From 0af627a4a0c2676e82cbe84cde514bf7a58899e9 Mon Sep 17 00:00:00 2001 From: Harshit Gangal Date: Mon, 3 Mar 2025 17:08:57 +0530 Subject: [PATCH] Faster Prepared Statement Execution by Using Raw SQL for Caching (#17777) Signed-off-by: Harshit Gangal Signed-off-by: Andres Taylor Co-authored-by: Andres Taylor --- go/cmd/vtgateclienttest/services/callerid.go | 11 +- go/cmd/vtgateclienttest/services/echo.go | 18 +- go/cmd/vtgateclienttest/services/errors.go | 19 +- go/cmd/vtgateclienttest/services/fallback.go | 15 +- go/cmd/vtgateclienttest/services/terminal.go | 13 +- go/mysql/conn.go | 65 +- go/mysql/conn_test.go | 45 +- go/mysql/fakesqldb/server.go | 4 +- go/mysql/query.go | 4 +- go/mysql/query_test.go | 2 +- go/mysql/server.go | 2 +- go/mysql/server_test.go | 4 +- go/test/endtoend/cluster/cluster_process.go | 2 +- go/test/endtoend/cluster/cluster_util.go | 2 +- go/test/endtoend/messaging/message_test.go | 8 +- .../endtoend/preparestmt/benchmark_test.go | 99 +- go/test/endtoend/recovery/recovery_util.go | 2 +- .../transaction/twopc/metric/metric_test.go | 34 +- .../endtoend/transaction/twopc/twopc_test.go | 58 +- .../vtcombo/recreate/recreate_test.go | 6 +- .../endtoend/vtcombo/vttest_sample_test.go | 26 +- go/test/endtoend/vtgate/grpc_api/acl_test.go | 10 +- .../endtoend/vtgate/grpc_api/execute_test.go | 2 +- .../vtgate/queries/reference/main_test.go | 8 +- go/test/utils/diff.go | 3 +- go/test/vschemawrapper/vschema_wrapper.go | 20 +- go/vt/key/destination.go | 64 +- go/vt/key/destination_test.go | 2 +- go/vt/proto/vtgate/vtgate.pb.go | 298 +-- go/vt/proto/vtgate/vtgate_vtproto.pb.go | 62 + go/vt/schemadiff/semantics.go | 2 +- go/vt/sqlparser/analyzer.go | 17 +- go/vt/sqlparser/cached_size.go | 18 + go/vt/sqlparser/normalizer.go | 57 +- go/vt/sqlparser/normalizer_test.go | 40 +- go/vt/sqlparser/redact_query.go | 2 +- go/vt/sqlparser/sql.go | 1684 ++++++++--------- go/vt/sqlparser/sql.y | 4 +- go/vt/sqlparser/utils.go | 2 +- go/vt/srvtopo/resolver.go | 10 +- go/vt/srvtopo/resolver_test.go | 14 +- go/vt/sysvars/sysvars.go | 28 + go/vt/topo/topoproto/destination.go | 6 +- go/vt/topo/topoproto/destination_test.go | 2 +- go/vt/vitessdriver/driver.go | 8 +- go/vt/vitessdriver/driver_test.go | 2 +- go/vt/vitessdriver/fakeserver_test.go | 255 +-- go/vt/vtexplain/vtexplain_vtgate.go | 2 +- go/vt/vtgate/autocommit_test.go | 11 +- go/vt/vtgate/bench_test.go | 2 + go/vt/vtgate/engine/cached_size.go | 26 +- go/vt/vtgate/engine/dbddl.go | 2 +- go/vt/vtgate/engine/delete.go | 21 +- go/vt/vtgate/engine/dml.go | 2 +- go/vt/vtgate/engine/dml_with_input.go | 14 +- go/vt/vtgate/engine/fake_vcursor_test.go | 8 +- go/vt/vtgate/engine/fk_cascade.go | 8 +- go/vt/vtgate/engine/fk_verify.go | 6 +- go/vt/vtgate/engine/insert.go | 12 +- go/vt/vtgate/engine/insert_common.go | 11 +- go/vt/vtgate/engine/insert_select.go | 12 +- go/vt/vtgate/engine/lock.go | 6 +- go/vt/vtgate/engine/mstream.go | 4 +- go/vt/vtgate/engine/online_ddl.go | 2 +- go/vt/vtgate/engine/plan.go | 179 +- go/vt/vtgate/engine/plan_description.go | 4 +- go/vt/vtgate/engine/primitive.go | 13 +- go/vt/vtgate/engine/revert_migration.go | 2 +- go/vt/vtgate/engine/route.go | 11 +- go/vt/vtgate/engine/routing.go | 16 +- go/vt/vtgate/engine/send.go | 7 +- go/vt/vtgate/engine/send_test.go | 31 +- go/vt/vtgate/engine/sequential.go | 9 +- go/vt/vtgate/engine/set.go | 18 +- go/vt/vtgate/engine/set_test.go | 6 +- go/vt/vtgate/engine/singlerow.go | 6 +- go/vt/vtgate/engine/update.go | 18 +- go/vt/vtgate/engine/upsert.go | 11 +- go/vt/vtgate/engine/vindex_func.go | 2 +- go/vt/vtgate/engine/vindex_func_test.go | 14 +- go/vt/vtgate/engine/vindex_lookup.go | 2 +- go/vt/vtgate/engine/vstream.go | 4 +- go/vt/vtgate/executor.go | 335 ++-- go/vt/vtgate/executor_ddl_test.go | 2 +- go/vt/vtgate/executor_dml_test.go | 96 +- go/vt/vtgate/executor_framework_test.go | 29 +- go/vt/vtgate/executor_scatter_stats_test.go | 12 +- go/vt/vtgate/executor_select_test.go | 227 +-- go/vt/vtgate/executor_set_test.go | 34 +- go/vt/vtgate/executor_test.go | 685 +++---- go/vt/vtgate/executor_vexplain_test.go | 2 +- go/vt/vtgate/executor_vschema_ddl_test.go | 102 +- go/vt/vtgate/executorcontext/vcursor_impl.go | 145 +- .../executorcontext/vcursor_impl_test.go | 75 +- go/vt/vtgate/fakerpcvtgateconn/conn.go | 28 +- go/vt/vtgate/grpcvtgateconn/conn.go | 20 +- go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go | 41 +- go/vt/vtgate/grpcvtgateconn/suite_test.go | 59 +- go/vt/vtgate/grpcvtgateservice/server.go | 11 +- go/vt/vtgate/legacy_scatter_conn_test.go | 10 +- go/vt/vtgate/plan_execute.go | 34 +- go/vt/vtgate/planbuilder/builder.go | 14 +- go/vt/vtgate/planbuilder/bypass.go | 4 +- go/vt/vtgate/planbuilder/ddl.go | 28 +- .../planbuilder/operators/misc_routing.go | 2 +- go/vt/vtgate/planbuilder/operators/route.go | 2 +- go/vt/vtgate/planbuilder/plan_test_vindex.go | 16 +- .../plancontext/planning_context_test.go | 10 +- .../vtgate/planbuilder/plancontext/vschema.go | 10 +- .../vtgate/planbuilder/prepared_statement.go | 22 +- go/vt/vtgate/planbuilder/set.go | 6 +- go/vt/vtgate/planbuilder/show.go | 6 +- go/vt/vtgate/planbuilder/simplifier_test.go | 8 +- .../planbuilder/testdata/dml_cases.json | 246 --- .../dml_cases_with_user_as_default.json | 1 - .../testdata/foreignkey_cases.json | 145 -- .../testdata/foreignkey_checks_off_cases.json | 18 - .../testdata/foreignkey_checks_on_cases.json | 77 - .../planbuilder/testdata/mirror_cases.json | 3 - .../planbuilder/testdata/oltp_cases.json | 4 - .../planbuilder/testdata/reference_cases.json | 23 - .../planbuilder/testdata/tpcc_cases.json | 17 - go/vt/vtgate/planbuilder/vexplain.go | 2 +- go/vt/vtgate/plugin_mysql_server.go | 12 +- go/vt/vtgate/plugin_mysql_server_test.go | 4 +- go/vt/vtgate/resolver.go | 2 +- go/vt/vtgate/scatter_conn_test.go | 6 +- go/vt/vtgate/semantics/FakeSI.go | 2 +- go/vt/vtgate/semantics/info_schema.go | 2 +- go/vt/vtgate/semantics/semantic_table.go | 2 +- go/vt/vtgate/semantics/typer_test.go | 4 +- go/vt/vtgate/vindexes/binary.go | 12 +- go/vt/vtgate/vindexes/binarymd5.go | 6 +- go/vt/vtgate/vindexes/cfc.go | 14 +- go/vt/vtgate/vindexes/cfc_test.go | 4 +- go/vt/vtgate/vindexes/consistent_lookup.go | 20 +- .../vtgate/vindexes/consistent_lookup_test.go | 10 +- go/vt/vtgate/vindexes/hash.go | 6 +- go/vt/vtgate/vindexes/hash_test.go | 2 +- go/vt/vtgate/vindexes/lookup.go | 22 +- go/vt/vtgate/vindexes/lookup_hash.go | 20 +- go/vt/vtgate/vindexes/lookup_hash_test.go | 12 +- .../vindexes/lookup_hash_unique_test.go | 8 +- go/vt/vtgate/vindexes/lookup_test.go | 12 +- .../vindexes/lookup_unicodeloosemd5_hash.go | 16 +- .../lookup_unicodeloosemd5_hash_test.go | 8 +- go/vt/vtgate/vindexes/lookup_unique_test.go | 6 +- go/vt/vtgate/vindexes/multicol.go | 4 +- go/vt/vtgate/vindexes/multicol_test.go | 2 +- go/vt/vtgate/vindexes/null.go | 6 +- go/vt/vtgate/vindexes/null_test.go | 2 +- go/vt/vtgate/vindexes/numeric.go | 10 +- go/vt/vtgate/vindexes/numeric_static_map.go | 6 +- .../vindexes/numeric_static_map_test.go | 4 +- go/vt/vtgate/vindexes/numeric_test.go | 2 +- go/vt/vtgate/vindexes/region_experimental.go | 4 +- .../vindexes/region_experimental_test.go | 4 +- go/vt/vtgate/vindexes/region_json.go | 4 +- go/vt/vtgate/vindexes/reverse_bits.go | 4 +- go/vt/vtgate/vindexes/reverse_bits_test.go | 2 +- go/vt/vtgate/vindexes/unicodeloosemd5.go | 6 +- go/vt/vtgate/vindexes/unicodeloosexxhash.go | 6 +- go/vt/vtgate/vindexes/vindex.go | 12 +- go/vt/vtgate/vindexes/vindex_test.go | 4 +- go/vt/vtgate/vindexes/vschema_test.go | 16 +- go/vt/vtgate/vindexes/xxhash.go | 6 +- go/vt/vtgate/vtgate.go | 32 +- go/vt/vtgate/vtgate_test.go | 48 +- go/vt/vtgate/vtgateconn/vtgateconn.go | 14 +- go/vt/vtgate/vtgateservice/interface.go | 4 +- go/vt/vthash/highway/highwayhash.go | 6 + go/vtbench/client.go | 2 +- proto/vtgate.proto | 4 + 173 files changed, 3145 insertions(+), 3453 deletions(-) diff --git a/go/cmd/vtgateclienttest/services/callerid.go b/go/cmd/vtgateclienttest/services/callerid.go index 0e4a7da8495..f66644d3996 100644 --- a/go/cmd/vtgateclienttest/services/callerid.go +++ b/go/cmd/vtgateclienttest/services/callerid.go @@ -75,11 +75,18 @@ func (c *callerIDClient) checkCallerID(ctx context.Context, received string) (bo return true, fmt.Errorf("SUCCESS: callerid matches") } -func (c *callerIDClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (c *callerIDClient) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { if ok, err := c.checkCallerID(ctx, sql); ok { return session, nil, err } - return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables) + return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables, prepared) } func (c *callerIDClient) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, sqlList []string, bindVariablesList []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) { diff --git a/go/cmd/vtgateclienttest/services/echo.go b/go/cmd/vtgateclienttest/services/echo.go index 5a4d78aeb3e..7ffe5b1cfd5 100644 --- a/go/cmd/vtgateclienttest/services/echo.go +++ b/go/cmd/vtgateclienttest/services/echo.go @@ -98,7 +98,14 @@ func echoQueryResult(vals map[string]any) *sqltypes.Result { return qr } -func (c *echoClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (c *echoClient) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { if strings.HasPrefix(sql, EchoPrefix) { return session, echoQueryResult(map[string]any{ "callerId": callerid.EffectiveCallerIDFromContext(ctx), @@ -107,7 +114,7 @@ func (c *echoClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLCo "session": session, }), nil } - return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables) + return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables, prepared) } func (c *echoClient) StreamExecute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable, callback func(*sqltypes.Result) error) (*vtgatepb.Session, error) { @@ -173,14 +180,13 @@ func (c *echoClient) VStream(ctx context.Context, tabletType topodatapb.TabletTy return c.fallbackClient.VStream(ctx, tabletType, vgtid, filter, flags, callback) } -func (c *echoClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { +func (c *echoClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { if strings.HasPrefix(sql, EchoPrefix) { return session, echoQueryResult(map[string]any{ "callerId": callerid.EffectiveCallerIDFromContext(ctx), "query": sql, - "bindVars": bindVariables, "session": session, - }).Fields, nil + }).Fields, 0, nil } - return c.fallbackClient.Prepare(ctx, session, sql, bindVariables) + return c.fallbackClient.Prepare(ctx, session, sql) } diff --git a/go/cmd/vtgateclienttest/services/errors.go b/go/cmd/vtgateclienttest/services/errors.go index ad877054850..f1438a105e4 100644 --- a/go/cmd/vtgateclienttest/services/errors.go +++ b/go/cmd/vtgateclienttest/services/errors.go @@ -110,14 +110,21 @@ func trimmedRequestToError(received string) error { } } -func (c *errorClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (c *errorClient) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { if err := requestToPartialError(sql, session); err != nil { return session, nil, err } if err := requestToError(sql); err != nil { return session, nil, err } - return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables) + return c.fallbackClient.Execute(ctx, mysqlCtx, session, sql, bindVariables, prepared) } func (c *errorClient) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, sqlList []string, bindVariablesList []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) { @@ -139,14 +146,14 @@ func (c *errorClient) StreamExecute(ctx context.Context, mysqlCtx vtgateservice. return c.fallbackClient.StreamExecute(ctx, mysqlCtx, session, sql, bindVariables, callback) } -func (c *errorClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { +func (c *errorClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { if err := requestToPartialError(sql, session); err != nil { - return session, nil, err + return session, nil, 0, err } if err := requestToError(sql); err != nil { - return session, nil, err + return session, nil, 0, err } - return c.fallbackClient.Prepare(ctx, session, sql, bindVariables) + return c.fallbackClient.Prepare(ctx, session, sql) } func (c *errorClient) CloseSession(ctx context.Context, session *vtgatepb.Session) error { diff --git a/go/cmd/vtgateclienttest/services/fallback.go b/go/cmd/vtgateclienttest/services/fallback.go index dab0e912ddb..00ad33776df 100644 --- a/go/cmd/vtgateclienttest/services/fallback.go +++ b/go/cmd/vtgateclienttest/services/fallback.go @@ -40,8 +40,15 @@ func newFallbackClient(fallback vtgateservice.VTGateService) fallbackClient { return fallbackClient{fallback: fallback} } -func (c fallbackClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { - return c.fallback.Execute(ctx, mysqlCtx, session, sql, bindVariables) +func (c fallbackClient) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { + return c.fallback.Execute(ctx, mysqlCtx, session, sql, bindVariables, prepared) } func (c fallbackClient) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, sqlList []string, bindVariablesList []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) { @@ -52,8 +59,8 @@ func (c fallbackClient) StreamExecute(ctx context.Context, mysqlCtx vtgateservic return c.fallback.StreamExecute(ctx, mysqlCtx, session, sql, bindVariables, callback) } -func (c fallbackClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { - return c.fallback.Prepare(ctx, session, sql, bindVariables) +func (c fallbackClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { + return c.fallback.Prepare(ctx, session, sql) } func (c fallbackClient) CloseSession(ctx context.Context, session *vtgatepb.Session) error { diff --git a/go/cmd/vtgateclienttest/services/terminal.go b/go/cmd/vtgateclienttest/services/terminal.go index 8fa321e2606..ad1937566f1 100644 --- a/go/cmd/vtgateclienttest/services/terminal.go +++ b/go/cmd/vtgateclienttest/services/terminal.go @@ -43,7 +43,14 @@ func newTerminalClient() *terminalClient { return &terminalClient{} } -func (c *terminalClient) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (c *terminalClient) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { if sql == "quit://" { log.Fatal("Received quit:// query. Going down.") } @@ -63,8 +70,8 @@ func (c *terminalClient) StreamExecute(ctx context.Context, mysqlCtx vtgateservi return session, errTerminal } -func (c *terminalClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { - return session, nil, errTerminal +func (c *terminalClient) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { + return session, nil, 0, errTerminal } func (c *terminalClient) CloseSession(ctx context.Context, session *vtgatepb.Session) error { diff --git a/go/mysql/conn.go b/go/mysql/conn.go index 7eedc094c1e..cfe65e07166 100644 --- a/go/mysql/conn.go +++ b/go/mysql/conn.go @@ -38,7 +38,6 @@ import ( "vitess.io/vitess/go/vt/log" querypb "vitess.io/vitess/go/vt/proto/query" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" ) @@ -1122,7 +1121,7 @@ func (c *Conn) handleComStmtExecute(handler Handler, data []byte) (kontinue bool return c.writeErrorPacketFromErrorAndLog(err) } - fieldSent := false + receivedResult := false // sendFinished is set if the response should just be an OK packet. sendFinished := false prepare := c.PrepareData[stmtID] @@ -1132,8 +1131,8 @@ func (c *Conn) handleComStmtExecute(handler Handler, data []byte) (kontinue bool return io.EOF } - if !fieldSent { - fieldSent = true + if !receivedResult { + receivedResult = true if len(qr.Fields) == 0 { sendFinished = true @@ -1157,7 +1156,7 @@ func (c *Conn) handleComStmtExecute(handler Handler, data []byte) (kontinue bool }) // If no field was sent, we expect an error. - if !fieldSent { + if !receivedResult { // This is just a failsafe. Should never happen. if err == nil || err == io.EOF { err = sqlerror.NewSQLErrorFromError(errors.New("unexpected: query ended without no results and no error")) @@ -1200,10 +1199,8 @@ func (c *Conn) handleComPrepare(handler Handler, data []byte) (kontinue bool) { query := c.parseComPrepare(data) c.recycleReadPacket() - var queries []string if c.Capabilities&CapabilityClientMultiStatements != 0 { - var err error - queries, err = handler.Env().Parser().SplitStatementToPieces(query) + queries, err := handler.Env().Parser().SplitStatementToPieces(query) if err != nil { log.Errorf("Conn %v: Error splitting query: %v", c, err) return c.writeErrorPacketFromErrorAndLog(err) @@ -1212,56 +1209,26 @@ func (c *Conn) handleComPrepare(handler Handler, data []byte) (kontinue bool) { log.Errorf("Conn %v: can not prepare multiple statements", c) return c.writeErrorPacketFromErrorAndLog(err) } - } else { - queries = []string{query} + query = queries[0] + } + + fld, paramsCount, err := handler.ComPrepare(c, query) + if err != nil { + return c.writeErrorPacketFromErrorAndLog(err) } // Populate PrepareData c.StatementID++ prepare := &PrepareData{ StatementID: c.StatementID, - PrepareStmt: queries[0], - } - - statement, err := handler.Env().Parser().ParseStrictDDL(query) - if err != nil { - log.Errorf("Conn %v: Error parsing prepared statement: %v", c, err) - if !c.writeErrorPacketFromErrorAndLog(err) { - return false - } + PrepareStmt: query, + ParamsCount: paramsCount, + ParamsType: make([]int32, paramsCount), + BindVars: make(map[string]*querypb.BindVariable, paramsCount), } - - paramsCount := uint16(0) - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { - switch node := node.(type) { - case *sqlparser.Argument: - if strings.HasPrefix(node.Name, "v") { - paramsCount++ - } - } - return true, nil - }, statement) - - if paramsCount > 0 { - prepare.ParamsCount = paramsCount - prepare.ParamsType = make([]int32, paramsCount) - prepare.BindVars = make(map[string]*querypb.BindVariable, paramsCount) - } - - bindVars := make(map[string]*querypb.BindVariable, paramsCount) - for i := range uint16(paramsCount) { - parameterID := fmt.Sprintf("v%d", i+1) - bindVars[parameterID] = &querypb.BindVariable{} - } - c.PrepareData[c.StatementID] = prepare - fld, err := handler.ComPrepare(c, queries[0], bindVars) - if err != nil { - return c.writeErrorPacketFromErrorAndLog(err) - } - - if err := c.writePrepare(fld, c.PrepareData[c.StatementID]); err != nil { + if err := c.writePrepare(fld, prepare); err != nil { log.Error("Error writing prepare data to client %v: %v", c.ConnectionID, err) return false } diff --git a/go/mysql/conn_test.go b/go/mysql/conn_test.go index 088e487e961..7520493dbfc 100644 --- a/go/mysql/conn_test.go +++ b/go/mysql/conn_test.go @@ -816,7 +816,7 @@ func TestMultiStatementStopsOnError(t *testing.T) { // this handler will return results according to the query. In case the query contains "error" it will return an error // panic if the query contains "panic" and it will return selectRowsResult in case of any other query - handler := &testRun{t: t, err: fmt.Errorf("execution failed")} + handler := &testRun{err: fmt.Errorf("execution failed")} res := sConn.handleNextCommand(handler) // Execution error will occur in this case because the query sent is error and testRun will throw an error. // We should send an error packet but not close the connection. @@ -842,7 +842,7 @@ func TestMultiStatement(t *testing.T) { // this handler will return results according to the query. In case the query contains "error" it will return an error // panic if the query contains "panic" and it will return selectRowsResult in case of any other query - handler := &testRun{t: t, err: sqlerror.NewSQLError(sqlerror.CRMalformedPacket, sqlerror.SSUnknownSQLState, "cannot get column number")} + handler := &testRun{err: sqlerror.NewSQLError(sqlerror.CRMalformedPacket, sqlerror.SSUnknownSQLState, "cannot get column number")} res := sConn.handleNextCommand(handler) // The queries run will be select 1; and select 2; These queries do not return any errors, so the connection should still be open require.True(t, res, "we should not break the connection in case of no errors") @@ -890,7 +890,7 @@ func TestMultiStatementOnSplitError(t *testing.T) { // this handler will return results according to the query. In case the query contains "error" it will return an error // panic if the query contains "panic" and it will return selectRowsResult in case of any other query - handler := &testRun{t: t, err: fmt.Errorf("execution failed")} + handler := &testRun{err: fmt.Errorf("execution failed")} // We will encounter an error in split statement when this multi statement is processed. res := sConn.handleNextCommand(handler) @@ -917,7 +917,7 @@ func TestInitDbAgainstWrongDbDoesNotDropConnection(t *testing.T) { // this handler will return results according to the query. In case the query contains "error" it will return an error // panic if the query contains "panic" and it will return selectRowsResult in case of any other query - handler := &testRun{t: t, err: fmt.Errorf("execution failed")} + handler := &testRun{err: fmt.Errorf("execution failed")} res := sConn.handleNextCommand(handler) require.True(t, res, "we should not break the connection because of execution errors") @@ -938,7 +938,7 @@ func TestConnectionErrorWhileWritingComQuery(t *testing.T) { // this handler will return an error on the first run, and fail the test if it's run more times errorString := make([]byte, 17000) - handler := &testRun{t: t, err: errors.New(string(errorString))} + handler := &testRun{err: errors.New(string(errorString))} res := sConn.handleNextCommand(handler) require.False(t, res, "we should beak the connection in case of error writing error packet") } @@ -953,7 +953,7 @@ func TestConnectionErrorWhileWritingComStmtSendLongData(t *testing.T) { }, DefaultFlushDelay, 0) // this handler will return an error on the first run, and fail the test if it's run more times - handler := &testRun{t: t, err: fmt.Errorf("not used")} + handler := &testRun{err: fmt.Errorf("not used")} res := sConn.handleNextCommand(handler) require.False(t, res, "we should beak the connection in case of error writing error packet") } @@ -967,7 +967,7 @@ func TestConnectionErrorWhileWritingComPrepare(t *testing.T) { }, DefaultFlushDelay, 0) sConn.Capabilities = sConn.Capabilities | CapabilityClientMultiStatements // this handler will return an error on the first run, and fail the test if it's run more times - handler := &testRun{t: t, err: fmt.Errorf("not used")} + handler := &testRun{err: fmt.Errorf("not used")} res := sConn.handleNextCommand(handler) require.False(t, res, "we should beak the connection in case of error writing error packet") } @@ -981,7 +981,7 @@ func TestConnectionErrorWhileWritingComStmtExecute(t *testing.T) { 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x20, 0x31}, }, DefaultFlushDelay, 0) // this handler will return an error on the first run, and fail the test if it's run more times - handler := &testRun{t: t, err: fmt.Errorf("not used")} + handler := &testRun{err: fmt.Errorf("not used")} res := sConn.handleNextCommand(handler) require.False(t, res, "we should beak the connection in case of error writing error packet") } @@ -1032,16 +1032,16 @@ func startGoRoutine(ctx context.Context, t *testing.T, s string) { err := cConn.writePacket(mockData) require.NoError(t, err) - handler := &testRun{ - t: t, - expParamCounts: 1, - expQuery: sql, - expStmtID: 1, - } + handler := &testRun{paramCounts: 1} ok := sConn.handleNextCommand(handler) require.True(t, ok, "error handling command for id: %s", s) + prepareData, ok := sConn.PrepareData[sConn.StatementID] + require.True(t, ok, "prepare data not found for id: %d", sConn.StatementID) + require.Equal(t, uint16(1), prepareData.ParamsCount) + require.NotNil(t, prepareData.BindVars) + resp, err := cConn.ReadPacket() require.NoError(t, err) require.EqualValues(t, 0, resp[0]) @@ -1083,11 +1083,9 @@ func createSendLongDataPacket(stmtID uint32, paramID uint16, data []byte) []byte type testRun struct { UnimplementedHandler - t *testing.T - err error - expParamCounts int - expQuery string - expStmtID int + paramCounts uint16 + err error + expQuery string } func (t testRun) ComStmtExecute(c *Conn, prepare *PrepareData, callback func(*sqltypes.Result) error) error { @@ -1129,13 +1127,8 @@ func (t testRun) ComQuery(c *Conn, query string, callback func(*sqltypes.Result) return nil } -func (t testRun) ComPrepare(c *Conn, query string, bv map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - assert.Equal(t.t, t.expQuery, query) - assert.EqualValues(t.t, t.expStmtID, c.StatementID) - assert.NotNil(t.t, c.PrepareData[c.StatementID]) - assert.EqualValues(t.t, t.expParamCounts, c.PrepareData[c.StatementID].ParamsCount) - assert.Len(t.t, c.PrepareData, int(c.PrepareData[c.StatementID].ParamsCount)) - return nil, nil +func (t testRun) ComPrepare(c *Conn, query string) ([]*querypb.Field, uint16, error) { + return nil, t.paramCounts, nil } func (t testRun) WarningCount(c *Conn) uint16 { diff --git a/go/mysql/fakesqldb/server.go b/go/mysql/fakesqldb/server.go index cc53ee19008..dfa016149bb 100644 --- a/go/mysql/fakesqldb/server.go +++ b/go/mysql/fakesqldb/server.go @@ -525,8 +525,8 @@ func (db *DB) comQueryOrdered(query string) (*sqltypes.Result, error) { } // ComPrepare is part of the mysql.Handler interface. -func (db *DB) ComPrepare(c *mysql.Conn, query string, bindVars map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - return nil, nil +func (db *DB) ComPrepare(*mysql.Conn, string) ([]*querypb.Field, uint16, error) { + return nil, 0, nil } // ComStmtExecute is part of the mysql.Handler interface. diff --git a/go/mysql/query.go b/go/mysql/query.go index 87a8c8e030a..2fe428164e1 100644 --- a/go/mysql/query.go +++ b/go/mysql/query.go @@ -630,7 +630,7 @@ func (c *Conn) parseComStmtExecute(prepareData map[uint32]*PrepareData, data []b newParamsBoundFlag, pos, ok := readByte(payload, pos) if ok && newParamsBoundFlag == 0x01 { var mysqlType, flags byte - for i := range uint16(prepare.ParamsCount) { + for i := range prepare.ParamsCount { mysqlType, pos, ok = readByte(payload, pos) if !ok { return stmtID, 0, sqlerror.NewSQLError(sqlerror.CRMalformedPacket, sqlerror.SSUnknownSQLState, "reading parameter type failed") @@ -651,7 +651,7 @@ func (c *Conn) parseComStmtExecute(prepareData map[uint32]*PrepareData, data []b } } - for i := range len(prepare.ParamsType) { + for i := range prepare.ParamsCount { var val sqltypes.Value parameterID := fmt.Sprintf("v%d", i+1) if v, ok := prepare.BindVars[parameterID]; ok { diff --git a/go/mysql/query_test.go b/go/mysql/query_test.go index 01cbd6a8f30..35e60045e29 100644 --- a/go/mysql/query_test.go +++ b/go/mysql/query_test.go @@ -365,7 +365,7 @@ func TestSQLErrorOnServerClose(t *testing.T) { err := cConn.WriteComQuery("close before rows read") require.NoError(t, err) - handler := &testRun{t: t} + handler := &testRun{} _ = sConn.handleNextCommand(handler) // From the server we will receive a field packet which the client will read diff --git a/go/mysql/server.go b/go/mysql/server.go index e21281710b7..55d3bf96f53 100644 --- a/go/mysql/server.go +++ b/go/mysql/server.go @@ -109,7 +109,7 @@ type Handler interface { // ComPrepare is called when a connection receives a prepared // statement query. - ComPrepare(c *Conn, query string, bindVars map[string]*querypb.BindVariable) ([]*querypb.Field, error) + ComPrepare(c *Conn, query string) ([]*querypb.Field, uint16, error) // ComStmtExecute is called when a connection receives a statement // execute query. diff --git a/go/mysql/server_test.go b/go/mysql/server_test.go index b301e956ef4..8494ba66151 100644 --- a/go/mysql/server_test.go +++ b/go/mysql/server_test.go @@ -231,8 +231,8 @@ func (th *testHandler) ComQuery(c *Conn, query string, callback func(*sqltypes.R return nil } -func (th *testHandler) ComPrepare(c *Conn, query string, bindVars map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - return nil, nil +func (th *testHandler) ComPrepare(*Conn, string) ([]*querypb.Field, uint16, error) { + return nil, 0, nil } func (th *testHandler) ComStmtExecute(c *Conn, prepare *PrepareData, callback func(*sqltypes.Result) error) error { diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index ed61f7fd0c9..dd5029f91d3 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -958,7 +958,7 @@ func (cluster *LocalProcessCluster) ExecOnVTGate(ctx context.Context, addr strin session := conn.Session(target, opts) defer conn.Close() - return session.Execute(ctx, sql, bindvars) + return session.Execute(ctx, sql, bindvars, false) } // StreamTabletHealth invokes a HealthStream on a local cluster Vttablet and diff --git a/go/test/endtoend/cluster/cluster_util.go b/go/test/endtoend/cluster/cluster_util.go index 18f78dcb3d0..962120fedbc 100644 --- a/go/test/endtoend/cluster/cluster_util.go +++ b/go/test/endtoend/cluster/cluster_util.go @@ -275,7 +275,7 @@ func positionAtLeast(t *testing.T, tablet *Vttablet, a string, b string) bool { // ExecuteQueriesUsingVtgate sends query to vtgate using vtgate session. func ExecuteQueriesUsingVtgate(t *testing.T, session *vtgateconn.VTGateSession, query string) { - _, err := session.Execute(context.Background(), query, nil) + _, err := session.Execute(context.Background(), query, nil, false) require.Nil(t, err) } diff --git a/go/test/endtoend/messaging/message_test.go b/go/test/endtoend/messaging/message_test.go index 7dbef603417..5939adfd0ea 100644 --- a/go/test/endtoend/messaging/message_test.go +++ b/go/test/endtoend/messaging/message_test.go @@ -428,7 +428,7 @@ func TestReparenting(t *testing.T) { assertClientCount(t, 0, shard0Replica) assertClientCount(t, 1, shard1Primary) - _, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (3) and time_acked is null", nil) + _, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (3) and time_acked is null", nil, false) require.Nil(t, err) } @@ -480,7 +480,7 @@ func TestConnection(t *testing.T) { _, err = stream.Next() require.Nil(t, err) - _, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (2, 5) and time_acked is null", nil) + _, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (2, 5) and time_acked is null", nil, false) require.Nil(t, err) // After closing one stream, ensure vttablets have dropped it. stream.Close() @@ -533,7 +533,7 @@ func testMessaging(t *testing.T, name, ks string) { resMap = make(map[string]string) stream.ClearMem() // validate message ack with id 4 - qr, err := session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (4) and time_acked is null", nil) + qr, err := session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (4) and time_acked is null", nil, false) require.Nil(t, err) assert.Equal(t, uint64(1), qr.RowsAffected) @@ -546,7 +546,7 @@ func testMessaging(t *testing.T, name, ks string) { assert.Equal(t, "1", resMap["1"]) // validate message ack with 1 and 4, only 1 should be ack - qr, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (1, 4) and time_acked is null", nil) + qr, err = session.Execute(context.Background(), "update "+name+" set time_acked = 1, time_next = null where id in (1, 4) and time_acked is null", nil, false) require.Nil(t, err) assert.Equal(t, uint64(1), qr.RowsAffected) } diff --git a/go/test/endtoend/preparestmt/benchmark_test.go b/go/test/endtoend/preparestmt/benchmark_test.go index fcf10a6948a..b9ef2ccfc65 100644 --- a/go/test/endtoend/preparestmt/benchmark_test.go +++ b/go/test/endtoend/preparestmt/benchmark_test.go @@ -17,6 +17,8 @@ limitations under the License. package preparestmt import ( + "context" + "fmt" "math/rand/v2" "testing" @@ -113,7 +115,7 @@ WHERE if err != nil { b.Fatal(err) } - defer sStmt.Close() + defer uStmt.Close() b.Run("Update", func(b *testing.B) { b.ResetTimer() @@ -129,7 +131,7 @@ WHERE if err != nil { b.Fatal(err) } - defer sStmt.Close() + defer dStmt.Close() b.Run("Delete", func(b *testing.B) { b.ResetTimer() @@ -142,3 +144,96 @@ WHERE }) } + +/* +export ver=v1 p=~/benchmark && go test \ +-run '^$' -bench '^BenchmarkExecuteStmt' \ +-benchtime 2s -count 6 -cpu 4 \ +| tee $p/${ver}.txt +*/ +func BenchmarkExecuteStmt(b *testing.B) { + dbo := Connect(b, "interpolateParams=false") + defer dbo.Close() + + // prepare statement + insertStmt := `insert into sks.t1 (name, age, email, created_at, is_active) values('%s', %d, '%s', current_timestamp, %d)` + selectStmt := `select id, name, age, email from sks.t1 where age between %d and %d and is_active = %d limit %d` + updateStmt := `update sks.t1 set is_active = %d where id = %d` + deleteStmt := `delete from sks.t1 where is_active = %d and age = %d` + + joinStmt := `SELECT + user.id AS user_id +FROM + sks.t1 AS user +LEFT JOIN + sks.t1 AS parent ON user.id = parent.id AND parent.age = %d +LEFT JOIN + sks.t1 AS manager ON user.id = manager.id AND manager.is_active = %d +LEFT JOIN + sks.t1 AS child ON user.id = child.id +WHERE + user.is_active = %d + AND user.id = %d + AND parent.id = %d + AND manager.id = %d` + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + b.Run("Insert", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := dbo.ExecContext(ctx, fmt.Sprintf(insertStmt, fake.FirstName(), rand.IntN(100), fake.EmailAddress(), rand.IntN(2))) + if err != nil { + b.Fatal(err) + } + } + }) + + b.Run("Select", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + age := rand.IntN(80) + r, err := dbo.QueryContext(ctx, fmt.Sprintf(selectStmt, age, age+20, rand.IntN(2), rand.IntN(10))) + if err != nil { + b.Fatal(err) + } + r.Close() + } + }) + + b.Run("Join Select:Simple Route", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + age := rand.IntN(80) + active := rand.IntN(2) + id := rand.IntN(2000) + r, err := dbo.QueryContext(ctx, fmt.Sprintf(joinStmt, age, active, active, id, id, id)) + if err != nil { + b.Fatal(err) + } + r.Close() + } + }) + + b.Run("Update", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := dbo.ExecContext(ctx, fmt.Sprintf(updateStmt, rand.IntN(2), rand.IntN(2000))) + if err != nil { + b.Fatal(err) + } + } + }) + + b.Run("Delete", func(b *testing.B) { + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := dbo.ExecContext(ctx, fmt.Sprintf(deleteStmt, rand.IntN(2), rand.IntN(100))) + if err != nil { + b.Fatal(err) + } + } + }) + +} diff --git a/go/test/endtoend/recovery/recovery_util.go b/go/test/endtoend/recovery/recovery_util.go index f862b30a9b3..7118507b685 100644 --- a/go/test/endtoend/recovery/recovery_util.go +++ b/go/test/endtoend/recovery/recovery_util.go @@ -45,7 +45,7 @@ var ( // VerifyQueriesUsingVtgate verifies queries using vtgate. func VerifyQueriesUsingVtgate(t *testing.T, session *vtgateconn.VTGateSession, query string, value string) { - qr, err := session.Execute(context.Background(), query, nil) + qr, err := session.Execute(context.Background(), query, nil, false) require.Nil(t, err) assert.Equal(t, value, fmt.Sprintf("%v", qr.Rows[0][0])) } diff --git a/go/test/endtoend/transaction/twopc/metric/metric_test.go b/go/test/endtoend/transaction/twopc/metric/metric_test.go index 40645628f45..4ad49373487 100644 --- a/go/test/endtoend/transaction/twopc/metric/metric_test.go +++ b/go/test/endtoend/transaction/twopc/metric/metric_test.go @@ -120,14 +120,14 @@ func TestVTGate2PCCommitMetricOnFailure(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - _, err = conn.Execute(ctx, "begin", nil) + _, err = conn.Execute(ctx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(7,'foo'), (8,'bar')", nil) + _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(7,'foo'), (8,'bar')", nil, false) require.NoError(t, err) // fail after mm commit. newCtx := callerid.NewContext(ctx, callerid.NewEffectiveCallerID("MMCommitted_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail After MM commit") updatedCount := getVarValue[float64](t, "CommitUnresolved", clusterInstance.VtgateProcess.GetVars) @@ -135,16 +135,16 @@ func TestVTGate2PCCommitMetricOnFailure(t *testing.T) { waitForResolve(ctx, t, conn, 5*time.Second) - _, err = conn.Execute(ctx, "begin", nil) + _, err = conn.Execute(ctx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(9,'foo')", nil) + _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(9,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(10,'apa')", nil) + _, err = conn.Execute(ctx, "insert into twopc_user(id, name) values(10,'apa')", nil, false) require.NoError(t, err) // fail during rm commit. newCtx = callerid.NewContext(ctx, callerid.NewEffectiveCallerID("RMCommit_-40_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail During RM commit") updatedCount = getVarValue[float64](t, "CommitUnresolved", clusterInstance.VtgateProcess.GetVars) @@ -167,16 +167,16 @@ func TestVTTablet2PCMetrics(t *testing.T) { defer cancel() for i := 1; i <= 20; i++ { - _, err = conn.Execute(ctx, "begin", nil) + _, err = conn.Execute(ctx, "begin", nil, false) require.NoError(t, err) query := fmt.Sprintf("insert into twopc_user(id, name) values(%d,'foo'), (%d,'bar'), (%d,'baz')", i, i*101, i+53) - _, err = conn.Execute(ctx, query, nil) + _, err = conn.Execute(ctx, query, nil, false) require.NoError(t, err) multi := len(conn.SessionPb().ShardSessions) > 1 // fail after mm commit. - _, err = conn.Execute(ctx, "commit", nil) + _, err = conn.Execute(ctx, "commit", nil, false) if multi { assert.ErrorContains(t, err, "Fail After MM commit") } else { @@ -224,7 +224,7 @@ func TestVTTablet2PCMetricsFailCommitPrepared(t *testing.T) { execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (4, 1)") execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (6, 2)") execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (9, 3)") - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "commit prepared: retryable error") dtidRetryable := getDTIDFromWarnings(ctx, t, conn) require.NotEmpty(t, dtidRetryable) @@ -238,7 +238,7 @@ func TestVTTablet2PCMetricsFailCommitPrepared(t *testing.T) { execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (20, 11)") execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (22, 21)") execute(t, newCtx, conn, "insert into twopc_t1(id, col) values (25, 31)") - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "commit prepared: non retryable error") dtidNonRetryable := getDTIDFromWarnings(ctx, t, conn) require.NotEmpty(t, dtidNonRetryable) @@ -255,7 +255,7 @@ func TestVTTablet2PCMetricsFailCommitPrepared(t *testing.T) { waitForDTIDResolve(ctx, t, conn, dtidRetryable, 5*time.Second) // dtid with non retryable error should remain unresolved. - qr, err := conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtidNonRetryable), nil) + qr, err := conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtidNonRetryable), nil, false) require.NoError(t, err) require.NotZero(t, qr.Rows, "should remain unresolved") @@ -265,7 +265,7 @@ func TestVTTablet2PCMetricsFailCommitPrepared(t *testing.T) { require.NoError(t, err) require.Contains(t, out, "Successfully concluded the distributed transaction") // now verifying - qr, err = conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtidNonRetryable), nil) + qr, err = conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtidNonRetryable), nil, false) require.NoError(t, err) require.Empty(t, qr.Rows) } @@ -273,7 +273,7 @@ func TestVTTablet2PCMetricsFailCommitPrepared(t *testing.T) { func execute(t *testing.T, ctx context.Context, conn *vtgateconn.VTGateSession, query string) { t.Helper() - _, err := conn.Execute(ctx, query, nil) + _, err := conn.Execute(ctx, query, nil, false) require.NoError(t, err) } @@ -364,7 +364,7 @@ func waitForResolve(ctx context.Context, t *testing.T, conn *vtgateconn.VTGateSe } func getDTIDFromWarnings(ctx context.Context, t *testing.T, conn *vtgateconn.VTGateSession) string { - qr, err := conn.Execute(ctx, "show warnings", nil) + qr, err := conn.Execute(ctx, "show warnings", nil, false) require.NoError(t, err) require.Len(t, qr.Rows, 1) @@ -388,7 +388,7 @@ func waitForDTIDResolve(ctx context.Context, t *testing.T, conn *vtgateconn.VTGa t.Errorf("transaction resolution exceeded wait time of %v", waitTime) unresolved = false // break the loop. case <-time.After(100 * time.Millisecond): - qr, err := conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtid), nil) + qr, err := conn.Execute(ctx, fmt.Sprintf(`show transaction status for '%s'`, dtid), nil, false) require.NoError(t, err) unresolved = len(qr.Rows) != 0 } diff --git a/go/test/endtoend/transaction/twopc/twopc_test.go b/go/test/endtoend/transaction/twopc/twopc_test.go index b7f7c11fba9..89dfe2c3548 100644 --- a/go/test/endtoend/transaction/twopc/twopc_test.go +++ b/go/test/endtoend/transaction/twopc/twopc_test.go @@ -615,24 +615,24 @@ func TestDTResolveAfterMMCommit(t *testing.T) { qCtx, cancel := context.WithCancel(context.Background()) defer cancel() // Insert into multiple shards - _, err = conn.Execute(qCtx, "begin", nil) + _, err = conn.Execute(qCtx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(9,'baz')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(9,'baz')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'apa')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'apa')", nil, false) require.NoError(t, err) // Also do an update to a table that has a consistent lookup vindex. // We expect to see only the pre-session changes in the logs. - _, err = conn.Execute(qCtx, "update twopc_consistent_lookup set col = 22 where id = 4", nil) + _, err = conn.Execute(qCtx, "update twopc_consistent_lookup set col = 22 where id = 4", nil, false) require.NoError(t, err) // The caller ID is used to simulate the failure at the desired point. newCtx := callerid.NewContext(qCtx, callerid.NewEffectiveCallerID("MMCommitted_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail After MM commit") testWarningAndTransactionStatus(t, conn, @@ -717,20 +717,20 @@ func TestDTResolveAfterRMPrepare(t *testing.T) { qCtx, cancel := context.WithCancel(context.Background()) defer cancel() // Insert into multiple shards - _, err = conn.Execute(qCtx, "begin", nil) + _, err = conn.Execute(qCtx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil, false) require.NoError(t, err) // Also do an update to a table that has a consistent lookup vindex. // We expect to see only the pre-session changes in the logs. - _, err = conn.Execute(qCtx, "update twopc_consistent_lookup set col = 22 where id = 4", nil) + _, err = conn.Execute(qCtx, "update twopc_consistent_lookup set col = 22 where id = 4", nil, false) require.NoError(t, err) // The caller ID is used to simulate the failure at the desired point. newCtx := callerid.NewContext(qCtx, callerid.NewEffectiveCallerID("RMPrepared_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail After RM prepared") testWarningAndTransactionStatus(t, conn, @@ -796,18 +796,18 @@ func TestDTResolveDuringRMPrepare(t *testing.T) { qCtx, cancel := context.WithCancel(context.Background()) defer cancel() // Insert into multiple shards - _, err = conn.Execute(qCtx, "begin", nil) + _, err = conn.Execute(qCtx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'bar')", nil, false) require.NoError(t, err) // The caller ID is used to simulate the failure at the desired point. newCtx := callerid.NewContext(qCtx, callerid.NewEffectiveCallerID("RMPrepare_-40_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail During RM prepare") testWarningAndTransactionStatus(t, conn, @@ -862,18 +862,18 @@ func TestDTResolveDuringRMCommit(t *testing.T) { qCtx, cancel := context.WithCancel(context.Background()) defer cancel() // Insert into multiple shards - _, err = conn.Execute(qCtx, "begin", nil) + _, err = conn.Execute(qCtx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'apa')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(10,'apa')", nil, false) require.NoError(t, err) // The caller ID is used to simulate the failure at the desired point. newCtx := callerid.NewContext(qCtx, callerid.NewEffectiveCallerID("RMCommit_-40_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail During RM commit") testWarningAndTransactionStatus(t, conn, @@ -945,16 +945,16 @@ func TestDTResolveAfterTransactionRecord(t *testing.T) { qCtx, cancel := context.WithCancel(context.Background()) defer cancel() // Insert into multiple shards - _, err = conn.Execute(qCtx, "begin", nil) + _, err = conn.Execute(qCtx, "begin", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(7,'foo')", nil, false) require.NoError(t, err) - _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil) + _, err = conn.Execute(qCtx, "insert into twopc_user(id, name) values(8,'bar')", nil, false) require.NoError(t, err) // The caller ID is used to simulate the failure at the desired point. newCtx := callerid.NewContext(qCtx, callerid.NewEffectiveCallerID("TRCreated_FailNow", "", ""), nil) - _, err = conn.Execute(newCtx, "commit", nil) + _, err = conn.Execute(newCtx, "commit", nil, false) require.ErrorContains(t, err, "Fail After TR created") testWarningAndTransactionStatus(t, conn, @@ -1000,7 +1000,7 @@ func testWarningAndTransactionStatus(t *testing.T, conn *vtgateconn.VTGateSessio txConcluded bool, txState string, txParticipants string) { t.Helper() - qr, err := conn.Execute(context.Background(), "show warnings", nil) + qr, err := conn.Execute(context.Background(), "show warnings", nil, false) require.NoError(t, err) require.Len(t, qr.Rows, 1) @@ -1015,7 +1015,7 @@ func testWarningAndTransactionStatus(t *testing.T, conn *vtgateconn.VTGateSessio require.Greater(t, indx, 0) dtid := w.Msg[:indx] - qr, err = conn.Execute(context.Background(), fmt.Sprintf(`show transaction status for '%v'`, dtid), nil) + qr, err = conn.Execute(context.Background(), fmt.Sprintf(`show transaction status for '%v'`, dtid), nil, false) require.NoError(t, err) // validate transaction status @@ -1304,7 +1304,7 @@ func executeReturnError(ctx context.Context, t *testing.T, ss *vtgateconn.VTGate // sort by shard sortShard(ss) } - _, err := ss.Execute(ctx, sql, nil) + _, err := ss.Execute(ctx, sql, nil, false) return err } diff --git a/go/test/endtoend/vtcombo/recreate/recreate_test.go b/go/test/endtoend/vtcombo/recreate/recreate_test.go index 496d26c8062..98cb871f276 100644 --- a/go/test/endtoend/vtcombo/recreate/recreate_test.go +++ b/go/test/endtoend/vtcombo/recreate/recreate_test.go @@ -105,10 +105,10 @@ func TestDropAndRecreateWithSameShards(t *testing.T) { mysqlConnCountBefore, err := getMySQLConnectionCount(ctx, cur) require.Nil(t, err) - _, err = cur.Execute(ctx, "DROP DATABASE "+ks1, nil) + _, err = cur.Execute(ctx, "DROP DATABASE "+ks1, nil, false) require.Nil(t, err) - _, err = cur.Execute(ctx, "CREATE DATABASE "+ks1, nil) + _, err = cur.Execute(ctx, "CREATE DATABASE "+ks1, nil, false) require.Nil(t, err) assertTabletsPresent(t) @@ -124,7 +124,7 @@ func TestDropAndRecreateWithSameShards(t *testing.T) { } func getMySQLConnectionCount(ctx context.Context, session *vtgateconn.VTGateSession) (int, error) { - result, err := session.Execute(ctx, "select variable_value from performance_schema.global_status where variable_name='threads_connected'", nil) + result, err := session.Execute(ctx, "select variable_value from performance_schema.global_status where variable_name='threads_connected'", nil, false) if err != nil { return 0, err } diff --git a/go/test/endtoend/vtcombo/vttest_sample_test.go b/go/test/endtoend/vtcombo/vttest_sample_test.go index af0decca3d3..1711523dc61 100644 --- a/go/test/endtoend/vtcombo/vttest_sample_test.go +++ b/go/test/endtoend/vtcombo/vttest_sample_test.go @@ -178,7 +178,7 @@ func assertInsertedRowsExist(ctx context.Context, t *testing.T, conn *vtgateconn bindVariables := map[string]*querypb.BindVariable{ "id_start": {Type: querypb.Type_UINT64, Value: []byte(strconv.FormatInt(int64(idStart), 10))}, } - res, err := cur.Execute(ctx, "select * from test_table where id >= :id_start", bindVariables) + res, err := cur.Execute(ctx, "select * from test_table where id >= :id_start", bindVariables, false) require.NoError(t, err) assert.Equal(t, rowCount, len(res.Rows)) @@ -199,7 +199,7 @@ func assertRouting(ctx context.Context, t *testing.T, db *sql.DB) { func assertCanInsertRow(ctx context.Context, t *testing.T, conn *vtgateconn.VTGateConn) { cur := conn.Session(ks1+":80-@primary", nil) - _, err := cur.Execute(ctx, "begin", nil) + _, err := cur.Execute(ctx, "begin", nil, false) require.NoError(t, err) i := 0x810000000000000 @@ -209,10 +209,10 @@ func assertCanInsertRow(ctx context.Context, t *testing.T, conn *vtgateconn.VTGa "keyspace_id": {Type: querypb.Type_UINT64, Value: []byte(strconv.FormatInt(int64(i), 10))}, } query := "insert into test_table (id, msg, keyspace_id) values (:id, :msg, :keyspace_id)" - _, err = cur.Execute(ctx, query, bindVariables) + _, err = cur.Execute(ctx, query, bindVariables, false) require.NoError(t, err) - _, err = cur.Execute(ctx, "commit", nil) + _, err = cur.Execute(ctx, "commit", nil, false) require.NoError(t, err) } @@ -220,7 +220,7 @@ func insertManyRows(ctx context.Context, t *testing.T, conn *vtgateconn.VTGateCo cur := conn.Session(ks1+":-80@primary", nil) query := "insert into test_table (id, msg, keyspace_id) values (:id, :msg, :keyspace_id)" - _, err := cur.Execute(ctx, "begin", nil) + _, err := cur.Execute(ctx, "begin", nil, false) require.NoError(t, err) for i := idStart; i < idStart+rowCount; i++ { @@ -229,11 +229,11 @@ func insertManyRows(ctx context.Context, t *testing.T, conn *vtgateconn.VTGateCo "msg": {Type: querypb.Type_VARCHAR, Value: []byte("test" + strconv.FormatInt(int64(i), 10))}, "keyspace_id": {Type: querypb.Type_UINT64, Value: []byte(strconv.FormatInt(int64(i), 10))}, } - _, err = cur.Execute(ctx, query, bindVariables) + _, err = cur.Execute(ctx, query, bindVariables, false) require.NoError(t, err) } - _, err = cur.Execute(ctx, "commit", nil) + _, err = cur.Execute(ctx, "commit", nil, false) require.NoError(t, err) } @@ -301,17 +301,17 @@ func assertTransactionalityAndRollbackObeyed(ctx context.Context, t *testing.T, "keyspace_id": {Type: querypb.Type_UINT64, Value: []byte(strconv.FormatInt(int64(i), 10))}, } query := "insert into test_table (id, msg, keyspace_id) values (:id, :msg, :keyspace_id)" - _, err := cur.Execute(ctx, query, bindVariables) + _, err := cur.Execute(ctx, query, bindVariables, false) require.NoError(t, err) bindVariables = map[string]*querypb.BindVariable{ "msg": {Type: querypb.Type_VARCHAR, Value: []byte(msg)}, } - res, err := cur.Execute(ctx, "select * from test_table where msg = :msg", bindVariables) + res, err := cur.Execute(ctx, "select * from test_table where msg = :msg", bindVariables, false) require.NoError(t, err) require.Equal(t, 1, len(res.Rows)) - _, err = cur.Execute(ctx, "begin", nil) + _, err = cur.Execute(ctx, "begin", nil, false) require.NoError(t, err) msg2 := msg + "2" @@ -320,16 +320,16 @@ func assertTransactionalityAndRollbackObeyed(ctx context.Context, t *testing.T, "msg": {Type: querypb.Type_VARCHAR, Value: []byte(msg2)}, } query = "update test_table set msg = :msg where id = :id" - _, err = cur.Execute(ctx, query, bindVariables) + _, err = cur.Execute(ctx, query, bindVariables, false) require.NoError(t, err) - _, err = cur.Execute(ctx, "rollback", nil) + _, err = cur.Execute(ctx, "rollback", nil, false) require.NoError(t, err) bindVariables = map[string]*querypb.BindVariable{ "msg": {Type: querypb.Type_VARCHAR, Value: []byte(msg2)}, } - res, err = cur.Execute(ctx, "select * from test_table where msg = :msg", bindVariables) + res, err = cur.Execute(ctx, "select * from test_table where msg = :msg", bindVariables, false) require.NoError(t, err) require.Equal(t, 0, len(res.Rows)) } diff --git a/go/test/endtoend/vtgate/grpc_api/acl_test.go b/go/test/endtoend/vtgate/grpc_api/acl_test.go index d0ac003169d..d14841cb523 100644 --- a/go/test/endtoend/vtgate/grpc_api/acl_test.go +++ b/go/test/endtoend/vtgate/grpc_api/acl_test.go @@ -39,7 +39,7 @@ func TestEffectiveCallerIDWithAccess(t *testing.T) { session := vtgateConn.Session(keyspaceName+"@primary", nil) query := "SELECT id FROM test_table" ctx = callerid.NewContext(ctx, callerid.NewEffectiveCallerID("user_with_access", "", ""), nil) - _, err = session.Execute(ctx, query, nil) + _, err = session.Execute(ctx, query, nil, false) assert.NoError(t, err) } @@ -55,7 +55,7 @@ func TestEffectiveCallerIDWithNoAccess(t *testing.T) { session := vtgateConn.Session(keyspaceName+"@primary", nil) query := "SELECT id FROM test_table" ctx = callerid.NewContext(ctx, callerid.NewEffectiveCallerID("user_no_access", "", ""), nil) - _, err = session.Execute(ctx, query, nil) + _, err = session.Execute(ctx, query, nil, false) require.Error(t, err) assert.ErrorContains(t, err, "Select command denied to user") assert.ErrorContains(t, err, "for table 'test_table' (ACL check error)") @@ -72,7 +72,7 @@ func TestAuthenticatedUserWithAccess(t *testing.T) { session := vtgateConn.Session(keyspaceName+"@primary", nil) query := "SELECT id FROM test_table" - _, err = session.Execute(ctx, query, nil) + _, err = session.Execute(ctx, query, nil, false) assert.NoError(t, err) } @@ -87,7 +87,7 @@ func TestAuthenticatedUserNoAccess(t *testing.T) { session := vtgateConn.Session(keyspaceName+"@primary", nil) query := "SELECT id FROM test_table" - _, err = session.Execute(ctx, query, nil) + _, err = session.Execute(ctx, query, nil, false) require.Error(t, err) assert.ErrorContains(t, err, "Select command denied to user") assert.ErrorContains(t, err, "for table 'test_table' (ACL check error)") @@ -104,7 +104,7 @@ func TestUnauthenticatedUser(t *testing.T) { session := vtgateConn.Session(keyspaceName+"@primary", nil) query := "SELECT id FROM test_table" - _, err = session.Execute(ctx, query, nil) + _, err = session.Execute(ctx, query, nil, false) require.Error(t, err) assert.ErrorContains(t, err, "invalid credentials") } diff --git a/go/test/endtoend/vtgate/grpc_api/execute_test.go b/go/test/endtoend/vtgate/grpc_api/execute_test.go index b1a5f3b8d80..8faaea8102f 100644 --- a/go/test/endtoend/vtgate/grpc_api/execute_test.go +++ b/go/test/endtoend/vtgate/grpc_api/execute_test.go @@ -102,7 +102,7 @@ func exec(ctx context.Context, conn *vtgateconn.VTGateSession, sql string, bv ma if options != nil && options.Workload == querypb.ExecuteOptions_OLAP { return streamExec(ctx, conn, sql, bv) } - res, err := conn.Execute(ctx, sql, bv) + res, err := conn.Execute(ctx, sql, bv, false) return res, conn.SessionPb(), err } diff --git a/go/test/endtoend/vtgate/queries/reference/main_test.go b/go/test/endtoend/vtgate/queries/reference/main_test.go index 95e19ef596a..22db6c6db63 100644 --- a/go/test/endtoend/vtgate/queries/reference/main_test.go +++ b/go/test/endtoend/vtgate/queries/reference/main_test.go @@ -128,13 +128,13 @@ func TestMain(m *testing.M) { } shard := fmt.Sprintf("%s/%s@primary", ks.Name, s.Name) session := vtgateConn.Session(shard, nil) - _, err := session.Execute(ctx, "SHOW CREATE TABLE zip_detail", map[string]*querypb.BindVariable{}) + _, err := session.Execute(ctx, "SHOW CREATE TABLE zip_detail", map[string]*querypb.BindVariable{}, false) if err != nil { fmt.Fprintf(os.Stderr, "Failed to SHOW CREATE TABLE zip_detail; might not exist yet: %v\n", err) time.Sleep(1 * time.Second) continue } - qr, err := session.Execute(ctx, "SELECT * FROM zip_detail", map[string]*querypb.BindVariable{}) + qr, err := session.Execute(ctx, "SELECT * FROM zip_detail", map[string]*querypb.BindVariable{}, false) if err != nil { fmt.Fprintf(os.Stderr, "Failed to query sharded keyspace for zip_detail rows: %v\n", err) done <- false @@ -184,7 +184,7 @@ func TestMain(m *testing.M) { INSERT INTO zip_detail(id, zip_id, discontinued_at) VALUES (1, 1, '2022-05-13'), (2, 2, '2022-08-15') - `, map[string]*querypb.BindVariable{}); err != nil { + `, map[string]*querypb.BindVariable{}, false); err != nil { return 1 } @@ -194,7 +194,7 @@ func TestMain(m *testing.M) { VALUES (1, 1, 'Failed delivery due to discontinued zipcode.'), (2, 2, 'Failed delivery due to discontinued zipcode.'), (3, 3, 'Failed delivery due to unknown reason.'); - `, map[string]*querypb.BindVariable{}); err != nil { + `, map[string]*querypb.BindVariable{}, false); err != nil { return 1 } diff --git a/go/test/utils/diff.go b/go/test/utils/diff.go index 2344c027c39..be9cf6adcc7 100644 --- a/go/test/utils/diff.go +++ b/go/test/utils/diff.go @@ -68,7 +68,8 @@ func MustMatchFn(ignoredFields ...string) func(t *testing.T, want, got any, errM t.Helper() diff := cmp.Diff(want, got, diffOpts...) if diff != "" { - require.FailNow(t, "%v: (-want +got)\n%v", errMsg, diff) + t.Errorf("%v: (-want +got)\n%v", errMsg, diff) + t.FailNow() } } } diff --git a/go/test/vschemawrapper/vschema_wrapper.go b/go/test/vschemawrapper/vschema_wrapper.go index b48d7f5d0b7..0eefe18e5b8 100644 --- a/go/test/vschemawrapper/vschema_wrapper.go +++ b/go/test/vschemawrapper/vschema_wrapper.go @@ -50,7 +50,7 @@ type VSchemaWrapper struct { V *vindexes.VSchema Keyspace *vindexes.Keyspace TabletType_ topodatapb.TabletType - Dest key.Destination + Dest key.ShardDestination SysVarEnabled bool ForeignKeyChecksState *bool Version plancontext.PlannerVersion @@ -109,16 +109,12 @@ func (vw *VSchemaWrapper) GetPrepareData(stmtName string) *vtgatepb.PrepareData return nil } -func (vw *VSchemaWrapper) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) { +func (vw *VSchemaWrapper) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, error) { plan, err := vw.TestBuilder(query, vw, vw.CurrentDb()) if err != nil { - return nil, nil, err - } - stmt, _, err := vw.Env.Parser().Parse2(query) - if err != nil { - return nil, nil, err + return nil, err } - return plan, stmt, nil + return plan, nil } func (vw *VSchemaWrapper) ClearPrepareData(string) {} @@ -227,7 +223,7 @@ func (vw *VSchemaWrapper) SysVarSetEnabled() bool { return vw.SysVarEnabled } -func (vw *VSchemaWrapper) TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) { +func (vw *VSchemaWrapper) TargetDestination(qualifier string) (key.ShardDestination, *vindexes.Keyspace, topodatapb.TabletType, error) { var keyspaceName string if vw.Keyspace != nil { keyspaceName = vw.Keyspace.Name @@ -249,11 +245,11 @@ func (vw *VSchemaWrapper) TabletType() topodatapb.TabletType { return vw.TabletType_ } -func (vw *VSchemaWrapper) Destination() key.Destination { +func (vw *VSchemaWrapper) ShardDestination() key.ShardDestination { return vw.Dest } -func (vw *VSchemaWrapper) FindTable(tab sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.Destination, error) { +func (vw *VSchemaWrapper) FindTable(tab sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.ShardDestination, error) { destKeyspace, destTabletType, destTarget, err := topoproto.ParseDestination(tab.Qualifier.String(), topodatapb.TabletType_PRIMARY) if err != nil { return nil, destKeyspace, destTabletType, destTarget, err @@ -284,7 +280,7 @@ func (vw *VSchemaWrapper) FindViewTarget(name sqlparser.TableName) (*vindexes.Ke return nil, nil } -func (vw *VSchemaWrapper) FindTableOrVindex(tab sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (vw *VSchemaWrapper) FindTableOrVindex(tab sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { return vw.Vcursor.FindTableOrVindex(tab) } diff --git a/go/vt/key/destination.go b/go/vt/key/destination.go index 77287c782e7..7f190c5e694 100644 --- a/go/vt/key/destination.go +++ b/go/vt/key/destination.go @@ -31,22 +31,22 @@ import ( // AnyShardPicker makes a choice on what shard to use when any shard will do. Used for testing. var AnyShardPicker DestinationAnyShardPicker = DestinationAnyShardPickerRandomShard{} -// Destination is an interface definition for a query destination, +// ShardDestination is an interface definition for a query destination, // within a given Keyspace / Tablet Type. It is meant to be an internal // data structure, with multiple possible implementations. // The srvtopo package can resolve Destinations into actual Targets. -type Destination interface { - // Resolve calls the callback for every shard Destination +type ShardDestination interface { + // Resolve calls the callback for every shard ShardDestination // resolves into, given the shards list. // The returned error must be generated by vterrors. Resolve([]*topodatapb.ShardReference, func(shard string) error) error - // String returns a printable version of the Destination. + // String returns a printable version of the ShardDestination. String() string } // DestinationsString returns a printed version of the destination array. -func DestinationsString(destinations []Destination) string { +func DestinationsString(destinations []ShardDestination) string { var buffer strings.Builder buffer.WriteString("Destinations:") for i, d := range destinations { @@ -63,15 +63,15 @@ func DestinationsString(destinations []Destination) string { // // DestinationShard is the destination for a single Shard. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationShard string -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationShard) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { return addShard(string(d)) } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationShard) String() string { return "DestinationShard(" + string(d) + ")" } @@ -81,10 +81,10 @@ func (d DestinationShard) String() string { // // DestinationShards is the destination for multiple shards. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationShards []string -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationShards) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { for _, shard := range d { if err := addShard(shard); err != nil { @@ -94,7 +94,7 @@ func (d DestinationShards) Resolve(allShards []*topodatapb.ShardReference, addSh return nil } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationShards) String() string { return "DestinationShards(" + strings.Join(d, ",") + ")" } @@ -106,7 +106,7 @@ func (d DestinationShards) String() string { // DestinationExactKeyRange is the destination for a single KeyRange. // The KeyRange must map exactly to one or more shards, and cannot // start or end in the middle of a shard. -// It implements the Destination interface. +// It implements the ShardDestination interface. // (it cannot be just a type *topodatapb.KeyRange, as then the receiver // methods don't work. And it can't be topodatapb.KeyRange either, // as then the methods are on *DestinationExactKeyRange, and the original @@ -115,12 +115,12 @@ type DestinationExactKeyRange struct { KeyRange *topodatapb.KeyRange } -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationExactKeyRange) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { return processExactKeyRange(allShards, d.KeyRange, addShard) } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationExactKeyRange) String() string { return "DestinationExactKeyRange(" + KeyRangeString(d.KeyRange) + ")" } @@ -159,7 +159,7 @@ func processExactKeyRange(allShards []*topodatapb.ShardReference, kr *topodatapb // // DestinationKeyRange is the destination for a single KeyRange. -// It implements the Destination interface. +// It implements the ShardDestination interface. // (it cannot be just a type *topodatapb.KeyRange, as then the receiver // methods don't work. And it can't be topodatapb.KeyRange either, // as then the methods are on *DestinationKeyRange, and the original @@ -168,12 +168,12 @@ type DestinationKeyRange struct { KeyRange *topodatapb.KeyRange } -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationKeyRange) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { return processKeyRange(allShards, d.KeyRange, addShard) } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationKeyRange) String() string { return "DestinationKeyRange(" + KeyRangeString(d.KeyRange) + ")" } @@ -196,10 +196,10 @@ func processKeyRange(allShards []*topodatapb.ShardReference, kr *topodatapb.KeyR // // DestinationKeyspaceID is the destination for a single KeyspaceID. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationKeyspaceID []byte -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationKeyspaceID) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { shard, err := GetShardForKeyspaceID(allShards, d) if err != nil { @@ -208,7 +208,7 @@ func (d DestinationKeyspaceID) Resolve(allShards []*topodatapb.ShardReference, a return addShard(shard) } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationKeyspaceID) String() string { return "DestinationKeyspaceID(" + hex.EncodeToString(d) + ")" } @@ -232,10 +232,10 @@ func GetShardForKeyspaceID(allShards []*topodatapb.ShardReference, keyspaceID [] // // DestinationKeyspaceIDs is the destination for multiple KeyspaceIDs. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationKeyspaceIDs [][]byte -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationKeyspaceIDs) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { for _, ksid := range d { shard, err := GetShardForKeyspaceID(allShards, ksid) @@ -249,7 +249,7 @@ func (d DestinationKeyspaceIDs) Resolve(allShards []*topodatapb.ShardReference, return nil } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationKeyspaceIDs) String() string { var buffer strings.Builder buffer.WriteString("DestinationKeyspaceIDs(") @@ -282,10 +282,10 @@ func (dp DestinationAnyShardPickerRandomShard) PickShard(shardCount int) int { // // DestinationAnyShard is the destination for any one shard in the keyspace. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationAnyShard struct{} -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationAnyShard) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { if len(allShards) == 0 { return vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "no shard in keyspace") @@ -293,7 +293,7 @@ func (d DestinationAnyShard) Resolve(allShards []*topodatapb.ShardReference, add return addShard(allShards[AnyShardPicker.PickShard(len(allShards))].Name) } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationAnyShard) String() string { return "DestinationAnyShard()" } @@ -304,10 +304,10 @@ func (d DestinationAnyShard) String() string { // DestinationAllShards is the destination for all the shards in the // keyspace. This usually maps to the first one in the list. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationAllShards struct{} -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationAllShards) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { for _, shard := range allShards { if err := addShard(shard.Name); err != nil { @@ -318,7 +318,7 @@ func (d DestinationAllShards) Resolve(allShards []*topodatapb.ShardReference, ad return nil } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationAllShards) String() string { return "DestinationAllShards()" } @@ -328,15 +328,15 @@ func (d DestinationAllShards) String() string { // // DestinationNone is a destination that doesn't resolve to any shard. -// It implements the Destination interface. +// It implements the ShardDestination interface. type DestinationNone struct{} -// Resolve is part of the Destination interface. +// Resolve is part of the ShardDestination interface. func (d DestinationNone) Resolve(allShards []*topodatapb.ShardReference, addShard func(shard string) error) error { return nil } -// String is part of the Destination interface. +// String is part of the ShardDestination interface. func (d DestinationNone) String() string { return "DestinationNone()" } diff --git a/go/vt/key/destination_test.go b/go/vt/key/destination_test.go index f348b9ffa25..82b1a278995 100644 --- a/go/vt/key/destination_test.go +++ b/go/vt/key/destination_test.go @@ -258,7 +258,7 @@ func TestDestinationsString(t *testing.T) { End: []byte{0x40}, } - got := DestinationsString([]Destination{ + got := DestinationsString([]ShardDestination{ DestinationShard("2"), DestinationShards{"2", "3"}, DestinationExactKeyRange{KeyRange: kr2040}, diff --git a/go/vt/proto/vtgate/vtgate.pb.go b/go/vt/proto/vtgate/vtgate.pb.go index 3fbbf494ede..c025506ebdf 100644 --- a/go/vt/proto/vtgate/vtgate.pb.go +++ b/go/vt/proto/vtgate/vtgate.pb.go @@ -566,6 +566,7 @@ type ExecuteRequest struct { Session *Session `protobuf:"bytes,2,opt,name=session,proto3" json:"session,omitempty"` // query is the query and bind variables to execute. Query *query.BoundQuery `protobuf:"bytes,3,opt,name=query,proto3" json:"query,omitempty"` + Prepared bool `protobuf:"varint,8,opt,name=prepared,proto3" json:"prepared,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -621,6 +622,13 @@ func (x *ExecuteRequest) GetQuery() *query.BoundQuery { return nil } +func (x *ExecuteRequest) GetPrepared() bool { + if x != nil { + return x.Prepared + } + return false +} + // ExecuteResponse is the returned value from Execute. type ExecuteResponse struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -1343,6 +1351,7 @@ type PrepareResponse struct { Session *Session `protobuf:"bytes,2,opt,name=session,proto3" json:"session,omitempty"` // fields contains the fields, only set if error is unset. Fields []*query.Field `protobuf:"bytes,3,rep,name=fields,proto3" json:"fields,omitempty"` + ParamsCount uint32 `protobuf:"varint,4,opt,name=params_count,json=paramsCount,proto3" json:"params_count,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -1398,6 +1407,13 @@ func (x *PrepareResponse) GetFields() []*query.Field { return nil } +func (x *PrepareResponse) GetParamsCount() uint32 { + if x != nil { + return x.ParamsCount + } + return 0 +} + // CloseSessionRequest is the payload to CloseSession. type CloseSessionRequest struct { state protoimpl.MessageState `protogen:"open.v1"` @@ -1735,7 +1751,7 @@ var file_vtgate_proto_rawDesc = string([]byte{ 0x6f, 0x75, 0x74, 0x12, 0x2e, 0x0a, 0x13, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x6b, 0x5f, 0x67, 0x74, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x72, 0x61, 0x63, 0x6b, 0x47, 0x74, - 0x69, 0x64, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, + 0x69, 0x64, 0x73, 0x22, 0xc6, 0x01, 0x0a, 0x0e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, @@ -1744,148 +1760,152 @@ var file_vtgate_proto_rawDesc = string([]byte{ 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, - 0x08, 0x05, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, - 0x22, 0x8f, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, - 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x18, 0x03, 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, 0xb3, 0x01, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x42, 0x61, - 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, + 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x65, 0x70, + 0x61, 0x72, 0x65, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x70, 0x72, 0x65, 0x70, + 0x61, 0x72, 0x65, 0x64, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, + 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x8f, 0x01, 0x0a, + 0x0f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x25, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, 0x6f, 0x72, + 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, + 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x03, 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, 0xb3, + 0x01, 0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, + 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, + 0x2b, 0x0a, 0x07, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x07, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x4a, 0x04, 0x08, 0x04, + 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x4a, 0x04, + 0x08, 0x07, 0x10, 0x08, 0x22, 0x9a, 0x01, 0x0a, 0x14, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, + 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, + 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, + 0x30, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x57, + 0x69, 0x74, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x14, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, - 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, - 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x07, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, - 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x07, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, - 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x4a, 0x04, 0x08, 0x06, - 0x10, 0x07, 0x4a, 0x04, 0x08, 0x07, 0x10, 0x08, 0x22, 0x9a, 0x01, 0x0a, 0x14, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x4a, 0x04, 0x08, 0x03, + 0x10, 0x04, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, 0x10, 0x06, 0x22, 0x6e, + 0x0a, 0x15, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 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, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x5d, + 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, + 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, + 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, + 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x74, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x74, 0x69, 0x64, 0x22, 0x1c, 0x0a, + 0x1a, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xef, 0x02, 0x0a, 0x0c, + 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x23, 0x0a, 0x0d, + 0x6d, 0x69, 0x6e, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x5f, 0x73, 0x6b, 0x65, 0x77, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x53, 0x6b, 0x65, + 0x77, 0x12, 0x2d, 0x0a, 0x12, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x5f, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x68, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, + 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x4f, + 0x6e, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, + 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x27, + 0x0a, 0x0f, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, + 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x65, 0x6c, 0x6c, 0x50, 0x72, 0x65, + 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x5f, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x3c, 0x0a, 0x1a, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x48, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x12, 0x43, 0x0a, 0x1e, 0x69, 0x6e, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, 0x6a, 0x6f, 0x75, 0x72, + 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x1b, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, + 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xf6, 0x01, + 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x35, + 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, + 0x01, 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, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x47, 0x74, 0x69, 0x64, 0x52, 0x05, 0x76, 0x67, 0x74, 0x69, 0x64, 0x12, 0x2a, + 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, + 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x2a, 0x0a, 0x05, 0x66, 0x6c, + 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x76, 0x74, 0x67, 0x61, + 0x74, 0x65, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x52, + 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x0e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, + 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, + 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, + 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x27, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x22, 0xac, 0x01, 0x0a, 0x0f, 0x50, + 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, + 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x05, + 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x70, 0x61, + 0x72, 0x61, 0x6d, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x6e, 0x0a, 0x13, 0x43, 0x6c, 0x6f, + 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, + 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, + 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x3d, 0x0a, 0x14, 0x43, 0x6c, 0x6f, + 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, - 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, - 0x75, 0x6c, 0x74, 0x57, 0x69, 0x74, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x07, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x14, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, - 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, - 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x05, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, - 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, - 0x4a, 0x04, 0x08, 0x03, 0x10, 0x04, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x4a, 0x04, 0x08, 0x05, - 0x10, 0x06, 0x22, 0x6e, 0x0a, 0x15, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x78, 0x65, 0x63, - 0x75, 0x74, 0x65, 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, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, - 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x22, 0x5d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x54, 0x72, 0x61, - 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, - 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, - 0x04, 0x64, 0x74, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x74, 0x69, - 0x64, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x54, 0x72, 0x61, 0x6e, - 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0xef, 0x02, 0x0a, 0x0c, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x6c, 0x61, 0x67, 0x73, - 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x69, 0x6e, 0x69, 0x6d, 0x69, 0x7a, 0x65, 0x5f, 0x73, 0x6b, 0x65, - 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x69, 0x6d, 0x69, 0x7a, - 0x65, 0x53, 0x6b, 0x65, 0x77, 0x12, 0x2d, 0x0a, 0x12, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, - 0x61, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x11, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x49, 0x6e, 0x74, 0x65, - 0x72, 0x76, 0x61, 0x6c, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x6f, 0x6e, 0x5f, - 0x72, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x73, - 0x74, 0x6f, 0x70, 0x4f, 0x6e, 0x52, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x12, 0x14, 0x0a, 0x05, - 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, - 0x6c, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x65, 0x6c, 0x6c, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, - 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x65, 0x6c, - 0x6c, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x3c, - 0x0a, 0x1a, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, - 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x18, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4b, 0x65, 0x79, 0x73, 0x70, 0x61, - 0x63, 0x65, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x73, 0x12, 0x43, 0x0a, 0x1e, - 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x68, 0x61, 0x72, 0x64, 0x5f, - 0x6a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x08, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x1b, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x52, 0x65, 0x73, - 0x68, 0x61, 0x72, 0x64, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x22, 0xf6, 0x01, 0x0a, 0x0e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x35, 0x0a, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x02, 0x20, 0x01, 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, 0x0a, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, 0x05, 0x76, 0x67, 0x74, - 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, - 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x47, 0x74, 0x69, 0x64, 0x52, 0x05, 0x76, 0x67, 0x74, - 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x2a, - 0x0a, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, - 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x56, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x46, 0x6c, - 0x61, 0x67, 0x73, 0x52, 0x05, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x22, 0x3d, 0x0a, 0x0f, 0x56, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, - 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, - 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x92, 0x01, 0x0a, 0x0e, 0x50, 0x72, - 0x65, 0x70, 0x61, 0x72, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x09, - 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x44, - 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, - 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, - 0x6e, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x22, 0x89, - 0x01, 0x0a, 0x0f, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, - 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, - 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x22, 0x6e, 0x0a, 0x13, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x2c, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x6c, - 0x6c, 0x65, 0x72, 0x49, 0x44, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x49, 0x64, 0x12, - 0x29, 0x0a, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, - 0x6e, 0x52, 0x07, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x3d, 0x0a, 0x14, 0x43, 0x6c, - 0x6f, 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0f, 0x2e, 0x76, 0x74, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x50, 0x43, 0x45, 0x72, 0x72, - 0x6f, 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x2a, 0x44, 0x0a, 0x0f, 0x54, 0x72, 0x61, - 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0f, 0x0a, 0x0b, - 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, - 0x06, 0x53, 0x49, 0x4e, 0x47, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x55, 0x4c, - 0x54, 0x49, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x57, 0x4f, 0x50, 0x43, 0x10, 0x03, 0x2a, - 0x3c, 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x0a, - 0x0a, 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x50, 0x52, - 0x45, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x4f, 0x53, 0x54, 0x10, 0x02, 0x12, 0x0e, 0x0a, - 0x0a, 0x41, 0x55, 0x54, 0x4f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x42, 0x36, 0x0a, - 0x0f, 0x69, 0x6f, 0x2e, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x5a, 0x23, 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, 0x76, - 0x74, 0x67, 0x61, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x2a, 0x44, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, + 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, + 0x53, 0x49, 0x4e, 0x47, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x4d, 0x55, 0x4c, 0x54, + 0x49, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x57, 0x4f, 0x50, 0x43, 0x10, 0x03, 0x2a, 0x3c, + 0x0a, 0x0b, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x0a, 0x0a, + 0x06, 0x4e, 0x4f, 0x52, 0x4d, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x50, 0x52, 0x45, + 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x50, 0x4f, 0x53, 0x54, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, + 0x41, 0x55, 0x54, 0x4f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x10, 0x03, 0x42, 0x36, 0x0a, 0x0f, + 0x69, 0x6f, 0x2e, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x5a, + 0x23, 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, 0x76, 0x74, + 0x67, 0x61, 0x74, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( diff --git a/go/vt/proto/vtgate/vtgate_vtproto.pb.go b/go/vt/proto/vtgate/vtgate_vtproto.pb.go index f4bdbeeec93..383ea6300eb 100644 --- a/go/vt/proto/vtgate/vtgate_vtproto.pb.go +++ b/go/vt/proto/vtgate/vtgate_vtproto.pb.go @@ -186,6 +186,7 @@ func (m *ExecuteRequest) CloneVT() *ExecuteRequest { r.CallerId = m.CallerId.CloneVT() r.Session = m.Session.CloneVT() r.Query = m.Query.CloneVT() + r.Prepared = m.Prepared if len(m.unknownFields) > 0 { r.unknownFields = make([]byte, len(m.unknownFields)) copy(r.unknownFields, m.unknownFields) @@ -431,6 +432,7 @@ func (m *PrepareResponse) CloneVT() *PrepareResponse { r := new(PrepareResponse) r.Error = m.Error.CloneVT() r.Session = m.Session.CloneVT() + r.ParamsCount = m.ParamsCount if rhs := m.Fields; rhs != nil { tmpContainer := make([]*query.Field, len(rhs)) for k, v := range rhs { @@ -1020,6 +1022,16 @@ func (m *ExecuteRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.Prepared { + i-- + if m.Prepared { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } if m.Query != nil { size, err := m.Query.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { @@ -1760,6 +1772,11 @@ func (m *PrepareResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } + if m.ParamsCount != 0 { + i = protohelpers.EncodeVarint(dAtA, i, uint64(m.ParamsCount)) + i-- + dAtA[i] = 0x20 + } if len(m.Fields) > 0 { for iNdEx := len(m.Fields) - 1; iNdEx >= 0; iNdEx-- { size, err := m.Fields[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) @@ -2116,6 +2133,9 @@ func (m *ExecuteRequest) SizeVT() (n int) { l = m.Query.SizeVT() n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) } + if m.Prepared { + n += 2 + } n += len(m.unknownFields) return n } @@ -2382,6 +2402,9 @@ func (m *PrepareResponse) SizeVT() (n int) { n += 1 + l + protohelpers.SizeOfVarint(uint64(l)) } } + if m.ParamsCount != 0 { + n += 1 + protohelpers.SizeOfVarint(uint64(m.ParamsCount)) + } n += len(m.unknownFields) return n } @@ -4119,6 +4142,26 @@ func (m *ExecuteRequest) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Prepared", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Prepared = bool(v != 0) default: iNdEx = preIndex skippy, err := protohelpers.Skip(dAtA[iNdEx:]) @@ -5905,6 +5948,25 @@ func (m *PrepareResponse) UnmarshalVT(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ParamsCount", wireType) + } + m.ParamsCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return protohelpers.ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ParamsCount |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := protohelpers.Skip(dAtA[iNdEx:]) diff --git a/go/vt/schemadiff/semantics.go b/go/vt/schemadiff/semantics.go index a2a4b4f954f..6bbe848b46f 100644 --- a/go/vt/schemadiff/semantics.go +++ b/go/vt/schemadiff/semantics.go @@ -50,7 +50,7 @@ func newDeclarativeSchemaInformation(env *Environment) *declarativeSchemaInforma } // FindTableOrVindex implements the SchemaInformation interface -func (si *declarativeSchemaInformation) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (si *declarativeSchemaInformation) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { table := si.Tables[tablename.Name.String()] return table, nil, "", 0, nil, nil } diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index 2aa7192bec2..c3a9b744c17 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -142,12 +142,11 @@ func CanNormalize(stmt Statement) bool { // CachePlan takes Statement and returns true if the query plan should be cached func CachePlan(stmt Statement) bool { - switch stmt.(type) { - case *Select, *Insert, *Update, *Delete, *Union, *Stream: - return !checkDirective(stmt, DirectiveSkipQueryPlanCache) - default: + _, supportSetVar := stmt.(SupportOptimizerHint) + if !supportSetVar { return false } + return !checkDirective(stmt, DirectiveSkipQueryPlanCache) } // MustRewriteAST takes Statement and returns true if RewriteAST must run on it for correct execution irrespective of user flags. @@ -422,11 +421,11 @@ func IsSimpleTuple(node Expr) bool { return false } -// IsLockingFunc returns true for all functions that are used to work with mysql advisory locks -func IsLockingFunc(node Expr) bool { - switch node.(type) { - case *LockingFunc: +func SupportsOptimizerHint(stmt StatementType) bool { + switch stmt { + case StmtSelect, StmtInsert, StmtUpdate, StmtDelete, StmtStream, StmtVStream: return true + default: + return false } - return false } diff --git a/go/vt/sqlparser/cached_size.go b/go/vt/sqlparser/cached_size.go index 36bc371c0c8..ffab0863f8e 100644 --- a/go/vt/sqlparser/cached_size.go +++ b/go/vt/sqlparser/cached_size.go @@ -3512,6 +3512,24 @@ func (cached *PurgeBinaryLogs) CachedSize(alloc bool) int64 { size += hack.RuntimeAllocSize(int64(len(cached.Before))) return size } +func (cached *QueryHints) CachedSize(alloc bool) int64 { + if cached == nil { + return int64(0) + } + size := int64(0) + if alloc { + size += int64(64) + } + // field Workload string + size += hack.RuntimeAllocSize(int64(len(cached.Workload))) + // field ForeignKeyChecks *bool + size += hack.RuntimeAllocSize(int64(1)) + // field Priority string + size += hack.RuntimeAllocSize(int64(len(cached.Priority))) + // field Timeout *int + size += hack.RuntimeAllocSize(int64(8)) + return size +} func (cached *ReferenceDefinition) CachedSize(alloc bool) int64 { if cached == nil { return int64(0) diff --git a/go/vt/sqlparser/normalizer.go b/go/vt/sqlparser/normalizer.go index dbded43765a..c0d0d8000ed 100644 --- a/go/vt/sqlparser/normalizer.go +++ b/go/vt/sqlparser/normalizer.go @@ -62,11 +62,13 @@ type ( onLeave map[*AliasedExpr]func(*AliasedExpr) parameterize bool + useASTQuery bool } // RewriteASTResult holds the result of rewriting the AST, including bind variable needs. RewriteASTResult struct { *BindVarNeeds - AST Statement // The rewritten AST + AST Statement // The rewritten AST + UpdateQueryFromAST bool } // VSchemaViews provides access to view definitions within the VSchema. VSchemaViews interface { @@ -99,8 +101,8 @@ var funcRewrites = map[string]string{ "row_count": RowCountName, } -// PrepareAST normalizes the input SQL statement and returns the rewritten AST along with bind variable information. -func PrepareAST( +// Normalize normalizes the input SQL statement and returns the rewritten AST along with bind variable information. +func Normalize( in Statement, reservedVars *ReservedVars, bindVars map[string]*querypb.BindVariable, @@ -114,16 +116,18 @@ func PrepareAST( ) (*RewriteASTResult, error) { nz := newNormalizer(reservedVars, bindVars, keyspace, selectLimit, setVarComment, sysVars, fkChecksState, views, parameterize) nz.shouldRewriteDatabaseFunc = shouldRewriteDatabaseFunc(in) + nz.determineQueryRewriteStrategy(in) + out := SafeRewrite(in, nz.walkDown, nz.walkUp) if nz.err != nil { return nil, nz.err } - r := &RewriteASTResult{ - AST: out.(Statement), - BindVarNeeds: nz.bindVarNeeds, - } - return r, nil + return &RewriteASTResult{ + AST: out.(Statement), + BindVarNeeds: nz.bindVarNeeds, + UpdateQueryFromAST: nz.useASTQuery, + }, nil } func newNormalizer( @@ -153,20 +157,29 @@ func newNormalizer( } } +func (nz *normalizer) determineQueryRewriteStrategy(in Statement) { + switch in.(type) { + case *Select, *Union, *Insert, *Update, *Delete, *CallProc, *Stream, *VExplainStmt: + nz.useASTQuery = true + case *Set: + nz.useASTQuery = true + nz.parameterize = false + default: + nz.parameterize = false + } +} + // walkDown processes nodes when traversing down the AST. // It handles normalization logic based on node types. func (nz *normalizer) walkDown(node, _ SQLNode) bool { switch node := node.(type) { - case *AssignmentExpr: - nz.err = vterrors.VT12001("Assignment expression") - return false case *Begin, *Commit, *Rollback, *Savepoint, *SRollback, *Release, *OtherAdmin, *Analyze, *PrepareStmt, *ExecuteStmt, *FramePoint, *ColName, TableName, *ConvertType: - // These statement don't need normalizing + // These statement do not need normalizing + return false + case *AssignmentExpr: + nz.err = vterrors.VT12001("Assignment expression") return false - case *Set: - // Disable parameterization within SET statements. - nz.parameterize = false case *DerivedTable: nz.inDerived++ case *Select: @@ -219,18 +232,22 @@ func (nz *normalizer) noteAliasedExprName(node *AliasedExpr) { // It finalizes normalization logic based on node types. func (nz *normalizer) walkUp(cursor *Cursor) bool { // Add SET_VAR comments if applicable. - if supportOptimizerHint, supports := cursor.Node().(SupportOptimizerHint); supports { + if stmt, supports := cursor.Node().(SupportOptimizerHint); supports { if nz.setVarComment != "" { - newComments, err := supportOptimizerHint.GetParsedComments().AddQueryHint(nz.setVarComment) + newComments, err := stmt.GetParsedComments().AddQueryHint(nz.setVarComment) if err != nil { nz.err = err return false } - supportOptimizerHint.SetComments(newComments) + stmt.SetComments(newComments) + nz.useASTQuery = true } + + // use foreign key checks of normalizer and set the query hint in the query. if nz.fkChecksState != nil { - newComments := supportOptimizerHint.GetParsedComments().SetMySQLSetVarValue(sysvars.ForeignKeyChecks, FkChecksStateString(nz.fkChecksState)) - supportOptimizerHint.SetComments(newComments) + newComments := stmt.GetParsedComments().SetMySQLSetVarValue(sysvars.ForeignKeyChecks, FkChecksStateString(nz.fkChecksState)) + stmt.SetComments(newComments) + nz.useASTQuery = true } } diff --git a/go/vt/sqlparser/normalizer_test.go b/go/vt/sqlparser/normalizer_test.go index 7c3e660ac9d..596df245c26 100644 --- a/go/vt/sqlparser/normalizer_test.go +++ b/go/vt/sqlparser/normalizer_test.go @@ -317,12 +317,10 @@ func TestNormalize(t *testing.T) { "bv1": sqltypes.TestBindVariable([]any{1, "2"}), }, }, { - // EXPLAIN queries - in: "explain select * from t where v1 in (1, '2')", - outstmt: "explain select * from t where v1 in ::bv1", - outbv: map[string]*querypb.BindVariable{ - "bv1": sqltypes.TestBindVariable([]any{1, "2"}), - }, + // EXPLAIN query will be normalized and not parameterized + in: "explain select @x from t where v1 in (1, '2')", + outstmt: "explain select :__vtudvx as `@x` from t where v1 in (1, '2')", + outbv: map[string]*querypb.BindVariable{}, }, { // NOT IN clause in: "select * from t where v1 not in (1, '2')", @@ -378,9 +376,9 @@ func TestNormalize(t *testing.T) { "bv1": sqltypes.ValueBindVariable(sqltypes.MakeTrusted(sqltypes.Datetime, []byte("2022-08-06 17:05:12"))), }, }, { - // TimestampVal should also be normalized - in: `explain select comms_by_companies.* from comms_by_companies where comms_by_companies.id = 'rjve634shXzaavKHbAH16ql6OrxJ' limit 1,1`, - outstmt: `explain select comms_by_companies.* from comms_by_companies where comms_by_companies.id = :comms_by_companies_id /* VARCHAR */ limit :bv1 /* INT64 */, :bv2 /* INT64 */`, + // TimestampVal should also be parameterized + in: `select comms_by_companies.* from comms_by_companies where comms_by_companies.id = 'rjve634shXzaavKHbAH16ql6OrxJ' limit 1,1`, + outstmt: `select comms_by_companies.* from comms_by_companies where comms_by_companies.id = :comms_by_companies_id /* VARCHAR */ limit :bv1 /* INT64 */, :bv2 /* INT64 */`, outbv: map[string]*querypb.BindVariable{ "bv1": sqltypes.Int64BindVariable(1), "bv2": sqltypes.Int64BindVariable(1), @@ -451,7 +449,7 @@ func TestNormalize(t *testing.T) { require.NoError(t, err) known := getBindvars(stmt) bv := make(map[string]*querypb.BindVariable) - out, err := PrepareAST(stmt, NewReservedVars(prefix, known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + out, err := Normalize(stmt, NewReservedVars(prefix, known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) require.NoError(t, err) assert.Equal(t, tc.outstmt, String(out.AST)) assert.Equal(t, tc.outbv, bv) @@ -480,7 +478,7 @@ func TestNormalizeInvalidDates(t *testing.T) { require.NoError(t, err) known := getBindvars(stmt) bv := make(map[string]*querypb.BindVariable) - _, err = PrepareAST(stmt, NewReservedVars("bv", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + _, err = Normalize(stmt, NewReservedVars("bv", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) require.EqualError(t, err, tc.err.Error()) }) } @@ -502,7 +500,7 @@ func TestNormalizeValidSQL(t *testing.T) { bv := make(map[string]*querypb.BindVariable) known := make(BindVars) - out, err := PrepareAST(tree, NewReservedVars("vtg", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + out, err := Normalize(tree, NewReservedVars("vtg", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) require.NoError(t, err) normalizerOutput := String(out.AST) if normalizerOutput == "otheradmin" || normalizerOutput == "otherread" { @@ -533,7 +531,7 @@ func TestNormalizeOneCasae(t *testing.T) { } bv := make(map[string]*querypb.BindVariable) known := make(BindVars) - out, err := PrepareAST(tree, NewReservedVars("vtg", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + out, err := Normalize(tree, NewReservedVars("vtg", known), bv, true, "ks", 0, "", map[string]string{}, nil, nil) require.NoError(t, err) normalizerOutput := String(out.AST) require.EqualValues(t, testOne.output, normalizerOutput) @@ -882,7 +880,7 @@ func TestRewrites(in *testing.T) { stmt, known, err := parser.Parse2(tc.in) require.NoError(err) vars := NewReservedVars("v", known) - result, err := PrepareAST( + result, err := Normalize( stmt, vars, map[string]*querypb.BindVariable{}, @@ -985,7 +983,7 @@ func TestRewritesWithSetVarComment(in *testing.T) { stmt, err := parser.Parse(tc.in) require.NoError(err) vars := NewReservedVars("v", nil) - result, err := PrepareAST( + result, err := Normalize( stmt, vars, map[string]*querypb.BindVariable{}, @@ -1046,7 +1044,7 @@ func TestRewritesSysVar(in *testing.T) { stmt, err := parser.Parse(tc.in) require.NoError(err) vars := NewReservedVars("v", nil) - result, err := PrepareAST( + result, err := Normalize( stmt, vars, map[string]*querypb.BindVariable{}, @@ -1109,7 +1107,7 @@ func TestRewritesWithDefaultKeyspace(in *testing.T) { stmt, err := parser.Parse(tc.in) require.NoError(err) vars := NewReservedVars("v", nil) - result, err := PrepareAST( + result, err := Normalize( stmt, vars, map[string]*querypb.BindVariable{}, @@ -1157,7 +1155,7 @@ func BenchmarkNormalize(b *testing.B) { b.Fatal(err) } for i := 0; i < b.N; i++ { - _, err := PrepareAST(ast, NewReservedVars("", reservedVars), map[string]*querypb.BindVariable{}, true, "ks", 0, "", map[string]string{}, nil, nil) + _, err := Normalize(ast, NewReservedVars("", reservedVars), map[string]*querypb.BindVariable{}, true, "ks", 0, "", map[string]string{}, nil, nil) require.NoError(b, err) } } @@ -1187,7 +1185,7 @@ func BenchmarkNormalizeTraces(b *testing.B) { for i := 0; i < b.N; i++ { for i, query := range parsed { - _, err := PrepareAST(query, NewReservedVars("", reservedVars[i]), map[string]*querypb.BindVariable{}, true, "ks", 0, "", map[string]string{}, nil, nil) + _, err := Normalize(query, NewReservedVars("", reservedVars[i]), map[string]*querypb.BindVariable{}, true, "ks", 0, "", map[string]string{}, nil, nil) require.NoError(b, err) } } @@ -1222,7 +1220,7 @@ func BenchmarkNormalizeVTGate(b *testing.B) { // Normalize if possible and retry. if CanNormalize(stmt) || MustRewriteAST(stmt, false) { - result, err := PrepareAST( + result, err := Normalize( stmt, NewReservedVars("vtg", reservedVars), bindVars, @@ -1514,7 +1512,7 @@ func benchmarkNormalization(b *testing.B, sqls []string) { } reservedVars := NewReservedVars("vtg", reserved) - _, err = PrepareAST( + _, err = Normalize( stmt, reservedVars, make(map[string]*querypb.BindVariable), diff --git a/go/vt/sqlparser/redact_query.go b/go/vt/sqlparser/redact_query.go index 2d018d7c0eb..d9ff5088dc8 100644 --- a/go/vt/sqlparser/redact_query.go +++ b/go/vt/sqlparser/redact_query.go @@ -28,7 +28,7 @@ func (p *Parser) RedactSQLQuery(sql string) (string, error) { return "", err } - out, err := PrepareAST(stmt, NewReservedVars("redacted", reservedVars), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + out, err := Normalize(stmt, NewReservedVars("redacted", reservedVars), bv, true, "ks", 0, "", map[string]string{}, nil, nil) if err != nil { return "", err } diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index ef2036265e6..259ecd1e760 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -1713,79 +1713,79 @@ var yyAct = [...]int{ 3654, 3338, 3343, 3340, 3339, 1819, 3337, 3342, 3341, 3667, 4028, 3095, 3358, 3357, 3180, 3296, 2535, 2388, 2088, 759, 3246, 5, 3243, 3562, 3069, 3760, 3556, 3094, 3230, 787, - 968, 2415, 2970, 3360, 928, 3051, 2431, 1770, 929, 2896, - 3546, 4099, 753, 2498, 3387, 754, 3001, 2522, 2972, 2971, - 2434, 2503, 3584, 2566, 1173, 165, 1050, 933, 87, 1876, - 42, 1103, 2448, 2921, 2902, 1071, 43, 1047, 2435, 1924, - 2886, 1078, 3240, 2870, 2436, 41, 2342, 2310, 2184, 1050, - 3042, 2146, 2544, 151, 2521, 1906, 2423, 2096, 2505, 2963, - 1131, 2583, 1136, 1808, 1113, 2938, 982, 1788, 2438, 1774, - 1731, 102, 1550, 2190, 2121, 2110, 1475, 2309, 106, 1458, - 107, 2025, 1913, 1110, 1107, 2005, 769, 3245, 1142, 1885, - 2494, 1139, 1111, 2909, 1137, 2495, 1138, 1807, 1088, 1149, - 2416, 764, 1090, 1793, 1533, 1509, 2087, 1282, 3755, 1059, - 1753, 2868, 2217, 1049, 109, 1053, 1070, 2037, 101, 1056, - 3747, 169, 85, 3517, 1055, 129, 127, 128, 134, 1898, - 2198, 135, 93, 1054, 1222, 1045, 1073, 1083, 3573, 1057, - 757, 100, 1554, 84, 4237, 746, 3655, 3326, 1290, 4083, - 1268, 2537, 2538, 2539, 2537, 930, 1082, 108, 3024, 3023, - 1476, 1559, 689, 2992, 2581, 3647, 756, 4185, 1776, 3609, - 3059, 1476, 3060, 1063, 130, 4084, 1044, 4079, 1175, 2383, - 2384, 2103, 1178, 136, 1104, 4078, 2102, 2101, 2100, 3722, - 2099, 1192, 1193, 1194, 2098, 1197, 1198, 1199, 1200, 2068, - 1238, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211, - 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 747, 1098, - 1097, 1064, 686, 4179, 687, 2866, 1061, 2, 3348, 1048, - 1152, 1128, 2635, 3226, 4206, 3184, 1492, 4242, 1153, 2570, - 3520, 1072, 1046, 1990, 4259, 1239, 1779, 4201, 130, 1179, - 1182, 1183, 1127, 1126, 1125, 4057, 1472, 2994, 4227, 1469, - 1186, 1777, 4241, 3722, 3519, 2407, 1772, 2898, 1486, 731, - 1195, 113, 114, 115, 731, 118, 1780, 2412, 124, 1486, - 2411, 193, 725, 2569, 681, 3721, 3348, 4189, 3346, 1120, - 4079, 1778, 981, 1115, 4187, 3017, 744, 745, 192, 3345, - 4031, 984, 985, 986, 2831, 1043, 1038, 1039, 1040, 1041, - 2108, 3406, 4188, 1052, 4153, 3949, 130, 3352, 3948, 4186, - 1177, 131, 3660, 725, 1129, 3661, 1764, 1176, 86, 4217, - 3959, 95, 4183, 984, 985, 986, 3679, 174, 86, 2642, - 3668, 3014, 1460, 1085, 1086, 4127, 3346, 1096, 1100, 932, - 86, 2912, 4135, 2563, 3958, 4139, 2155, 4162, 3432, 3721, - 1887, 1809, 2947, 1810, 1471, 2946, 720, 3031, 2948, 3032, - 3278, 3279, 2455, 2456, 2514, 3352, 1096, 1100, 932, 2913, - 2867, 1482, 4167, 2952, 1474, 1489, 1488, 1490, 1491, 97, - 2080, 2081, 1482, 3277, 3058, 2639, 725, 171, 2508, 2454, - 172, 3774, 4165, 1275, 3041, 1277, 2568, 1258, 1036, 2640, - 1035, 95, 4172, 4173, 705, 1246, 1124, 3678, 1231, 1232, - 1247, 95, 3298, 3299, 4049, 725, 191, 703, 1245, 4166, - 1244, 4143, 3349, 95, 725, 1246, 1263, 1264, 1259, 1287, - 1247, 1252, 2959, 1274, 1276, 3414, 2033, 2473, 2472, 3384, - 1234, 3382, 2905, 2906, 725, 3070, 2385, 725, 3116, 86, - 3412, 1453, 88, 4143, 2633, 2079, 739, 700, 3002, 1470, - 1805, 737, 2083, 3388, 1122, 743, 715, 2419, 1735, 3043, - 4001, 2545, 4002, 1221, 3375, 3027, 1980, 2584, 4207, 4239, - 3349, 710, 3376, 2006, 1510, 1279, 1196, 726, 2609, 1452, - 2610, 2590, 2611, 713, 1284, 1459, 723, 2588, 1267, 4208, - 97, 1261, 1262, 1227, 724, 3046, 3403, 3649, 1511, 1512, - 1513, 1514, 1515, 1516, 1517, 1519, 1518, 1520, 1521, 2586, - 1981, 3297, 1982, 3933, 3648, 2612, 1151, 1260, 726, 3072, - 1253, 175, 95, 3300, 1151, 1265, 3385, 1202, 3383, 2587, - 181, 2507, 1201, 1286, 1272, 1266, 2591, 2548, 1273, 1285, - 3645, 2636, 2589, 2637, 3726, 1132, 1162, 2432, 1278, 1133, - 1133, 1171, 1170, 1169, 690, 1168, 692, 706, 3559, 728, - 1167, 727, 696, 2995, 694, 698, 707, 699, 2034, 693, - 1166, 704, 1165, 1271, 695, 708, 709, 712, 716, 717, - 718, 714, 711, 1164, 702, 729, 1159, 1123, 1089, 1738, - 3300, 726, 1891, 3082, 3081, 3080, 1172, 1524, 3074, 4260, - 3078, 1108, 3073, 1524, 3071, 1181, 1145, 3117, 1108, 3076, - 4180, 1144, 1106, 4214, 1144, 1180, 1108, 2026, 3075, 1150, - 726, 1886, 3320, 1084, 3047, 2574, 2573, 1150, 2022, 726, - 3183, 1461, 1189, 3026, 1884, 1883, 1882, 3077, 3079, 3012, - 1291, 2023, 1291, 1291, 1880, 1237, 680, 3040, 3192, 726, - 3039, 4181, 726, 2417, 2418, 3644, 1099, 1093, 1091, 166, - 3211, 1481, 1478, 1479, 1480, 1485, 1487, 1484, 3209, 1483, - 1525, 1526, 1481, 1478, 1479, 1480, 1485, 1487, 1484, 1477, - 1483, 4044, 1806, 3063, 3598, 1099, 1093, 1091, 3580, 2657, - 1477, 2943, 1050, 1534, 1539, 1540, 1163, 1543, 1545, 1546, - 1547, 1548, 1549, 3029, 1552, 1553, 1555, 1555, 2908, 1555, + 928, 2415, 2970, 3360, 968, 3051, 2431, 1770, 929, 2896, + 3546, 4099, 753, 2498, 3387, 754, 3001, 2434, 2972, 1103, + 2522, 2503, 3584, 2971, 1173, 2566, 1050, 933, 87, 1876, + 42, 2448, 165, 2921, 2902, 1071, 43, 1047, 1078, 2870, + 1924, 2436, 41, 3240, 2886, 2310, 2342, 2096, 2184, 1050, + 3042, 2544, 151, 2435, 2521, 2423, 1906, 2146, 2505, 2963, + 1136, 2583, 1808, 2938, 1113, 1131, 1788, 1731, 2438, 982, + 1774, 102, 1550, 2190, 2121, 2110, 1475, 2309, 106, 1458, + 107, 2025, 1913, 1110, 1107, 1885, 1142, 3245, 769, 2005, + 2494, 1137, 1111, 2909, 1139, 2495, 1138, 1807, 1088, 1793, + 2416, 764, 1090, 1059, 1533, 1149, 1509, 2087, 3755, 2217, + 1753, 2868, 1282, 1049, 109, 1053, 1070, 2037, 101, 1056, + 3747, 169, 85, 129, 1055, 3517, 1898, 134, 127, 128, + 1054, 2198, 1222, 135, 93, 1045, 1073, 1083, 3573, 1057, + 757, 100, 84, 1554, 4237, 108, 2537, 2538, 2539, 4083, + 746, 1290, 3655, 1082, 3326, 930, 2537, 3024, 3023, 2581, + 1559, 1268, 689, 2992, 3647, 1044, 756, 4185, 1476, 1776, + 3059, 3060, 4078, 1063, 130, 4084, 4079, 1476, 1175, 1990, + 136, 2103, 1104, 2102, 3609, 2383, 2384, 2101, 2100, 1178, + 3722, 1192, 1193, 1194, 2099, 1197, 1198, 1199, 1200, 2098, + 2068, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211, + 1212, 1213, 1214, 1215, 1216, 1217, 1218, 1219, 1238, 1098, + 1097, 1064, 686, 747, 687, 2866, 1061, 2, 1048, 2635, + 1046, 1128, 3226, 1152, 2898, 1492, 2570, 1779, 4206, 1777, + 3722, 4179, 4259, 1072, 1153, 1239, 731, 4201, 130, 1772, + 1127, 3184, 1179, 1182, 1183, 1126, 1125, 4057, 3014, 2994, + 1115, 4227, 3520, 4242, 3348, 2407, 1186, 1780, 4189, 1778, + 1195, 113, 114, 115, 4187, 118, 1486, 4079, 124, 1120, + 2569, 193, 2412, 3519, 681, 1486, 3721, 3348, 4241, 2411, + 3017, 725, 981, 4188, 1043, 1472, 744, 745, 1469, 4186, + 3345, 4031, 2831, 1764, 2108, 3406, 1038, 1039, 1040, 1041, + 4153, 3949, 3948, 1052, 3660, 1177, 130, 3661, 4183, 731, + 1176, 1032, 4217, 3959, 3346, 3679, 970, 1033, 984, 985, + 986, 971, 95, 1129, 972, 973, 3721, 974, 1460, 1096, + 1100, 932, 3668, 1085, 1086, 4127, 4135, 3346, 1096, 1100, + 932, 2563, 3958, 3352, 2155, 987, 988, 4162, 3432, 1887, + 2947, 3278, 3279, 2946, 2455, 2456, 2948, 3031, 725, 3032, + 2080, 2081, 1809, 3277, 1810, 3058, 3352, 3678, 2867, 2639, + 2454, 3041, 4139, 1287, 1489, 1258, 1490, 1491, 725, 1482, + 1036, 1035, 1474, 2642, 1246, 2568, 1246, 725, 1482, 1247, + 4049, 1247, 2995, 1471, 1488, 2033, 2959, 1245, 725, 1244, + 86, 720, 2473, 2472, 2905, 2906, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, + 1013, 1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, + 1023, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 4143, 705, + 725, 97, 2385, 2640, 4167, 1263, 1264, 86, 3349, 3384, + 3382, 725, 703, 1221, 2514, 3414, 1259, 1252, 3412, 1453, + 2633, 2079, 2083, 3116, 4165, 1124, 86, 1231, 1232, 739, + 2912, 3349, 4207, 95, 4172, 4173, 743, 1286, 2508, 3369, + 1510, 1459, 1196, 1285, 3298, 3299, 726, 3388, 1470, 3043, + 737, 4166, 700, 4208, 2419, 1735, 3002, 86, 2913, 1234, + 88, 715, 3403, 2545, 1511, 1512, 1513, 1514, 1515, 1516, + 1517, 1519, 1518, 1520, 1521, 3027, 710, 3774, 3375, 4001, + 2588, 4002, 2590, 1122, 1452, 4239, 3376, 2034, 713, 2584, + 95, 723, 1980, 2609, 2006, 2610, 1279, 2611, 2636, 724, + 2637, 1261, 1262, 1284, 1267, 1227, 3385, 3383, 97, 95, + 3046, 1089, 3649, 3648, 1265, 1260, 1253, 2612, 1202, 2586, + 3370, 3371, 2587, 726, 1266, 2597, 2593, 2595, 2596, 2594, + 2598, 2599, 2600, 2601, 1201, 2589, 1981, 2591, 1982, 1738, + 95, 3559, 3933, 726, 1132, 2548, 3726, 2432, 1133, 1133, + 1171, 3300, 726, 3297, 1170, 1169, 1168, 1167, 1166, 690, + 1165, 692, 706, 726, 728, 3300, 727, 696, 1164, 694, + 698, 707, 699, 1159, 693, 1891, 704, 1172, 4260, 695, + 708, 709, 712, 716, 717, 718, 714, 711, 4180, 702, + 729, 2507, 3117, 1108, 4214, 1108, 1108, 1144, 1145, 1106, + 1291, 1181, 1291, 1291, 2026, 1084, 3183, 1144, 1099, 1093, + 1091, 1180, 1886, 3047, 2574, 726, 1123, 1099, 1093, 1091, + 1805, 2573, 2022, 1461, 1189, 3320, 726, 3026, 1884, 1481, + 1478, 1479, 1480, 1485, 1487, 1484, 1883, 1483, 1481, 1478, + 1479, 1480, 1485, 1487, 1484, 1882, 1483, 1477, 2996, 3012, + 2417, 2418, 1050, 1534, 1539, 1540, 1477, 1543, 1545, 1546, + 1547, 1548, 1549, 3029, 1552, 1553, 1555, 1555, 2023, 1555, 1555, 1560, 1560, 1560, 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, @@ -1798,294 +1798,294 @@ var yyAct = [...]int{ 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, 1531, 4056, - 1280, 2993, 1687, 1130, 1689, 1690, 1691, 1692, 1693, 3720, - 1450, 1451, 3607, 3608, 3610, 1776, 1560, 1560, 1560, 1560, - 1560, 1560, 1449, 4141, 730, 2511, 2567, 4102, 3560, 2996, - 2961, 1700, 1701, 1702, 1703, 1704, 1705, 1706, 1707, 1708, - 1709, 1710, 1711, 1712, 1713, 721, 1527, 1528, 1529, 1530, - 2641, 3350, 3351, 94, 1225, 4141, 1541, 4171, 167, 4140, - 722, 725, 3504, 94, 3354, 179, 2512, 1468, 1535, 1556, - 3677, 1557, 1558, 2510, 89, 94, 1242, 3404, 1248, 1249, - 1250, 1251, 1160, 3720, 1544, 2640, 1992, 1991, 1993, 1994, - 1995, 4140, 3016, 1561, 1562, 1151, 4169, 1092, 1912, 1243, - 1151, 4170, 1288, 1289, 2419, 95, 187, 2513, 2565, 3350, - 3351, 1233, 1728, 2843, 1763, 1230, 2158, 2509, 1734, 1797, - 922, 4253, 3354, 2871, 2873, 1688, 1092, 1050, 3050, 1236, - 3191, 1050, 2903, 126, 688, 3276, 3015, 1050, 2461, 1521, - 2668, 1524, 1764, 1119, 1504, 1067, 1121, 1283, 1724, 1256, - 2010, 4070, 1188, 168, 173, 170, 176, 177, 178, 180, - 182, 183, 184, 185, 1174, 2226, 1725, 1269, 3640, 186, - 188, 189, 190, 2011, 121, 3090, 195, 2038, 3572, 684, - 2126, 2585, 2092, 1241, 94, 2597, 2593, 2595, 2596, 2594, - 2598, 2599, 2600, 2601, 2127, 1522, 1523, 2125, 1150, 684, - 2019, 1811, 2191, 1150, 1144, 1147, 1148, 1911, 1108, 1151, - 1490, 1491, 1141, 1145, 1742, 1060, 1124, 1220, 1746, 2527, - 2191, 3769, 2677, 4218, 1049, 1744, 2668, 1745, 1491, 106, - 2562, 107, 1161, 1140, 1081, 1081, 1725, 1694, 1695, 1696, - 1697, 1698, 1699, 684, 1732, 1124, 2199, 1116, 3615, 3614, - 2552, 1921, 122, 1920, 1118, 1117, 3053, 1910, 3284, 2560, - 1162, 3052, 1160, 2200, 2218, 109, 4261, 4103, 1775, 2220, - 4209, 3599, 4036, 2225, 2221, 1151, 726, 2222, 2223, 2224, - 2557, 1224, 2219, 2227, 2228, 2229, 2230, 2231, 2232, 2233, - 2234, 2235, 1062, 1888, 1889, 1890, 4255, 2872, 4062, 3053, - 2417, 2418, 1740, 1122, 3052, 2007, 3674, 2008, 3675, 3285, - 2009, 1904, 1150, 1766, 1151, 2564, 4104, 1154, 1144, 1741, - 1729, 4037, 1156, 2561, 1743, 1270, 1157, 1155, 1226, 1897, - 2557, 2031, 1151, 1975, 3287, 2039, 2185, 1240, 1769, 1926, - 1048, 1927, 4211, 1929, 1931, 1916, 1046, 1935, 1937, 1939, - 1941, 1943, 95, 4251, 3282, 1957, 4252, 2182, 4250, 1291, - 1965, 1966, 4262, 1802, 1803, 1255, 1971, 1972, 2000, 1764, - 1871, 1915, 2124, 2559, 3298, 3299, 1257, 2196, 1150, 3941, - 1187, 3283, 1492, 3940, 1184, 3931, 3691, 1914, 1914, 1879, - 3690, 1223, 3622, 2015, 3621, 2013, 2014, 2012, 2016, 2017, - 2018, 3611, 2197, 1895, 1894, 1907, 1893, 1123, 2424, 2425, - 1998, 749, 3327, 1492, 3316, 3289, 2968, 1150, 1747, 984, - 985, 986, 1154, 1144, 2967, 2347, 1987, 1156, 2347, 2344, - 1961, 1157, 1155, 2966, 2517, 1150, 1123, 2704, 2346, 1999, - 2001, 1144, 1147, 1148, 1985, 1108, 1984, 1983, 1918, 1141, - 1145, 1973, 1158, 1967, 979, 2027, 2174, 2163, 2164, 2165, + 1679, 1680, 1681, 1682, 1683, 1684, 1685, 1686, 1776, 1449, + 1531, 4056, 1687, 2993, 1689, 1690, 1691, 1692, 1693, 3016, + 3720, 1280, 1130, 1450, 1451, 2567, 1560, 1560, 1560, 1560, + 1560, 1560, 1992, 1991, 1993, 1994, 1995, 3607, 3608, 3610, + 3677, 1700, 1701, 1702, 1703, 1704, 1705, 1706, 1707, 1708, + 1709, 1710, 1711, 1712, 1713, 4102, 1527, 1528, 1529, 1530, + 4141, 3560, 1806, 3015, 1151, 1275, 1541, 1277, 1524, 3645, + 3720, 3504, 1162, 1160, 3284, 1151, 1243, 2961, 1535, 730, + 1556, 1880, 1557, 1558, 1237, 1524, 1242, 95, 1248, 1249, + 1250, 1251, 1230, 3404, 1544, 4253, 4140, 3350, 3351, 1092, + 721, 1119, 1561, 1562, 1121, 1274, 1276, 1151, 1092, 1225, + 3354, 1233, 1288, 1289, 2641, 722, 1468, 1912, 2871, 2873, + 3350, 3351, 1728, 680, 1763, 3285, 2640, 4181, 1734, 3040, + 922, 4044, 3039, 3354, 3598, 2511, 3580, 1050, 2010, 2943, + 1188, 1050, 89, 3050, 2565, 94, 2908, 1050, 1256, 4171, + 3287, 2843, 1525, 1526, 3063, 2158, 1797, 1688, 1724, 1236, + 3192, 3191, 126, 2903, 688, 2226, 2461, 1150, 1524, 1521, + 3282, 2657, 3276, 1725, 1764, 725, 2512, 2199, 1150, 2668, + 1504, 1067, 1269, 2510, 3644, 1283, 195, 121, 4169, 684, + 3298, 3299, 4070, 4170, 2200, 2527, 1174, 3283, 3640, 3572, + 2011, 2585, 94, 1124, 2092, 1116, 3090, 1151, 3211, 684, + 1150, 1241, 1118, 1117, 2019, 1811, 1272, 2513, 2419, 3209, + 1273, 94, 1163, 1161, 1742, 1060, 1911, 2509, 1746, 2038, + 1278, 3289, 2191, 3070, 1049, 1744, 2191, 1745, 2677, 106, + 4218, 107, 1491, 1725, 1081, 1081, 1694, 1695, 1696, 1697, + 1698, 1699, 94, 684, 1732, 1271, 3769, 1492, 2668, 1490, + 1491, 1122, 3615, 1151, 2218, 122, 3614, 2552, 1921, 2220, + 1920, 1124, 1220, 2225, 2221, 109, 1910, 2222, 2223, 2224, + 4062, 1775, 2219, 2227, 2228, 2229, 2230, 2231, 2232, 2233, + 2234, 2235, 2872, 2007, 2557, 2008, 2562, 3053, 2009, 3297, + 2560, 1162, 3052, 1888, 1889, 1890, 1160, 4209, 3599, 1904, + 1150, 3300, 1740, 1062, 3053, 1154, 1144, 3072, 2000, 3052, + 1156, 4255, 4211, 1766, 1157, 1155, 3674, 4251, 3675, 4103, + 4252, 1741, 4250, 2197, 1743, 1729, 1224, 2561, 1897, 1151, + 1270, 2031, 2557, 1975, 1046, 1158, 2185, 3941, 1769, 1048, + 1926, 2564, 1927, 1957, 1929, 1931, 1916, 1226, 1935, 1937, + 1939, 1941, 1943, 95, 1255, 1240, 1150, 2182, 1187, 1291, + 1965, 1966, 1184, 1802, 1803, 1257, 1971, 1972, 4104, 3940, + 4036, 1915, 1871, 2124, 1998, 2559, 1151, 2039, 3931, 1999, + 726, 3082, 3081, 3080, 1123, 3691, 3074, 1879, 3078, 1151, + 3073, 3690, 3071, 1914, 1914, 3622, 1489, 3076, 1490, 1491, + 3621, 1894, 1895, 4261, 2417, 2418, 3075, 1907, 1893, 3611, + 2015, 749, 2013, 2014, 2012, 2016, 2017, 2018, 1747, 4037, + 2347, 3327, 3286, 1764, 2344, 3077, 3079, 984, 985, 986, + 1961, 2196, 1150, 2346, 1987, 3316, 1223, 2968, 1144, 1147, + 1148, 2967, 1108, 2966, 2517, 1997, 1141, 1145, 1918, 979, + 2001, 1985, 1123, 1984, 1983, 2027, 2174, 2163, 2164, 2165, 2166, 2176, 2167, 2168, 2169, 2181, 2177, 2170, 2171, 2178, - 2179, 2180, 2172, 2173, 2175, 1953, 1492, 1510, 1956, 3062, - 1958, 1997, 1964, 3297, 2114, 2115, 2647, 2648, 130, 1963, - 1962, 1510, 1127, 1126, 1125, 3300, 1933, 1986, 1739, 3379, - 1455, 1511, 1512, 1513, 1514, 1515, 1516, 1517, 1519, 1518, - 1520, 1521, 2044, 2718, 731, 1511, 1512, 1513, 1514, 1515, - 1516, 1517, 1519, 1518, 1520, 1521, 3604, 2066, 731, 1291, - 1291, 1489, 1805, 1490, 1491, 2040, 2041, 1514, 1515, 1516, - 1517, 1519, 1518, 1520, 1521, 87, 2892, 4238, 87, 2045, - 1516, 1517, 1519, 1518, 1520, 1521, 2052, 2053, 2054, 4210, - 1492, 192, 1489, 4065, 1490, 1491, 1492, 2065, 1498, 1499, - 1500, 1501, 1502, 1503, 1497, 1494, 2950, 1510, 731, 2533, - 1506, 2532, 1507, 4064, 131, 1511, 1512, 1513, 1514, 1515, - 1516, 1517, 1519, 1518, 1520, 1521, 1508, 1522, 1523, 1505, + 2179, 2180, 2172, 2173, 2175, 1953, 2347, 1973, 1956, 1150, + 1958, 2114, 2115, 2112, 2113, 1144, 1147, 1148, 130, 1108, + 1127, 1510, 1150, 1141, 1145, 1126, 1125, 1154, 1144, 4262, + 1967, 1964, 1156, 1963, 1962, 1986, 1157, 1155, 2111, 1510, + 1933, 3062, 2044, 2718, 1140, 1511, 1512, 1513, 1514, 1515, + 1516, 1517, 1519, 1518, 1520, 1521, 1739, 1492, 3379, 1291, + 1291, 1455, 2066, 1511, 1512, 1513, 1514, 1515, 1516, 1517, + 1519, 1518, 1520, 1521, 731, 87, 2040, 2041, 87, 1511, + 1512, 1513, 1514, 1515, 1516, 1517, 1519, 1518, 1520, 1521, + 2045, 192, 2114, 2115, 2647, 2648, 1805, 2052, 2053, 2054, + 2065, 1516, 1517, 1519, 1518, 1520, 1521, 1510, 4210, 3604, + 1506, 731, 1507, 4065, 131, 1512, 1513, 1514, 1515, 1516, + 1517, 1519, 1518, 1520, 1521, 4064, 1508, 1522, 1523, 1505, 174, 1511, 1512, 1513, 1514, 1515, 1516, 1517, 1519, 1518, - 1520, 1521, 1492, 2531, 4040, 2530, 3286, 4039, 2153, 2153, - 2154, 4038, 2151, 2151, 1492, 1489, 1720, 1490, 1491, 1492, - 3936, 1718, 42, 1492, 3092, 42, 1716, 3920, 2042, 1717, - 1715, 1492, 1719, 3919, 1510, 2046, 2656, 2048, 2049, 2050, - 2051, 2116, 4223, 1764, 2055, 2529, 3768, 2528, 4221, 1764, - 171, 1782, 3766, 172, 3687, 2237, 2067, 1510, 1511, 1512, - 1513, 1514, 1515, 1516, 1517, 1519, 1518, 1520, 1521, 1512, - 1513, 1514, 1515, 1516, 1517, 1519, 1518, 1520, 1521, 191, - 1723, 1511, 1512, 1513, 1514, 1515, 1516, 1517, 1519, 1518, - 1520, 1521, 1492, 1722, 4149, 1764, 2666, 4197, 1764, 1489, - 1783, 1490, 1491, 1724, 2093, 1489, 2665, 1490, 1491, 1492, - 1721, 4147, 1764, 1764, 85, 4145, 1764, 85, 2892, 1764, - 4058, 1725, 2123, 4014, 1764, 2716, 2186, 2073, 2074, 1492, - 2114, 2115, 2112, 2113, 103, 1488, 1764, 1492, 1488, 1764, - 105, 3619, 2128, 3603, 104, 1492, 2892, 4134, 3968, 112, - 1492, 1489, 3389, 1490, 1491, 2892, 4113, 2111, 1492, 2673, - 111, 3386, 110, 1489, 3319, 1490, 1491, 3318, 1489, 2157, - 1490, 1491, 1489, 2357, 1490, 1491, 112, 3044, 2356, 2130, - 1489, 2261, 1490, 1491, 4012, 1764, 2355, 111, 2977, 110, - 2964, 684, 1492, 684, 175, 1720, 1492, 105, 1552, 2892, - 4109, 4009, 1764, 181, 2129, 1764, 2131, 2132, 2133, 2134, + 1520, 1521, 2950, 1782, 731, 2533, 4040, 2532, 2153, 2153, + 2154, 1764, 2151, 2151, 1514, 1515, 1516, 1517, 1519, 1518, + 1520, 1521, 42, 1492, 2531, 42, 2530, 2529, 2042, 2528, + 4039, 2126, 2716, 4038, 192, 2046, 2952, 2048, 2049, 2050, + 2051, 2116, 1488, 1764, 2055, 2127, 1522, 1523, 2125, 112, + 171, 3936, 1783, 172, 3290, 2237, 2067, 131, 3294, 1510, + 111, 2656, 110, 2892, 4238, 3293, 1489, 3920, 1490, 1491, + 105, 103, 3919, 174, 3768, 3092, 3766, 105, 3687, 191, + 1723, 104, 1722, 1511, 1512, 1513, 1514, 1515, 1516, 1517, + 1519, 1518, 1520, 1521, 4197, 1764, 103, 1492, 1721, 3295, + 2892, 1764, 4058, 1724, 3291, 2093, 104, 1492, 3619, 3292, + 1488, 1764, 3968, 1492, 85, 4223, 1764, 85, 1725, 3603, + 1510, 2892, 4134, 3967, 2123, 3389, 2186, 2892, 4113, 3924, + 2073, 2074, 3386, 171, 3319, 1764, 172, 2892, 4109, 2424, + 2425, 3318, 1764, 2128, 1511, 1512, 1513, 1514, 1515, 1516, + 1517, 1519, 1518, 1520, 1521, 4021, 1764, 3923, 1492, 3658, + 4055, 3666, 191, 3944, 1764, 2892, 3932, 1764, 1492, 2157, + 3658, 1764, 1489, 2357, 1490, 1491, 2892, 3656, 2356, 2130, + 3044, 2261, 1492, 2557, 1764, 2939, 2355, 1492, 2977, 4221, + 1764, 684, 1492, 684, 175, 2964, 1492, 1720, 1552, 4149, + 1764, 1492, 1714, 181, 2129, 1492, 2131, 2132, 2133, 2134, 2135, 2136, 2138, 2140, 2141, 2142, 2143, 2144, 2145, 2343, - 1535, 3991, 1764, 2345, 2236, 2201, 2202, 2203, 2204, 3545, - 1764, 4021, 1764, 3658, 4055, 1764, 3967, 2672, 2192, 2215, - 1714, 1489, 3963, 1490, 1491, 1492, 103, 2410, 3290, 2630, - 3538, 1764, 3294, 3944, 1764, 684, 104, 2622, 1489, 3293, - 1490, 1491, 1764, 3575, 1492, 2621, 2354, 2892, 3932, 2360, - 2361, 3658, 1764, 2440, 1492, 2892, 3656, 105, 1489, 2579, - 1490, 1491, 2357, 1538, 3535, 1764, 1489, 2429, 1490, 1491, - 106, 2253, 107, 3295, 1489, 2355, 1490, 1491, 3291, 1489, - 2578, 1490, 1491, 3292, 2557, 1764, 1492, 1489, 2470, 1490, - 1491, 2414, 106, 2390, 107, 1492, 2391, 1764, 3924, 1492, - 2069, 2402, 3578, 1764, 2798, 1764, 3923, 1492, 2122, 3309, - 3308, 3574, 166, 3306, 3307, 3666, 1786, 3533, 1764, 3304, - 3305, 1489, 1764, 1490, 1491, 1489, 1113, 1490, 1491, 3304, - 3303, 2918, 1764, 1492, 2640, 3025, 3496, 1764, 1492, 2035, - 2442, 2939, 2479, 2480, 2481, 2482, 3494, 1764, 1063, 1875, - 3006, 3003, 2464, 1996, 2378, 2999, 3000, 2982, 1492, 2465, - 1113, 2939, 1492, 1988, 2474, 1978, 2475, 2476, 2477, 2478, - 2396, 2446, 2397, 2403, 1489, 1974, 1490, 1491, 3490, 1764, - 2484, 1492, 2664, 2486, 2487, 2488, 2489, 1785, 2405, 111, - 2468, 3487, 1764, 1489, 2500, 1490, 1491, 1970, 2546, 3485, - 1764, 1492, 2506, 1489, 1969, 1490, 1491, 1968, 2427, 2940, - 2892, 2891, 2469, 2451, 2452, 1098, 1097, 2450, 1784, 2942, - 2156, 1764, 2467, 1281, 2466, 3483, 1764, 2558, 1492, 2940, - 3481, 1764, 1492, 1875, 1874, 1489, 2917, 1490, 1491, 2640, - 1817, 1816, 2714, 2516, 1489, 2543, 1490, 1491, 1489, 1492, - 1490, 1491, 2910, 1492, 3479, 1764, 1489, 3271, 1490, 1491, - 2910, 3571, 1492, 1488, 1051, 1488, 2501, 2640, 1492, 2497, - 2490, 2492, 2493, 3477, 1764, 2519, 2551, 3241, 2515, 2554, - 2520, 2555, 1489, 1492, 1490, 1491, 2889, 1489, 3571, 1490, - 1491, 2557, 4097, 3475, 1764, 4069, 2892, 2918, 2571, 2918, - 105, 3524, 2501, 2550, 1152, 2553, 2549, 1489, 3623, 1490, - 1491, 1489, 1153, 1490, 1491, 97, 3306, 3214, 2453, 2798, - 2572, 1764, 1914, 2575, 3473, 1764, 684, 2576, 2577, 2918, - 1489, 2701, 1490, 1491, 1492, 2700, 2557, 3571, 2540, 1765, - 1767, 3471, 1764, 2422, 2409, 3469, 1764, 95, 95, 1060, - 1489, 167, 1490, 1491, 3467, 1764, 1768, 2381, 179, 2645, - 3465, 1764, 2156, 2094, 2887, 3624, 3625, 3626, 1050, 1050, - 1050, 2582, 2078, 2021, 684, 3463, 1764, 1489, 1804, 1490, - 1491, 1489, 1949, 1490, 1491, 1492, 1135, 1134, 1545, 4176, - 1545, 1492, 4116, 684, 3955, 1771, 1492, 3921, 1489, 187, - 1490, 1491, 1489, 3781, 1490, 1491, 2660, 1492, 3639, 3636, - 3617, 1489, 3437, 1490, 1491, 2615, 3436, 1489, 1877, 1490, - 1491, 2499, 3377, 2357, 3332, 1492, 3461, 1764, 2356, 3328, - 3007, 1492, 1489, 2496, 1490, 1491, 2663, 1492, 2491, 1950, - 1951, 1952, 2485, 1538, 2483, 2003, 168, 173, 170, 176, - 177, 178, 180, 182, 183, 184, 185, 1909, 1905, 3421, - 1492, 1873, 186, 188, 189, 190, 123, 2973, 2632, 3330, - 1538, 3381, 2974, 1538, 1225, 1538, 684, 3459, 1764, 1492, - 3956, 2638, 2514, 1489, 1764, 1490, 1491, 1726, 3457, 1764, - 3585, 3586, 1492, 2394, 4233, 4231, 1976, 2646, 1492, 3443, - 1764, 4204, 1492, 4077, 3996, 3588, 4073, 3324, 2071, 3323, - 2652, 684, 3591, 3322, 2649, 2650, 2651, 3419, 1764, 3241, - 1492, 2974, 2123, 2863, 1764, 1492, 2986, 2030, 684, 2616, - 1764, 3957, 3590, 3263, 1489, 3260, 1490, 1491, 3264, 3259, - 1489, 1492, 1490, 1491, 684, 1489, 685, 1490, 1491, 3261, - 2413, 684, 2861, 1764, 3262, 3579, 1489, 1065, 1490, 1491, - 2056, 2057, 684, 684, 684, 684, 684, 684, 684, 1492, - 3627, 2836, 1764, 2676, 1489, 2072, 1490, 1491, 2654, 1781, - 1489, 3219, 1490, 1491, 2813, 1764, 1489, 1492, 1490, 1491, - 2805, 1764, 1492, 2400, 2796, 1764, 2624, 2625, 3231, 3233, - 4035, 2627, 3218, 2653, 1492, 2655, 2842, 3234, 1066, 1489, - 2628, 1490, 1491, 4212, 2658, 1492, 2659, 2794, 1764, 3564, - 748, 1492, 2661, 3628, 3629, 3630, 3759, 3563, 1489, 2712, - 1490, 1491, 2830, 2781, 1764, 3761, 3567, 3265, 2874, 2927, - 2928, 1489, 1492, 1490, 1491, 3228, 2020, 1489, 1945, 1490, - 1491, 1489, 1068, 1490, 1491, 2153, 2877, 1050, 1034, 2151, - 1069, 2779, 1764, 3302, 2957, 1492, 2978, 2608, 2607, 1489, - 2606, 1490, 1491, 2605, 1489, 1077, 1490, 1491, 2604, 2777, - 1764, 2199, 2915, 2916, 2775, 1764, 2603, 2875, 2602, 1076, - 1489, 2440, 1490, 1491, 1050, 2935, 2773, 1764, 2200, 3397, - 1492, 1946, 1947, 1948, 3746, 1492, 3745, 2771, 1764, 1492, - 2119, 2117, 2118, 2769, 1764, 2878, 1191, 2880, 1489, 1492, - 1490, 1491, 1454, 1492, 1190, 2973, 2895, 1492, 2122, 3056, - 2914, 3013, 103, 131, 2767, 1764, 1489, 3569, 1490, 1491, - 105, 1489, 104, 1490, 1491, 2923, 2926, 2927, 2928, 2924, - 4247, 2925, 2929, 1489, 1492, 1490, 1491, 2765, 1764, 3744, - 1732, 2990, 2865, 2904, 1489, 684, 1490, 1491, 2619, 42, - 1489, 2933, 1490, 1491, 1492, 2424, 2425, 4152, 2932, 4054, - 3951, 2934, 3301, 2885, 1492, 2931, 1775, 2960, 2962, 2893, - 1492, 1489, 3593, 1490, 1491, 1492, 1725, 2763, 1764, 2953, - 2907, 2761, 1764, 2890, 1492, 2406, 110, 3547, 1492, 3011, - 2644, 2759, 1764, 2937, 1489, 3540, 1490, 1491, 103, 2757, - 1764, 2077, 1492, 2941, 105, 1538, 3217, 2944, 104, 1492, - 4020, 2076, 2506, 2951, 3216, 1492, 4019, 3999, 2954, 1492, - 2194, 111, 2976, 1538, 3022, 2195, 3536, 2979, 2980, 1489, - 3767, 1490, 1491, 3765, 1489, 2965, 1490, 1491, 1489, 3764, - 1490, 1491, 3757, 3637, 3568, 3566, 2755, 1764, 1489, 1492, - 1490, 1491, 1489, 2975, 1490, 1491, 1489, 4053, 1490, 1491, - 1492, 3333, 3502, 2257, 1492, 2983, 2541, 3498, 2987, 2988, - 2989, 2984, 1492, 1892, 1075, 3756, 2753, 1764, 112, 3019, - 2751, 1764, 3557, 1489, 1897, 1490, 1491, 4234, 2910, 111, - 3730, 3066, 3067, 2889, 2749, 1764, 1492, 3120, 3008, 3009, - 1492, 2747, 1764, 1489, 1492, 1490, 1491, 2742, 1764, 4235, - 4234, 2738, 1764, 1489, 3018, 1490, 1491, 112, 3, 1489, - 2702, 1490, 1491, 2392, 1489, 1798, 1490, 1491, 111, 1790, - 110, 116, 117, 1489, 1492, 1490, 1491, 1489, 1492, 1490, - 1491, 2736, 1764, 2340, 1492, 4235, 99, 3045, 4041, 3602, - 3083, 1489, 3434, 1490, 1491, 1, 2729, 1764, 1489, 2030, - 1490, 1491, 3064, 3048, 1489, 3928, 1490, 1491, 1489, 2091, - 1490, 1491, 10, 2372, 1042, 1457, 3101, 3102, 3103, 3104, - 3105, 3106, 3107, 3108, 3109, 3110, 1456, 3606, 2727, 1764, - 1765, 2379, 2089, 3641, 4164, 9, 3118, 2969, 1489, 2090, - 1490, 1491, 8, 701, 1976, 2382, 1492, 1730, 3084, 1489, - 4205, 1490, 1491, 1489, 112, 1490, 1491, 4160, 4161, 3020, - 1081, 1489, 1989, 1490, 1491, 111, 3433, 110, 1979, 3669, - 3425, 2404, 1726, 2308, 3952, 105, 3423, 3336, 1492, 1060, - 2547, 3635, 2504, 1143, 156, 1489, 1492, 1490, 1491, 1489, - 1492, 1490, 1491, 1489, 2462, 1490, 1491, 2463, 3122, 4129, - 120, 684, 3178, 1101, 3054, 119, 1146, 3055, 2030, 684, - 1254, 684, 2542, 684, 2449, 3659, 2958, 2471, 1492, 2251, - 1823, 1821, 1822, 1489, 1820, 1490, 1491, 1489, 3065, 1490, - 1491, 1825, 1824, 1489, 4101, 1490, 1491, 2998, 2859, 3405, - 2703, 3196, 3503, 2082, 738, 3185, 2930, 732, 194, 1812, - 3187, 1791, 2923, 2926, 2927, 2928, 2924, 3068, 2925, 2929, - 2440, 2075, 3585, 3586, 3111, 3085, 1492, 1185, 691, 3310, - 2858, 2580, 697, 2343, 1542, 2343, 3158, 2345, 2854, 2345, - 2070, 3215, 2853, 2518, 3248, 2945, 87, 1095, 1087, 2440, - 2440, 2440, 2440, 2440, 2393, 1489, 2526, 1490, 1491, 2334, - 2335, 2336, 2337, 2338, 2879, 1094, 3196, 3929, 3249, 2440, - 2852, 3186, 2440, 3188, 3561, 1492, 2359, 3253, 3227, 2362, + 1535, 2236, 2630, 2345, 3578, 1764, 2201, 2202, 2203, 2204, + 1498, 1499, 1500, 1501, 1502, 1503, 1497, 1494, 2192, 2622, + 2215, 112, 2798, 1764, 3309, 3308, 1489, 2410, 1490, 1491, + 4147, 1764, 111, 2940, 110, 684, 1489, 2621, 1490, 1491, + 4145, 1764, 1489, 2942, 1490, 1491, 2354, 3306, 3307, 2360, + 2361, 3304, 3305, 2440, 4014, 1764, 3003, 175, 1492, 4012, + 1764, 2939, 2357, 1538, 4009, 1764, 181, 2429, 3991, 1764, + 106, 2253, 107, 3545, 1764, 2355, 2579, 3538, 1764, 3304, + 3303, 2918, 1764, 1492, 2640, 3025, 2982, 1489, 2470, 1490, + 1491, 2578, 106, 2390, 107, 2414, 1492, 1489, 2402, 1490, + 1491, 1875, 3006, 2889, 1492, 2999, 3000, 1764, 2122, 1492, + 2714, 1489, 166, 1490, 1491, 2391, 1489, 105, 1490, 1491, + 2069, 1489, 105, 1490, 1491, 1489, 1113, 1490, 1491, 2940, + 1489, 2035, 1490, 1491, 1489, 1996, 1490, 1491, 1492, 2640, + 2442, 2910, 2479, 2480, 2481, 2482, 1492, 2465, 1063, 1988, + 3535, 1764, 2892, 2891, 2378, 2156, 1764, 1492, 1978, 2464, + 1113, 2446, 1974, 1970, 2474, 1492, 2475, 2476, 2477, 2478, + 2396, 1492, 2397, 2403, 1969, 3533, 1764, 2910, 1968, 2673, + 2484, 2887, 2405, 2486, 2487, 2488, 2489, 1764, 2468, 1764, + 1875, 1874, 1492, 1784, 2500, 166, 3496, 1764, 2546, 1817, + 1816, 3494, 1764, 2469, 2427, 2506, 1281, 1489, 3571, 1490, + 1491, 3241, 1488, 2451, 2452, 1098, 1097, 2450, 2918, 2467, + 111, 4097, 3571, 3575, 2466, 4069, 3271, 1492, 2558, 2892, + 3490, 1764, 1489, 2543, 1490, 1491, 2640, 2918, 3487, 1764, + 2917, 3524, 2516, 1492, 3306, 1489, 3214, 1490, 1491, 3485, + 1764, 2704, 2453, 1489, 3571, 1490, 1491, 2672, 1489, 1051, + 1490, 1491, 1492, 3483, 1764, 2798, 2501, 1492, 2701, 2497, + 2490, 2492, 2493, 1492, 2700, 2557, 2520, 2551, 2515, 2519, + 2554, 2540, 2555, 2422, 3481, 1764, 2571, 1489, 1492, 1490, + 1491, 3574, 2557, 1492, 1488, 1489, 2409, 1490, 1491, 1768, + 2381, 2918, 2501, 2550, 2549, 2553, 1489, 1152, 1490, 1491, + 97, 2156, 2094, 2078, 1489, 2021, 1490, 1491, 1153, 2572, + 1489, 1804, 1490, 1491, 1135, 3623, 684, 1134, 1914, 2575, + 95, 4176, 4116, 2576, 2577, 3479, 1764, 1492, 3955, 1765, + 1767, 1489, 95, 1490, 1491, 1949, 1771, 3921, 3781, 1060, + 3639, 167, 3636, 1492, 3477, 1764, 3617, 3437, 179, 2645, + 1764, 2582, 3436, 1877, 3330, 3475, 1764, 2499, 1050, 1050, + 1050, 3377, 3332, 3328, 684, 3007, 1489, 2664, 1490, 1491, + 3473, 1764, 3624, 3625, 3626, 3471, 1764, 2496, 1545, 2491, + 1545, 1492, 1489, 684, 1490, 1491, 2485, 2483, 2003, 187, + 1909, 1492, 1950, 1951, 1952, 1905, 2660, 1492, 1873, 123, + 2974, 1489, 2973, 1490, 1491, 2615, 1489, 1225, 1490, 1491, + 1492, 3381, 1489, 2357, 1490, 1491, 3585, 3586, 2356, 3469, + 1764, 3956, 2514, 1492, 167, 4233, 2663, 1489, 2394, 1490, + 1491, 179, 1489, 1538, 1490, 1491, 168, 173, 170, 176, + 177, 178, 180, 182, 183, 184, 185, 4231, 4204, 4077, + 3996, 2400, 186, 188, 189, 190, 2974, 3627, 2632, 3588, + 1538, 3324, 4073, 1538, 3323, 1538, 684, 3322, 3241, 2986, + 2616, 2638, 187, 3467, 1764, 1492, 1489, 1726, 1490, 1491, + 2071, 1781, 3591, 3465, 1764, 1492, 1976, 2646, 3263, 3463, + 1764, 1492, 1489, 3264, 1490, 1491, 1786, 3590, 3957, 1492, + 2652, 684, 3461, 1764, 3260, 1492, 2649, 2650, 2651, 1492, + 3628, 3629, 3630, 3259, 2123, 3459, 1764, 2030, 684, 168, + 173, 170, 176, 177, 178, 180, 182, 183, 184, 185, + 1489, 3421, 1490, 1491, 684, 186, 188, 189, 190, 1492, + 1489, 684, 1490, 1491, 2413, 3579, 1489, 2072, 1490, 1491, + 2056, 2057, 684, 684, 684, 684, 684, 684, 684, 1489, + 1492, 1490, 1491, 2676, 3261, 3219, 3218, 1785, 2654, 3262, + 1492, 4035, 1489, 3564, 1490, 1491, 1492, 3457, 1764, 3231, + 3233, 3563, 1492, 3443, 1764, 685, 2624, 2625, 3234, 3759, + 1492, 2627, 4212, 2653, 1492, 2655, 2842, 3419, 1764, 3761, + 2628, 3567, 1764, 3228, 2658, 1492, 2659, 3265, 2020, 2927, + 2928, 1492, 2661, 1065, 2923, 2926, 2927, 2928, 2924, 2712, + 2925, 2929, 2830, 1492, 1489, 1720, 1490, 1491, 2874, 1034, + 1718, 2863, 1764, 2957, 1489, 1716, 1490, 1491, 1717, 1715, + 1489, 1719, 1490, 1491, 2978, 2153, 2877, 1050, 1489, 2151, + 1490, 1491, 2861, 1764, 1489, 1492, 1490, 1491, 1489, 748, + 1490, 1491, 2836, 1764, 1066, 3302, 1077, 2608, 2813, 1764, + 1068, 1492, 2915, 2916, 2805, 1764, 2199, 2875, 1069, 2607, + 1076, 2440, 2796, 1764, 1050, 2935, 2794, 1764, 1489, 2606, + 1490, 1491, 1492, 2200, 2119, 2117, 2118, 2781, 1764, 1492, + 2605, 2604, 2603, 2779, 1764, 2878, 2602, 2880, 1492, 1489, + 1191, 1490, 1491, 1190, 3397, 3963, 2895, 1492, 2122, 1489, + 2914, 1490, 1491, 2973, 1454, 1489, 103, 1490, 1491, 3056, + 3013, 1489, 105, 1490, 1491, 131, 104, 4152, 3569, 1489, + 105, 1490, 1491, 1489, 4247, 1490, 1491, 2777, 1764, 2865, + 1732, 103, 2990, 2904, 1489, 684, 1490, 1491, 2933, 42, + 1489, 104, 1490, 1491, 4053, 2424, 2425, 2619, 2932, 1492, + 4054, 2934, 1489, 2885, 1490, 1491, 3951, 2960, 2962, 1775, + 2893, 3301, 1492, 1725, 2775, 1764, 2931, 2406, 3547, 2907, + 1492, 2773, 1764, 2937, 1492, 2890, 2953, 3217, 1492, 3011, + 2771, 1764, 2644, 1492, 1489, 3216, 1490, 1491, 2077, 2769, + 1764, 2941, 3746, 1492, 3745, 1538, 2944, 1945, 112, 1492, + 1489, 2076, 1490, 1491, 1492, 2506, 2954, 110, 2951, 111, + 2194, 110, 2976, 1538, 3022, 2195, 112, 2979, 2980, 105, + 4020, 1489, 4019, 1490, 1491, 2965, 3999, 111, 1489, 110, + 1490, 1491, 3767, 3765, 3764, 3757, 3637, 1489, 3568, 1490, + 1491, 2767, 1764, 2975, 3566, 3333, 1489, 3744, 1490, 1491, + 1946, 1947, 1948, 2257, 2983, 3928, 2541, 1892, 2987, 2988, + 2989, 2984, 2765, 1764, 1075, 1492, 2763, 1764, 112, 3019, + 2761, 1764, 111, 1897, 3756, 2759, 1764, 3730, 3557, 111, + 2889, 3066, 3067, 2910, 1492, 2757, 1764, 4235, 4234, 3008, + 3009, 2755, 1764, 3120, 2702, 1492, 2753, 1764, 1489, 2392, + 1490, 1491, 1798, 3018, 1492, 1790, 116, 117, 3, 4234, + 4235, 1489, 4041, 1490, 1491, 3602, 99, 2091, 1, 1489, + 10, 1490, 1491, 1489, 1042, 1490, 1491, 1489, 1457, 1490, + 1491, 1492, 1489, 2340, 1490, 1491, 2089, 3045, 2090, 9, + 3083, 8, 1489, 3048, 1490, 1491, 1456, 1492, 1489, 2030, + 1490, 1491, 3064, 1489, 3606, 1490, 1491, 2751, 1764, 4164, + 701, 2382, 1730, 2372, 4205, 4160, 3101, 3102, 3103, 3104, + 3105, 3106, 3107, 3108, 3109, 3110, 2749, 1764, 4161, 1492, + 1765, 2379, 1989, 1979, 3669, 2308, 3118, 2747, 1764, 3952, + 3336, 2547, 3635, 2504, 1976, 1143, 2742, 1764, 156, 2462, + 2463, 4129, 120, 1101, 3084, 119, 1146, 1254, 2542, 3020, + 1081, 3659, 2958, 2471, 1489, 1823, 1490, 1491, 1821, 1822, + 1820, 2404, 1726, 2738, 1764, 1825, 1824, 4101, 1492, 1060, + 3405, 2703, 3503, 1489, 1492, 1490, 1491, 2082, 738, 2736, + 1764, 2930, 1492, 732, 1489, 194, 1490, 1491, 1812, 1791, + 2075, 684, 3178, 1489, 3122, 1490, 1491, 1185, 2030, 684, + 691, 684, 3310, 684, 2449, 3054, 2580, 697, 3055, 2251, + 1542, 2729, 1764, 2070, 3215, 2945, 1095, 1087, 3065, 2393, + 1489, 2879, 1490, 1491, 1094, 3929, 3249, 2998, 3561, 3227, + 3229, 3196, 2897, 3232, 3225, 3185, 1489, 4034, 1490, 1491, + 3187, 3758, 2923, 2926, 2927, 2928, 2924, 3068, 2925, 2929, + 2440, 4114, 3585, 3586, 3111, 3085, 2666, 1492, 2955, 1787, + 2727, 1764, 3523, 2343, 3158, 2343, 2665, 2345, 1489, 2345, + 1490, 1491, 2675, 2518, 3248, 3641, 87, 2189, 1532, 2440, + 2440, 2440, 2440, 2440, 763, 2439, 2526, 934, 1773, 2334, + 2335, 2336, 2337, 2338, 3725, 2109, 3196, 761, 760, 2440, + 758, 3186, 2440, 3188, 2881, 2911, 2359, 3253, 1492, 2362, 2363, 3195, 3168, 3169, 3170, 3171, 3172, 1489, 2031, 1490, - 1491, 3270, 3229, 2897, 1492, 1489, 3232, 1490, 1491, 1489, - 3223, 1490, 1491, 3207, 3213, 1492, 3225, 2442, 4034, 3758, - 1492, 4114, 2955, 1787, 3523, 2380, 3220, 2675, 2851, 3222, - 1492, 2189, 1532, 763, 3235, 3236, 2439, 1489, 1492, 1490, - 1491, 934, 1773, 1053, 3725, 3353, 2442, 2442, 2442, 2442, - 2442, 2109, 1055, 1492, 761, 3361, 3252, 3254, 760, 3272, - 3257, 1054, 3273, 3266, 3255, 3256, 2442, 3258, 106, 2442, - 107, 3208, 3210, 3212, 758, 3274, 2881, 2850, 3280, 2911, - 3221, 1496, 1495, 969, 684, 1489, 2869, 1490, 1491, 1799, - 3312, 684, 3313, 3311, 1492, 2922, 2849, 2920, 3314, 3315, - 2919, 2617, 684, 684, 2447, 3587, 684, 2840, 2620, 3583, - 4156, 2441, 2839, 2437, 3362, 2888, 920, 919, 684, 3366, - 770, 762, 2838, 3365, 2506, 684, 3238, 1492, 3355, 752, - 2837, 3334, 983, 918, 1489, 3372, 1490, 1491, 917, 3363, - 3364, 1492, 3028, 3378, 3030, 2834, 1492, 2956, 3374, 1473, - 1492, 684, 3393, 1489, 3392, 1490, 1491, 1749, 2631, 1492, - 3390, 1752, 2401, 1114, 1489, 3402, 1490, 1491, 3244, 1489, - 4060, 1490, 1491, 3244, 2643, 3431, 1748, 4067, 3400, 1489, - 3344, 1490, 1491, 3653, 3325, 3004, 2829, 1489, 3410, 1490, - 1491, 2534, 69, 46, 3426, 3427, 3428, 3429, 3430, 4029, - 3407, 3408, 1489, 3409, 1490, 1491, 3411, 4098, 3413, 912, - 3415, 909, 3727, 3728, 3729, 3181, 3182, 4080, 4081, 2822, - 908, 4082, 2246, 1467, 1464, 4178, 2084, 98, 36, 35, - 1545, 34, 2662, 2821, 1545, 33, 2667, 1538, 2820, 2030, - 32, 26, 2819, 1489, 25, 1490, 1491, 24, 23, 22, - 3548, 2818, 3550, 29, 19, 21, 20, 18, 3335, 2670, - 3347, 2671, 3518, 4200, 4246, 125, 55, 2679, 52, 3522, - 50, 2681, 2682, 133, 132, 53, 1489, 49, 1490, 1491, + 1491, 3270, 1496, 1489, 1495, 1490, 1491, 3223, 969, 3207, + 2869, 1489, 1492, 1490, 1491, 3213, 1799, 2442, 1492, 2922, + 2920, 2919, 2617, 2447, 3587, 2380, 3222, 3583, 1492, 3593, + 3220, 4156, 1492, 2441, 3235, 3236, 2437, 2888, 1492, 920, + 919, 770, 762, 1053, 752, 3353, 2442, 2442, 2442, 2442, + 2442, 983, 1055, 1492, 918, 3361, 3272, 3252, 1054, 3273, + 917, 3255, 3256, 3266, 3258, 1492, 2442, 3363, 106, 2442, + 107, 3364, 3254, 1492, 3274, 3257, 3028, 3208, 3210, 3212, + 3221, 2969, 3378, 3030, 684, 3280, 1489, 2956, 1490, 1491, + 3312, 684, 3374, 1492, 1473, 3311, 3313, 1749, 3314, 3315, + 1752, 2401, 684, 684, 3540, 1114, 684, 3402, 2620, 4060, + 3536, 2643, 3431, 1748, 3362, 3366, 4067, 3344, 684, 3334, + 3502, 3653, 3325, 3365, 3498, 684, 3238, 2506, 3355, 3004, + 3434, 2534, 69, 46, 4029, 3372, 4098, 1489, 912, 1490, + 1491, 1492, 909, 3727, 3728, 3433, 1492, 3729, 3181, 3182, + 4080, 684, 3393, 4081, 3392, 908, 1492, 3425, 2631, 4082, + 3390, 1489, 2246, 1490, 1491, 3423, 1467, 1489, 3244, 1490, + 1491, 1464, 4178, 3244, 2084, 98, 36, 1489, 3400, 1490, + 1491, 1489, 35, 1490, 1491, 2859, 34, 1489, 3410, 1490, + 1491, 33, 32, 26, 3426, 3427, 3428, 3429, 3430, 25, + 3407, 3408, 1489, 3409, 1490, 1491, 3411, 24, 3413, 23, + 3415, 22, 29, 19, 1489, 21, 1490, 1491, 20, 18, + 3347, 4200, 1489, 4246, 1490, 1491, 125, 55, 52, 50, + 1545, 133, 2662, 2858, 1545, 132, 2667, 1538, 2854, 2030, + 53, 49, 1489, 1228, 1490, 1491, 47, 96, 2853, 31, + 3548, 30, 3550, 17, 16, 15, 14, 13, 3335, 2670, + 12, 2671, 3518, 11, 7, 6, 39, 2679, 38, 3522, + 37, 2681, 2682, 28, 27, 40, 4, 2991, 2536, 0, 2688, 2689, 2690, 2691, 2692, 2693, 2694, 2695, 2696, 2697, - 1489, 2699, 1490, 1491, 1228, 1489, 47, 1490, 1491, 1489, - 96, 1490, 1491, 31, 3401, 30, 3247, 17, 1489, 16, - 1490, 1491, 2440, 15, 2705, 2706, 2707, 2708, 14, 2710, - 2711, 3549, 2713, 3551, 13, 3600, 2715, 3553, 3565, 12, - 2720, 2721, 11, 2722, 7, 3558, 2725, 2726, 2728, 2730, + 1489, 2699, 1490, 1491, 0, 1489, 0, 1490, 1491, 0, + 0, 0, 0, 0, 3401, 1489, 3247, 1490, 1491, 0, + 0, 0, 2440, 0, 2705, 2706, 2707, 2708, 0, 2710, + 2711, 3549, 2713, 3551, 0, 3600, 2715, 3553, 3565, 0, + 2720, 2721, 0, 2722, 0, 3558, 2725, 2726, 2728, 2730, 2731, 2732, 2733, 2734, 2735, 2737, 2739, 2740, 2741, 2743, 1492, 2745, 2746, 2748, 2750, 2752, 2754, 2756, 2758, 2760, 2762, 2764, 2766, 2768, 2770, 2772, 2774, 2776, 2778, 2780, 2782, 2783, 2784, 3356, 2786, 3592, 2788, 3594, 2790, 2791, - 3589, 2793, 2795, 2797, 3362, 3570, 1492, 2800, 3601, 3366, - 1492, 2804, 3595, 3365, 1492, 2809, 2810, 2811, 2812, 2442, - 3395, 3396, 1492, 6, 39, 3618, 1492, 3620, 2823, 2824, - 2825, 2826, 2827, 2828, 3555, 1492, 2832, 2833, 38, 3612, - 3613, 37, 28, 27, 2835, 1492, 40, 4, 2991, 2841, - 1492, 2536, 0, 0, 2844, 2845, 2846, 2847, 2848, 0, - 0, 0, 2817, 0, 0, 2855, 2856, 3582, 2857, 0, + 3589, 2793, 2795, 2797, 3362, 3366, 3570, 2800, 3601, 3595, + 1492, 2804, 0, 3365, 1492, 2809, 2810, 2811, 2812, 2442, + 3395, 3396, 1492, 0, 0, 3618, 1492, 3620, 2823, 2824, + 2825, 2826, 2827, 2828, 3555, 1492, 2832, 2833, 0, 3612, + 3613, 0, 0, 0, 2835, 1492, 0, 0, 0, 2841, + 1492, 0, 0, 0, 2844, 2845, 2846, 2847, 2848, 0, + 0, 0, 2852, 0, 0, 2855, 2856, 3582, 2857, 0, 0, 2860, 2862, 2404, 0, 2864, 3525, 1492, 3527, 3528, - 3529, 0, 684, 0, 0, 2876, 3596, 3597, 3663, 3664, - 1976, 3646, 0, 1492, 0, 3650, 3651, 3652, 2816, 0, - 0, 3665, 2815, 1492, 0, 2894, 2814, 0, 1492, 0, - 0, 0, 1492, 0, 2808, 2683, 0, 0, 2807, 1489, - 0, 1490, 1491, 0, 0, 0, 0, 2806, 1492, 0, - 3681, 0, 2698, 0, 0, 0, 0, 2803, 0, 0, - 0, 684, 2802, 0, 1492, 0, 684, 0, 0, 1492, - 0, 0, 0, 0, 0, 1489, 3692, 1490, 1491, 1489, - 0, 1490, 1491, 1489, 0, 1490, 1491, 0, 0, 2801, + 3529, 0, 684, 0, 0, 2876, 3596, 3597, 0, 0, + 1976, 3646, 3663, 3664, 0, 3650, 3651, 3652, 0, 0, + 0, 3665, 2851, 1492, 0, 2894, 2850, 0, 1492, 0, + 0, 0, 1492, 0, 2849, 2683, 0, 0, 2840, 1489, + 0, 1490, 1491, 0, 0, 0, 0, 2839, 1492, 0, + 3681, 0, 2698, 0, 0, 0, 0, 2838, 0, 0, + 0, 684, 2837, 0, 1492, 0, 684, 0, 0, 1492, + 0, 0, 0, 1492, 0, 0, 3692, 0, 1492, 1489, + 0, 1490, 1491, 1489, 0, 1490, 1491, 0, 0, 2834, 0, 1489, 0, 1490, 1491, 1489, 0, 1490, 1491, 1492, - 0, 0, 0, 0, 1489, 2799, 1490, 1491, 3733, 0, - 3734, 3735, 3736, 0, 1489, 2792, 1490, 1491, 0, 1489, - 2789, 1490, 1491, 0, 2787, 0, 0, 3743, 0, 0, - 3750, 0, 3752, 0, 0, 1492, 0, 0, 0, 3723, - 2785, 684, 1492, 0, 0, 0, 1489, 0, 1490, 1491, - 0, 0, 0, 0, 0, 3248, 2744, 3753, 87, 0, - 3248, 2724, 1489, 0, 1490, 1491, 0, 0, 0, 0, - 0, 1492, 1489, 0, 1490, 1491, 0, 1489, 0, 1490, + 0, 0, 0, 0, 1489, 0, 1490, 1491, 3733, 0, + 3734, 3735, 3736, 0, 1489, 2829, 1490, 1491, 0, 1489, + 2822, 1490, 1491, 0, 2821, 0, 0, 3743, 0, 0, + 3750, 0, 3752, 0, 0, 0, 1492, 0, 0, 3723, + 2820, 684, 1492, 0, 0, 0, 1489, 0, 1490, 1491, + 0, 0, 0, 0, 0, 3248, 2819, 3753, 87, 0, + 3248, 2818, 0, 0, 0, 2817, 0, 0, 0, 0, + 2816, 1492, 1489, 0, 1490, 1491, 0, 1489, 0, 1490, 1491, 1489, 1492, 1490, 1491, 0, 2153, 3783, 3686, 0, - 2151, 2723, 3754, 1492, 1538, 3763, 3762, 1489, 1492, 1490, + 2151, 2815, 3754, 0, 1538, 3763, 3762, 1489, 1492, 1490, 1491, 0, 0, 3773, 3770, 3772, 3775, 684, 684, 684, - 684, 684, 684, 1489, 0, 1490, 1491, 0, 1489, 0, - 1490, 1491, 0, 0, 0, 0, 3935, 2719, 0, 0, - 0, 0, 3787, 0, 2717, 0, 684, 684, 3642, 3643, + 684, 684, 684, 1489, 1492, 1490, 1491, 0, 1489, 0, + 1490, 1491, 1489, 0, 1490, 1491, 3935, 1489, 2814, 1490, + 1491, 0, 3787, 0, 2808, 0, 684, 684, 3642, 3643, 0, 0, 0, 0, 0, 42, 0, 2380, 1489, 0, 1490, 1491, 0, 0, 0, 0, 0, 0, 0, 0, - 3926, 684, 0, 2709, 3927, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2680, 0, 3954, 0, 0, 3925, - 3942, 3947, 3946, 0, 1489, 2674, 1490, 1491, 0, 0, - 2669, 1489, 0, 1490, 1491, 0, 0, 0, 3096, 3097, - 3098, 3099, 3100, 3993, 3994, 0, 0, 3777, 0, 0, + 3926, 684, 0, 2807, 3927, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 2806, 0, 3954, 0, 0, 3925, + 3942, 3947, 3946, 0, 0, 1489, 0, 1490, 1491, 0, + 2803, 1489, 0, 1490, 1491, 0, 0, 0, 3096, 3097, + 3098, 3099, 3100, 3993, 3994, 0, 2802, 3777, 0, 0, 0, 3751, 3937, 3938, 3939, 2153, 3997, 0, 3115, 2151, 1489, 0, 1490, 1491, 0, 0, 0, 0, 3784, 3785, 0, 1489, 0, 1490, 1491, 0, 0, 0, 0, 0, - 0, 0, 1489, 0, 1490, 1491, 0, 1489, 0, 1490, + 0, 0, 0, 0, 0, 0, 0, 1489, 0, 1490, 1491, 0, 4000, 0, 4042, 3248, 4003, 0, 3779, 0, - 0, 0, 0, 0, 0, 0, 0, 3244, 0, 1563, + 0, 0, 0, 1489, 0, 1490, 1491, 3244, 0, 1563, 1564, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 1575, 1576, 1577, 1578, 1579, 1580, 1581, 1583, 1584, 1585, 1586, 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, @@ -2100,27 +2100,27 @@ var yyAct = [...]int{ 1676, 1677, 1683, 1684, 1685, 1686, 1700, 1701, 1702, 1703, 1704, 1705, 1706, 1707, 1708, 1709, 1710, 1711, 1712, 1713, 750, 3998, 4043, 4027, 4017, 4026, 3086, 3247, 1081, 0, - 684, 4023, 3247, 4025, 4061, 0, 0, 0, 0, 0, + 684, 4023, 3247, 4025, 4061, 0, 0, 0, 1492, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3250, 0, 87, 1538, 0, 0, 4046, 0, 1538, 684, - 684, 684, 684, 684, 1754, 3930, 0, 0, 3268, 0, - 0, 3267, 1754, 0, 0, 4066, 4045, 1976, 1762, 684, - 4050, 1755, 684, 3275, 2030, 4063, 1762, 0, 0, 1755, + 684, 684, 684, 684, 1492, 3930, 0, 0, 3268, 0, + 0, 3267, 1754, 0, 0, 4066, 4045, 1976, 0, 684, + 4050, 1492, 684, 3275, 2030, 4063, 1762, 0, 0, 1755, 0, 0, 0, 0, 0, 0, 0, 0, 3934, 0, - 4068, 0, 0, 0, 0, 0, 2398, 2399, 1761, 1759, - 1760, 1756, 0, 1757, 1750, 1751, 1761, 1759, 1760, 1756, + 4068, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 2398, 2399, 1761, 1759, 1760, 1756, 0, 1757, 0, 0, 0, 0, 0, 684, 0, 0, - 0, 1074, 0, 0, 1080, 1080, 1758, 0, 0, 42, + 2801, 1074, 0, 0, 1080, 1080, 0, 0, 0, 42, 0, 0, 4086, 1538, 1758, 4087, 0, 0, 0, 0, - 4111, 0, 0, 684, 0, 87, 0, 0, 3160, 0, + 4111, 0, 0, 684, 0, 87, 2799, 0, 3160, 0, 3162, 0, 0, 0, 0, 0, 0, 684, 4096, 0, - 0, 4105, 0, 0, 0, 0, 3173, 3174, 3175, 3176, - 0, 0, 0, 0, 0, 0, 4117, 0, 684, 0, - 0, 684, 0, 0, 0, 4128, 3399, 3954, 4131, 4120, + 0, 4105, 0, 2792, 0, 0, 3173, 3174, 3175, 3176, + 0, 0, 0, 0, 0, 0, 4117, 1489, 684, 1490, + 1491, 684, 0, 0, 0, 4128, 3399, 3954, 4131, 4120, 4125, 4122, 4121, 4115, 4119, 4124, 4123, 4047, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3416, 3417, + 0, 0, 0, 1489, 0, 1490, 1491, 0, 3416, 3417, 0, 3418, 3420, 3422, 4150, 0, 4174, 3247, 0, 0, - 0, 0, 0, 0, 0, 4155, 0, 4163, 4168, 0, + 1489, 0, 1490, 1491, 0, 4155, 0, 4163, 4168, 0, 0, 0, 42, 4142, 4182, 0, 0, 0, 0, 3435, 0, 0, 0, 4195, 3438, 4184, 3440, 3441, 3442, 3444, 3445, 3446, 3447, 3448, 3449, 3450, 3451, 3452, 3453, 3454, @@ -2129,91 +2129,91 @@ var yyAct = [...]int{ 3492, 3493, 3495, 4219, 4199, 3497, 2031, 3499, 3500, 3501, 4142, 4215, 3505, 3506, 3507, 3508, 3509, 3510, 3511, 3512, 3513, 3514, 3515, 2153, 4229, 4232, 4225, 2151, 4228, 4226, - 4230, 3521, 4216, 4194, 0, 3526, 0, 4112, 192, 3530, - 3531, 0, 3532, 3534, 4240, 3537, 3539, 4248, 3541, 3542, + 4230, 3521, 4216, 4194, 0, 3526, 1492, 4112, 192, 3530, + 3531, 1492, 3532, 3534, 4240, 3537, 3539, 4248, 3541, 3542, 3543, 3544, 4107, 4256, 4254, 0, 0, 4052, 3552, 0, 0, 131, 0, 153, 0, 0, 684, 3244, 0, 0, - 0, 0, 0, 0, 0, 4264, 0, 174, 0, 4265, + 0, 0, 0, 0, 1510, 4264, 0, 174, 0, 4265, 4266, 3994, 2153, 4263, 0, 0, 2151, 0, 0, 4142, - 0, 4071, 0, 3576, 3577, 0, 4059, 3581, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1032, - 0, 164, 2347, 0, 0, 1033, 0, 152, 0, 0, - 0, 0, 684, 0, 0, 2152, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 171, 0, 0, + 0, 4071, 0, 3576, 3577, 0, 4059, 3581, 1511, 1512, + 1513, 1514, 1515, 1516, 1517, 1519, 1518, 1520, 1521, 1492, + 1032, 164, 4198, 2347, 0, 0, 1033, 152, 0, 0, + 0, 1841, 684, 0, 0, 0, 2152, 0, 2789, 0, + 0, 0, 0, 2787, 0, 0, 0, 171, 0, 0, 172, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 684, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 140, 141, 163, 162, 191, 0, 0, 0, - 0, 684, 0, 684, 684, 684, 0, 0, 4106, 0, - 4190, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3657, 0, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014, - 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023, 1024, - 1025, 1026, 1027, 1028, 1029, 1030, 0, 3676, 0, 0, - 3680, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 157, 138, - 160, 145, 137, 0, 158, 159, 0, 0, 0, 0, - 0, 175, 0, 0, 3693, 0, 0, 0, 0, 0, - 181, 146, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 140, 141, 163, 162, 191, 1492, 0, 0, + 0, 684, 1492, 684, 684, 684, 0, 0, 4106, 0, + 4190, 1492, 0, 0, 0, 1489, 0, 1490, 1491, 0, + 1489, 1492, 1490, 1491, 0, 1492, 0, 0, 0, 0, + 1492, 2785, 3657, 0, 1492, 989, 990, 991, 992, 993, + 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, + 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, + 1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, 1022, 1023, + 1024, 1025, 1026, 1027, 1028, 1029, 1030, 3676, 0, 1492, + 3680, 0, 0, 1492, 0, 0, 0, 0, 1489, 2744, + 1490, 1491, 0, 0, 2724, 0, 0, 0, 157, 138, + 160, 145, 137, 2723, 158, 159, 0, 0, 1828, 0, + 2182, 175, 0, 2719, 3693, 0, 0, 2717, 0, 0, + 181, 146, 2709, 0, 0, 0, 2680, 0, 0, 0, 0, 0, 0, 0, 0, 149, 147, 142, 143, 144, - 148, 0, 4213, 0, 0, 1726, 0, 139, 0, 0, - 0, 0, 0, 0, 0, 0, 150, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3716, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3724, 1510, 0, 0, 0, 0, 0, 3731, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 86, 44, 45, 88, 0, 1511, 1512, 1513, 1514, - 1515, 1516, 1517, 1519, 1518, 1520, 1521, 0, 0, 0, - 92, 0, 0, 1976, 48, 76, 77, 0, 74, 78, + 148, 0, 4213, 0, 0, 1726, 1489, 139, 1490, 1491, + 0, 1489, 0, 1490, 1491, 0, 150, 0, 0, 0, + 1489, 0, 1490, 1491, 0, 0, 0, 0, 0, 0, + 1489, 2674, 1490, 1491, 1489, 2669, 1490, 1491, 3716, 1489, + 0, 1490, 1491, 1489, 0, 1490, 1491, 0, 0, 0, + 0, 3724, 1842, 0, 0, 0, 0, 0, 3731, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2174, + 2163, 2164, 2165, 2166, 2176, 2167, 2168, 2169, 2181, 2177, + 2170, 2171, 2178, 2179, 2180, 2172, 2173, 2175, 1489, 0, + 1490, 1491, 1489, 1976, 1490, 1491, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 166, - 75, 0, 1538, 0, 0, 0, 1976, 0, 0, 0, - 0, 0, 97, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 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, 0, 0, 0, 0, 0, + 0, 0, 1538, 0, 0, 0, 1976, 0, 1855, 1858, + 1859, 1860, 1861, 1862, 1863, 0, 1864, 1865, 1867, 1868, + 1866, 1869, 1870, 1843, 1844, 1845, 1846, 1826, 1827, 1856, + 0, 1829, 0, 1830, 1831, 1832, 1833, 1834, 1835, 1836, + 1837, 1838, 0, 0, 1839, 1847, 1848, 1849, 1850, 0, + 1851, 1852, 1853, 1854, 0, 0, 1840, 0, 0, 0, 0, 0, 1493, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3943, 0, 0, 0, 0, 1976, 0, 0, 0, 3950, 0, 0, - 0, 0, 0, 1551, 83, 161, 0, 0, 0, 0, + 0, 0, 0, 1551, 0, 161, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3960, 3961, 3962, 0, 3964, 0, 3965, 3966, 0, 0, 0, 0, 3969, 3970, 3971, 3972, 3973, 3974, 3975, 3976, 3977, 3978, 3979, 3980, 3981, 3982, 3983, 3984, 3985, 3986, 3987, 3988, 3989, - 3990, 0, 3992, 3995, 0, 3714, 0, 0, 2182, 0, + 3990, 0, 3992, 3995, 0, 3714, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4004, 4005, 4006, 4007, 4008, 4010, 4011, 4013, 4015, 4016, 4018, 0, - 0, 0, 4022, 0, 0, 0, 4024, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 51, - 54, 57, 56, 59, 1032, 73, 0, 0, 82, 79, - 1033, 0, 0, 0, 154, 0, 0, 155, 0, 0, - 2152, 4051, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 61, 91, 90, 0, 0, 71, 72, 58, - 0, 0, 0, 0, 0, 80, 81, 0, 167, 0, - 0, 0, 0, 0, 0, 179, 0, 2174, 2163, 2164, - 2165, 2166, 2176, 2167, 2168, 2169, 2181, 2177, 2170, 2171, - 2178, 2179, 2180, 2172, 2173, 2175, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 63, - 64, 0, 65, 66, 67, 68, 187, 0, 0, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, - 1010, 1011, 1012, 1013, 1014, 1015, 1016, 1017, 1018, 1019, - 1020, 1021, 1022, 1023, 1024, 1025, 1026, 1027, 1028, 1029, - 1030, 0, 0, 168, 173, 170, 176, 177, 178, 180, - 182, 183, 184, 185, 0, 0, 60, 0, 0, 186, + 1754, 0, 4022, 0, 0, 0, 4024, 0, 0, 0, + 0, 0, 0, 0, 1762, 0, 0, 1755, 0, 0, + 0, 0, 1032, 0, 0, 0, 0, 970, 1033, 984, + 985, 986, 971, 0, 154, 972, 973, 155, 974, 0, + 0, 4051, 1750, 1751, 1761, 1759, 1760, 1756, 0, 1757, + 0, 0, 0, 0, 0, 0, 987, 988, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 167, 0, + 0, 0, 1758, 0, 0, 179, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1857, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 187, 989, 990, 991, + 992, 993, 994, 995, 996, 997, 998, 999, 1000, 1001, + 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, + 1012, 1013, 1014, 1015, 1016, 1017, 1018, 1019, 1020, 1021, + 1022, 1023, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 4143, + 0, 0, 0, 168, 173, 170, 176, 177, 178, 180, + 182, 183, 184, 185, 0, 0, 0, 0, 0, 186, 188, 189, 190, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1976, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3369, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1789, 684, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4033, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4076, 0, 0, 0, 0, 0, 1878, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 4198, 0, 89, 0, 0, 0, - 0, 0, 4091, 1841, 0, 0, 0, 0, 4094, 0, - 4095, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 4091, 0, 0, 0, 0, 0, 4094, 0, + 4095, 3370, 3371, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1538, 4110, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2222,43 +2222,43 @@ var yyAct = [...]int{ 0, 0, 0, 4144, 4146, 4148, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4154, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1976, 0, 4177, 0, 0, 94, 0, 0, 0, + 0, 1976, 0, 4177, 0, 0, 0, 0, 0, 0, 0, 0, 2036, 197, 0, 0, 197, 0, 0, 0, 736, 0, 0, 0, 0, 742, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 197, 0, 0, 0, 0, 4196, 1726, 0, 0, 0, 0, 0, 0, 0, 0, 0, 197, 0, 0, 0, 0, 0, 0, 0, - 1828, 0, 0, 192, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 192, 0, 0, 0, 0, 0, 0, 0, 0, 2030, 0, 2997, 0, 4220, 4222, 4224, 742, 197, 742, 0, 0, 0, 0, 131, 0, 153, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4075, 0, 0, 0, 174, 0, 0, 0, 4085, 0, 0, 0, - 4245, 0, 0, 0, 0, 0, 0, 70, 0, 0, + 4245, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 4257, 4258, 0, 0, 0, 0, 0, 0, 164, 0, 0, 0, - 0, 964, 152, 0, 1842, 0, 0, 0, 0, 0, + 0, 964, 152, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 0, 0, 172, 0, 0, 1726, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1900, 1901, 163, 162, 191, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 719, 0, 0, 0, 0, 0, 741, - 1855, 1858, 1859, 1860, 1861, 1862, 1863, 0, 1864, 1865, - 1867, 1868, 1866, 1869, 1870, 1843, 1844, 1845, 1846, 1826, - 1827, 1856, 0, 1829, 0, 1830, 1831, 1832, 1833, 1834, - 1835, 1836, 1837, 1838, 0, 0, 1839, 1847, 1848, 1849, - 1850, 0, 1851, 1852, 1853, 1854, 0, 0, 1840, 2104, - 2105, 2106, 2107, 741, 0, 741, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 86, 44, 45, 88, 0, 0, 0, 0, 0, 0, + 0, 4141, 0, 0, 0, 0, 0, 0, 0, 92, + 0, 0, 0, 48, 76, 77, 0, 74, 78, 2104, + 2105, 2106, 2107, 741, 0, 741, 0, 0, 0, 75, + 0, 0, 0, 0, 0, 2120, 0, 4140, 0, 0, + 0, 97, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 157, 1902, 160, 0, 1899, 0, 158, - 159, 0, 0, 0, 0, 0, 175, 0, 0, 0, - 2159, 2160, 0, 0, 0, 181, 2183, 0, 0, 2187, + 159, 62, 0, 0, 0, 0, 175, 0, 0, 0, + 2159, 2160, 0, 95, 0, 181, 2183, 0, 0, 2187, 2188, 0, 0, 0, 2193, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2205, 2206, 2207, 2208, 2209, 2210, 2211, 2212, 2213, 2214, 0, 2216, 0, 0, 0, 2238, 2239, 2240, 2241, 2242, 2243, - 2244, 2245, 2247, 0, 2252, 0, 2254, 2255, 2256, 0, + 2244, 2245, 2247, 83, 2252, 0, 2254, 2255, 2256, 0, 2258, 2259, 2260, 0, 2262, 2263, 2264, 2265, 2266, 2267, 2268, 2269, 2270, 2271, 2272, 2273, 2274, 2275, 2276, 2277, 2278, 2279, 2280, 2281, 2282, 2283, 2284, 2285, 2286, 2287, @@ -2267,54 +2267,54 @@ var yyAct = [...]int{ 2311, 2312, 2313, 2314, 2315, 2316, 2317, 2318, 2319, 2320, 2321, 2322, 2323, 2324, 2325, 2326, 2327, 2328, 2329, 2330, 2331, 2332, 2333, 0, 166, 0, 0, 0, 2339, 0, - 2341, 0, 2348, 2349, 2350, 2351, 2352, 2353, 0, 0, - 0, 0, 0, 1857, 0, 0, 0, 0, 0, 0, + 2341, 0, 2348, 2349, 2350, 2351, 2352, 2353, 51, 54, + 57, 56, 59, 0, 73, 0, 0, 82, 79, 0, 0, 2364, 2365, 2366, 2367, 2368, 2369, 2370, 2371, 0, 2373, 2374, 2375, 2376, 2377, 0, 0, 0, 0, 0, - 0, 1841, 95, 0, 0, 1032, 0, 0, 0, 0, - 970, 1033, 984, 985, 986, 971, 0, 0, 972, 973, - 0, 974, 0, 0, 0, 0, 0, 0, 0, 0, - 1080, 0, 0, 0, 0, 0, 0, 979, 0, 987, - 988, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 161, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2420, 2421, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3367, 3368, + 0, 61, 91, 90, 0, 0, 71, 72, 58, 0, + 0, 1841, 0, 0, 80, 81, 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, 0, + 161, 0, 0, 0, 0, 0, 0, 0, 63, 64, + 0, 65, 66, 67, 68, 0, 2420, 2421, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2459, 0, 0, 0, 0, 0, 0, 0, - 989, 990, 991, 992, 993, 994, 995, 996, 997, 998, - 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, - 1009, 1010, 1011, 1012, 1013, 1014, 1015, 1016, 1017, 1018, - 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, 1027, 1028, - 1029, 1030, 0, 0, 0, 0, 0, 0, 197, 0, - 197, 0, 0, 0, 2502, 0, 0, 0, 1828, 154, - 0, 0, 155, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3369, 0, 0, 0, 742, 0, 742, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 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, 197, 0, + 197, 0, 0, 0, 2502, 0, 0, 0, 0, 154, + 0, 0, 155, 0, 0, 0, 0, 0, 1828, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 742, 0, 742, 742, 0, 0, 167, 0, 0, 0, 0, 0, 0, 179, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 742, 197, 0, 0, 0, 0, 0, 0, 0, + 0, 742, 197, 0, 0, 89, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1537, 187, 1842, 0, 0, 0, 0, 0, 0, 0, + 1537, 187, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1842, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3370, 3371, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 168, 173, 170, 176, 177, 178, 180, 182, 183, 184, 185, 0, - 0, 0, 0, 0, 186, 188, 189, 190, 1855, 1858, - 1859, 1860, 1861, 1862, 1863, 0, 1864, 1865, 1867, 1868, + 0, 0, 0, 0, 186, 188, 189, 190, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1855, 1858, + 1859, 1860, 1861, 1862, 1863, 94, 1864, 1865, 1867, 1868, 1866, 1869, 1870, 1843, 1844, 1845, 1846, 1826, 1827, 1856, 0, 1829, 0, 1830, 1831, 1832, 1833, 1834, 1835, 1836, - 1837, 1838, 0, 0, 1839, 1847, 1848, 1849, 1850, 935, - 1851, 1852, 1853, 1854, 0, 939, 1840, 0, 0, 936, - 937, 0, 0, 0, 938, 940, 0, 0, 0, 0, + 1837, 1838, 0, 0, 1839, 1847, 1848, 1849, 1850, 0, + 1851, 1852, 1853, 1854, 0, 0, 1840, 0, 0, 0, 0, 741, 1448, 741, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1536, 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, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2327,8 +2327,8 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 0, 197, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1857, 0, 742, 0, 0, 0, 0, 0, 0, - 197, 0, 0, 0, 742, 0, 0, 0, 0, 0, + 0, 0, 0, 742, 0, 0, 0, 0, 0, 0, + 197, 1857, 0, 0, 742, 0, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 742, @@ -2555,9 +2555,9 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1293, 0, 0, 0, 0, 0, 742, - 742, 742, 742, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2097, + 0, 0, 0, 1293, 0, 0, 0, 1032, 0, 742, + 742, 742, 742, 1033, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2152, 0, 0, 0, 0, 0, 2097, 0, 0, 0, 0, 0, 0, 2592, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2613, 2614, 0, 0, 2618, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2565,36 +2565,36 @@ var yyAct = [...]int{ 2626, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2629, 0, 0, 0, - 0, 3605, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 741, 741, 0, 0, 0, 0, + 0, 3605, 989, 990, 991, 992, 993, 994, 995, 996, + 997, 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, + 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014, 1015, 1016, + 1017, 1018, 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, + 1027, 1028, 1029, 1030, 741, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1293, 1293, 0, 742, 0, 742, 0, 197, 0, 0, 0, 0, 0, 0, 0, 2085, 0, 0, 0, 0, 741, 741, 741, 741, 0, 0, 1537, 0, 0, 0, 197, 0, 0, 742, 0, 742, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2147, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 95, 0, 0, 1032, 0, 0, 0, + 0, 970, 1033, 984, 985, 986, 971, 0, 0, 972, + 973, 0, 974, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 979, 2147, + 987, 988, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 197, 0, 0, 742, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 742, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 742, 0, 0, 3367, + 3368, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 1008, 1009, 1010, 1011, 1012, 1013, 1014, 1015, 1016, 1017, + 1018, 1019, 1020, 1021, 1022, 1023, 1024, 1025, 1026, 1027, + 1028, 1029, 1030, 0, 0, 0, 0, 0, 0, 0, 741, 0, 741, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 0, 0, 0, 0, 0, 742, 0, 742, 0, 0, 1536, 0, 0, 0, 0, 0, 0, - 741, 0, 741, 0, 0, 0, 0, 0, 0, 0, + 741, 0, 741, 0, 3369, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 742, 0, 0, 0, 1293, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2602,7 +2602,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2395, 0, 0, 0, 741, 0, 0, - 0, 2936, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 2936, 0, 0, 0, 3370, 3371, 0, 0, 0, 741, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2408, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2611,8 +2611,8 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1105, 0, 0, 0, 2985, 0, 0, 742, 0, 0, 0, 0, 0, 741, 0, 0, 0, 0, - 0, 0, 741, 0, 741, 197, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 935, 0, 741, 0, 741, 197, 939, 0, 0, 0, + 936, 937, 0, 0, 0, 938, 940, 0, 0, 0, 0, 0, 0, 742, 197, 0, 0, 0, 0, 0, 0, 0, 0, 741, 0, 0, 0, 0, 0, 0, 1112, 0, 0, 0, 0, 0, 0, 2523, 2524, 2525, @@ -7331,26 +7331,26 @@ var yyAct = [...]int{ } var yyPact = [...]int{ - -1000, -1000, 4559, -1000, -555, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 5368, -1000, -555, -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, 2542, 2773, -1000, -1000, -1000, -1000, 2685, -1000, 999, - 2121, -1000, 2449, 4217, -1000, 55454, 473, -1000, 52470, -428, - 862, 234, 36058, -1000, 199, -1000, 189, 53962, 200, -1000, - -1000, -1000, -1000, -428, 21884, 2358, 54, 52, 55454, -1000, - -1000, -1000, -1000, -362, 2637, 2002, -1000, 386, -1000, -1000, - -1000, -1000, -1000, -1000, 51724, -1000, -1000, -1000, 1103, -1000, - -1000, 2458, 2456, 2291, 889, 2354, -1000, 2532, 2002, -1000, - 21884, 2621, 2398, 21138, 21138, 446, -1000, -1000, 336, -1000, - -1000, 30836, 55454, 39042, 913, -1000, 2449, -1000, -1000, -1000, - 193, -1000, 340, 2041, -1000, 2040, -1000, 864, 1083, 377, - 863, 487, 374, 363, 361, 351, 346, 344, 343, 342, - 392, -1000, 919, 919, -214, -221, 1470, 431, 434, 434, - 1044, 456, 2436, 2428, -1000, -1000, 919, 919, 919, 365, - 919, 919, 919, 919, 315, 310, 919, 919, 919, 919, - 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, - 919, 919, 919, 884, 2449, 274, -1000, -1000, -1000, -1000, + -1000, 2460, 2567, -1000, -1000, -1000, -1000, 2670, -1000, 1002, + 2064, -1000, 2461, 4217, -1000, 55454, 780, -1000, 52470, -428, + 882, 279, 36058, -1000, 238, -1000, 212, 53962, 221, -1000, + -1000, -1000, -1000, -428, 21884, 2339, 35, 34, 55454, -1000, + -1000, -1000, -1000, -373, 2637, 1997, -1000, 458, -1000, -1000, + -1000, -1000, -1000, -1000, 51724, -1000, -1000, -1000, 1114, -1000, + -1000, 2468, 2485, 2357, 915, 2382, -1000, 2563, 1997, -1000, + 21884, 2621, 2399, 21138, 21138, 468, -1000, -1000, 299, -1000, + -1000, 30836, 55454, 39042, 861, -1000, 2461, -1000, -1000, -1000, + 202, -1000, 379, 1981, -1000, 1978, -1000, 1145, 966, 404, + 844, 843, 399, 391, 389, 388, 387, 386, 385, 381, + 413, -1000, 941, 941, -221, -226, 1553, 467, 457, 457, + 1022, 488, 2435, 2432, -1000, -1000, 941, 941, 941, 371, + 941, 941, 941, 941, 357, 341, 941, 941, 941, 941, + 941, 941, 941, 941, 941, 941, 941, 941, 941, 941, + 941, 941, 941, 929, 2461, 326, -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, @@ -7399,67 +7399,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, - 55454, 224, 55454, -1000, 813, 472, -1000, -1000, -453, 1084, - 1084, 74, 1084, 1084, 1084, 1084, 201, 1031, 51, -1000, - 198, 271, 196, 268, 1069, 210, -1000, -1000, 254, 1069, - 1875, -1000, 894, 264, 203, -1000, 1084, 1084, -1000, 14400, - 229, 14400, 14400, 259, 180, -1000, 2437, -1000, -1000, -1000, - -1000, -1000, 1319, -1000, -1000, -1000, -1000, -32, 455, -1000, - -1000, -1000, -1000, 53962, 50978, 278, -1000, -1000, 28, 1919, - 1722, 21884, 1323, 887, -1000, -1000, 1343, 870, -1000, -1000, - -1000, -1000, -1000, 512, -1000, 24122, 24122, 24122, 24122, -1000, - -1000, 2003, 50232, 2003, 2003, 24122, 2003, 24122, 2003, 2003, - 2003, 2003, 21884, 2003, 2003, 2003, 2003, -1000, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, -1000, -1000, -1000, - -1000, 2003, 809, 2003, 2003, 2003, 2003, 2003, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 2003, 2003, 2003, 2003, 2003, - 2003, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, -1000, -1000, -1000, 1672, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 1448, 1542, 1525, 1512, -1000, + 55454, 293, 55454, -1000, 833, 741, -1000, -1000, -435, 1102, + 1102, 53, 1102, 1102, 1102, 1102, 237, 1020, 29, -1000, + 236, 321, 225, 324, 1074, 722, -1000, -1000, 315, 1074, + 1858, -1000, 922, 323, 147, -1000, 1102, 1102, -1000, 14400, + 206, 14400, 14400, 304, 198, -1000, 2449, -1000, -1000, -1000, + -1000, -1000, 1350, -1000, -1000, -1000, -1000, -36, 487, -1000, + -1000, -1000, -1000, 53962, 50978, 317, -1000, -1000, 36, 1866, + 1163, 21884, 1595, 913, -1000, -1000, 1343, 887, -1000, -1000, + -1000, -1000, -1000, 824, -1000, 24122, 24122, 24122, 24122, -1000, + -1000, 1985, 50232, 1985, 1985, 24122, 1985, 24122, 1985, 1985, + 1985, 1985, 21884, 1985, 1985, 1985, 1985, -1000, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, -1000, -1000, -1000, + -1000, 1985, 831, 1985, 1985, 1985, 1985, 1985, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 1985, 1985, 1985, 1985, 1985, + 1985, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, -1000, -1000, -1000, 1634, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 2287, 1530, 1514, 1512, -1000, 18900, -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, 2003, -1000, -1000, -1000, 55454, -1000, 2003, 221, 53962, - 53962, 383, 1313, -1000, -1000, 2532, 2002, -1000, 2637, 2676, - 386, -1000, 3945, 1695, 1638, 1546, 2002, 2010, 55454, -1000, - 2050, 211, -1000, -1000, -1000, -332, -347, 2262, 1532, 1870, - -1000, -1000, -1000, -1000, 1841, 21884, -1000, -1000, 2681, -1000, - 28598, 803, 2677, 49486, -1000, 446, 446, 2032, 430, -11, - -1000, -1000, -1000, -1000, 956, 35312, -1000, -1000, -1000, -1000, - -1000, 1894, 55454, -1000, -1000, 5606, 53962, -1000, 2116, -1000, - 1887, -1000, 2073, 21884, 2134, 471, 53962, 462, 461, 460, - 443, -67, -1000, -1000, -1000, -1000, -1000, -1000, 919, 919, - 919, -1000, 388, 2620, 4217, 6836, -1000, -1000, -1000, 48740, - 2113, 53962, -1000, 2112, -1000, 1026, 868, 869, 869, 53962, - -1000, -1000, 54708, 53962, 1022, 1020, 53962, 53962, 53962, 53962, - -1000, 47994, -1000, 47248, 46502, 1311, 53962, 45756, 45010, 44264, - 43518, 42772, -1000, 2364, -1000, 2098, -1000, -1000, -1000, 54708, - 53962, 53962, 54708, 53962, 54708, 55454, 53962, -1000, -1000, 368, - -1000, -1000, 1305, 1304, 1297, 919, 919, 1268, 1859, 1856, - 1849, 919, 919, 1266, 1827, 37550, 1817, 276, 1262, 1261, - 1259, 1312, 1815, 269, 1805, 1296, 1254, 1255, 53962, 2100, - 55454, -1000, 249, 985, 995, 955, 2449, 2346, 2027, 452, - 468, 53962, 439, 439, 53962, -1000, 15152, 55454, 250, -1000, - 1791, 21884, -1000, 1079, 1069, 1069, -1000, -1000, -1000, -1000, - -1000, -1000, 1084, 55454, 1079, -1000, -1000, -1000, 1069, 1084, - 55454, 1084, 1084, 1084, 1084, 1069, 1069, 1069, 1084, 55454, - 55454, 55454, 55454, 55454, 55454, 55454, 55454, 55454, 14400, 894, - 1084, -454, -1000, 1752, -1000, -1000, -1000, 2240, -1000, -1000, + -1000, 1985, -1000, -1000, -1000, 55454, -1000, 1985, 258, 53962, + 53962, 373, 1341, -1000, -1000, 2563, 1997, -1000, 2637, 2585, + 458, -1000, 4753, 1568, 1575, 1444, 1997, 1953, 55454, -1000, + 2001, 194, -1000, -1000, -1000, -354, -356, 2174, 1484, 1845, + -1000, -1000, -1000, -1000, 2231, 21884, -1000, -1000, 2667, -1000, + 28598, 830, 2664, 49486, -1000, 468, 468, 1975, 640, 0, + -1000, -1000, -1000, -1000, 960, 35312, -1000, -1000, -1000, -1000, + -1000, 1853, 55454, -1000, -1000, 5616, 53962, -1000, 2063, -1000, + 1844, -1000, 2018, 21884, 2077, 738, 53962, 511, 502, 494, + 474, -68, -1000, -1000, -1000, -1000, -1000, -1000, 941, 941, + 941, -1000, 411, 2614, 4217, 6836, -1000, -1000, -1000, 48740, + 2060, 53962, -1000, 2055, -1000, 1035, 877, 866, 866, 53962, + -1000, -1000, 54708, 53962, 1029, 1027, 53962, 53962, 53962, 53962, + -1000, 47994, -1000, 47248, 46502, 1325, 53962, 45756, 45010, 44264, + 43518, 42772, -1000, 2543, -1000, 2061, -1000, -1000, -1000, 54708, + 53962, 53962, 54708, 53962, 54708, 55454, 53962, -1000, -1000, 319, + -1000, -1000, 1319, 1318, 1316, 941, 941, 1315, 1830, 1826, + 1815, 941, 941, 1292, 1814, 37550, 1810, 342, 1269, 1268, + 1266, 1320, 1801, 205, 1787, 1260, 1194, 1265, 53962, 2053, + 55454, -1000, 310, 963, 1012, 959, 2461, 2318, 1969, 486, + 535, 53962, 466, 466, 53962, -1000, 15152, 55454, 209, -1000, + 1783, 21884, -1000, 1121, 1074, 1074, -1000, -1000, -1000, -1000, + -1000, -1000, 1102, 55454, 1121, -1000, -1000, -1000, 1074, 1102, + 55454, 1102, 1102, 1102, 1102, 1074, 1074, 1074, 1102, 55454, + 55454, 55454, 55454, 55454, 55454, 55454, 55454, 55454, 14400, 922, + 1102, -453, -1000, 1772, -1000, -1000, -1000, 2222, -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, @@ -7476,21 +7476,21 @@ 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, 14400, 14400, - -1000, -1000, 2555, 2545, -1000, -1000, -1000, 2026, -1000, 187, - 15, 197, -1000, 42026, 507, 937, -1000, 507, -1000, -1000, - -1000, -1000, -1000, 2017, 41280, -1000, -459, -463, -465, -466, - -1000, -1000, -1000, -467, -472, -1000, -1000, -1000, 21884, 21884, - 21884, 21884, -251, -1000, 1530, 24122, 2419, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 21884, 1177, 941, 24122, 24122, 24122, + -1000, -1000, 2555, 2542, -1000, -1000, -1000, 1967, -1000, 203, + -5, 207, -1000, 42026, 565, 949, -1000, 565, -1000, -1000, + -1000, -1000, -1000, 1966, 41280, -1000, -454, -459, -465, -466, + -1000, -1000, -1000, -470, -472, -1000, -1000, -1000, 21884, 21884, + 21884, 21884, -257, -1000, 1271, 24122, 2413, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 21884, 1178, 1402, 24122, 24122, 24122, 24122, 24122, 24122, 24122, 25614, 24868, 24122, 24122, 24122, 24122, - 24122, 24122, -1000, -1000, 33074, 4706, 4706, 870, 870, 870, - 870, -1000, -178, 2016, 54708, -1000, -1000, -1000, 800, 21884, - 21884, 870, -1000, 1334, 4398, 18900, 21884, 21884, 21884, 21884, - 961, 1722, 54708, 21884, -1000, 1546, -1000, -1000, -1000, -1000, - 1192, -1000, -1000, 1139, 2404, 2404, 2404, 2404, 21884, 21884, - 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 2404, 21884, - 725, 725, 917, 21884, 21884, 21884, 21884, 21884, 21884, 21884, - 21884, 17408, 21884, 21884, 24122, 21884, 21884, 21884, 1546, 21884, + 24122, 24122, -1000, -1000, 33074, 8389, 8389, 887, 887, 887, + 887, -1000, -180, 1965, 54708, -1000, -1000, -1000, 829, 21884, + 21884, 887, -1000, 1364, 4110, 18900, 21884, 21884, 21884, 21884, + 981, 1163, 54708, 21884, -1000, 1444, -1000, -1000, -1000, -1000, + 1246, -1000, -1000, 1060, 2409, 2409, 2409, 2409, 21884, 21884, + 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 2409, 21884, + 715, 715, 917, 21884, 21884, 21884, 21884, 21884, 21884, 21884, + 21884, 17408, 21884, 21884, 24122, 21884, 21884, 21884, 1444, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, @@ -7498,111 +7498,111 @@ var yyPact = [...]int{ 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, - 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 1546, 21884, - 1244, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 16656, 21884, + 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 1444, 21884, + 1239, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 16656, 21884, 21884, 21884, 21884, 21884, -1000, -1000, -1000, -1000, -1000, -1000, - 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 1546, 21884, + 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 1444, 21884, 21884, 21884, 21884, 21884, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 1668, 1750, 1572, 2003, 21884, -1000, - 2011, -1000, -195, 30090, 21884, 1748, 2675, 2157, 53962, -1000, - -1000, -1000, -1000, 2532, -1000, 2532, 1668, 3937, 2278, 21138, - -1000, -1000, 3937, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 1775, -1000, 55454, 2010, - 2527, 53962, -1000, 1998, -1000, -1000, 2003, -1000, -306, -1000, - -309, 2232, 1743, 832, -1000, 21884, 21884, 1997, -1000, 1298, - 55454, -1000, -251, -1000, 40534, -1000, -1000, 13648, 55454, 338, - 55454, -1000, 29344, 39788, 307, -1000, -11, 1972, -1000, 26, - -3, 18154, 867, -1000, -1000, -1000, 1470, 26360, 1865, 867, - 113, -1000, -1000, -1000, 2073, -1000, 2073, 2073, 2073, 2073, - 832, 832, 832, 832, -1000, -1000, -1000, -1000, -1000, 2099, - 2073, 2097, -1000, 2073, 2073, 2073, 2073, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 1760, 1600, 1544, 1985, 21884, -1000, + 1954, -1000, -189, 30090, 21884, 1767, 2661, 2102, 53962, -1000, + -1000, -1000, -1000, 2563, -1000, 2563, 1760, 3945, 2146, 21138, + -1000, -1000, 3945, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 1840, -1000, 55454, 1953, + 2519, 53962, -1000, 1950, -1000, -1000, 1985, -1000, -297, -1000, + -304, 2236, 1747, 936, -1000, 21884, 21884, 1937, -1000, 1639, + 55454, -1000, -257, -1000, 40534, -1000, -1000, 13648, 55454, 378, + 55454, -1000, 29344, 39788, 308, -1000, 0, 1906, -1000, 7, + -11, 18154, 885, -1000, -1000, -1000, 1553, 26360, 1856, 885, + 78, -1000, -1000, -1000, 2018, -1000, 2018, 2018, 2018, 2018, + 936, 936, 936, 936, -1000, -1000, -1000, -1000, -1000, 2052, + 2018, 2051, -1000, 2018, 2018, 2018, 2018, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 2093, 2093, 2093, 2088, 2088, 2076, - 2076, 432, -1000, 21884, 348, 39042, 2495, 1249, 2137, 249, - 443, 2146, 53962, 53962, 53962, 55454, 982, -1000, 1477, 1435, - 1401, -1000, -539, 1992, -1000, -1000, 2613, -1000, -1000, 1101, - 1043, 1041, 978, 53962, 227, 328, -1000, 417, -1000, 39042, - 53962, 1019, 869, 53962, -1000, 53962, -1000, -1000, -1000, -1000, - -1000, 53962, -1000, -1000, 1990, -1000, 1945, 1154, 1040, 1104, - 1011, 1990, -1000, -1000, -184, 1990, -1000, 1990, -1000, 1990, - -1000, 1990, -1000, 1990, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 993, 309, -346, 53962, 227, 450, -1000, - 449, 33074, -1000, -1000, -1000, 33074, 33074, -1000, -1000, -1000, - -1000, 1732, 1711, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 2044, 2044, 2044, 2042, 2042, 2022, + 2022, 452, -1000, 21884, 448, 39042, 2495, 1259, 2013, 310, + 474, 2096, 53962, 53962, 53962, 55454, 948, -1000, 1459, 1456, + 1437, -1000, -544, 1935, -1000, -1000, 2613, -1000, -1000, 1098, + 1077, 1072, 1158, 53962, 269, 376, -1000, 449, -1000, 39042, + 53962, 1026, 866, 53962, -1000, 53962, -1000, -1000, -1000, -1000, + -1000, 53962, -1000, -1000, 1929, -1000, 1946, 1156, 1071, 1108, + 1067, 1929, -1000, -1000, -186, 1929, -1000, 1929, -1000, 1929, + -1000, 1929, -1000, 1929, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1019, 298, -349, 53962, 269, 485, -1000, + 478, 33074, -1000, -1000, -1000, 33074, 33074, -1000, -1000, -1000, + -1000, 1743, 1728, -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, -522, 55454, -1000, 241, 936, 294, 314, 308, - 55454, 787, 2397, 2395, 2387, 2382, 2379, 2377, 2376, 251, - 298, 55454, 55454, 439, 2194, 55454, 2489, 55454, -1000, -1000, - -1000, -1000, -1000, 1697, 1689, -1000, 1722, 55454, -1000, -1000, - 1084, 1084, -1000, -1000, 55454, 1084, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 1084, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -527, 55454, -1000, 303, 946, 344, 347, 349, + 55454, 327, 2425, 2421, 2420, 2419, 2408, 2398, 2386, 306, + 340, 55454, 55454, 466, 2145, 55454, 2498, 55454, -1000, -1000, + -1000, -1000, -1000, 1689, 1671, -1000, 1163, 55454, -1000, -1000, + 1102, 1102, -1000, -1000, 55454, 1102, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 1102, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 55454, -1000, -1000, -1000, -1000, 1681, -1000, 55454, -32, 185, + 55454, -1000, -1000, -1000, -1000, 1654, -1000, 55454, -36, 201, -1000, -1000, 53962, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -83, -1000, 39, 21, 373, -1000, -1000, -1000, - -1000, -1000, 2534, -1000, 1722, 975, 992, -1000, 2003, -1000, - -1000, 1274, -1000, -1000, -1000, -1000, -1000, 2003, 2003, 2003, - -1000, -1000, -1000, -1000, -1000, 1177, 24122, 24122, 24122, 1410, - 492, 1433, 1327, 1420, 1276, 1276, 1287, 24122, 1287, 24122, - 871, 871, 871, 871, 871, -1000, -1000, -1000, -1000, -1000, - -1000, 1672, -1000, 1627, -1000, 2003, 54708, 1874, 16656, 1904, - 1540, 1546, 879, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -86, -1000, 45, 5, 427, -1000, -1000, -1000, + -1000, -1000, 2536, -1000, 1163, 1014, 996, -1000, 1985, -1000, + -1000, 1352, -1000, -1000, -1000, -1000, -1000, 1985, 1985, 1985, + -1000, -1000, -1000, -1000, -1000, 1178, 24122, 24122, 24122, 1435, + 804, 1496, 1291, 1326, 1363, 1363, 1308, 24122, 1308, 24122, + 891, 891, 891, 891, 891, -1000, -1000, -1000, -1000, -1000, + -1000, 1634, -1000, 1629, -1000, 1985, 54708, 1809, 16656, 2099, + 2810, 1444, 908, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 3614, 1546, 1919, 1546, 1671, 3609, 979, - -1000, 21884, 1546, 3598, -1000, -1000, 1546, 1546, 21884, -1000, - -1000, 21884, 21884, 21884, 21884, 2137, 2137, 2137, 2137, 2137, - 2137, 2137, 2137, 2137, 2137, 21884, 2137, 1989, -1000, -1000, + -1000, -1000, -1000, 4439, 1444, 1866, 1444, 1911, 4435, 985, + -1000, 21884, 1444, 4390, -1000, -1000, 1444, 1444, 21884, -1000, + -1000, 21884, 21884, 21884, 21884, 2013, 2013, 2013, 2013, 2013, + 2013, 2013, 2013, 2013, 2013, 21884, 2013, 1928, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 1985, 2672, 1329, 2137, - 2137, 2137, 2137, 3587, 2137, 2137, 21884, 1974, -1000, -1000, - -1000, 1569, 3558, 1257, 3551, 2137, 2137, -1000, 2137, 3515, - 3485, 1546, 2662, 2630, 2137, 2137, 2137, 2137, 2137, 2615, - 2585, 2137, 2137, 2581, 2137, 3480, 2137, 2575, 2568, 2554, - 2550, 2520, 2473, 2465, 2455, 2451, 2411, 2388, 2367, 2361, - 2350, 2338, 2333, 2315, 2287, 2137, 2137, 2137, 3464, 2137, - 3448, 2137, 3444, 2137, 2137, 3439, 2271, 2248, 1546, 1973, - -1000, 3429, 2137, 3413, 3386, 3381, 2244, 3371, 3362, 3358, - 2137, 2137, 2137, 2238, 3350, 3346, 3342, 3306, 3115, 3106, - 3102, 3097, 3083, 2137, 1572, 1572, 1572, 1572, 1572, 3050, - -258, 2137, 1546, -1000, -1000, -1000, -1000, -1000, 3009, 2225, - 2994, 2986, 2976, 2971, 1546, 2003, 797, -1000, -1000, 1572, - 1546, 1546, 1572, 1572, 2960, 2941, 2892, 2844, 2816, 2812, - 2137, 2137, -1000, 2137, 2804, 2772, 2206, 2177, 1546, -1000, - 1572, 55454, -1000, -425, -1000, 5, 960, 2003, -1000, 37550, - 1546, -1000, 4201, -1000, 1247, -1000, -1000, -1000, -1000, -1000, - 34566, 2018, -1000, -1000, -1000, -1000, 2003, 1864, -1000, -496, - 20392, -1000, -1000, -1000, 832, 94, 33820, 860, 860, 124, - 1722, 1722, 21884, -1000, -1000, -1000, -1000, -1000, -1000, 552, - 2639, 376, 2003, -1000, 1951, 2420, -1000, -1000, -1000, 2507, - 27106, -1000, -1000, 2003, 2003, 55454, 1893, 1873, -1000, 535, - -1000, 1352, 1972, -11, -12, -1000, -1000, -1000, -1000, 1722, - -1000, 1398, 341, 347, -1000, 424, -1000, -1000, -1000, -1000, - 2368, 106, -1000, -1000, -1000, 345, 832, -1000, -1000, -1000, - -1000, -1000, -1000, 1622, -1000, 1622, -1000, -1000, -1000, -1000, - -1000, 1248, -1000, -1000, -1000, -1000, 1239, -1000, -1000, 1231, - -1000, -1000, 2670, 2181, 348, -1000, -1000, 919, 1620, -1000, - -1000, 2371, 919, 919, 53962, -1000, -1000, 1810, 2495, 241, - 55454, 2191, -1000, 2146, 2146, 2146, -1000, 2482, -1000, -1000, - -1000, -1000, -1000, -1000, -526, 166, 548, -1000, -1000, -1000, - 5202, 53962, 1809, -1000, 213, -1000, 1804, -1000, 53962, -1000, - 1803, 2085, 53962, 53962, -1000, -1000, -1000, 53962, 2003, -1000, - -1000, -1000, -1000, 466, 2447, 367, -1000, -1000, -282, -1000, - -1000, 227, 213, 54708, 53962, 867, -1000, -1000, -1000, -1000, - -1000, -529, 1788, 458, 232, 329, 55454, 55454, 55454, 55454, - 55454, 55454, 485, -1000, -1000, 40, -1000, -1000, 212, -1000, - -1000, 1609, -1000, -1000, -1000, -1000, 212, -1000, -1000, -1000, - -1000, -1000, 277, 448, -1000, 55454, 55454, 952, -1000, -1000, - -1000, -1000, -1000, 1069, -1000, -1000, 1069, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2443, - 55454, 20, -492, -1000, -488, 21884, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 1243, 486, 1433, 24122, 24122, 4398, 4398, - 24122, -1000, -1000, -1000, 323, 323, 33074, -1000, 24122, 21884, - -1000, -1000, 21884, 21884, 21884, 931, -1000, 21884, 1392, -1000, - 21884, -1000, -258, 1572, 2137, 2137, 2137, 2137, -258, -258, - -258, -258, -258, -258, -258, -258, -258, -258, 1917, -1000, - 21884, 21884, 21884, 1546, 326, -1000, -1000, -1000, -258, 21884, - -1000, -1000, 2649, -1000, 21884, -1000, 33074, 21884, 21884, 21884, + -1000, -1000, -1000, -1000, -1000, -1000, 1922, 2656, 1973, 2013, + 2013, 2013, 2013, 4386, 2013, 2013, 21884, 1842, -1000, -1000, + -1000, 1476, 4381, 1257, 4377, 2013, 2013, -1000, 2013, 4367, + 4358, 1444, 2804, 2755, 2013, 2013, 2013, 2013, 2013, 2723, + 2707, 2013, 2013, 2680, 2013, 4353, 2013, 2671, 2660, 2641, + 2580, 2575, 2569, 2559, 2554, 2550, 2546, 2525, 2473, 2464, + 2455, 2448, 2411, 2367, 2361, 2013, 2013, 2013, 4295, 2013, + 4227, 2013, 4222, 2013, 2013, 3977, 2350, 2346, 1444, 1919, + -1000, 3960, 2013, 3934, 3630, 3614, 2338, 3598, 3587, 3558, + 2013, 2013, 2013, 2332, 3552, 3515, 3494, 3489, 3485, 3480, + 3464, 3448, 3444, 2013, 1544, 1544, 1544, 1544, 1544, 3439, + -260, 2013, 1444, -1000, -1000, -1000, -1000, -1000, 3413, 2326, + 3386, 3381, 3371, 3362, 1444, 1985, 825, -1000, -1000, 1544, + 1444, 1444, 1544, 1544, 3358, 3350, 3346, 3306, 3112, 3102, + 2013, 2013, -1000, 2013, 3097, 3049, 2316, 2295, 1444, -1000, + 1544, 55454, -1000, -425, -1000, 3, 945, 1985, -1000, 37550, + 1444, -1000, 4202, -1000, 1295, -1000, -1000, -1000, -1000, -1000, + 34566, 1835, -1000, -1000, -1000, -1000, 1985, 1806, -1000, -495, + 20392, -1000, -1000, -1000, 936, 55, 33820, 881, 881, 86, + 1163, 1163, 21884, -1000, -1000, -1000, -1000, -1000, -1000, 820, + 2644, 515, 1985, -1000, 1955, 2299, -1000, -1000, -1000, 2518, + 27106, -1000, -1000, 1985, 1985, 55454, 1793, 1697, -1000, 813, + -1000, 1386, 1906, 0, -14, -1000, -1000, -1000, -1000, 1163, + -1000, 1434, 380, 1470, -1000, 451, -1000, -1000, -1000, -1000, + 2347, 70, -1000, -1000, -1000, 382, 936, -1000, -1000, -1000, + -1000, -1000, -1000, 1627, -1000, 1627, -1000, -1000, -1000, -1000, + -1000, 1258, -1000, -1000, -1000, -1000, 1256, -1000, -1000, 1252, + -1000, -1000, 2944, 2126, 448, -1000, -1000, 941, 1620, -1000, + -1000, 2359, 941, 941, 53962, -1000, -1000, 1739, 2495, 303, + 55454, 2144, -1000, 2096, 2096, 2096, -1000, 2483, -1000, -1000, + -1000, -1000, -1000, -1000, -526, 168, 367, -1000, -1000, -1000, + 5202, 53962, 1759, -1000, 261, -1000, 1709, -1000, 53962, -1000, + 1755, 2030, 53962, 53962, -1000, -1000, -1000, 53962, 1985, -1000, + -1000, -1000, -1000, 516, 2456, 284, -1000, -1000, -287, -1000, + -1000, 269, 261, 54708, 53962, 885, -1000, -1000, -1000, -1000, + -1000, -530, 1738, 492, 282, 329, 55454, 55454, 55454, 55454, + 55454, 55454, 797, -1000, -1000, 17, -1000, -1000, 242, -1000, + -1000, 1612, -1000, -1000, -1000, -1000, 242, -1000, -1000, -1000, + -1000, -1000, 332, 477, -1000, 55454, 55454, 957, -1000, -1000, + -1000, -1000, -1000, 1074, -1000, -1000, 1074, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2453, + 55454, 1, -492, -1000, -489, 21884, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1275, 787, 1496, 24122, 24122, 4110, 4110, + 24122, -1000, -1000, -1000, 951, 951, 33074, -1000, 24122, 21884, + -1000, -1000, 21884, 21884, 21884, 952, -1000, 21884, 1443, -1000, + 21884, -1000, -260, 1544, 2013, 2013, 2013, 2013, -260, -260, + -260, -260, -260, -260, -260, -260, -260, -260, 1948, -1000, + 21884, 21884, 21884, 1444, 351, -1000, -1000, -1000, -260, 21884, + -1000, -1000, 2655, -1000, 21884, -1000, 33074, 21884, 21884, 21884, -1000, -1000, -1000, 21884, 21884, -1000, -1000, 21884, -1000, 21884, -1000, -1000, -1000, -1000, -1000, -1000, 21884, -1000, 21884, -1000, -1000, -1000, 21884, -1000, 21884, -1000, -1000, 21884, -1000, 21884, @@ -7614,196 +7614,196 @@ var yyPact = [...]int{ -1000, 21884, 21884, 21884, -1000, 21884, 21884, 21884, 21884, -1000, -1000, -1000, -1000, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, 21884, -1000, -1000, -1000, -1000, -1000, -1000, 21884, - -1000, 39042, 38, -258, 1244, 38, 1244, 23376, 815, 493, + -1000, 39042, 54, -260, 1239, 54, 1239, 23376, 836, 835, 22630, -1000, 21884, 15904, -1000, -1000, -1000, -1000, -1000, 21884, 21884, 21884, 21884, 21884, 21884, -1000, -1000, -1000, 21884, 21884, - -1000, 21884, -1000, 21884, -1000, -1000, -1000, -1000, -1000, 960, - -1000, 453, 445, 869, 53962, -1000, -1000, -1000, -1000, 1971, - -1000, 2558, -1000, 2289, 2268, 2645, 2639, 21138, -1000, 29344, - -1000, -1000, 53962, -1000, -1000, -410, -1000, 2343, 2295, 860, - 860, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 12896, 2532, - 21884, 2184, 54708, 2003, -1000, 27852, 53962, 54708, 29344, 29344, - 29344, 29344, 29344, -1000, 2214, 2210, -1000, 2224, 2208, 2312, - 55454, -1000, 1668, 1785, -1000, 21884, 31582, 1921, 29344, -1000, - -1000, 29344, 55454, 12144, -1000, -1000, 19, -8, -1000, -1000, - -1000, -1000, 1470, -1000, -1000, 1144, 2504, 2366, -1000, -1000, - -1000, -1000, -1000, 1783, -1000, 1773, 1970, 1767, 1763, 309, - -1000, 2132, 2439, 919, 919, -1000, 1229, -1000, 1334, 1599, - 1596, -1000, -1000, -1000, 447, -1000, 55454, 2178, 2174, 2172, - -1000, -546, 1227, 2084, 2129, 21884, 2079, 2608, 1950, 53962, - -1000, -1000, 54708, -1000, 296, -1000, 348, 53962, -1000, -1000, - -1000, 328, 55454, -1000, 5547, -1000, -1000, -1000, 213, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 55454, 240, -1000, 2077, - 1318, -1000, -1000, 2135, -1000, -1000, -1000, -1000, -1000, 216, - 214, 1593, 205, 1584, -1000, 205, -1000, 55454, 909, 2181, - 55454, -1000, -1000, -1000, 1084, 1084, -1000, -1000, 2413, -1000, - 1334, 2137, 24122, 24122, -1000, 870, -1000, -1000, 400, -233, - 2073, 2073, -1000, 2073, 2076, -1000, 2073, 173, 2073, 158, - 2073, -1000, -1000, 1546, 1546, -1000, 1572, 2171, 2183, 2710, - -1000, 1722, 21884, 2704, -1000, -1000, -258, -258, -258, -258, - -258, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -75, 2700, 2626, 2137, -1000, 2071, 2067, -1000, 2137, - 21884, 2137, 1546, 2153, 2137, 2137, 2137, 2137, 2137, 2137, - 2137, 2137, 2137, 2137, 2137, 2137, 2142, 2131, 2080, 2029, - 2014, 2008, 1999, 1995, 1978, 1947, 1927, 1908, 1884, 1879, - 1853, 1845, 2137, 2137, 1832, 2137, 1800, 1790, -1000, 1722, - 1572, 2541, 1572, 2137, 2137, 2536, 292, 2137, 1758, 1758, - 1758, 1758, 1758, 1572, 1572, 1572, 1572, 2137, 53962, -1000, - -258, -1000, -1000, -330, -354, -1000, 1546, -258, 1955, 24122, - 2137, 24122, 24122, 24122, 2137, 1546, -1000, 1771, 1718, 2500, - 1684, 2137, 2469, 2137, 2137, 2137, 1663, -1000, 2531, 2003, - 2531, 2003, 2531, 1738, 1247, 55454, -1000, -1000, -1000, -1000, - 2639, 2632, -1000, 1953, -1000, 94, 581, -1000, 2316, 2295, - -1000, 2592, 2331, 2591, -1000, -1000, -1000, -1000, -1000, 1722, - -1000, 2454, 1942, -1000, 933, 1765, -1000, -1000, 20392, 1756, - 2242, 532, 1738, 1991, 2420, 2155, 2170, 2807, -1000, -1000, - -1000, -1000, 2207, -1000, 2187, -1000, -1000, 2050, -1000, 2446, - 338, 29344, 1983, 1983, -1000, 528, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 1061, 5547, 2703, -1000, 1575, -1000, 1348, - 195, 1216, -1000, -1000, 919, 919, -1000, 1018, 1017, -1000, - 55454, 2065, -1000, 832, 1573, 832, 1209, -1000, -1000, 1207, - -1000, -1000, -1000, -1000, 2024, 2286, -1000, -1000, -1000, -1000, - 55454, -1000, 55454, 55454, 55454, 2064, 2590, -1000, 21884, 2063, - 923, 2666, 53962, 53962, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 444, 919, -506, 297, 280, - 919, 919, 919, -547, -1000, -1000, 1709, 1705, -1000, -215, - -1000, 21884, -1000, -1000, -1000, -1000, -1000, 1234, 1234, 1542, - 1525, 1512, -1000, 2050, -1000, -1000, -1000, 1768, -1000, -1000, - -197, 53962, 53962, 53962, 53962, -1000, -1000, -1000, 1121, -1000, + -1000, 21884, -1000, 21884, -1000, -1000, -1000, -1000, -1000, 945, + -1000, 834, 823, 866, 53962, -1000, -1000, -1000, -1000, 1900, + -1000, 2539, -1000, 2263, 2262, 2642, 2644, 21138, -1000, 29344, + -1000, -1000, 53962, -1000, -1000, -411, -1000, 2311, 2286, 881, + 881, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 12896, 2563, + 21884, 2143, 54708, 1985, -1000, 27852, 53962, 54708, 29344, 29344, + 29344, 29344, 29344, -1000, 2198, 2189, -1000, 2249, 2173, 2292, + 55454, -1000, 1760, 1735, -1000, 21884, 31582, 1890, 29344, -1000, + -1000, 29344, 55454, 12144, -1000, -1000, -1, -17, -1000, -1000, + -1000, -1000, 1553, -1000, -1000, 950, 2513, 2378, -1000, -1000, + -1000, -1000, -1000, 1733, -1000, 1705, 1898, 1701, 1678, 298, + -1000, 2070, 2447, 941, 941, -1000, 1250, -1000, 1364, 1573, + 1566, -1000, -1000, -1000, 490, -1000, 55454, 2142, 2139, 2136, + -1000, -539, 1236, 2028, 2024, 21884, 2027, 2602, 1883, 53962, + -1000, -1000, 54708, -1000, 297, -1000, 448, 53962, -1000, -1000, + -1000, 376, 55454, -1000, 8598, -1000, -1000, -1000, 261, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 55454, 294, -1000, 2026, + 1347, -1000, -1000, 2085, -1000, -1000, -1000, -1000, -1000, 235, + 234, 1564, 239, 1557, -1000, 239, -1000, 55454, 940, 2126, + 55454, -1000, -1000, -1000, 1102, 1102, -1000, -1000, 2438, -1000, + 1364, 2013, 24122, 24122, -1000, 887, -1000, -1000, 406, -239, + 2018, 2018, -1000, 2018, 2022, -1000, 2018, 191, 2018, 188, + 2018, -1000, -1000, 1444, 1444, -1000, 1544, 2261, 2265, 3029, + -1000, 1163, 21884, 3021, -1000, -1000, -260, -260, -260, -260, + -260, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -75, 3009, 2994, 2013, -1000, 2017, 2012, -1000, 2013, + 21884, 2013, 1444, 2247, 2013, 2013, 2013, 2013, 2013, 2013, + 2013, 2013, 2013, 2013, 2013, 2013, 2241, 2179, 2166, 2153, + 2147, 2137, 2083, 2039, 2034, 2019, 2008, 1989, 1938, 1917, + 1903, 1892, 2013, 2013, 1884, 2013, 1855, 1850, -1000, 1163, + 1544, 2988, 1544, 2013, 2013, 2984, 291, 2013, 1676, 1676, + 1676, 1676, 1676, 1544, 1544, 1544, 1544, 2013, 53962, -1000, + -260, -1000, -1000, -311, -332, -1000, 1444, -260, 1895, 24122, + 2013, 24122, 24122, 24122, 2013, 1444, -1000, 1829, 1804, 2974, + 1731, 2013, 2968, 2013, 2013, 2013, 1727, -1000, 2522, 1985, + 2522, 1985, 2522, 1617, 1295, 55454, -1000, -1000, -1000, -1000, + 2644, 2638, -1000, 1891, -1000, 55, 604, -1000, 2280, 2286, + -1000, 2601, 2306, 2595, -1000, -1000, -1000, -1000, -1000, 1163, + -1000, 2465, 1876, -1000, 944, 1945, -1000, -1000, 20392, 1658, + 2242, 810, 1617, 1908, 2299, 2091, 2134, 2807, -1000, -1000, + -1000, -1000, 2182, -1000, 2167, -1000, -1000, 2001, -1000, 2893, + 378, 29344, 1872, 1872, -1000, 808, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1088, 8598, 2679, -1000, 1551, -1000, 1401, + 210, 1224, -1000, -1000, 941, 941, -1000, 1025, 1021, -1000, + 55454, 2011, -1000, 936, 1540, 936, 1215, -1000, -1000, 1210, + -1000, -1000, -1000, -1000, 2041, 2183, -1000, -1000, -1000, -1000, + 55454, -1000, 55454, 55454, 55454, 2007, 2593, -1000, 21884, 2005, + 943, 2818, 53962, 53962, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 803, 941, -507, 336, 335, + 941, 941, 941, -541, -1000, -1000, 1610, 1604, -1000, -223, + -1000, 21884, -1000, -1000, -1000, -1000, -1000, 1242, 1242, 1530, + 1514, 1512, -1000, 2001, -1000, -1000, -1000, 1594, -1000, -1000, + -195, 53962, 53962, 53962, 53962, -1000, -1000, -1000, 1131, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 870, 1546, 395, -202, 1546, -1000, -1000, 832, -1000, + -1000, 887, 1444, 355, -213, 1444, -1000, -1000, 936, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 21884, - -1000, 21884, -1000, 21884, 1722, 21884, -1000, -1000, -1000, -1000, - -1000, 2532, 1486, 21884, 21884, -1000, 1205, 1201, -258, 2137, + -1000, 21884, -1000, 21884, 1163, 21884, -1000, -1000, -1000, -1000, + -1000, 2563, 1510, 21884, 21884, -1000, 1206, 1200, -260, 2013, -1000, -1000, -1000, 21884, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, 21884, -1000, -1000, 21884, -1000, -1000, -1000, 21884, -1000, 21884, -1000, 21884, -1000, - -1000, -1000, 21884, 289, 323, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 1546, 335, -1000, -1000, - -1000, 2642, -1000, 1546, 21884, 4398, -1000, 4398, 4398, 4398, + -1000, -1000, 21884, 266, 951, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 1444, 377, -1000, -1000, + -1000, 2639, -1000, 1444, 21884, 4110, -1000, 4110, 4110, 4110, -1000, -1000, -1000, 21884, -1000, 21884, 21884, -1000, 21884, -1000, - 21884, -1000, -1000, -1000, -1000, 21884, 2003, 2434, 38296, 2003, - 38296, 2003, 31582, -1000, -1000, 2632, 2624, 2589, 2317, 2327, - 2327, 2316, -1000, 2586, 2580, -1000, 1484, 2577, 1478, 990, - -1000, 54708, 21884, -1000, 2003, 37550, -1000, 398, 53962, 2003, - 53962, -1000, 2569, -1000, -1000, 21884, 2058, -1000, 21884, -1000, - -1000, -1000, -1000, 4706, 2639, 1983, -1000, -1000, 872, -1000, + 21884, -1000, -1000, -1000, -1000, 21884, 1985, 2552, 38296, 1985, + 38296, 1985, 31582, -1000, -1000, 2638, 2633, 2592, 2290, 2301, + 2301, 2280, -1000, 2591, 2590, -1000, 1508, 2589, 1506, 1015, + -1000, 54708, 21884, -1000, 1985, 37550, -1000, 534, 53962, 1985, + 53962, -1000, 2630, -1000, -1000, 21884, 2003, -1000, 21884, -1000, + -1000, -1000, -1000, 8389, 2644, 1872, -1000, -1000, 899, -1000, 21884, -1000, 9719, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 1465, 1459, -1000, -1000, 2052, 21884, -1000, -1000, - -1000, 1759, 1751, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 2050, -1000, -1000, -1000, -1000, 328, -536, 2638, 53962, - 1200, -1000, 1701, 1950, 301, 2003, 1452, 919, 919, 919, - 1198, 1194, 37550, 1687, -1000, 53962, 399, -1000, 328, -1000, - -223, -226, 2137, -1000, -1000, 2502, -1000, -1000, 15904, -1000, - -1000, 2049, 2144, -1000, -1000, -1000, -1000, 2213, -182, -209, - -1000, -1000, 2137, 2137, 2137, 1676, 1546, -1000, 2137, 2137, - 1669, 1581, -1000, -1000, 2137, 2137, 2137, 2137, 2137, 2137, - 2137, 2137, 2137, 2137, 2137, 2137, 2137, 2137, 2137, 2137, - 2137, 2137, 2137, 2137, 1572, 1655, -1000, 289, 1546, 2169, - -1000, -1000, 4706, -1000, -1000, 2569, 2564, 38, -1000, -1000, - 226, 38, 1722, 965, 1546, 1546, 965, 1635, 2137, 1618, - 1557, 2137, 2137, 32328, -1000, 2563, 2557, 1665, -1000, -1000, - 38296, 1665, 38296, 960, 2624, -268, 21884, 21884, 2290, 1107, - -1000, -1000, -1000, -1000, 1443, 1439, -1000, 1436, -1000, 2702, - -1000, 1722, -1000, 2003, 2003, -1000, 525, 1765, -1000, 2532, - 1722, 53962, 1722, 83, 2569, -1000, 2137, -1000, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, - 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, 2003, -1000, - -1000, 53962, 2530, -1000, -1000, 2501, 1667, 164, -1000, 1553, - 1950, -1000, -1000, 930, -1000, 21884, -1000, 37550, 1405, 1385, - -1000, -1000, -1000, -1000, -547, -1000, -1000, -1000, -1000, -1000, - -1000, 386, 1949, -1000, 906, 53962, 55454, -1000, 2188, -1000, + -1000, -1000, 1504, 1499, -1000, -1000, 2002, 21884, -1000, -1000, + -1000, 1590, 1562, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 2001, -1000, -1000, -1000, -1000, 376, -534, 2538, 53962, + 1193, -1000, 1599, 1883, 370, 1985, 1483, 941, 941, 941, + 1184, 1152, 37550, 1597, -1000, 53962, 400, -1000, 376, -1000, + -229, -230, 2013, -1000, -1000, 2508, -1000, -1000, 15904, -1000, + -1000, 1993, 2095, -1000, -1000, -1000, -1000, 2190, -184, -216, + -1000, -1000, 2013, 2013, 2013, 2379, 1444, -1000, 2013, 2013, + 1556, 1545, -1000, -1000, 2013, 2013, 2013, 2013, 2013, 2013, + 2013, 2013, 2013, 2013, 2013, 2013, 2013, 2013, 2013, 2013, + 2013, 2013, 2013, 2013, 1544, 1722, -1000, 266, 1444, 2125, + -1000, -1000, 8389, -1000, -1000, 2630, 2583, 54, -1000, -1000, + 285, 54, 1163, 987, 1444, 1444, 987, 1718, 2013, 1713, + 1708, 2013, 2013, 32328, -1000, 2579, 2577, 1589, -1000, -1000, + 38296, 1589, 38296, 945, 2633, -267, 21884, 21884, 2271, 1205, + -1000, -1000, -1000, -1000, 1465, 1462, -1000, 1438, -1000, 2676, + -1000, 1163, -1000, 1985, 1985, -1000, 805, 1945, -1000, 2563, + 1163, 53962, 1163, 59, 2630, -1000, 2013, -1000, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, + 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, 1985, -1000, + -1000, 53962, 2427, -1000, -1000, 2502, 1593, 166, -1000, 1535, + 1883, -1000, -1000, 902, -1000, 21884, -1000, 37550, 1417, 1405, + -1000, -1000, -1000, -1000, -541, -1000, -1000, -1000, -1000, -1000, + -1000, 458, 1879, -1000, 937, 53962, 55454, -1000, 2154, -1000, -1000, -1000, -1000, 21884, -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, 21884, -1000, 1546, 2168, -1000, -364, -1000, -516, 21884, - -258, -1000, -1000, -258, -1000, -1000, -1000, -1000, -1000, 21884, - -1000, -1000, 21884, -1000, 21884, -1000, -1000, 1665, -1000, -1000, - -1000, 36804, -1000, 1665, -1000, 1665, -1000, -268, -1000, 1946, - -1000, 53962, 1722, 350, -1000, 1102, -1000, -1000, -1000, -1000, - -1000, 54708, 53962, 1765, 53962, -1000, -1000, 1633, 1546, 2003, - 2532, -1000, 1589, -1000, 386, -1000, 2047, 2129, -1000, -1000, - -1000, 19646, -1000, -1000, -1000, -1000, -1000, 238, -192, 15904, - 11392, 1580, -1000, -185, 2137, 1572, -1000, -477, -1000, -1000, - -1000, -1000, 256, -1000, -1000, 1919, -1000, -1000, 1549, 1545, - 1528, -1000, -1000, -1000, -1000, -1000, -1000, -268, -1000, -1000, - 2499, -1000, -228, -1000, -1000, 1915, 1552, -1000, -1000, -1000, - 31582, 53216, -1000, -176, 332, -192, 21884, 2044, 1546, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -20, -1000, -1000, - 495, -1000, -1000, -1000, 2135, -206, -1000, -1000, -1000, 288, - -497, -286, -293, 24122, -1000, 21884, -1000, 21884, -1000, 21884, - -1000, 53962, 2003, -1000, -1000, -1000, 1531, -1000, 5018, -355, - 2166, -1000, -113, -1000, -1000, -1000, 1060, 1381, -1000, 1157, - -1000, -1000, -1000, -1000, -1000, 2266, 53962, -1000, 429, -1000, - -1000, 15152, -197, -210, 987, -1000, -1000, -1000, -1000, -1000, - 4398, 1482, 1476, 2137, -1000, 53962, -1000, 53216, -340, 867, - 4706, -1000, 2160, 2159, 2663, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -550, 1370, 244, -1000, -1000, -1000, 288, -343, - -1000, 21884, -1000, 21884, -1000, 1546, -1000, -1000, 2471, 83, - -1000, 2699, -1000, 2640, 1006, 1006, -1000, 1111, -550, -1000, - -1000, -1000, -1000, 2137, 2137, -1000, -358, -1000, -1000, -1000, - -1000, -1000, 413, 1152, -1000, -1000, -1000, -1000, -1000, 4706, - -1000, -1000, -1000, 215, 215, -1000, -1000, + -1000, 21884, -1000, 1444, 2124, -1000, -377, -1000, -516, 21884, + -260, -1000, -1000, -260, -1000, -1000, -1000, -1000, -1000, 21884, + -1000, -1000, 21884, -1000, 21884, -1000, -1000, 1589, -1000, -1000, + -1000, 36804, -1000, 1589, -1000, 1589, -1000, -267, -1000, 1875, + -1000, 53962, 1163, 368, -1000, 1154, -1000, -1000, -1000, -1000, + -1000, 54708, 53962, 1945, 53962, -1000, -1000, 1571, 1444, 1985, + 2563, -1000, 1561, -1000, 458, -1000, 1987, 2024, -1000, -1000, + -1000, 19646, -1000, -1000, -1000, -1000, -1000, 274, -192, 15904, + 11392, 1555, -1000, -191, 2013, 1544, -1000, -478, -1000, -1000, + -1000, -1000, 283, -1000, -1000, 1866, -1000, -1000, 1694, 1684, + 1633, -1000, -1000, -1000, -1000, -1000, -1000, -267, -1000, -1000, + 2469, -1000, -232, -1000, -1000, 1862, 1534, -1000, -1000, -1000, + 31582, 53216, -1000, -176, 414, -192, 21884, 1986, 1444, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -2, -1000, -1000, + 801, -1000, -1000, -1000, 2085, -220, -1000, -1000, -1000, 4704, + -497, -306, -312, 24122, -1000, 21884, -1000, 21884, -1000, 21884, + -1000, 53962, 1985, -1000, -1000, -1000, 1528, -1000, 4276, -355, + 2123, -1000, -109, -1000, -1000, -1000, 1087, 1400, -1000, 1127, + -1000, -1000, -1000, -1000, -1000, 2255, 53962, -1000, 450, -1000, + -1000, 15152, -195, -217, 994, -1000, -1000, -1000, -1000, -1000, + 4110, 1623, 1549, 2013, -1000, 53962, -1000, 53216, -337, 885, + 8389, -1000, 2122, 2100, 2651, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -550, 1497, 300, -1000, -1000, -1000, 4704, -317, + -1000, 21884, -1000, 21884, -1000, 1444, -1000, -1000, 2475, 59, + -1000, 2674, -1000, 2672, 970, 970, -1000, 1126, -550, -1000, + -1000, -1000, -1000, 2013, 2013, -1000, -360, -1000, -1000, -1000, + -1000, -1000, 432, 1289, -1000, -1000, -1000, -1000, -1000, 8389, + -1000, -1000, -1000, 216, 216, -1000, -1000, } var yyPgo = [...]int{ - 0, 3391, 3388, 28, 4, 38, 37, 3387, 3386, 3383, - 166, 3382, 3381, 3378, 3364, 3363, 3294, 2769, 2762, 2739, - 3292, 3289, 3284, 3278, 3273, 3269, 3267, 3265, 3263, 48, - 96, 87, 105, 203, 192, 3260, 3256, 179, 169, 199, - 3254, 3237, 3235, 113, 189, 79, 78, 191, 3234, 3233, - 62, 3230, 3228, 3226, 187, 186, 185, 1053, 3225, 181, - 112, 46, 3224, 3223, 3220, 3217, 3216, 3215, 3214, 3213, - 3209, 3208, 3207, 3204, 3201, 3200, 3195, 3191, 3189, 3188, - 286, 3187, 3186, 10, 3185, 65, 3184, 3183, 3182, 3181, - 3180, 15, 3178, 3177, 20, 41, 57, 3176, 3175, 44, - 3174, 3173, 3172, 3171, 3169, 71, 3167, 26, 3159, 40, - 3153, 3152, 120, 3151, 3145, 3144, 30, 3143, 3140, 3137, - 2, 172, 3136, 3135, 136, 3134, 3130, 3125, 190, 217, - 3123, 2296, 3122, 101, 3121, 3117, 3109, 170, 194, 3108, - 121, 3107, 3104, 3103, 145, 3102, 3920, 3100, 3099, 68, - 63, 226, 3098, 3093, 200, 64, 60, 3092, 16, 13, - 3089, 3081, 59, 75, 3080, 106, 3077, 3076, 100, 80, - 3075, 98, 104, 3073, 3071, 21, 7, 3070, 5, 1, - 6, 82, 3069, 3065, 117, 3064, 3061, 3060, 93, 3057, - 3055, 6505, 3049, 92, 128, 103, 70, 3046, 171, 153, - 3043, 3042, 3041, 3039, 3036, 3034, 49, 3018, 3014, 3011, - 134, 1256, 108, 3004, 3002, 147, 352, 126, 50, 129, - 3001, 146, 2996, 215, 76, 2993, 164, 2992, 2991, 133, - 132, 2987, 2984, 56, 168, 195, 2983, 97, 127, 116, - 176, 95, 131, 2982, 2981, 55, 69, 2979, 2978, 2976, - 2966, 165, 2963, 2962, 58, 2948, 53, 2944, 178, 2938, - 325, 67, 2937, 180, 161, 2935, 72, 2934, 2924, 66, - 99, 102, 29, 2918, 157, 162, 123, 160, 2917, 2915, - 52, 2911, 2910, 2904, 197, 327, 2902, 2901, 305, 167, - 141, 149, 83, 2899, 353, 2898, 2897, 2891, 14, 5107, - 7204, 183, 27, 163, 2881, 2879, 1040, 19, 47, 24, - 2878, 210, 2877, 198, 2876, 2874, 2873, 205, 208, 111, - 158, 54, 2872, 2870, 2869, 2864, 35, 2862, 2861, 2854, - 2852, 2851, 2850, 36, 34, 33, 89, 221, 61, 18, - 94, 155, 150, 73, 2847, 2846, 2845, 119, 91, 2842, - 156, 154, 122, 159, 2840, 177, 142, 115, 2836, 84, - 32, 2835, 2833, 2830, 2829, 85, 2827, 2824, 2814, 2813, - 152, 143, 118, 77, 2812, 81, 114, 148, 144, 43, - 2811, 42, 2810, 2807, 31, 188, 25, 2804, 39, 107, - 137, 2803, 5291, 2799, 9, 349, 151, 2798, 2792, 8, - 11, 17, 2788, 2787, 2780, 2777, 130, 2775, 2773, 2764, - 2757, 23, 45, 22, 12, 110, 135, 74, 2756, 2745, - 139, 2744, 2725, 2716, 0, 1034, 125, 2688, 202, + 0, 3238, 3237, 28, 4, 38, 37, 3236, 3235, 3234, + 167, 3233, 3230, 3228, 3226, 3225, 3224, 2718, 2716, 2697, + 3223, 3220, 3217, 3216, 3215, 3214, 3213, 3211, 3209, 48, + 96, 87, 102, 202, 194, 3207, 3206, 179, 163, 199, + 3203, 3201, 3200, 112, 186, 83, 78, 193, 3195, 3191, + 62, 3189, 3188, 3187, 189, 188, 183, 1072, 3186, 181, + 111, 46, 3183, 3181, 3180, 3179, 3178, 3175, 3173, 3172, + 3171, 3169, 3167, 3159, 3153, 3152, 3151, 3146, 3142, 3136, + 286, 3135, 3134, 10, 3132, 65, 3131, 3126, 3122, 3119, + 3115, 15, 3113, 3110, 20, 41, 57, 3109, 3108, 44, + 3107, 3104, 3103, 3102, 3098, 71, 3096, 26, 3094, 40, + 3093, 3092, 125, 3091, 3089, 3082, 30, 3081, 3077, 3076, + 2, 169, 3073, 3072, 136, 3071, 3069, 3067, 191, 205, + 3065, 2345, 3061, 98, 3060, 3057, 3054, 170, 192, 3052, + 121, 3047, 3043, 3042, 149, 3036, 3920, 3031, 3027, 68, + 63, 226, 3020, 3014, 200, 60, 64, 3011, 16, 13, + 3004, 3002, 59, 75, 3001, 106, 3000, 2999, 104, 77, + 2997, 113, 101, 2996, 2993, 21, 7, 2991, 5, 1, + 6, 82, 2987, 2984, 107, 2983, 2982, 2981, 93, 2980, + 2979, 6505, 2976, 91, 128, 99, 70, 2970, 171, 153, + 2968, 2964, 2962, 2945, 2944, 2940, 49, 2938, 2937, 2935, + 134, 1256, 108, 2934, 2928, 147, 352, 129, 50, 130, + 2927, 148, 2925, 215, 76, 2924, 164, 2918, 2917, 133, + 132, 2912, 2902, 56, 168, 195, 2899, 97, 126, 115, + 176, 95, 131, 2898, 2891, 55, 69, 2881, 2877, 2874, + 2873, 166, 2872, 2870, 58, 2869, 53, 2868, 178, 2866, + 325, 67, 2865, 180, 161, 2864, 72, 2861, 2859, 66, + 100, 103, 29, 2857, 157, 162, 122, 160, 2856, 2855, + 52, 2854, 2853, 2850, 197, 294, 2847, 2846, 305, 172, + 141, 145, 85, 2842, 320, 2840, 2837, 2830, 14, 5107, + 7204, 185, 27, 159, 2829, 2828, 1040, 19, 47, 24, + 2825, 221, 2823, 198, 2821, 2818, 2817, 210, 211, 117, + 158, 54, 2812, 2811, 2810, 2807, 35, 2806, 2805, 2800, + 2799, 2798, 2795, 36, 34, 33, 89, 220, 61, 18, + 94, 155, 150, 73, 2793, 2792, 2791, 119, 79, 2788, + 156, 151, 120, 165, 2787, 177, 142, 116, 2786, 84, + 32, 2785, 2783, 2782, 2781, 92, 2780, 2779, 2778, 2775, + 152, 143, 118, 80, 2773, 81, 114, 146, 144, 43, + 2772, 42, 2771, 2770, 31, 187, 25, 2769, 39, 105, + 137, 2765, 5291, 2764, 9, 339, 154, 2763, 2762, 8, + 11, 17, 2758, 2745, 2744, 2742, 127, 2741, 2740, 2739, + 2734, 23, 45, 22, 12, 110, 135, 74, 2726, 2708, + 139, 2704, 2698, 2696, 0, 1034, 123, 2688, 203, } //line sql.y:8822 @@ -9393,7 +9393,7 @@ var yyChk = [...]int{ -425, -424, -240, 97, -244, -29, 95, -3, 288, -333, -384, -334, -335, -5, -6, -360, -83, 597, -386, -364, -306, -302, 98, 105, 97, 597, -425, -425, -91, 159, - 723, 687, -156, 235, -425, 96, -425, 96, -425, 96, + 723, 687, -155, 235, -425, 96, -425, 96, -425, 96, -107, 96, 28, 602, -425, -309, -177, -175, -300, 651, -403, -402, 593, -413, -409, 130, 157, 110, -411, 684, 689, 645, 140, 141, -83, -146, 95, -425, -84, 303, diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index 5ecdeabe108..f631074c784 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -5709,7 +5709,7 @@ frame_point: { $$ = &FramePoint{Type:UnboundedFollowingType} } -| NUM_literal PRECEDING +| literal PRECEDING { $$ = &FramePoint{Type:ExprPrecedingType, Expr:$1} } @@ -5717,7 +5717,7 @@ frame_point: { $$ = &FramePoint{Type:ExprPrecedingType, Expr:$2, Unit: $3} } -| NUM_literal FOLLOWING +| literal FOLLOWING { $$ = &FramePoint{Type:ExprFollowingType, Expr:$1} } diff --git a/go/vt/sqlparser/utils.go b/go/vt/sqlparser/utils.go index c56e7740fc5..e805b0a46dc 100644 --- a/go/vt/sqlparser/utils.go +++ b/go/vt/sqlparser/utils.go @@ -42,7 +42,7 @@ func (p *Parser) QueryMatchesTemplates(query string, queryTemplates []string) (m return "", err } - out, err := PrepareAST(stmt, NewReservedVars("", reservedVars), bv, true, "ks", 0, "", map[string]string{}, nil, nil) + out, err := Normalize(stmt, NewReservedVars("", reservedVars), bv, true, "ks", 0, "", map[string]string{}, nil, nil) if err != nil { return "", err } diff --git a/go/vt/srvtopo/resolver.go b/go/vt/srvtopo/resolver.go index 0ccfb0fd872..c1057faeb15 100644 --- a/go/vt/srvtopo/resolver.go +++ b/go/vt/srvtopo/resolver.go @@ -168,7 +168,7 @@ func (r *Resolver) GetAllKeyspaces(ctx context.Context) ([]string, error) { // If dst1 is in shard1, and dst2 and dst3 are in shard2, the output will be: // - []*ResolvedShard: shard1, shard2 // - [][][]sqltypes.Value: [[id1a,id1b]], [[id2a,id2b], [id3a,id3b]] -func (r *Resolver) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids [][]sqltypes.Value, destinations []key.Destination) ([]*ResolvedShard, [][][]sqltypes.Value, error) { +func (r *Resolver) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*ResolvedShard, [][][]sqltypes.Value, error) { keyspace, _, allShards, err := r.GetKeyspaceShards(ctx, keyspace, tabletType) if err != nil { return nil, nil, err @@ -245,7 +245,7 @@ func (acc *resultAcc) resolveShard(idx int) func(shard string) error { // If dst1 is in shard1, and dst2 and dst3 are in shard2, the output will be: // - []*ResolvedShard: shard1, shard2 // - [][]*querypb.Value: [id1], [id2, id3] -func (r *Resolver) ResolveDestinations(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids []*querypb.Value, destinations []key.Destination) ([]*ResolvedShard, [][]*querypb.Value, error) { +func (r *Resolver) ResolveDestinations(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids []*querypb.Value, destinations []key.ShardDestination) ([]*ResolvedShard, [][]*querypb.Value, error) { keyspace, _, allShards, err := r.GetKeyspaceShards(ctx, keyspace, tabletType) if err != nil { return nil, nil, err @@ -290,9 +290,9 @@ func (r *Resolver) ResolveDestinations(ctx context.Context, keyspace string, tab } // ResolveDestination is a shortcut to ResolveDestinations with only -// one Destination, and no ids. -func (r *Resolver) ResolveDestination(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, destination key.Destination) ([]*ResolvedShard, error) { - rss, _, err := r.ResolveDestinations(ctx, keyspace, tabletType, nil, []key.Destination{destination}) +// one ShardDestination, and no ids. +func (r *Resolver) ResolveDestination(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, destination key.ShardDestination) ([]*ResolvedShard, error) { + rss, _, err := r.ResolveDestinations(ctx, keyspace, tabletType, nil, []key.ShardDestination{destination}) return rss, err } diff --git a/go/vt/srvtopo/resolver_test.go b/go/vt/srvtopo/resolver_test.go index fae8bef1fb2..7c1c25ac6f4 100644 --- a/go/vt/srvtopo/resolver_test.go +++ b/go/vt/srvtopo/resolver_test.go @@ -127,7 +127,7 @@ func TestResolveDestinations(t *testing.T) { name string keyspace string ids []*querypb.Value - destinations []key.Destination + destinations []key.ShardDestination errString string expectedShards []string expectedValues [][]*querypb.Value @@ -135,7 +135,7 @@ func TestResolveDestinations(t *testing.T) { { name: "unsharded keyspace, regular shard, no ids", keyspace: "uks", - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationShard("0"), }, expectedShards: []string{"0"}, @@ -144,7 +144,7 @@ func TestResolveDestinations(t *testing.T) { name: "unsharded keyspace, regular shard, with ids", keyspace: "uks", ids: []*querypb.Value{id1, id2}, - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationShard("0"), key.DestinationShard("0"), }, @@ -157,7 +157,7 @@ func TestResolveDestinations(t *testing.T) { name: "sharded keyspace, keyrange destinations, with ids", keyspace: "sks", ids: []*querypb.Value{id1, id2}, - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationExactKeyRange{KeyRange: kr2040}, key.DestinationExactKeyRange{KeyRange: kr80a0}, }, @@ -171,7 +171,7 @@ func TestResolveDestinations(t *testing.T) { name: "sharded keyspace, keyspace id destinations, with ids", keyspace: "sks", ids: []*querypb.Value{id1, id2}, - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationKeyspaceID{0x28}, key.DestinationKeyspaceID{0x78, 0x23}, }, @@ -185,7 +185,7 @@ func TestResolveDestinations(t *testing.T) { name: "sharded keyspace, multi keyspace id destinations, with ids", keyspace: "sks", ids: []*querypb.Value{id1, id2}, - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationKeyspaceIDs{ {0x28}, {0x47}, @@ -205,7 +205,7 @@ func TestResolveDestinations(t *testing.T) { { name: "using non-mapping keyranges should fail", keyspace: "sks", - destinations: []key.Destination{ + destinations: []key.ShardDestination{ key.DestinationExactKeyRange{ KeyRange: kr2830, }, diff --git a/go/vt/sysvars/sysvars.go b/go/vt/sysvars/sysvars.go index 297ed956bf8..17e495ee9ca 100644 --- a/go/vt/sysvars/sysvars.go +++ b/go/vt/sysvars/sysvars.go @@ -87,6 +87,9 @@ var ( ReadAfterWriteTimeOut = SystemVariable{Name: "read_after_write_timeout"} SessionTrackGTIDs = SystemVariable{Name: "session_track_gtids", IdentifierAsString: true} + // Filled in from VitessAware, ReadOnly, IgnoreThese, NotSupported, UseReservedConn, CheckAndIgnore + AllSystemVariables map[string]SystemVariable + VitessAware = []SystemVariable{ Autocommit, ClientFoundRows, @@ -269,6 +272,31 @@ var ( } ) +func init() { + AllSystemVariables = make(map[string]SystemVariable) + for _, set := range [][]SystemVariable{ + VitessAware, + ReadOnly, + IgnoreThese, + NotSupported, + UseReservedConn, + CheckAndIgnore, + } { + for _, v := range set { + AllSystemVariables[v.Name] = v + } + } +} + +func SupportsSetVar(name string) bool { + sys, ok := AllSystemVariables[name] + if !ok { + return false + } + + return sys.SupportSetVar +} + // GetInterestingVariables is used to return all the variables that may be listed in a SHOW VARIABLES command. func GetInterestingVariables() []string { var res []string diff --git a/go/vt/topo/topoproto/destination.go b/go/vt/topo/topoproto/destination.go index c86affd1569..19a4965e62b 100644 --- a/go/vt/topo/topoproto/destination.go +++ b/go/vt/topo/topoproto/destination.go @@ -27,10 +27,10 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -// ParseDestination parses the string representation of a Destination +// ParseDestination parses the string representation of a ShardDestination // of the form keyspace:shard@tablet_type. You can use a / instead of a :. -func ParseDestination(targetString string, defaultTabletType topodatapb.TabletType) (string, topodatapb.TabletType, key.Destination, error) { - var dest key.Destination +func ParseDestination(targetString string, defaultTabletType topodatapb.TabletType) (string, topodatapb.TabletType, key.ShardDestination, error) { + var dest key.ShardDestination var keyspace string tabletType := defaultTabletType diff --git a/go/vt/topo/topoproto/destination_test.go b/go/vt/topo/topoproto/destination_test.go index 13de245d14e..553f749021b 100644 --- a/go/vt/topo/topoproto/destination_test.go +++ b/go/vt/topo/topoproto/destination_test.go @@ -32,7 +32,7 @@ func TestParseDestination(t *testing.T) { testcases := []struct { targetString string - dest key.Destination + dest key.ShardDestination keyspace string tabletType topodatapb.TabletType }{{ diff --git a/go/vt/vitessdriver/driver.go b/go/vt/vitessdriver/driver.go index 4a965399e9c..25da8c3bb8d 100644 --- a/go/vt/vitessdriver/driver.go +++ b/go/vt/vitessdriver/driver.go @@ -491,7 +491,7 @@ func (c *conn) Exec(query string, args []driver.Value) (driver.Result, error) { return nil, err } - qr, err := c.session.Execute(ctx, query, bindVars) + qr, err := c.session.Execute(ctx, query, bindVars, false) if err != nil { return nil, err } @@ -507,7 +507,7 @@ func (c *conn) ExecContext(ctx context.Context, query string, args []driver.Name if err != nil { return nil, err } - qr, err := c.session.Execute(ctx, query, bv) + qr, err := c.session.Execute(ctx, query, bv, false) if err != nil { return nil, err } @@ -529,7 +529,7 @@ func (c *conn) Query(query string, args []driver.Value) (driver.Rows, error) { return newStreamingRows(stream, c.convert), nil } - qr, err := c.session.Execute(ctx, query, bindVars) + qr, err := c.session.Execute(ctx, query, bindVars, false) if err != nil { return nil, err } @@ -555,7 +555,7 @@ func (c *conn) QueryContext(ctx context.Context, query string, args []driver.Nam return newStreamingRows(stream, c.convert), nil } - qr, err := c.session.Execute(ctx, query, bv) + qr, err := c.session.Execute(ctx, query, bv, false) if err != nil { return nil, err } diff --git a/go/vt/vitessdriver/driver_test.go b/go/vt/vitessdriver/driver_test.go index bd49a0acd0a..c3b12526300 100644 --- a/go/vt/vitessdriver/driver_test.go +++ b/go/vt/vitessdriver/driver_test.go @@ -730,7 +730,7 @@ func TestStreamExec(t *testing.T) { require.NoError(t, err) defer db.Close() - for k, v := range execMap { + for k, v := range createExecMap() { t.Run(k, func(t *testing.T) { s, err := db.Prepare(k) require.NoError(t, err) diff --git a/go/vt/vitessdriver/fakeserver_test.go b/go/vt/vitessdriver/fakeserver_test.go index a4b17fc65d6..a2b43caefde 100644 --- a/go/vt/vitessdriver/fakeserver_test.go +++ b/go/vt/vitessdriver/fakeserver_test.go @@ -32,14 +32,26 @@ import ( ) // fakeVTGateService has the server side of this fake -type fakeVTGateService struct{} +type ( + fakeVTGateService struct { + execMap map[string]execMapResponse + } -// queryExecute contains all the fields we use to test Execute -type queryExecute struct { - SQL string - BindVariables map[string]*querypb.BindVariable - Session *vtgatepb.Session -} + execMapResponse struct { + execQuery *queryExecute + paramsCount uint16 + result *sqltypes.Result + session *vtgatepb.Session + err error + } + + // queryExecute contains all the fields we use to test Execute + queryExecute struct { + SQL string + BindVariables map[string]*querypb.BindVariable + Session *vtgatepb.Session + } +) func (q *queryExecute) Equal(q2 *queryExecute) bool { return q.SQL == q2.SQL && @@ -48,8 +60,15 @@ func (q *queryExecute) Equal(q2 *queryExecute) bool { } // Execute is part of the VTGateService interface -func (f *fakeVTGateService) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { - execCase, ok := execMap[sql] +func (f *fakeVTGateService) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { + execCase, ok := f.execMap[sql] if !ok { return session, nil, fmt.Errorf("no match for: %s", sql) } @@ -71,7 +90,7 @@ func (f *fakeVTGateService) Execute(ctx context.Context, mysqlCtx vtgateservice. // ExecuteBatch is part of the VTGateService interface func (f *fakeVTGateService) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, sql []string, bindVariables []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) { if len(sql) == 1 { - execCase, ok := execMap[sql[0]] + execCase, ok := f.execMap[sql[0]] if !ok { return session, nil, fmt.Errorf("no match for: %s", sql) } @@ -99,7 +118,7 @@ func (f *fakeVTGateService) ExecuteBatch(ctx context.Context, session *vtgatepb. // StreamExecute is part of the VTGateService interface func (f *fakeVTGateService) StreamExecute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable, callback func(*sqltypes.Result) error) (*vtgatepb.Session, error) { - execCase, ok := execMap[sql] + execCase, ok := f.execMap[sql] if !ok { return session, fmt.Errorf("no match for: %s", sql) } @@ -131,24 +150,23 @@ func (f *fakeVTGateService) StreamExecute(ctx context.Context, mysqlCtx vtgatese } // Prepare is part of the VTGateService interface -func (f *fakeVTGateService) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { - execCase, ok := execMap[sql] +func (f *fakeVTGateService) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { + execCase, ok := f.execMap[sql] if !ok { - return session, nil, fmt.Errorf("no match for: %s", sql) + return session, nil, 0, fmt.Errorf("no match for: %s", sql) } query := &queryExecute{ - SQL: sql, - BindVariables: bindVariables, - Session: session, + SQL: sql, + Session: session, } if !query.Equal(execCase.execQuery) { - return session, nil, fmt.Errorf("Prepare request mismatch: got %+v, want %+v", query, execCase.execQuery) + return session, nil, 0, fmt.Errorf("Prepare request mismatch: got %+v, want %+v", query, execCase.execQuery) } if execCase.session != nil { proto.Reset(session) proto.Merge(session, execCase.session) } - return session, execCase.result.Fields, nil + return session, execCase.result.Fields, execCase.paramsCount, nil } func (f *fakeVTGateService) CloseSession(ctx context.Context, session *vtgatepb.Session) error { @@ -168,124 +186,127 @@ func (f *fakeVTGateService) HandlePanic(err *error) { // CreateFakeServer returns the fake server for the tests func CreateFakeServer() vtgateservice.VTGateService { - return &fakeVTGateService{} + return &fakeVTGateService{ + execMap: createExecMap(), + } } -var execMap = map[string]struct { - execQuery *queryExecute - result *sqltypes.Result - session *vtgatepb.Session - err error -}{ - "request": { - execQuery: &queryExecute{ - SQL: "request", - BindVariables: map[string]*querypb.BindVariable{ - "v1": sqltypes.Int64BindVariable(0), - }, - Session: &vtgatepb.Session{ - TargetString: "@rdonly", - Autocommit: true, +func createExecMap() map[string]execMapResponse { + return map[string]execMapResponse{ + "request": { + execQuery: &queryExecute{ + SQL: "request", + BindVariables: map[string]*querypb.BindVariable{ + "v1": sqltypes.Int64BindVariable(0), + }, + Session: &vtgatepb.Session{ + TargetString: "@rdonly", + Autocommit: true, + }, }, + result: &result1, + paramsCount: 1, + session: nil, }, - result: &result1, - session: nil, - }, - "requestDates": { - execQuery: &queryExecute{ - SQL: "requestDates", - BindVariables: map[string]*querypb.BindVariable{ - "v1": sqltypes.Int64BindVariable(0), - }, - Session: &vtgatepb.Session{ - TargetString: "@rdonly", - Autocommit: true, + "requestDates": { + execQuery: &queryExecute{ + SQL: "requestDates", + BindVariables: map[string]*querypb.BindVariable{ + "v1": sqltypes.Int64BindVariable(0), + }, + Session: &vtgatepb.Session{ + TargetString: "@rdonly", + Autocommit: true, + }, }, + result: &result2, + paramsCount: 1, + session: nil, }, - result: &result2, - session: nil, - }, - "txRequest": { - execQuery: &queryExecute{ - SQL: "txRequest", - BindVariables: map[string]*querypb.BindVariable{ - "v1": sqltypes.Int64BindVariable(0), + "txRequest": { + execQuery: &queryExecute{ + SQL: "txRequest", + BindVariables: map[string]*querypb.BindVariable{ + "v1": sqltypes.Int64BindVariable(0), + }, + Session: session1, }, - Session: session1, + result: &sqltypes.Result{}, + paramsCount: 1, + session: session2, }, - result: &sqltypes.Result{}, - session: session2, - }, - "distributedTxRequest": { - execQuery: &queryExecute{ - SQL: "distributedTxRequest", - BindVariables: map[string]*querypb.BindVariable{ - "v1": sqltypes.Int64BindVariable(1), - }, - Session: &vtgatepb.Session{ - InTransaction: true, - ShardSessions: []*vtgatepb.Session_ShardSession{ - { - Target: &querypb.Target{ - Keyspace: "ks", - Shard: "1", - TabletType: topodatapb.TabletType_PRIMARY, + "distributedTxRequest": { + execQuery: &queryExecute{ + SQL: "distributedTxRequest", + BindVariables: map[string]*querypb.BindVariable{ + "v1": sqltypes.Int64BindVariable(1), + }, + Session: &vtgatepb.Session{ + InTransaction: true, + ShardSessions: []*vtgatepb.Session_ShardSession{ + { + Target: &querypb.Target{ + Keyspace: "ks", + Shard: "1", + TabletType: topodatapb.TabletType_PRIMARY, + }, + TransactionId: 1, }, - TransactionId: 1, }, + TargetString: "@rdonly", }, - TargetString: "@rdonly", }, + result: &sqltypes.Result{}, + paramsCount: 1, + session: session2, }, - result: &sqltypes.Result{}, - session: session2, - }, - "begin": { - execQuery: &queryExecute{ - SQL: "begin", - Session: &vtgatepb.Session{ + "begin": { + execQuery: &queryExecute{ + SQL: "begin", + Session: &vtgatepb.Session{ + TargetString: "@primary", + Autocommit: true, + }, + }, + result: &sqltypes.Result{}, + session: session1, + }, + "commit": { + execQuery: &queryExecute{ + SQL: "commit", + Session: session2, + }, + result: &sqltypes.Result{}, + session: &vtgatepb.Session{ TargetString: "@primary", Autocommit: true, }, }, - result: &sqltypes.Result{}, - session: session1, - }, - "commit": { - execQuery: &queryExecute{ - SQL: "commit", - Session: session2, - }, - result: &sqltypes.Result{}, - session: &vtgatepb.Session{ - TargetString: "@primary", - Autocommit: true, - }, - }, - "rollback": { - execQuery: &queryExecute{ - SQL: "rollback", - Session: session2, - }, - result: &sqltypes.Result{}, - session: &vtgatepb.Session{ - TargetString: "@primary", - }, - }, - "use @rdonly": { - execQuery: &queryExecute{ - SQL: "use @rdonly", - Session: &vtgatepb.Session{ + "rollback": { + execQuery: &queryExecute{ + SQL: "rollback", + Session: session2, + }, + result: &sqltypes.Result{}, + session: &vtgatepb.Session{ TargetString: "@primary", - Autocommit: true, }, }, - result: &sqltypes.Result{}, - session: &vtgatepb.Session{ - TargetString: "@rdonly", - SessionUUID: "1111", + "use @rdonly": { + execQuery: &queryExecute{ + SQL: "use @rdonly", + Session: &vtgatepb.Session{ + TargetString: "@primary", + Autocommit: true, + }, + }, + result: &sqltypes.Result{}, + session: &vtgatepb.Session{ + TargetString: "@rdonly", + SessionUUID: "1111", + }, }, - }, + } } var result1 = sqltypes.Result{ @@ -357,5 +378,3 @@ var session2 = &vtgatepb.Session{ }, TargetString: "@rdonly", } - -var dtid2 = "aa" diff --git a/go/vt/vtexplain/vtexplain_vtgate.go b/go/vt/vtexplain/vtexplain_vtgate.go index 155fe6de743..7d25758a30e 100644 --- a/go/vt/vtexplain/vtexplain_vtgate.go +++ b/go/vt/vtexplain/vtexplain_vtgate.go @@ -243,7 +243,7 @@ func (vte *VTExplain) vtgateExecute(sql string) ([]*engine.Plan, map[string]*Tab // This will ensure that the commit/rollback order is predictable. vte.sortShardSession() - _, err := vte.vtgateExecutor.Execute(context.Background(), nil, "VtexplainExecute", econtext.NewSafeSession(vte.vtgateSession), sql, nil) + _, err := vte.vtgateExecutor.Execute(context.Background(), nil, "VtexplainExecute", econtext.NewSafeSession(vte.vtgateSession), sql, nil, false) if err != nil { for _, tc := range vte.explainTopo.TabletConns { tc.tabletQueries = nil diff --git a/go/vt/vtgate/autocommit_test.go b/go/vt/vtgate/autocommit_test.go index 2e65cefbabe..33d78be889c 100644 --- a/go/vt/vtgate/autocommit_test.go +++ b/go/vt/vtgate/autocommit_test.go @@ -26,7 +26,6 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - econtext "vitess.io/vitess/go/vt/vtgate/executorcontext" ) // This file contains tests for all the autocommit code paths @@ -382,7 +381,7 @@ func TestAutocommitTransactionStarted(t *testing.T) { // single shard query - no savepoint needed sql := "update `user` set a = 2 where id = 1" - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + _, err := executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) require.Len(t, sbc1.Queries, 1) require.Equal(t, sql, sbc1.Queries[0].Sql) @@ -394,7 +393,7 @@ func TestAutocommitTransactionStarted(t *testing.T) { // multi shard query - savepoint needed sql = "update `user` set a = 2 where id in (1, 4)" expectedSql := "update `user` set a = 2 where id in ::__vals" - _, err = executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + _, err = executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) require.Len(t, sbc1.Queries, 2) require.Contains(t, sbc1.Queries[0].Sql, "savepoint") @@ -413,7 +412,7 @@ func TestAutocommitDirectTarget(t *testing.T) { } sql := "insert into `simple`(val) values ('val')" - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + _, err := executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) assertQueries(t, sbclookup, []*querypb.BoundQuery{{ @@ -434,7 +433,7 @@ func TestAutocommitDirectRangeTarget(t *testing.T) { } sql := "delete from sharded_user_msgs limit 1000" - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + _, err := executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) assertQueries(t, sbc1, []*querypb.BoundQuery{{ @@ -451,5 +450,5 @@ func autocommitExec(executor *Executor, sql string) (*sqltypes.Result, error) { TransactionMode: vtgatepb.TransactionMode_MULTI, } - return executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + return executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) } diff --git a/go/vt/vtgate/bench_test.go b/go/vt/vtgate/bench_test.go index c2336662c69..d1875f00508 100644 --- a/go/vt/vtgate/bench_test.go +++ b/go/vt/vtgate/bench_test.go @@ -70,6 +70,7 @@ func BenchmarkWithNormalizer(b *testing.B) { }, benchQuery, nil, + false, ) if err != nil { panic(err) @@ -92,6 +93,7 @@ func BenchmarkWithoutNormalizer(b *testing.B) { }, benchQuery, nil, + false, ) if err != nil { panic(err) diff --git a/go/vt/vtgate/engine/cached_size.go b/go/vt/vtgate/engine/cached_size.go index e2c59b0b26b..4a3365623fb 100644 --- a/go/vt/vtgate/engine/cached_size.go +++ b/go/vt/vtgate/engine/cached_size.go @@ -224,7 +224,7 @@ func (cached *Delete) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(8) } // field DML *vitess.io/vitess/go/vt/vtgate/engine.DML size += cached.DML.CachedSize(true) @@ -608,7 +608,7 @@ func (cached *Lock) CachedSize(alloc bool) int64 { } // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -649,7 +649,7 @@ func (cached *MStream) CachedSize(alloc bool) int64 { } // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -738,7 +738,7 @@ func (cached *OnlineDDL) CachedSize(alloc bool) int64 { size += hack.RuntimeAllocSize(int64(len(cached.SQL))) // field DDLStrategySetting *vitess.io/vitess/go/vt/schema.DDLStrategySetting size += cached.DDLStrategySetting.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -778,7 +778,7 @@ func (cached *Plan) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(160) + size += int64(224) } // field Original string size += hack.RuntimeAllocSize(int64(len(cached.Original))) @@ -802,6 +802,8 @@ func (cached *Plan) CachedSize(alloc bool) int64 { size += hack.RuntimeAllocSize(int64(len(elem))) } } + // field QueryHints vitess.io/vitess/go/vt/sqlparser.QueryHints + size += cached.QueryHints.CachedSize(false) return size } func (cached *Projection) CachedSize(alloc bool) int64 { @@ -914,7 +916,7 @@ func (cached *RevertMigration) CachedSize(alloc bool) int64 { size += cached.Stmt.CachedSize(true) // field Query string size += hack.RuntimeAllocSize(int64(len(cached.Query))) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -926,7 +928,7 @@ func (cached *Route) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(128) + size += int64(112) } // field Query string size += hack.RuntimeAllocSize(int64(len(cached.Query))) @@ -976,7 +978,7 @@ func (cached *RoutingParameters) CachedSize(alloc bool) int64 { } } } - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -1100,7 +1102,7 @@ func (cached *Send) CachedSize(alloc bool) int64 { } // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -1211,7 +1213,7 @@ func (cached *SysVarCheckAndIgnore) CachedSize(alloc bool) int64 { size += hack.RuntimeAllocSize(int64(len(cached.Name))) // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -1245,7 +1247,7 @@ func (cached *SysVarReservedConn) CachedSize(alloc bool) int64 { size += hack.RuntimeAllocSize(int64(len(cached.Name))) // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } @@ -1410,7 +1412,7 @@ func (cached *VStream) CachedSize(alloc bool) int64 { } // field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace size += cached.Keyspace.CachedSize(true) - // field TargetDestination vitess.io/vitess/go/vt/key.Destination + // field TargetDestination vitess.io/vitess/go/vt/key.ShardDestination if cc, ok := cached.TargetDestination.(cachedObject); ok { size += cc.CachedSize(true) } diff --git a/go/vt/vtgate/engine/dbddl.go b/go/vt/vtgate/engine/dbddl.go index 152894d5eab..27ae6c6a254 100644 --- a/go/vt/vtgate/engine/dbddl.go +++ b/go/vt/vtgate/engine/dbddl.go @@ -126,7 +126,7 @@ func (c *DBDDL) createDatabase(ctx context.Context, vcursor VCursor, plugin DBDD var destinations []*srvtopo.ResolvedShard for { // loop until we have found a valid shard - destinations, _, err = vcursor.ResolveDestinations(ctx, c.name, nil, []key.Destination{key.DestinationAllShards{}}) + destinations, _, err = vcursor.ResolveDestinations(ctx, c.name, nil, []key.ShardDestination{key.DestinationAllShards{}}) if err == nil { break } diff --git a/go/vt/vtgate/engine/delete.go b/go/vt/vtgate/engine/delete.go index f8425420974..f20f70ba187 100644 --- a/go/vt/vtgate/engine/delete.go +++ b/go/vt/vtgate/engine/delete.go @@ -20,7 +20,6 @@ import ( "context" "fmt" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/sqltypes" @@ -34,10 +33,10 @@ var _ Primitive = (*Delete)(nil) // Delete represents the instructions to perform a delete. type Delete struct { - *DML - - // Delete does not take inputs noInputs + noFields + + *DML } // TryExecute performs a non-streaming exec. @@ -71,11 +70,6 @@ func (del *Delete) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVa return callback(res) } -// GetFields fetches the field info. -func (del *Delete) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, fmt.Errorf("BUG: unreachable code for %q", del.Query) -} - // deleteVindexEntries performs an delete if table owns vindex. // Note: the commit order may be different from the DML order because it's possible // for DMLs to reuse existing transactions. @@ -140,11 +134,10 @@ func (del *Delete) description() PrimitiveDescription { } return PrimitiveDescription{ - OperatorType: "Delete", - Keyspace: del.Keyspace, - Variant: del.Opcode.String(), - TargetTabletType: topodatapb.TabletType_PRIMARY, - Other: other, + OperatorType: "Delete", + Keyspace: del.Keyspace, + Variant: del.Opcode.String(), + Other: other, } } diff --git a/go/vt/vtgate/engine/dml.go b/go/vt/vtgate/engine/dml.go index 9a0a044a3c4..31fc1a2d88e 100644 --- a/go/vt/vtgate/engine/dml.go +++ b/go/vt/vtgate/engine/dml.go @@ -139,7 +139,7 @@ func (dml *DML) execMultiShard(ctx context.Context, primitive Primitive, vcursor } func resolveKeyspaceID(ctx context.Context, vcursor VCursor, vindex vindexes.Vindex, vindexKey []sqltypes.Value) ([]byte, error) { - var destinations []key.Destination + var destinations []key.ShardDestination var err error switch vdx := vindex.(type) { case vindexes.MultiColumn: diff --git a/go/vt/vtgate/engine/dml_with_input.go b/go/vt/vtgate/engine/dml_with_input.go index e0eb3b03592..2ab6464792b 100644 --- a/go/vt/vtgate/engine/dml_with_input.go +++ b/go/vt/vtgate/engine/dml_with_input.go @@ -20,9 +20,6 @@ import ( "context" "fmt" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" - "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" ) @@ -34,6 +31,7 @@ const DmlVals = "dml_vals" // DMLWithInput represents the instructions to perform a DML operation based on the input result. type DMLWithInput struct { txNeeded + noFields Input Primitive @@ -161,11 +159,6 @@ func (dml *DMLWithInput) TryStreamExecute(ctx context.Context, vcursor VCursor, return callback(res) } -// GetFields fetches the field info. -func (dml *DMLWithInput) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.VT13001("unreachable code for DMLs") -} - func (dml *DMLWithInput) description() PrimitiveDescription { var offsets []string for idx, offset := range dml.OutputCols { @@ -185,8 +178,7 @@ func (dml *DMLWithInput) description() PrimitiveDescription { other["BindVars"] = bvList } return PrimitiveDescription{ - OperatorType: "DMLWithInput", - TargetTabletType: topodatapb.TabletType_PRIMARY, - Other: other, + OperatorType: "DMLWithInput", + Other: other, } } diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index 3b04ce3aa63..08636fdac78 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -388,11 +388,11 @@ func (t *noopVCursor) ExecuteKeyspaceID(ctx context.Context, keyspace string, ks panic("unimplemented") } -func (t *noopVCursor) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { +func (t *noopVCursor) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { return nil, nil, nil } -func (t *noopVCursor) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { +func (t *noopVCursor) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { panic("unimplemented") } @@ -640,7 +640,7 @@ func (f *loggingVCursor) StreamExecuteMulti(ctx context.Context, primitive Primi return []error{callback(r)} } -func (f *loggingVCursor) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { +func (f *loggingVCursor) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { f.log = append(f.log, fmt.Sprintf("ResolveDestinations %v %v %v", keyspace, ids, key.DestinationsString(destinations))) if f.shardErr != nil { return nil, nil, f.shardErr @@ -713,7 +713,7 @@ func (f *loggingVCursor) ResolveDestinations(ctx context.Context, keyspace strin return rss, values, nil } -func (f *loggingVCursor) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { +func (f *loggingVCursor) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { f.log = append(f.log, fmt.Sprintf("ResolveDestinationsMultiCol %v %v %v", keyspace, ids, key.DestinationsString(destinations))) if f.shardErr != nil { return nil, nil, f.shardErr diff --git a/go/vt/vtgate/engine/fk_cascade.go b/go/vt/vtgate/engine/fk_cascade.go index 35122ac9563..b73ab15546b 100644 --- a/go/vt/vtgate/engine/fk_cascade.go +++ b/go/vt/vtgate/engine/fk_cascade.go @@ -23,8 +23,6 @@ import ( "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/vterrors" ) // FkChild contains the Child Primitive to be executed collecting the values from the Selection Primitive using the column indexes. @@ -54,6 +52,7 @@ type NonLiteralUpdateInfo struct { // On success, it executes the Parent Primitive. type FkCascade struct { txNeeded + noFields // Selection is the Primitive that is used to find the rows that are going to be modified in the child tables. Selection Primitive @@ -78,11 +77,6 @@ func (fkc *FkCascade) GetTableName() string { return fkc.Parent.GetTableName() } -// GetFields implements the Primitive interface. -func (fkc *FkCascade) GetFields(_ context.Context, _ VCursor, _ map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] GetFields should not be called") -} - // TryExecute implements the Primitive interface. func (fkc *FkCascade) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { // Execute the Selection primitive to find the rows that are going to modified. diff --git a/go/vt/vtgate/engine/fk_verify.go b/go/vt/vtgate/engine/fk_verify.go index 7184e5d8381..da850f7a366 100644 --- a/go/vt/vtgate/engine/fk_verify.go +++ b/go/vt/vtgate/engine/fk_verify.go @@ -36,6 +36,7 @@ type Verify struct { // It does this by executing a select distinct query on the parent table with the values that are being inserted/updated. type FkVerify struct { txNeeded + noFields Verify []*Verify Exec Primitive @@ -62,11 +63,6 @@ func (f *FkVerify) GetTableName() string { return f.Exec.GetTableName() } -// GetFields implements the Primitive interface -func (f *FkVerify) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] GetFields should not be called") -} - // TryExecute implements the Primitive interface func (f *FkVerify) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { for _, v := range f.Verify { diff --git a/go/vt/vtgate/engine/insert.go b/go/vt/vtgate/engine/insert.go index c51c490ada6..74b0e954fe9 100644 --- a/go/vt/vtgate/engine/insert.go +++ b/go/vt/vtgate/engine/insert.go @@ -25,7 +25,6 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/key" querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/srvtopo" @@ -238,7 +237,7 @@ func (ins *Insert) getInsertShardedQueries( // each RSS. So we pass the ksid indexes in as ids, and get them back // as values. We also skip nil KeyspaceIds, no need to resolve them. var indexes []*querypb.Value - var destinations []key.Destination + var destinations []key.ShardDestination for i, ksid := range keyspaceIDs { if ksid != nil { indexes = append(indexes, &querypb.Value{ @@ -389,11 +388,10 @@ func (ins *Insert) description() PrimitiveDescription { } return PrimitiveDescription{ - OperatorType: "Insert", - Keyspace: ins.Keyspace, - Variant: ins.Opcode.String(), - TargetTabletType: topodatapb.TabletType_PRIMARY, - Other: other, + OperatorType: "Insert", + Keyspace: ins.Keyspace, + Variant: ins.Opcode.String(), + Other: other, } } diff --git a/go/vt/vtgate/engine/insert_common.go b/go/vt/vtgate/engine/insert_common.go index d4cae045e86..e325ab7b6cc 100644 --- a/go/vt/vtgate/engine/insert_common.go +++ b/go/vt/vtgate/engine/insert_common.go @@ -37,8 +37,8 @@ import ( type ( InsertCommon struct { - // Insert needs tx handling txNeeded + noFields // Opcode is the execution opcode. Opcode InsertOpcode @@ -144,13 +144,8 @@ func (ic *InsertCommon) GetTableName() string { return ic.TableName } -// GetFields fetches the field info. -func (ic *InsertCommon) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.VT13001("unexpected fields call for insert query") -} - func (ins *InsertCommon) executeUnshardedTableQuery(ctx context.Context, vcursor VCursor, loggingPrimitive Primitive, bindVars map[string]*querypb.BindVariable, query string, insertID uint64) (*sqltypes.Result, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, ins.Keyspace.Name, nil, []key.Destination{key.DestinationAllShards{}}) + rss, _, err := vcursor.ResolveDestinations(ctx, ins.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAllShards{}}) if err != nil { return nil, err } @@ -446,7 +441,7 @@ func (ic *InsertCommon) processGenerateFromValues( func (ic *InsertCommon) execGenerate(ctx context.Context, vcursor VCursor, loggingPrimitive Primitive, count int64) (int64, error) { // If generation is needed, generate the requested number of values (as one call). - rss, _, err := vcursor.ResolveDestinations(ctx, ic.Generate.Keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) + rss, _, err := vcursor.ResolveDestinations(ctx, ic.Generate.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAnyShard{}}) if err != nil { return 0, err } diff --git a/go/vt/vtgate/engine/insert_select.go b/go/vt/vtgate/engine/insert_select.go index 1e1ea5b20f9..4ccbb5cc071 100644 --- a/go/vt/vtgate/engine/insert_select.go +++ b/go/vt/vtgate/engine/insert_select.go @@ -27,7 +27,6 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/key" querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/srvtopo" @@ -235,7 +234,7 @@ func (ins *InsertSelect) getInsertShardedQueries( } var indexes []*querypb.Value - var destinations []key.Destination + var destinations []key.ShardDestination for i, ksid := range keyspaceIDs { if ksid != nil { indexes = append(indexes, &querypb.Value{ @@ -336,11 +335,10 @@ func (ins *InsertSelect) description() PrimitiveDescription { } return PrimitiveDescription{ - OperatorType: "Insert", - Keyspace: ins.Keyspace, - Variant: "Select", - TargetTabletType: topodatapb.TabletType_PRIMARY, - Other: other, + OperatorType: "Insert", + Keyspace: ins.Keyspace, + Variant: "Select", + Other: other, } } diff --git a/go/vt/vtgate/engine/lock.go b/go/vt/vtgate/engine/lock.go index a8ba9f46df3..2b962a76345 100644 --- a/go/vt/vtgate/engine/lock.go +++ b/go/vt/vtgate/engine/lock.go @@ -45,7 +45,7 @@ type Lock struct { Keyspace *vindexes.Keyspace // TargetDestination specifies an explicit target destination to send the query to. - TargetDestination key.Destination + TargetDestination key.ShardDestination FieldQuery string @@ -78,7 +78,7 @@ func (l *Lock) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[str } func (l *Lock) execLock(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, l.Keyspace.Name, nil, []key.Destination{l.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, l.Keyspace.Name, nil, []key.ShardDestination{l.TargetDestination}) if err != nil { return nil, err } @@ -162,7 +162,7 @@ func (l *Lock) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars m // GetFields is part of the Primitive interface func (l *Lock) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, l.Keyspace.Name, nil, []key.Destination{l.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, l.Keyspace.Name, nil, []key.ShardDestination{l.TargetDestination}) if err != nil { return nil, err } diff --git a/go/vt/vtgate/engine/mstream.go b/go/vt/vtgate/engine/mstream.go index af24199026b..a5b1eba05e6 100644 --- a/go/vt/vtgate/engine/mstream.go +++ b/go/vt/vtgate/engine/mstream.go @@ -37,7 +37,7 @@ type MStream struct { Keyspace *vindexes.Keyspace // TargetDestination specifies an explicit target destination to stream messages from - TargetDestination key.Destination + TargetDestination key.ShardDestination // TableName specifies the table on which stream will be executed. TableName string @@ -65,7 +65,7 @@ func (m *MStream) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[ // TryStreamExecute implements the Primitive interface func (m *MStream) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error { - rss, _, err := vcursor.ResolveDestinations(ctx, m.Keyspace.Name, nil, []key.Destination{m.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, m.Keyspace.Name, nil, []key.ShardDestination{m.TargetDestination}) if err != nil { return err } diff --git a/go/vt/vtgate/engine/online_ddl.go b/go/vt/vtgate/engine/online_ddl.go index 9acf55869bc..4bcbcc16c5c 100644 --- a/go/vt/vtgate/engine/online_ddl.go +++ b/go/vt/vtgate/engine/online_ddl.go @@ -42,7 +42,7 @@ type OnlineDDL struct { SQL string DDLStrategySetting *schema.DDLStrategySetting // TargetDestination specifies an explicit target destination to send the query to. - TargetDestination key.Destination + TargetDestination key.ShardDestination } func (v *OnlineDDL) description() PrimitiveDescription { diff --git a/go/vt/vtgate/engine/plan.go b/go/vt/vtgate/engine/plan.go index 33a32aecda9..a932dae2657 100644 --- a/go/vt/vtgate/engine/plan.go +++ b/go/vt/vtgate/engine/plan.go @@ -23,34 +23,48 @@ import ( "sync/atomic" "time" + "vitess.io/vitess/go/cache/theine" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vthash" ) -// Plan represents the execution strategy for a given query. -// For now it's a simple wrapper around the real instructions. -// An instruction (aka Primitive) is typically a tree where -// each node does its part by combining the results of the -// sub-nodes. type ( PlanType int8 + // Plan represents the execution strategy for a given query in Vitess. It is + // primarily a wrapper around Primitives (the execution instructions). Each + // Primitive may form a subtree, combining results from its children to + // achieve the overall query result. Plan struct { - Type PlanType // Type of plan - QueryType sqlparser.StatementType // Type of query - Original string // Original is the original query. - Instructions Primitive // Instructions contains the instructions needed to fulfil the query. - BindVarNeeds *sqlparser.BindVarNeeds // Stores BindVars needed to be provided as part of expression rewriting - Warnings []*query.QueryWarning // Warnings that need to be yielded every time this query runs - TablesUsed []string // TablesUsed is the list of tables that this plan will query + Type PlanType // Type of plan (Passthrough, Scatter, JoinOp, Complex, etc.) + QueryType sqlparser.StatementType // QueryType indicates the SQL statement type (SELECT, UPDATE, etc.) + Original string // Original holds the raw query text + Instructions Primitive // Instructions define how the query is executed. + BindVarNeeds *sqlparser.BindVarNeeds // BindVarNeeds lists required bind vars discovered during planning. + Warnings []*query.QueryWarning // Warnings accumulates any warnings generated for this plan. + TablesUsed []string // TablesUsed enumerates the tables this query accesses. + QueryHints sqlparser.QueryHints // QueryHints stores any SET_VAR hints that influenced plan generation. + ParamsCount uint16 // ParamsCount is the total number of bind parameters (?) in the query. - ExecCount uint64 // Count of times this plan was executed - ExecTime uint64 // Total execution time - ShardQueries uint64 // Total number of shard queries - RowsReturned uint64 // Total number of rows - RowsAffected uint64 // Total number of rows - Errors uint64 // Total number of errors + ExecCount uint64 // ExecCount is how many times this plan has been executed. + ExecTime uint64 // ExecTime is the total accumulated execution time in nanoseconds. + ShardQueries uint64 // ShardQueries is the total count of shard-level queries performed. + RowsReturned uint64 // RowsReturned is the total number of rows returned to clients. + RowsAffected uint64 // RowsAffected is the total number of rows affected by DML operations. + Errors uint64 // Errors is the total count of errors encountered during execution. + } + + // PlanKey identifies a plan uniquely based on keyspace, destination, query, + // SET_VAR comment, and collation. It is primarily used as a cache key. + PlanKey struct { + CurrentKeyspace string // CurrentKeyspace is the name of the keyspace associated with the plan. + Destination string // Destination specifies the shard or routing destination for the plan. + Query string // Query is the original or normalized SQL statement used to build the plan. + SetVarComment string // SetVarComment holds any embedded SET_VAR hints within the query. + Collation collations.ID // Collation is the character collation ID that governs string comparison. } ) @@ -77,62 +91,6 @@ func higher(a, b PlanType) PlanType { return b } -func NewPlan(query string, stmt sqlparser.Statement, primitive Primitive, bindVarNeeds *sqlparser.BindVarNeeds, tablesUsed []string) *Plan { - return &Plan{ - Type: getPlanType(primitive), - QueryType: sqlparser.ASTToStatementType(stmt), - Original: query, - Instructions: primitive, - BindVarNeeds: bindVarNeeds, - TablesUsed: tablesUsed, - } -} - -// MarshalJSON serializes the plan into a JSON representation. -func (p *Plan) MarshalJSON() ([]byte, error) { - var instructions *PrimitiveDescription - if p.Instructions != nil { - description := PrimitiveToPlanDescription(p.Instructions, nil) - instructions = &description - } - - marshalPlan := struct { - Type string - QueryType string - Original string `json:",omitempty"` - Instructions *PrimitiveDescription `json:",omitempty"` - ExecCount uint64 `json:",omitempty"` - ExecTime time.Duration `json:",omitempty"` - ShardQueries uint64 `json:",omitempty"` - RowsAffected uint64 `json:",omitempty"` - RowsReturned uint64 `json:",omitempty"` - Errors uint64 `json:",omitempty"` - TablesUsed []string `json:",omitempty"` - }{ - Type: p.Type.String(), - QueryType: p.QueryType.String(), - Original: p.Original, - Instructions: instructions, - ExecCount: atomic.LoadUint64(&p.ExecCount), - ExecTime: time.Duration(atomic.LoadUint64(&p.ExecTime)), - ShardQueries: atomic.LoadUint64(&p.ShardQueries), - RowsAffected: atomic.LoadUint64(&p.RowsAffected), - RowsReturned: atomic.LoadUint64(&p.RowsReturned), - Errors: atomic.LoadUint64(&p.Errors), - TablesUsed: p.TablesUsed, - } - - b := new(bytes.Buffer) - enc := json.NewEncoder(b) - enc.SetEscapeHTML(false) - err := enc.Encode(marshalPlan) - if err != nil { - return nil, err - } - - return b.Bytes(), nil -} - func (p PlanType) String() string { switch p { case PlanLocal: @@ -293,6 +251,79 @@ func getPlanTypeForUpsert(prim *Upsert) PlanType { return finalPlanType } +func (pk PlanKey) DebugString() string { + return fmt.Sprintf("CurrentKeyspace: %s, Destination: %s, Query: %s, SetVarComment: %s, Collation: %d", pk.CurrentKeyspace, pk.Destination, pk.Query, pk.SetVarComment, pk.Collation) +} + +func (pk PlanKey) Hash() theine.HashKey256 { + hasher := vthash.New256() + _, _ = hasher.WriteUint16(uint16(pk.Collation)) + _, _ = hasher.WriteString(pk.CurrentKeyspace) + _, _ = hasher.WriteString(pk.Destination) + _, _ = hasher.WriteString(pk.SetVarComment) + _, _ = hasher.WriteString(pk.Query) + + var planKey theine.HashKey256 + hasher.Sum(planKey[:0]) + return planKey +} + +func NewPlan(query string, stmt sqlparser.Statement, primitive Primitive, bindVarNeeds *sqlparser.BindVarNeeds, tablesUsed []string) *Plan { + return &Plan{ + Type: getPlanType(primitive), + QueryType: sqlparser.ASTToStatementType(stmt), + Original: query, + Instructions: primitive, + BindVarNeeds: bindVarNeeds, + TablesUsed: tablesUsed, + } +} + +// MarshalJSON serializes the plan into a JSON representation. +func (p *Plan) MarshalJSON() ([]byte, error) { + var instructions *PrimitiveDescription + if p.Instructions != nil { + description := PrimitiveToPlanDescription(p.Instructions, nil) + instructions = &description + } + + marshalPlan := struct { + Type string + QueryType string + Original string `json:",omitempty"` + Instructions *PrimitiveDescription `json:",omitempty"` + ExecCount uint64 `json:",omitempty"` + ExecTime time.Duration `json:",omitempty"` + ShardQueries uint64 `json:",omitempty"` + RowsAffected uint64 `json:",omitempty"` + RowsReturned uint64 `json:",omitempty"` + Errors uint64 `json:",omitempty"` + TablesUsed []string `json:",omitempty"` + }{ + Type: p.Type.String(), + QueryType: p.QueryType.String(), + Original: p.Original, + Instructions: instructions, + ExecCount: atomic.LoadUint64(&p.ExecCount), + ExecTime: time.Duration(atomic.LoadUint64(&p.ExecTime)), + ShardQueries: atomic.LoadUint64(&p.ShardQueries), + RowsAffected: atomic.LoadUint64(&p.RowsAffected), + RowsReturned: atomic.LoadUint64(&p.RowsReturned), + Errors: atomic.LoadUint64(&p.Errors), + TablesUsed: p.TablesUsed, + } + + b := new(bytes.Buffer) + enc := json.NewEncoder(b) + enc.SetEscapeHTML(false) + err := enc.Encode(marshalPlan) + if err != nil { + return nil, err + } + + return b.Bytes(), nil +} + // AddStats updates the plan execution statistics func (p *Plan) AddStats(execCount uint64, execTime time.Duration, shardQueries, rowsAffected, rowsReturned, errors uint64) { atomic.AddUint64(&p.ExecCount, execCount) diff --git a/go/vt/vtgate/engine/plan_description.go b/go/vt/vtgate/engine/plan_description.go index f19be96123e..84ffe9de0f9 100644 --- a/go/vt/vtgate/engine/plan_description.go +++ b/go/vt/vtgate/engine/plan_description.go @@ -39,7 +39,7 @@ type PrimitiveDescription struct { // Keyspace specifies the keyspace to send the query to. Keyspace *vindexes.Keyspace // TargetDestination specifies an explicit target destination to send the query to. - TargetDestination key.Destination + TargetDestination key.ShardDestination // TargetTabletType specifies an explicit target destination tablet type // this is only used in conjunction with TargetDestination TargetTabletType topodatapb.TabletType @@ -82,7 +82,7 @@ func (pd PrimitiveDescription) MarshalJSON() ([]byte, error) { } if pd.TargetDestination != nil { s := pd.TargetDestination.String() - dest := s[11:] // TODO: All these start with Destination. We should fix that instead if trimming it out here + dest := s[11:] // TODO: All these start with ShardDestination. We should fix that instead if trimming it out here if err := marshalAdd(prepend, buf, "TargetDestination", dest); err != nil { return nil, err diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index fe991a670b6..d1222475148 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -77,11 +77,11 @@ type ( // Keyspace ID level functions. ExecuteKeyspaceID(ctx context.Context, keyspace string, ksid []byte, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError, autocommit bool) (*sqltypes.Result, error) - // Resolver methods, from key.Destination to srvtopo.ResolvedShard. + // Resolver methods, from key.ShardDestination to srvtopo.ResolvedShard. // Will replace all of the Topo functions. - ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) + ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) - ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) + ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) ExecuteVSchema(ctx context.Context, keyspace string, vschemaDDL *sqlparser.AlterVschema) error @@ -269,6 +269,9 @@ type ( // txNeeded is a default implementation for Primitives that need transaction handling txNeeded struct{} + + // noFields is a default implementation for Primitives that do not return fields + noFields struct{} ) // Find will return the first Primitive that matches the evaluate function. If no match is found, nil will be returned @@ -303,3 +306,7 @@ func (noTxNeeded) NeedsTransaction() bool { func (txNeeded) NeedsTransaction() bool { return true } + +func (noFields) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + return &sqltypes.Result{}, nil +} diff --git a/go/vt/vtgate/engine/revert_migration.go b/go/vt/vtgate/engine/revert_migration.go index 2eb2af061a5..7d43056fad5 100644 --- a/go/vt/vtgate/engine/revert_migration.go +++ b/go/vt/vtgate/engine/revert_migration.go @@ -40,7 +40,7 @@ type RevertMigration struct { Keyspace *vindexes.Keyspace Stmt *sqlparser.RevertMigration Query string - TargetDestination key.Destination + TargetDestination key.ShardDestination } func (v *RevertMigration) description() PrimitiveDescription { diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 20ab7695f70..05329e74a02 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -37,7 +37,6 @@ import ( "vitess.io/vitess/go/vt/vtgate/vindexes" querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) var _ Primitive = (*Route)(nil) @@ -58,10 +57,6 @@ type Route struct { // Route does not need transaction handling noTxNeeded - // TargetTabletType specifies an explicit target destination tablet type - // this is only used in conjunction with TargetDestination - TargetTabletType topodatapb.TabletType - // Query specifies the query to be executed. Query string @@ -345,7 +340,7 @@ func (route *Route) GetFields(ctx context.Context, vcursor VCursor, bindVars map // If not find, then pick any shard. if rs == nil { - rss, _, err := vcursor.ResolveDestinations(ctx, route.Keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) + rss, _, err := vcursor.ResolveDestinations(ctx, route.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAnyShard{}}) if err != nil { return nil, err } @@ -438,7 +433,7 @@ func (route *Route) executeAfterLookup( bindVars map[string]*querypb.BindVariable, wantfields bool, ids []sqltypes.Value, - dest []key.Destination, + dest []key.ShardDestination, ) (*sqltypes.Result, error) { protoIds := make([]*querypb.Value, 0, len(ids)) for _, id := range ids { @@ -462,7 +457,7 @@ func (route *Route) streamExecuteAfterLookup( wantfields bool, callback func(*sqltypes.Result) error, ids []sqltypes.Value, - dest []key.Destination, + dest []key.ShardDestination, ) error { protoIds := make([]*querypb.Value, 0, len(ids)) for _, id := range ids { diff --git a/go/vt/vtgate/engine/routing.go b/go/vt/vtgate/engine/routing.go index dd6143f6aa4..ca7a814676e 100644 --- a/go/vt/vtgate/engine/routing.go +++ b/go/vt/vtgate/engine/routing.go @@ -116,7 +116,7 @@ type RoutingParameters struct { // TargetDestination specifies an explicit target destination to send the query to. // This will bypass the routing logic. - TargetDestination key.Destination // update `user[-]@replica`.user set .... + TargetDestination key.ShardDestination // update `user[-]@replica`.user set .... // Vindex specifies the vindex to be used. Vindex vindexes.Vindex @@ -194,7 +194,7 @@ func (rp *RoutingParameters) systemQuery(ctx context.Context, vcursor VCursor, b func (rp *RoutingParameters) routeInfoSchemaQuery(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) ([]*srvtopo.ResolvedShard, error) { defaultRoute := func() ([]*srvtopo.ResolvedShard, error) { ks := rp.Keyspace.Name - destinations, _, err := vcursor.ResolveDestinations(ctx, ks, nil, []key.Destination{key.DestinationAnyShard{}}) + destinations, _, err := vcursor.ResolveDestinations(ctx, ks, nil, []key.ShardDestination{key.DestinationAnyShard{}}) return destinations, vterrors.Wrapf(err, "failed to find information about keyspace `%s`", ks) } @@ -258,7 +258,7 @@ func (rp *RoutingParameters) routeInfoSchemaQuery(ctx context.Context, vcursor V } // we only have table_schema to work with - destinations, _, err := vcursor.ResolveDestinations(ctx, specifiedKS, nil, []key.Destination{key.DestinationAnyShard{}}) + destinations, _, err := vcursor.ResolveDestinations(ctx, specifiedKS, nil, []key.ShardDestination{key.DestinationAnyShard{}}) if err != nil { log.Errorf("failed to route information_schema query to keyspace [%s]", specifiedKS) bindVars[sqltypes.BvSchemaName] = sqltypes.StringBindVariable(specifiedKS) @@ -291,7 +291,7 @@ func (rp *RoutingParameters) routedTable(ctx context.Context, vcursor VCursor, b return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "cannot send the query to multiple keyspace due to different table_name: %s, %s", routedKs.Name, routedTable.Keyspace.Name) } - shards, _, err := vcursor.ResolveDestinations(ctx, routedTable.Keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) + shards, _, err := vcursor.ResolveDestinations(ctx, routedTable.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAnyShard{}}) bindVars[tblBvName] = sqltypes.StringBindVariable(routedTable.Name.String()) if tableSchema != "" { setReplaceSchemaName(bindVars) @@ -305,7 +305,7 @@ func (rp *RoutingParameters) routedTable(ctx context.Context, vcursor VCursor, b } func (rp *RoutingParameters) anyShard(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) + rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAnyShard{}}) if err != nil { return nil, nil, err } @@ -317,7 +317,7 @@ func (rp *RoutingParameters) anyShard(ctx context.Context, vcursor VCursor, bind } func (rp *RoutingParameters) unsharded(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.Destination{key.DestinationAllShards{}}) + rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.ShardDestination{key.DestinationAllShards{}}) if err != nil { return nil, nil, err } @@ -331,8 +331,8 @@ func (rp *RoutingParameters) unsharded(ctx context.Context, vcursor VCursor, bin return rss, multiBindVars, nil } -func (rp *RoutingParameters) byDestination(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, destination key.Destination) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.Destination{destination}) +func (rp *RoutingParameters) byDestination(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, destination key.ShardDestination) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { + rss, _, err := vcursor.ResolveDestinations(ctx, rp.Keyspace.Name, nil, []key.ShardDestination{destination}) if err != nil { return nil, nil, err } diff --git a/go/vt/vtgate/engine/send.go b/go/vt/vtgate/engine/send.go index 4655f680675..01a87bd76b9 100644 --- a/go/vt/vtgate/engine/send.go +++ b/go/vt/vtgate/engine/send.go @@ -39,7 +39,7 @@ type Send struct { Keyspace *vindexes.Keyspace // TargetDestination specifies an explicit target destination to send the query to. - TargetDestination key.Destination + TargetDestination key.ShardDestination // Query specifies the query to be executed. Query string @@ -125,7 +125,7 @@ func (s *Send) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[str } func (s *Send) checkAndReturnShards(ctx context.Context, vcursor VCursor) ([]*srvtopo.ResolvedShard, error) { - rss, _, err := vcursor.ResolveDestinations(ctx, s.Keyspace.Name, nil, []key.Destination{s.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, s.Keyspace.Name, nil, []key.ShardDestination{s.TargetDestination}) if err != nil { return nil, err } @@ -185,6 +185,9 @@ func (s *Send) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars m // GetFields implements Primitive interface func (s *Send) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + if s.IsDML || s.IsDDL { + return &sqltypes.Result{}, nil + } qr, err := vcursor.ExecutePrimitive(ctx, s, bindVars, false) if err != nil { return nil, err diff --git a/go/vt/vtgate/engine/send_test.go b/go/vt/vtgate/engine/send_test.go index e44b6c30a0d..68bbdb10726 100644 --- a/go/vt/vtgate/engine/send_test.go +++ b/go/vt/vtgate/engine/send_test.go @@ -35,7 +35,7 @@ func TestSendTable(t *testing.T) { testName string sharded bool shards []string - destination key.Destination + destination key.ShardDestination expectedQueryLog []string expectedError string isDML bool @@ -174,7 +174,7 @@ func TestSendTable_StreamExecute(t *testing.T) { testName string sharded bool shards []string - destination key.Destination + destination key.ShardDestination expectedQueryLog []string expectedError string isDML bool @@ -305,16 +305,27 @@ func TestSendGetFields(t *testing.T) { }, Query: "dummy_query", TargetDestination: key.DestinationAllShards{}, - IsDML: true, SingleShardOnly: false, } vc := &loggingVCursor{shards: []string{"-20", "20-"}, results: results} - qr, err := send.GetFields(context.Background(), vc, map[string]*querypb.BindVariable{}) - require.NoError(t, err) - vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, - `ExecuteMultiShard ks.-20: dummy_query {} ks.20-: dummy_query {} true false`, + + t.Run("GetFields - not a dml query", func(t *testing.T) { + qr, err := send.GetFields(context.Background(), vc, map[string]*querypb.BindVariable{}) + require.NoError(t, err) + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ExecuteMultiShard ks.-20: dummy_query {} ks.20-: dummy_query {} false false`, + }) + require.Nil(t, qr.Rows) + require.Equal(t, 4, len(qr.Fields)) + }) + + vc.Rewind() + t.Run("GetFields - a dml query", func(t *testing.T) { + send.IsDML = true + qr, err := send.GetFields(context.Background(), vc, map[string]*querypb.BindVariable{}) + require.NoError(t, err) + require.Empty(t, qr.Fields) + vc.ExpectLog(t, nil) }) - require.Nil(t, qr.Rows) - require.Equal(t, 4, len(qr.Fields)) } diff --git a/go/vt/vtgate/engine/sequential.go b/go/vt/vtgate/engine/sequential.go index ecf74d663a2..56be78c12e1 100644 --- a/go/vt/vtgate/engine/sequential.go +++ b/go/vt/vtgate/engine/sequential.go @@ -21,14 +21,14 @@ import ( "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/vterrors" ) // Sequential Primitive is used to execute DML statements in a fixed order. // Any failure, stops the execution and returns. type Sequential struct { txNeeded + noFields + Sources []Primitive } @@ -92,11 +92,6 @@ func (s *Sequential) TryStreamExecute(ctx context.Context, vcursor VCursor, bind return callback(qr) } -// GetFields fetches the field info. -func (s *Sequential) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unreachable code for Sequential engine") -} - // Inputs returns the input primitives for this func (s *Sequential) Inputs() ([]Primitive, []map[string]any) { return s.Sources, nil diff --git a/go/vt/vtgate/engine/set.go b/go/vt/vtgate/engine/set.go index f0de330cbed..32a297404f3 100644 --- a/go/vt/vtgate/engine/set.go +++ b/go/vt/vtgate/engine/set.go @@ -41,6 +41,7 @@ type ( // Set contains the instructions to perform set. Set struct { noTxNeeded + noFields Ops []SetOp Input Primitive @@ -68,7 +69,7 @@ type ( SysVarCheckAndIgnore struct { Name string Keyspace *vindexes.Keyspace - TargetDestination key.Destination `json:",omitempty"` + TargetDestination key.ShardDestination `json:",omitempty"` Expr string } @@ -76,7 +77,7 @@ type ( SysVarReservedConn struct { Name string Keyspace *vindexes.Keyspace - TargetDestination key.Destination `json:",omitempty"` + TargetDestination key.ShardDestination `json:",omitempty"` Expr string SupportSetVar bool } @@ -142,11 +143,6 @@ func (s *Set) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars ma return callback(result) } -// GetFields implements the Primitive interface method. -func (s *Set) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return &sqltypes.Result{}, nil -} - // Inputs implements the Primitive interface func (s *Set) Inputs() ([]Primitive, []map[string]any) { return []Primitive{s.Input}, nil @@ -234,7 +230,7 @@ func (svci *SysVarCheckAndIgnore) VariableName() string { // Execute implements the SetOp interface method func (svci *SysVarCheckAndIgnore) Execute(ctx context.Context, vcursor VCursor, env *evalengine.ExpressionEnv) error { - rss, _, err := vcursor.ResolveDestinations(ctx, svci.Keyspace.Name, nil, []key.Destination{svci.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, svci.Keyspace.Name, nil, []key.ShardDestination{svci.TargetDestination}) if err != nil { return err } @@ -276,7 +272,7 @@ func (svs *SysVarReservedConn) VariableName() string { func (svs *SysVarReservedConn) Execute(ctx context.Context, vcursor VCursor, env *evalengine.ExpressionEnv) error { // For those running on advanced vitess settings. if svs.TargetDestination != nil { - rss, _, err := vcursor.ResolveDestinations(ctx, svs.Keyspace.Name, nil, []key.Destination{svs.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, svs.Keyspace.Name, nil, []key.ShardDestination{svs.TargetDestination}) if err != nil { return err } @@ -324,7 +320,7 @@ func (svs *SysVarReservedConn) checkAndUpdateSysVar(ctx context.Context, vcursor if svs.Name == "sql_mode" { sysVarExprValidationQuery = fmt.Sprintf("select @@%s orig, %s new", svs.Name, svs.Expr) } - rss, _, err := vcursor.ResolveDestinations(ctx, svs.Keyspace.Name, nil, []key.Destination{key.DestinationKeyspaceID{0}}) + rss, _, err := vcursor.ResolveDestinations(ctx, svs.Keyspace.Name, nil, []key.ShardDestination{key.DestinationKeyspaceID{0}}) if err != nil { return false, err } @@ -356,7 +352,7 @@ func (svs *SysVarReservedConn) checkAndUpdateSysVar(ctx context.Context, vcursor // If the condition below is true, we want to use reserved connection instead of SET_VAR query hint. // MySQL supports SET_VAR only in MySQL80 and for a limited set of system variables. - if !svs.SupportSetVar || s == "''" || !vcursor.CanUseSetVar() { + if !svs.SupportSetVar || !vcursor.CanUseSetVar() { vcursor.Session().NeedsReservedConn() return true, nil } diff --git a/go/vt/vtgate/engine/set_test.go b/go/vt/vtgate/engine/set_test.go index 0677ee40bd8..688c47c6f47 100644 --- a/go/vt/vtgate/engine/set_test.go +++ b/go/vt/vtgate/engine/set_test.go @@ -452,7 +452,7 @@ func TestSetTable(t *testing.T) { `ResolveDestinations ks [] Destinations:DestinationKeyspaceID(00)`, `ExecuteMultiShard ks.-20: select @@sql_mode orig, '' new {} false false`, "SysVar set with (sql_mode,'')", - "Needs Reserved Conn", + "SET_VAR can be used", }, qr: []*sqltypes.Result{sqltypes.MakeTestResult(sqltypes.MakeTestFields("orig|new", "varchar|varchar"), "a|", @@ -478,7 +478,7 @@ func TestSetTable(t *testing.T) { "|a", )}, }, { - testName: "sql_mode change to empty - non empty orig - MySQL80 - should use reserved conn", + testName: "sql_mode change to empty - non empty orig - MySQL80 - set_var allowed", mysqlVersion: "8.0.0", setOps: []SetOp{ &SysVarReservedConn{ @@ -492,7 +492,7 @@ func TestSetTable(t *testing.T) { `ResolveDestinations ks [] Destinations:DestinationKeyspaceID(00)`, `ExecuteMultiShard ks.-20: select @@sql_mode orig, '' new {} false false`, "SysVar set with (sql_mode,'')", - "Needs Reserved Conn", + "SET_VAR can be used", }, qr: []*sqltypes.Result{sqltypes.MakeTestResult(sqltypes.MakeTestFields("orig|new", "varchar|varchar"), "a|", diff --git a/go/vt/vtgate/engine/singlerow.go b/go/vt/vtgate/engine/singlerow.go index 35ecdaff90b..8a3fff2cacd 100644 --- a/go/vt/vtgate/engine/singlerow.go +++ b/go/vt/vtgate/engine/singlerow.go @@ -29,6 +29,7 @@ var _ Primitive = (*SingleRow)(nil) type SingleRow struct { noInputs noTxNeeded + noFields } // RouteType returns a description of the query routing type used by the primitive @@ -65,11 +66,6 @@ func (s *SingleRow) TryStreamExecute(ctx context.Context, vcursor VCursor, bindV return callback(res) } -// GetFields fetches the field info. -func (s *SingleRow) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return &sqltypes.Result{}, nil -} - func (s *SingleRow) description() PrimitiveDescription { return PrimitiveDescription{ OperatorType: "SingleRow", diff --git a/go/vt/vtgate/engine/update.go b/go/vt/vtgate/engine/update.go index 6e153692089..9e878ffce20 100644 --- a/go/vt/vtgate/engine/update.go +++ b/go/vt/vtgate/engine/update.go @@ -23,8 +23,6 @@ import ( "vitess.io/vitess/go/vt/vtgate/evalengine" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/srvtopo" "vitess.io/vitess/go/vt/vtgate/vindexes" @@ -42,8 +40,8 @@ type VindexValues struct { // Update represents the instructions to perform an update. type Update struct { - // Update does not take inputs noInputs + noFields *DML @@ -83,11 +81,6 @@ func (upd *Update) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVa } -// GetFields fetches the field info. -func (upd *Update) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, fmt.Errorf("BUG: unreachable code for %q", upd.Query) -} - // updateVindexEntries performs an update when a vindex is being modified // by the statement. // Note: the commit order may be different from the DML order because it's possible @@ -223,10 +216,9 @@ func (upd *Update) description() PrimitiveDescription { } return PrimitiveDescription{ - OperatorType: "Update", - Keyspace: upd.Keyspace, - Variant: upd.Opcode.String(), - TargetTabletType: topodatapb.TabletType_PRIMARY, - Other: other, + OperatorType: "Update", + Keyspace: upd.Keyspace, + Variant: upd.Opcode.String(), + Other: other, } } diff --git a/go/vt/vtgate/engine/upsert.go b/go/vt/vtgate/engine/upsert.go index 2e42452a7a4..58d996b2b2b 100644 --- a/go/vt/vtgate/engine/upsert.go +++ b/go/vt/vtgate/engine/upsert.go @@ -22,7 +22,6 @@ import ( "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/vterrors" ) @@ -33,6 +32,8 @@ var _ Primitive = (*Upsert)(nil) // if there is `Duplicate Key` error, it executes the update primitive. type Upsert struct { txNeeded + noFields + Upserts []upsert } @@ -70,11 +71,6 @@ func (u *Upsert) GetTableName() string { return "" } -// GetFields implements Primitive interface type. -func (u *Upsert) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return nil, vterrors.VT13001("unexpected to receive GetFields call for insert on duplicate key update query") -} - // TryExecute implements Primitive interface type. func (u *Upsert) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { result := &sqltypes.Result{} @@ -131,7 +127,6 @@ func (u *Upsert) Inputs() ([]Primitive, []map[string]any) { func (u *Upsert) description() PrimitiveDescription { return PrimitiveDescription{ - OperatorType: "Upsert", - TargetTabletType: topodatapb.TabletType_PRIMARY, + OperatorType: "Upsert", } } diff --git a/go/vt/vtgate/engine/vindex_func.go b/go/vt/vtgate/engine/vindex_func.go index 13507631716..0c43449d4ca 100644 --- a/go/vt/vtgate/engine/vindex_func.go +++ b/go/vt/vtgate/engine/vindex_func.go @@ -156,7 +156,7 @@ func (vf *VindexFunc) mapVindex(ctx context.Context, vcursor VCursor, bindVars m if vcursor.GetKeyspace() == "" { return nil, vterrors.VT09005() } - resolvedShards, _, err := vcursor.ResolveDestinations(ctx, vcursor.GetKeyspace(), nil, []key.Destination{d}) + resolvedShards, _, err := vcursor.ResolveDestinations(ctx, vcursor.GetKeyspace(), nil, []key.ShardDestination{d}) if err != nil { return nil, err } diff --git a/go/vt/vtgate/engine/vindex_func_test.go b/go/vt/vtgate/engine/vindex_func_test.go index 2805d85cdd3..602eb7c52bc 100644 --- a/go/vt/vtgate/engine/vindex_func_test.go +++ b/go/vt/vtgate/engine/vindex_func_test.go @@ -43,8 +43,8 @@ func (*uvindex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][] panic("unimplemented") } -func (v *uvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - destinations := make([]key.Destination, 0, len(ids)) +func (v *uvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + destinations := make([]key.ShardDestination, 0, len(ids)) dkid := []byte("foo") for i := 0; i < len(ids); i++ { if v.matchkr { @@ -75,12 +75,12 @@ func (*nvindex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][] panic("unimplemented") } -func (v *nvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - destinations := make([]key.Destination, 0) +func (v *nvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + destinations := make([]key.ShardDestination, 0) for i := 0; i < len(ids); i++ { if v.matchid { destinations = append(destinations, - []key.Destination{ + []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("foo"), []byte("bar"), @@ -88,7 +88,7 @@ func (v *nvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqlty }...) } else if v.matchkr { destinations = append(destinations, - []key.Destination{ + []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{ Start: []byte{0x40}, @@ -97,7 +97,7 @@ func (v *nvindex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqlty }, }...) } else { - destinations = append(destinations, []key.Destination{key.DestinationNone{}}...) + destinations = append(destinations, []key.ShardDestination{key.DestinationNone{}}...) } } return destinations, nil diff --git a/go/vt/vtgate/engine/vindex_lookup.go b/go/vt/vtgate/engine/vindex_lookup.go index 2e0e2857498..f9b491ae878 100644 --- a/go/vt/vtgate/engine/vindex_lookup.go +++ b/go/vt/vtgate/engine/vindex_lookup.go @@ -99,7 +99,7 @@ func (vr *VindexLookup) TryExecute(ctx context.Context, vcursor VCursor, bindVar return vr.SendTo.executeAfterLookup(ctx, vcursor, bindVars, wantfields, ids, dest) } -func (vr *VindexLookup) mapVindexToDestination(ids []sqltypes.Value, results []*sqltypes.Result, bindVars map[string]*querypb.BindVariable) ([]key.Destination, error) { +func (vr *VindexLookup) mapVindexToDestination(ids []sqltypes.Value, results []*sqltypes.Result, bindVars map[string]*querypb.BindVariable) ([]key.ShardDestination, error) { dest, err := vr.Vindex.MapResult(ids, results) if err != nil { return nil, err diff --git a/go/vt/vtgate/engine/vstream.go b/go/vt/vtgate/engine/vstream.go index 5d3c92c4d98..aafc500aba0 100644 --- a/go/vt/vtgate/engine/vstream.go +++ b/go/vt/vtgate/engine/vstream.go @@ -39,7 +39,7 @@ type VStream struct { noInputs Keyspace *vindexes.Keyspace - TargetDestination key.Destination + TargetDestination key.ShardDestination TableName string Position string Limit int @@ -67,7 +67,7 @@ func (v *VStream) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[ // TryStreamExecute implements the Primitive interface func (v *VStream) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error { - rss, _, err := vcursor.ResolveDestinations(ctx, v.Keyspace.Name, nil, []key.Destination{v.TargetDestination}) + rss, _, err := vcursor.ResolveDestinations(ctx, v.Keyspace.Name, nil, []key.ShardDestination{v.TargetDestination}) if err != nil { return err } diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 99805a0de5e..89103d4f160 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -23,6 +23,7 @@ import ( "fmt" "io" "net/http" + "sort" "strings" "sync" "sync/atomic" @@ -30,9 +31,6 @@ import ( "github.com/spf13/pflag" - vschemapb "vitess.io/vitess/go/vt/proto/vschema" - "vitess.io/vitess/go/vt/vtgate/dynamicconfig" - "vitess.io/vitess/go/acl" "vitess.io/vitess/go/cache/theine" "vitess.io/vitess/go/mysql/capabilities" @@ -49,6 +47,7 @@ import ( binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/servenv" @@ -58,6 +57,7 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/dynamicconfig" "vitess.io/vitess/go/vt/vtgate/engine" "vitess.io/vitess/go/vt/vtgate/evalengine" econtext "vitess.io/vitess/go/vt/vtgate/executorcontext" @@ -67,7 +67,6 @@ import ( "vitess.io/vitess/go/vt/vtgate/vindexes" "vitess.io/vitess/go/vt/vtgate/vschemaacl" "vitess.io/vitess/go/vt/vtgate/vtgateservice" - "vitess.io/vitess/go/vt/vthash" ) var ( @@ -233,14 +232,22 @@ func NewExecutor( } // Execute executes a non-streaming query. -func (e *Executor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (result *sqltypes.Result, err error) { +func (e *Executor) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + method string, + safeSession *econtext.SafeSession, + sql string, + bindVars map[string]*querypb.BindVariable, + prepared bool, +) (result *sqltypes.Result, err error) { span, ctx := trace.NewSpan(ctx, "executor.Execute") span.Annotate("method", method) trace.AnnotateSQL(span, sqlparser.Preview(sql)) defer span.Finish() logStats := logstats.NewLogStats(ctx, method, sql, safeSession.GetSessionUUID(), bindVars, streamlog.GetQueryLogConfig()) - stmtType, result, err := e.execute(ctx, mysqlCtx, safeSession, sql, bindVars, logStats) + stmtType, result, err := e.execute(ctx, mysqlCtx, safeSession, sql, bindVars, prepared, logStats) logStats.Error = err if result == nil { saveSessionStats(safeSession, stmtType, 0, 0, err) @@ -382,7 +389,7 @@ func (e *Executor) StreamExecute( return err } - err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, logStats, resultHandler, srr.storeResultStats) + err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, false, logStats, resultHandler, srr.storeResultStats) logStats.Error = err saveSessionStats(safeSession, srr.stmtType, srr.rowsAffected, srr.rowsReturned, err) @@ -434,11 +441,11 @@ func saveSessionStats(safeSession *econtext.SafeSession, stmtType sqlparser.Stat } } -func (e *Executor) execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) (sqlparser.StatementType, *sqltypes.Result, error) { +func (e *Executor) execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, prepared bool, logStats *logstats.LogStats) (sqlparser.StatementType, *sqltypes.Result, error) { var err error var qr *sqltypes.Result var stmtType sqlparser.StatementType - err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, logStats, func(ctx context.Context, plan *engine.Plan, vc *econtext.VCursorImpl, bindVars map[string]*querypb.BindVariable, time time.Time) error { + err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, prepared, logStats, func(ctx context.Context, plan *engine.Plan, vc *econtext.VCursorImpl, bindVars map[string]*querypb.BindVariable, time time.Time) error { stmtType = plan.QueryType qr, err = e.executePlan(ctx, safeSession, plan, vc, bindVars, logStats, time) return err @@ -1087,48 +1094,100 @@ func (e *Executor) SaveVSchema(vschema *vindexes.VSchema, stats *VSchemaStats) { } // ParseDestinationTarget parses destination target string and sets default keyspace if possible. -func (e *Executor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) { +func (e *Executor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.ShardDestination, error) { return econtext.ParseDestinationTarget(targetString, defaultTabletType, e.VSchema()) } -// getPlan computes the plan for the given query. If one is in -// the cache, it reuses it. -func (e *Executor) getPlan( +func (e *Executor) fetchOrCreatePlan( ctx context.Context, - vcursor *econtext.VCursorImpl, - query string, - stmt sqlparser.Statement, - comments sqlparser.MarginComments, + safeSession *econtext.SafeSession, + sql string, bindVars map[string]*querypb.BindVariable, - reservedVars *sqlparser.ReservedVars, - allowParameterization bool, + parameterize bool, + preparedPlan bool, logStats *logstats.LogStats, -) (*engine.Plan, error) { +) ( + plan *engine.Plan, vcursor *econtext.VCursorImpl, stmt sqlparser.Statement, err error) { if e.VSchema() == nil { - return nil, vterrors.VT13001("vschema not initialized") + return nil, nil, nil, vterrors.VT13001("vschema not initialized") } - qh, err := sqlparser.BuildQueryHints(stmt) - if err != nil { - return nil, err + query, comments := sqlparser.SplitMarginComments(sql) + vcursor, _ = econtext.NewVCursorImpl(safeSession, comments, e, logStats, e.vm, e.VSchema(), e.resolver.resolver, e.serv, nullResultsObserver{}, e.vConfig) + + var setVarComment string + if e.vConfig.SetVarEnabled { + setVarComment = vcursor.PrepareSetVarComment() + } + + var planKey engine.PlanKey + if preparedPlan { + planKey = buildPlanKey(ctx, vcursor, query, setVarComment) + plan, logStats.CachedPlan = e.plans.Get(planKey.Hash(), e.epoch.Load()) } + + if plan == nil { + plan, logStats.CachedPlan, stmt, err = e.getCachedOrBuildPlan(ctx, vcursor, query, bindVars, setVarComment, parameterize, planKey) + if err != nil { + return nil, nil, nil, err + } + } + + qh := plan.QueryHints vcursor.SetIgnoreMaxMemoryRows(qh.IgnoreMaxMemoryRows) vcursor.SetConsolidator(qh.Consolidator) vcursor.SetWorkloadName(qh.Workload) - vcursor.UpdateForeignKeyChecksState(qh.ForeignKeyChecks) vcursor.SetPriority(qh.Priority) vcursor.SetExecQueryTimeout(qh.Timeout) - setVarComment, err := prepareSetVarComment(vcursor, stmt) + logStats.SQL = comments.Leading + plan.Original + comments.Trailing + logStats.BindVariables = sqltypes.CopyBindVariables(bindVars) + + return plan, vcursor, stmt, nil +} + +func (e *Executor) getCachedOrBuildPlan( + ctx context.Context, + vcursor *econtext.VCursorImpl, + query string, + bindVars map[string]*querypb.BindVariable, + setVarComment string, + parameterize bool, + planKey engine.PlanKey, +) (plan *engine.Plan, cached bool, stmt sqlparser.Statement, err error) { + stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser()) if err != nil { - return nil, err + return nil, false, nil, err + } + + defer func() { + if err == nil { + vcursor.CheckForReservedConnection(setVarComment, stmt) + } + }() + + qh, err := sqlparser.BuildQueryHints(stmt) + if err != nil { + return nil, false, nil, err + } + + if qh.ForeignKeyChecks == nil { + qh.ForeignKeyChecks = vcursor.SafeSession.ForeignKeyChecks() } + vcursor.SetForeignKeyCheckState(qh.ForeignKeyChecks) - // Normalize if possible - shouldNormalize := e.canNormalizeStatement(stmt, setVarComment) - parameterize := allowParameterization && shouldNormalize + paramsCount := uint16(0) + preparedPlan := planKey.Query != "" + if preparedPlan { + // We need to count the number of arguments in the statement before we plan the query. + // Planning could add additional arguments to the statement. + paramsCount = countArguments(stmt) + if bindVars == nil { + bindVars = make(map[string]*querypb.BindVariable) + } + } - rewriteASTResult, err := sqlparser.PrepareAST( + rewriteASTResult, err := sqlparser.Normalize( stmt, reservedVars, bindVars, @@ -1137,31 +1196,70 @@ func (e *Executor) getPlan( vcursor.SafeSession.GetSelectLimit(), setVarComment, vcursor.GetSystemVariablesCopy(), - vcursor.GetForeignKeyChecksState(), + qh.ForeignKeyChecks, vcursor, ) if err != nil { - return nil, err + return nil, false, nil, err } stmt = rewriteASTResult.AST bindVarNeeds := rewriteASTResult.BindVarNeeds - if shouldNormalize { + if rewriteASTResult.UpdateQueryFromAST && !preparedPlan { query = sqlparser.String(stmt) } - logStats.SQL = comments.Leading + query + comments.Trailing - logStats.BindVariables = sqltypes.CopyBindVariables(bindVars) + planCachable := sqlparser.CachePlan(stmt) && vcursor.CachePlan() + if planCachable { + if !preparedPlan { + // build Plan key + planKey = buildPlanKey(ctx, vcursor, query, setVarComment) + } + plan, cached, err = e.plans.GetOrLoad(planKey.Hash(), e.epoch.Load(), func() (*engine.Plan, error) { + return e.buildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds, qh, paramsCount) + }) + return plan, cached, stmt, err + } + plan, err = e.buildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds, qh, paramsCount) + return plan, false, stmt, err +} + +func buildPlanKey(ctx context.Context, vcursor *econtext.VCursorImpl, query string, setVarComment string) engine.PlanKey { + allDest := getDestinations(ctx, vcursor) - return e.cacheAndBuildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds, logStats) + return engine.PlanKey{ + CurrentKeyspace: vcursor.GetKeyspace(), + Destination: strings.Join(allDest, ","), + Query: query, + SetVarComment: setVarComment, + Collation: vcursor.ConnCollation(), + } } -func (e *Executor) hashPlan(ctx context.Context, vcursor *econtext.VCursorImpl, query string) PlanCacheKey { - hasher := vthash.New256() - vcursor.KeyForPlan(ctx, query, hasher) +func getDestinations(ctx context.Context, vcursor *econtext.VCursorImpl) []string { + currDest := vcursor.ShardDestination() + if currDest == nil { + return nil + } - var planKey PlanCacheKey - hasher.Sum(planKey[:0]) - return planKey + switch currDest.(type) { + case key.DestinationKeyspaceID, key.DestinationKeyspaceIDs: + // these need to be resolved to shards + default: + return []string{currDest.String()} + } + + resolved, _, err := vcursor.ResolveDestinations(ctx, vcursor.GetKeyspace(), nil, []key.ShardDestination{currDest}) + if err != nil || len(resolved) <= 0 { + return nil + } + + shards := make([]string, len(resolved)) + for i := 0; i < len(shards); i++ { + shards[i] = resolved[i].Target.GetShard() + } + sort.Strings(shards) + + return shards } func (e *Executor) buildStatement( @@ -1171,73 +1269,22 @@ func (e *Executor) buildStatement( stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, bindVarNeeds *sqlparser.BindVarNeeds, + qh sqlparser.QueryHints, + paramsCount uint16, ) (*engine.Plan, error) { plan, err := planbuilder.BuildFromStmt(ctx, query, stmt, reservedVars, vcursor, bindVarNeeds, e.ddlConfig) if err != nil { return nil, err } + plan.ParamsCount = paramsCount plan.Warnings = vcursor.GetAndEmptyWarnings() + plan.QueryHints = qh err = e.checkThatPlanIsValid(stmt, plan) return plan, err } -func (e *Executor) cacheAndBuildStatement( - ctx context.Context, - vcursor *econtext.VCursorImpl, - query string, - stmt sqlparser.Statement, - reservedVars *sqlparser.ReservedVars, - bindVarNeeds *sqlparser.BindVarNeeds, - logStats *logstats.LogStats, -) (*engine.Plan, error) { - planCachable := sqlparser.CachePlan(stmt) && vcursor.CachePlan() - if planCachable { - planKey := e.hashPlan(ctx, vcursor, query) - - var plan *engine.Plan - var err error - plan, logStats.CachedPlan, err = e.plans.GetOrLoad(planKey, e.epoch.Load(), func() (*engine.Plan, error) { - return e.buildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds) - }) - return plan, err - } - return e.buildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds) -} - -func (e *Executor) canNormalizeStatement(stmt sqlparser.Statement, setVarComment string) bool { - return sqlparser.CanNormalize(stmt) || setVarComment != "" -} - -func prepareSetVarComment(vcursor *econtext.VCursorImpl, stmt sqlparser.Statement) (string, error) { - if vcursor == nil || vcursor.Session().InReservedConn() { - return "", nil - } - - if !vcursor.Session().HasSystemVariables() { - return "", nil - } - - switch stmt.(type) { - // If the statement is a transaction statement or a set no reserved connection / SET_VAR is needed - case *sqlparser.Begin, *sqlparser.Commit, *sqlparser.Rollback, *sqlparser.Savepoint, - *sqlparser.SRollback, *sqlparser.Release, *sqlparser.Set, *sqlparser.Show: - return "", nil - case sqlparser.SupportOptimizerHint: - break - default: - vcursor.NeedsReservedConn() - return "", nil - } - - var res strings.Builder - vcursor.Session().GetSystemVariables(func(k, v string) { - res.WriteString(fmt.Sprintf("SET_VAR(%s = %s) ", k, v)) - }) - return strings.TrimSpace(res.String()), nil -} - func (e *Executor) debugCacheEntries() (items map[string]*engine.Plan) { items = make(map[string]*engine.Plan) e.ForEachPlan(func(plan *engine.Plan) bool { @@ -1359,9 +1406,9 @@ func isValidPayloadSize(query string) bool { } // Prepare executes a prepare statements. -func (e *Executor) Prepare(ctx context.Context, method string, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (fld []*querypb.Field, err error) { - logStats := logstats.NewLogStats(ctx, method, sql, safeSession.GetSessionUUID(), bindVars, streamlog.GetQueryLogConfig()) - fld, err = e.prepare(ctx, safeSession, sql, bindVars, logStats) +func (e *Executor) Prepare(ctx context.Context, method string, safeSession *econtext.SafeSession, sql string) (fld []*querypb.Field, paramsCount uint16, err error) { + logStats := logstats.NewLogStats(ctx, method, sql, safeSession.GetSessionUUID(), nil, streamlog.GetQueryLogConfig()) + fld, paramsCount, err = e.prepare(ctx, safeSession, sql, logStats) logStats.Error = err // The mysql plugin runs an implicit rollback whenever a connection closes. @@ -1375,21 +1422,16 @@ func (e *Executor) Prepare(ctx context.Context, method string, safeSession *econ err = errorTransform.TransformError(err) err = vterrors.TruncateError(err, truncateErrorLen) - return fld, err + return fld, paramsCount, err } -func (e *Executor) prepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) { +func (e *Executor) prepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, logStats *logstats.LogStats) ([]*querypb.Field, uint16, error) { // Start an implicit transaction if necessary. if !safeSession.Autocommit && !safeSession.InTransaction() { if err := e.txConn.Begin(ctx, safeSession, nil); err != nil { - return nil, err + return nil, 0, err } } - - if bindVars == nil { - bindVars = make(map[string]*querypb.BindVariable) - } - stmtType := sqlparser.Preview(sql) logStats.StmtType = stmtType.String() @@ -1405,13 +1447,14 @@ func (e *Executor) prepare(ctx context.Context, safeSession *econtext.SafeSessio } switch stmtType { - case sqlparser.StmtSelect, sqlparser.StmtShow: - return e.handlePrepare(ctx, safeSession, sql, bindVars, logStats) - case sqlparser.StmtDDL, sqlparser.StmtBegin, sqlparser.StmtCommit, sqlparser.StmtRollback, sqlparser.StmtSet, sqlparser.StmtInsert, sqlparser.StmtReplace, sqlparser.StmtUpdate, sqlparser.StmtDelete, + case sqlparser.StmtSelect, sqlparser.StmtShow, + sqlparser.StmtInsert, sqlparser.StmtReplace, sqlparser.StmtUpdate, sqlparser.StmtDelete: + return e.handlePrepare(ctx, safeSession, sql, logStats) + case sqlparser.StmtDDL, sqlparser.StmtBegin, sqlparser.StmtCommit, sqlparser.StmtRollback, sqlparser.StmtSet, sqlparser.StmtUse, sqlparser.StmtOther, sqlparser.StmtAnalyze, sqlparser.StmtComment, sqlparser.StmtExplain, sqlparser.StmtFlush, sqlparser.StmtKill: - return nil, nil + return nil, 0, nil } - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unrecognized prepare statement: %s", sql) + return nil, 0, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unrecognized prepare statement: %s", sql) } func (e *Executor) initVConfig(warnOnShardedOnly bool, pv plancontext.PlannerVersion) { @@ -1445,29 +1488,43 @@ func (e *Executor) initVConfig(warnOnShardedOnly bool, pv plancontext.PlannerVer } } -func (e *Executor) handlePrepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) { - query, comments := sqlparser.SplitMarginComments(sql) - - vcursor, _ := econtext.NewVCursorImpl(safeSession, comments, e, logStats, e.vm, e.VSchema(), e.resolver.resolver, e.serv, nullResultsObserver{}, e.vConfig) +func countArguments(statement sqlparser.Statement) (paramsCount uint16) { + _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { + switch node := node.(type) { + case *sqlparser.Argument: + if strings.HasPrefix(node.Name, "v") { + paramsCount++ + } + } + return true, nil + }, statement) + return +} - stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser()) - if err != nil { - return nil, err +func prepareBindVars(paramsCount uint16) map[string]*querypb.BindVariable { + bindVars := make(map[string]*querypb.BindVariable, paramsCount) + for i := range paramsCount { + parameterID := fmt.Sprintf("v%d", i+1) + bindVars[parameterID] = &querypb.BindVariable{} } + return bindVars +} - plan, err := e.getPlan(ctx, vcursor, sql, stmt, comments, bindVars, reservedVars /* parameterize */, false, logStats) +func (e *Executor) handlePrepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, logStats *logstats.LogStats) ([]*querypb.Field, uint16, error) { + plan, vcursor, _, err := e.fetchOrCreatePlan(ctx, safeSession, sql, nil, false, true, logStats) execStart := time.Now() logStats.PlanTime = execStart.Sub(logStats.StartTime) if err != nil { logStats.Error = err - return nil, err + return nil, 0, err } + bindVars := prepareBindVars(plan.ParamsCount) err = e.addNeededBindVars(vcursor, plan.BindVarNeeds, bindVars, safeSession) if err != nil { logStats.Error = err - return nil, err + return nil, 0, err } qr, err := plan.Instructions.GetFields(ctx, vcursor, bindVars) @@ -1476,13 +1533,13 @@ func (e *Executor) handlePrepare(ctx context.Context, safeSession *econtext.Safe if err != nil { logStats.Error = err errCount = 1 // nolint - return nil, err + return nil, 0, err } logStats.RowsAffected = qr.RowsAffected plan.AddStats(1, time.Since(logStats.StartTime), logStats.ShardQueries, qr.RowsAffected, uint64(len(qr.Rows)), errCount) - return qr.Fields, err + return qr.Fields, plan.ParamsCount, err } func parseAndValidateQuery(query string, parser *sqlparser.Parser) (sqlparser.Statement, *sqlparser.ReservedVars, error) { @@ -1623,29 +1680,11 @@ func (e *Executor) ReleaseLock(ctx context.Context, session *econtext.SafeSessio } // PlanPrepareStmt implements the IExecutor interface -func (e *Executor) PlanPrepareStmt(ctx context.Context, vcursor *econtext.VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) { - stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser()) - if err != nil { - return nil, nil, err - } - +func (e *Executor) PlanPrepareStmt(ctx context.Context, safeSession *econtext.SafeSession, query string) (*engine.Plan, error) { // creating this log stats to not interfere with the original log stats. - lStats := logstats.NewLogStats(ctx, "prepare", query, vcursor.Session().GetSessionUUID(), nil, streamlog.GetQueryLogConfig()) - plan, err := e.getPlan( - ctx, - vcursor, - query, - sqlparser.Clone(stmt), - vcursor.GetMarginComments(), - map[string]*querypb.BindVariable{}, - reservedVars, /* normalize */ - false, - lStats, - ) - if err != nil { - return nil, nil, err - } - return plan, stmt, nil + lStats := logstats.NewLogStats(ctx, "prepare", query, safeSession.GetSessionUUID(), nil, streamlog.GetQueryLogConfig()) + plan, _, _, err := e.fetchOrCreatePlan(ctx, safeSession, query, nil, false, true, lStats) + return plan, err } func (e *Executor) Close() { diff --git a/go/vt/vtgate/executor_ddl_test.go b/go/vt/vtgate/executor_ddl_test.go index bf117856e08..08c4091b41b 100644 --- a/go/vt/vtgate/executor_ddl_test.go +++ b/go/vt/vtgate/executor_ddl_test.go @@ -60,7 +60,7 @@ func TestDDLFlags(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) enableDirectDDL.Set(testcase.enableDirectDDL) enableOnlineDDL.Set(testcase.enableOnlineDDL) - _, err := executor.Execute(ctx, nil, "TestDDLFlags", session, testcase.sql, nil) + _, err := executor.Execute(ctx, nil, "TestDDLFlags", session, testcase.sql, nil, false) if testcase.wantErr { require.EqualError(t, err, testcase.err) } else { diff --git a/go/vt/vtgate/executor_dml_test.go b/go/vt/vtgate/executor_dml_test.go index 2846c763ae9..1078d4203e1 100644 --- a/go/vt/vtgate/executor_dml_test.go +++ b/go/vt/vtgate/executor_dml_test.go @@ -238,9 +238,9 @@ func TestUpdateInTransactionLookupDefaultReadLock(t *testing.T) { safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}) _, err := executorExecSession(ctx, executor, + safeSession, "update t2_lookup set lu_col = 5 where nv_lu_col = 2", nil, - safeSession.Session, ) require.NoError(t, err) @@ -300,9 +300,9 @@ func TestUpdateInTransactionLookupExclusiveReadLock(t *testing.T) { safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}) _, err := executorExecSession(ctx, executor, + safeSession, "update t2_lookup set lu_col = 5 where erl_lu_col = 2", nil, - safeSession.Session, ) require.NoError(t, err) @@ -362,9 +362,9 @@ func TestUpdateInTransactionLookupSharedReadLock(t *testing.T) { safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}) _, err := executorExecSession(ctx, executor, + safeSession, "update t2_lookup set lu_col = 5 where srl_lu_col = 2", nil, - safeSession.Session, ) require.NoError(t, err) @@ -424,9 +424,9 @@ func TestUpdateInTransactionLookupNoReadLock(t *testing.T) { safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}) _, err := executorExecSession(ctx, executor, + safeSession, "update t2_lookup set lu_col = 5 where nrl_lu_col = 2", nil, - safeSession.Session, ) require.NoError(t, err) @@ -1474,7 +1474,7 @@ func TestInsertShardedAutocommitLookup(t *testing.T) { ` executor, sbc1, sbc2, sbclookup, ctx := createCustomExecutor(t, vschema, config.DefaultMySQLVersion) - _, err := executorExecSession(ctx, executor, "insert into user(id, v, name, music) values (1, 2, 'myname', 'star')", nil, &vtgatepb.Session{}) + _, err := executorExec(ctx, executor, &vtgatepb.Session{}, "insert into user(id, v, name, music) values (1, 2, 'myname', 'star')", nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ Sql: "insert into `user`(id, v, `name`, music) values (:_Id_0, 2, :_name_0, :_music_0)", @@ -2075,6 +2075,7 @@ func TestInsertPartialFail1(t *testing.T) { econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}), "insert into user(id, v, name) values (1, 2, 'myname')", nil, + false, ) require.Error(t, err) } @@ -2096,6 +2097,7 @@ func TestInsertPartialFail2(t *testing.T) { safeSession, "insert into user(id, v, name) values (1, 2, 'myname')", nil, + false, ) want := "reverted partial DML execution failure" @@ -2471,11 +2473,9 @@ func TestUpdateEqualWithPrepare(t *testing.T) { session := &vtgatepb.Session{ TargetString: "@primary", } - _, err := executorPrepare(ctx, executor, session, "update music set a = :a0 where id = :id0", map[string]*querypb.BindVariable{ - "a0": sqltypes.Int64BindVariable(3), - "id0": sqltypes.Int64BindVariable(2), - }) + _, paramsCount, err := executorPrepare(ctx, executor, session, "update music set a = ? where id = ?") require.NoError(t, err) + assert.EqualValues(t, 2, paramsCount) var wantQueries []*querypb.BoundQuery @@ -2483,6 +2483,7 @@ func TestUpdateEqualWithPrepare(t *testing.T) { assertQueries(t, sbc2, nil) assertQueries(t, sbc1, nil) } + func TestInsertShardedWithPrepare(t *testing.T) { executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t) @@ -2492,12 +2493,9 @@ func TestInsertShardedWithPrepare(t *testing.T) { session := &vtgatepb.Session{ TargetString: "@primary", } - _, err := executorPrepare(ctx, executor, session, "insert into user(id, v, name) values (:_Id0, 2, ':_name_0')", map[string]*querypb.BindVariable{ - "_Id0": sqltypes.Int64BindVariable(1), - "_name_0": sqltypes.BytesBindVariable([]byte("myname")), - "__seq0": sqltypes.Int64BindVariable(1), - }) + _, paramsCount, err := executorPrepare(ctx, executor, session, "insert into user(id, v, name) values (?, 2, ?)") require.NoError(t, err) + assert.EqualValues(t, 2, paramsCount) var wantQueries []*querypb.BoundQuery @@ -2513,15 +2511,12 @@ func TestDeleteEqualWithPrepare(t *testing.T) { session := &vtgatepb.Session{ TargetString: "@primary", } - _, err := executorPrepare(ctx, executor, session, "delete from user where id = :id0", map[string]*querypb.BindVariable{ - "id0": sqltypes.Int64BindVariable(1), - }) + _, paramsCount, err := executorPrepare(ctx, executor, session, "delete from user where id = ?") require.NoError(t, err) + assert.EqualValues(t, 1, paramsCount) var wantQueries []*querypb.BoundQuery - assertQueries(t, sbc, wantQueries) - assertQueries(t, sbclookup, wantQueries) } @@ -2666,7 +2661,7 @@ func TestReservedConnDML(t *testing.T) { session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true}) - _, err := executor.Execute(ctx, nil, "TestReservedConnDML", session, "use "+KsTestUnsharded, nil) + _, err := executor.Execute(ctx, nil, "TestReservedConnDML", session, "use "+KsTestUnsharded, nil, false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ @@ -2675,24 +2670,24 @@ func TestReservedConnDML(t *testing.T) { sbc.SetResults([]*sqltypes.Result{ sqltypes.MakeTestResult(sqltypes.MakeTestFields("id", "int64"), "1"), }) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "set default_week_format = 1", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "set default_week_format = 1", nil, false) require.NoError(t, err) assertQueries(t, sbc, wantQueries) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "begin", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "begin", nil, false) require.NoError(t, err) wantQueries = append(wantQueries, &querypb.BoundQuery{Sql: "set default_week_format = 1", BindVariables: map[string]*querypb.BindVariable{}}, &querypb.BoundQuery{Sql: "insert into `simple`() values ()", BindVariables: map[string]*querypb.BindVariable{}}) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "insert into `simple`() values ()", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "insert into `simple`() values ()", nil, false) require.NoError(t, err) assertQueries(t, sbc, wantQueries) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "commit", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "commit", nil, false) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "begin", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "begin", nil, false) require.NoError(t, err) sbc.EphemeralShardErr = sqlerror.NewSQLError(sqlerror.CRServerGone, sqlerror.SSNetError, "connection gone") @@ -2700,11 +2695,11 @@ func TestReservedConnDML(t *testing.T) { wantQueries = append(wantQueries, &querypb.BoundQuery{Sql: "set default_week_format = 1", BindVariables: map[string]*querypb.BindVariable{}}, &querypb.BoundQuery{Sql: "insert into `simple`() values ()", BindVariables: map[string]*querypb.BindVariable{}}) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "insert into `simple`() values ()", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "insert into `simple`() values ()", nil, false) require.NoError(t, err) assertQueries(t, sbc, wantQueries) - _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "commit", nil) + _, err = executor.Execute(ctx, nil, "TestReservedConnDML", session, "commit", nil, false) require.NoError(t, err) } @@ -2804,7 +2799,7 @@ func TestPartialVindexInsertQueryFailure(t *testing.T) { require.True(t, session.GetAutocommit()) require.False(t, session.InTransaction()) - _, err := executorExecSession(ctx, executor, "begin", nil, session.Session) + _, err := executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) require.True(t, session.GetAutocommit()) require.True(t, session.InTransaction()) @@ -2825,7 +2820,7 @@ func TestPartialVindexInsertQueryFailure(t *testing.T) { BindVariables: map[string]*querypb.BindVariable{}, }} - _, err = executorExecSession(ctx, executor, "insert into t1(id, unq_col) values (1, 1), (2, 3)", nil, session.Session) + _, err = executorExecSession(ctx, executor, session, "insert into t1(id, unq_col) values (1, 1), (2, 3)", nil) require.Error(t, err) require.Contains(t, err.Error(), "reverted partial DML execution failure") require.True(t, session.GetAutocommit()) @@ -2867,7 +2862,7 @@ func TestPartialVindexInsertQueryFailureAutoCommit(t *testing.T) { }, }} - _, err := executorExecSession(ctx, executor, "insert into t1(id, unq_col) values (1, 1), (2, 3)", nil, session.Session) + _, err := executorExecSession(ctx, executor, session, "insert into t1(id, unq_col) values (1, 1), (2, 3)", nil) require.Error(t, err) assert.Contains(t, err.Error(), "transaction rolled back to reverse changes of partial DML execution") assert.True(t, session.GetAutocommit()) @@ -2895,11 +2890,11 @@ func TestMultiInternalSavepoint(t *testing.T) { executor, sbc1, sbc2, _, ctx := createExecutorEnv(t) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - _, err := executorExecSession(ctx, executor, "begin", nil, session.Session) + _, err := executorExec(ctx, executor, session.Session, "begin", nil) require.NoError(t, err) // this query goes to multiple shards so internal savepoint will be created. - _, err = executorExecSession(ctx, executor, "insert into user_extra(user_id) values (1), (4)", nil, session.Session) + _, err = executorExec(ctx, executor, session.Session, "insert into user_extra(user_id) values (1), (4)", nil) require.NoError(t, err) wantQ := []*querypb.BoundQuery{{ @@ -2915,7 +2910,7 @@ func TestMultiInternalSavepoint(t *testing.T) { require.Len(t, sbc2.Queries, 0) sbc1.Queries = nil - _, err = executorExecSession(ctx, executor, "insert into user_extra(user_id) values (3), (6)", nil, session.Session) + _, err = executorExec(ctx, executor, session.Session, "insert into user_extra(user_id) values (3), (6)", nil) require.NoError(t, err) wantQ = []*querypb.BoundQuery{{ Sql: "savepoint x", @@ -2975,19 +2970,19 @@ func TestInsertSelectFromDual(t *testing.T) { // set result for dual query. sbc1.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult(sqltypes.MakeTestFields("1|2|myname", "int64|int64|varchar"), "1|2|myname")}) - _, err := executor.Execute(context.Background(), nil, "TestInsertSelect", session, wQuery, nil) + _, err := executorExecSession(context.Background(), executor, session, wQuery, nil) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestInsertSelect", session, query, nil) + _, err = executorExecSession(context.Background(), executor, session, query, nil) require.NoError(t, err) assertQueries(t, sbc1, wantQueries) assertQueries(t, sbc2, nil) assertQueries(t, sbclookup, wantlkpQueries) - testQueryLog(t, executor, logChan, "TestInsertSelect", "SET", wQuery, 0) + testQueryLog(t, executor, logChan, "TestExecute", "SET", wQuery, 0) testQueryLog(t, executor, logChan, "VindexCreate", "INSERT", "insert into name_user_map(`name`, user_id) values (:name_0, :user_id_0)", 1) - testQueryLog(t, executor, logChan, "TestInsertSelect", "INSERT", "insert into `user`(id, v, `name`) select 1, 2, 'myname' from dual", 2) + testQueryLog(t, executor, logChan, "TestExecute", "INSERT", "insert into `user`(id, v, `name`) select 1, 2, 'myname' from dual", 2) }) } } @@ -3037,19 +3032,19 @@ func TestInsertSelectFromTable(t *testing.T) { sbc2.Queries = nil sbclookup.Queries = nil wQuery := fmt.Sprintf("set @@workload = %s", workload) - _, err := executor.Execute(context.Background(), nil, "TestInsertSelect", session, wQuery, nil) + _, err := executorExecSession(context.Background(), executor, session, wQuery, nil) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestInsertSelect", session, query, nil) + _, err = executorExecSession(context.Background(), executor, session, query, nil) require.NoError(t, err) assertQueries(t, sbc1, wantQueries) assertQueries(t, sbc2, wantQueries[:1]) // select scatter query went scatter. assertQueries(t, sbclookup, wantlkpQueries) - testQueryLog(t, executor, logChan, "TestInsertSelect", "SET", wQuery, 0) + testQueryLog(t, executor, logChan, "TestExecute", "SET", wQuery, 0) testQueryLog(t, executor, logChan, "VindexCreate", "INSERT", "insert into name_user_map(`name`, user_id) values (:name_0, :user_id_0), (:name_1, :user_id_1), (:name_2, :user_id_2), (:name_3, :user_id_3), (:name_4, :user_id_4), (:name_5, :user_id_5), (:name_6, :user_id_6), (:name_7, :user_id_7)", 1) - testQueryLog(t, executor, logChan, "TestInsertSelect", "INSERT", "insert into `user`(id, `name`) select c1, c2 from music", 9) // 8 from select and 1 from insert. + testQueryLog(t, executor, logChan, "TestExecute", "INSERT", "insert into `user`(id, `name`) select c1, c2 from music", 9) // 8 from select and 1 from insert. } } @@ -3151,59 +3146,58 @@ func TestDeleteMultiTable(t *testing.T) { // TestSessionRowsAffected test that rowsAffected is set correctly for each shard session. func TestSessionRowsAffected(t *testing.T) { - method := t.Name() executor, _, sbc4060, _, ctx := createExecutorEnv(t) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) // start the transaction - _, err := executor.Execute(ctx, nil, method, session, "begin", nil) + _, err := executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) // -20 - select query - _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 1", nil) + _, err = executorExecSession(ctx, executor, session, "select * from user where id = 1", nil) require.NoError(t, err) require.Len(t, session.ShardSessions, 1) require.False(t, session.ShardSessions[0].RowsAffected) // -20 - update query (rows affected) - _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 41 where id = 1", nil) + _, err = executorExecSession(ctx, executor, session, "update user set foo = 41 where id = 1", nil) require.NoError(t, err) require.True(t, session.ShardSessions[0].RowsAffected) // e0- - select query - _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 7", nil) + _, err = executorExecSession(ctx, executor, session, "select * from user where id = 7", nil) require.NoError(t, err) assert.Len(t, session.ShardSessions, 2) require.False(t, session.ShardSessions[1].RowsAffected) // c0-e0 - update query (rows affected) - _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 42 where id = 5", nil) + _, err = executorExecSession(ctx, executor, session, "update user set foo = 42 where id = 5", nil) require.NoError(t, err) require.Len(t, session.ShardSessions, 3) require.True(t, session.ShardSessions[2].RowsAffected) // 40-60 - update query (no rows affected) sbc4060.SetResults([]*sqltypes.Result{{RowsAffected: 0}}) - _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 42 where id = 3", nil) + _, err = executorExecSession(ctx, executor, session, "update user set foo = 42 where id = 3", nil) require.NoError(t, err) assert.Len(t, session.ShardSessions, 4) require.False(t, session.ShardSessions[3].RowsAffected) // 40-60 - select query - _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 3", nil) + _, err = executorExecSession(ctx, executor, session, "select * from user where id = 3", nil) require.NoError(t, err) require.False(t, session.ShardSessions[3].RowsAffected) // 40-60 - delete query (rows affected) - _, err = executor.Execute(ctx, nil, method, session, "delete from user where id = 3", nil) + _, err = executorExecSession(ctx, executor, session, "delete from user where id = 3", nil) require.NoError(t, err) require.True(t, session.ShardSessions[0].RowsAffected) require.False(t, session.ShardSessions[1].RowsAffected) require.True(t, session.ShardSessions[2].RowsAffected) require.True(t, session.ShardSessions[3].RowsAffected) - _, err = executor.Execute(ctx, nil, method, session, "commit", nil) + _, err = executorExecSession(ctx, executor, session, "commit", nil) require.NoError(t, err) require.Zero(t, session.ShardSessions) } diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index 099b785b446..88455e8331f 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -82,8 +82,8 @@ func (*keyRangeLookuper) NeedsVCursor() bool { return false } func (*keyRangeLookuper) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*keyRangeLookuper) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - return []key.Destination{ +func (*keyRangeLookuper) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + return []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{ End: []byte{0x10}, @@ -107,8 +107,8 @@ func (*keyRangeLookuperUnique) NeedsVCursor() bool { return false } func (*keyRangeLookuperUnique) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*keyRangeLookuperUnique) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - return []key.Destination{ +func (*keyRangeLookuperUnique) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + return []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{ End: []byte{0x10}, @@ -320,27 +320,16 @@ func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, ctx context.Context, return executor, primary, replica } -func executorExecSession(ctx context.Context, executor *Executor, sql string, bv map[string]*querypb.BindVariable, session *vtgatepb.Session) (*sqltypes.Result, error) { - return executor.Execute( - ctx, - nil, - "TestExecute", - econtext.NewSafeSession(session), - sql, - bv) +func executorExecSession(ctx context.Context, executor *Executor, session *econtext.SafeSession, sql string, bv map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + return executor.Execute(ctx, nil, "TestExecute", session, sql, bv, false) } func executorExec(ctx context.Context, executor *Executor, session *vtgatepb.Session, sql string, bv map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return executorExecSession(ctx, executor, sql, bv, session) + return executorExecSession(ctx, executor, econtext.NewSafeSession(session), sql, bv) } -func executorPrepare(ctx context.Context, executor *Executor, session *vtgatepb.Session, sql string, bv map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - return executor.Prepare( - ctx, - "TestExecute", - econtext.NewSafeSession(session), - sql, - bv) +func executorPrepare(ctx context.Context, executor *Executor, session *vtgatepb.Session, sql string) ([]*querypb.Field, uint16, error) { + return executor.Prepare(ctx, "TestExecute", econtext.NewSafeSession(session), sql) } func executorStream(ctx context.Context, executor *Executor, sql string) (qr *sqltypes.Result, err error) { diff --git a/go/vt/vtgate/executor_scatter_stats_test.go b/go/vt/vtgate/executor_scatter_stats_test.go index b665f850a23..487b2ea4df6 100644 --- a/go/vt/vtgate/executor_scatter_stats_test.go +++ b/go/vt/vtgate/executor_scatter_stats_test.go @@ -32,7 +32,7 @@ func TestScatterStatsWithNoScatterQuery(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) - _, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from main1", nil) + _, err := executorExecSession(ctx, executor, session, "select * from main1", nil) require.NoError(t, err) result, err := executor.gatherScatterStats() @@ -44,7 +44,7 @@ func TestScatterStatsWithSingleScatterQuery(t *testing.T) { executor, _, _, _, ctx := createExecutorEnv(t) session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) - _, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user", nil) + _, err := executorExecSession(ctx, executor, session, "select * from user", nil) require.NoError(t, err) result, err := executor.gatherScatterStats() @@ -56,17 +56,17 @@ func TestScatterStatsHttpWriting(t *testing.T) { executor, _, _, _, ctx := createExecutorEnv(t) session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) - _, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user", nil) + _, err := executorExecSession(ctx, executor, session, "select * from user", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user where Id = 15", nil) + _, err = executorExecSession(ctx, executor, session, "select * from user where Id = 15", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user where Id > 15", nil) + _, err = executorExecSession(ctx, executor, session, "select * from user where Id > 15", nil) require.NoError(t, err) query4 := "select * from user as u1 join user as u2 on u1.Id = u2.Id" - _, err = executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, query4, nil) + _, err = executorExecSession(ctx, executor, session, query4, nil) require.NoError(t, err) time.Sleep(500 * time.Millisecond) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 3d8261495fe..20290922bca 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "os" + "reflect" "runtime" "strconv" "strings" @@ -29,6 +30,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/proto" econtext "vitess.io/vitess/go/vt/vtgate/executorcontext" @@ -62,7 +64,7 @@ func TestSelectNext(t *testing.T) { // Autocommit session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - _, err := executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv) + _, err := executorExecSession(context.Background(), executor, session, query, bv) require.NoError(t, err) utils.MustMatch(t, wantQueries, sbclookup.Queries) @@ -73,7 +75,7 @@ func TestSelectNext(t *testing.T) { // Txn session = econtext.NewAutocommitSession(&vtgatepb.Session{}) session.Session.InTransaction = true - _, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv) + _, err = executorExecSession(context.Background(), executor, session, query, bv) require.NoError(t, err) utils.MustMatch(t, wantQueries, sbclookup.Queries) @@ -84,7 +86,7 @@ func TestSelectNext(t *testing.T) { // Reserve session = econtext.NewAutocommitSession(&vtgatepb.Session{}) session.Session.InReservedConn = true - _, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv) + _, err = executorExecSession(context.Background(), executor, session, query, bv) require.NoError(t, err) utils.MustMatch(t, wantQueries, sbclookup.Queries) @@ -96,7 +98,7 @@ func TestSelectNext(t *testing.T) { session = econtext.NewAutocommitSession(&vtgatepb.Session{}) session.Session.InReservedConn = true session.Session.InTransaction = true - _, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv) + _, err = executorExecSession(context.Background(), executor, session, query, bv) require.NoError(t, err) utils.MustMatch(t, wantQueries, sbclookup.Queries) @@ -111,7 +113,7 @@ func TestSelectDBA(t *testing.T) { _, err := executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{Sql: query, BindVariables: map[string]*querypb.BindVariable{}}} utils.MustMatch(t, wantQueries, sbc1.Queries) @@ -121,7 +123,7 @@ func TestSelectDBA(t *testing.T) { _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name /* VARCHAR */", @@ -137,7 +139,7 @@ func TestSelectDBA(t *testing.T) { _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name /* VARCHAR */", @@ -153,7 +155,7 @@ func TestSelectDBA(t *testing.T) { _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */", @@ -182,24 +184,24 @@ func TestSystemVariablesMySQLBelow80(t *testing.T) { }}, }}) - _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err := executorExecSession(context.Background(), executor, session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executorExecSession(context.Background(), executor, session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) require.NoError(t, err) require.True(t, session.InReservedConn()) wantQueries := []*querypb.BoundQuery{ {Sql: "select @@sql_mode orig, 'only_full_group_by' new"}, {Sql: "set sql_mode = 'only_full_group_by'", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, - {Sql: "select :vtg1 /* INT64 */ from information_schema.`table`", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, + {Sql: "select /*+ SET_VAR(sql_mode = 'only_full_group_by') */ :vtg1 /* INT64 */ from information_schema.`table`", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, } - + require.Equal(t, len(wantQueries), len(sbc1.Queries)) utils.MustMatch(t, wantQueries, sbc1.Queries) } func TestSystemVariablesWithSetVarDisabled(t *testing.T) { - executor, sbc1, _, _, _ := createCustomExecutor(t, "{}", "8.0.0") + executor, sbc1, _, _, _ := createExecutorEnv(t) executor.config.Normalize = true executor.vConfig.SetVarEnabled = false session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"}) @@ -215,12 +217,12 @@ func TestSystemVariablesWithSetVarDisabled(t *testing.T) { }}, }}) - _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err := executorExecSession(context.Background(), executor, session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) require.NoError(t, err) + require.True(t, session.InReservedConn()) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executorExecSession(context.Background(), executor, session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) require.NoError(t, err) - require.True(t, session.InReservedConn()) wantQueries := []*querypb.BoundQuery{ {Sql: "select @@sql_mode orig, 'only_full_group_by' new"}, @@ -237,10 +239,10 @@ func TestSetSystemVariablesTx(t *testing.T) { session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"}) - _, err := executor.Execute(context.Background(), nil, "TestBegin", session, "begin", map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestBegin", session, "begin", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.NotZero(t, session.ShardSessions) @@ -255,14 +257,14 @@ func TestSetSystemVariablesTx(t *testing.T) { }}, }}) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestCommit", session, "commit", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestCommit", session, "commit", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) @@ -293,10 +295,10 @@ func TestSetSystemVariables(t *testing.T) { sqltypes.NewVarChar("only_full_group_by"), }}, }}) - _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) wantQueries := []*querypb.BoundQuery{ @@ -308,7 +310,7 @@ func TestSetSystemVariables(t *testing.T) { // Execute a select with a comment that needs a query hint - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select /* comment */ 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select /* comment */ 1 from information_schema.table", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) wantQueries = []*querypb.BoundQuery{ @@ -325,7 +327,7 @@ func TestSetSystemVariables(t *testing.T) { sqltypes.NewVarChar("0"), }}, }}) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_safe_updates = 0", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_safe_updates = 0", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) wantQueries = []*querypb.BoundQuery{ @@ -334,7 +336,7 @@ func TestSetSystemVariables(t *testing.T) { utils.MustMatch(t, wantQueries, lookup.Queries) lookup.Queries = nil - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @var = @@sql_mode", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @var = @@sql_mode", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) require.Nil(t, lookup.Queries) @@ -348,7 +350,7 @@ func TestSetSystemVariables(t *testing.T) { sqltypes.NewVarChar("4"), }}, }}) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @x = @@sql_mode, @y = @@max_tmp_tables", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @x = @@sql_mode, @y = @@max_tmp_tables", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.False(t, session.InReservedConn()) wantQueries = []*querypb.BoundQuery{ @@ -371,22 +373,39 @@ func TestSetSystemVariables(t *testing.T) { sqltypes.NewVarChar("1"), }}, }}) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@max_tmp_tables = 1", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@max_tmp_tables = 1", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) require.True(t, session.InReservedConn()) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select 1 from information_schema.table", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ {Sql: "select 1 from dual where @@max_tmp_tables != 1"}, {Sql: "set max_tmp_tables = '1', sql_mode = 'only_full_group_by', sql_safe_updates = '0'", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, - {Sql: "select :vtg1 /* INT64 */ from information_schema.`table`", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, - } - utils.MustMatch(t, wantQueries, lookup.Queries) + // we don't need the set_var since we are in a reserved connection, but since the plan is in the cache, we'll use it + {Sql: "select /*+ SET_VAR(sql_mode = 'only_full_group_by') SET_VAR(sql_safe_updates = '0') */ :vtg1 /* INT64 */ from information_schema.`table`", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, + } + + diffOpts := []cmp.Option{ + cmp.Comparer(func(a, b proto.Message) bool { + return proto.Equal(a, b) + }), + cmp.Exporter(func(reflect.Type) bool { + return true + }), + } + diff := cmp.Diff(wantQueries, lookup.Queries, diffOpts...) + if diff == "" { + return + } + // try again with rearranged SET_VAR hints + wantQueries[2].Sql = "select /*+ SET_VAR(sql_safe_updates = '0') SET_VAR(sql_mode = 'only_full_group_by') */ :vtg1 /* INT64 */ from information_schema.`table`" + diff = cmp.Diff(wantQueries, lookup.Queries, diffOpts...) + assert.Empty(t, diff) } -func TestSetSystemVariablesWithReservedConnection(t *testing.T) { +func TestSetSystemVariablesWithSetVarInvalidSQLMode(t *testing.T) { executor, sbc1, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, SystemVariables: map[string]string{}}) @@ -401,31 +420,28 @@ func TestSetSystemVariablesWithReservedConnection(t *testing.T) { sqltypes.NewVarChar(""), }}, }}) - _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = ''", map[string]*querypb.BindVariable{}) + + _, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, "set @@sql_mode = ''", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) + require.False(t, session.InReservedConn()) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user group by age", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user group by age", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - require.True(t, session.InReservedConn()) wantQueries := []*querypb.BoundQuery{ {Sql: "select @@sql_mode orig, '' new"}, - {Sql: "set sql_mode = ''"}, - {Sql: "select age, city, weight_string(age) from `user` group by age, weight_string(age) order by age asc"}, + {Sql: "select /*+ SET_VAR(sql_mode = ' ') */ age, city, weight_string(age) from `user` group by age, weight_string(age) order by age asc"}, } utils.MustMatch(t, wantQueries, sbc1.Queries) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city+1 from user group by age", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city+1 from user group by age", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - require.True(t, session.InReservedConn()) wantQueries = []*querypb.BoundQuery{ {Sql: "select @@sql_mode orig, '' new"}, - {Sql: "set sql_mode = ''"}, - {Sql: "select age, city, weight_string(age) from `user` group by age, weight_string(age) order by age asc"}, - {Sql: "select age, city + :vtg1 /* INT64 */, weight_string(age) from `user` group by age, weight_string(age) order by age asc", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, + {Sql: "select /*+ SET_VAR(sql_mode = ' ') */ age, city, weight_string(age) from `user` group by age, weight_string(age) order by age asc"}, + {Sql: "select /*+ SET_VAR(sql_mode = ' ') */ age, city + :vtg1 /* INT64 */, weight_string(age) from `user` group by age, weight_string(age) order by age asc", BindVariables: map[string]*querypb.BindVariable{"vtg1": {Type: sqltypes.Int64, Value: []byte("1")}}}, } utils.MustMatch(t, wantQueries, sbc1.Queries) require.Equal(t, "''", session.SystemVariables["sql_mode"]) - sbc1.Queries = nil } func TestSelectVindexFunc(t *testing.T) { @@ -433,11 +449,11 @@ func TestSelectVindexFunc(t *testing.T) { query := "select * from hash_index where id = 1" session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - _, err := executor.Execute(context.Background(), nil, "TestSelectVindexFunc", session, query, nil) + _, err := executor.Execute(context.Background(), nil, "TestSelectVindexFunc", session, query, nil, false) require.ErrorContains(t, err, "VT09005: no database selected") session.TargetString = KsTestSharded - _, err = executor.Execute(context.Background(), nil, "TestSelectVindexFunc", session, query, nil) + _, err = executor.Execute(context.Background(), nil, "TestSelectVindexFunc", session, query, nil, false) require.NoError(t, err) } @@ -447,9 +463,9 @@ func TestCreateTableValidTimestamp(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor", SystemVariables: map[string]string{"sql_mode": "ALLOW_INVALID_DATES"}}) query := "create table aa(t timestamp default 0)" - _, err := executor.Execute(context.Background(), nil, "TestSelect", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestSelect", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) - require.True(t, session.InReservedConn()) + assert.True(t, session.InReservedConn()) wantQueries := []*querypb.BoundQuery{ {Sql: "set sql_mode = ALLOW_INVALID_DATES", BindVariables: map[string]*querypb.BindVariable{}}, @@ -466,7 +482,7 @@ func TestGen4SelectDBA(t *testing.T) { _, err := executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) expected := "select CONSTRAINT_CATALOG, CONSTRAINT_SCHEMA, CONSTRAINT_NAME, TABLE_SCHEMA, TABLE_NAME, CONSTRAINT_TYPE, `ENFORCED` from INFORMATION_SCHEMA.TABLE_CONSTRAINTS" wantQueries := []*querypb.BoundQuery{{Sql: expected, BindVariables: map[string]*querypb.BindVariable{}}} @@ -477,7 +493,7 @@ func TestGen4SelectDBA(t *testing.T) { _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name1 /* VARCHAR */", @@ -495,7 +511,7 @@ func TestGen4SelectDBA(t *testing.T) { _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name1 /* VARCHAR */", @@ -511,7 +527,7 @@ func TestGen4SelectDBA(t *testing.T) { sbc1.Queries = nil query = "select 1 from information_schema.table_constraints where constraint_schema = 'vt_ks'" - _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */", @@ -525,10 +541,9 @@ func TestGen4SelectDBA(t *testing.T) { sbc1.Queries = nil query = "select t.table_schema,t.table_name,c.column_name,c.column_type from tables t join columns c on c.table_schema = t.table_schema and c.table_name = t.table_name where t.table_schema = 'TestExecutor' and c.table_schema = 'TestExecutor' order by t.table_schema,t.table_name,c.column_name" - _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", + _, err = executorExecSession(context.Background(), executor, econtext.NewSafeSession(&vtgatepb.Session{TargetString: "information_schema"}), - query, map[string]*querypb.BindVariable{}, - ) + query, map[string]*querypb.BindVariable{}) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select t.table_schema, t.table_name, c.column_name, c.column_type from information_schema.`tables` as t, information_schema.`columns` as c where t.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = t.table_schema and c.table_name = t.table_name order by t.table_schema asc, t.table_name asc, c.column_name asc", @@ -1062,18 +1077,11 @@ func TestLastInsertIDInSubQueryExpression(t *testing.T) { func TestSelectDatabase(t *testing.T) { executor, _, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) sql := "select database()" - newSession := &vtgatepb.Session{ - TargetString: "@primary", + session := &vtgatepb.Session{ + TargetString: "TestExecutor@primary", } - session := econtext.NewSafeSession(newSession) - session.TargetString = "TestExecutor@primary" - result, err := executor.Execute( - context.Background(), - nil, - "TestExecute", - session, - sql, - map[string]*querypb.BindVariable{}) + + result, err := executorExec(context.Background(), executor, session, sql, map[string]*querypb.BindVariable{}) wantResult := &sqltypes.Result{ Fields: []*querypb.Field{ {Name: "database()", Type: sqltypes.VarChar, Charset: uint32(collations.MySQL8().DefaultConnectionCharset())}, @@ -3089,18 +3097,16 @@ func TestSelectBindvarswithPrepare(t *testing.T) { logChan := executor.queryLogger.Subscribe("Test") defer executor.queryLogger.Unsubscribe(logChan) - sql := "select id from `user` where id = :id" + sql := "select id from `user` where id = ?" session := &vtgatepb.Session{ TargetString: "@primary", } - _, err := executorPrepare(ctx, executor, session, sql, map[string]*querypb.BindVariable{ - "id": sqltypes.Int64BindVariable(1), - }) + _, paramsCount, err := executorPrepare(ctx, executor, session, sql) require.NoError(t, err) - + assert.EqualValues(t, 1, paramsCount) wantQueries := []*querypb.BoundQuery{{ Sql: "select id from `user` where 1 != 1", - BindVariables: map[string]*querypb.BindVariable{"id": sqltypes.Int64BindVariable(1)}, + BindVariables: map[string]*querypb.BindVariable{"v1": sqltypes.NullBindVariable}, }} utils.MustMatch(t, wantQueries, sbc1.Queries) assert.Empty(t, sbc2.Queries) @@ -3115,8 +3121,9 @@ func TestSelectDatabasePrepare(t *testing.T) { session := &vtgatepb.Session{ TargetString: "@primary", } - _, err := executorPrepare(ctx, executor, session, sql, map[string]*querypb.BindVariable{}) + _, paramsCount, err := executorPrepare(ctx, executor, session, sql) require.NoError(t, err) + assert.Zero(t, paramsCount) } func TestSelectWithUnionAll(t *testing.T) { @@ -3406,25 +3413,25 @@ func TestSelectScatterFails(t *testing.T) { logChan := executor.queryLogger.Subscribe("Test") defer executor.queryLogger.Unsubscribe(logChan) - _, err := executorExecSession(ctx, executor, "select id from `user`", nil, sess) + _, err := executorExec(ctx, executor, sess, "select id from `user`", nil) require.Error(t, err) assert.Contains(t, err.Error(), "scatter") // Run the test again, to ensure it behaves the same for a cached query - _, err = executorExecSession(ctx, executor, "select id from `user`", nil, sess) + _, err = executorExec(ctx, executor, sess, "select id from `user`", nil) require.Error(t, err) assert.Contains(t, err.Error(), "scatter") - _, err = executorExecSession(ctx, executor, "select /*vt+ ALLOW_SCATTER */ id from user", nil, sess) + _, err = executorExec(ctx, executor, sess, "select /*vt+ ALLOW_SCATTER */ id from user", nil) require.NoError(t, err) - _, err = executorExecSession(ctx, executor, "begin", nil, sess) + _, err = executorExec(ctx, executor, sess, "begin", nil) require.NoError(t, err) - _, err = executorExecSession(ctx, executor, "commit", nil, sess) + _, err = executorExec(ctx, executor, sess, "commit", nil) require.NoError(t, err) - _, err = executorExecSession(ctx, executor, "savepoint a", nil, sess) + _, err = executorExec(ctx, executor, sess, "savepoint a", nil) require.NoError(t, err) } @@ -3437,7 +3444,7 @@ func TestGen4SelectStraightJoin(t *testing.T) { "TestGen4SelectStraightJoin", session, query, map[string]*querypb.BindVariable{}, - ) + false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ { @@ -3454,7 +3461,7 @@ func TestGen4MultiColumnVindexEqual(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) query := "select * from user_region where cola = 1 and colb = 2" - _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ { @@ -3471,7 +3478,7 @@ func TestGen4MultiColumnVindexEqual(t *testing.T) { sbc1.Queries = nil query = "select * from user_region where cola = 17984 and colb = 1" - _, err = executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ { @@ -3491,7 +3498,7 @@ func TestGen4MultiColumnVindexIn(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) query := "select * from user_region where cola IN (1,17984) and colb IN (2,3,4)" - _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) bv1, _ := sqltypes.BuildBindVariable([]int64{1}) bv2, _ := sqltypes.BuildBindVariable([]int64{17984}) @@ -3528,7 +3535,7 @@ func TestGen4MultiColMixedColComparision(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) query := "select * from user_region where colb = 2 and cola IN (1,17984)" - _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMixedColComparision", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMixedColComparision", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) bvtg1 := sqltypes.Int64BindVariable(2) bvtg2, _ := sqltypes.BuildBindVariable([]int64{1, 17984}) @@ -3563,7 +3570,7 @@ func TestGen4MultiColBestVindexSel(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) query := "select * from user_region where colb = 2 and cola IN (1,17984) and cola = 1" - _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColBestVindexSel", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColBestVindexSel", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) bvtg2, _ := sqltypes.BuildBindVariable([]int64{1, 17984}) wantQueries := []*querypb.BoundQuery{ @@ -3583,7 +3590,7 @@ func TestGen4MultiColBestVindexSel(t *testing.T) { sbc1.Queries = nil query = "select * from user_region where colb in (10,20) and cola IN (1,17984) and cola = 1 and colb = 2" - _, err = executor.Execute(context.Background(), nil, "TestGen4MultiColBestVindexSel", session, query, map[string]*querypb.BindVariable{}) + _, err = executor.Execute(context.Background(), nil, "TestGen4MultiColBestVindexSel", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) bvtg1, _ := sqltypes.BuildBindVariable([]int64{10, 20}) @@ -3607,7 +3614,7 @@ func TestGen4MultiColMultiEqual(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) query := "select * from user_region where (cola,colb) in ((17984,2),(17984,3))" - _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMultiEqual", session, query, map[string]*querypb.BindVariable{}) + _, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMultiEqual", session, query, map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ { @@ -3774,7 +3781,7 @@ func TestRegionRange(t *testing.T) { for _, tcase := range tcases { t.Run(strconv.Itoa(tcase.regionID), func(t *testing.T) { sql := fmt.Sprintf("select * from user_region where cola = %d", tcase.regionID) - _, err := executor.Execute(context.Background(), nil, "TestRegionRange", econtext.NewAutocommitSession(&vtgatepb.Session{}), sql, nil) + _, err := executor.Execute(context.Background(), nil, "TestRegionRange", econtext.NewAutocommitSession(&vtgatepb.Session{}), sql, nil, false) require.NoError(t, err) count := 0 for _, sbc := range conns { @@ -3826,7 +3833,7 @@ func TestMultiCol(t *testing.T) { for _, tcase := range tcases { t.Run(fmt.Sprintf("%d_%d_%d", tcase.cola, tcase.colb, tcase.colc), func(t *testing.T) { sql := fmt.Sprintf("select * from multicoltbl where cola = %d and colb = %d and colc = '%d'", tcase.cola, tcase.colb, tcase.colc) - _, err := executor.Execute(ctx, nil, "TestMultiCol", session, sql, nil) + _, err := executor.Execute(ctx, nil, "TestMultiCol", session, sql, nil, false) require.NoError(t, err) var shards []string for _, sbc := range conns { @@ -3915,7 +3922,7 @@ func TestMultiColPartial(t *testing.T) { for _, tcase := range tcases { t.Run(tcase.where, func(t *testing.T) { sql := fmt.Sprintf("select * from multicoltbl where %s", tcase.where) - _, err := executor.Execute(ctx, nil, "TestMultiCol", session, sql, nil) + _, err := executor.Execute(ctx, nil, "TestMultiCol", session, sql, nil, false) require.NoError(t, err) var shards []string for _, sbc := range conns { @@ -4199,7 +4206,7 @@ func TestSelectAggregationRandom(t *testing.T) { defer executor.Close() session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - rs, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ A.a, A.b, (A.a / A.b) as c from (select sum(a) as a, sum(b) as b from user) A", nil) + rs, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ A.a, A.b, (A.a / A.b) as c from (select sum(a) as a, sum(b) as b from user) A", nil, false) require.NoError(t, err) assert.Equal(t, `[[DECIMAL(10) DECIMAL(1) DECIMAL(10.0000)]]`, fmt.Sprintf("%v", rs.Rows)) } @@ -4208,7 +4215,7 @@ func TestSelectDateTypes(t *testing.T) { executor, _, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - qr, err := executor.Execute(context.Background(), nil, "TestSelectDateTypes", session, "select '2020-01-01' + interval month(date_sub(FROM_UNIXTIME(1234), interval 1 month))-1 month", nil) + qr, err := executor.Execute(context.Background(), nil, "TestSelectDateTypes", session, "select '2020-01-01' + interval month(date_sub(FROM_UNIXTIME(1234), interval 1 month))-1 month", nil, false) require.NoError(t, err) require.Equal(t, sqltypes.Char, qr.Fields[0].Type) require.Equal(t, `[[CHAR("2020-12-01")]]`, fmt.Sprintf("%v", qr.Rows)) @@ -4218,11 +4225,11 @@ func TestSelectHexAndBit(t *testing.T) { executor, _, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - qr, err := executor.Execute(context.Background(), nil, "TestSelectHexAndBit", session, "select 0b1001, b'1001', 0x9, x'09'", nil) + qr, err := executor.Execute(context.Background(), nil, "TestSelectHexAndBit", session, "select 0b1001, b'1001', 0x9, x'09'", nil, false) require.NoError(t, err) require.Equal(t, `[[VARBINARY("\t") VARBINARY("\t") VARBINARY("\t") VARBINARY("\t")]]`, fmt.Sprintf("%v", qr.Rows)) - qr, err = executor.Execute(context.Background(), nil, "TestSelectHexAndBit", session, "select 1 + 0b1001, 1 + b'1001', 1 + 0x9, 1 + x'09'", nil) + qr, err = executor.Execute(context.Background(), nil, "TestSelectHexAndBit", session, "select 1 + 0b1001, 1 + b'1001', 1 + 0x9, 1 + x'09'", nil, false) require.NoError(t, err) require.Equal(t, `[[INT64(10) INT64(10) UINT64(10) UINT64(10)]]`, fmt.Sprintf("%v", qr.Rows)) } @@ -4233,7 +4240,7 @@ func TestSelectCFC(t *testing.T) { executor, _, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - _, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ c2 from tbl_cfc where c1 like 'A%'", nil) + _, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ c2 from tbl_cfc where c1 like 'A%'", nil, false) require.NoError(t, err) timeout := time.After(30 * time.Second) @@ -4261,7 +4268,7 @@ func TestSelectView(t *testing.T) { session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view", nil) + _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view", nil, false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ Sql: "select id, col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = user_extra.user_id) as user_details_view", @@ -4270,7 +4277,7 @@ func TestSelectView(t *testing.T) { utils.MustMatch(t, wantQueries, sbc.Queries) sbc.Queries = nil - _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view where id = 2", nil) + _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view where id = 2", nil, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{{ Sql: "select id, col from (select `user`.id, user_extra.col from `user`, user_extra where `user`.id = :id /* INT64 */ and `user`.id = user_extra.user_id) as user_details_view", @@ -4281,7 +4288,7 @@ func TestSelectView(t *testing.T) { utils.MustMatch(t, wantQueries, sbc.Queries) sbc.Queries = nil - _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view where id in (1,2,3,4,5)", nil) + _, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view where id in (1,2,3,4,5)", nil, false) require.NoError(t, err) bvtg1, _ := sqltypes.BuildBindVariable([]int64{1, 2, 3, 4, 5}) bvals, _ := sqltypes.BuildBindVariable([]int64{1, 2}) @@ -4304,7 +4311,7 @@ func TestWarmingReads(t *testing.T) { // Since queries on the replica will run in a separate go-routine, we need synchronization for the Queries field in the sandboxconn. replica.RequireQueriesLocking() - _, err := executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err := executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ {Sql: "select age, city from `user`"}, @@ -4319,7 +4326,7 @@ func TestWarmingReads(t *testing.T) { utils.MustMatch(t, wantQueriesReplica, replica.GetQueries()) replica.ClearQueries() - _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ {Sql: "select age, city from `user` /* already has a comment */"}, @@ -4334,17 +4341,17 @@ func TestWarmingReads(t *testing.T) { utils.MustMatch(t, wantQueriesReplica, replica.GetQueries()) replica.ClearQueries() - _, err = executor.Execute(ctx, nil, "TestSelect", session, "insert into user (age, city) values (5, 'Boston')", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestSelect", session, "insert into user (age, city) values (5, 'Boston')", map[string]*querypb.BindVariable{}, false) waitUntilQueryCount(t, replica, 0) require.NoError(t, err) require.Nil(t, replica.GetQueries()) - _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}, false) waitUntilQueryCount(t, replica, 0) require.NoError(t, err) require.Nil(t, replica.GetQueries()) - _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}, false) waitUntilQueryCount(t, replica, 0) require.NoError(t, err) require.Nil(t, replica.GetQueries()) @@ -4352,7 +4359,7 @@ func TestWarmingReads(t *testing.T) { executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, ctx, 0) replica.RequireQueriesLocking() - _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}, false) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ {Sql: "select age, city from `user`"}, @@ -4434,32 +4441,32 @@ func TestSysVarGlobalAndSession(t *testing.T) { sqltypes.MakeTestResult(sqltypes.MakeTestFields("reserve_execute", "uint64")), sqltypes.MakeTestResult(sqltypes.MakeTestFields("@@global.innodb_lock_wait_timeout", "uint64"), "20"), }) - qr, err := executor.Execute(context.Background(), nil, "TestSetStmt", session, + qr, err := executorExecSession(context.Background(), executor, session, "select @@innodb_lock_wait_timeout", nil) require.NoError(t, err) require.Equal(t, `[[UINT64(20)]]`, fmt.Sprintf("%v", qr.Rows)) - qr, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, + qr, err = executorExecSession(context.Background(), executor, session, "select @@global.innodb_lock_wait_timeout", nil) require.NoError(t, err) require.Equal(t, `[[UINT64(20)]]`, fmt.Sprintf("%v", qr.Rows)) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, + _, err = executorExecSession(context.Background(), executor, session, "set @@global.innodb_lock_wait_timeout = 120", nil) require.NoError(t, err) require.Empty(t, session.SystemVariables["innodb_lock_wait_timeout"]) - _, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, + _, err = executorExecSession(context.Background(), executor, session, "set @@innodb_lock_wait_timeout = 40", nil) require.NoError(t, err) require.EqualValues(t, "40", session.SystemVariables["innodb_lock_wait_timeout"]) - qr, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, + qr, err = executorExecSession(context.Background(), executor, session, "select @@innodb_lock_wait_timeout", nil) require.NoError(t, err) require.Equal(t, `[[INT64(40)]]`, fmt.Sprintf("%v", qr.Rows)) - qr, err = executor.Execute(context.Background(), nil, "TestSetStmt", session, + qr, err = executorExecSession(context.Background(), executor, session, "select @@global.innodb_lock_wait_timeout", nil) require.NoError(t, err) require.Equal(t, `[[UINT64(20)]]`, fmt.Sprintf("%v", qr.Rows)) diff --git a/go/vt/vtgate/executor_set_test.go b/go/vt/vtgate/executor_set_test.go index 310d885a134..c48232c5dd8 100644 --- a/go/vt/vtgate/executor_set_test.go +++ b/go/vt/vtgate/executor_set_test.go @@ -268,7 +268,7 @@ func TestExecutorSet(t *testing.T) { for i, tcase := range testcases { t.Run(fmt.Sprintf("%d-%s", i, tcase.in), func(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true}) - _, err := executorEnv.Execute(ctx, nil, "TestExecute", session, tcase.in, nil) + _, err := executorExecSession(ctx, executorEnv, session, tcase.in, nil) if tcase.err == "" { require.NoError(t, err) utils.MustMatch(t, tcase.out, session.Session, "new executor") @@ -381,7 +381,7 @@ func TestExecutorSetOp(t *testing.T) { session.TargetString = KsTestUnsharded session.EnableSystemSettings = !tcase.disallowResConn sbclookup.SetResults([]*sqltypes.Result{tcase.result}) - _, err := executor.Execute(ctx, nil, "TestExecute", session, tcase.in, nil) + _, err := executorExecSession(ctx, executor, session, tcase.in, nil) require.NoError(t, err) utils.MustMatch(t, tcase.warning, session.Warnings, "") utils.MustMatch(t, tcase.sysVars, session.SystemVariables, "") @@ -396,7 +396,7 @@ func TestExecutorSetMetadata(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set := "set @@vitess_metadata.app_keyspace_v1= '1'" - _, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil) + _, err := executorExecSession(ctx, executor, session, set, nil) assert.Equalf(t, vtrpcpb.Code_PERMISSION_DENIED, vterrors.Code(err), "expected error %v, got error: %v", vtrpcpb.Code_PERMISSION_DENIED, err) }) @@ -410,11 +410,11 @@ func TestExecutorSetMetadata(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set := "set @@vitess_metadata.app_keyspace_v1= '1'" - _, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil) + _, err := executorExecSession(ctx, executor, session, set, nil) require.NoError(t, err, "%s error: %v", set, err) show := `show vitess_metadata variables like 'app\\_keyspace\\_v_'` - result, err := executor.Execute(ctx, nil, "TestExecute", session, show, nil) + result, err := executorExecSession(ctx, executor, session, show, nil) require.NoError(t, err) want := "1" @@ -423,11 +423,11 @@ func TestExecutorSetMetadata(t *testing.T) { // Update metadata set = "set @@vitess_metadata.app_keyspace_v2='2'" - _, err = executor.Execute(ctx, nil, "TestExecute", session, set, nil) + _, err = executorExecSession(ctx, executor, session, set, nil) require.NoError(t, err, "%s error: %v", set, err) show = `show vitess_metadata variables like 'app\\_keyspace\\_v%'` - gotqr, err := executor.Execute(ctx, nil, "TestExecute", session, show, nil) + gotqr, err := executorExecSession(ctx, executor, session, show, nil) require.NoError(t, err) wantqr := &sqltypes.Result{ @@ -443,7 +443,7 @@ func TestExecutorSetMetadata(t *testing.T) { assert.ElementsMatch(t, wantqr.Rows, gotqr.Rows) show = "show vitess_metadata variables" - gotqr, err = executor.Execute(ctx, nil, "TestExecute", session, show, nil) + gotqr, err = executorExecSession(ctx, executor, session, show, nil) require.NoError(t, err) assert.Equal(t, wantqr.Fields, gotqr.Fields) @@ -471,7 +471,7 @@ func TestPlanExecutorSetUDV(t *testing.T) { for _, tcase := range testcases { t.Run(tcase.in, func(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true}) - _, err := executor.Execute(ctx, nil, "TestExecute", session, tcase.in, nil) + _, err := executorExecSession(ctx, executor, session, tcase.in, nil) if err != nil { require.EqualError(t, err, tcase.err) } else { @@ -522,7 +522,7 @@ func TestSetVar(t *testing.T) { sqltypes.MakeTestFields("orig|new", "varchar|varchar"), "|only_full_group_by")}) - _, err := executor.Execute(ctx, nil, "TestSetVar", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err := executorExecSession(ctx, executor, session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) require.NoError(t, err) tcases := []struct { @@ -544,7 +544,7 @@ func TestSetVar(t *testing.T) { // reset reserved conn need. session.SetReservedConn(false) - _, err = executor.Execute(ctx, nil, "TestSetVar", session, tc.sql, map[string]*querypb.BindVariable{}) + _, err = executorExecSession(ctx, executor, session, tc.sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) assert.Equal(t, tc.rc, session.InReservedConn()) }) @@ -565,11 +565,11 @@ func TestSetVarShowVariables(t *testing.T) { sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|varchar"), "sql_mode|ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE")}) - _, err := executor.Execute(ctx, nil, "TestSetVar", session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) + _, err := executorExecSession(ctx, executor, session, "set @@sql_mode = only_full_group_by", map[string]*querypb.BindVariable{}) require.NoError(t, err) // this should return the updated value of sql_mode. - qr, err := executor.Execute(ctx, nil, "TestSetVar", session, "show variables like 'sql_mode'", map[string]*querypb.BindVariable{}) + qr, err := executorExecSession(ctx, executor, session, "show variables like 'sql_mode'", map[string]*querypb.BindVariable{}) require.NoError(t, err) assert.False(t, session.InReservedConn(), "reserved connection should not be used") assert.Equal(t, `[[VARCHAR("sql_mode") VARCHAR("only_full_group_by")]]`, fmt.Sprintf("%v", qr.Rows)) @@ -611,7 +611,7 @@ func TestExecutorSetAndSelect(t *testing.T) { sqltypes.MakeTestResult(nil)}) // third one for new set query setQ := fmt.Sprintf("set %s = '%s'", tcase.sysVar, tcase.val) - _, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, setQ, nil) + _, err := executorExecSession(ctx, e, session, setQ, nil) require.NoError(t, err) } @@ -619,7 +619,7 @@ func TestExecutorSetAndSelect(t *testing.T) { // if the query reaches the shard, it will return REPEATABLE-READ isolation level. sbc.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult(sqltypes.MakeTestFields(tcase.sysVar, "varchar"), "REPEATABLE-READ")}) - qr, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, selectQ, nil) + qr, err := executorExecSession(ctx, e, session, selectQ, nil) require.NoError(t, err) assert.Equal(t, tcase.exp, fmt.Sprintf("%v", qr.Rows)) }) @@ -634,12 +634,12 @@ func TestExecutorTimeZone(t *testing.T) { session := econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded, EnableSystemSettings: true}) session.SetSystemVariable("time_zone", "'+08:00'") - qr, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, "select now()", nil) + qr, err := executorExecSession(ctx, e, session, "select now()", nil) require.NoError(t, err) session.SetSystemVariable("time_zone", "'+02:00'") - qrWith, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, "select now()", nil) + qrWith, err := executorExecSession(ctx, e, session, "select now()", nil) require.NoError(t, err) assert.False(t, qr.Rows[0][0].Equal(qrWith.Rows[0][0]), "%v vs %v", qr.Rows[0][0].ToString(), qrWith.Rows[0][0].ToString()) diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 5e7e5c2a07d..970fa6a00be 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -29,6 +29,10 @@ import ( "strings" "testing" "time" + "unsafe" + + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/srvtopo" "github.com/google/go-cmp/cmp" "github.com/google/safehtml/template" @@ -61,6 +65,80 @@ import ( "vitess.io/vitess/go/vt/vtgate/vtgateservice" ) +type fakeResolver struct { + gw srvtopo.Gateway + resolveShards []*srvtopo.ResolvedShard +} + +func (f *fakeResolver) GetGateway() srvtopo.Gateway { + return f.gw +} + +func (f *fakeResolver) ResolveDestinations(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids []*querypb.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { + return f.resolveShards, nil, nil +} + +func (f *fakeResolver) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, tabletType topodatapb.TabletType, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { + panic("implement me") +} + +var _ econtext.Resolver = (*fakeResolver)(nil) + +func TestPlanKey(t *testing.T) { + // TestPlanKey verifies that buildPlanKey produces the expected PlanKey.DebugString + // values under different target, shard, and SET_VAR comment conditions. + ks1 := &vindexes.Keyspace{Name: "ks1"} + ks1Schema := &vindexes.KeyspaceSchema{Keyspace: ks1} + vschemaWith1KS := &vindexes.VSchema{ + Keyspaces: map[string]*vindexes.KeyspaceSchema{ + ks1.Name: ks1Schema, + }, + } + + type testCase struct { + targetString string + expectedPlanPrefixKey string + resolvedShard []*srvtopo.ResolvedShard + setVarComment string + } + + tests := []testCase{{ + targetString: "", + expectedPlanPrefixKey: "CurrentKeyspace: ks1, Destination: , Query: SELECT 1, SetVarComment: , Collation: 255", + }, { + setVarComment: "sEtVaRcOmMeNt", + expectedPlanPrefixKey: "CurrentKeyspace: ks1, Destination: , Query: SELECT 1, SetVarComment: sEtVaRcOmMeNt, Collation: 255", + }, { + targetString: "ks1@replica", + expectedPlanPrefixKey: "CurrentKeyspace: ks1, Destination: , Query: SELECT 1, SetVarComment: , Collation: 255", + }, { + targetString: "ks1:-80", + expectedPlanPrefixKey: "CurrentKeyspace: ks1, Destination: DestinationShard(-80), Query: SELECT 1, SetVarComment: , Collation: 255", + }, { + targetString: "ks1[deadbeef]", + resolvedShard: []*srvtopo.ResolvedShard{ + {Target: &querypb.Target{Keyspace: "ks1", Shard: "-66"}}, + {Target: &querypb.Target{Keyspace: "ks1", Shard: "66-"}}}, + expectedPlanPrefixKey: "CurrentKeyspace: ks1, Destination: -66,66-, Query: SELECT 1, SetVarComment: , Collation: 255", + }} + cfg := econtext.VCursorConfig{ + Collation: collations.CollationUtf8mb4ID, + DefaultTabletType: topodatapb.TabletType_PRIMARY, + } + + e, _, _, _, ctx := createExecutorEnv(t) + e.vschema = vschemaWith1KS + for i, tc := range tests { + t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) { + ss := econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetString}) + resolver := &fakeResolver{resolveShards: tc.resolvedShard} + vc, _ := econtext.NewVCursorImpl(ss, makeComments(""), e, nil, e.vm, e.VSchema(), resolver, nil, nullResultsObserver{}, cfg) + key := buildPlanKey(ctx, vc, "SELECT 1", tc.setVarComment) + require.Equal(t, tc.expectedPlanPrefixKey, key.DebugString(), "test case %d", i) + }) + } +} + func TestExecutorResultsExceeded(t *testing.T) { executor, _, _, sbclookup, ctx := createExecutorEnv(t) @@ -76,11 +154,11 @@ func TestExecutorResultsExceeded(t *testing.T) { result2 := sqltypes.MakeTestResult(sqltypes.MakeTestFields("col", "int64"), "1", "2", "3", "4") sbclookup.SetResults([]*sqltypes.Result{result1, result2}) - _, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from main1", nil) + _, err := executorExecSession(ctx, executor, session, "select * from main1", nil) require.NoError(t, err) assert.Equal(t, initial, warnings.Counts()["ResultsExceeded"], "warnings count") - _, err = executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select * from main1", nil) require.NoError(t, err) assert.Equal(t, initial+1, warnings.Counts()["ResultsExceeded"], "warnings count") } @@ -110,7 +188,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) { stmt, err := sqlparser.NewTestParser().Parse(test.query) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecutorMaxMemoryRowsExceeded", session, test.query, nil) + _, err = executorExecSession(ctx, executor, session, test.query, nil) if sqlparser.IgnoreMaxMaxMemoryRowsDirective(stmt) { require.NoError(t, err, "no error when DirectiveIgnoreMaxMemoryRows is provided") } else { @@ -132,7 +210,7 @@ func TestExecutorTransactionsNoAutoCommit(t *testing.T) { defer executor.queryLogger.Unsubscribe(logChan) // begin. - _, err := executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err := executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) wantSession := &vtgatepb.Session{InTransaction: true, TargetString: "@primary", SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -142,35 +220,27 @@ func TestExecutorTransactionsNoAutoCommit(t *testing.T) { assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // commit. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) logStats = testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select id from main1", 1) assert.EqualValues(t, 0, logStats.CommitTime, "logstats: expected zero CommitTime") assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") - _, err = executor.Execute(context.Background(), nil, "TestExecute", session, "commit", nil) - if err != nil { - t.Fatal(err) - } + _, err = executorExecSession(context.Background(), executor, session, "commit", nil) + require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", SessionUUID: "suuid"} - if !proto.Equal(session.Session, wantSession) { - t.Errorf("begin: %v, want %v", session.Session, wantSession) - } - if commitCount := sbclookup.CommitCount.Load(); commitCount != 1 { - t.Errorf("want 1, got %d", commitCount) - } + assert.Truef(t, proto.Equal(session.Session, wantSession), "begin: %v, want %v", session.Session, wantSession) + assert.EqualValues(t, 1, sbclookup.CommitCount.Load(), "commit count") logStats = testQueryLog(t, executor, logChan, "TestExecute", "COMMIT", "commit", 1) - if logStats.CommitTime == 0 { - t.Errorf("logstats: expected non-zero CommitTime") - } + assert.NotZero(t, logStats.CommitTime) assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // rollback. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err = executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "rollback", nil) + _, err = executorExecSession(ctx, executor, session, "rollback", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -178,22 +248,18 @@ func TestExecutorTransactionsNoAutoCommit(t *testing.T) { _ = testQueryLog(t, executor, logChan, "TestExecute", "BEGIN", "begin", 0) _ = testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select id from main1", 1) logStats = testQueryLog(t, executor, logChan, "TestExecute", "ROLLBACK", "rollback", 1) - if logStats.CommitTime == 0 { - t.Errorf("logstats: expected non-zero CommitTime") - } + assert.NotZero(t, logStats.CommitTime) assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // CloseSession doesn't log anything err = executor.CloseSession(ctx, session) require.NoError(t, err) logStats = getQueryLog(logChan) - if logStats != nil { - t.Errorf("logstats: expected no record for no-op rollback, got %v", logStats) - } + assert.Nil(t, logStats, "logstats: expected nil") // Prevent use of non-primary if in_transaction is on. session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", InTransaction: true}) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "use @replica", nil) + _, err = executorExecSession(ctx, executor, session, "use @replica", nil) require.EqualError(t, err, `can't execute the given command because you have an active transaction`) } @@ -207,7 +273,7 @@ func TestDirectTargetRewrites(t *testing.T) { } sql := "select database()" - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{}) + _, err := executorExec(ctx, executor, session, sql, map[string]*querypb.BindVariable{}) require.NoError(t, err) assertQueries(t, sbclookup, []*querypb.BoundQuery{{ Sql: "select :__vtdbname as `database()` from dual", @@ -224,7 +290,7 @@ func TestExecutorTransactionsAutoCommit(t *testing.T) { defer executor.queryLogger.Unsubscribe(logChan) // begin. - _, err := executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err := executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) wantSession := &vtgatepb.Session{InTransaction: true, TargetString: "@primary", Autocommit: true, SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -235,9 +301,9 @@ func TestExecutorTransactionsAutoCommit(t *testing.T) { assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // commit. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "commit", nil) + _, err = executorExecSession(ctx, executor, session, "commit", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", Autocommit: true, SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -251,11 +317,11 @@ func TestExecutorTransactionsAutoCommit(t *testing.T) { assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // rollback. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err = executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "rollback", nil) + _, err = executorExecSession(ctx, executor, session, "rollback", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", Autocommit: true, SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -307,9 +373,9 @@ func TestExecutorTransactionsAutoCommitStreaming(t *testing.T) { assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // commit. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "commit", nil) + _, err = executorExecSession(ctx, executor, session, "commit", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", Autocommit: true, Options: oltpOptions, SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -323,11 +389,11 @@ func TestExecutorTransactionsAutoCommitStreaming(t *testing.T) { assert.EqualValues(t, "suuid", logStats.SessionUUID, "logstats: expected non-empty SessionUUID") // rollback. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err = executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err = executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "rollback", nil) + _, err = executorExecSession(ctx, executor, session, "rollback", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{TargetString: "@primary", Autocommit: true, Options: oltpOptions, SessionUUID: "suuid"} utils.MustMatch(t, wantSession, session.Session, "session") @@ -344,25 +410,25 @@ func TestExecutorDeleteMetadata(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true}) set := "set @@vitess_metadata.app_v1= '1'" - _, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil) + _, err := executorExecSession(ctx, executor, session, set, nil) assert.NoError(t, err, "%s error: %v", set, err) show := `show vitess_metadata variables like 'app\\_%'` - result, _ := executor.Execute(ctx, nil, "TestExecute", session, show, nil) + result, _ := executorExecSession(ctx, executor, session, show, nil) assert.Len(t, result.Rows, 1) // Fails if deleting key that doesn't exist delQuery := "set @@vitess_metadata.doesn't_exist=''" - _, err = executor.Execute(ctx, nil, "TestExecute", session, delQuery, nil) + _, err = executorExecSession(ctx, executor, session, delQuery, nil) assert.True(t, topo.IsErrType(err, topo.NoNode)) // Delete existing key, show should fail given the node doesn't exist delQuery = "set @@vitess_metadata.app_v1=''" - _, err = executor.Execute(ctx, nil, "TestExecute", session, delQuery, nil) + _, err = executorExecSession(ctx, executor, session, delQuery, nil) assert.NoError(t, err) show = `show vitess_metadata variables like 'app\\_%'` - _, err = executor.Execute(ctx, nil, "TestExecute", session, show, nil) + _, err = executorExecSession(ctx, executor, session, show, nil) assert.True(t, topo.IsErrType(err, topo.NoNode)) } @@ -376,7 +442,7 @@ func TestExecutorAutocommit(t *testing.T) { // autocommit = 0 startCount := sbclookup.CommitCount.Load() - _, err := executor.Execute(ctx, nil, "TestExecute", session, "select id from main1", nil) + _, err := executorExecSession(ctx, executor, session, "select id from main1", nil) require.NoError(t, err) wantSession := &vtgatepb.Session{TargetString: "@primary", InTransaction: true, FoundRows: 1, RowCount: -1} testSession := session.Session.CloneVT() @@ -392,7 +458,7 @@ func TestExecutorAutocommit(t *testing.T) { } // autocommit = 1 - _, err = executor.Execute(ctx, nil, "TestExecute", session, "set autocommit=1", nil) + _, err = executorExecSession(ctx, executor, session, "set autocommit=1", nil) require.NoError(t, err) _ = testQueryLog(t, executor, logChan, "TestExecute", "SET", "set @@autocommit = 1", 0) @@ -401,7 +467,7 @@ func TestExecutorAutocommit(t *testing.T) { t.Errorf("Commit count: %d, want %d", got, want) } - _, err = executor.Execute(ctx, nil, "TestExecute", session, "update main1 set id=1", nil) + _, err = executorExecSession(ctx, executor, session, "update main1 set id=1", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{Autocommit: true, TargetString: "@primary", FoundRows: 0, RowCount: 1} utils.MustMatch(t, wantSession, session.Session, "session does not match for autocommit=1") @@ -413,11 +479,11 @@ func TestExecutorAutocommit(t *testing.T) { // autocommit = 1, "begin" session.ResetTx() startCount = sbclookup.CommitCount.Load() - _, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err = executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) _ = testQueryLog(t, executor, logChan, "TestExecute", "BEGIN", "begin", 0) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "update main1 set id=1", nil) + _, err = executorExecSession(ctx, executor, session, "update main1 set id=1", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{InTransaction: true, Autocommit: true, TargetString: "@primary", FoundRows: 0, RowCount: 1} testSession = session.Session.CloneVT() @@ -435,7 +501,7 @@ func TestExecutorAutocommit(t *testing.T) { t.Errorf("logstats: expected non-zero RowsAffected") } - _, err = executor.Execute(ctx, nil, "TestExecute", session, "commit", nil) + _, err = executorExecSession(ctx, executor, session, "commit", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{Autocommit: true, TargetString: "@primary"} if !proto.Equal(session.Session, wantSession) { @@ -449,14 +515,14 @@ func TestExecutorAutocommit(t *testing.T) { // transition autocommit from 0 to 1 in the middle of a transaction. startCount = sbclookup.CommitCount.Load() session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil) + _, err = executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "update main1 set id=1", nil) + _, err = executorExecSession(ctx, executor, session, "update main1 set id=1", nil) require.NoError(t, err) if got, want := sbclookup.CommitCount.Load(), startCount; got != want { t.Errorf("Commit count: %d, want %d", got, want) } - _, err = executor.Execute(ctx, nil, "TestExecute", session, "set autocommit=1", nil) + _, err = executorExecSession(ctx, executor, session, "set autocommit=1", nil) require.NoError(t, err) wantSession = &vtgatepb.Session{Autocommit: true, TargetString: "@primary"} if !proto.Equal(session.Session, wantSession) { @@ -480,7 +546,7 @@ func TestExecutorShowColumns(t *testing.T) { } for _, query := range queries { t.Run(query, func(t *testing.T) { - _, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{{ @@ -525,28 +591,28 @@ func TestExecutorShow(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) for _, query := range []string{"show vitess_keyspaces", "show keyspaces"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) assertMatchesNoOrder(t, `[[VARCHAR("TestUnsharded")] [VARCHAR("TestMultiCol")] [VARCHAR("TestXBadVSchema")] [VARCHAR("TestXBadSharding")] [VARCHAR("TestExecutor")]]`, fmt.Sprintf("%v", qr.Rows)) } for _, query := range []string{"show databases", "show DATABASES", "show schemas", "show SCHEMAS"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Showing default tables (5+4[default]) assertMatchesNoOrder(t, `[[VARCHAR("TestUnsharded")] [VARCHAR("TestMultiCol")] [VARCHAR("TestXBadVSchema")] [VARCHAR("TestXBadSharding")] [VARCHAR("TestExecutor")]] [VARCHAR("information_schema")] [VARCHAR("mysql")] [VARCHAR("sys")] [VARCHAR("performance_schema")]`, fmt.Sprintf("%v", qr.Rows)) } - _, err := executor.Execute(ctx, nil, "TestExecute", session, "show variables", nil) + _, err := executorExecSession(ctx, executor, session, "show variables", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show collation", nil) + _, err = executorExecSession(ctx, executor, session, "show collation", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show collation where `Charset` = 'utf8' and `Collation` = 'utf8_bin'", nil) + _, err = executorExecSession(ctx, executor, session, "show collation where `Charset` = 'utf8' and `Collation` = 'utf8_bin'", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "use @primary", nil) + _, err = executorExecSession(ctx, executor, session, "use @primary", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show tables", nil) + _, err = executorExecSession(ctx, executor, session, "show tables", nil) assert.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "'show tables' should fail without a keyspace") assert.Empty(t, sbclookup.Queries, "sbclookup unexpectedly has queries already") @@ -563,7 +629,7 @@ func TestExecutorShow(t *testing.T) { sbclookup.SetResults([]*sqltypes.Result{showResults}) query := fmt.Sprintf("show tables from %v", KsTestUnsharded) - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) assert.Equal(t, 1, len(sbclookup.Queries), "Tablet should have received one 'show' query. Instead received: %v", sbclookup.Queries) @@ -575,88 +641,88 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, fmt.Sprintf("unexpected results running query: %s", query)) wantErrNoTable := "table unknown_table not found" - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show create table unknown_table", nil) + _, err = executorExecSession(ctx, executor, session, "show create table unknown_table", nil) assert.EqualErrorf(t, err, wantErrNoTable, "Got: %v. Want: %v", wantErrNoTable) // SHOW CREATE table using vschema to find keyspace. - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show create table user_seq", nil) + _, err = executorExecSession(ctx, executor, session, "show create table user_seq", nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql wantQuery := "show create table user_seq" assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // SHOW CREATE table with query-provided keyspace - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show create table %v.unknown", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show create table %v.unknown", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql wantQuery = "show create table `unknown`" assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // SHOW KEYS with two different syntax - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show keys from %v.unknown", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show keys from %v.unknown", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql wantQuery = "show indexes from `unknown`" assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show keys from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show keys from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // SHOW INDEX with two different syntax - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show index from %v.unknown", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show index from %v.unknown", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show index from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show index from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // SHOW INDEXES with two different syntax - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show indexes from %v.unknown", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show indexes from %v.unknown", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show indexes from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show indexes from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // SHOW EXTENDED {INDEX | INDEXES | KEYS} - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show extended index from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show extended index from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show extended indexes from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show extended indexes from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show extended keys from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show extended keys from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) lastQuery = sbclookup.Queries[len(sbclookup.Queries)-1].Sql assert.Equal(t, wantQuery, lastQuery, "Got: %v. Want: %v", lastQuery, wantQuery) // Set destination keyspace in session session.TargetString = KsTestUnsharded - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show create table unknown", nil) + _, err = executorExecSession(ctx, executor, session, "show create table unknown", nil) require.NoError(t, err) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show full columns from table1", nil) + _, err = executorExecSession(ctx, executor, session, "show full columns from table1", nil) require.NoError(t, err) // Reset target string so other tests dont fail. session.TargetString = "@primary" - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show full columns from unknown from %v", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show full columns from unknown from %v", KsTestUnsharded), nil) require.NoError(t, err) for _, query := range []string{"show charset like 'utf8%'", "show character set like 'utf8%'"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr := &sqltypes.Result{ Fields: append(buildVarCharFields("Charset", "Description", "Default collation"), &querypb.Field{Name: "Maxlen", Type: sqltypes.Uint32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG | querypb.MySqlFlag_NOT_NULL_FLAG | querypb.MySqlFlag_UNSIGNED_FLAG | querypb.MySqlFlag_NO_DEFAULT_VALUE_FLAG)}), @@ -678,7 +744,7 @@ func TestExecutorShow(t *testing.T) { } for _, query := range []string{"show charset like '%foo'", "show character set like 'foo%'", "show charset like 'foo%'", "show character set where charset like '%foo'", "show charset where charset = '%foo'"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr := &sqltypes.Result{ Fields: append(buildVarCharFields("Charset", "Description", "Default collation"), &querypb.Field{Name: "Maxlen", Type: sqltypes.Uint32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG | querypb.MySqlFlag_NOT_NULL_FLAG | querypb.MySqlFlag_UNSIGNED_FLAG | querypb.MySqlFlag_NO_DEFAULT_VALUE_FLAG)}), @@ -689,7 +755,7 @@ func TestExecutorShow(t *testing.T) { } for _, query := range []string{"show charset like 'utf8mb3'", "show character set like 'utf8mb3'", "show charset where charset = 'utf8mb3'", "show character set where charset = 'utf8mb3'"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr := &sqltypes.Result{ Fields: append(buildVarCharFields("Charset", "Description", "Default collation"), &querypb.Field{Name: "Maxlen", Type: sqltypes.Uint32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG | querypb.MySqlFlag_NOT_NULL_FLAG | querypb.MySqlFlag_UNSIGNED_FLAG | querypb.MySqlFlag_NO_DEFAULT_VALUE_FLAG)}), @@ -706,7 +772,7 @@ func TestExecutorShow(t *testing.T) { } for _, query := range []string{"show charset like 'utf8mb4'", "show character set like 'utf8mb4'", "show charset where charset = 'utf8mb4'", "show character set where charset = 'utf8mb4'"} { - qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err := executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr := &sqltypes.Result{ Fields: append(buildVarCharFields("Charset", "Description", "Default collation"), &querypb.Field{Name: "Maxlen", Type: sqltypes.Uint32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG | querypb.MySqlFlag_NOT_NULL_FLAG | querypb.MySqlFlag_UNSIGNED_FLAG | querypb.MySqlFlag_NO_DEFAULT_VALUE_FLAG)}), @@ -722,12 +788,12 @@ func TestExecutorShow(t *testing.T) { } for _, query := range []string{"show character set where foo like '%foo'"} { - _, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err := executorExecSession(ctx, executor, session, query, nil) require.Error(t, err) } query = "show engines" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Engine", "Support", "Comment", "Transactions", "XA", "Savepoints"), @@ -744,7 +810,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show plugins" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Name", "Status", "Type", "Library", "License"), @@ -760,7 +826,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) for _, sql := range []string{"show session status", "show session status like 'Ssl_cipher'"} { - qr, err = executor.Execute(ctx, nil, "TestExecute", session, sql, nil) + qr, err = executorExecSession(ctx, executor, session, sql, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: []*querypb.Field{ @@ -776,11 +842,11 @@ func TestExecutorShow(t *testing.T) { } // Test SHOW FULL COLUMNS FROM where query has a qualifier - _, err = executor.Execute(ctx, nil, "TestExecute", session, fmt.Sprintf("show full columns from %v.table1", KsTestUnsharded), nil) + _, err = executorExecSession(ctx, executor, session, fmt.Sprintf("show full columns from %v.table1", KsTestUnsharded), nil) require.NoError(t, err) query = "show vitess_shards" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Just test for first & last. @@ -795,7 +861,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vitess_shards like 'TestExecutor/%'" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Just test for first & last. @@ -810,7 +876,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vitess_shards like 'TestExec%/%'" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Just test for first & last. @@ -825,7 +891,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vitess_replication_status" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) qr.Rows = [][]sqltypes.Value{} wantqr = &sqltypes.Result{ @@ -834,7 +900,7 @@ func TestExecutorShow(t *testing.T) { } utils.MustMatch(t, wantqr, qr, query) query = "show vitess_replication_status like 'x'" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) qr.Rows = [][]sqltypes.Value{} wantqr = &sqltypes.Result{ @@ -844,7 +910,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vitess_tablets" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Just test for first & last. qr.Rows = [][]sqltypes.Value{qr.Rows[0], qr.Rows[len(qr.Rows)-1]} @@ -858,7 +924,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vitess_tablets like 'x'" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Cell", "Keyspace", "Shard", "TabletType", "State", "Alias", "Hostname", "PrimaryTermStartTime"), @@ -867,7 +933,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, fmt.Sprintf("%q should be empty", query)) query = "show vitess_tablets like '-20%'" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Cell", "Keyspace", "Shard", "TabletType", "State", "Alias", "Hostname", "PrimaryTermStartTime"), @@ -878,7 +944,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vschema vindexes" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Keyspace", "Name", "Type", "Params", "Owner"), @@ -909,7 +975,7 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vschema vindexes on TestExecutor.user" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -921,18 +987,18 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vschema vindexes on user" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) wantErr := econtext.ErrNoKeyspace.Error() assert.EqualError(t, err, wantErr, query) query = "show vschema vindexes on TestExecutor.garbage" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) wantErr = "VT05005: table 'garbage' does not exist in keyspace 'TestExecutor'" assert.EqualError(t, err, wantErr, query) query = "show vschema vindexes on user" session.TargetString = "TestExecutor" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -945,7 +1011,7 @@ func TestExecutorShow(t *testing.T) { query = "show vschema vindexes on user2" session.TargetString = "TestExecutor" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -957,12 +1023,12 @@ func TestExecutorShow(t *testing.T) { utils.MustMatch(t, wantqr, qr, query) query = "show vschema vindexes on garbage" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) wantErr = "VT05005: table 'garbage' does not exist in keyspace 'TestExecutor'" assert.EqualError(t, err, wantErr, query) query = "show warnings" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: []*querypb.Field{ @@ -976,7 +1042,7 @@ func TestExecutorShow(t *testing.T) { query = "show warnings" session.Warnings = []*querypb.QueryWarning{} - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: []*querypb.Field{ @@ -993,7 +1059,7 @@ func TestExecutorShow(t *testing.T) { {Code: uint32(sqlerror.ERBadTable), Message: "bad table"}, {Code: uint32(sqlerror.EROutOfResources), Message: "ks/-40: query timed out"}, } - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: []*querypb.Field{ @@ -1012,7 +1078,7 @@ func TestExecutorShow(t *testing.T) { // Make sure it still works when one of the keyspaces is in a bad state getSandbox(KsTestSharded).SrvKeyspaceMustFail++ query = "show vitess_shards" - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) // Just test for first & last. qr.Rows = [][]sqltypes.Value{qr.Rows[0], qr.Rows[len(qr.Rows)-1]} @@ -1027,7 +1093,7 @@ func TestExecutorShow(t *testing.T) { query = "show vschema tables" session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) - qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + qr, err = executorExecSession(ctx, executor, session, query, nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Tables"), @@ -1053,28 +1119,28 @@ func TestExecutorShow(t *testing.T) { query = "show vschema tables" session = econtext.NewSafeSession(&vtgatepb.Session{}) - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) want = econtext.ErrNoKeyspace.Error() assert.EqualError(t, err, want, query) query = "show 10" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) want = "syntax error at position 8 near '10'" assert.EqualError(t, err, want, query) query = "show vschema tables" session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "no_such_keyspace"}) - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) want = "VT05003: unknown database 'no_such_keyspace' in vschema" assert.EqualError(t, err, want, query) query = "show vitess_migrations" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) want = "VT05003: unknown database 'no_such_keyspace' in vschema" assert.EqualError(t, err, want, query) query = "show vitess_migrations from ks like '9748c3b7_7fdb_11eb_ac2c_f875a4d24e90'" - _, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil) + _, err = executorExecSession(ctx, executor, session, query, nil) want = "VT05003: unknown database 'ks' in vschema" assert.EqualError(t, err, want, query) } @@ -1098,7 +1164,7 @@ func TestExecutorShowTargeted(t *testing.T) { } for _, sql := range queries { - _, err := executor.Execute(ctx, nil, "TestExecutorShowTargeted", session, sql, nil) + _, err := executorExecSession(ctx, executor, session, sql, nil) require.NoError(t, err) assert.NotZero(t, len(sbc2.Queries), "Tablet should have received 'show' query") lastQuery := sbc2.Queries[len(sbc2.Queries)-1].Sql @@ -1111,7 +1177,7 @@ func TestExecutorShowFromSystemSchema(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "mysql"}) - _, err := executor.Execute(ctx, nil, "TestExecutorShowFromSystemSchema", session, "show tables", nil) + _, err := executorExecSession(ctx, executor, session, "show tables", nil) require.NoError(t, err) } @@ -1129,21 +1195,19 @@ func TestExecutorUse(t *testing.T) { "TestExecutor:-80@primary", } for i, stmt := range stmts { - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) - if err != nil { - t.Error(err) - } + _, err := executorExecSession(ctx, executor, session, stmt, nil) + require.NoError(t, err) wantSession := &vtgatepb.Session{Autocommit: true, TargetString: want[i], RowCount: -1} utils.MustMatch(t, wantSession, session.Session, "session does not match") } - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "use 1", nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{}, "use 1", nil) wantErr := "syntax error at position 6 near '1'" if err == nil || err.Error() != wantErr { t.Errorf("got: %v, want %v", err, wantErr) } - _, err = executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "use UnexistentKeyspace", nil) + _, err = executorExec(ctx, executor, &vtgatepb.Session{}, "use UnexistentKeyspace", nil) require.EqualError(t, err, "VT05003: unknown database 'UnexistentKeyspace' in vschema") } @@ -1157,10 +1221,8 @@ func TestExecutorComment(t *testing.T) { wantResult := &sqltypes.Result{} for _, stmt := range stmts { - gotResult, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil) - if err != nil { - t.Error(err) - } + gotResult, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: KsTestUnsharded}, stmt, nil) + require.NoError(t, err) if !gotResult.Equal(wantResult) { t.Errorf("Exec %s: %v, want %v", stmt, gotResult, wantResult) } @@ -1242,7 +1304,7 @@ func TestExecutorDDL(t *testing.T) { sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) stmtType := "DDL" - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: tc.targetStr}, stmt, nil) if tc.hasNoKeyspaceErr { require.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "expect query to fail: %q", stmt) stmtType = "" // For error case, plan is not generated to query log will not contain any stmtType. @@ -1281,7 +1343,7 @@ func TestExecutorDDL(t *testing.T) { sbc1.ExecCount.Store(0) sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: ""}), stmt.input, nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: ""}, stmt.input, nil) if stmt.hasErr { assert.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "expect query to fail") testQueryLog(t, executor, logChan, "TestExecute", "", stmt.input, 0) @@ -1294,7 +1356,6 @@ func TestExecutorDDL(t *testing.T) { } func TestExecutorDDLFk(t *testing.T) { - mName := "TestExecutorDDLFk" stmts := []string{ "create table t1(id bigint primary key, foreign key (id) references t2(id))", "alter table t2 add foreign key (id) references t1(id) on delete cascade", @@ -1306,7 +1367,7 @@ func TestExecutorDDLFk(t *testing.T) { executor, _, _, sbc, ctx := createExecutorEnv(t) sbc.ExecCount.Store(0) executor.vConfig.ForeignKeyMode = fkMode(mode) - _, err := executor.Execute(ctx, nil, mName, econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: KsTestUnsharded}, stmt, nil) if mode == "allow" { require.NoError(t, err) require.EqualValues(t, 1, sbc.ExecCount.Load()) @@ -1341,7 +1402,7 @@ func TestExecutorAlterVSchemaKeyspace(t *testing.T) { } stmt := "alter vschema create vindex TestExecutor.test_vindex using hash" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, "TestExecutor", "test_vindex", vschemaUpdates, executor) @@ -1370,7 +1431,7 @@ func TestExecutorCreateVindexDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema create vindex test_vindex using hash" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, ks, "test_vindex", vschemaUpdates, executor) @@ -1378,7 +1439,7 @@ func TestExecutorCreateVindexDDL(t *testing.T) { t.Errorf("updated vschema did not contain test_vindex") } - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr := "vindex test_vindex already exists in keyspace TestExecutor" if err == nil || err.Error() != wantErr { t.Errorf("create duplicate vindex: %v, want %s", err, wantErr) @@ -1394,10 +1455,8 @@ func TestExecutorCreateVindexDDL(t *testing.T) { // ksNew := "test_new_keyspace" session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt = "alter vschema create vindex test_vindex2 using hash" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) - if err != nil { - t.Fatalf("error in %s: %v", stmt, err) - } + _, err = executorExecSession(ctx, executor, session, stmt, nil) + require.NoError(t, err) vschema, vindex = waitForVindex(t, ks, "test_vindex2", vschemaUpdates, executor) if vindex.Type != "hash" { @@ -1445,19 +1504,19 @@ func TestExecutorAddDropVschemaTableDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema add table test_table" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, append([]string{"test_table"}, vschemaTables...), executor) stmt = "alter vschema add table test_table2" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, append([]string{"test_table", "test_table2"}, vschemaTables...), executor) // Should fail adding a table on a sharded keyspace session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) stmt = "alter vschema add table test_table" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "add vschema table: unsupported on sharded keyspace TestExecutor") // No queries should have gone to any tablets @@ -1481,34 +1540,28 @@ func TestExecutorVindexDDLACL(t *testing.T) { // test that by default no users can perform the operation stmt := "alter vschema create vindex test_hash using hash" - _, err := executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctxRedUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`) - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'blueUser' is not authorized to perform vschema operations`) // test when all users are enabled vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%")) - _, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) - if err != nil { - t.Errorf("unexpected error '%v'", err) - } + _, err = executorExecSession(ctxRedUser, executor, session, stmt, nil) + require.NoError(t, err) stmt = "alter vschema create vindex test_hash2 using hash" - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) - if err != nil { - t.Errorf("unexpected error '%v'", err) - } + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) + require.NoError(t, err) // test when only one user is enabled vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("orangeUser, blueUser, greenUser")) - _, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxRedUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`) stmt = "alter vschema create vindex test_hash3 using hash" - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) - if err != nil { - t.Errorf("unexpected error '%v'", err) - } + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) + require.NoError(t, err) // restore the disallowed state vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("")) @@ -1517,7 +1570,7 @@ func TestExecutorVindexDDLACL(t *testing.T) { func TestExecutorUnrecognized(t *testing.T) { executor, _, _, _, ctx := createExecutorEnv(t) - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "invalid statement", nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{}, "invalid statement", nil) require.Error(t, err, "unrecognized statement: invalid statement'") } @@ -1529,7 +1582,7 @@ func TestExecutorDeniedErrorNoBuffer(t *testing.T) { session := econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: "@primary"}) startExec := time.Now() - _, err := executor.Execute(ctx, nil, "TestExecutorDeniedErrorNoBuffer", session, "select * from user", nil) + _, err := executorExecSession(ctx, executor, session, "select * from user", nil) require.NoError(t, err, "enforce denied tables not buffered") endExec := time.Now() require.GreaterOrEqual(t, endExec.Sub(startExec).Milliseconds(), int64(500)) @@ -1543,9 +1596,7 @@ func TestVSchemaStats(t *testing.T) { templ := template.New("") templ, err := templ.Parse(VSchemaTemplate) - if err != nil { - t.Fatalf("error parsing template: %v", err) - } + require.NoError(t, err) wr := &bytes.Buffer{} if err := templ.Execute(wr, stats); err != nil { t.Fatalf("error executing template: %v", err) @@ -1561,40 +1612,28 @@ var pv = querypb.ExecuteOptions_Gen4 func TestGetPlanUnnormalized(t *testing.T) { r, _, _, _, ctx := createExecutorEnv(t) - emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) - unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) + emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) + unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) query1 := "select * from music_user_map where id = 1" - plan1, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + plan1, logStats := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) wantSQL := query1 + " /* comment */" - if logStats1.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats1.SQL) - } + assert.Equal(t, wantSQL, logStats.SQL) - plan2, logStats2 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) - if plan1 != plan2 { - t.Errorf("getPlan(query1): plans must be equal: %p %p", plan1, plan2) - } + plan2, logStats := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + assert.Equal(t, unsafe.Pointer(plan1), unsafe.Pointer(plan2)) assertCacheContains(t, r, emptyvc, query1) - if logStats2.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats2.SQL) - } - plan3, logStats3 := getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) - if plan1 == plan3 { - t.Errorf("getPlan(query1, ks): plans must not be equal: %p %p", plan1, plan3) - } - if logStats3.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats3.SQL) - } - plan4, logStats4 := getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) - if plan3 != plan4 { - t.Errorf("getPlan(query1, ks): plans must be equal: %p %p", plan3, plan4) - } + assert.Equal(t, wantSQL, logStats.SQL) + + plan3, logStats := getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + assert.NotEqual(t, unsafe.Pointer(plan1), unsafe.Pointer(plan3)) + assert.Equal(t, wantSQL, logStats.SQL) + + plan4, logStats := getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + assert.Equal(t, unsafe.Pointer(plan3), unsafe.Pointer(plan4)) assertCacheContains(t, r, emptyvc, query1) assertCacheContains(t, r, unshardedvc, query1) - if logStats4.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats4.SQL) - } + assert.Equal(t, wantSQL, logStats.SQL) } func assertCacheSize(t *testing.T, c *PlanCache, expected int) { @@ -1617,23 +1656,18 @@ func assertCacheContains(t *testing.T, e *Executor, vc *econtext.VCursorImpl, sq return true }) } else { - h := e.hashPlan(context.Background(), vc, sql) - plan, _ = e.plans.Get(h, e.epoch.Load()) + h := buildPlanKey(context.Background(), vc, sql, "") + plan, _ = e.plans.Get(h.Hash(), e.epoch.Load()) } - require.Truef(t, plan != nil, "plan not found for query: %s", sql) + assert.Truef(t, plan != nil, "plan not found for query: %s", sql) return plan } -func getPlanCached(t *testing.T, ctx context.Context, e *Executor, vcursor *econtext.VCursorImpl, sql string, comments sqlparser.MarginComments, bindVars map[string]*querypb.BindVariable, skipQueryPlanCache bool) (*engine.Plan, *logstats.LogStats) { +func getPlanCached(t *testing.T, ctx context.Context, e *Executor, session *econtext.SafeSession, sql string, comments sqlparser.MarginComments, bindVars map[string]*querypb.BindVariable, skipQueryPlanCache bool) (*engine.Plan, *logstats.LogStats) { logStats := logstats.NewLogStats(ctx, "Test", "", "", nil, streamlog.NewQueryLogConfigForTest()) - vcursor.SafeSession = &econtext.SafeSession{ - Session: &vtgatepb.Session{ - Options: &querypb.ExecuteOptions{SkipQueryPlanCache: skipQueryPlanCache}}, - } + session.GetOrCreateOptions().SkipQueryPlanCache = skipQueryPlanCache - stmt, reservedVars, err := parseAndValidateQuery(sql, sqlparser.NewTestParser()) - require.NoError(t, err) - plan, err := e.getPlan(context.Background(), vcursor, sql, stmt, comments, bindVars, reservedVars /* normalize */, e.config.Normalize, logStats) + plan, _, _, err := e.fetchOrCreatePlan(ctx, session, comments.Leading+sql+comments.Trailing, bindVars, e.config.Normalize, false, logStats) require.NoError(t, err) // Wait for cache to settle @@ -1645,48 +1679,45 @@ func TestGetPlanCacheUnnormalized(t *testing.T) { t.Run("Cache", func(t *testing.T) { r, _, _, _, ctx := createExecutorEnv(t) - emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) + emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) query1 := "select * from music_user_map where id = 1" - _, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true) + _, logStats1 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true) assertCacheSize(t, r.plans, 0) wantSQL := query1 + " /* comment */" - if logStats1.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats1.SQL) - } + assert.Equal(t, wantSQL, logStats1.SQL) - _, logStats2 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment 2 */"), map[string]*querypb.BindVariable{}, false) + _, logStats2 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment 2 */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 1) wantSQL = query1 + " /* comment 2 */" - if logStats2.SQL != wantSQL { - t.Errorf("logstats sql want \"%s\" got \"%s\"", wantSQL, logStats2.SQL) - } + assert.Equal(t, wantSQL, logStats2.SQL) }) t.Run("Skip Cache", func(t *testing.T) { - // Skip cache using directive r, _, _, _, ctx := createExecutorEnv(t) unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) + // Skip cache using directive query1 := "insert /*vt+ SKIP_QUERY_PLAN_CACHE=1 */ into user(id) values (1), (2)" - getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 0) + // it will be cached now. query1 = "insert into user(id) values (1), (2)" - getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 1) // the target string will be resolved and become part of the plan cache key, which adds a new entry ksIDVc1, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) - getPlanCached(t, ctx, r, ksIDVc1, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, ksIDVc1.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 2) // the target string will be resolved and become part of the plan cache key, as it's an unsharded ks, it will be the same entry as above ksIDVc2, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) - getPlanCached(t, ctx, r, ksIDVc2, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, ksIDVc2.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 2) }) } @@ -1697,12 +1728,12 @@ func TestGetPlanCacheNormalized(t *testing.T) { emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) query1 := "select * from music_user_map where id = 1" - _, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true /* skipQueryPlanCache */) + _, logStats1 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true /* skipQueryPlanCache */) assertCacheSize(t, r.plans, 0) wantSQL := "select * from music_user_map where id = :id /* INT64 */ /* comment */" assert.Equal(t, wantSQL, logStats1.SQL) - _, logStats2 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false /* skipQueryPlanCache */) + _, logStats2 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false /* skipQueryPlanCache */) assertCacheSize(t, r.plans, 1) assert.Equal(t, wantSQL, logStats2.SQL) }) @@ -1713,21 +1744,21 @@ func TestGetPlanCacheNormalized(t *testing.T) { unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) query1 := "insert /*vt+ SKIP_QUERY_PLAN_CACHE=1 */ into user(id) values (1), (2)" - getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 0) query1 = "insert into user(id) values (1), (2)" - getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 1) // the target string will be resolved and become part of the plan cache key, which adds a new entry ksIDVc1, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) - getPlanCached(t, ctx, r, ksIDVc1, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, ksIDVc1.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 2) // the target string will be resolved and become part of the plan cache key, as it's an unsharded ks, it will be the same entry as above ksIDVc2, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) - getPlanCached(t, ctx, r, ksIDVc2, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) + getPlanCached(t, ctx, r, ksIDVc2.SafeSession, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false) assertCacheSize(t, r.plans, 2) }) } @@ -1735,15 +1766,15 @@ func TestGetPlanCacheNormalized(t *testing.T) { func TestGetPlanNormalized(t *testing.T) { r, _, _, _, ctx := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) - emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) - unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) + emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) + unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig) - query1 := "select * from music_user_map where id = 1" + query1 := "select * from music_user_map where id = 1" // 163 -- 80 query2 := "select * from music_user_map where id = 2" normalized := "select * from music_user_map where id = :id /* INT64 */" - plan1, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment 1 */"), map[string]*querypb.BindVariable{}, false) - plan2, logStats2 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment 2 */"), map[string]*querypb.BindVariable{}, false) + plan1, logStats1 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment 1 */"), map[string]*querypb.BindVariable{}, false) + plan2, logStats2 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query1, makeComments(" /* comment 2 */"), map[string]*querypb.BindVariable{}, false) assert.Equal(t, plan1, plan2) assertCacheContains(t, r, emptyvc, normalized) @@ -1753,18 +1784,18 @@ func TestGetPlanNormalized(t *testing.T) { wantSQL = normalized + " /* comment 2 */" assert.Equal(t, wantSQL, logStats2.SQL) - plan3, logStats3 := getPlanCached(t, ctx, r, emptyvc, query2, makeComments(" /* comment 3 */"), map[string]*querypb.BindVariable{}, false) + plan3, logStats3 := getPlanCached(t, ctx, r, emptyvc.SafeSession, query2, makeComments(" /* comment 3 */"), map[string]*querypb.BindVariable{}, false) assert.Equal(t, plan1, plan3) wantSQL = normalized + " /* comment 3 */" assert.Equal(t, wantSQL, logStats3.SQL) var logStats5 *logstats.LogStats - plan3, logStats5 = getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment 5 */"), map[string]*querypb.BindVariable{}, false) + plan3, logStats5 = getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment 5 */"), map[string]*querypb.BindVariable{}, false) assert.Equal(t, plan1, plan3) wantSQL = normalized + " /* comment 5 */" assert.Equal(t, wantSQL, logStats5.SQL) - plan4, _ := getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment 6 */"), map[string]*querypb.BindVariable{}, false) + plan4, _ := getPlanCached(t, ctx, r, unshardedvc.SafeSession, query1, makeComments(" /* comment 6 */"), map[string]*querypb.BindVariable{}, false) assert.Equal(t, plan1, plan4) assertCacheContains(t, r, emptyvc, normalized) assertCacheContains(t, r, unshardedvc, normalized) @@ -1792,21 +1823,14 @@ func TestGetPlanPriority(t *testing.T) { r, _, _, _, ctx := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) logStats := logstats.NewLogStats(ctx, "Test", "", "", nil, streamlog.NewQueryLogConfigForTest()) - vCursor, err := econtext.NewVCursorImpl(session, makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{}) - assert.NoError(t, err) - - stmt, err := sqlparser.NewTestParser().Parse(testCase.sql) - assert.NoError(t, err) - qh, _ := sqlparser.BuildQueryHints(stmt) - priorityFromStatement := qh.Priority - _, err = r.getPlan(context.Background(), vCursor, testCase.sql, stmt, makeComments("/* some comment */"), map[string]*querypb.BindVariable{}, nil, true, logStats) + plan, _, _, err := r.fetchOrCreatePlan(context.Background(), session, testCase.sql, map[string]*querypb.BindVariable{}, r.config.Normalize, false, logStats) if testCase.expectedError != nil { assert.ErrorIs(t, err, testCase.expectedError) } else { assert.NoError(t, err) - assert.Equal(t, testCase.expectedPriority, priorityFromStatement) - assert.Equal(t, testCase.expectedPriority, vCursor.SafeSession.Options.Priority) + assert.Equal(t, testCase.expectedPriority, plan.QueryHints.Priority) + assert.Equal(t, testCase.expectedPriority, session.Options.Priority) } }) } @@ -1994,7 +2018,7 @@ func TestExecutorMaxPayloadSizeExceeded(t *testing.T) { "delete from main1 where id=1", } for _, query := range testMaxPayloadSizeExceeded { - _, err := executor.Execute(context.Background(), nil, "TestExecutorMaxPayloadSizeExceeded", session, query, nil) + _, err := executorExecSession(context.Background(), executor, session, query, nil) require.NotNil(t, err) assert.EqualError(t, err, "query payload size above threshold") } @@ -2007,14 +2031,14 @@ func TestExecutorMaxPayloadSizeExceeded(t *testing.T) { "delete /*vt+ IGNORE_MAX_PAYLOAD_SIZE=1 */ from main1 where id=1", } for _, query := range testMaxPayloadSizeOverride { - _, err := executor.Execute(context.Background(), nil, "TestExecutorMaxPayloadSizeWithOverride", session, query, nil) + _, err := executorExecSession(context.Background(), executor, session, query, nil) assert.Equal(t, nil, err, "err should be nil") } assert.Equal(t, warningCount, warnings.Counts()["WarnPayloadSizeExceeded"], "warnings count") maxPayloadSize = 1000 for _, query := range testMaxPayloadSizeExceeded { - _, err := executor.Execute(context.Background(), nil, "TestExecutorMaxPayloadSizeExceeded", session, query, nil) + _, err := executorExecSession(context.Background(), executor, session, query, nil) assert.Equal(t, nil, err, "err should be nil") } assert.Equal(t, warningCount+4, warnings.Counts()["WarnPayloadSizeExceeded"], "warnings count") @@ -2041,7 +2065,7 @@ func TestExecutorClearsWarnings(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{ Warnings: []*querypb.QueryWarning{{Code: 234, Message: "oh noes"}}, }) - _, err := executor.Execute(context.Background(), nil, "TestExecute", session, "select 42", nil) + _, err := executorExecSession(context.Background(), executor, session, "select 42", nil) require.NoError(t, err) require.Empty(t, session.Warnings) } @@ -2073,7 +2097,7 @@ func TestServingKeyspaces(t *testing.T) { }) require.ElementsMatch(t, []string{"TestExecutor", "TestUnsharded"}, gw.GetServingKeyspaces()) - result, err := executor.Execute(ctx, nil, "TestServingKeyspaces", econtext.NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil) + result, err := executorExec(ctx, executor, &vtgatepb.Session{}, "select keyspace_name from dual", nil) require.NoError(t, err) require.Equal(t, `[[VARCHAR("TestExecutor")]]`, fmt.Sprintf("%v", result.Rows)) @@ -2089,7 +2113,7 @@ func TestServingKeyspaces(t *testing.T) { // Clear plan cache, to force re-planning of the query. executor.ClearPlans() require.ElementsMatch(t, []string{"TestUnsharded"}, gw.GetServingKeyspaces()) - result, err = executor.Execute(ctx, nil, "TestServingKeyspaces", econtext.NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil) + result, err = executorExec(ctx, executor, &vtgatepb.Session{}, "select keyspace_name from dual", nil) require.NoError(t, err) require.Equal(t, `[[VARCHAR("TestUnsharded")]]`, fmt.Sprintf("%v", result.Rows)) } @@ -2165,11 +2189,11 @@ func TestExecutorOther(t *testing.T) { sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: tc.targetStr}, stmt, nil) if tc.hasNoKeyspaceErr { assert.Error(t, err, econtext.ErrNoKeyspace.Error()) } else if tc.hasDestinationShardErr { - assert.Errorf(t, err, "Destination can only be a single shard for statement: %s", stmt) + assert.Errorf(t, err, "ShardDestination can only be a single shard for statement: %s", stmt) } else { assert.NoError(t, err) } @@ -2221,7 +2245,7 @@ func TestExecutorAnalyze(t *testing.T) { sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil) + _, err := executorExec(context.Background(), executor, &vtgatepb.Session{TargetString: tc.targetStr}, stmt, nil) require.NoError(t, err) utils.MustMatch(t, tc.wantCnts, cnts{ @@ -2285,7 +2309,7 @@ func TestExecutorExplainStmt(t *testing.T) { sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) - _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil) + _, err := executorExec(ctx, executor, &vtgatepb.Session{TargetString: tc.targetStr}, stmt, nil) assert.NoError(t, err) utils.MustMatch(t, tc.wantCnts, cnts{ @@ -2373,11 +2397,11 @@ func TestExecutorOtherAdmin(t *testing.T) { sbc2.ExecCount.Store(0) sbclookup.ExecCount.Store(0) - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil) + _, err := executorExec(context.Background(), executor, &vtgatepb.Session{TargetString: tc.targetStr}, stmt, nil) if tc.hasNoKeyspaceErr { assert.Error(t, err, econtext.ErrNoKeyspace.Error()) } else if tc.hasDestinationShardErr { - assert.Errorf(t, err, "Destination can only be a single shard for statement: %s, got: DestinationExactKeyRange(-)", stmt) + assert.Errorf(t, err, "ShardDestination can only be a single shard for statement: %s, got: DestinationExactKeyRange(-)", stmt) } else { assert.NoError(t, err) } @@ -2487,60 +2511,47 @@ func TestExecutorSavepointInTxWithReservedConn(t *testing.T) { sbc1.SetResults([]*sqltypes.Result{ sqltypes.MakeTestResult(sqltypes.MakeTestFields("orig|new", "varchar|varchar"), "a|"), }) - _, err := exec(executor, session, "set sql_mode = ''") - require.NoError(t, err) + queries := []string{ + "set sql_mode = ''", + "begin", + "savepoint a", + "select id from user where id = 1", + "savepoint b", + "release savepoint a", + "select id from user where id = 3", + "commit", + } + + for _, query := range queries { + _, err := exec(executor, session, query) + require.NoError(t, err) + } - _, err = exec(executor, session, "begin") - require.NoError(t, err) - _, err = exec(executor, session, "savepoint a") - require.NoError(t, err) - _, err = exec(executor, session, "select id from user where id = 1") - require.NoError(t, err) - _, err = exec(executor, session, "savepoint b") - require.NoError(t, err) - _, err = exec(executor, session, "release savepoint a") - require.NoError(t, err) - _, err = exec(executor, session, "select id from user where id = 3") - require.NoError(t, err) - _, err = exec(executor, session, "commit") - require.NoError(t, err) emptyBV := map[string]*querypb.BindVariable{} - sbc1WantQueries := []*querypb.BoundQuery{{ - Sql: "select @@sql_mode orig, '' new", BindVariables: emptyBV, - }, { - Sql: "set sql_mode = ''", BindVariables: emptyBV, - }, { - Sql: "savepoint a", BindVariables: emptyBV, - }, { - Sql: "select id from `user` where id = 1", BindVariables: emptyBV, - }, { - Sql: "savepoint b", BindVariables: emptyBV, - }, { - Sql: "release savepoint a", BindVariables: emptyBV, - }} + sbc1WantQueries := []*querypb.BoundQuery{ + {Sql: "select @@sql_mode orig, '' new", BindVariables: emptyBV}, + {Sql: "savepoint a", BindVariables: emptyBV}, + {Sql: "select /*+ SET_VAR(sql_mode = ' ') */ id from `user` where id = 1", BindVariables: emptyBV}, + {Sql: "savepoint b", BindVariables: emptyBV}, + {Sql: "release savepoint a", BindVariables: emptyBV}, + } - sbc2WantQueries := []*querypb.BoundQuery{{ - Sql: "set sql_mode = ''", BindVariables: emptyBV, - }, { - Sql: "savepoint a", BindVariables: emptyBV, - }, { - Sql: "savepoint b", BindVariables: emptyBV, - }, { - Sql: "release savepoint a", BindVariables: emptyBV, - }, { - Sql: "select id from `user` where id = 3", BindVariables: emptyBV, - }} + sbc2WantQueries := []*querypb.BoundQuery{ + {Sql: "savepoint a", BindVariables: emptyBV}, + {Sql: "savepoint b", BindVariables: emptyBV}, + {Sql: "release savepoint a", BindVariables: emptyBV}, + {Sql: "select /*+ SET_VAR(sql_mode = ' ') */ id from `user` where id = 3", BindVariables: emptyBV}} utils.MustMatch(t, sbc1WantQueries, sbc1.Queries, "") utils.MustMatch(t, sbc2WantQueries, sbc2.Queries, "") testQueryLog(t, executor, logChan, "TestExecute", "SET", "set @@sql_mode = ''", 1) testQueryLog(t, executor, logChan, "TestExecute", "BEGIN", "begin", 0) testQueryLog(t, executor, logChan, "TestExecute", "SAVEPOINT", "savepoint a", 0) - testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select id from `user` where id = 1", 1) + testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select /*+ SET_VAR(sql_mode = ' ') */ id from `user` where id = 1", 1) testQueryLog(t, executor, logChan, "TestExecute", "SAVEPOINT", "savepoint b", 1) testQueryLog(t, executor, logChan, "TestExecute", "RELEASE", "release savepoint a", 1) - testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select id from `user` where id = 3", 1) + testQueryLog(t, executor, logChan, "TestExecute", "SELECT", "select /*+ SET_VAR(sql_mode = ' ') */ id from `user` where id = 3", 1) testQueryLog(t, executor, logChan, "TestExecute", "COMMIT", "commit", 2) } @@ -2635,7 +2646,7 @@ func TestExecutorCallProc(t *testing.T) { sbc2.ExecCount.Store(0) sbcUnsharded.ExecCount.Store(0) - _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), "CALL proc()", nil) + _, err := executorExec(context.Background(), executor, &vtgatepb.Session{TargetString: tc.targetStr}, "CALL proc()", nil) if tc.hasNoKeyspaceErr { assert.EqualError(t, err, econtext.ErrNoKeyspace.Error()) } else if tc.unshardedOnlyErr { @@ -2660,7 +2671,7 @@ func TestExecutorTempTable(t *testing.T) { executor.vConfig.WarnShardedOnly = true creatQuery := "create temporary table temp_t(id bigint primary key)" session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) - _, err := executor.Execute(ctx, nil, "TestExecutorTempTable", session, creatQuery, nil) + _, err := executorExecSession(ctx, executor, session, creatQuery, nil) require.NoError(t, err) assert.EqualValues(t, 1, sbcUnsharded.ExecCount.Load()) assert.NotEmpty(t, session.Warnings) @@ -2668,7 +2679,7 @@ func TestExecutorTempTable(t *testing.T) { before := executor.plans.Len() - _, err = executor.Execute(ctx, nil, "TestExecutorTempTable", session, "select * from temp_t", nil) + _, err = executorExecSession(ctx, executor, session, "select * from temp_t", nil) require.NoError(t, err) assert.Equal(t, before, executor.plans.Len()) @@ -2679,7 +2690,7 @@ func TestExecutorShowVitessMigrations(t *testing.T) { showQuery := "show vitess_migrations" session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) - _, err := executor.Execute(ctx, nil, "", session, showQuery, nil) + _, err := executorExecSession(ctx, executor, session, showQuery, nil) require.NoError(t, err) assert.Contains(t, sbc1.StringQueries(), "show vitess_migrations") assert.Contains(t, sbc2.StringQueries(), "show vitess_migrations") @@ -2689,10 +2700,10 @@ func TestExecutorDescHash(t *testing.T) { executor, _, _, _, ctx := createExecutorEnv(t) session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) - _, err := executor.Execute(ctx, nil, "", session, "desc hash_index", nil) + _, err := executorExecSession(ctx, executor, session, "desc hash_index", nil) require.EqualError(t, err, "VT05004: table 'hash_index' does not exist") - _, err = executor.Execute(ctx, nil, "", session, "desc music", nil) + _, err = executorExecSession(ctx, executor, session, "desc music", nil) require.NoError(t, err) } @@ -2704,7 +2715,7 @@ func TestExecutorVExplainQueries(t *testing.T) { sbclookup.SetResults([]*sqltypes.Result{ sqltypes.MakeTestResult(sqltypes.MakeTestFields("name|user_id", "varchar|int64"), "apa|1", "apa|2"), }) - qr, err := executor.Execute(ctx, nil, "TestExecutorVExplainQueries", session, "vexplain queries select * from user where name = 'apa'", nil) + qr, err := executorExecSession(ctx, executor, session, "vexplain queries select * from user where name = 'apa'", nil) require.NoError(t, err) txt := fmt.Sprintf("%v\n", qr.Rows) lookupQuery := "select `name`, user_id from name_user_map where `name` in" @@ -2756,12 +2767,12 @@ func TestExecutorStartTxnStmt(t *testing.T) { for _, tcase := range tcases { t.Run(tcase.beginSQL, func(t *testing.T) { - _, err := executor.Execute(ctx, nil, "TestExecutorStartTxnStmt", session, tcase.beginSQL, nil) + _, err := executorExecSession(ctx, executor, session, tcase.beginSQL, nil) require.NoError(t, err) assert.Equal(t, tcase.expTxAccessMode, session.GetOrCreateOptions().TransactionAccessMode) - _, err = executor.Execute(ctx, nil, "TestExecutorStartTxnStmt", session, "rollback", nil) + _, err = executorExecSession(ctx, executor, session, "rollback", nil) require.NoError(t, err) }) @@ -2772,38 +2783,48 @@ func TestExecutorPrepareExecute(t *testing.T) { executor, _, _, _, _ := createExecutorEnvWithConfig(t, createExecutorConfigWithNormalizer()) session := econtext.NewAutocommitSession(&vtgatepb.Session{}) - // prepare statement. - _, err := executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user from 'select * from user where id = ?'", nil) - require.NoError(t, err) - prepData := session.PrepareStatement["prep_user"] - require.NotNil(t, prepData) - require.Equal(t, "select * from `user` where id = :v1", prepData.PrepareStatement) - require.EqualValues(t, 1, prepData.ParamsCount) + t.Run("prepare statement", func(t *testing.T) { + _, err := executorExecSession(context.Background(), executor, session, "prepare prep_user from 'select * from user where id = ?'", nil) + require.NoError(t, err) - // prepare statement using user defined variable - _, err = executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "set @udv_query = 'select * from user where id in (?,?,?)'", nil) - require.NoError(t, err) + prepData := session.PrepareStatement["prep_user"] + require.NotNil(t, prepData) + assert.Equal(t, "select * from user where id = ?", prepData.PrepareStatement) + assert.EqualValues(t, 1, prepData.ParamsCount) + }) - _, err = executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user2 from @udv_query", nil) - require.NoError(t, err) - prepData = session.PrepareStatement["prep_user2"] - require.NotNil(t, prepData) - require.Equal(t, "select * from `user` where id in (:v1, :v2, :v3)", prepData.PrepareStatement) - require.EqualValues(t, 3, prepData.ParamsCount) + t.Run("prepare statement using user defined variable", func(t *testing.T) { + _, err := executorExecSession(context.Background(), executor, session, "set @udv_query = 'select * from user where id in (?,?,?)'", nil) + require.NoError(t, err) + _, err = executorExecSession(context.Background(), executor, session, "prepare prep_user2 from @udv_query", nil) + require.NoError(t, err) - // syntax error on prepared query - _, err = executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user2 from 'select'", nil) - require.Error(t, err) - require.Nil(t, session.PrepareStatement["prep_user2"]) // prepared statement is cleared from the session. + prepData := session.PrepareStatement["prep_user2"] + require.NotNil(t, prepData) + assert.Equal(t, "select * from user where id in (?,?,?)", prepData.PrepareStatement) + assert.EqualValues(t, 3, prepData.ParamsCount) + }) - // user defined variable does not exists on prepared query - _, err = executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user from @foo", nil) - require.Error(t, err) - require.Nil(t, session.PrepareStatement["prep_user"]) // prepared statement is cleared from the session. + t.Run("syntax error on prepared query", func(t *testing.T) { + _, err := executorExecSession(context.Background(), executor, session, "prepare prep_user2 from 'select'", nil) + require.Error(t, err) - // empty prepared query - _, err = executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user from ''", nil) - require.Error(t, err) + // prepared statement is cleared from the session. + require.Nil(t, session.PrepareStatement["prep_user2"]) + }) + + t.Run("user defined variable does not exists on prepared query", func(t *testing.T) { + _, err := executorExecSession(context.Background(), executor, session, "prepare prep_user from @foo", nil) + require.Error(t, err) + + // prepared statement is cleared from the session. + require.Nil(t, session.PrepareStatement["prep_user"]) + }) + + t.Run("empty prepared query", func(t *testing.T) { + _, err := executorExecSession(context.Background(), executor, session, "prepare prep_user from ''", nil) + require.Error(t, err) + }) } // TestExecutorSettingsInTwoPC tests that settings are supported for multi-shard atomic commit. @@ -2854,17 +2875,17 @@ func TestExecutorSettingsInTwoPC(t *testing.T) { }) // start transaction - _, err := executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, "begin", nil) + _, err := executorExecSession(ctx, executor, session, "begin", nil) require.NoError(t, err) // execute queries for _, sql := range tcase.sqls { - _, err = executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, sql, nil) + _, err = executorExecSession(ctx, executor, session, sql, nil) require.NoError(t, err) } // commit 2pc - _, err = executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, "commit", nil) + _, err = executorExecSession(ctx, executor, session, "commit", nil) require.NoError(t, err) queriesRecvd, err := sbc1.GetFinalQueries() @@ -2889,13 +2910,13 @@ func TestExecutorTruncateErrors(t *testing.T) { return nil } - _, err := executor.Execute(ctx, nil, "TestExecute", session, "invalid statement", nil) + _, err := executorExecSession(ctx, executor, session, "invalid statement", nil) assert.EqualError(t, err, "syntax error at posi [TRUNCATED]") err = executor.StreamExecute(ctx, nil, "TestExecute", session, "invalid statement", nil, fn) assert.EqualError(t, err, "syntax error at posi [TRUNCATED]") - _, err = executor.Prepare(context.Background(), "TestExecute", session, "invalid statement", nil) + _, _, err = executor.Prepare(context.Background(), "TestExecute", session, "invalid statement") assert.EqualError(t, err, "[BUG] unrecognized p [TRUNCATED]") } @@ -2944,7 +2965,7 @@ func TestExecutorFlushStmt(t *testing.T) { for _, tc := range tcs { t.Run(tc.query+tc.targetStr, func(t *testing.T) { - _, err := executor.Execute(context.Background(), nil, "TestExecutorFlushStmt", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), tc.query, nil) + _, err := executorExec(context.Background(), executor, &vtgatepb.Session{TargetString: tc.targetStr}, tc.query, nil) if tc.expectedErr == "" { require.NoError(t, err) } else { @@ -2991,7 +3012,7 @@ func TestExecutorKillStmt(t *testing.T) { allowKillStmt = !tc.disallow t.Run("execute:"+tc.query+tc.errStr, func(t *testing.T) { mysqlCtx := &fakeMysqlConnection{ErrMsg: tc.errStr} - _, err := executor.Execute(context.Background(), mysqlCtx, "TestExecutorKillStmt", econtext.NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil) + _, err := executor.Execute(context.Background(), mysqlCtx, "TestExecutorKillStmt", econtext.NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil, false) if tc.errStr != "" { require.ErrorContains(t, err, tc.errStr) } else { @@ -3038,7 +3059,7 @@ func (f *fakeMysqlConnection) KillConnection(ctx context.Context, connID uint32) var _ vtgateservice.MySQLConnection = (*fakeMysqlConnection)(nil) func exec(executor *Executor, session *econtext.SafeSession, sql string) (*sqltypes.Result, error) { - return executor.Execute(context.Background(), nil, "TestExecute", session, sql, nil) + return executorExecSession(context.Background(), executor, session, sql, nil) } func makeComments(text string) sqlparser.MarginComments { diff --git a/go/vt/vtgate/executor_vexplain_test.go b/go/vt/vtgate/executor_vexplain_test.go index a9516492f1b..0e929a76291 100644 --- a/go/vt/vtgate/executor_vexplain_test.go +++ b/go/vt/vtgate/executor_vexplain_test.go @@ -138,7 +138,7 @@ func TestVExplainKeys(t *testing.T) { t.Run(tt.Query, func(t *testing.T) { executor, _, _, _, _ := createExecutorEnv(t) session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"}) - gotResult, err := executor.Execute(context.Background(), nil, "Execute", session, "vexplain keys "+tt.Query, nil) + gotResult, err := executorExecSession(context.Background(), executor, session, "vexplain keys "+tt.Query, nil) require.NoError(t, err) gotRowString := gotResult.Rows[0][0].ToString() diff --git a/go/vt/vtgate/executor_vschema_ddl_test.go b/go/vt/vtgate/executor_vschema_ddl_test.go index 1acc1ba2362..b8eca5654c8 100644 --- a/go/vt/vtgate/executor_vschema_ddl_test.go +++ b/go/vt/vtgate/executor_vschema_ddl_test.go @@ -155,7 +155,7 @@ func TestPlanExecutorAlterVSchemaKeyspace(t *testing.T) { } stmt := "alter vschema create vindex TestExecutor.test_vindex using hash" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, "TestExecutor", "test_vindex", vschemaUpdates, executor) @@ -184,7 +184,7 @@ func TestPlanExecutorCreateVindexDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema create vindex test_vindex using hash" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, ks, "test_vindex", vschemaUpdates, executor) @@ -192,7 +192,7 @@ func TestPlanExecutorCreateVindexDDL(t *testing.T) { t.Errorf("updated vschema did not contain test_vindex") } - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr := "vindex test_vindex already exists in keyspace TestExecutor" if err == nil || err.Error() != wantErr { t.Errorf("create duplicate vindex: %v, want %s", err, wantErr) @@ -226,14 +226,14 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema drop vindex test_vindex" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) wantErr := "vindex test_vindex does not exists in keyspace TestExecutor" if err == nil || err.Error() != wantErr { t.Errorf("want error %v got %v", wantErr, err) } stmt = "alter vschema drop vindex TestExecutor.test_vindex" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr = "vindex test_vindex does not exists in keyspace TestExecutor" if err == nil || err.Error() != wantErr { t.Errorf("want error %v got %v", wantErr, err) @@ -241,7 +241,7 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) { // add one vindex that has never been used by the tables stmt = "alter vschema create vindex test_vindex using hash" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, ks, "test_vindex", vschemaUpdates, executor) @@ -251,7 +251,7 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) { // drop an existing vindex that has never been used by the tables stmt = "alter vschema drop vindex TestExecutor.test_vindex" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) vschema = <-vschemaUpdates _, ok = vschema.Keyspaces[ks].Vindexes["test_vindex"] @@ -261,7 +261,7 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) { // drop an existing vindex that is used by at least one table stmt = "alter vschema drop vindex TestExecutor.keyspace_id" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr = "can not drop vindex cause keyspace_id still defined on table ksid_table" if err == nil || err.Error() != wantErr { t.Errorf("drop vindex still defined: %v, want %s", err, wantErr) @@ -300,19 +300,19 @@ func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema add table test_table" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, append([]string{"test_table"}, vschemaTables...), executor) stmt = "alter vschema add table test_table2" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, append([]string{"test_table", "test_table2"}, vschemaTables...), executor) // Should fail adding a table on a sharded keyspace session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}) stmt = "alter vschema add table test_table" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr := "add vschema table: unsupported on sharded keyspace TestExecutor" if err == nil || err.Error() != wantErr { t.Errorf("want error %v got %v", wantErr, err) @@ -347,7 +347,7 @@ func TestExecutorAddSequenceDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema add sequence test_seq" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, append(vschemaTables, []string{"test_seq"}...), executor) vschema = executor.vm.GetCurrentSrvVschema() @@ -362,7 +362,7 @@ func TestExecutorAddSequenceDDL(t *testing.T) { session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ksSharded}) stmt = "alter vschema add sequence sequence_table" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr := "add sequence table: unsupported on sharded keyspace TestExecutor" if err == nil || err.Error() != wantErr { @@ -371,13 +371,13 @@ func TestExecutorAddSequenceDDL(t *testing.T) { // Should be able to add autoincrement to table in sharded keyspace stmt = "alter vschema on test_table add vindex hash_index (id)" - if _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil); err != nil { + if _, err = executorExecSession(ctx, executor, session, stmt, nil); err != nil { t.Error(err) } time.Sleep(10 * time.Millisecond) stmt = "alter vschema on test_table add auto_increment id using `db-name`.`test_seq`" - if _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil); err != nil { + if _, err = executorExecSession(ctx, executor, session, stmt, nil); err != nil { t.Error(err) } time.Sleep(10 * time.Millisecond) @@ -409,7 +409,7 @@ func TestExecutorDropSequenceDDL(t *testing.T) { // add test sequence stmt := "alter vschema add sequence test_seq" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, []string{"test_seq"}, executor) vschema = executor.vm.GetCurrentSrvVschema() @@ -422,7 +422,7 @@ func TestExecutorDropSequenceDDL(t *testing.T) { // drop existing test sequence stmt = "alter vschema drop sequence test_seq" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) if !waitForNewerVSchema(ctx, executor, ts, 5*time.Second) { @@ -433,7 +433,7 @@ func TestExecutorDropSequenceDDL(t *testing.T) { session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt = "alter vschema drop sequence test_seq" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) wantErr := "vschema does not contain sequence test_seq in keyspace TestUnsharded" if err == nil || err.Error() != wantErr { @@ -452,14 +452,14 @@ func TestExecutorDropAutoIncDDL(t *testing.T) { session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks}) stmt := "alter vschema add table test_table" - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _ = waitForVschemaTables(t, ks, []string{"test_table"}, executor) ts := executor.VSchema().GetCreated() stmt = "alter vschema on test_table add auto_increment id using `db-name`.`test_seq`" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) if !waitForNewerVSchema(ctx, executor, ts, 5*time.Second) { t.Fatalf("vschema did not update with auto_increment for 'test_table'") @@ -472,7 +472,7 @@ func TestExecutorDropAutoIncDDL(t *testing.T) { utils.MustMatch(t, wantAutoInc, gotAutoInc) stmt = "alter vschema on test_table drop auto_increment" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) if !waitForNewerVSchema(ctx, executor, ts, 5*time.Second) { @@ -503,14 +503,14 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // Create a new vindex implicitly with the statement stmt := "alter vschema on test add vindex test_hash (id) using hash " - _, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex := waitForVindex(t, ks, "test_hash", vschemaUpdates, executor) require.Equal(t, "hash", vindex.Type) _ = waitForColVindexes(t, ks, "test", []string{"test_hash"}, executor) - qr, err := executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + qr, err := executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.NoError(t, err) wantqr := &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -522,17 +522,17 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // Drop it stmt = "alter vschema on test drop vindex test_hash" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, _ = waitForVindex(t, ks, "test_hash", vschemaUpdates, executor) _ = waitForColVindexes(t, ks, "test", []string{}, executor) - _, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + _, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.EqualError(t, err, "VT05005: table 'test' does not exist in keyspace 'TestExecutor'") // add it again using the same syntax stmt = "alter vschema on test add vindex test_hash (id) using hash " - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex = waitForVindex(t, ks, "test_hash", vschemaUpdates, executor) @@ -540,7 +540,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { _ = waitForColVindexes(t, ks, "test", []string{"test_hash"}, executor) - qr, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + qr, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -553,7 +553,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // add another stmt = "alter vschema on test add vindex test_lookup (c1,c2) using lookup with owner=`test`, from=`c1,c2`, table=test_lookup, to=keyspace_id" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) vschema, vindex = waitForVindex(t, ks, "test_lookup", vschemaUpdates, executor) @@ -570,7 +570,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { t.Fatalf("table test not defined in vschema") } - qr, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + qr, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -582,7 +582,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { utils.MustMatch(t, wantqr, qr) stmt = "alter vschema on test add vindex test_hash_id2 (id2) using hash" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) vschema, vindex = waitForVindex(t, ks, "test_hash_id2", vschemaUpdates, executor) @@ -599,7 +599,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { t.Fatalf("table test not defined in vschema") } - qr, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + qr, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -613,13 +613,13 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // drop one stmt = "alter vschema on test drop vindex test_lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) // wait for up to 50ms for it to disappear deadline := time.Now().Add(50 * time.Millisecond) for { - qr, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test", nil) + qr, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test", nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -640,7 +640,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // use the newly created vindex on a new table stmt = "alter vschema on test2 add vindex test_hash (id)" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) vschema, vindex = waitForVindex(t, ks, "test_hash", vschemaUpdates, executor) @@ -653,7 +653,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // create an identical vindex definition on a different table stmt = "alter vschema on test2 add vindex test_lookup (c1,c2) using lookup with owner=`test`, from=`c1,c2`, table=test_lookup, to=keyspace_id" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) vschema, vindex = waitForVindex(t, ks, "test_lookup", vschemaUpdates, executor) @@ -664,7 +664,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { require.Len(t, table.ColumnVindexes, 2) require.Equal(t, "test_lookup", table.ColumnVindexes[1].Name) - qr, err = executor.Execute(ctx, nil, "TestExecute", session, "show vschema vindexes on TestExecutor.test2", nil) + qr, err = executorExecSession(ctx, executor, session, "show vschema vindexes on TestExecutor.test2", nil) require.NoError(t, err) wantqr = &sqltypes.Result{ Fields: buildVarCharFields("Columns", "Name", "Type", "Params", "Owner"), @@ -677,7 +677,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { // now make sure we can create another vindex that references a table with dashes (i.e. escaping is necessary) stmt = "alter vschema on test2 add vindex test_lookup_fqn(c1,c2) using consistent_lookup_unique with owner=`test`, from=`c1,c2`, table=`test-keyspace`.`lookup-fqn`, to=`keyspace_id`" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.NoError(t, err) _, vindex = waitForVindex(t, ks, "test_lookup_fqn", vschemaUpdates, executor) @@ -688,35 +688,35 @@ func TestExecutorAddDropVindexDDL(t *testing.T) { require.Equal(t, "keyspace_id", vindex.Params["to"]) stmt = "alter vschema on test2 add vindex nonexistent (c1,c2)" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "vindex nonexistent does not exist in keyspace TestExecutor") stmt = "alter vschema on test2 add vindex test_hash (c1,c2) using lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "vindex test_hash defined with type hash not lookup") stmt = "alter vschema on test2 add vindex test_lookup (c1,c2) using lookup with owner=xyz" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "vindex test_lookup defined with owner test not xyz") stmt = "alter vschema on test2 add vindex test_lookup (c1,c2) using lookup with owner=`test`, foo=bar" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "vindex test_lookup defined with different parameters") stmt = "alter vschema on nonexistent drop vindex test_lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "table TestExecutor.nonexistent not defined in vschema") stmt = "alter vschema on nonexistent drop vindex test_lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "InvalidKeyspace"}), stmt, nil) + _, err = executorExec(ctx, executor, &vtgatepb.Session{TargetString: "InvalidKeyspace"}, stmt, nil) require.EqualError(t, err, "VT05003: unknown database 'InvalidKeyspace' in vschema") stmt = "alter vschema on nowhere.nohow drop vindex test_lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "VT05003: unknown database 'nowhere' in vschema") stmt = "alter vschema on test drop vindex test_lookup" - _, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctx, executor, session, stmt, nil) require.EqualError(t, err, "vindex test_lookup not defined in table TestExecutor.test") // no queries should have gone to any tablets @@ -740,31 +740,31 @@ func TestPlanExecutorVindexDDLACL(t *testing.T) { // test that by default no users can perform the operation stmt := "alter vschema create vindex test_hash using hash" - _, err := executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) + _, err := executorExecSession(ctxRedUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`) - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'blueUser' is not authorized to perform vschema operations`) // test when all users are enabled vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%")) - _, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxRedUser, executor, session, stmt, nil) if err != nil { t.Errorf("unexpected error '%v'", err) } stmt = "alter vschema create vindex test_hash2 using hash" - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) if err != nil { t.Errorf("unexpected error '%v'", err) } // test when only one user is enabled vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("orangeUser, blueUser, greenUser")) - _, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxRedUser, executor, session, stmt, nil) require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`) stmt = "alter vschema create vindex test_hash3 using hash" - _, err = executor.Execute(ctxBlueUser, nil, "TestExecute", session, stmt, nil) + _, err = executorExecSession(ctxBlueUser, executor, session, stmt, nil) if err != nil { t.Errorf("unexpected error '%v'", err) } diff --git a/go/vt/vtgate/executorcontext/vcursor_impl.go b/go/vt/vtgate/executorcontext/vcursor_impl.go index 19b8f3cbe3a..a4c800e5053 100644 --- a/go/vt/vtgate/executorcontext/vcursor_impl.go +++ b/go/vt/vtgate/executorcontext/vcursor_impl.go @@ -19,7 +19,6 @@ package executorcontext import ( "context" "fmt" - "io" "sort" "strings" "sync" @@ -29,6 +28,8 @@ import ( "github.com/google/uuid" "golang.org/x/exp/maps" + "vitess.io/vitess/go/vt/sysvars" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/config" "vitess.io/vitess/go/mysql/sqlerror" @@ -95,7 +96,7 @@ type ( // vcursor_impl needs these facilities to be able to be able to execute queries for vindexes iExecute interface { - Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable) (*sqltypes.Result, error) + Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable, prepared bool) (*sqltypes.Result, error) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *SafeSession, autocommit bool, ignoreMaxMemoryRows bool, resultsObserver ResultsObserver, fetchLastInsertID bool) (qr *sqltypes.Result, errs []error) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error, observer ResultsObserver, fetchLastInsertID bool) []error ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) @@ -112,7 +113,7 @@ type ( // TODO: remove when resolver is gone VSchema() *vindexes.VSchema - PlanPrepareStmt(ctx context.Context, vcursor *VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) + PlanPrepareStmt(ctx context.Context, safeSession *SafeSession, query string) (*engine.Plan, error) Environment() *vtenv.Environment ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error) @@ -126,6 +127,24 @@ type ( UpdateVSchema(ctx context.Context, ks *topo.KeyspaceVSchemaInfo, vschema *vschemapb.SrvVSchema) error } + Resolver interface { + GetGateway() srvtopo.Gateway + ResolveDestinations( + ctx context.Context, + keyspace string, + tabletType topodatapb.TabletType, + ids []*querypb.Value, + destinations []key.ShardDestination, + ) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) + ResolveDestinationsMultiCol( + ctx context.Context, + keyspace string, + tabletType topodatapb.TabletType, + ids [][]sqltypes.Value, + destinations []key.ShardDestination, + ) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) + } + // VCursorImpl implements the VCursor functionality used by dependent // packages to call back into VTGate. VCursorImpl struct { @@ -133,10 +152,10 @@ type ( SafeSession *SafeSession keyspace string tabletType topodatapb.TabletType - destination key.Destination + destination key.ShardDestination marginComments sqlparser.MarginComments executor iExecute - resolver *srvtopo.Resolver + resolver Resolver topoServer *topo.Server logStats *logstats.LogStats @@ -175,7 +194,7 @@ func NewVCursorImpl( logStats *logstats.LogStats, vm VSchemaOperator, vschema *vindexes.VSchema, - resolver *srvtopo.Resolver, + resolver Resolver, serv srvtopo.Server, observer ResultsObserver, cfg VCursorConfig, @@ -213,6 +232,26 @@ func NewVCursorImpl( }, nil } +func (vc *VCursorImpl) GetSafeSession() *SafeSession { + return vc.SafeSession +} + +func (vc *VCursorImpl) PrepareSetVarComment() string { + var res []string + vc.Session().GetSystemVariables(func(k, v string) { + if sysvars.SupportsSetVar(k) { + if k == "sql_mode" && v == "''" { + // SET_VAR(sql_mode, '') is not accepted by MySQL, giving a warning: + // | Warning | 1064 | Optimizer hint syntax error near ''') */ + v = "' '" + } + res = append(res, fmt.Sprintf("SET_VAR(%s = %s)", k, v)) + } + }) + + return strings.Join(res, " ") +} + func (vc *VCursorImpl) CloneForMirroring(ctx context.Context) engine.VCursor { callerId := callerid.EffectiveCallerIDFromContext(ctx) immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) @@ -367,7 +406,7 @@ func (vc *VCursorImpl) UnresolvedTransactions(ctx context.Context, keyspace stri if keyspace == "" { keyspace = vc.GetKeyspace() } - rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.Destination{key.DestinationAllShards{}}) + rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.ShardDestination{key.DestinationAllShards{}}) if err != nil { return nil, err } @@ -391,7 +430,7 @@ func (vc *VCursorImpl) StartPrimitiveTrace() func() engine.Stats { // FindTable finds the specified table. If the keyspace what specified in the input, it gets used as qualifier. // Otherwise, the keyspace from the request is used, if one was provided. -func (vc *VCursorImpl) FindTable(name sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.Destination, error) { +func (vc *VCursorImpl) FindTable(name sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.ShardDestination, error) { destKeyspace, destTabletType, dest, err := vc.parseDestinationTarget(name.Qualifier.String()) if err != nil { return nil, "", destTabletType, nil, err @@ -435,7 +474,7 @@ func (vc *VCursorImpl) FindRoutedTable(name sqlparser.TableName) (*vindexes.Base } // FindTableOrVindex finds the specified table or vindex. -func (vc *VCursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (vc *VCursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { if name.Qualifier.IsEmpty() && name.Name.String() == "dual" { // The magical MySQL dual table should only be resolved // when it is not qualified by a database name. @@ -473,12 +512,12 @@ func (vc *VCursorImpl) FindViewTarget(name sqlparser.TableName) (*vindexes.Keysp return tbl.Keyspace, nil } -func (vc *VCursorImpl) parseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) { +func (vc *VCursorImpl) parseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.ShardDestination, error) { return ParseDestinationTarget(targetString, vc.tabletType, vc.vschema) } // ParseDestinationTarget parses destination target string and provides a keyspace if possible. -func ParseDestinationTarget(targetString string, tablet topodatapb.TabletType, vschema *vindexes.VSchema) (string, topodatapb.TabletType, key.Destination, error) { +func ParseDestinationTarget(targetString string, tablet topodatapb.TabletType, vschema *vindexes.VSchema) (string, topodatapb.TabletType, key.ShardDestination, error) { destKeyspace, destTabletType, dest, err := topoprotopb.ParseDestination(targetString, tablet) // If the keyspace is not specified, and there is only one keyspace in the VSchema, use that. if destKeyspace == "" && len(vschema.Keyspaces) == 1 { @@ -489,7 +528,7 @@ func ParseDestinationTarget(targetString string, tablet topodatapb.TabletType, v return destKeyspace, destTabletType, dest, err } -func (vc *VCursorImpl) getDualTable() (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (vc *VCursorImpl) getDualTable() (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { ksName := vc.getActualKeyspace() var ks *vindexes.Keyspace if ksName == "" { @@ -765,7 +804,7 @@ func (vc *VCursorImpl) Execute(ctx context.Context, method string, query string, return nil, err } - qr, err := vc.executor.Execute(ctx, nil, method, session, vc.marginComments.Leading+query+vc.marginComments.Trailing, bindVars) + qr, err := vc.executor.Execute(ctx, nil, method, session, vc.marginComments.Leading+query+vc.marginComments.Trailing, bindVars, false) vc.setRollbackOnPartialExecIfRequired(err != nil, rollbackOnError) return qr, err @@ -780,7 +819,7 @@ func (vc *VCursorImpl) markSavepoint(ctx context.Context, needsRollbackOnParialE } uID := fmt.Sprintf("_vt%s", strings.ReplaceAll(uuid.NewString(), "-", "_")) spQuery := fmt.Sprintf("%ssavepoint %s%s", vc.marginComments.Leading, uID, vc.marginComments.Trailing) - _, err := vc.executor.Execute(ctx, nil, "MarkSavepoint", vc.SafeSession, spQuery, bindVars) + _, err := vc.executor.Execute(ctx, nil, "MarkSavepoint", vc.SafeSession, spQuery, bindVars, false) if err != nil { return err } @@ -851,7 +890,7 @@ func (vc *VCursorImpl) ExecuteStandalone(ctx context.Context, primitive engine.P // ExecuteKeyspaceID is part of the engine.VCursor interface. func (vc *VCursorImpl) ExecuteKeyspaceID(ctx context.Context, keyspace string, ksid []byte, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError, autocommit bool) (*sqltypes.Result, error) { atomic.AddUint64(&vc.logStats.ShardQueries, 1) - rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.Destination{key.DestinationKeyspaceID(ksid)}) + rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.ShardDestination{key.DestinationKeyspaceID(ksid)}) if err != nil { return nil, err } @@ -928,7 +967,7 @@ func (vc *VCursorImpl) fixupPartiallyMovedShards(rss []*srvtopo.ResolvedShard) ( return rss, nil } -func (vc *VCursorImpl) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { +func (vc *VCursorImpl) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) { rss, values, err := vc.resolver.ResolveDestinations(ctx, keyspace, vc.tabletType, ids, destinations) if err != nil { return nil, nil, err @@ -942,7 +981,7 @@ func (vc *VCursorImpl) ResolveDestinations(ctx context.Context, keyspace string, return rss, values, err } -func (vc *VCursorImpl) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { +func (vc *VCursorImpl) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.ShardDestination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) { rss, values, err := vc.resolver.ResolveDestinationsMultiCol(ctx, keyspace, vc.tabletType, ids, destinations) if err != nil { return nil, nil, err @@ -995,6 +1034,21 @@ func (vc *VCursorImpl) SetSysVar(name string, expr string) { vc.SafeSession.SetSystemVariable(name, expr) } +func (vc *VCursorImpl) CheckForReservedConnection(setVarComment string, stmt sqlparser.Statement) { + if setVarComment == "" { + return + } + switch stmt.(type) { + // If the statement supports optimizer hints or a transaction statement or a SET statement + // no reserved connection is needed + case *sqlparser.Begin, *sqlparser.Commit, *sqlparser.Rollback, *sqlparser.Savepoint, + *sqlparser.SRollback, *sqlparser.Release, *sqlparser.Set, *sqlparser.Show, + sqlparser.SupportOptimizerHint: + default: + vc.NeedsReservedConn() + } +} + // NeedsReservedConn implements the SessionActions interface func (vc *VCursorImpl) NeedsReservedConn() { vc.SafeSession.SetReservedConn(true) @@ -1020,7 +1074,7 @@ func (vc *VCursorImpl) ShardSession() []*srvtopo.ResolvedShard { } // Destination implements the ContextVSchema interface -func (vc *VCursorImpl) Destination() key.Destination { +func (vc *VCursorImpl) ShardDestination() key.ShardDestination { return vc.destination } @@ -1044,7 +1098,7 @@ func commentedShardQueries(shardQueries []*querypb.BoundQuery, marginComments sq } // TargetDestination implements the ContextVSchema interface -func (vc *VCursorImpl) TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) { +func (vc *VCursorImpl) TargetDestination(qualifier string) (key.ShardDestination, *vindexes.Keyspace, topodatapb.TabletType, error) { keyspaceName := vc.getActualKeyspace() if vc.destination == nil && qualifier != "" { keyspaceName = qualifier @@ -1350,40 +1404,6 @@ func (vc *VCursorImpl) FindMirrorRule(name sqlparser.TableName) (*vindexes.Mirro return mirrorRule, err } -func (vc *VCursorImpl) KeyForPlan(ctx context.Context, query string, buf io.StringWriter) { - _, _ = buf.WriteString(vc.keyspace) - _, _ = buf.WriteString(vindexes.TabletTypeSuffix[vc.tabletType]) - _, _ = buf.WriteString("+Collate:") - _, _ = buf.WriteString(vc.Environment().CollationEnv().LookupName(vc.config.Collation)) - - if vc.destination != nil { - switch vc.destination.(type) { - case key.DestinationKeyspaceID, key.DestinationKeyspaceIDs: - resolved, _, err := vc.ResolveDestinations(ctx, vc.keyspace, nil, []key.Destination{vc.destination}) - if err == nil && len(resolved) > 0 { - shards := make([]string, len(resolved)) - for i := 0; i < len(shards); i++ { - shards[i] = resolved[i].Target.GetShard() - } - sort.Strings(shards) - - _, _ = buf.WriteString("+KsIDsResolved:") - for i, s := range shards { - if i > 0 { - _, _ = buf.WriteString(",") - } - _, _ = buf.WriteString(s) - } - } - default: - _, _ = buf.WriteString("+") - _, _ = buf.WriteString(vc.destination.String()) - } - } - _, _ = buf.WriteString("+Query:") - _, _ = buf.WriteString(query) -} - func (vc *VCursorImpl) GetKeyspace() string { return vc.keyspace } @@ -1545,8 +1565,8 @@ func (vc *VCursorImpl) GetUDV(name string) *querypb.BindVariable { return vc.SafeSession.GetUDV(name) } -func (vc *VCursorImpl) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) { - return vc.executor.PlanPrepareStmt(ctx, vc, query) +func (vc *VCursorImpl) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, error) { + return vc.executor.PlanPrepareStmt(ctx, vc.SafeSession, query) } func (vc *VCursorImpl) ClearPrepareData(name string) { @@ -1569,18 +1589,9 @@ func (vc *VCursorImpl) GetWarmingReadsChannel() chan bool { return vc.config.WarmingReadsChannel } -// UpdateForeignKeyChecksState updates the foreign key checks state of the vcursor. -func (vc *VCursorImpl) UpdateForeignKeyChecksState(fkStateFromQuery *bool) { - // Initialize the state to unspecified. - vc.fkChecksState = nil - // If the query has a SET_VAR optimizer hint that explicitly sets the foreign key checks state, - // we should use that. - if fkStateFromQuery != nil { - vc.fkChecksState = fkStateFromQuery - return - } - // If the query doesn't have anything, then we consult the session state. - vc.fkChecksState = vc.SafeSession.ForeignKeyChecks() +// SetForeignKeyCheckState updates the foreign key checks state of the vcursor. +func (vc *VCursorImpl) SetForeignKeyCheckState(fkChecksState *bool) { + vc.fkChecksState = fkChecksState } // GetForeignKeyChecksState gets the stored foreign key checks state in the vcursor. diff --git a/go/vt/vtgate/executorcontext/vcursor_impl_test.go b/go/vt/vtgate/executorcontext/vcursor_impl_test.go index 54173bf63b0..cb6a9b69986 100644 --- a/go/vt/vtgate/executorcontext/vcursor_impl_test.go +++ b/go/vt/vtgate/executorcontext/vcursor_impl_test.go @@ -21,7 +21,6 @@ import ( "errors" "fmt" "strconv" - "strings" "testing" "time" @@ -29,7 +28,6 @@ import ( "vitess.io/vitess/go/streamlog" - "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/sqltypes" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" "vitess.io/vitess/go/vt/proto/vschema" @@ -103,7 +101,7 @@ func TestDestinationKeyspace(t *testing.T) { targetString, qualifier string expectedError string expectedKeyspace string - expectedDest key.Destination + expectedDest key.ShardDestination expectedTabletType topodatapb.TabletType } @@ -193,15 +191,10 @@ func TestDestinationKeyspace(t *testing.T) { } var ( - ks1 = &vindexes.Keyspace{Name: "ks1"} - ks1Schema = &vindexes.KeyspaceSchema{Keyspace: ks1} - ks2 = &vindexes.Keyspace{Name: "ks2"} - ks2Schema = &vindexes.KeyspaceSchema{Keyspace: ks2} - vschemaWith1KS = &vindexes.VSchema{ - Keyspaces: map[string]*vindexes.KeyspaceSchema{ - ks1.Name: ks1Schema, - }, - } + ks1 = &vindexes.Keyspace{Name: "ks1"} + ks1Schema = &vindexes.KeyspaceSchema{Keyspace: ks1} + ks2 = &vindexes.Keyspace{Name: "ks2"} + ks2Schema = &vindexes.KeyspaceSchema{Keyspace: ks2} ) var vschemaWith2KS = &vindexes.VSchema{ @@ -253,58 +246,6 @@ func TestSetTarget(t *testing.T) { } } -func TestKeyForPlan(t *testing.T) { - type testCase struct { - vschema *vindexes.VSchema - targetString string - expectedPlanPrefixKey string - } - - tests := []testCase{{ - vschema: vschemaWith1KS, - targetString: "", - expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1", - }, { - vschema: vschemaWith1KS, - targetString: "ks1@replica", - expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1", - }, { - vschema: vschemaWith1KS, - targetString: "ks1:-80", - expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+DestinationShard(-80)+Query:SELECT 1", - }, { - vschema: vschemaWith1KS, - targetString: "ks1[deadbeef]", - expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+KsIDsResolved:80-+Query:SELECT 1", - }, { - vschema: vschemaWith1KS, - targetString: "", - expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1", - }, { - vschema: vschemaWith1KS, - targetString: "ks1@replica", - expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1", - }} - - for i, tc := range tests { - t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) { - ss := NewSafeSession(&vtgatepb.Session{InTransaction: false}) - ss.SetTargetString(tc.targetString) - cfg := VCursorConfig{ - Collation: collations.CollationUtf8mb4ID, - DefaultTabletType: topodatapb.TabletType_PRIMARY, - } - vc, err := NewVCursorImpl(ss, sqlparser.MarginComments{}, &fakeExecutor{}, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, srvtopo.NewResolver(&FakeTopoServer{}, nil, ""), nil, fakeObserver{}, cfg) - require.NoError(t, err) - vc.vschema = tc.vschema - - var buf strings.Builder - vc.KeyForPlan(context.Background(), "SELECT 1", &buf) - require.Equal(t, tc.expectedPlanPrefixKey, buf.String()) - }) - } -} - func TestFirstSortedKeyspace(t *testing.T) { ks1Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "xks1"}} ks2Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "aks2"}} @@ -388,7 +329,7 @@ func TestRecordMirrorStats(t *testing.T) { type fakeExecutor struct{} -func (f fakeExecutor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { +func (f fakeExecutor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable, prepared bool) (*sqltypes.Result, error) { // TODO implement me panic("implement me") } @@ -453,7 +394,7 @@ func (f fakeExecutor) SetVitessMetadata(ctx context.Context, name, value string) panic("implement me") } -func (f fakeExecutor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) { +func (f fakeExecutor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.ShardDestination, error) { // TODO implement me panic("implement me") } @@ -463,7 +404,7 @@ func (f fakeExecutor) VSchema() *vindexes.VSchema { panic("implement me") } -func (f fakeExecutor) PlanPrepareStmt(ctx context.Context, vcursor *VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) { +func (f fakeExecutor) PlanPrepareStmt(context.Context, *SafeSession, string) (*engine.Plan, error) { // TODO implement me panic("implement me") } diff --git a/go/vt/vtgate/fakerpcvtgateconn/conn.go b/go/vt/vtgate/fakerpcvtgateconn/conn.go index 372ddfb8cfc..894ac5e2193 100644 --- a/go/vt/vtgate/fakerpcvtgateconn/conn.go +++ b/go/vt/vtgate/fakerpcvtgateconn/conn.go @@ -43,9 +43,10 @@ type queryExecute struct { } type queryResponse struct { - execQuery *queryExecute - reply *sqltypes.Result - err error + execQuery *queryExecute + reply *sqltypes.Result + paramsCount uint16 + err error } // FakeVTGateConn provides a fake implementation of vtgateconn.Impl @@ -84,7 +85,13 @@ func (conn *FakeVTGateConn) AddQuery( } // Execute please see vtgateconn.Impl.Execute -func (conn *FakeVTGateConn) Execute(ctx context.Context, session *vtgatepb.Session, sql string, bindVars map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (conn *FakeVTGateConn) Execute( + ctx context.Context, + session *vtgatepb.Session, + sql string, + bindVars map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { response, ok := conn.execMap[sql] if !ok { return nil, nil, fmt.Errorf("no match for: %s", sql) @@ -158,23 +165,22 @@ func (a *streamExecuteAdapter) Recv() (*sqltypes.Result, error) { } // Prepare please see vtgateconn.Impl.Prepare -func (conn *FakeVTGateConn) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVars map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { +func (conn *FakeVTGateConn) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { response, ok := conn.execMap[sql] if !ok { - return nil, nil, fmt.Errorf("no match for: %s", sql) + return nil, nil, 0, fmt.Errorf("no match for: %s", sql) } query := &queryExecute{ - SQL: sql, - BindVariables: bindVars, - Session: session, + SQL: sql, + Session: session, } if !reflect.DeepEqual(query, response.execQuery) { - return nil, nil, fmt.Errorf( + return nil, nil, 0, fmt.Errorf( "Prepare: %+v, want %+v", query, response.execQuery) } reply := *response.reply s := newSession(true, "test_keyspace", []string{}, topodatapb.TabletType_PRIMARY) - return s, reply.Fields, nil + return s, reply.Fields, response.paramsCount, nil } // CloseSession please see vtgateconn.Impl.CloseSession diff --git a/go/vt/vtgate/grpcvtgateconn/conn.go b/go/vt/vtgate/grpcvtgateconn/conn.go index fa16fa0d602..f37f61fd9cc 100644 --- a/go/vt/vtgate/grpcvtgateconn/conn.go +++ b/go/vt/vtgate/grpcvtgateconn/conn.go @@ -107,7 +107,13 @@ func DialWithOpts(_ context.Context, opts ...grpc.DialOption) vtgateconn.DialerF return Dial(opts...) } -func (conn *vtgateConn) Execute(ctx context.Context, session *vtgatepb.Session, query string, bindVars map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (conn *vtgateConn) Execute( + ctx context.Context, + session *vtgatepb.Session, + query string, + bindVars map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { request := &vtgatepb.ExecuteRequest{ CallerId: callerid.EffectiveCallerIDFromContext(ctx), Session: session, @@ -115,6 +121,7 @@ func (conn *vtgateConn) Execute(ctx context.Context, session *vtgatepb.Session, Sql: query, BindVariables: bindVars, }, + Prepared: prepared, } response, err := conn.c.Execute(ctx, request) if err != nil { @@ -200,23 +207,22 @@ func (conn *vtgateConn) StreamExecute(ctx context.Context, session *vtgatepb.Ses }, nil } -func (conn *vtgateConn) Prepare(ctx context.Context, session *vtgatepb.Session, query string, bindVars map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { +func (conn *vtgateConn) Prepare(ctx context.Context, session *vtgatepb.Session, query string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { request := &vtgatepb.PrepareRequest{ CallerId: callerid.EffectiveCallerIDFromContext(ctx), Session: session, Query: &querypb.BoundQuery{ - Sql: query, - BindVariables: bindVars, + Sql: query, }, } response, err := conn.c.Prepare(ctx, request) if err != nil { - return session, nil, vterrors.FromGRPC(err) + return session, nil, 0, vterrors.FromGRPC(err) } if response.Error != nil { - return response.Session, nil, vterrors.FromVTRPC(response.Error) + return response.Session, nil, 0, vterrors.FromVTRPC(response.Error) } - return response.Session, response.Fields, nil + return response.Session, response.Fields, uint16(response.ParamsCount), nil } func (conn *vtgateConn) CloseSession(ctx context.Context, session *vtgatepb.Session) error { diff --git a/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go b/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go index 55a067807bd..4cd52d0f5a0 100644 --- a/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go +++ b/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go @@ -40,9 +40,7 @@ func TestGRPCVTGateConn(t *testing.T) { // listen on a random port listener, err := net.Listen("tcp", "127.0.0.1:0") - if err != nil { - t.Fatalf("Cannot listen: %v", err) - } + require.NoError(t, err) // Create a gRPC server and listen on the port server := grpc.NewServer() @@ -52,9 +50,7 @@ func TestGRPCVTGateConn(t *testing.T) { // Create a Go RPC client connecting to the server ctx := context.Background() client, err := dial(ctx, listener.Addr().String()) - if err != nil { - t.Fatalf("dial failed: %v", err) - } + require.NoError(t, err) RegisterTestDialProtocol(client) // run the test suite @@ -92,16 +88,12 @@ func TestGRPCVTGateConnAuth(t *testing.T) { }` f, err := os.CreateTemp("", "static_auth_creds.json") - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) defer os.Remove(f.Name()) - if _, err := io.WriteString(f, authJSON); err != nil { - t.Fatal(err) - } - if err := f.Close(); err != nil { - t.Fatal(err) - } + _, err = io.WriteString(f, authJSON) + require.NoError(t, err) + err = f.Close() + require.NoError(t, err) // Create a Go RPC client connecting to the server ctx := context.Background() @@ -115,9 +107,7 @@ func TestGRPCVTGateConnAuth(t *testing.T) { }) require.NoError(t, err, "failed to set `--grpc_auth_static_client_creds=%s`", f.Name()) client, err := dial(ctx, listener.Addr().String()) - if err != nil { - t.Fatalf("dial failed: %v", err) - } + require.NoError(t, err) RegisterTestDialProtocol(client) // run the test suite @@ -133,16 +123,11 @@ func TestGRPCVTGateConnAuth(t *testing.T) { }` f, err = os.CreateTemp("", "static_auth_creds.json") - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) defer os.Remove(f.Name()) - if _, err := io.WriteString(f, invalidAuthJSON); err != nil { - t.Fatal(err) - } - if err := f.Close(); err != nil { - t.Fatal(err) - } + _, err = io.WriteString(f, invalidAuthJSON) + require.NoError(t, err) + require.NoError(t, f.Close()) // Create a Go RPC client connecting to the server ctx = context.Background() @@ -162,7 +147,7 @@ func TestGRPCVTGateConnAuth(t *testing.T) { RegisterTestDialProtocol(client) conn, _ := vtgateconn.DialProtocol(context.Background(), "test", "") // run the test suite - _, err = conn.Session("", nil).Execute(context.Background(), "select * from t", nil) + _, err = conn.Session("", nil).Execute(context.Background(), "select * from t", nil, false) want := "rpc error: code = Unauthenticated desc = username and password must be provided" if err == nil || err.Error() != want { t.Errorf("expected auth failure:\n%v, want\n%s", err, want) diff --git a/go/vt/vtgate/grpcvtgateconn/suite_test.go b/go/vt/vtgate/grpcvtgateconn/suite_test.go index 32df3a2ab34..f1a6d6ba7e3 100644 --- a/go/vt/vtgate/grpcvtgateconn/suite_test.go +++ b/go/vt/vtgate/grpcvtgateconn/suite_test.go @@ -91,7 +91,14 @@ func (q *queryExecute) equal(q2 *queryExecute) bool { } // Execute is part of the VTGateService interface -func (f *fakeVTGateService) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) { +func (f *fakeVTGateService) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (*vtgatepb.Session, *sqltypes.Result, error) { if f.hasError { return session, nil, errTestVtGateError } @@ -197,9 +204,9 @@ func (f *fakeVTGateService) StreamExecute(ctx context.Context, mysqlCtx vtgatese } // Prepare is part of the VTGateService interface -func (f *fakeVTGateService) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) { +func (f *fakeVTGateService) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) { if f.hasError { - return session, nil, errTestVtGateError + return session, nil, 0, errTestVtGateError } if f.panics { panic(fmt.Errorf("test forced panic")) @@ -207,22 +214,21 @@ func (f *fakeVTGateService) Prepare(ctx context.Context, session *vtgatepb.Sessi f.checkCallerID(ctx, "Prepare") execCase, ok := execMap[sql] if !ok { - return session, nil, fmt.Errorf("no match for: %s", sql) + return session, nil, 0, fmt.Errorf("no match for: %s", sql) } query := &queryExecute{ - SQL: sql, - BindVariables: bindVariables, - Session: session, + SQL: sql, + Session: session, } if !query.equal(execCase.execQuery) { f.t.Errorf("Prepare:\n%+v, want\n%+v", query, execCase.execQuery) - return session, nil, nil + return session, nil, 0, nil } if execCase.outSession != nil { proto.Reset(session) proto.Merge(session, execCase.outSession) } - return session, execCase.result.Fields, nil + return session, execCase.result.Fields, execCase.paramsCount, nil } // CloseSession is part of the VTGateService interface @@ -341,13 +347,13 @@ func verifyErrorString(t *testing.T, err error, method string) { func testExecute(t *testing.T, session *vtgateconn.VTGateSession) { ctx := newContext() execCase := execMap["request1"] - qr, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + qr, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables, false) require.NoError(t, err) if !qr.Equal(execCase.result) { t.Errorf("Unexpected result from Execute: got\n%#v want\n%#v", qr, execCase.result) } - _, err = session.Execute(ctx, "none", nil) + _, err = session.Execute(ctx, "none", nil, false) want := "no match for: none" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("none request: %v, want %v", err, want) @@ -358,14 +364,14 @@ func testExecuteError(t *testing.T, session *vtgateconn.VTGateSession, fake *fak ctx := newContext() execCase := execMap["errorRequst"] - _, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + _, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables, false) verifyError(t, err, "Execute") } func testExecutePanic(t *testing.T, session *vtgateconn.VTGateSession) { ctx := newContext() execCase := execMap["request1"] - _, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + _, err := session.Execute(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables, false) expectPanic(t, err) } @@ -484,13 +490,12 @@ func testStreamExecutePanic(t *testing.T, session *vtgateconn.VTGateSession) { func testPrepare(t *testing.T, session *vtgateconn.VTGateSession) { ctx := newContext() execCase := execMap["request1"] - _, err := session.Prepare(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + fields, paramsCount, err := session.Prepare(ctx, execCase.execQuery.SQL) require.NoError(t, err) - // if !qr.Equal(execCase.result) { - // t.Errorf("Unexpected result from Execute: got\n%#v want\n%#v", qr, execCase.result) - // } + require.True(t, sqltypes.FieldsEqual(fields, execCase.result.Fields)) + require.Equal(t, execCase.paramsCount, paramsCount) - _, err = session.Prepare(ctx, "none", nil) + _, _, err = session.Prepare(ctx, "none") require.EqualError(t, err, "no match for: none") } @@ -498,14 +503,14 @@ func testPrepareError(t *testing.T, session *vtgateconn.VTGateSession, fake *fak ctx := newContext() execCase := execMap["errorRequst"] - _, err := session.Prepare(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + _, _, err := session.Prepare(ctx, execCase.execQuery.SQL) verifyError(t, err, "Prepare") } func testPreparePanic(t *testing.T, session *vtgateconn.VTGateSession) { ctx := newContext() execCase := execMap["request1"] - _, err := session.Prepare(ctx, execCase.execQuery.SQL, execCase.execQuery.BindVariables) + _, _, err := session.Prepare(ctx, execCase.execQuery.SQL) expectPanic(t, err) } @@ -520,17 +525,15 @@ var testExecuteOptions = &querypb.ExecuteOptions{ } var execMap = map[string]struct { - execQuery *queryExecute - result *sqltypes.Result - outSession *vtgatepb.Session - err error + execQuery *queryExecute + paramsCount uint16 + result *sqltypes.Result + outSession *vtgatepb.Session + err error }{ "request1": { execQuery: &queryExecute{ SQL: "request1", - BindVariables: map[string]*querypb.BindVariable{ - "bind1": sqltypes.Int64BindVariable(0), - }, Session: &vtgatepb.Session{ TargetString: "connection_ks@rdonly", Options: testExecuteOptions, @@ -584,5 +587,3 @@ var streamResultFields = sqltypes.Result{ Fields: result1.Fields, Rows: [][]sqltypes.Value{}, } - -var dtid2 = "aa" diff --git a/go/vt/vtgate/grpcvtgateservice/server.go b/go/vt/vtgate/grpcvtgateservice/server.go index d9fef3d4e31..d4b759ed75d 100644 --- a/go/vt/vtgate/grpcvtgateservice/server.go +++ b/go/vt/vtgate/grpcvtgateservice/server.go @@ -145,7 +145,7 @@ func (vtg *VTGate) Execute(ctx context.Context, request *vtgatepb.ExecuteRequest if session == nil { session = &vtgatepb.Session{Autocommit: true} } - session, result, err := vtg.server.Execute(ctx, nil, session, request.Query.Sql, request.Query.BindVariables) + session, result, err := vtg.server.Execute(ctx, nil, session, request.Query.Sql, request.Query.BindVariables, request.Prepared) return &vtgatepb.ExecuteResponse{ Result: sqltypes.ResultToProto3(result), Session: session, @@ -224,11 +224,12 @@ func (vtg *VTGate) Prepare(ctx context.Context, request *vtgatepb.PrepareRequest session = &vtgatepb.Session{Autocommit: true} } - session, fields, err := vtg.server.Prepare(ctx, session, request.Query.Sql, request.Query.BindVariables) + session, fields, paramsCount, err := vtg.server.Prepare(ctx, session, request.Query.Sql) return &vtgatepb.PrepareResponse{ - Fields: fields, - Session: session, - Error: vterrors.ToVTRPC(err), + Session: session, + Fields: fields, + ParamsCount: uint32(paramsCount), + Error: vterrors.ToVTRPC(err), }, nil } diff --git a/go/vt/vtgate/legacy_scatter_conn_test.go b/go/vt/vtgate/legacy_scatter_conn_test.go index fecd6c2a8b1..ec345e4308e 100644 --- a/go/vt/vtgate/legacy_scatter_conn_test.go +++ b/go/vt/vtgate/legacy_scatter_conn_test.go @@ -279,7 +279,7 @@ func TestMaxMemoryRows(t *testing.T) { res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa") rss, _, err := res.ResolveDestinations(ctx, "TestMaxMemoryRows", topodatapb.TabletType_REPLICA, nil, - []key.Destination{key.DestinationShard("0"), key.DestinationShard("1")}) + []key.ShardDestination{key.DestinationShard("0"), key.DestinationShard("1")}) require.NoError(t, err) session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}) @@ -331,7 +331,7 @@ func TestLegaceHealthCheckFailsOnReservedConnections(t *testing.T) { res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa") session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true}) - destinations := []key.Destination{key.DestinationShard("0")} + destinations := []key.ShardDestination{key.DestinationShard("0")} rss, _, err := res.ResolveDestinations(ctx, keyspace, topodatapb.TabletType_REPLICA, nil, destinations) require.NoError(t, err) @@ -348,12 +348,12 @@ func TestLegaceHealthCheckFailsOnReservedConnections(t *testing.T) { require.Error(t, vterrors.Aggregate(errs)) } -func executeOnShards(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.Destination) { +func executeOnShards(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.ShardDestination) { t.Helper() require.Empty(t, executeOnShardsReturnsErr(t, ctx, res, keyspace, sc, session, destinations)) } -func executeOnShardsReturnsErr(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.Destination) error { +func executeOnShardsReturnsErr(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.ShardDestination) error { t.Helper() rss, _, err := res.ResolveDestinations(ctx, keyspace, topodatapb.TabletType_REPLICA, nil, destinations) require.NoError(t, err) @@ -611,7 +611,7 @@ func TestReservePrequeries(t *testing.T) { "s2": "42", }, }) - destinations := []key.Destination{key.DestinationShard("0")} + destinations := []key.ShardDestination{key.DestinationShard("0")} executeOnShards(t, ctx, res, keyspace, sc, session, destinations) assert.Equal(t, 1+1, len(sbc0.StringQueries())) diff --git a/go/vt/vtgate/plan_execute.go b/go/vt/vtgate/plan_execute.go index 50f4ab30bb7..7fb0bec33e2 100644 --- a/go/vt/vtgate/plan_execute.go +++ b/go/vt/vtgate/plan_execute.go @@ -65,12 +65,11 @@ func (e *Executor) newExecute( safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, + prepared bool, logStats *logstats.LogStats, execPlan planExec, // used when there is a plan to execute recResult txResult, // used when it's something simple like begin/commit/rollback/savepoint ) (err error) { - // 1: Prepare before planning and execution. - // Start an implicit transaction if necessary. err = e.startTxIfNecessary(ctx, safeSession) if err != nil { @@ -81,19 +80,13 @@ func (e *Executor) newExecute( bindVars = make(map[string]*querypb.BindVariable) } - query, comments := sqlparser.SplitMarginComments(sql) - - // 2: Parse and Validate query. - stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser()) - if err != nil { - return err - } - var ( vs = e.VSchema() lastVSchemaCreated = vs.GetCreated() result *sqltypes.Result plan *engine.Plan + vcursor *econtext.VCursorImpl + stmt sqlparser.Statement cancel context.CancelFunc ) @@ -119,19 +112,14 @@ func (e *Executor) newExecute( } } - vcursor, err := econtext.NewVCursorImpl(safeSession, comments, e, logStats, e.vm, vs, e.resolver.resolver, e.serv, nullResultsObserver{}, e.vConfig) - if err != nil { - return err - } - - // 3: Create a plan for the query. + // Create a plan for the query. // If we are retrying, it is likely that the routing rules have changed and hence we need to // replan the query since the target keyspace of the resolved shards may have changed as a // result of MoveTables SwitchTraffic which does a RebuildSrvVSchema which in turn causes // the vtgate to clear the cached plans when processing the new serving vschema. // When buffering ends, many queries might be getting planned at the same time and we then // take full advatange of the cached plan. - plan, err = e.getPlan(ctx, vcursor, query, stmt, comments, bindVars, reservedVars, e.config.Normalize, logStats) + plan, vcursor, stmt, err = e.fetchOrCreatePlan(ctx, safeSession, sql, bindVars, e.config.Normalize, prepared, logStats) execStart := e.logPlanningFinished(logStats, plan) if err != nil { @@ -160,14 +148,14 @@ func (e *Executor) newExecute( return recResult(plan.QueryType, result) } - // 4: Prepare for execution. + // Prepare for execution. err = e.addNeededBindVars(vcursor, plan.BindVarNeeds, bindVars, safeSession) if err != nil { logStats.Error = err return err } - // 5: Execute the plan. + // Execute the plan. if plan.Instructions.NeedsTransaction() { err = e.insideTransaction(ctx, safeSession, logStats, func() error { @@ -181,10 +169,10 @@ func (e *Executor) newExecute( return err } - // 6: Retry if needed. + // Retry if needed. rootCause := vterrors.RootCause(err) if rootCause != nil && strings.Contains(rootCause.Error(), "enforce denied tables") { - log.V(2).Infof("Retry: %d, will retry query %s due to %v", try, query, err) + log.V(2).Infof("Retry: %d, will retry query %s due to %v", try, sql, err) if try == 0 { // We are going to retry at least once defer func() { // Prevent any plan cache pollution from queries planned against the wrong keyspace during a MoveTables @@ -219,7 +207,7 @@ func (e *Executor) newExecute( return err } - return vterrors.New(vtrpcpb.Code_INTERNAL, fmt.Sprintf("query %s failed after retries: %v ", query, err)) + return vterrors.New(vtrpcpb.Code_INTERNAL, fmt.Sprintf("query %s failed after retries: %v ", sql, err)) } // handleTransactions deals with transactional queries: begin, commit, rollback and savepoint management @@ -371,7 +359,7 @@ func (e *Executor) rollbackPartialExec(ctx context.Context, safeSession *econtex rQuery := safeSession.GetRollbackOnPartialExec() if rQuery != econtext.TxRollback { safeSession.SavepointRollback() - _, _, err = e.execute(ctx, nil, safeSession, rQuery, bindVars, logStats) + _, _, err = e.execute(ctx, nil, safeSession, rQuery, bindVars, false, logStats) // If no error, the revert is successful with the savepoint. Notify the reason as error to the client. if err == nil { errMsg.WriteString("reverted partial DML execution failure") diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index baec8b3175f..8cf01f645e9 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -94,7 +94,7 @@ func TestBuilder(query string, vschema plancontext.VSchema, keyspace string) (*e } } reservedVars := sqlparser.NewReservedVars("vtg", known) - result, err := sqlparser.PrepareAST(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, keyspace, sqlparser.SQLSelectLimitUnset, "", nil, vschema.GetForeignKeyChecksState(), vschema) + result, err := sqlparser.Normalize(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, keyspace, sqlparser.SQLSelectLimitUnset, "", nil, vschema.GetForeignKeyChecksState(), vschema) if err != nil { return nil, err } @@ -148,7 +148,7 @@ func getPlannerFromQueryHint(stmt sqlparser.Statement) (plancontext.PlannerVersi } func buildRoutePlan(stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, f func(statement sqlparser.Statement, reservedVars *sqlparser.ReservedVars, schema plancontext.VSchema) (*planResult, error)) (*planResult, error) { - if vschema.Destination() != nil { + if vschema.ShardDestination() != nil { return buildPlanForBypass(stmt, reservedVars, vschema) } return f(stmt, reservedVars, vschema) @@ -239,7 +239,7 @@ func buildAnalyzePlan(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vsche var ks *vindexes.Keyspace var err error - dest := key.Destination(key.DestinationAllShards{}) + dest := key.ShardDestination(key.DestinationAllShards{}) if analyzeStmt.Table.Qualifier.NotEmpty() && sqlparser.SystemSchema(analyzeStmt.Table.Qualifier.String()) { ks, err = vschema.AnyKeyspace() @@ -315,7 +315,7 @@ func buildLoadPlan(query string, vschema plancontext.VSchema) (*planResult, erro return nil, err } - destination := vschema.Destination() + destination := vschema.ShardDestination() if destination == nil { if err := vschema.ErrorIfShardedF(keyspace, "LOAD", "LOAD is not supported on sharded keyspace"); err != nil { return nil, err @@ -360,7 +360,7 @@ func buildFlushOptions(stmt *sqlparser.Flush, vschema plancontext.VSchema) (*pla return nil, err } - dest := vschema.Destination() + dest := vschema.ShardDestination() if dest == nil { dest = key.DestinationAllShards{} } @@ -380,10 +380,10 @@ func buildFlushTables(stmt *sqlparser.Flush, vschema plancontext.VSchema) (*plan tc := &tableCollector{} type sendDest struct { ks *vindexes.Keyspace - dest key.Destination + dest key.ShardDestination } - dest := vschema.Destination() + dest := vschema.ShardDestination() if dest == nil { dest = key.DestinationAllShards{} } diff --git a/go/vt/vtgate/planbuilder/bypass.go b/go/vt/vtgate/planbuilder/bypass.go index 6e3f64990d7..70beaabe25a 100644 --- a/go/vt/vtgate/planbuilder/bypass.go +++ b/go/vt/vtgate/planbuilder/bypass.go @@ -30,7 +30,7 @@ func buildPlanForBypass(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vsc if err != nil { return nil, err } - switch dest := vschema.Destination().(type) { + switch dest := vschema.ShardDestination().(type) { case key.DestinationExactKeyRange: if _, ok := stmt.(*sqlparser.Insert); ok { return nil, vterrors.VT03023(vschema.TargetString()) @@ -60,7 +60,7 @@ func buildPlanForBypass(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vsc send := &engine.Send{ Keyspace: keyspace, - TargetDestination: vschema.Destination(), + TargetDestination: vschema.ShardDestination(), Query: sqlparser.String(stmt), IsDML: sqlparser.IsDMLStatement(stmt), SingleShardOnly: false, diff --git a/go/vt/vtgate/planbuilder/ddl.go b/go/vt/vtgate/planbuilder/ddl.go index 112d0280813..7c859a13f25 100644 --- a/go/vt/vtgate/planbuilder/ddl.go +++ b/go/vt/vtgate/planbuilder/ddl.go @@ -45,7 +45,7 @@ func (fk *fkContraint) FkWalk(node sqlparser.SQLNode) (kontinue bool, err error) // This is why we return a compound primitive (DDL) which contains fully populated primitives (Send & OnlineDDL), // and which chooses which of the two to invoke at runtime. func buildGeneralDDLPlan(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) { - if vschema.Destination() != nil { + if vschema.ShardDestination() != nil { return buildByPassPlan(sql, vschema, true) } normalDDLPlan, onlineDDLPlan, err := buildDDLPlans(ctx, sql, ddlStatement, reservedVars, vschema, cfg) @@ -85,7 +85,7 @@ func buildByPassPlan(sql string, vschema plancontext.VSchema, isDDL bool) (*plan } send := &engine.Send{ Keyspace: keyspace, - TargetDestination: vschema.Destination(), + TargetDestination: vschema.ShardDestination(), Query: sql, IsDDL: isDDL, } @@ -93,7 +93,7 @@ func buildByPassPlan(sql string, vschema plancontext.VSchema, isDDL bool) (*plan } func buildDDLPlans(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*engine.Send, *engine.OnlineDDL, error) { - var destination key.Destination + var destination key.ShardDestination var keyspace *vindexes.Keyspace var err error @@ -165,9 +165,9 @@ func checkFKError(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStateme return nil } -func findTableDestinationAndKeyspace(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.Destination, *vindexes.Keyspace, error) { +func findTableDestinationAndKeyspace(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.ShardDestination, *vindexes.Keyspace, error) { var table *vindexes.BaseTable - var destination key.Destination + var destination key.ShardDestination var keyspace *vindexes.Keyspace var err error table, _, _, _, destination, err = vschema.FindTableOrVindex(ddlStatement.GetTable()) @@ -198,7 +198,7 @@ func buildCreateViewCommon( cfg dynamicconfig.DDL, ddlSelect sqlparser.TableStatement, ddl sqlparser.DDLStatement, -) (key.Destination, *vindexes.Keyspace, error) { +) (key.ShardDestination, *vindexes.Keyspace, error) { if vschema.IsViewsEnabled() { return createViewEnabled(vschema, reservedVars, ddlSelect, ddl) } @@ -242,7 +242,7 @@ func buildCreateViewCommon( return destination, keyspace, nil } -func createViewEnabled(vschema plancontext.VSchema, reservedVars *sqlparser.ReservedVars, ddlSelect sqlparser.TableStatement, ddl sqlparser.DDLStatement) (key.Destination, *vindexes.Keyspace, error) { +func createViewEnabled(vschema plancontext.VSchema, reservedVars *sqlparser.ReservedVars, ddlSelect sqlparser.TableStatement, ddl sqlparser.DDLStatement) (key.ShardDestination, *vindexes.Keyspace, error) { // For Create View, we require that the keyspace exist and the select query can be satisfied within the keyspace itself // We should remove the keyspace name from the table name, as the database name in MySQL might be different than the keyspace name destination, keyspace, err := findTableDestinationAndKeyspace(vschema, ddl) @@ -291,7 +291,7 @@ func createViewEnabled(vschema plancontext.VSchema, reservedVars *sqlparser.Rese return destination, keyspace, nil } -func buildDropView(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.Destination, *vindexes.Keyspace, error) { +func buildDropView(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.ShardDestination, *vindexes.Keyspace, error) { if !vschema.IsViewsEnabled() { return buildDropTable(vschema, ddlStatement) } @@ -319,11 +319,11 @@ func buildDropView(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatem return key.DestinationAllShards{}, ks, nil } -func buildDropTable(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.Destination, *vindexes.Keyspace, error) { - var destination key.Destination +func buildDropTable(vschema plancontext.VSchema, ddlStatement sqlparser.DDLStatement) (key.ShardDestination, *vindexes.Keyspace, error) { + var destination key.ShardDestination var keyspace *vindexes.Keyspace for i, tab := range ddlStatement.GetFromTables() { - var destinationTab key.Destination + var destinationTab key.ShardDestination var keyspaceTab *vindexes.Keyspace var table *vindexes.BaseTable var err error @@ -362,12 +362,12 @@ func buildDropTable(vschema plancontext.VSchema, ddlStatement sqlparser.DDLState return destination, keyspace, nil } -func buildRenameTable(vschema plancontext.VSchema, renameTable *sqlparser.RenameTable) (key.Destination, *vindexes.Keyspace, error) { - var destination key.Destination +func buildRenameTable(vschema plancontext.VSchema, renameTable *sqlparser.RenameTable) (key.ShardDestination, *vindexes.Keyspace, error) { + var destination key.ShardDestination var keyspace *vindexes.Keyspace for _, tabPair := range renameTable.TablePairs { - var destinationFrom key.Destination + var destinationFrom key.ShardDestination var keyspaceFrom *vindexes.Keyspace var table *vindexes.BaseTable var err error diff --git a/go/vt/vtgate/planbuilder/operators/misc_routing.go b/go/vt/vtgate/planbuilder/operators/misc_routing.go index 575aa7b4e9a..5cfb52c0248 100644 --- a/go/vt/vtgate/planbuilder/operators/misc_routing.go +++ b/go/vt/vtgate/planbuilder/operators/misc_routing.go @@ -37,7 +37,7 @@ type ( keyspace *vindexes.Keyspace // targetDestination specifies an explicit target destination tablet type - TargetDestination key.Destination + TargetDestination key.ShardDestination } // AnyShardRouting is used for routing logic where any shard in the keyspace can be used. diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go index 9aeafec2799..9f455e1acec 100644 --- a/go/vt/vtgate/planbuilder/operators/route.go +++ b/go/vt/vtgate/planbuilder/operators/route.go @@ -365,7 +365,7 @@ func findVSchemaTableAndCreateRoute( ) } -func createTargetedRouting(ctx *plancontext.PlanningContext, target key.Destination, tabletType topodatapb.TabletType, vschemaTable *vindexes.BaseTable) Routing { +func createTargetedRouting(ctx *plancontext.PlanningContext, target key.ShardDestination, tabletType topodatapb.TabletType, vschemaTable *vindexes.BaseTable) Routing { switch ctx.Statement.(type) { case *sqlparser.Update: if tabletType != topodatapb.TabletType_PRIMARY { diff --git a/go/vt/vtgate/planbuilder/plan_test_vindex.go b/go/vt/vtgate/planbuilder/plan_test_vindex.go index 30d72f8c03a..ee1369f1d87 100644 --- a/go/vt/vtgate/planbuilder/plan_test_vindex.go +++ b/go/vt/vtgate/planbuilder/plan_test_vindex.go @@ -36,7 +36,7 @@ func (*hashIndex) NeedsVCursor() bool { return false } func (*hashIndex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*hashIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*hashIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func newHashIndex(name string, _ map[string]string) (vindexes.Vindex, error) { @@ -53,7 +53,7 @@ func (*lookupIndex) NeedsVCursor() bool { return false } func (*lookupIndex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*lookupIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*lookupIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*lookupIndex) Create(context.Context, vindexes.VCursor, [][]sqltypes.Value, [][]byte, bool) error { @@ -87,7 +87,7 @@ func (*nameLkpIndex) GetCommitOrder() vtgatepb.CommitOrder { return vtgatepb.Com func (*nameLkpIndex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*nameLkpIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*nameLkpIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*nameLkpIndex) Create(context.Context, vindexes.VCursor, [][]sqltypes.Value, [][]byte, bool) error { @@ -102,7 +102,7 @@ func (*nameLkpIndex) Update(context.Context, vindexes.VCursor, []sqltypes.Value, func (*nameLkpIndex) Query() (string, []string) { return "select name, keyspace_id from name_user_vdx where name in ::name", []string{"name"} } -func (*nameLkpIndex) MapResult([]sqltypes.Value, []*sqltypes.Result) ([]key.Destination, error) { +func (*nameLkpIndex) MapResult([]sqltypes.Value, []*sqltypes.Result) ([]key.ShardDestination, error) { return nil, nil } @@ -131,7 +131,7 @@ func (*costlyIndex) NeedsVCursor() bool { return false } func (*costlyIndex) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*costlyIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*costlyIndex) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*costlyIndex) Create(context.Context, vindexes.VCursor, [][]sqltypes.Value, [][]byte, bool) error { @@ -157,7 +157,7 @@ func (m *multiColIndex) String() string { return m.name } func (*multiColIndex) Cost() int { return 1 } func (*multiColIndex) IsUnique() bool { return true } func (*multiColIndex) NeedsVCursor() bool { return false } -func (*multiColIndex) Map(ctx context.Context, vcursor vindexes.VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { +func (*multiColIndex) Map(ctx context.Context, vcursor vindexes.VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*multiColIndex) Verify(ctx context.Context, vcursor vindexes.VCursor, rowsColValues [][]sqltypes.Value, ksids [][]byte) ([]bool, error) { @@ -187,7 +187,7 @@ func (*unqLkpVdxBackfill) GetCommitOrder() vtgatepb.CommitOrder { return vtgatep func (*unqLkpVdxBackfill) Verify(context.Context, vindexes.VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*unqLkpVdxBackfill) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*unqLkpVdxBackfill) Map(ctx context.Context, vcursor vindexes.VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*unqLkpVdxBackfill) Create(context.Context, vindexes.VCursor, [][]sqltypes.Value, [][]byte, bool) error { @@ -202,7 +202,7 @@ func (*unqLkpVdxBackfill) Update(context.Context, vindexes.VCursor, []sqltypes.V func (*unqLkpVdxBackfill) Query() (string, []string) { return "select unq_key, keyspace_id from unq_lkp_idx where unq_key in ::unq_key", []string{"unq_key"} } -func (*unqLkpVdxBackfill) MapResult([]sqltypes.Value, []*sqltypes.Result) ([]key.Destination, error) { +func (*unqLkpVdxBackfill) MapResult([]sqltypes.Value, []*sqltypes.Result) ([]key.ShardDestination, error) { return nil, nil } func (u *unqLkpVdxBackfill) IsBackfilling() bool { return u.inBackfill } diff --git a/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go b/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go index b13c2112e3a..82f97e49a56 100644 --- a/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go +++ b/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go @@ -191,7 +191,7 @@ func (v *vschema) FindViewTarget(name sqlparser.TableName) (*vindexes.Keyspace, panic("implement me") } -func (v *vschema) FindTable(tablename sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.Destination, error) { +func (v *vschema) FindTable(tablename sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.ShardDestination, error) { // TODO implement me panic("implement me") } @@ -201,7 +201,7 @@ func (v *vschema) FindView(name sqlparser.TableName) sqlparser.TableStatement { panic("implement me") } -func (v *vschema) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (v *vschema) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { // TODO implement me panic("implement me") } @@ -216,7 +216,7 @@ func (v *vschema) TargetString() string { panic("implement me") } -func (v *vschema) Destination() key.Destination { +func (v *vschema) ShardDestination() key.ShardDestination { // TODO implement me panic("implement me") } @@ -226,7 +226,7 @@ func (v *vschema) TabletType() topodatapb.TabletType { panic("implement me") } -func (v *vschema) TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) { +func (v *vschema) TargetDestination(qualifier string) (key.ShardDestination, *vindexes.Keyspace, topodatapb.TabletType, error) { // TODO implement me panic("implement me") } @@ -345,7 +345,7 @@ func (v *vschema) GetUDV(name string) *querypb.BindVariable { panic("implement me") } -func (v *vschema) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) { +func (v *vschema) PlanPrepareStatement(context.Context, string) (*engine.Plan, error) { // TODO implement me panic("implement me") } diff --git a/go/vt/vtgate/planbuilder/plancontext/vschema.go b/go/vt/vtgate/planbuilder/plancontext/vschema.go index 1b86225382c..a62fb17d3ef 100644 --- a/go/vt/vtgate/planbuilder/plancontext/vschema.go +++ b/go/vt/vtgate/planbuilder/plancontext/vschema.go @@ -24,18 +24,18 @@ type PlannerVersion = querypb.ExecuteOptions_PlannerVersion // VSchema defines the interface for this package to fetch // info about tables. type VSchema interface { - FindTable(tablename sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.Destination, error) + FindTable(tablename sqlparser.TableName) (*vindexes.BaseTable, string, topodatapb.TabletType, key.ShardDestination, error) FindView(name sqlparser.TableName) sqlparser.TableStatement // FindViewTarget finds the target keyspace for the view table provided. FindViewTarget(name sqlparser.TableName) (*vindexes.Keyspace, error) - FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) + FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) // SelectedKeyspace returns the current keyspace if set, otherwise returns an error SelectedKeyspace() (*vindexes.Keyspace, error) TargetString() string - Destination() key.Destination + ShardDestination() key.ShardDestination TabletType() topodatapb.TabletType - TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) + TargetDestination(qualifier string) (key.ShardDestination, *vindexes.Keyspace, topodatapb.TabletType, error) AnyKeyspace() (*vindexes.Keyspace, error) FirstSortedKeyspace() (*vindexes.Keyspace, error) SysVarSetEnabled() bool @@ -87,7 +87,7 @@ type VSchema interface { GetUDV(name string) *querypb.BindVariable // PlanPrepareStatement plans the prepared statement. - PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) + PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, error) // ClearPrepareData clears the prepared data from the session. ClearPrepareData(stmtName string) diff --git a/go/vt/vtgate/planbuilder/prepared_statement.go b/go/vt/vtgate/planbuilder/prepared_statement.go index 88be1f2b168..b9fb832ae10 100644 --- a/go/vt/vtgate/planbuilder/prepared_statement.go +++ b/go/vt/vtgate/planbuilder/prepared_statement.go @@ -53,15 +53,14 @@ func prepareStmt(ctx context.Context, vschema plancontext.VSchema, pStmt *sqlpar return nil, err } - plan, stmt, err := vschema.PlanPrepareStatement(ctx, pQuery) + plan, err := vschema.PlanPrepareStatement(ctx, pQuery) if err != nil { return nil, err } - count := countArguments(stmt) vschema.StorePrepareData(stmtName, &vtgatepb.PrepareData{ - PrepareStatement: sqlparser.String(stmt), - ParamsCount: count, + PrepareStatement: plan.Original, + ParamsCount: int32(plan.ParamsCount), }) return &planResult{ @@ -70,19 +69,6 @@ func prepareStmt(ctx context.Context, vschema plancontext.VSchema, pStmt *sqlpar }, nil } -func countArguments(stmt sqlparser.Statement) (paramsCount int32) { - _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) { - switch node := node.(type) { - case *sqlparser.Argument: - if regexParams.MatchString(node.Name) { - paramsCount++ - } - } - return true, nil - }, stmt) - return -} - func fetchUDVValue(vschema plancontext.VSchema, udv string) (string, error) { bv := vschema.GetUDV(udv) if bv == nil { @@ -105,7 +91,7 @@ func buildExecuteStmtPlan(ctx context.Context, vschema plancontext.VSchema, eStm return nil, vterrors.VT03025("EXECUTE") } - plan, _, err := vschema.PlanPrepareStatement(ctx, prepareData.PrepareStatement) + plan, err := vschema.PlanPrepareStatement(ctx, prepareData.PrepareStatement) if err != nil { return nil, err } diff --git a/go/vt/vtgate/planbuilder/set.go b/go/vt/vtgate/planbuilder/set.go index 77f20be40f9..f8b0f3e6443 100644 --- a/go/vt/vtgate/planbuilder/set.go +++ b/go/vt/vtgate/planbuilder/set.go @@ -190,7 +190,7 @@ func buildSetOpReservedConn(s setting) planFunc { return &engine.SysVarReservedConn{ Name: expr.Var.Name.Lowered(), Keyspace: ks, - TargetDestination: vschema.Destination(), + TargetDestination: vschema.ShardDestination(), Expr: value, SupportSetVar: s.supportSetVar, }, nil @@ -234,13 +234,13 @@ func buildSetOpVitessAware(s setting) planFunc { } } -func resolveDestination(vschema plancontext.VSchema) (*vindexes.Keyspace, key.Destination, error) { +func resolveDestination(vschema plancontext.VSchema) (*vindexes.Keyspace, key.ShardDestination, error) { keyspace, err := vschema.AnyKeyspace() if err != nil { return nil, nil, err } - dest := vschema.Destination() + dest := vschema.ShardDestination() if dest == nil { dest = key.DestinationAnyShard{} } diff --git a/go/vt/vtgate/planbuilder/show.go b/go/vt/vtgate/planbuilder/show.go index 40cf7b2411f..6195cb16a90 100644 --- a/go/vt/vtgate/planbuilder/show.go +++ b/go/vt/vtgate/planbuilder/show.go @@ -44,7 +44,7 @@ const ( ) func buildShowPlan(sql string, stmt *sqlparser.Show, _ *sqlparser.ReservedVars, vschema plancontext.VSchema) (*planResult, error) { - if vschema.Destination() != nil { + if vschema.ShardDestination() != nil { return buildByPassPlan(sql, vschema, false) } @@ -183,7 +183,7 @@ func buildShowTblPlan(show *sqlparser.ShowBasic, vschema plancontext.VSchema) (e show.DbName = sqlparser.NewIdentifierCS("") } - dest := key.Destination(key.DestinationAnyShard{}) + dest := key.ShardDestination(key.DestinationAnyShard{}) var ks *vindexes.Keyspace var err error @@ -495,7 +495,7 @@ func buildCreateDbPlan(show *sqlparser.ShowCreate, vschema plancontext.VSchema) } func buildCreateTblPlan(show *sqlparser.ShowCreate, vschema plancontext.VSchema) (engine.Primitive, error) { - dest := key.Destination(key.DestinationAnyShard{}) + dest := key.ShardDestination(key.DestinationAnyShard{}) var ks *vindexes.Keyspace var err error diff --git a/go/vt/vtgate/planbuilder/simplifier_test.go b/go/vt/vtgate/planbuilder/simplifier_test.go index 012475ba021..634e7e2351b 100644 --- a/go/vt/vtgate/planbuilder/simplifier_test.go +++ b/go/vt/vtgate/planbuilder/simplifier_test.go @@ -48,7 +48,7 @@ func TestSimplifyBuggyQuery(t *testing.T) { stmt, reserved, err := sqlparser.NewTestParser().Parse2(query) require.NoError(t, err) reservedVars := sqlparser.NewReservedVars("vtg", reserved) - rewritten, _ := sqlparser.PrepareAST(sqlparser.Clone(stmt), reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) + rewritten, _ := sqlparser.Normalize(sqlparser.Clone(stmt), reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) simplified := simplifier.SimplifyStatement( stmt.(sqlparser.TableStatement), @@ -72,7 +72,7 @@ func TestSimplifyPanic(t *testing.T) { stmt, reserved, err := sqlparser.NewTestParser().Parse2(query) require.NoError(t, err) reservedVars := sqlparser.NewReservedVars("vtg", reserved) - rewritten, _ := sqlparser.PrepareAST(sqlparser.Clone(stmt), reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) + rewritten, _ := sqlparser.Normalize(sqlparser.Clone(stmt), reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) simplified := simplifier.SimplifyStatement( stmt.(sqlparser.TableStatement), @@ -103,7 +103,7 @@ func TestUnsupportedFile(t *testing.T) { return } reservedVars := sqlparser.NewReservedVars("vtg", reserved) - rewritten, err := sqlparser.PrepareAST(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) + rewritten, err := sqlparser.Normalize(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) if err != nil { t.Skip() } @@ -135,7 +135,7 @@ func keepSameError(query string, reservedVars *sqlparser.ReservedVars, vschema * if err != nil { panic(err) } - rewritten, _ := sqlparser.PrepareAST(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) + rewritten, _ := sqlparser.Normalize(stmt, reservedVars, map[string]*querypb.BindVariable{}, false, vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil) ast := rewritten.AST _, expected := BuildFromStmt(context.Background(), query, ast, reservedVars, vschema, rewritten.BindVarNeeds, staticConfig{}) if expected == nil { diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases.json b/go/vt/vtgate/planbuilder/testdata/dml_cases.json index 9b00821fb7b..53937fa739c 100644 --- a/go/vt/vtgate/planbuilder/testdata/dml_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/dml_cases.json @@ -25,7 +25,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update m1 set val = 1", "Table": "m1" }, @@ -49,7 +48,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set val = 1", "Table": "unsharded" }, @@ -73,7 +71,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = (select col from unsharded limit 1)", "Table": "unsharded" }, @@ -97,7 +94,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = (select id from unsharded union select id from unsharded)", "Table": "unsharded" }, @@ -121,7 +117,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = (select id from unsharded as a join unsharded as b on a.id = b.id)", "Table": "unsharded" }, @@ -145,7 +140,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id set col1 = 'asdf' where keepers.id is null and foo.col is not null and foo.col < 1000", "Table": "unsharded" }, @@ -169,7 +163,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` as route1 set a = 1 where id = 1", "Table": "user", "Values": [ @@ -197,7 +190,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded_a set a = (select a from unsharded as route2)", "Table": "unsharded, unsharded_a" }, @@ -222,7 +214,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded", "Table": "unsharded" }, @@ -246,7 +237,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from seq", "Table": "seq" }, @@ -270,7 +260,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded_ref", "Table": "unsharded_ref" }, @@ -294,7 +283,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where id = 1", "Table": "user", "Values": [ @@ -322,7 +310,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` as user_alias set val = 1 where user_alias.id = 1", "Table": "user", "Values": [ @@ -350,7 +337,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where id = 1", "Table": "user", "Values": [ @@ -378,7 +364,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where `name` = 'foo' and id = 1", "Table": "user", "Values": [ @@ -406,7 +391,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "email_user_map:4" ], @@ -446,7 +430,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "address_user_map:5", "email_user_map:4" @@ -481,7 +464,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "email_user_map:4" ], @@ -515,7 +497,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "music_user_map:1" ], @@ -549,7 +530,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where id = id2 and id = 1", "Table": "user", "Values": [ @@ -577,7 +557,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where id = 18446744073709551616 and id = 1", "Table": "user", "Values": [ @@ -605,7 +584,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where id = 1 for update", @@ -636,7 +614,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete a from unsharded_a as a, unsharded_b as b where a.id = b.id and b.val = 1", "Table": "unsharded_a, unsharded_b" }, @@ -661,7 +638,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete a from unsharded_a as a join unsharded_b as b on a.id = b.id where b.val = 1", "Table": "unsharded_a, unsharded_b" }, @@ -686,7 +662,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete foo from unsharded as foo left join (select id from unsharded where col is not null order by col desc limit 10) as keepers on foo.id = keepers.id where keepers.id is null and foo.col is not null and foo.col < 1000", "Table": "unsharded" }, @@ -710,7 +685,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as route1 where id = 1 for update", @@ -741,7 +715,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded_a where a = (select a from unsharded as route2)", "Table": "unsharded, unsharded_a" }, @@ -766,7 +739,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update music set val = 1 where id = 1", "Table": "music", "Values": [ @@ -794,7 +766,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded_a as a join unsharded_b as b on a.id = b.id set a.val = 'foo' where b.val = 1", "Table": "unsharded_a, unsharded_b" }, @@ -819,7 +790,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded_a as a, unsharded_b as b set a.val = 'foo' where a.id = b.id and b.val = 1", "Table": "unsharded_a, unsharded_b" }, @@ -844,7 +814,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music where id = 1 for update", @@ -875,7 +844,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from music_extra where user_id = 1", "Table": "music_extra", "Values": [ @@ -903,7 +871,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded values ()", "TableName": "unsharded" }, @@ -927,7 +894,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded values (1, 2)", "TableName": "unsharded" }, @@ -951,7 +917,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded values (1, 2) on duplicate key update x = 3", "TableName": "unsharded" }, @@ -975,7 +940,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into unsharded_authoritative(col1, col2) values (:__seq0, 1)", "TableName": "unsharded_authoritative" @@ -1000,7 +964,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into music(user_id, id) values (:_user_id_0, :_id_0) on duplicate key update user_id = values(user_id)", "TableName": "music", @@ -1029,7 +992,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into music(user_id, id) values (:_user_id_0, :_id_0), (:_user_id_1, :_id_1) on duplicate key update user_id = values(user_id)", "TableName": "music", @@ -1058,7 +1020,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded select id from unsharded_auto", "TableName": "unsharded" }, @@ -1083,7 +1044,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded select id from unsharded join unsharded_auto", "TableName": "unsharded" }, @@ -1108,7 +1068,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(18446744073709551616)", "Query": "insert into unsharded_auto(id, val) values (:__seq0, 'aa')", "TableName": "unsharded_auto" @@ -1133,7 +1092,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into unsharded_auto(id, val) values (:__seq0, 'aa')", "TableName": "unsharded_auto" @@ -1158,7 +1116,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into unsharded_auto(val, id) values ('aa', :__seq0)", "TableName": "unsharded_auto" @@ -1183,7 +1140,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into unsharded_auto(val, id) values (false, :__seq0)", "TableName": "unsharded_auto" @@ -1208,7 +1164,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, null)", "Query": "insert into unsharded_auto(id, val) values (:__seq0, 'aa'), (:__seq1, 'bb')", "TableName": "unsharded_auto" @@ -1233,7 +1188,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded values (1, 1)", "TableName": "unsharded" }, @@ -1257,7 +1211,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into `user`(id, val, `Name`, Costly) values (:_Id_0, 1, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1287,7 +1240,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1323,7 +1275,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into authoritative(user_id, col1, col2) values (:_user_id_0, 2, 3)", "TableName": "authoritative", "VindexValues": { @@ -1350,7 +1301,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1380,7 +1330,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1410,7 +1359,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "InsertIgnore": true, "Query": "insert ignore into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0)", @@ -1441,7 +1389,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "InsertIgnore": true, "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0) on duplicate key update col = 2", @@ -1472,7 +1419,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(:aa)", "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1502,7 +1448,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into `user`(nonid, id, `Name`, Costly) values (2, :_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1532,7 +1477,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into `user`(id, nonid, `Name`, Costly) values (:_Id_0, 2, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1562,7 +1506,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into `user`(nonid, id, `Name`, Costly) values (true, :_Id_0, :_Name_0, :_Costly_0)", "TableName": "user", @@ -1592,7 +1535,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "insert into `user`(nonid, `name`, id, Costly) values (2, :_Name_0, :_Id_0, :_Costly_0)", "TableName": "user", @@ -1622,7 +1564,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "insert into user_extra(nonid, extra_id, user_id) values (2, :__seq0, :_user_id_0)", "TableName": "user_extra", @@ -1650,7 +1591,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into `weird``name`(`a``b*c`, `b*c`) values (:_a_b_c_0, 2)", "TableName": "weird`name", "VindexValues": { @@ -1677,7 +1617,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded select 1 from dual union select 1 from dual", "TableName": "unsharded" }, @@ -1702,7 +1641,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(18446744073709551616)", "Query": "insert into user_extra(nonid, extra_id, user_id) values (2, :__seq0, :_user_id_0)", "TableName": "user_extra", @@ -1730,7 +1668,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into music_extra(music_id, user_id) values (:_music_id_0, :_user_id_0)", "TableName": "music_extra", "VindexValues": { @@ -1770,7 +1707,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, 2)", "Query": "insert into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0), (:_Id_1, :_Name_1, :_Costly_1)", "TableName": "user", @@ -1800,7 +1736,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, 2)", "Query": "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0), (:_Id_1, :_Name_1, :_Costly_1)", "QueryTimeout": 1, @@ -1831,7 +1766,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, 2)", "MultiShardAutocommit": true, "Query": "insert /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ into `user`(id, `Name`, Costly) values (:_Id_0, :_Name_0, :_Costly_0), (:_Id_1, :_Name_1, :_Costly_1)", @@ -1868,7 +1802,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "replace into unsharded values (1, 2)", "TableName": "unsharded" }, @@ -1892,7 +1825,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "replace into unsharded select id from unsharded_auto", "TableName": "unsharded" }, @@ -1917,7 +1849,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(18446744073709551616)", "Query": "replace into unsharded_auto(id, val) values (:__seq0, 'aa')", "TableName": "unsharded_auto" @@ -1942,7 +1873,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1)", "Query": "replace into unsharded_auto(id, val) values (:__seq0, 'aa')", "TableName": "unsharded_auto" @@ -1967,7 +1897,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null)", "Query": "replace into unsharded_auto(val, id) values ('aa', :__seq0)", "TableName": "unsharded_auto" @@ -1992,7 +1921,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, null)", "Query": "replace into unsharded_auto(id, val) values (:__seq0, 'aa'), (:__seq1, 'bb')", "TableName": "unsharded_auto" @@ -2023,7 +1951,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into multicolvin(column_a, column_b, column_c, kid) values (:_column_a_0, :_column_b_0, :_column_c_0, :_kid_0)", "TableName": "multicolvin", "VindexValues": { @@ -2052,7 +1979,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into overlap_vindex(kid, column_a, column_b) values (:_kid_0, :_column_a_0, 3)", "TableName": "overlap_vindex", "VindexValues": { @@ -2080,7 +2006,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into multicolvin(column_a, column_b, column_c, kid) values (:_column_a_0, :_column_b_0, :_column_c_0, :_kid_0), (:_column_a_1, :_column_b_1, :_column_c_1, :_kid_1)", "TableName": "multicolvin", "VindexValues": { @@ -2109,7 +2034,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "kid_index", "OwnedVindexQuery": "select kid, column_a, column_b, column_c from multicolvin where kid = 1 for update", @@ -2140,7 +2064,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "colb_colc_map:4" ], @@ -2174,7 +2097,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "cola_map:4", "colb_colc_map:5" @@ -2209,7 +2131,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1", "Table": "user_extra" }, @@ -2233,7 +2154,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1", "Table": "user_extra" }, @@ -2257,7 +2177,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "MultiShardAutocommit": true, "Query": "update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ user_extra set val = 1", "Table": "user_extra" @@ -2282,7 +2201,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*vt+ QUERY_TIMEOUT_MS=1 */ user_extra set val = 1", "QueryTimeout": 1, "Table": "user_extra" @@ -2307,7 +2225,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1 where id between 1 and 2", "Table": "user_extra" }, @@ -2331,7 +2248,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1 where user_id in ::__vals", "Table": "user_extra", "Values": [ @@ -2359,7 +2275,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1 where `name` = 'foo'", "Table": "user_extra" }, @@ -2383,7 +2298,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1 where id in (1, 2)", "Table": "user_extra" }, @@ -2407,7 +2321,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1 where `name` = 'foo' or id = 1", "Table": "user_extra" }, @@ -2431,7 +2344,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "FetchLastInsertID": true, "Query": "update user_extra set col = last_insert_id(123)", "Table": "user_extra" @@ -2455,7 +2367,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "FetchLastInsertID": true, "Query": "delete from user_extra where col = last_insert_id(123)", "Table": "user_extra" @@ -2479,7 +2390,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra", "Table": "user_extra" }, @@ -2503,7 +2413,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra", "Table": "user_extra" }, @@ -2527,7 +2436,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra where user_id between 1 and 2", "Table": "user_extra" }, @@ -2551,7 +2459,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra where `name` = 'jose'", "Table": "user_extra" }, @@ -2575,7 +2482,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "MultiShardAutocommit": true, "Query": "delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from user_extra where `name` = 'jose'", "Table": "user_extra" @@ -2600,7 +2506,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*vt+ QUERY_TIMEOUT_MS=1 */ from user_extra where `name` = 'jose'", "QueryTimeout": 1, "Table": "user_extra" @@ -2625,7 +2530,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra where user_id in ::__vals", "Table": "user_extra", "Values": [ @@ -2653,7 +2557,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = (select id from unsharded_a where id = unsharded.col) where col = (select id from unsharded_b)", "Table": "unsharded, unsharded_a, unsharded_b" }, @@ -2679,7 +2582,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded where col = (select id from unsharded_a where id = unsharded.col)", "Table": "unsharded, unsharded_a" }, @@ -2704,7 +2606,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -2738,7 +2639,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded values (:__lastInsertId, 2)", "TableName": "unsharded" }, @@ -2762,7 +2662,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "FetchLastInsertID": true, "Query": "insert into unsharded values (last_insert_id(789), 2)", "TableName": "unsharded" @@ -2787,7 +2686,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -2821,7 +2719,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -2851,7 +2748,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -2881,7 +2777,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where id in (1, 2, 3) for update", @@ -2912,7 +2807,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where id + 1 = 2 for update", @@ -2939,7 +2833,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` for update", @@ -2966,7 +2859,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music where id = 1 for update", @@ -2997,7 +2889,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1", "Table": "user" }, @@ -3021,7 +2912,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` for update", @@ -3048,7 +2938,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "colb_colc_map:4" ], @@ -3082,7 +2971,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -3116,7 +3004,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where `name` = _binary 'abc' for update", @@ -3147,7 +3034,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` for update", @@ -3174,7 +3060,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -3204,7 +3089,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra set val = 1", "Table": "user_extra" }, @@ -3228,7 +3112,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where `user`.id * `user`.col = `user`.foo for update", @@ -3273,7 +3156,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into user_privacy_consents(user_id, accepted_at) select user_id, accepted_at from (select 1 as user_id, 1629194864 as accepted_at from dual) as tmp where not exists (select 1 from user_privacy_consents where user_id = 1)", "TableName": "user_privacy_consents" }, @@ -3298,7 +3180,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "xxhash", "OwnedVindexQuery": "select c1, c2, c3 from t1 where c2 = 20 for update", @@ -3325,7 +3206,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "lookup_t1:3" ], @@ -3355,7 +3235,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "xxhash", "OwnedVindexQuery": "select c1, c2, c3 from t1 where c2 = 10 and c3 = 20 for update", @@ -3386,7 +3265,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "lookup_t1:3" ], @@ -3420,7 +3298,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "xxhash", "OwnedVindexQuery": "select c1, c2, c3 from t1 where c2 = 10 and c3 in (20, 21) for update", @@ -3451,7 +3328,6 @@ "Name": "zlookup_unique", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "lookup_t1:3" ], @@ -3485,7 +3361,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -3515,7 +3390,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as u where u.col > 20 for update", @@ -3542,7 +3416,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 1 where cola = 1 and colb = 2", "Table": "multicol_tbl", "Values": [ @@ -3571,7 +3444,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 1 where colb = 2 and cola = 1", "Table": "multicol_tbl", "Values": [ @@ -3600,7 +3472,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 1 where colb in ::__vals1 and cola = 1", "Table": "multicol_tbl", "Values": [ @@ -3629,7 +3500,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 1 where colb in ::__vals1 and cola in ::__vals0", "Table": "multicol_tbl", "Values": [ @@ -3658,7 +3528,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where cola = 1 and colb = 2 for update", @@ -3690,7 +3559,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where colb = 2 and cola = 1 for update", @@ -3722,7 +3590,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where colb in (1, 2) and cola = 1 for update", @@ -3754,7 +3621,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where colb in (1, 2) and cola in (3, 4) for update", @@ -3786,7 +3652,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "colc_map:4" ], @@ -3821,7 +3686,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 42 where `name` = 'foo'", "Table": "multicol_tbl", "Values": [ @@ -3849,7 +3713,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 42 where cola = 1", "Table": "multicol_tbl", "Values": [ @@ -3877,7 +3740,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_muticoltbl_map:4" ], @@ -3911,7 +3773,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_muticoltbl_map:4" ], @@ -3945,7 +3806,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update multicol_tbl set x = 1 where `name` = 'foo' and cola = 2", "Table": "multicol_tbl", "Values": [ @@ -3973,7 +3833,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where `name` = 'foo' for update", @@ -4004,7 +3863,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where cola = 1 for update", @@ -4035,7 +3893,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where cola in (1, 2) for update", @@ -4066,7 +3923,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 2, "KsidVindex": "multicolIdx", "OwnedVindexQuery": "select cola, colb, colc, `name` from multicol_tbl where `name` = 'foo' and cola = 2 for update", @@ -4097,7 +3953,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "TableName": "music", "VindexOffsetFromSelect": { "music_user_map": "[0]", @@ -4156,7 +4011,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(1)", "TableName": "user_extra", "VindexOffsetFromSelect": { @@ -4197,7 +4051,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(2)", "TableName": "user_extra", "VindexOffsetFromSelect": { @@ -4238,7 +4091,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(0)", "TableName": "user", "VindexOffsetFromSelect": { @@ -4281,7 +4133,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(1)", "TableName": "user", "VindexOffsetFromSelect": { @@ -4330,7 +4181,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(2)", "TableName": "user_extra", "VindexOffsetFromSelect": { @@ -4371,7 +4221,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into unsharded(col) select col from unsharded_auto", "TableName": "unsharded" }, @@ -4396,7 +4245,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(2)", "TableName": "user_extra", "VindexOffsetFromSelect": { @@ -4437,7 +4285,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(2)", "TableName": "user_extra", "VindexOffsetFromSelect": { @@ -4478,7 +4325,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "TableName": "unsharded", "Inputs": [ { @@ -4515,7 +4361,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "TableName": "unsharded", "Inputs": [ { @@ -4572,7 +4417,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set col = :__sq1", "Table": "user" } @@ -4619,7 +4463,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = :__sq1", "Table": "unsharded" } @@ -4692,7 +4535,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update unsharded set col = :__sq1", "Table": "unsharded" } @@ -4719,7 +4561,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set col = (select count(*) from user_extra where user_extra.user_id = 5) where id = 5", "Table": "user", "Values": [ @@ -4748,7 +4589,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set col = (select count(*) from user_extra where user_extra.user_id = `user`.id) where id = 5", "Table": "user", "Values": [ @@ -4777,7 +4617,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set col = (select count(*) from user_extra where user_extra.user_id = `user`.id) where id > 5", "Table": "user" }, @@ -4802,7 +4641,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into authoritative(user_id) values (:_user_id_0)", "TableName": "authoritative", "VindexValues": { @@ -4839,7 +4677,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` for update", @@ -4872,7 +4709,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` for update", @@ -4901,7 +4737,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete foo from unsharded as foo join (select id from unsharded as a join unsharded_b as b on a.user_id = b.user_id) as keepers on foo.id = keepers.id where keepers.id is null and foo.col is not null and foo.col < 1000", "Table": "unsharded, unsharded_b" }, @@ -4926,7 +4761,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set col = 1 where (`name`, col) in (('aa', 'bb'), ('cc', 'dd'))", "Table": "user", "Values": [ @@ -4954,7 +4788,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where (`name`, col) in (('aa', 'bb'), ('cc', 'dd')) for update", @@ -4985,7 +4818,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into ref(col) values (1)", "TableName": "ref" }, @@ -5009,7 +4841,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update m1 set foo = last_insert_id(foo + 1) where id = 12345", "Table": "m1" }, @@ -5033,7 +4864,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*vt+ QUERY_TIMEOUT_MS=1 */ unsharded set val = 1", "QueryTimeout": 1, "Table": "unsharded" @@ -5058,7 +4888,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into unsharded values ()", "QueryTimeout": 1, "TableName": "unsharded" @@ -5083,7 +4912,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InputAsNonStreaming": true, "TableName": "music", "VindexOffsetFromSelect": { @@ -5128,7 +4956,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(null, null, null)", "Query": "insert into mixed_tbl(shard_key, lkp_key) values (:_shard_key_0, :_lkp_key_0), (:_shard_key_1, :_lkp_key_1), (:_shard_key_2, :_lkp_key_2)", "TableName": "mixed_tbl", @@ -5157,7 +4984,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Values::(1, null, 27)", "Query": "insert into mixed_tbl(shard_key, lkp_key) values (:_shard_key_0, :_lkp_key_0), (:_shard_key_1, :_lkp_key_1), (:_shard_key_2, :_lkp_key_2)", "TableName": "mixed_tbl", @@ -5186,7 +5012,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(1)", "TableName": "mixed_tbl", "VindexOffsetFromSelect": { @@ -5232,7 +5057,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(1)", "TableName": "mixed_tbl", "VindexOffsetFromSelect": { @@ -5284,7 +5108,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "AutoIncrement": "select next :n /* INT64 */ values from seq:Offset(0)", "InputAsNonStreaming": true, "TableName": "user", @@ -5357,7 +5180,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into music(id, user_id, col) values (:_id_0, :_user_id_0, 3) on duplicate key update music.col = 5", "TableName": "music", @@ -5386,7 +5208,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref where col = 1", "Table": "source_of_ref" }, @@ -5410,7 +5231,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from ref", "Table": "ref" }, @@ -5434,7 +5254,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` limit 20 for update", @@ -5461,7 +5280,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u, ref_with_source as r where u.col = r.col for update", @@ -5489,7 +5307,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u, music as m where u.id = m.user_id for update", @@ -5512,7 +5329,6 @@ "Original": "delete user from user join user_extra on user.id = user_extra.id where user.name = 'foo'", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5561,7 +5377,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select `user`.Id, `user`.`Name`, `user`.Costly from `user` where `user`.id in ::dml_vals for update", @@ -5590,7 +5405,6 @@ "Original": "delete u from user u join music m on u.col = m.col", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5635,7 +5449,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u where u.id in ::dml_vals for update", @@ -5664,7 +5477,6 @@ "Original": "delete u from music m join user u where u.col = m.col and m.foo = 42", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5709,7 +5521,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u where u.id in ::dml_vals for update", @@ -5743,7 +5554,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u, music as m where m.foo = 42 and u.id = m.user_id for update", @@ -5766,7 +5576,6 @@ "Original": "delete u from user u join music m on u.col = m.col join user_extra ue on m.user_id = ue.user_id where ue.foo = 20 and u.col = 30 and m.bar = 40", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5811,7 +5620,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select u.Id, u.`Name`, u.Costly from `user` as u where u.id in ::dml_vals for update", @@ -5841,7 +5649,6 @@ "Original": "delete m from user u join music m on u.col = m.col join user_extra ue on m.user_id = ue.user_id where ue.foo = 20 and u.col = 30 and m.bar = 40", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5886,7 +5693,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select m.user_id, m.id from music as m where m.id in ::dml_vals for update", @@ -5916,7 +5722,6 @@ "Original": "delete from user limit 10", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -5945,7 +5750,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where `user`.id in ::dml_vals for update", @@ -5973,7 +5777,6 @@ "Original": "delete from user order by name, col limit 5", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -6003,7 +5806,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where `user`.id in ::dml_vals for update", @@ -6031,7 +5833,6 @@ "Original": "update user set val = 1 where (name = 'foo' or id = 1) limit 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -6060,7 +5861,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` set val = 1 where `user`.id in ::dml_vals", "Table": "user", "Values": [ @@ -6085,7 +5885,6 @@ "Original": "update user set name = 'abc' where id > 10 limit 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -6114,7 +5913,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -6145,7 +5943,6 @@ "Original": "update user as u, user_extra as ue set u.name = 'foo' where u.id = ue.id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -6194,7 +5991,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -6226,7 +6022,6 @@ "Original": "update user join user_extra on user.id = user_extra.id set user.name = 'foo'", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -6275,7 +6070,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -6307,7 +6101,6 @@ "Original": "update user as u, user_extra as ue set u.col = ue.col where u.id = ue.id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "BindVars": [ "0:[ue_col:1]" ], @@ -6359,7 +6152,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` as u set u.col = :ue_col /* INT16 */ where u.id in ::dml_vals", "Table": "user", "Values": [ @@ -6385,7 +6177,6 @@ "Original": "update user as u, user_extra as ue set u.col = ue.foo + ue.bar + u.baz where u.id = ue.id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "BindVars": [ "0:[ue_bar:2 ue_foo:1]" ], @@ -6437,7 +6228,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update `user` as u set u.col = :ue_foo + :ue_bar + u.baz where u.id in ::dml_vals", "Table": "user", "Values": [ @@ -6463,7 +6253,6 @@ "Original": "update user, user_extra ue set user.name = ue.id + 'foo', ue.bar = user.baz where user.id = ue.id and user.id = 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "BindVars": [ "0:[ue_id:1]", "1:[user_baz:3]" @@ -6517,7 +6306,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "ChangedVindexValues": [ "name_user_map:3" ], @@ -6538,7 +6326,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update user_extra as ue set ue.bar = :user_baz where (ue.id, ue.user_id) in ::dml_vals", "Table": "user_extra", "Values": [ @@ -6581,7 +6368,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music as bar for update", @@ -6608,7 +6394,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update music as bar set col = 23", "Table": "music" }, @@ -6632,7 +6417,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into music(id, user_id) values (:_id_0, :_user_id_0)", "TableName": "music", "VindexValues": { @@ -6684,7 +6468,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where id = :__sq1 for update", @@ -6738,7 +6521,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where col = :__sq1 for update", @@ -6788,7 +6570,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded where col = :__sq1", "Table": "unsharded" } @@ -6857,7 +6638,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded where col = :__sq1", "Table": "unsharded" } @@ -6930,7 +6710,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from unsharded where col = :__sq1", "Table": "unsharded" } @@ -6952,7 +6731,6 @@ "Original": "delete u, m from user u, music m where u.col = m.col and u.foo = m.bar and u.baz = 12 and m.baz = 21", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -6999,7 +6777,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as u where u.id in ::dml_vals for update", @@ -7017,7 +6794,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music as m where m.id in ::dml_vals for update", @@ -7046,7 +6822,6 @@ "Original": "delete music,user from music inner join user where music.id = user.id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -7096,7 +6871,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music where music.id in ::dml_vals for update", @@ -7114,7 +6888,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` where `user`.id in ::dml_vals for update", @@ -7143,7 +6916,6 @@ "Original": "delete u, m from user u join music m on u.id = m.user_id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -7167,7 +6939,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as u where u.id in ::dml_vals for update", @@ -7185,7 +6956,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music as m where m.id in ::dml_vals for update", @@ -7214,7 +6984,6 @@ "Original": "delete u, m from user u join music m on u.col = m.col", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -7260,7 +7029,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as u where u.id in ::dml_vals for update", @@ -7278,7 +7046,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select user_id, id from music as m where m.id in ::dml_vals for update", @@ -7307,7 +7074,6 @@ "Original": "delete u, ue from user u join user_extra ue on u.id = ue.user_id", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1 2]" @@ -7331,7 +7097,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "user_index", "OwnedVindexQuery": "select Id, `Name`, Costly from `user` as u where u.id in ::dml_vals for update", @@ -7349,7 +7114,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from user_extra as ue where (ue.id, ue.user_id) in ::dml_vals", "Table": "user_extra", "Values": [ @@ -7375,7 +7139,6 @@ "Original": "delete o, ev from `order` o join order_event ev where o.oid = ev.oid and ev.ename = 'a'", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0 1]", "1:[2 3]" @@ -7399,7 +7162,6 @@ "Name": "ordering", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from order_event as ev where (ev.oid, ev.ename) in ::dml_vals", "Table": "order_event", "Values": [ @@ -7414,7 +7176,6 @@ "Name": "ordering", "Sharded": true }, - "TargetTabletType": "PRIMARY", "KsidLength": 1, "KsidVindex": "xxhash", "OwnedVindexQuery": "select region_id, oid from `order` as o where (o.oid, o.region_id) in ::dml_vals for update", @@ -7443,7 +7204,6 @@ "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]" @@ -7489,7 +7249,6 @@ "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": [ @@ -7504,7 +7263,6 @@ "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": [ @@ -7535,7 +7293,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into authoritative(user_id, col1, col2) values (:_user_id_0, '2', 3), (:_user_id_1, '5', 6) as new on duplicate key update col2 = new.user_id + new.col1", "TableName": "authoritative", @@ -7563,7 +7320,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into authoritative(user_id, col1, col2) values (:_user_id_0, '2', 3), (:_user_id_1, '5', 6) as new (a, b, c) on duplicate key update col1 = a + c", "TableName": "authoritative", @@ -7591,7 +7347,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into authoritative(user_id, col1, col2) values (:_user_id_0, '2', 3), (:_user_id_1, '5', 6) as new on duplicate key update col2 = new.user_id + new.col1", "TableName": "authoritative", @@ -7619,7 +7374,6 @@ "Name": "user", "Sharded": true }, - "TargetTabletType": "PRIMARY", "InsertIgnore": true, "Query": "insert into authoritative(user_id, col1, col2) values (:_user_id_0, '2', 3), (:_user_id_1, '5', 6) as new (a, b, c) on duplicate key update col1 = a + c", "TableName": "authoritative", diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json b/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json index cf956a1685b..5d2dfc3bbdd 100644 --- a/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json +++ b/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json @@ -13,7 +13,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update ambiguous_ref_with_source set done = true where id = 1", "Table": "ambiguous_ref_with_source" }, diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json index f48521890d8..74a53146278 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json @@ -18,7 +18,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into tbl2(col2, coly) values (:_col2_0, 3)", "TableName": "tbl2", "VindexValues": { @@ -44,7 +43,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into multicol_tbl2(cola, colb, colc) values (:_cola_0, :_colb_0, :_colc_0)", "TableName": "multicol_tbl2", "VindexValues": { @@ -102,7 +100,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0, @@ -128,7 +125,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -176,7 +172,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -196,7 +191,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 1 @@ -212,7 +206,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from tbl5", "Table": "tbl5" } @@ -259,7 +252,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -275,7 +267,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl9 where col9 = 5", "Table": "u_tbl9" } @@ -301,7 +292,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl5 set col5 = 'foo' where id = 1", "Table": "u_tbl5" }, @@ -340,7 +330,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -356,7 +345,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set col2 = 'bar' where id = 1", "Table": "u_tbl2" } @@ -382,7 +370,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set col_no_ref = 'baz' where id = 1", "Table": "u_tbl2" }, @@ -410,7 +397,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update tbl1 set not_ref_col = 'foo' where id = 1", "Table": "tbl1" }, @@ -454,7 +440,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -470,7 +455,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update tbl5 set t5col5 = 'foo'", "Table": "tbl5" } @@ -501,7 +485,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update tbl2 set col = 'foo'", "Table": "tbl2" }, @@ -579,7 +562,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update tbl10 set col = 'foo'", "Table": "tbl10" } @@ -630,7 +612,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -646,7 +627,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from tbl9 where col9 = 34", "Table": "tbl9", "Values": [ @@ -712,7 +692,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -728,7 +707,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 'foo' where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -762,7 +740,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -778,7 +755,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast('foo' as CHAR)))", "Table": "u_tbl9" } @@ -792,7 +768,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl1 set col1 = 'foo'", "Table": "u_tbl1" } @@ -853,7 +828,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -876,7 +850,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set m = 2, col2 = col1 + 'bar' where id = 1", "Table": "u_tbl2" } @@ -948,7 +921,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -964,7 +936,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = :fkc_upd where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1005,7 +976,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1021,7 +991,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (:fkc_upd1 is null or (col9) not in ((:fkc_upd1)))", "Table": "u_tbl9" } @@ -1035,7 +1004,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl1 set m = 2, col1 = x + 'bar' where id = 1", "Table": "u_tbl1" } @@ -1080,7 +1048,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -1096,7 +1063,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set m = col1 + 'bar', col2 = 2 where id = 1", "Table": "u_tbl2" } @@ -1158,7 +1124,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -1174,7 +1139,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1208,7 +1172,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1224,7 +1187,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast(2 as CHAR)))", "Table": "u_tbl9" } @@ -1238,7 +1200,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl1 set m = x + 'bar', col1 = 2 where id = 1", "Table": "u_tbl1" } @@ -1267,7 +1228,6 @@ "Original": "delete from u_tbl2 limit 2", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -1306,7 +1266,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -1322,7 +1281,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl2 where u_tbl2.id in ::dml_vals", "Table": "u_tbl2" } @@ -1408,7 +1366,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ tbl3 set coly = colx + 10 where coly = 10", "Table": "tbl3" } @@ -1489,7 +1446,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update tbl3 set coly = 20 where coly = 10", "Table": "tbl3" } @@ -1551,7 +1507,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl8 set col8 = 'foo' where (col8) in ::fkc_vals", "Table": "u_tbl8" } @@ -1565,7 +1520,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl6 set col6 = 'foo'", "Table": "u_tbl6" } @@ -1640,7 +1594,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col4 = 'foo' where (col4) in ::fkc_vals", "Table": "u_tbl4" } @@ -1654,7 +1607,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl7 set col7 = 'foo'", "Table": "u_tbl7" } @@ -1730,7 +1682,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col4 = :v1 where (col4) in ::fkc_vals", "Table": "u_tbl4" } @@ -1744,7 +1695,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl7 set col7 = :v1", "Table": "u_tbl7" } @@ -1767,7 +1717,6 @@ "Original": "insert into u_tbl1 (id, col1) values (1, 3) on duplicate key update col1 = 5", "Instructions": { "OperatorType": "Upsert", - "TargetTabletType": "PRIMARY", "Inputs": [ { "InputName": "Insert-1", @@ -1777,7 +1726,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl1(id, col1) values (1, 3)", "TableName": "u_tbl1" @@ -1826,7 +1774,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -1842,7 +1789,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 5 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1876,7 +1822,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1892,7 +1837,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast(5 as CHAR)))", "Table": "u_tbl9" } @@ -1906,7 +1850,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl1 set col1 = 5 where id = 1", "Table": "u_tbl1" } @@ -1937,7 +1880,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into u_tbl1(id, col1, foo) values (1, 3, 'bar') on duplicate key update foo = 'baz'", "TableName": "u_tbl1" }, @@ -2004,7 +1946,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -2020,7 +1961,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -2034,7 +1974,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl1 where (id) in ((1))", "Table": "u_tbl1" } @@ -2047,7 +1986,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl1(id, col1) values (1, 2)", "TableName": "u_tbl1" @@ -2112,7 +2050,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0, @@ -2129,7 +2066,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals and (cola, colb) not in ((1, 2))", "Table": "u_multicol_tbl2" } @@ -2143,7 +2079,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl1 set cola = 1, colb = 2 where id = 3", "Table": "u_multicol_tbl1" } @@ -2207,7 +2142,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0, @@ -2224,7 +2158,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals and (:v2 is null or (:v1 is null or (cola, colb) not in ((:v1, :v2))))", "Table": "u_multicol_tbl2" } @@ -2238,7 +2171,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl1 set cola = :v1, colb = :v2 where id = :v3", "Table": "u_multicol_tbl1" } @@ -2287,7 +2219,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -2307,7 +2238,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 1 @@ -2323,7 +2253,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from tbl5 where id = :v1", "Table": "tbl5" } @@ -2411,7 +2340,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col4 = :fkc_upd where (col4) in ::fkc_vals", "Table": "u_tbl4" } @@ -2425,7 +2353,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl7 set foo = 100, col7 = baz + 1 + col7 where bar = 42", "Table": "u_tbl7" } @@ -2497,7 +2424,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0, @@ -2514,7 +2440,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals and (:fkc_upd is null or (cola) not in ((:fkc_upd)))", "Table": "u_multicol_tbl2" } @@ -2528,7 +2453,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_multicol_tbl1 set cola = cola + 3 where id = 3", "Table": "u_multicol_tbl1" } @@ -2592,7 +2516,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0, @@ -2621,7 +2544,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_multicol_tbl2 set cola = 2, colb = colc - 2 where id = 7", "Table": "u_multicol_tbl2" } @@ -2669,7 +2591,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -2685,7 +2606,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl9 where (col9) in ((10), (20), (30)) or (col9 * foo) in ((10 * null), (20 * null), (30 * null)) or (bar, col9) in ((1, 10), (1, 20), (1, 30)) or (id) in ((1), (2), (3))", "Table": "u_tbl9" } @@ -2698,7 +2618,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl9(id, col9) values (1, 10), (2, 20), (3, 30)", "TableName": "u_tbl9" @@ -2725,7 +2644,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -2754,7 +2672,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=Off) */ u_multicol_tbl1 set cola = 1, colb = 2 where id = 3", "Table": "u_multicol_tbl1" }, @@ -2777,7 +2694,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into tbl3(col3, coly) values (:_col3_0, 3)", "TableName": "tbl3", "VindexValues": { @@ -2798,7 +2714,6 @@ "Original": "insert into u_tbl1 (id, col1) values (1, 3) on duplicate key update col1 = values(col1)", "Instructions": { "OperatorType": "Upsert", - "TargetTabletType": "PRIMARY", "Inputs": [ { "InputName": "Insert-1", @@ -2808,7 +2723,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl1(id, col1) values (1, 3)", "TableName": "u_tbl1" @@ -2857,7 +2771,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -2873,7 +2786,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 3 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -2907,7 +2819,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -2923,7 +2834,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast(3 as CHAR)))", "Table": "u_tbl9" } @@ -2937,7 +2847,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl1 set col1 = 3 where id = 1", "Table": "u_tbl1" } @@ -2963,7 +2872,6 @@ "Original": "insert into u_tbl2 (id, col2) values (:v1, :v2),(:v3, :v4), (:v5, :v6) on duplicate key update col2 = values(col2)", "Instructions": { "OperatorType": "Upsert", - "TargetTabletType": "PRIMARY", "Inputs": [ { "InputName": "Insert-1", @@ -2973,7 +2881,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl2(id, col2) values (:v1, :v2)", "TableName": "u_tbl2" @@ -3002,7 +2909,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -3018,7 +2924,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set col2 = :v2 where id = :v1", "Table": "u_tbl2" } @@ -3032,7 +2937,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl2(id, col2) values (:v3, :v4)", "TableName": "u_tbl2" @@ -3061,7 +2965,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -3077,7 +2980,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set col2 = :v4 where id = :v3", "Table": "u_tbl2" } @@ -3091,7 +2993,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert into u_tbl2(id, col2) values (:v5, :v6)", "TableName": "u_tbl2" @@ -3120,7 +3021,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals2", "Cols": [ 0 @@ -3136,7 +3036,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl2 set col2 = :v6 where id = :v5", "Table": "u_tbl2" } @@ -3164,7 +3063,6 @@ "Original": "delete u from u_tbl6 u join u_tbl5 m on u.col = m.col where u.col2 = 4 and m.col3 = 6", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3203,7 +3101,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -3219,7 +3116,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl6 as u where u.id in ::dml_vals", "Table": "u_tbl6" } @@ -3243,7 +3139,6 @@ "Original": "delete u_tbl10 from u_tbl10 join u_tbl11 using (id) where id = 5", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3282,7 +3177,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -3298,7 +3192,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl10 where u_tbl10.id in ::dml_vals", "Table": "u_tbl10" } @@ -3321,7 +3214,6 @@ "Original": "delete u_tbl1 from u_tbl10 join u_tbl1 on u_tbl10.col = u_tbl1.col", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3380,7 +3272,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -3396,7 +3287,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -3410,7 +3300,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl1 where u_tbl1.id in ::dml_vals", "Table": "u_tbl1" } @@ -3435,7 +3324,6 @@ "Original": "delete from u_tbl1 order by id limit 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3494,7 +3382,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -3510,7 +3397,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -3524,7 +3410,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl1 where u_tbl1.id in ::dml_vals", "Table": "u_tbl1" } @@ -3589,7 +3474,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col41 = :__sq1 where col4 = 3", "Table": "u_tbl4" } @@ -3673,7 +3557,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -3689,7 +3572,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = :__sq1 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -3723,7 +3605,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -3739,7 +3620,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "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" } @@ -3753,7 +3633,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl1 set col1 = :__sq1 order by id desc", "Table": "u_tbl1" } @@ -3779,7 +3658,6 @@ "Original": "delete u_tbl6 from u_tbl6 join u_tbl8 on u_tbl6.id = u_tbl8.id where u_tbl6.id = 4", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3818,7 +3696,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -3834,7 +3711,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl6 where u_tbl6.id in ::dml_vals", "Table": "u_tbl6" } @@ -3857,7 +3733,6 @@ "Original": "delete u, m from u_tbl6 u join u_tbl5 m on u.col = m.col where u.col2 = 4 and m.col3 = 6", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -3897,7 +3772,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -3913,7 +3787,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl6 as u where u.id in ::dml_vals", "Table": "u_tbl6" } @@ -3926,7 +3799,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from u_tbl5 as m where m.id in ::dml_vals", "Table": "u_tbl5" } @@ -3948,7 +3820,6 @@ "Original": "update u_tbl2 set col2 = 'bar' limit 2", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -3987,7 +3858,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -4003,7 +3873,6 @@ "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" } @@ -4026,7 +3895,6 @@ "Original": "update u_tbl2 set col2 = id + 1 order by id limit 2", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -4081,7 +3949,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -4104,7 +3971,6 @@ "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" } @@ -4171,7 +4037,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl8 set col8 = 'foo' where (col8) in ::fkc_vals", "Table": "u_tbl8" } @@ -4185,7 +4050,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl6 as u, u_tbl5 as m set u.col6 = 'foo' where u.col2 = 4 and m.col3 = 6 and u.col = m.col", "Table": "u_tbl6" } @@ -4208,7 +4072,6 @@ "Original": "update u_tbl1 u join u_multicol_tbl1 m on u.col = m.col set u.col1 = 'foo', m.cola = 'bar' where u.foo = 4 and m.bar = 6", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]", "1:[1]" @@ -4268,7 +4131,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -4284,7 +4146,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 'foo' where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -4318,7 +4179,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -4334,7 +4194,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast('foo' as CHAR)))", "Table": "u_tbl9" } @@ -4348,7 +4207,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_tbl1 as u set u.col1 = 'foo' where u.id in ::dml_vals", "Table": "u_tbl1" } @@ -4398,7 +4256,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals5", "Cols": [ 0, @@ -4415,7 +4272,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals4 and (cola) not in (('bar'))", "Table": "u_multicol_tbl2" } @@ -4429,7 +4285,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update u_multicol_tbl1 as m set m.cola = 'bar' where m.id in ::dml_vals", "Table": "u_multicol_tbl1" } diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_off_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_off_cases.json index 55d5ee83539..3a5105224d2 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_off_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_off_cases.json @@ -13,7 +13,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into tbl3(col3, coly) values (:_col3_0, 3)", "TableName": "tbl3", "VindexValues": { @@ -39,7 +38,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into multicol_tbl2(cola, colb, colc) values (:_cola_0, :_colb_0, :_colc_0)", "TableName": "multicol_tbl2", "VindexValues": { @@ -65,7 +63,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from tbl1", "Table": "tbl1" }, @@ -88,7 +85,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from tbl7", "Table": "tbl7" }, @@ -133,7 +129,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0, @@ -159,7 +154,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -191,7 +185,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from tbl8 where col8 = 1", "Table": "tbl8", "Values": [ @@ -218,7 +211,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=Off) */ tbl1 set t1col1 = 'foo' where col1 = 1", "Table": "tbl1", "Values": [ @@ -245,7 +237,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=Off) */ tbl7 set t7col7 = 'foo', t7col72 = 42", "Table": "tbl7" }, @@ -284,7 +275,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -300,7 +290,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl5 set t5col5 = 'foo'", "Table": "tbl5" } @@ -326,7 +315,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into tbl6(col6, t6col6) values (:_col6_0, 'foo')", "TableName": "tbl6", "VindexValues": { @@ -352,7 +340,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from tbl20 where col = 'bar'", "Table": "tbl20", "Values": [ @@ -399,7 +386,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -415,7 +401,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from tbl9 where col9 = 34", "Table": "tbl9", "Values": [ @@ -445,7 +430,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -474,7 +458,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=Off) */ u_multicol_tbl1 set cola = 1, colb = 2 where id = 3", "Table": "u_multicol_tbl1" }, @@ -497,7 +480,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into tbl3(col3, coly) values (:_col3_0, 3)", "TableName": "tbl3", "VindexValues": { 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 d9b2f7e8a3e..9d9de507022 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json @@ -18,7 +18,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=On) */ into tbl2(col2, coly) values (:_col2_0, 3)", "TableName": "tbl2", "VindexValues": { @@ -44,7 +43,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=On) */ into multicol_tbl2(cola, colb, colc) values (:_cola_0, :_colb_0, :_colc_0)", "TableName": "multicol_tbl2", "VindexValues": { @@ -102,7 +100,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0, @@ -128,7 +125,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -176,7 +172,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -196,7 +191,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 1 @@ -212,7 +206,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from tbl5", "Table": "tbl5" } @@ -259,7 +252,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -275,7 +267,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from u_tbl9 where col9 = 5", "Table": "u_tbl9" } @@ -301,7 +292,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl5 set col5 = 'foo' where id = 1", "Table": "u_tbl5" }, @@ -340,7 +330,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -356,7 +345,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2 set col2 = 'bar' where id = 1", "Table": "u_tbl2" } @@ -382,7 +370,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2 set col_no_ref = 'baz' where id = 1", "Table": "u_tbl2" }, @@ -410,7 +397,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl1 set not_ref_col = 'foo' where id = 1", "Table": "tbl1" }, @@ -454,7 +440,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -470,7 +455,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl5 set t5col5 = 'foo'", "Table": "tbl5" } @@ -501,7 +485,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl2 set col = 'foo'", "Table": "tbl2" }, @@ -579,7 +562,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl10 set col = 'foo'", "Table": "tbl10" } @@ -630,7 +612,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -646,7 +627,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from tbl9 where col9 = 34", "Table": "tbl9", "Values": [ @@ -712,7 +692,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -728,7 +707,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 'foo' where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -762,7 +740,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -778,7 +755,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast('foo' as CHAR)))", "Table": "u_tbl9" } @@ -792,7 +768,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl1 set col1 = 'foo'", "Table": "u_tbl1" } @@ -816,7 +791,6 @@ "Original": "update u_tbl2 set col2 = 'bar' limit 2", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -855,7 +829,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -871,7 +844,6 @@ "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" } @@ -931,7 +903,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -954,7 +925,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set m = 2, col2 = col1 + 'bar' where id = 1", "Table": "u_tbl2" } @@ -1026,7 +996,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -1042,7 +1011,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = :fkc_upd where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1083,7 +1051,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1099,7 +1066,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (:fkc_upd1 is null or (col9) not in ((:fkc_upd1)))", "Table": "u_tbl9" } @@ -1113,7 +1079,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl1 set m = 2, col1 = x + 'bar' where id = 1", "Table": "u_tbl1" } @@ -1158,7 +1123,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -1174,7 +1138,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2 set m = col1 + 'bar', col2 = 2 where id = 1", "Table": "u_tbl2" } @@ -1236,7 +1199,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -1252,7 +1214,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1286,7 +1247,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1302,7 +1262,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast(2 as CHAR)))", "Table": "u_tbl9" } @@ -1316,7 +1275,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl1 set m = x + 'bar', col1 = 2 where id = 1", "Table": "u_tbl1" } @@ -1345,7 +1303,6 @@ "Original": "delete from u_tbl2 limit 2", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -1384,7 +1341,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -1400,7 +1356,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from u_tbl2 where u_tbl2.id in ::dml_vals", "Table": "u_tbl2" } @@ -1486,7 +1441,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ tbl3 set coly = colx + 10 where coly = 10", "Table": "tbl3" } @@ -1567,7 +1521,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ tbl3 set coly = 20 where coly = 10", "Table": "tbl3" } @@ -1629,7 +1582,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl8 set col8 = 'foo' where (col8) in ::fkc_vals", "Table": "u_tbl8" } @@ -1643,7 +1595,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl6 set col6 = 'foo'", "Table": "u_tbl6" } @@ -1718,7 +1669,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col4 = 'foo' where (col4) in ::fkc_vals", "Table": "u_tbl4" } @@ -1732,7 +1682,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl7 set col7 = 'foo'", "Table": "u_tbl7" } @@ -1808,7 +1757,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl4 set col4 = :v1 where (col4) in ::fkc_vals", "Table": "u_tbl4" } @@ -1822,7 +1770,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl7 set col7 = :v1", "Table": "u_tbl7" } @@ -1845,7 +1792,6 @@ "Original": "insert into u_tbl1 (id, col1) values (1, 3) on duplicate key update col1 = 5", "Instructions": { "OperatorType": "Upsert", - "TargetTabletType": "PRIMARY", "Inputs": [ { "InputName": "Insert-1", @@ -1855,7 +1801,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert /*+ SET_VAR(foreign_key_checks=On) */ into u_tbl1(id, col1) values (1, 3)", "TableName": "u_tbl1" @@ -1904,7 +1849,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -1920,7 +1864,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = 5 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -1954,7 +1897,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals3", "Cols": [ 0 @@ -1970,7 +1912,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (col9) not in ((cast(5 as CHAR)))", "Table": "u_tbl9" } @@ -1984,7 +1925,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl1 set col1 = 5 where id = 1", "Table": "u_tbl1" } @@ -2015,7 +1955,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=On) */ into u_tbl1(id, col1, foo) values (1, 3, 'bar') on duplicate key update foo = 'baz'", "TableName": "u_tbl1" }, @@ -2082,7 +2021,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0 @@ -2098,7 +2036,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=ON) */ from u_tbl2 where (col2) in ::fkc_vals", "Table": "u_tbl2" } @@ -2112,7 +2049,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from u_tbl1 where (id) in ((1))", "Table": "u_tbl1" } @@ -2125,7 +2061,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "NoAutoCommit": true, "Query": "insert /*+ SET_VAR(foreign_key_checks=On) */ into u_tbl1(id, col1) values (1, 2)", "TableName": "u_tbl1" @@ -2190,7 +2125,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0, @@ -2207,7 +2141,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals and (cola, colb) not in ((1, 2))", "Table": "u_multicol_tbl2" } @@ -2221,7 +2154,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_multicol_tbl1 set cola = 1, colb = 2 where id = 3", "Table": "u_multicol_tbl1" } @@ -2285,7 +2217,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 0, @@ -2302,7 +2233,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_multicol_tbl2 set cola = null, colb = null where (cola, colb) in ::fkc_vals and (:v2 is null or (:v1 is null or (cola, colb) not in ((:v1, :v2))))", "Table": "u_multicol_tbl2" } @@ -2316,7 +2246,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_multicol_tbl1 set cola = :v1, colb = :v2 where id = :v3", "Table": "u_multicol_tbl1" } @@ -2365,7 +2294,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals", "Cols": [ 0 @@ -2385,7 +2313,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "BvName": "fkc_vals1", "Cols": [ 1 @@ -2401,7 +2328,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=On) */ from tbl5 where id = :v1", "Table": "tbl5" } @@ -2429,7 +2355,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete /*+ SET_VAR(foreign_key_checks=Off) */ from multicol_tbl1 where cola = 1 and colb = 2 and colc = 3", "Table": "multicol_tbl1", "Values": [ @@ -2458,7 +2383,6 @@ "Name": "unsharded_fk_allow", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update /*+ SET_VAR(foreign_key_checks=Off) */ u_multicol_tbl1 set cola = 1, colb = 2 where id = 3", "Table": "u_multicol_tbl1" }, @@ -2481,7 +2405,6 @@ "Name": "sharded_fk_allow", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert /*+ SET_VAR(foreign_key_checks=Off) */ into tbl3(col3, coly) values (:_col3_0, 3)", "TableName": "tbl3", "VindexValues": { diff --git a/go/vt/vtgate/planbuilder/testdata/mirror_cases.json b/go/vt/vtgate/planbuilder/testdata/mirror_cases.json index ed564b6a61e..1edbec3d1a6 100644 --- a/go/vt/vtgate/planbuilder/testdata/mirror_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/mirror_cases.json @@ -262,7 +262,6 @@ "Name": "unsharded_src1", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into t1(id) values (1)", "TableName": "t1" }, @@ -285,7 +284,6 @@ "Name": "unsharded_src1", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update t1 set `data` = 'a' where id = 1", "Table": "t1" }, @@ -308,7 +306,6 @@ "Name": "unsharded_src1", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from t1 where id = 1", "Table": "t1" }, diff --git a/go/vt/vtgate/planbuilder/testdata/oltp_cases.json b/go/vt/vtgate/planbuilder/testdata/oltp_cases.json index bc64519a4f3..8db0ef3cafb 100644 --- a/go/vt/vtgate/planbuilder/testdata/oltp_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/oltp_cases.json @@ -148,7 +148,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update sbtest6 set k = k + 1 where id = 5", "Table": "sbtest6", "Values": [ @@ -175,7 +174,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update sbtest9 set c = 7 where id = 8", "Table": "sbtest9", "Values": [ @@ -202,7 +200,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from sbtest15 where id = 7525", "Table": "sbtest15", "Values": [ @@ -229,7 +226,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into sbtest16(id, k, c, pad) values (:_id_0, 1, 2, 50)", "TableName": "sbtest16", "VindexValues": { diff --git a/go/vt/vtgate/planbuilder/testdata/reference_cases.json b/go/vt/vtgate/planbuilder/testdata/reference_cases.json index 825ebb00f18..534c41719b0 100644 --- a/go/vt/vtgate/planbuilder/testdata/reference_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/reference_cases.json @@ -165,7 +165,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into ambiguous_ref_with_source(col) values (1)", "TableName": "ambiguous_ref_with_source" }, @@ -235,7 +234,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into ambiguous_ref_with_source(col) values (1)", "TableName": "ambiguous_ref_with_source" }, @@ -259,7 +257,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update ambiguous_ref_with_source set col = 1", "Table": "ambiguous_ref_with_source" }, @@ -283,7 +280,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update ambiguous_ref_with_source set col = 1", "Table": "ambiguous_ref_with_source" }, @@ -307,7 +303,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from ambiguous_ref_with_source where col = 1", "Table": "ambiguous_ref_with_source" }, @@ -331,7 +326,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from ambiguous_ref_with_source where col = 1", "Table": "ambiguous_ref_with_source" }, @@ -455,7 +449,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into global_ref(col) values (1)", "TableName": "global_ref" }, @@ -479,7 +472,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref where col = 1", "Table": "source_of_ref" }, @@ -503,7 +495,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update source_of_ref set x = 4 where col = 1", "Table": "source_of_ref" }, @@ -527,7 +518,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into source_of_ref(x) values (4)", "TableName": "source_of_ref" }, @@ -551,7 +541,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref where col = 1", "Table": "source_of_ref" }, @@ -575,7 +564,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update source_of_ref set x = 4 where col = 1", "Table": "source_of_ref" }, @@ -599,7 +587,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into source_of_ref(x) values (4)", "TableName": "source_of_ref" }, @@ -623,7 +610,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref where col = 1", "Table": "source_of_ref" }, @@ -647,7 +633,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update source_of_ref set x = 4 where col = 1", "Table": "source_of_ref" }, @@ -671,7 +656,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into source_of_ref(x) values (4)", "TableName": "source_of_ref" }, @@ -695,7 +679,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref where col = 1", "Table": "source_of_ref" }, @@ -719,7 +702,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update source_of_ref set x = 4 where col = 1", "Table": "source_of_ref" }, @@ -743,7 +725,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "insert into source_of_ref(x) values (4)", "TableName": "source_of_ref" }, @@ -847,7 +828,6 @@ "Original": "update main.source_of_ref as sr join main.rerouted_ref as rr on sr.id = rr.id inner join user.music as m on sr.col = m.col set sr.tt = 5 where m.user_id = 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -896,7 +876,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "update source_of_ref as sr set sr.tt = 5 where sr.id in ::dml_vals", "Table": "source_of_ref" } @@ -918,7 +897,6 @@ "Original": "delete sr from main.source_of_ref as sr join main.rerouted_ref as rr on sr.id = rr.id inner join user.music as m on sr.col = m.col where m.user_id = 1", "Instructions": { "OperatorType": "DMLWithInput", - "TargetTabletType": "PRIMARY", "Offset": [ "0:[0]" ], @@ -967,7 +945,6 @@ "Name": "main", "Sharded": false }, - "TargetTabletType": "PRIMARY", "Query": "delete from source_of_ref as sr where sr.id in ::dml_vals", "Table": "source_of_ref" } diff --git a/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json b/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json index 95f520330fd..8345b55e936 100644 --- a/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/tpcc_cases.json @@ -68,7 +68,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update district1 set d_next_o_id = 56 where d_id = 9842 and d_w_id = 8546", "Table": "district1", "Values": [ @@ -95,7 +94,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into orders1(o_id, o_d_id, o_w_id, o_c_id, o_entry_d, o_ol_cnt, o_all_local) values (334983, 59896, :_o_w_id_0, 156, now(), 781038, 'hello')", "TableName": "orders1", "VindexValues": { @@ -121,7 +119,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into new_orders1(no_o_id, no_d_id, no_w_id) values (8, 9, :_no_w_id_0)", "TableName": "new_orders1", "VindexValues": { @@ -201,7 +198,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update stock1 set s_quantity = 894 where s_i_id = 156 and s_w_id = 6", "Table": "stock1", "Values": [ @@ -228,7 +224,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into order_line1(ol_o_id, ol_d_id, ol_w_id, ol_number, ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_dist_info) values (648, 36812, :_ol_w_id_0, 4946378, 3, 7, 89, 1, 'info')", "TableName": "order_line1", "VindexValues": { @@ -254,7 +249,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update warehouse1 set w_ytd = w_ytd + 946879 where w_id = 3", "Table": "warehouse1", "Values": [ @@ -308,7 +302,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update district1 set d_ytd = d_ytd + 2 where d_w_id = 89 and d_id = 9", "Table": "district1", "Values": [ @@ -470,7 +463,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update customer1 set c_balance = 508.98, c_ytd_payment = 48941.980301, c_data = 'i am data' where c_w_id = 20 and c_d_id = 387 and c_id = 98", "Table": "customer1", "Values": [ @@ -497,7 +489,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update customer1 set c_balance = 508.98, c_ytd_payment = 48941.980301 where c_w_id = 20 and c_d_id = 387 and c_id = 98", "Table": "customer1", "Values": [ @@ -524,7 +515,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "insert into history1(h_c_d_id, h_c_w_id, h_c_id, h_d_id, h_w_id, h_date, h_amount, h_data) values (6809887, 38748, 8746, 210, :_h_w_id_0, now(), 8907, 'data')", "TableName": "history1", "VindexValues": { @@ -712,7 +702,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from new_orders1 where no_o_id = 2218 and no_d_id = 358 and no_w_id = 98465", "Table": "new_orders1", "Values": [ @@ -766,7 +755,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update orders1 set o_carrier_id = 9 where o_id = 56 and o_d_id = 98 and o_w_id = 897", "Table": "orders1", "Values": [ @@ -793,7 +781,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update order_line1 set ol_delivery_d = now() where ol_o_id = 235 and ol_d_id = 315 and ol_w_id = 8", "Table": "order_line1", "Values": [ @@ -847,7 +834,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "update customer1 set c_balance = c_balance + 988.01, c_delivery_cnt = c_delivery_cnt + 1 where c_id = 6 and c_d_id = 5 and c_w_id = 160", "Table": "customer1", "Values": [ @@ -1010,7 +996,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from order_line1 where ol_w_id = 178 and ol_d_id = 1 and ol_o_id = 84", "Table": "order_line1", "Values": [ @@ -1037,7 +1022,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from orders1 where o_w_id = 1 and o_d_id = 3 and o_id = 384", "Table": "orders1", "Values": [ @@ -1064,7 +1048,6 @@ "Name": "main", "Sharded": true }, - "TargetTabletType": "PRIMARY", "Query": "delete from history1 where h_w_id = 75 and h_d_id = 102 limit 10", "Table": "history1", "Values": [ diff --git a/go/vt/vtgate/planbuilder/vexplain.go b/go/vt/vtgate/planbuilder/vexplain.go index 061fa85473d..89f4bd2403c 100644 --- a/go/vt/vtgate/planbuilder/vexplain.go +++ b/go/vt/vtgate/planbuilder/vexplain.go @@ -55,7 +55,7 @@ func buildVExplainPlan( func explainTabPlan(explain *sqlparser.ExplainTab, vschema plancontext.VSchema) (*planResult, error) { var keyspace *vindexes.Keyspace - var dest key.Destination + var dest key.ShardDestination if sqlparser.SystemSchema(explain.Table.Qualifier.String()) { var err error diff --git a/go/vt/vtgate/plugin_mysql_server.go b/go/vt/vtgate/plugin_mysql_server.go index dec7794c9bd..512a278e0ff 100644 --- a/go/vt/vtgate/plugin_mysql_server.go +++ b/go/vt/vtgate/plugin_mysql_server.go @@ -258,7 +258,7 @@ func (vh *vtgateHandler) ComQuery(c *mysql.Conn, query string, callback func(*sq fillInTxStatusFlags(c, session) return nil } - session, result, err := vh.vtg.Execute(ctx, vh, session, query, make(map[string]*querypb.BindVariable)) + session, result, err := vh.vtg.Execute(ctx, vh, session, query, make(map[string]*querypb.BindVariable), false) if err := sqlerror.NewSQLErrorFromError(err); err != nil { return err @@ -281,7 +281,7 @@ func fillInTxStatusFlags(c *mysql.Conn, session *vtgatepb.Session) { } // ComPrepare is the handler for command prepare. -func (vh *vtgateHandler) ComPrepare(c *mysql.Conn, query string, bindVars map[string]*querypb.BindVariable) ([]*querypb.Field, error) { +func (vh *vtgateHandler) ComPrepare(c *mysql.Conn, query string) ([]*querypb.Field, uint16, error) { var ctx context.Context var cancel context.CancelFunc if mysqlQueryTimeout != 0 { @@ -315,12 +315,12 @@ func (vh *vtgateHandler) ComPrepare(c *mysql.Conn, query string, bindVars map[st } }() - session, fld, err := vh.vtg.Prepare(ctx, session, query, bindVars) + session, fld, paramsCount, err := vh.vtg.Prepare(ctx, session, query) err = sqlerror.NewSQLErrorFromError(err) if err != nil { - return nil, err + return nil, 0, err } - return fld, nil + return fld, paramsCount, nil } func (vh *vtgateHandler) ComStmtExecute(c *mysql.Conn, prepare *mysql.PrepareData, callback func(*sqltypes.Result) error) error { @@ -364,7 +364,7 @@ func (vh *vtgateHandler) ComStmtExecute(c *mysql.Conn, prepare *mysql.PrepareDat fillInTxStatusFlags(c, session) return nil } - _, qr, err := vh.vtg.Execute(ctx, vh, session, prepare.PrepareStmt, prepare.BindVars) + _, qr, err := vh.vtg.Execute(ctx, vh, session, prepare.PrepareStmt, prepare.BindVars, true) if err != nil { return sqlerror.NewSQLErrorFromError(err) } diff --git a/go/vt/vtgate/plugin_mysql_server_test.go b/go/vt/vtgate/plugin_mysql_server_test.go index ceb4cea0d42..73636f3463b 100644 --- a/go/vt/vtgate/plugin_mysql_server_test.go +++ b/go/vt/vtgate/plugin_mysql_server_test.go @@ -58,8 +58,8 @@ func (th *testHandler) ComQuery(c *mysql.Conn, q string, callback func(*sqltypes return callback(&sqltypes.Result{Fields: []*querypb.Field{}, Rows: [][]sqltypes.Value{}}) } -func (th *testHandler) ComPrepare(c *mysql.Conn, q string, b map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - return nil, nil +func (th *testHandler) ComPrepare(*mysql.Conn, string) ([]*querypb.Field, uint16, error) { + return nil, 0, nil } func (th *testHandler) ComStmtExecute(c *mysql.Conn, prepare *mysql.PrepareData, callback func(*sqltypes.Result) error) error { diff --git a/go/vt/vtgate/resolver.go b/go/vt/vtgate/resolver.go index 780d028e0c6..195b1f36cfa 100644 --- a/go/vt/vtgate/resolver.go +++ b/go/vt/vtgate/resolver.go @@ -48,7 +48,7 @@ func NewResolver(resolver *srvtopo.Resolver, serv srvtopo.Server, cell string, s // MessageStream streams messages. func (res *Resolver) MessageStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, name string, callback func(*sqltypes.Result) error) error { - var destination key.Destination + var destination key.ShardDestination if shard != "" { // If we pass in a shard, resolve the keyspace/shard // following redirects. diff --git a/go/vt/vtgate/scatter_conn_test.go b/go/vt/vtgate/scatter_conn_test.go index e5c27c0de33..c09aa84b09d 100644 --- a/go/vt/vtgate/scatter_conn_test.go +++ b/go/vt/vtgate/scatter_conn_test.go @@ -308,7 +308,7 @@ func TestReservedOnMultiReplica(t *testing.T) { res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa") session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true}) - destinations := []key.Destination{key.DestinationShard("0")} + destinations := []key.ShardDestination{key.DestinationShard("0")} for i := 0; i < 10; i++ { executeOnShards(t, ctx, res, keyspace, sc, session, destinations) assert.EqualValues(t, 1, sbc0_1.ReserveCount.Load()+sbc0_2.ReserveCount.Load(), "sbc0 reserve count") @@ -458,7 +458,7 @@ func TestReservedBeginTableDriven(t *testing.T) { for _, action := range test.actions { session.Session.InTransaction = action.transaction session.Session.InReservedConn = action.reserved - var destinations []key.Destination + var destinations []key.ShardDestination for _, shard := range action.shards { destinations = append(destinations, key.DestinationShard(shard)) } @@ -488,7 +488,7 @@ func TestReservedConnFail(t *testing.T) { res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa") session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true}) - destinations := []key.Destination{key.DestinationShard("0")} + destinations := []key.ShardDestination{key.DestinationShard("0")} executeOnShards(t, ctx, res, keyspace, sc, session, destinations) assert.Equal(t, 1, len(session.ShardSessions)) diff --git a/go/vt/vtgate/semantics/FakeSI.go b/go/vt/vtgate/semantics/FakeSI.go index 8e24a029002..9dfab741584 100644 --- a/go/vt/vtgate/semantics/FakeSI.go +++ b/go/vt/vtgate/semantics/FakeSI.go @@ -40,7 +40,7 @@ type FakeSI struct { } // FindTableOrVindex implements the SchemaInformation interface -func (s *FakeSI) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (s *FakeSI) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { table, ok := s.Tables[sqlparser.String(tablename)] if ok { return table, nil, "", 0, nil, nil diff --git a/go/vt/vtgate/semantics/info_schema.go b/go/vt/vtgate/semantics/info_schema.go index c15056ce033..24736fa6833 100644 --- a/go/vt/vtgate/semantics/info_schema.go +++ b/go/vt/vtgate/semantics/info_schema.go @@ -1627,7 +1627,7 @@ func loadSchemaInfo(version string) map[string][]vindexes.Column { } // FindTableOrVindex implements the SchemaInformation interface -func (i *infoSchemaWithColumns) FindTableOrVindex(tbl sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (i *infoSchemaWithColumns) FindTableOrVindex(tbl sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) { if !strings.EqualFold(tbl.Qualifier.String(), "information_schema") { return i.inner.FindTableOrVindex(tbl) } diff --git a/go/vt/vtgate/semantics/semantic_table.go b/go/vt/vtgate/semantics/semantic_table.go index c8e4ca35350..9afd8bf6406 100644 --- a/go/vt/vtgate/semantics/semantic_table.go +++ b/go/vt/vtgate/semantics/semantic_table.go @@ -165,7 +165,7 @@ type ( // SchemaInformation is used to provide table information from Vschema. SchemaInformation interface { - FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) + FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.BaseTable, vindexes.Vindex, string, topodatapb.TabletType, key.ShardDestination, error) ConnCollation() collations.ID Environment() *vtenv.Environment // ForeignKeyMode returns the foreign_key flag value diff --git a/go/vt/vtgate/semantics/typer_test.go b/go/vt/vtgate/semantics/typer_test.go index bfb8cb1119e..608d62d5e32 100644 --- a/go/vt/vtgate/semantics/typer_test.go +++ b/go/vt/vtgate/semantics/typer_test.go @@ -45,7 +45,7 @@ func TestNormalizerAndSemanticAnalysisIntegration(t *testing.T) { require.NoError(t, err) rv := sqlparser.NewReservedVars("", known) - out, err := sqlparser.PrepareAST(parse, rv, map[string]*querypb.BindVariable{}, true, "d", 0, "", map[string]string{}, nil, nil) + out, err := sqlparser.Normalize(parse, rv, map[string]*querypb.BindVariable{}, true, "d", 0, "", map[string]string{}, nil, nil) require.NoError(t, err) st, err := Analyze(out.AST, "d", fakeSchemaInfo()) @@ -73,7 +73,7 @@ func TestColumnCollations(t *testing.T) { ast, err := sqlparser.NewTestParser().Parse(test.query) require.NoError(t, err) - out, err := sqlparser.PrepareAST(ast, sqlparser.NewReservedVars("bv", sqlparser.BindVars{}), map[string]*querypb.BindVariable{}, true, "d", 0, "", map[string]string{}, nil, nil) + out, err := sqlparser.Normalize(ast, sqlparser.NewReservedVars("bv", sqlparser.BindVars{}), map[string]*querypb.BindVariable{}, true, "d", 0, "", map[string]string{}, nil, nil) require.NoError(t, err) st, err := Analyze(out.AST, "d", fakeSchemaInfo()) diff --git a/go/vt/vtgate/vindexes/binary.go b/go/vt/vtgate/vindexes/binary.go index 96a72b2c3f4..d664b6e4464 100644 --- a/go/vt/vtgate/vindexes/binary.go +++ b/go/vt/vtgate/vindexes/binary.go @@ -80,9 +80,9 @@ func (vind *Binary) Verify(ctx context.Context, vcursor VCursor, ids []sqltypes. return out, nil } -// Map can map ids to key.Destination objects. -func (vind *Binary) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *Binary) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { idBytes, err := vind.Hash(id) if err != nil { @@ -109,8 +109,8 @@ func (*Binary) ReverseMap(_ VCursor, ksids [][]byte) ([]sqltypes.Value, error) { return reverseIds, nil } -// RangeMap can map ids to key.Destination objects. -func (vind *Binary) RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.Destination, error) { +// RangeMap can map ids to key.ShardDestination objects. +func (vind *Binary) RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.ShardDestination, error) { startKsId, err := vind.Hash(startId) if err != nil { return nil, err @@ -119,7 +119,7 @@ func (vind *Binary) RangeMap(ctx context.Context, vcursor VCursor, startId sqlty if err != nil { return nil, err } - out := []key.Destination{&key.DestinationKeyRange{KeyRange: key.NewKeyRange(startKsId, endKsId)}} + out := []key.ShardDestination{&key.DestinationKeyRange{KeyRange: key.NewKeyRange(startKsId, endKsId)}} return out, nil } diff --git a/go/vt/vtgate/vindexes/binarymd5.go b/go/vt/vtgate/vindexes/binarymd5.go index d3495e28deb..94d5804dea2 100644 --- a/go/vt/vtgate/vindexes/binarymd5.go +++ b/go/vt/vtgate/vindexes/binarymd5.go @@ -78,9 +78,9 @@ func (vind *BinaryMD5) Verify(ctx context.Context, vcursor VCursor, ids []sqltyp return out, nil } -// Map can map ids to key.Destination objects. -func (vind *BinaryMD5) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *BinaryMD5) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { ksid, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/cfc.go b/go/vt/vtgate/vindexes/cfc.go index af269b1a0d9..26bb5231697 100644 --- a/go/vt/vtgate/vindexes/cfc.go +++ b/go/vt/vtgate/vindexes/cfc.go @@ -257,9 +257,9 @@ func (vind *CFC) Verify(_ context.Context, _ VCursor, ids []sqltypes.Value, ksid return vind.verify(ids, ksids) } -// Map can map ids to key.Destination objects. -func (vind *CFC) Map(_ context.Context, _ VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *CFC) Map(_ context.Context, _ VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, len(ids)) for i, id := range ids { idBytes, err := id.ToBytes() if err != nil { @@ -280,7 +280,7 @@ func (vind *CFC) PrefixVindex() SingleColumn { } // NewKeyRangeFromPrefix creates a keyspace range from a prefix of keyspace id. -func NewKeyRangeFromPrefix(begin []byte) key.Destination { +func NewKeyRangeFromPrefix(begin []byte) key.ShardDestination { if len(begin) == 0 { return key.DestinationAllShards{} } @@ -345,9 +345,9 @@ func (vind *prefixCFC) IsUnique() bool { return false } -// Map can map ids to key.Destination objects. -func (vind *prefixCFC) Map(_ context.Context, _ VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *prefixCFC) Map(_ context.Context, _ VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, len(ids)) for i, id := range ids { value, err := id.ToBytes() if err != nil { diff --git a/go/vt/vtgate/vindexes/cfc_test.go b/go/vt/vtgate/vindexes/cfc_test.go index aaf639adec6..bc789dd96f3 100644 --- a/go/vt/vtgate/vindexes/cfc_test.go +++ b/go/vt/vtgate/vindexes/cfc_test.go @@ -405,7 +405,7 @@ func TestCFCPrefixMap(t *testing.T) { cases := []struct { testName string id string - dest key.Destination + dest key.ShardDestination }{ { testName: "literal regular", @@ -540,7 +540,7 @@ func TestCFCFindPrefixEscape(t *testing.T) { func TestDestinationKeyRangeFromPrefix(t *testing.T) { testCases := []struct { start []byte - dest key.Destination + dest key.ShardDestination }{ { start: []byte{3, 123, 255}, diff --git a/go/vt/vtgate/vindexes/consistent_lookup.go b/go/vt/vtgate/vindexes/consistent_lookup.go index d231f358a37..637368d2af3 100644 --- a/go/vt/vtgate/vindexes/consistent_lookup.go +++ b/go/vt/vtgate/vindexes/consistent_lookup.go @@ -101,9 +101,9 @@ func (lu *ConsistentLookup) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lu *ConsistentLookup) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (lu *ConsistentLookup) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -127,8 +127,8 @@ func (lu *ConsistentLookup) Map(ctx context.Context, vcursor VCursor, ids []sqlt } // MapResult implements the LookupPlanable interface -func (lu *ConsistentLookup) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (lu *ConsistentLookup) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -214,9 +214,9 @@ func (lu *ConsistentLookupUnique) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lu *ConsistentLookupUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (lu *ConsistentLookupUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -232,8 +232,8 @@ func (lu *ConsistentLookupUnique) Map(ctx context.Context, vcursor VCursor, ids } // MapResult implements the LookupPlanable interface -func (lu *ConsistentLookupUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (lu *ConsistentLookupUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) diff --git a/go/vt/vtgate/vindexes/consistent_lookup_test.go b/go/vt/vtgate/vindexes/consistent_lookup_test.go index 0279ecaba78..410783512e3 100644 --- a/go/vt/vtgate/vindexes/consistent_lookup_test.go +++ b/go/vt/vtgate/vindexes/consistent_lookup_test.go @@ -121,7 +121,7 @@ func TestConsistentLookupMap(t *testing.T) { got, err := lookup.Map(ctx, vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("1"), []byte("2"), @@ -153,7 +153,7 @@ func TestConsistentLookupMapWriteOnly(t *testing.T) { got, err := lookup.Map(context.Background(), nil, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, @@ -174,7 +174,7 @@ func TestConsistentLookupUniqueMap(t *testing.T) { got, err := lookup.Map(ctx, vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationNone{}, key.DestinationKeyspaceID([]byte("1")), } @@ -200,7 +200,7 @@ func TestConsistentLookupUniqueMapWriteOnly(t *testing.T) { got, err := lookup.Map(context.Background(), nil, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, @@ -221,7 +221,7 @@ func TestConsistentLookupMapAbsent(t *testing.T) { got, err := lookup.Map(ctx, vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } diff --git a/go/vt/vtgate/vindexes/hash.go b/go/vt/vtgate/vindexes/hash.go index d30895be48a..97ebcb93de4 100644 --- a/go/vt/vtgate/vindexes/hash.go +++ b/go/vt/vtgate/vindexes/hash.go @@ -75,9 +75,9 @@ func (vind *Hash) NeedsVCursor() bool { return false } -// Map can map ids to key.Destination objects. -func (vind *Hash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *Hash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, len(ids)) for i, id := range ids { ksid, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/hash_test.go b/go/vt/vtgate/vindexes/hash_test.go index 749457f71f9..cec01edf5de 100644 --- a/go/vt/vtgate/vindexes/hash_test.go +++ b/go/vt/vtgate/vindexes/hash_test.go @@ -105,7 +105,7 @@ func TestHashMap(t *testing.T) { sqltypes.NewInt64(-9223372036854775808), // - 2^63 }) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x16k@\xb4J\xbaK\xd6")), key.DestinationKeyspaceID([]byte("\x06\xe7\xea\"Βp\x8f")), key.DestinationKeyspaceID([]byte("N\xb1\x90ɢ\xfa\x16\x9c")), diff --git a/go/vt/vtgate/vindexes/lookup.go b/go/vt/vtgate/vindexes/lookup.go index 33462470010..392bbaa91c5 100644 --- a/go/vt/vtgate/vindexes/lookup.go +++ b/go/vt/vtgate/vindexes/lookup.go @@ -96,9 +96,9 @@ func (ln *LookupNonUnique) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (ln *LookupNonUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (ln *LookupNonUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if ln.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -123,8 +123,8 @@ func (ln *LookupNonUnique) Map(ctx context.Context, vcursor VCursor, ids []sqlty } // MapResult implements the LookupPlanable interface -func (ln *LookupNonUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (ln *LookupNonUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if ln.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -244,7 +244,7 @@ func ksidsToValues(ksids [][]byte) []sqltypes.Value { return values } -//==================================================================== +// ==================================================================== // LookupUnique defines a vindex that uses a lookup table. // The table is expected to define the id column as unique. It's @@ -327,10 +327,10 @@ func (lu *LookupUnique) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lu *LookupUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +// Map can map ids to key.ShardDestination objects. +func (lu *LookupUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { if lu.writeOnly { - out := make([]key.Destination, 0, len(ids)) + out := make([]key.ShardDestination, 0, len(ids)) for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) } @@ -343,8 +343,8 @@ func (lu *LookupUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes return lu.MapResult(ids, results) } -func (lu *LookupUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (lu *LookupUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for i, result := range results { switch len(result.Rows) { case 0: diff --git a/go/vt/vtgate/vindexes/lookup_hash.go b/go/vt/vtgate/vindexes/lookup_hash.go index 28f38942afa..5ba74e04a32 100644 --- a/go/vt/vtgate/vindexes/lookup_hash.go +++ b/go/vt/vtgate/vindexes/lookup_hash.go @@ -120,9 +120,9 @@ func (lh *LookupHash) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lh *LookupHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (lh *LookupHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lh.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -146,8 +146,8 @@ func (lh *LookupHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.V } // MapResult implements the LookupPlanable interface -func (lh *LookupHash) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (lh *LookupHash) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lh.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -329,10 +329,10 @@ func (lhu *LookupHashUnique) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lhu *LookupHashUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +// Map can map ids to key.ShardDestination objects. +func (lhu *LookupHashUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { if lhu.writeOnly { - out := make([]key.Destination, 0, len(ids)) + out := make([]key.ShardDestination, 0, len(ids)) for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) } @@ -346,8 +346,8 @@ func (lhu *LookupHashUnique) Map(ctx context.Context, vcursor VCursor, ids []sql return lhu.MapResult(ids, results) } -func (lhu *LookupHashUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (lhu *LookupHashUnique) MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lhu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) diff --git a/go/vt/vtgate/vindexes/lookup_hash_test.go b/go/vt/vtgate/vindexes/lookup_hash_test.go index fd07f6ab7d8..b4722719763 100644 --- a/go/vt/vtgate/vindexes/lookup_hash_test.go +++ b/go/vt/vtgate/vindexes/lookup_hash_test.go @@ -87,7 +87,7 @@ func TestLookupHashMap(t *testing.T) { got, err := lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("\x16k@\xb4J\xbaK\xd6"), []byte("\x06\xe7\xea\"Βp\x8f"), @@ -108,7 +108,7 @@ func TestLookupHashMap(t *testing.T) { ) got, err = lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1)}) require.NoError(t, err) - want = []key.Destination{key.DestinationKeyspaceIDs([][]byte{})} + want = []key.ShardDestination{key.DestinationKeyspaceIDs([][]byte{})} if !reflect.DeepEqual(got, want) { t.Errorf("Map(): %#v, want %#v", got, want) } @@ -129,7 +129,7 @@ func TestLookupHashMapAbsent(t *testing.T) { got, err := lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } @@ -141,7 +141,7 @@ func TestLookupHashMapAbsent(t *testing.T) { lookuphash = createLookup(t, "lookup_hash", true) got, err = lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, @@ -160,7 +160,7 @@ func TestLookupHashMapNull(t *testing.T) { got, err := lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NULL}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("\x16k@\xb4J\xbaK\xd6"), }), @@ -173,7 +173,7 @@ func TestLookupHashMapNull(t *testing.T) { lookuphash = createLookup(t, "lookup_hash", true) got, err = lookuphash.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NULL}) require.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, diff --git a/go/vt/vtgate/vindexes/lookup_hash_unique_test.go b/go/vt/vtgate/vindexes/lookup_hash_unique_test.go index 67697fb5eac..9929609889a 100644 --- a/go/vt/vtgate/vindexes/lookup_hash_unique_test.go +++ b/go/vt/vtgate/vindexes/lookup_hash_unique_test.go @@ -97,7 +97,7 @@ func TestLookupHashUniqueMap(t *testing.T) { got, err := lhu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x16k@\xb4J\xbaK\xd6")), key.DestinationNone{}, } @@ -108,7 +108,7 @@ func TestLookupHashUniqueMap(t *testing.T) { vc.numRows = 0 got, err = lhu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } @@ -130,7 +130,7 @@ func TestLookupHashUniqueMap(t *testing.T) { ) got, err = lhu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1)}) require.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationNone{}, } if !reflect.DeepEqual(got, want) { @@ -153,7 +153,7 @@ func TestLookupHashUniqueMapWriteOnly(t *testing.T) { got, err := lhu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, diff --git a/go/vt/vtgate/vindexes/lookup_test.go b/go/vt/vtgate/vindexes/lookup_test.go index 8041a395a8e..e0e89c30937 100644 --- a/go/vt/vtgate/vindexes/lookup_test.go +++ b/go/vt/vtgate/vindexes/lookup_test.go @@ -383,7 +383,7 @@ func TestLookupNonUniqueMap(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("1"), []byte("2"), @@ -425,7 +425,7 @@ func TestLookupNonUniqueMapAutocommit(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("1"), []byte("2"), @@ -455,7 +455,7 @@ func TestLookupNonUniqueMapWriteOnly(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, @@ -472,7 +472,7 @@ func TestLookupNonUniqueMapAbsent(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } @@ -774,7 +774,7 @@ func TestLookupMapResult(t *testing.T) { got, err := lookup.(LookupPlanable).MapResult(ids, results) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("1"), []byte("3"), @@ -798,7 +798,7 @@ func TestLookupUniqueMapResult(t *testing.T) { got, err := lookup.(LookupPlanable).MapResult(ids, results) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID("1"), } utils.MustMatch(t, want, got) diff --git a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash.go b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash.go index f7af93187da..a6ba96a6127 100644 --- a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash.go +++ b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash.go @@ -51,7 +51,7 @@ func init() { Register("lookup_unicodeloosemd5_hash_unique", newLookupUnicodeLooseMD5HashUnique) } -//==================================================================== +// ==================================================================== // LookupUnicodeLooseMD5Hash defines a vindex that uses a lookup table. // The table is expected to define the id column as unique. It's @@ -117,9 +117,9 @@ func (lh *LookupUnicodeLooseMD5Hash) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lh *LookupUnicodeLooseMD5Hash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (lh *LookupUnicodeLooseMD5Hash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lh.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) @@ -241,7 +241,7 @@ func (lh *LookupUnicodeLooseMD5Hash) UnknownParams() []string { return lh.unknownParams } -//==================================================================== +// ==================================================================== // LookupUnicodeLooseMD5HashUnique defines a vindex that uses a lookup table. // The table is expected to define the id column as unique. It's @@ -307,9 +307,9 @@ func (lhu *LookupUnicodeLooseMD5HashUnique) NeedsVCursor() bool { return true } -// Map can map ids to key.Destination objects. -func (lhu *LookupUnicodeLooseMD5HashUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (lhu *LookupUnicodeLooseMD5HashUnique) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) if lhu.writeOnly { for range ids { out = append(out, key.DestinationKeyRange{KeyRange: &topodatapb.KeyRange{}}) diff --git a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go index c0e4611d684..a24f4c71ead 100644 --- a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go +++ b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go @@ -69,7 +69,7 @@ func TestLookupUnicodeLooseMD5HashMap(t *testing.T) { got, err := lookup.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(10), sqltypes.NewInt64(20)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("\x16k@\xb4J\xbaK\xd6"), []byte("\x06\xe7\xea\"Βp\x8f"), @@ -122,7 +122,7 @@ func TestLookupUnicodeLooseMD5HashMapAutocommit(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(10), sqltypes.NewInt64(20)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceIDs([][]byte{ []byte("\x16k@\xb4J\xbaK\xd6"), []byte("\x06\xe7\xea\"Βp\x8f"), @@ -159,7 +159,7 @@ func TestLookupUnicodeLooseMD5HashMapWriteOnly(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(10), sqltypes.NewInt64(20)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, @@ -178,7 +178,7 @@ func TestLookupUnicodeLooseMD5HashMapAbsent(t *testing.T) { got, err := lnu.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(10), sqltypes.NewInt64(20)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } diff --git a/go/vt/vtgate/vindexes/lookup_unique_test.go b/go/vt/vtgate/vindexes/lookup_unique_test.go index fd2a62c4d21..ab3119c8ee2 100644 --- a/go/vt/vtgate/vindexes/lookup_unique_test.go +++ b/go/vt/vtgate/vindexes/lookup_unique_test.go @@ -67,7 +67,7 @@ func TestLookupUniqueMap(t *testing.T) { got, err := lookupUnique.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("1")), key.DestinationNone{}, } @@ -78,7 +78,7 @@ func TestLookupUniqueMap(t *testing.T) { vc.numRows = 0 got, err = lookupUnique.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationNone{}, key.DestinationNone{}, } @@ -109,7 +109,7 @@ func TestLookupUniqueMapWriteOnly(t *testing.T) { got, err := lookupUnique.Map(context.Background(), vc, []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyRange{ KeyRange: &topodatapb.KeyRange{}, }, diff --git a/go/vt/vtgate/vindexes/multicol.go b/go/vt/vtgate/vindexes/multicol.go index ee53ea5bb60..7eb532ca797 100644 --- a/go/vt/vtgate/vindexes/multicol.go +++ b/go/vt/vtgate/vindexes/multicol.go @@ -88,8 +88,8 @@ func (m *MultiCol) NeedsVCursor() bool { return false } -func (m *MultiCol) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(rowsColValues)) +func (m *MultiCol) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(rowsColValues)) for _, colValues := range rowsColValues { partial, ksid, err := m.mapKsid(colValues) if err != nil { diff --git a/go/vt/vtgate/vindexes/multicol_test.go b/go/vt/vtgate/vindexes/multicol_test.go index e4e2098dd1b..9e38ad9811a 100644 --- a/go/vt/vtgate/vindexes/multicol_test.go +++ b/go/vt/vtgate/vindexes/multicol_test.go @@ -242,7 +242,7 @@ func TestMultiColMap(t *testing.T) { }}) assert.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID("\x16\x6b\x40\x16\x6b\x40\x16\x6b"), key.DestinationKeyspaceID("\x25\x4e\x88\x16\x6b\x40\x16\x6b"), key.DestinationKeyspaceID("\xdd\x7c\x0b\x16\x6b\x40\x16\x6b"), diff --git a/go/vt/vtgate/vindexes/null.go b/go/vt/vtgate/vindexes/null.go index 58435643ea7..3b115e847d7 100644 --- a/go/vt/vtgate/vindexes/null.go +++ b/go/vt/vtgate/vindexes/null.go @@ -70,9 +70,9 @@ func (vind *Null) NeedsVCursor() bool { return false } -// Map can map ids to key.Destination objects. -func (vind *Null) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *Null) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for i := 0; i < len(ids); i++ { out = append(out, key.DestinationKeyspaceID(nullksid)) } diff --git a/go/vt/vtgate/vindexes/null_test.go b/go/vt/vtgate/vindexes/null_test.go index 03b97fe651b..090a91d5599 100644 --- a/go/vt/vtgate/vindexes/null_test.go +++ b/go/vt/vtgate/vindexes/null_test.go @@ -100,7 +100,7 @@ func TestNullMap(t *testing.T) { sqltypes.NULL, }) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte{0}), key.DestinationKeyspaceID([]byte{0}), key.DestinationKeyspaceID([]byte{0}), diff --git a/go/vt/vtgate/vindexes/numeric.go b/go/vt/vtgate/vindexes/numeric.go index b40df13a997..c801a671bc3 100644 --- a/go/vt/vtgate/vindexes/numeric.go +++ b/go/vt/vtgate/vindexes/numeric.go @@ -82,9 +82,9 @@ func (vind *Numeric) Verify(ctx context.Context, vcursor VCursor, ids []sqltypes return out, nil } -// Map can map ids to key.Destination objects. -func (vind *Numeric) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *Numeric) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { ksid, err := vind.Hash(id) if err != nil { @@ -110,7 +110,7 @@ func (*Numeric) ReverseMap(_ VCursor, ksids [][]byte) ([]sqltypes.Value, error) } // RangeMap implements Between. -func (vind *Numeric) RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.Destination, error) { +func (vind *Numeric) RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.ShardDestination, error) { startKsId, err := vind.Hash(startId) if err != nil { return nil, err @@ -119,7 +119,7 @@ func (vind *Numeric) RangeMap(ctx context.Context, vcursor VCursor, startId sqlt if err != nil { return nil, err } - out := []key.Destination{&key.DestinationKeyRange{KeyRange: key.NewKeyRange(startKsId, endKsId)}} + out := []key.ShardDestination{&key.DestinationKeyRange{KeyRange: key.NewKeyRange(startKsId, endKsId)}} return out, nil } diff --git a/go/vt/vtgate/vindexes/numeric_static_map.go b/go/vt/vtgate/vindexes/numeric_static_map.go index f97016d915f..2313bf1e301 100644 --- a/go/vt/vtgate/vindexes/numeric_static_map.go +++ b/go/vt/vtgate/vindexes/numeric_static_map.go @@ -145,9 +145,9 @@ func (vind *NumericStaticMap) Verify(ctx context.Context, vcursor VCursor, ids [ return out, nil } -// Map can map ids to key.Destination objects. -func (vind *NumericStaticMap) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *NumericStaticMap) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { ksid, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/numeric_static_map_test.go b/go/vt/vtgate/vindexes/numeric_static_map_test.go index 7a373060f16..6d262c89a3b 100644 --- a/go/vt/vtgate/vindexes/numeric_static_map_test.go +++ b/go/vt/vtgate/vindexes/numeric_static_map_test.go @@ -165,7 +165,7 @@ func TestNumericStaticMapMap(t *testing.T) { // in the third slice, we expect 2 instead of 3 as numeric_static_map_test.json // has 3 mapped to 2 - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x01")), key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x02")), key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x02")), @@ -271,7 +271,7 @@ func TestNumericStaticMapWithFallback(t *testing.T) { }) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x02")), key.DestinationKeyspaceID([]byte("\x8b\x59\x80\x16\x62\xb5\x21\x60")), key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x04")), diff --git a/go/vt/vtgate/vindexes/numeric_test.go b/go/vt/vtgate/vindexes/numeric_test.go index 612c0f3c5e7..65e58b54323 100644 --- a/go/vt/vtgate/vindexes/numeric_test.go +++ b/go/vt/vtgate/vindexes/numeric_test.go @@ -99,7 +99,7 @@ func TestNumericMap(t *testing.T) { sqltypes.NULL, }) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x01")), key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x02")), key.DestinationKeyspaceID([]byte("\x00\x00\x00\x00\x00\x00\x00\x03")), diff --git a/go/vt/vtgate/vindexes/region_experimental.go b/go/vt/vtgate/vindexes/region_experimental.go index c6b867647aa..c757b689c77 100644 --- a/go/vt/vtgate/vindexes/region_experimental.go +++ b/go/vt/vtgate/vindexes/region_experimental.go @@ -99,8 +99,8 @@ func (ge *RegionExperimental) NeedsVCursor() bool { } // Map satisfies MultiColumn. -func (ge *RegionExperimental) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { - destinations := make([]key.Destination, 0, len(rowsColValues)) +func (ge *RegionExperimental) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { + destinations := make([]key.ShardDestination, 0, len(rowsColValues)) for _, row := range rowsColValues { if len(row) > 2 { destinations = append(destinations, key.DestinationNone{}) diff --git a/go/vt/vtgate/vindexes/region_experimental_test.go b/go/vt/vtgate/vindexes/region_experimental_test.go index 56b16b8f3ee..8fd521b1e48 100644 --- a/go/vt/vtgate/vindexes/region_experimental_test.go +++ b/go/vt/vtgate/vindexes/region_experimental_test.go @@ -145,7 +145,7 @@ func TestRegionExperimentalMap(t *testing.T) { }}) assert.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x01\x16k@\xb4J\xbaK\xd6")), key.DestinationKeyspaceID([]byte("\xff\x16k@\xb4J\xbaK\xd6")), key.DestinationKeyspaceID([]byte("\x00\x16k@\xb4J\xbaK\xd6")), @@ -171,7 +171,7 @@ func TestRegionExperimentalMapMulti2(t *testing.T) { }}) assert.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x00\x01\x16k@\xb4J\xbaK\xd6")), key.DestinationKeyspaceID([]byte("\x00\xff\x16k@\xb4J\xbaK\xd6")), key.DestinationKeyspaceID([]byte("\x01\x00\x16k@\xb4J\xbaK\xd6")), diff --git a/go/vt/vtgate/vindexes/region_json.go b/go/vt/vtgate/vindexes/region_json.go index f0ac2ef18fa..1fa91f38411 100644 --- a/go/vt/vtgate/vindexes/region_json.go +++ b/go/vt/vtgate/vindexes/region_json.go @@ -118,8 +118,8 @@ func (rv *RegionJSON) NeedsVCursor() bool { } // Map satisfies MultiColumn. -func (rv *RegionJSON) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { - destinations := make([]key.Destination, 0, len(rowsColValues)) +func (rv *RegionJSON) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { + destinations := make([]key.ShardDestination, 0, len(rowsColValues)) for _, row := range rowsColValues { if len(row) != 2 { destinations = append(destinations, key.DestinationNone{}) diff --git a/go/vt/vtgate/vindexes/reverse_bits.go b/go/vt/vtgate/vindexes/reverse_bits.go index 80c72ca6924..7900b05f3f6 100644 --- a/go/vt/vtgate/vindexes/reverse_bits.go +++ b/go/vt/vtgate/vindexes/reverse_bits.go @@ -70,8 +70,8 @@ func (vind *ReverseBits) NeedsVCursor() bool { } // Map returns the corresponding KeyspaceId values for the given ids. -func (vind *ReverseBits) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +func (vind *ReverseBits) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { num, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/reverse_bits_test.go b/go/vt/vtgate/vindexes/reverse_bits_test.go index dbc2d207919..5753947f2de 100644 --- a/go/vt/vtgate/vindexes/reverse_bits_test.go +++ b/go/vt/vtgate/vindexes/reverse_bits_test.go @@ -101,7 +101,7 @@ func TestReverseBitsMap(t *testing.T) { sqltypes.NewInt64(6), }) require.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x80\x00\x00\x00\x00\x00\x00\x00")), key.DestinationKeyspaceID([]byte("@\x00\x00\x00\x00\x00\x00\x00")), key.DestinationKeyspaceID([]byte("\xc0\x00\x00\x00\x00\x00\x00\x00")), diff --git a/go/vt/vtgate/vindexes/unicodeloosemd5.go b/go/vt/vtgate/vindexes/unicodeloosemd5.go index 6b7ac52a0d6..1bda47fcad6 100644 --- a/go/vt/vtgate/vindexes/unicodeloosemd5.go +++ b/go/vt/vtgate/vindexes/unicodeloosemd5.go @@ -82,9 +82,9 @@ func (vind *UnicodeLooseMD5) Verify(ctx context.Context, vcursor VCursor, ids [] return out, nil } -// Map can map ids to key.Destination objects. -func (vind *UnicodeLooseMD5) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *UnicodeLooseMD5) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { data, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/unicodeloosexxhash.go b/go/vt/vtgate/vindexes/unicodeloosexxhash.go index 727efacd4b2..7b73704dbd7 100644 --- a/go/vt/vtgate/vindexes/unicodeloosexxhash.go +++ b/go/vt/vtgate/vindexes/unicodeloosexxhash.go @@ -82,9 +82,9 @@ func (vind *UnicodeLooseXXHash) Verify(ctx context.Context, vcursor VCursor, ids return out, nil } -// Map can map ids to key.Destination objects. -func (vind *UnicodeLooseXXHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *UnicodeLooseXXHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { data, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vindexes/vindex.go b/go/vt/vtgate/vindexes/vindex.go index 947877108e0..92d26c57f95 100644 --- a/go/vt/vtgate/vindexes/vindex.go +++ b/go/vt/vtgate/vindexes/vindex.go @@ -95,12 +95,12 @@ type ( // SingleColumn defines the interface for a single column vindex. SingleColumn interface { Vindex - // Map can map ids to key.Destination objects. + // Map can map ids to key.ShardDestination objects. // If the Vindex is unique, each id would map to either // a KeyRange, or a single KeyspaceID. // If the Vindex is non-unique, each id would map to either // a KeyRange, or a list of KeyspaceID. - Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) + Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) // Verify returns true for every id that successfully maps to the // specified keyspace id. @@ -110,7 +110,7 @@ type ( // MultiColumn defines the interface for a multi-column vindex. MultiColumn interface { Vindex - Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) + Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) Verify(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value, ksids [][]byte) ([]bool, error) // PartialVindex returns true if subset of columns can be passed in to the vindex Map and Verify function. PartialVindex() bool @@ -134,7 +134,7 @@ type ( // instead of a single keyspace id. It's being used to reduce the fan out for // 'BETWEEN' expressions. Sequential interface { - RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.Destination, error) + RangeMap(ctx context.Context, vcursor VCursor, startId sqltypes.Value, endId sqltypes.Value) ([]key.ShardDestination, error) } // A Prefixable vindex is one that maps the prefix of a id to a keyspace range @@ -168,7 +168,7 @@ type ( LookupPlanable interface { String() string Query() (selQuery string, arguments []string) - MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.Destination, error) + MapResult(ids []sqltypes.Value, results []*sqltypes.Result) ([]key.ShardDestination, error) AllowBatch() bool GetCommitOrder() vtgatepb.CommitOrder AutoCommitEnabled() bool @@ -216,7 +216,7 @@ func CreateVindex(vindexType, name string, params map[string]string) (vindex Vin } // Map invokes the Map implementation supplied by the vindex. -func Map(ctx context.Context, vindex Vindex, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { +func Map(ctx context.Context, vindex Vindex, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { switch vindex := vindex.(type) { case MultiColumn: return vindex.Map(ctx, vcursor, rowsColValues) diff --git a/go/vt/vtgate/vindexes/vindex_test.go b/go/vt/vtgate/vindexes/vindex_test.go index 97b17da2a4b..5d89ba06192 100644 --- a/go/vt/vtgate/vindexes/vindex_test.go +++ b/go/vt/vtgate/vindexes/vindex_test.go @@ -89,7 +89,7 @@ func TestVindexMap(t *testing.T) { }}) assert.NoError(t, err) - want := []key.Destination{ + want := []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x01\x16k@\xb4J\xbaK\xd6")), } assert.Equal(t, want, got) @@ -101,7 +101,7 @@ func TestVindexMap(t *testing.T) { sqltypes.NewInt64(1), }}) assert.NoError(t, err) - want = []key.Destination{ + want = []key.ShardDestination{ key.DestinationKeyspaceID([]byte("\x16k@\xb4J\xbaK\xd6")), } assert.Equal(t, want, got) diff --git a/go/vt/vtgate/vindexes/vschema_test.go b/go/vt/vtgate/vindexes/vschema_test.go index 5ebb5564ed5..eba29e8d94e 100644 --- a/go/vt/vtgate/vindexes/vschema_test.go +++ b/go/vt/vtgate/vindexes/vschema_test.go @@ -53,7 +53,7 @@ func (*cheapVindex) NeedsVCursor() bool { return false } func (*cheapVindex) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*cheapVindex) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*cheapVindex) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } @@ -75,7 +75,7 @@ func (*stFU) NeedsVCursor() bool { return false } func (*stFU) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*stFU) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*stFU) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } @@ -97,7 +97,7 @@ func (*stFN) NeedsVCursor() bool { return false } func (*stFN) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*stFN) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*stFN) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } @@ -119,7 +119,7 @@ func (*stLN) NeedsVCursor() bool { return true } func (*stLN) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*stLN) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*stLN) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*stLN) Create(context.Context, VCursor, [][]sqltypes.Value, [][]byte, bool) error { return nil } @@ -147,7 +147,7 @@ func (*stLU) NeedsVCursor() bool { return true } func (*stLU) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*stLU) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*stLU) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*stLU) Create(context.Context, VCursor, [][]sqltypes.Value, [][]byte, bool) error { return nil } @@ -179,7 +179,7 @@ func (*stLO) NeedsVCursor() bool { return true } func (*stLO) Verify(context.Context, VCursor, []sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*stLO) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { +func (*stLO) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*stLO) Create(context.Context, VCursor, [][]sqltypes.Value, [][]byte, bool) error { return nil } @@ -213,7 +213,7 @@ func (*mcFU) NeedsVCursor() bool { return false } func (*mcFU) Verify(context.Context, VCursor, [][]sqltypes.Value, [][]byte) ([]bool, error) { return []bool{}, nil } -func (*mcFU) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.Destination, error) { +func (*mcFU) Map(ctx context.Context, vcursor VCursor, rowsColValues [][]sqltypes.Value) ([]key.ShardDestination, error) { return nil, nil } func (*mcFU) PartialVindex() bool { return false } @@ -805,7 +805,7 @@ func TestVSchemaMirrorRules(t *testing.T) { ToTable: "ks4.ks4t1", Percent: 50, }, - // Destination sharded table must be defined in VSchema. + // ShardDestination sharded table must be defined in VSchema. { FromTable: "ks1.ks1t8", ToTable: "ks4.ks4t2", diff --git a/go/vt/vtgate/vindexes/xxhash.go b/go/vt/vtgate/vindexes/xxhash.go index 3362cd0aab1..9cc9590572d 100644 --- a/go/vt/vtgate/vindexes/xxhash.go +++ b/go/vt/vtgate/vindexes/xxhash.go @@ -68,9 +68,9 @@ func (vind *XXHash) NeedsVCursor() bool { return false } -// Map can map ids to key.Destination objects. -func (vind *XXHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.Destination, error) { - out := make([]key.Destination, 0, len(ids)) +// Map can map ids to key.ShardDestination objects. +func (vind *XXHash) Map(ctx context.Context, vcursor VCursor, ids []sqltypes.Value) ([]key.ShardDestination, error) { + out := make([]key.ShardDestination, 0, len(ids)) for _, id := range ids { ksid, err := vind.Hash(id) if err != nil { diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 8b8302d77d4..568ae22f840 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -488,7 +488,14 @@ func (vtg *VTGate) Gateway() *TabletGateway { } // Execute executes a non-streaming query. -func (vtg *VTGate) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (newSession *vtgatepb.Session, qr *sqltypes.Result, err error) { +func (vtg *VTGate) Execute( + ctx context.Context, + mysqlCtx vtgateservice.MySQLConnection, + session *vtgatepb.Session, + sql string, + bindVariables map[string]*querypb.BindVariable, + prepared bool, +) (newSession *vtgatepb.Session, qr *sqltypes.Result, err error) { // In this context, we don't care if we can't fully parse destination destKeyspace, destTabletType, _, _ := vtg.executor.ParseDestinationTarget(session.TargetString) statsKey := []string{"Execute", destKeyspace, topoproto.TabletTypeLString(destTabletType)} @@ -498,7 +505,7 @@ func (vtg *VTGate) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConn err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", bvErr) } else { safeSession := econtext.NewSafeSession(session) - qr, err = vtg.executor.Execute(ctx, mysqlCtx, "Execute", safeSession, sql, bindVariables) + qr, err = vtg.executor.Execute(ctx, mysqlCtx, "Execute", safeSession, sql, bindVariables, prepared) safeSession.RemoveInternalSavepoint() } if err == nil { @@ -536,7 +543,7 @@ func (vtg *VTGate) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, if len(bindVariablesList) != 0 { bv = bindVariablesList[i] } - session, qrl[i].QueryResult, qrl[i].QueryError = vtg.Execute(ctx, nil, session, sql, bv) + session, qrl[i].QueryResult, qrl[i].QueryError = vtg.Execute(ctx, nil, session, sql, bv, false) if qr := qrl[i].QueryResult; qr != nil { vtg.rowsReturned.Add(statsKey, int64(len(qr.Rows))) vtg.rowsAffected.Add(statsKey, int64(qr.RowsAffected)) @@ -592,30 +599,23 @@ func (vtg *VTGate) CloseSession(ctx context.Context, session *vtgatepb.Session) } // Prepare supports non-streaming prepare statement query with multi shards -func (vtg *VTGate) Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (newSession *vtgatepb.Session, fld []*querypb.Field, err error) { +func (vtg *VTGate) Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (newSession *vtgatepb.Session, fld []*querypb.Field, paramsCount uint16, err error) { // In this context, we don't care if we can't fully parse destination destKeyspace, destTabletType, _, _ := vtg.executor.ParseDestinationTarget(session.TargetString) statsKey := []string{"Prepare", destKeyspace, topoproto.TabletTypeLString(destTabletType)} defer vtg.timings.Record(statsKey, time.Now()) - if bvErr := sqltypes.ValidateBindVariables(bindVariables); bvErr != nil { - err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", bvErr) - goto handleError - } - - fld, err = vtg.executor.Prepare(ctx, "Prepare", econtext.NewSafeSession(session), sql, bindVariables) + fld, paramsCount, err = vtg.executor.Prepare(ctx, "Prepare", econtext.NewSafeSession(session), sql) if err == nil { - return session, fld, nil + return session, fld, paramsCount, nil } -handleError: query := map[string]any{ - "Sql": sql, - "BindVariables": bindVariables, - "Session": session, + "Sql": sql, + "Session": session, } err = recordAndAnnotateError(err, statsKey, query, vtg.logPrepare, vtg.executor.vm.parser) - return session, nil, err + return session, nil, 0, err } // VStream streams binlog events. diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index f05f63474d0..1b056cc378d 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -58,6 +58,7 @@ func TestVTGateExecute(t *testing.T) { }, "select id from t1", nil, + false, ) if err != nil { t.Errorf("want nil, got %v", err) @@ -98,6 +99,7 @@ func TestVTGateExecuteError(t *testing.T) { }, "bad select id from t1", nil, + false, ) require.Error(t, err) require.Nil(t, qr) @@ -117,19 +119,17 @@ func TestVTGatePrepare(t *testing.T) { vtg, sbc, ctx := createVtgateEnv(t) counts := vtg.timings.Timings.Counts() - _, qr, err := vtg.Prepare( + _, qr, paramsCount, err := vtg.Prepare( ctx, &vtgatepb.Session{ Autocommit: true, TargetString: KsTestUnsharded + "@primary", Options: executeOptions, }, - "select id from t1", - nil, + "select id from t1 where id = ? and name = ?", ) - if err != nil { - t.Errorf("want nil, got %v", err) - } + require.NoError(t, err) + assert.EqualValues(t, 2, paramsCount) want := sandboxconn.SingleRowResult.Fields utils.MustMatch(t, want, qr) @@ -155,7 +155,7 @@ func TestVTGatePrepareError(t *testing.T) { counts := errorCounts.Counts() - _, qr, err := vtg.Prepare( + _, qr, _, err := vtg.Prepare( ctx, &vtgatepb.Session{ Autocommit: true, @@ -163,7 +163,6 @@ func TestVTGatePrepareError(t *testing.T) { Options: executeOptions, }, "bad select id from t1", - nil, ) require.Error(t, err) require.Nil(t, qr) @@ -191,6 +190,7 @@ func TestVTGateExecuteWithKeyspaceShard(t *testing.T) { }, "select id from none", nil, + false, ) if err != nil { t.Errorf("want nil, got %v", err) @@ -208,6 +208,7 @@ func TestVTGateExecuteWithKeyspaceShard(t *testing.T) { }, "select id from none", nil, + false, ) want := "VT05003: unknown database 'invalid_keyspace' in vschema" assert.EqualError(t, err, want) @@ -221,6 +222,7 @@ func TestVTGateExecuteWithKeyspaceShard(t *testing.T) { }, "select id from none", nil, + false, ) if err != nil { t.Errorf("want nil, got %v", err) @@ -236,6 +238,7 @@ func TestVTGateExecuteWithKeyspaceShard(t *testing.T) { }, "select id from none", nil, + false, ) require.Error(t, err) require.Contains(t, err.Error(), `no healthy tablet available for 'keyspace:"TestExecutor" shard:"noshard" tablet_type:PRIMARY`) @@ -289,7 +292,7 @@ func TestVTGateBindVarError(t *testing.T) { }{{ name: "Execute", f: func() error { - _, _, err := vtg.Execute(ctx, nil, session, "", bindVars) + _, _, err := vtg.Execute(ctx, nil, session, "", bindVars, false) return err }, }, { @@ -328,6 +331,7 @@ func testErrorPropagation(t *testing.T, ctx context.Context, vtg *VTGate, sbcs [ session, "select id from t1", nil, + false, ) if err == nil { t.Errorf("error %v not propagated for Execute", expected) @@ -460,11 +464,11 @@ func TestErrorIssuesRollback(t *testing.T) { // Start a transaction, send one statement. // Simulate an error that should trigger a rollback: // vtrpcpb.Code_ABORTED case. - session, _, err := vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil) + session, _, err := vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil, false) if err != nil { t.Fatalf("cannot start a transaction: %v", err) } - session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err != nil { t.Fatalf("want nil, got %v", err) } @@ -472,7 +476,7 @@ func TestErrorIssuesRollback(t *testing.T) { t.Errorf("want 0, got %d", sbc.RollbackCount.Load()) } sbc.MustFailCodes[vtrpcpb.Code_ABORTED] = 20 - _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err == nil { t.Fatalf("want error but got nil") } @@ -485,11 +489,11 @@ func TestErrorIssuesRollback(t *testing.T) { // Start a transaction, send one statement. // Simulate an error that should trigger a rollback: // vtrpcpb.ErrorCode_RESOURCE_EXHAUSTED case. - session, _, err = vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil) + session, _, err = vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil, false) if err != nil { t.Fatalf("cannot start a transaction: %v", err) } - session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err != nil { t.Fatalf("want nil, got %v", err) } @@ -497,7 +501,7 @@ func TestErrorIssuesRollback(t *testing.T) { t.Errorf("want 0, got %d", sbc.RollbackCount.Load()) } sbc.MustFailCodes[vtrpcpb.Code_RESOURCE_EXHAUSTED] = 20 - _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err == nil { t.Fatalf("want error but got nil") } @@ -510,11 +514,11 @@ func TestErrorIssuesRollback(t *testing.T) { // Start a transaction, send one statement. // Simulate an error that should *not* trigger a rollback: // vtrpcpb.Code_ALREADY_EXISTS case. - session, _, err = vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil) + session, _, err = vtg.Execute(ctx, nil, &vtgatepb.Session{TargetString: KsTestUnsharded + "@primary"}, "begin", nil, false) if err != nil { t.Fatalf("cannot start a transaction: %v", err) } - session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + session, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err != nil { t.Fatalf("want nil, got %v", err) } @@ -522,7 +526,7 @@ func TestErrorIssuesRollback(t *testing.T) { t.Errorf("want 0, got %d", sbc.RollbackCount.Load()) } sbc.MustFailCodes[vtrpcpb.Code_ALREADY_EXISTS] = 20 - _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil) + _, _, err = vtg.Execute(ctx, nil, session, "select id from t1", nil, false) if err == nil { t.Fatalf("want error but got nil") } @@ -608,13 +612,13 @@ func TestMultiInternalSavepointVtGate(t *testing.T) { require.False(t, session.InTransaction) var err error - session, _, err = vtg.Execute(ctx, nil, session, "begin", nil) + session, _, err = vtg.Execute(ctx, nil, session, "begin", nil, false) require.NoError(t, err) require.True(t, session.GetAutocommit()) require.True(t, session.InTransaction) // this query goes to multiple shards so internal savepoint will be created. - session, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (1), (3)", nil) + session, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (1), (3)", nil, false) require.NoError(t, err) require.True(t, session.GetAutocommit()) require.True(t, session.InTransaction) @@ -642,7 +646,7 @@ func TestMultiInternalSavepointVtGate(t *testing.T) { sbc2.Queries = nil // multi shard so new savepoint will be created. - session, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (2), (4)", nil) + session, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (2), (4)", nil, false) require.NoError(t, err) wantQ = []*querypb.BoundQuery{{ Sql: "savepoint x", @@ -660,7 +664,7 @@ func TestMultiInternalSavepointVtGate(t *testing.T) { sbc3.Queries = nil // single shard so no savepoint will be created and neither any old savepoint will be executed - _, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (5)", nil) + _, _, err = vtg.Execute(ctx, nil, session, "insert into sp_tbl(user_id) values (5)", nil, false) require.NoError(t, err) wantQ = []*querypb.BoundQuery{{ Sql: "insert into sp_tbl(user_id) values (:_user_id_0)", diff --git a/go/vt/vtgate/vtgateconn/vtgateconn.go b/go/vt/vtgate/vtgateconn/vtgateconn.go index 38899550c1d..7455bd6dd88 100644 --- a/go/vt/vtgate/vtgateconn/vtgateconn.go +++ b/go/vt/vtgate/vtgateconn/vtgateconn.go @@ -118,8 +118,8 @@ type VTGateSession struct { } // Execute performs a VTGate Execute. -func (sn *VTGateSession) Execute(ctx context.Context, query string, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - session, res, err := sn.impl.Execute(ctx, sn.session, query, bindVars) +func (sn *VTGateSession) Execute(ctx context.Context, query string, bindVars map[string]*querypb.BindVariable, prepared bool) (*sqltypes.Result, error) { + session, res, err := sn.impl.Execute(ctx, sn.session, query, bindVars, prepared) sn.session = session return res, err } @@ -145,10 +145,10 @@ func (sn *VTGateSession) StreamExecute(ctx context.Context, query string, bindVa } // Prepare performs a VTGate Prepare. -func (sn *VTGateSession) Prepare(ctx context.Context, query string, bindVars map[string]*querypb.BindVariable) ([]*querypb.Field, error) { - session, fields, err := sn.impl.Prepare(ctx, sn.session, query, bindVars) +func (sn *VTGateSession) Prepare(ctx context.Context, query string) ([]*querypb.Field, uint16, error) { + session, fields, paramsCount, err := sn.impl.Prepare(ctx, sn.session, query) sn.session = session - return fields, err + return fields, paramsCount, err } // @@ -159,7 +159,7 @@ func (sn *VTGateSession) Prepare(ctx context.Context, query string, bindVars map // implementation. It can be used concurrently across goroutines. type Impl interface { // Execute executes a non-streaming query on vtgate. - Execute(ctx context.Context, session *vtgatepb.Session, query string, bindVars map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) + Execute(ctx context.Context, session *vtgatepb.Session, query string, bindVars map[string]*querypb.BindVariable, prepared bool) (*vtgatepb.Session, *sqltypes.Result, error) // ExecuteBatch executes a non-streaming queries on vtgate. ExecuteBatch(ctx context.Context, session *vtgatepb.Session, queryList []string, bindVarsList []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) @@ -168,7 +168,7 @@ type Impl interface { StreamExecute(ctx context.Context, session *vtgatepb.Session, query string, bindVars map[string]*querypb.BindVariable, processResponse func(*vtgatepb.StreamExecuteResponse)) (sqltypes.ResultStream, error) // Prepare returns the fields information for the query as part of supporting prepare statements. - Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) + Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) // CloseSession closes the session provided by rolling back any active transaction. CloseSession(ctx context.Context, session *vtgatepb.Session) error diff --git a/go/vt/vtgate/vtgateservice/interface.go b/go/vt/vtgate/vtgateservice/interface.go index c829f553d26..e97020651d5 100644 --- a/go/vt/vtgate/vtgateservice/interface.go +++ b/go/vt/vtgate/vtgateservice/interface.go @@ -31,11 +31,11 @@ import ( // VTGateService is the interface implemented by the VTGate service, // that RPC server implementations will call. type VTGateService interface { - Execute(ctx context.Context, mysqlCtx MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, *sqltypes.Result, error) + Execute(ctx context.Context, mysqlCtx MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable, prepared bool) (*vtgatepb.Session, *sqltypes.Result, error) ExecuteBatch(ctx context.Context, session *vtgatepb.Session, sqlList []string, bindVariablesList []map[string]*querypb.BindVariable) (*vtgatepb.Session, []sqltypes.QueryResponse, error) StreamExecute(ctx context.Context, mysqlCtx MySQLConnection, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable, callback func(*sqltypes.Result) error) (*vtgatepb.Session, error) // Prepare statement support - Prepare(ctx context.Context, session *vtgatepb.Session, sql string, bindVariables map[string]*querypb.BindVariable) (*vtgatepb.Session, []*querypb.Field, error) + Prepare(ctx context.Context, session *vtgatepb.Session, sql string) (*vtgatepb.Session, []*querypb.Field, uint16, error) // CloseSession closes the session, rolling back any implicit transactions. // This has the same effect as if a "rollback" statement was executed, diff --git a/go/vt/vthash/highway/highwayhash.go b/go/vt/vthash/highway/highwayhash.go index a922b435d9d..20b9ca4d3a8 100644 --- a/go/vt/vthash/highway/highwayhash.go +++ b/go/vt/vthash/highway/highwayhash.go @@ -142,6 +142,12 @@ func (d *Digest) Write(p []byte) (n int, err error) { return } +func (d *Digest) WriteUint16(x uint16) (int, error) { + var b [2]byte + binary.LittleEndian.PutUint16(b[:], x) + return d.Write(b[:]) +} + func (d *Digest) Sum(b []byte) []byte { state := d.state if d.offset > 0 { diff --git a/go/vtbench/client.go b/go/vtbench/client.go index 585a66d356f..869b0c92917 100644 --- a/go/vtbench/client.go +++ b/go/vtbench/client.go @@ -106,7 +106,7 @@ func (c *grpcVtgateConn) connect(ctx context.Context, cp ConnParams) error { } func (c *grpcVtgateConn) execute(ctx context.Context, query string, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return c.session.Execute(ctx, query, bindVars) + return c.session.Execute(ctx, query, bindVars, false) } type grpcVttabletConn struct { diff --git a/proto/vtgate.proto b/proto/vtgate.proto index aadf211f0a2..189428ec0c7 100644 --- a/proto/vtgate.proto +++ b/proto/vtgate.proto @@ -194,6 +194,8 @@ message ExecuteRequest { // Deprecated: use session.in_transaction instead // bool not_in_transaction = 5; reserved 4, 5, 6, 7; + + bool prepared = 8; } // ExecuteResponse is the returned value from Execute. @@ -354,6 +356,8 @@ message PrepareResponse { // fields contains the fields, only set if error is unset. repeated query.Field fields = 3; + + uint32 params_count = 4; } // CloseSessionRequest is the payload to CloseSession.