From 481ff933ab3ea5c20b482ad2fc34db199c2c215a Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 10 Mar 2020 16:15:14 -0700 Subject: [PATCH 01/18] tabletserver: remove TxPoolController abstraction It was not necessary, and was obfuscating the fact that it's wrapping TxEngine. Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/tabletserver.go | 60 ++++----------------- 1 file changed, 9 insertions(+), 51 deletions(-) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 9bc79c1dd82..259d055136d 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -163,7 +163,6 @@ type TabletServer struct { se *schema.Engine qe *QueryEngine te *TxEngine - teCtrl TxPoolController hw *heartbeat.Writer hr *heartbeat.Reader watcher *ReplicationWatcher @@ -207,46 +206,6 @@ func NewServer(topoServer *topo.Server, alias topodatapb.TabletAlias) *TabletSer return NewTabletServer(tabletenv.Config, topoServer, alias) } -// TxPoolController is how the tablet server interacts with the tx-pool. -// It is responsible for keeping it's own state - knowing when different types -// of transactions are allowed, and how to do state transitions. -type TxPoolController interface { - // Stop will stop accepting any new transactions. Transactions are immediately aborted. - Stop() error - - // Will start accepting all transactions. If transitioning from RO mode, transactions - // might need to be rolled back before new transactions can be accepts. - AcceptReadWrite() error - - // Will start accepting read-only transactions, but not full read and write transactions. - // If the engine is currently accepting full read and write transactions, they need to - // given a chance to clean up before they are forcefully rolled back. - AcceptReadOnly() error - - // InitDBConfig must be called before Init. - InitDBConfig(dbcfgs *dbconfigs.DBConfigs) - - // Init must be called once when vttablet starts for setting - // up the metadata tables. - Init() error - - // StopGently will change the state to NotServing but first wait for transactions to wrap up - StopGently() - - // Begin begins a transaction, and returns the associated transaction id and the - // statement(s) used to execute the begin (if any). - // - // Subsequent statements can access the connection through the transaction id. - Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, string, error) - - // Commit commits the specified transaction, returning the statement used to execute - // the commit or "" in autocommit settings. - Commit(ctx context.Context, transactionID int64) (string, error) - - // Rollback rolls back the specified transaction. - Rollback(ctx context.Context, transactionID int64) error -} - var tsOnce sync.Once var srvTopoServer srvtopo.Server @@ -266,7 +225,6 @@ func NewTabletServer(config tabletenv.TabletConfig, topoServer *topo.Server, ali tsv.se = schema.NewEngine(tsv, config) tsv.qe = NewQueryEngine(tsv, tsv.se, config) tsv.te = NewTxEngine(tsv, config) - tsv.teCtrl = tsv.te tsv.hw = heartbeat.NewWriter(tsv, alias, config) tsv.hr = heartbeat.NewReader(tsv, config) tsv.txThrottler = txthrottler.CreateTxThrottlerFromTabletConfig(topoServer) @@ -378,7 +336,7 @@ func (tsv *TabletServer) InitDBConfig(target querypb.Target, dbcfgs *dbconfigs.D tsv.se.InitDBConfig(tsv.dbconfigs.DbaWithDB()) tsv.qe.InitDBConfig(tsv.dbconfigs) - tsv.teCtrl.InitDBConfig(tsv.dbconfigs) + tsv.te.InitDBConfig(tsv.dbconfigs) tsv.hw.InitDBConfig(tsv.dbconfigs) tsv.hr.InitDBConfig(tsv.dbconfigs) tsv.watcher.InitDBConfig(tsv.dbconfigs) @@ -539,7 +497,7 @@ func (tsv *TabletServer) fullStart() (err error) { if err := tsv.qe.Open(); err != nil { return err } - if err := tsv.teCtrl.Init(); err != nil { + if err := tsv.te.Init(); err != nil { return err } if err := tsv.hw.Init(tsv.target); err != nil { @@ -556,7 +514,7 @@ func (tsv *TabletServer) serveNewType() (err error) { // transactional requests are not allowed. So, we can // be sure that the tx pool won't change after the wait. if tsv.target.TabletType == topodatapb.TabletType_MASTER { - tsv.teCtrl.AcceptReadWrite() + tsv.te.AcceptReadWrite() if err := tsv.txThrottler.Open(tsv.target.Keyspace, tsv.target.Shard); err != nil { return err } @@ -565,7 +523,7 @@ func (tsv *TabletServer) serveNewType() (err error) { tsv.hr.Close() tsv.hw.Open() } else { - tsv.teCtrl.AcceptReadOnly() + tsv.te.AcceptReadOnly() tsv.messager.Close() tsv.hr.Open() tsv.hw.Close() @@ -620,7 +578,7 @@ func (tsv *TabletServer) waitForShutdown() { // will be allowed. They will enable the conclusion of outstanding // transactions. tsv.messager.Close() - tsv.teCtrl.StopGently() + tsv.te.StopGently() tsv.qe.streamQList.TerminateAll() tsv.watcher.Close() tsv.requests.Wait() @@ -634,7 +592,7 @@ func (tsv *TabletServer) closeAll() { tsv.vstreamer.Close() tsv.hr.Close() tsv.hw.Close() - tsv.teCtrl.StopGently() + tsv.te.StopGently() tsv.watcher.Close() tsv.qe.Close() tsv.se.Close() @@ -766,7 +724,7 @@ func (tsv *TabletServer) Begin(ctx context.Context, target *querypb.Target, opti return vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "Transaction throttled") } var beginSQL string - transactionID, beginSQL, err = tsv.teCtrl.Begin(ctx, options) + transactionID, beginSQL, err = tsv.te.Begin(ctx, options) logStats.TransactionID = transactionID // Record the actual statements that were executed in the logStats. @@ -796,7 +754,7 @@ func (tsv *TabletServer) Commit(ctx context.Context, target *querypb.Target, tra logStats.TransactionID = transactionID var commitSQL string - commitSQL, err = tsv.teCtrl.Commit(ctx, transactionID) + commitSQL, err = tsv.te.Commit(ctx, transactionID) // If nothing was actually executed, don't count the operation in // the tablet metrics, and clear out the logStats Method so that @@ -820,7 +778,7 @@ func (tsv *TabletServer) Rollback(ctx context.Context, target *querypb.Target, t func(ctx context.Context, logStats *tabletenv.LogStats) error { defer tabletenv.QueryStats.Record("ROLLBACK", time.Now()) logStats.TransactionID = transactionID - return tsv.teCtrl.Rollback(ctx, transactionID) + return tsv.te.Rollback(ctx, transactionID) }, ) } From 1d66aa041c1103e79f05a55b6fbc4dfdd34930dd Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 10 Mar 2020 19:01:20 -0700 Subject: [PATCH 02/18] deprecation: delete unused event token code There was an aspiration that applications will use an event token to validate how fresh a replica read was. The feature was neither very usable nor used by anyone. This has now been deleted. Signed-off-by: Sugu Sougoumarane --- go/cmd/vtgateclienttest/goclienttest/echo.go | 25 +- go/cmd/vtgateclienttest/services/echo.go | 10 - go/sqltypes/proto3.go | 3 - go/sqltypes/proto3_test.go | 70 --- go/sqltypes/result.go | 63 +- go/sqltypes/result_test.go | 24 - go/vt/proto/query/query.pb.go | 569 ++++++++---------- go/vt/vtgate/engine/ordered_aggregate.go | 1 - go/vt/vtgate/grpcvtgateconn/suite_test.go | 22 +- .../tabletconntest/fakequeryservice.go | 24 +- .../tabletmanager/vreplication/engine_test.go | 4 +- go/vt/vttablet/tabletserver/query_executor.go | 6 +- .../tabletserver/replication_watcher.go | 161 +---- go/vt/vttablet/tabletserver/tabletserver.go | 9 +- proto/query.proto | 30 +- 15 files changed, 295 insertions(+), 726 deletions(-) diff --git a/go/cmd/vtgateclienttest/goclienttest/echo.go b/go/cmd/vtgateclienttest/goclienttest/echo.go index 31236ce911d..0e8f4c1d89d 100644 --- a/go/cmd/vtgateclienttest/goclienttest/echo.go +++ b/go/cmd/vtgateclienttest/goclienttest/echo.go @@ -85,20 +85,10 @@ var ( callerID = callerid.NewEffectiveCallerID("test_principal", "test_component", "test_subcomponent") callerIDEcho = "principal:\"test_principal\" component:\"test_component\" subcomponent:\"test_subcomponent\" " - eventToken = &querypb.EventToken{ - Timestamp: 876543, - Shard: shards[0], - Position: "test_position", - } - eventTokenEcho = "timestamp:876543 shard:\"-80\" position:\"test_position\" " - options = &querypb.ExecuteOptions{ - IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, - IncludeEventToken: true, - CompareEventToken: eventToken, + IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, } - optionsEcho = "include_event_token:true compare_event_token:<" + eventTokenEcho + "> included_fields:TYPE_ONLY " - extrasEcho = "event_token:<" + eventTokenEcho + "> fresher:true " + optionsEcho = "included_fields:TYPE_ONLY " ) // testEcho exercises the test cases provided by the "echo" service. @@ -130,7 +120,6 @@ func testEchoExecute(t *testing.T, conn *vtgateconn.VTGateConn, session *vtgatec "bindVars": bindVarsEcho, "tabletType": tabletTypeEcho, "options": optionsEcho, - "extras": extrasEcho, }) qr, err = conn.ExecuteKeyspaceIds(ctx, echoPrefix+query, keyspace, keyspaceIDs, bindVars, tabletType, options) @@ -142,7 +131,6 @@ func testEchoExecute(t *testing.T, conn *vtgateconn.VTGateConn, session *vtgatec "bindVars": bindVarsEcho, "tabletType": tabletTypeEcho, "options": optionsEcho, - "extras": extrasEcho, }) qr, err = conn.ExecuteKeyRanges(ctx, echoPrefix+query, keyspace, keyRanges, bindVars, tabletType, options) @@ -154,7 +142,6 @@ func testEchoExecute(t *testing.T, conn *vtgateconn.VTGateConn, session *vtgatec "bindVars": bindVarsEcho, "tabletType": tabletTypeEcho, "options": optionsEcho, - "extras": extrasEcho, }) qr, err = conn.ExecuteEntityIds(ctx, echoPrefix+query, keyspace, "column1", entityKeyspaceIDs, bindVars, tabletType, options) @@ -167,7 +154,6 @@ func testEchoExecute(t *testing.T, conn *vtgateconn.VTGateConn, session *vtgatec "bindVars": bindVarsEcho, "tabletType": tabletTypeEcho, "options": optionsEcho, - "extras": extrasEcho, }) var qrs []sqltypes.Result @@ -418,13 +404,6 @@ func checkEcho(t *testing.T, name string, qr *sqltypes.Result, err error, want m } got := getEcho(qr) for k, v := range want { - if k == "extras" { - gotExtras := qr.Extras.String() - if gotExtras != v { - t.Errorf("%v: extras = \n%q, want \n%q", name, gotExtras, v) - } - continue - } if got[k].ToString() != v { t.Errorf("%v: %v = \n%q, want \n%q", name, k, got[k], v) } diff --git a/go/cmd/vtgateclienttest/services/echo.go b/go/cmd/vtgateclienttest/services/echo.go index f6725c994a6..f200d07bdab 100644 --- a/go/cmd/vtgateclienttest/services/echo.go +++ b/go/cmd/vtgateclienttest/services/echo.go @@ -95,16 +95,6 @@ func echoQueryResult(vals map[string]interface{}) *sqltypes.Result { } qr.Rows = [][]sqltypes.Value{row} - if options, ok := vals["options"]; ok { - o := options.(*querypb.ExecuteOptions) - if o != nil && o.CompareEventToken != nil { - qr.Extras = &querypb.ResultExtras{ - Fresher: true, - EventToken: o.CompareEventToken, - } - } - } - return qr } diff --git a/go/sqltypes/proto3.go b/go/sqltypes/proto3.go index c42cc81c3ce..284e27fadbd 100644 --- a/go/sqltypes/proto3.go +++ b/go/sqltypes/proto3.go @@ -90,7 +90,6 @@ func ResultToProto3(qr *Result) *querypb.QueryResult { RowsAffected: qr.RowsAffected, InsertId: qr.InsertID, Rows: RowsToProto3(qr.Rows), - Extras: qr.Extras, } } @@ -105,7 +104,6 @@ func Proto3ToResult(qr *querypb.QueryResult) *Result { RowsAffected: qr.RowsAffected, InsertID: qr.InsertId, Rows: proto3ToRows(qr.Fields, qr.Rows), - Extras: qr.Extras, } } @@ -121,7 +119,6 @@ func CustomProto3ToResult(fields []*querypb.Field, qr *querypb.QueryResult) *Res RowsAffected: qr.RowsAffected, InsertID: qr.InsertId, Rows: proto3ToRows(fields, qr.Rows), - Extras: qr.Extras, } } diff --git a/go/sqltypes/proto3_test.go b/go/sqltypes/proto3_test.go index 0d55fff019d..5cb88ebe353 100644 --- a/go/sqltypes/proto3_test.go +++ b/go/sqltypes/proto3_test.go @@ -49,13 +49,6 @@ func TestResult(t *testing.T) { NULL, NULL, }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, } p3Result := &querypb.QueryResult{ Fields: fields, @@ -68,13 +61,6 @@ func TestResult(t *testing.T) { Lengths: []int64{2, -1, -1}, Values: []byte("bb"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, } p3converted := ResultToProto3(sqlResult) if !proto.Equal(p3converted, p3Result) { @@ -125,13 +111,6 @@ func TestResults(t *testing.T) { TestValue(Int64, "1"), TestValue(Float64, "2"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, }, { Fields: fields2, InsertID: 3, @@ -141,13 +120,6 @@ func TestResults(t *testing.T) { TestValue(Int64, "3"), TestValue(Float64, "4"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard1", - Position: "position1", - }, - }, }} p3Results := []*querypb.QueryResult{{ Fields: fields1, @@ -157,13 +129,6 @@ func TestResults(t *testing.T) { Lengths: []int64{2, 1, 1}, Values: []byte("aa12"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, }, { Fields: fields2, InsertId: 3, @@ -172,13 +137,6 @@ func TestResults(t *testing.T) { Lengths: []int64{2, 1, 1}, Values: []byte("bb34"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard1", - Position: "position1", - }, - }, }} p3converted := ResultsToProto3(sqlResults) if !Proto3ResultsEqual(p3converted, p3Results) { @@ -224,13 +182,6 @@ func TestQueryReponses(t *testing.T) { TestValue(Int64, "1"), TestValue(Float64, "2"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, }, QueryError: nil, }, { @@ -243,13 +194,6 @@ func TestQueryReponses(t *testing.T) { TestValue(Int64, "3"), TestValue(Float64, "4"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard1", - Position: "position1", - }, - }, }, QueryError: nil, }, { @@ -269,13 +213,6 @@ func TestQueryReponses(t *testing.T) { Lengths: []int64{2, 1, 1}, Values: []byte("aa12"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard0", - Position: "position0", - }, - }, }, }, { Error: nil, @@ -287,13 +224,6 @@ func TestQueryReponses(t *testing.T) { Lengths: []int64{2, 1, 1}, Values: []byte("bb34"), }}, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "shard1", - Position: "position1", - }, - }, }, }, { Error: &vtrpcpb.RPCError{ diff --git a/go/sqltypes/result.go b/go/sqltypes/result.go index 2f308f6d210..b12378b2ebf 100644 --- a/go/sqltypes/result.go +++ b/go/sqltypes/result.go @@ -25,11 +25,10 @@ import ( // Result represents a query result. type Result struct { - Fields []*querypb.Field `json:"fields"` - RowsAffected uint64 `json:"rows_affected"` - InsertID uint64 `json:"insert_id"` - Rows [][]Value `json:"rows"` - Extras *querypb.ResultExtras `json:"extras"` + Fields []*querypb.Field `json:"fields"` + RowsAffected uint64 `json:"rows_affected"` + InsertID uint64 `json:"insert_id"` + Rows [][]Value `json:"rows"` } // ResultStream is an interface for receiving Result. It is used for @@ -74,18 +73,6 @@ func (result *Result) Copy() *Result { out.Rows = append(out.Rows, CopyRow(r)) } } - if result.Extras != nil { - out.Extras = &querypb.ResultExtras{ - Fresher: result.Extras.Fresher, - } - if result.Extras.EventToken != nil { - out.Extras.EventToken = &querypb.EventToken{ - Timestamp: result.Extras.EventToken.Timestamp, - Shard: result.Extras.EventToken.Shard, - Position: result.Extras.EventToken.Position, - } - } - } return out } @@ -118,18 +105,6 @@ func (result *Result) Truncate(l int) *Result { out.Rows = append(out.Rows, r[:l]) } } - if result.Extras != nil { - out.Extras = &querypb.ResultExtras{ - Fresher: result.Extras.Fresher, - } - if result.Extras.EventToken != nil { - out.Extras.EventToken = &querypb.EventToken{ - Timestamp: result.Extras.EventToken.Timestamp, - Shard: result.Extras.EventToken.Shard, - Position: result.Extras.EventToken.Position, - } - } - } return out } @@ -158,12 +133,11 @@ func (result *Result) Equal(other *Result) bool { return false } - // Compare Fields, RowsAffected, InsertID, Rows, Extras. + // Compare Fields, RowsAffected, InsertID, Rows. return FieldsEqual(result.Fields, other.Fields) && result.RowsAffected == other.RowsAffected && result.InsertID == other.InsertID && - reflect.DeepEqual(result.Rows, other.Rows) && - proto.Equal(result.Extras, other.Extras) + reflect.DeepEqual(result.Rows, other.Rows) } // ResultsEqual compares two arrays of Result. @@ -242,30 +216,5 @@ func (result *Result) AppendResult(src *Result) { if src.InsertID != 0 { result.InsertID = src.InsertID } - if len(result.Rows) == 0 { - // we haven't gotten any result yet, just save the new extras. - result.Extras = src.Extras - } else { - // Merge the EventTokens / Fresher flags within Extras. - if src.Extras == nil { - // We didn't get any from innerq. Have to clear any - // we'd have gotten already. - if result.Extras != nil { - result.Extras.EventToken = nil - result.Extras.Fresher = false - } - } else { - // We may have gotten an EventToken from - // innerqr. If we also got one earlier, merge - // it. If we didn't get one earlier, we - // discard the new one. - if result.Extras != nil { - // Note if any of the two is nil, we get nil. - result.Extras.EventToken = EventTokenMinimum(result.Extras.EventToken, src.Extras.EventToken) - - result.Extras.Fresher = result.Extras.Fresher && src.Extras.Fresher - } - } - } result.Rows = append(result.Rows, src.Rows...) } diff --git a/go/sqltypes/result_test.go b/go/sqltypes/result_test.go index 2ddeaa59b34..bf2d9fd87fe 100644 --- a/go/sqltypes/result_test.go +++ b/go/sqltypes/result_test.go @@ -61,14 +61,6 @@ func TestCopy(t *testing.T) { {TestValue(Int64, "2"), MakeTrusted(VarChar, nil)}, {TestValue(Int64, "3"), TestValue(VarChar, "")}, }, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "sh", - Position: "po", - }, - Fresher: true, - }, } out := in.Copy() if !reflect.DeepEqual(out, in) { @@ -90,14 +82,6 @@ func TestTruncate(t *testing.T) { {TestValue(Int64, "2"), MakeTrusted(VarChar, nil)}, {TestValue(Int64, "3"), TestValue(VarChar, "")}, }, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "sh", - Position: "po", - }, - Fresher: true, - }, } out := in.Truncate(0) @@ -117,14 +101,6 @@ func TestTruncate(t *testing.T) { {TestValue(Int64, "2")}, {TestValue(Int64, "3")}, }, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "sh", - Position: "po", - }, - Fresher: true, - }, } if !reflect.DeepEqual(out, want) { t.Errorf("Truncate(1):\n%v, want\n%v", out, want) diff --git a/go/vt/proto/query/query.pb.go b/go/vt/proto/query/query.pb.go index 120dcb99879..45f9128cebd 100644 --- a/go/vt/proto/query/query.pb.go +++ b/go/vt/proto/query/query.pb.go @@ -488,7 +488,7 @@ func (x StreamEvent_Statement_Category) String() string { } func (StreamEvent_Statement_Category) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{12, 0, 0} + return fileDescriptor_5c6ac9b241082464, []int{11, 0, 0} } // Target describes what the client expects the tablet is. @@ -835,12 +835,6 @@ func (m *BoundQuery) GetBindVariables() map[string]*BindVariable { // ExecuteOptions is passed around for all Execute calls. type ExecuteOptions struct { - // If set, we will try to include an EventToken with the responses. - IncludeEventToken bool `protobuf:"varint,2,opt,name=include_event_token,json=includeEventToken,proto3" json:"include_event_token,omitempty"` - // If set, the fresher field may be set as a result comparison to this token. - // This is a shortcut so the application doesn't need to care about - // comparing EventTokens. - CompareEventToken *EventToken `protobuf:"bytes,3,opt,name=compare_event_token,json=compareEventToken,proto3" json:"compare_event_token,omitempty"` // Controls what fields are returned in Field message responses from mysql, i.e. // field name, table name, etc. This is an optimization for high-QPS queries where // the client knows what it's getting @@ -894,20 +888,6 @@ func (m *ExecuteOptions) XXX_DiscardUnknown() { var xxx_messageInfo_ExecuteOptions proto.InternalMessageInfo -func (m *ExecuteOptions) GetIncludeEventToken() bool { - if m != nil { - return m.IncludeEventToken - } - return false -} - -func (m *ExecuteOptions) GetCompareEventToken() *EventToken { - if m != nil { - return m.CompareEventToken - } - return nil -} - func (m *ExecuteOptions) GetIncludedFields() ExecuteOptions_IncludedFields { if m != nil { return m.IncludedFields @@ -1124,59 +1104,6 @@ func (m *Row) GetValues() []byte { return nil } -// ResultExtras contains optional out-of-band information. Usually the -// extras are requested by adding ExecuteOptions flags. -type ResultExtras struct { - // event_token is populated if the include_event_token flag is set - // in ExecuteOptions. - EventToken *EventToken `protobuf:"bytes,1,opt,name=event_token,json=eventToken,proto3" json:"event_token,omitempty"` - // If set, it means the data returned with this result is fresher - // than the compare_token passed in the ExecuteOptions. - Fresher bool `protobuf:"varint,2,opt,name=fresher,proto3" json:"fresher,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ResultExtras) Reset() { *m = ResultExtras{} } -func (m *ResultExtras) String() string { return proto.CompactTextString(m) } -func (*ResultExtras) ProtoMessage() {} -func (*ResultExtras) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{9} -} - -func (m *ResultExtras) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ResultExtras.Unmarshal(m, b) -} -func (m *ResultExtras) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ResultExtras.Marshal(b, m, deterministic) -} -func (m *ResultExtras) XXX_Merge(src proto.Message) { - xxx_messageInfo_ResultExtras.Merge(m, src) -} -func (m *ResultExtras) XXX_Size() int { - return xxx_messageInfo_ResultExtras.Size(m) -} -func (m *ResultExtras) XXX_DiscardUnknown() { - xxx_messageInfo_ResultExtras.DiscardUnknown(m) -} - -var xxx_messageInfo_ResultExtras proto.InternalMessageInfo - -func (m *ResultExtras) GetEventToken() *EventToken { - if m != nil { - return m.EventToken - } - return nil -} - -func (m *ResultExtras) GetFresher() bool { - if m != nil { - return m.Fresher - } - return false -} - // QueryResult is returned by Execute and ExecuteStream. // // As returned by Execute, len(fields) is always equal to len(row) @@ -1187,21 +1114,20 @@ func (m *ResultExtras) GetFresher() bool { // len(QueryResult[0].fields) is always equal to len(row) (for each // row in rows for each QueryResult in QueryResult[1:]). type QueryResult struct { - Fields []*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` - RowsAffected uint64 `protobuf:"varint,2,opt,name=rows_affected,json=rowsAffected,proto3" json:"rows_affected,omitempty"` - InsertId uint64 `protobuf:"varint,3,opt,name=insert_id,json=insertId,proto3" json:"insert_id,omitempty"` - Rows []*Row `protobuf:"bytes,4,rep,name=rows,proto3" json:"rows,omitempty"` - Extras *ResultExtras `protobuf:"bytes,5,opt,name=extras,proto3" json:"extras,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Fields []*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` + RowsAffected uint64 `protobuf:"varint,2,opt,name=rows_affected,json=rowsAffected,proto3" json:"rows_affected,omitempty"` + InsertId uint64 `protobuf:"varint,3,opt,name=insert_id,json=insertId,proto3" json:"insert_id,omitempty"` + Rows []*Row `protobuf:"bytes,4,rep,name=rows,proto3" json:"rows,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *QueryResult) Reset() { *m = QueryResult{} } func (m *QueryResult) String() string { return proto.CompactTextString(m) } func (*QueryResult) ProtoMessage() {} func (*QueryResult) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{10} + return fileDescriptor_5c6ac9b241082464, []int{9} } func (m *QueryResult) XXX_Unmarshal(b []byte) error { @@ -1250,13 +1176,6 @@ func (m *QueryResult) GetRows() []*Row { return nil } -func (m *QueryResult) GetExtras() *ResultExtras { - if m != nil { - return m.Extras - } - return nil -} - // QueryWarning is used to convey out of band query execution warnings // by storing in the vtgate.Session type QueryWarning struct { @@ -1271,7 +1190,7 @@ func (m *QueryWarning) Reset() { *m = QueryWarning{} } func (m *QueryWarning) String() string { return proto.CompactTextString(m) } func (*QueryWarning) ProtoMessage() {} func (*QueryWarning) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{11} + return fileDescriptor_5c6ac9b241082464, []int{10} } func (m *QueryWarning) XXX_Unmarshal(b []byte) error { @@ -1323,7 +1242,7 @@ func (m *StreamEvent) Reset() { *m = StreamEvent{} } func (m *StreamEvent) String() string { return proto.CompactTextString(m) } func (*StreamEvent) ProtoMessage() {} func (*StreamEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{12} + return fileDescriptor_5c6ac9b241082464, []int{11} } func (m *StreamEvent) XXX_Unmarshal(b []byte) error { @@ -1377,7 +1296,7 @@ func (m *StreamEvent_Statement) Reset() { *m = StreamEvent_Statement{} } func (m *StreamEvent_Statement) String() string { return proto.CompactTextString(m) } func (*StreamEvent_Statement) ProtoMessage() {} func (*StreamEvent_Statement) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{12, 0} + return fileDescriptor_5c6ac9b241082464, []int{11, 0} } func (m *StreamEvent_Statement) XXX_Unmarshal(b []byte) error { @@ -1450,7 +1369,7 @@ func (m *ExecuteRequest) Reset() { *m = ExecuteRequest{} } func (m *ExecuteRequest) String() string { return proto.CompactTextString(m) } func (*ExecuteRequest) ProtoMessage() {} func (*ExecuteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{13} + return fileDescriptor_5c6ac9b241082464, []int{12} } func (m *ExecuteRequest) XXX_Unmarshal(b []byte) error { @@ -1525,7 +1444,7 @@ func (m *ExecuteResponse) Reset() { *m = ExecuteResponse{} } func (m *ExecuteResponse) String() string { return proto.CompactTextString(m) } func (*ExecuteResponse) ProtoMessage() {} func (*ExecuteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{14} + return fileDescriptor_5c6ac9b241082464, []int{13} } func (m *ExecuteResponse) XXX_Unmarshal(b []byte) error { @@ -1570,7 +1489,7 @@ func (m *ResultWithError) Reset() { *m = ResultWithError{} } func (m *ResultWithError) String() string { return proto.CompactTextString(m) } func (*ResultWithError) ProtoMessage() {} func (*ResultWithError) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{15} + return fileDescriptor_5c6ac9b241082464, []int{14} } func (m *ResultWithError) XXX_Unmarshal(b []byte) error { @@ -1623,7 +1542,7 @@ func (m *ExecuteBatchRequest) Reset() { *m = ExecuteBatchRequest{} } func (m *ExecuteBatchRequest) String() string { return proto.CompactTextString(m) } func (*ExecuteBatchRequest) ProtoMessage() {} func (*ExecuteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{16} + return fileDescriptor_5c6ac9b241082464, []int{15} } func (m *ExecuteBatchRequest) XXX_Unmarshal(b []byte) error { @@ -1705,7 +1624,7 @@ func (m *ExecuteBatchResponse) Reset() { *m = ExecuteBatchResponse{} } func (m *ExecuteBatchResponse) String() string { return proto.CompactTextString(m) } func (*ExecuteBatchResponse) ProtoMessage() {} func (*ExecuteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{17} + return fileDescriptor_5c6ac9b241082464, []int{16} } func (m *ExecuteBatchResponse) XXX_Unmarshal(b []byte) error { @@ -1750,7 +1669,7 @@ func (m *StreamExecuteRequest) Reset() { *m = StreamExecuteRequest{} } func (m *StreamExecuteRequest) String() string { return proto.CompactTextString(m) } func (*StreamExecuteRequest) ProtoMessage() {} func (*StreamExecuteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{18} + return fileDescriptor_5c6ac9b241082464, []int{17} } func (m *StreamExecuteRequest) XXX_Unmarshal(b []byte) error { @@ -1825,7 +1744,7 @@ func (m *StreamExecuteResponse) Reset() { *m = StreamExecuteResponse{} } func (m *StreamExecuteResponse) String() string { return proto.CompactTextString(m) } func (*StreamExecuteResponse) ProtoMessage() {} func (*StreamExecuteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{19} + return fileDescriptor_5c6ac9b241082464, []int{18} } func (m *StreamExecuteResponse) XXX_Unmarshal(b []byte) error { @@ -1868,7 +1787,7 @@ func (m *BeginRequest) Reset() { *m = BeginRequest{} } func (m *BeginRequest) String() string { return proto.CompactTextString(m) } func (*BeginRequest) ProtoMessage() {} func (*BeginRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{20} + return fileDescriptor_5c6ac9b241082464, []int{19} } func (m *BeginRequest) XXX_Unmarshal(b []byte) error { @@ -1929,7 +1848,7 @@ func (m *BeginResponse) Reset() { *m = BeginResponse{} } func (m *BeginResponse) String() string { return proto.CompactTextString(m) } func (*BeginResponse) ProtoMessage() {} func (*BeginResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{21} + return fileDescriptor_5c6ac9b241082464, []int{20} } func (m *BeginResponse) XXX_Unmarshal(b []byte) error { @@ -1972,7 +1891,7 @@ func (m *CommitRequest) Reset() { *m = CommitRequest{} } func (m *CommitRequest) String() string { return proto.CompactTextString(m) } func (*CommitRequest) ProtoMessage() {} func (*CommitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{22} + return fileDescriptor_5c6ac9b241082464, []int{21} } func (m *CommitRequest) XXX_Unmarshal(b []byte) error { @@ -2032,7 +1951,7 @@ func (m *CommitResponse) Reset() { *m = CommitResponse{} } func (m *CommitResponse) String() string { return proto.CompactTextString(m) } func (*CommitResponse) ProtoMessage() {} func (*CommitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{23} + return fileDescriptor_5c6ac9b241082464, []int{22} } func (m *CommitResponse) XXX_Unmarshal(b []byte) error { @@ -2068,7 +1987,7 @@ func (m *RollbackRequest) Reset() { *m = RollbackRequest{} } func (m *RollbackRequest) String() string { return proto.CompactTextString(m) } func (*RollbackRequest) ProtoMessage() {} func (*RollbackRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{24} + return fileDescriptor_5c6ac9b241082464, []int{23} } func (m *RollbackRequest) XXX_Unmarshal(b []byte) error { @@ -2128,7 +2047,7 @@ func (m *RollbackResponse) Reset() { *m = RollbackResponse{} } func (m *RollbackResponse) String() string { return proto.CompactTextString(m) } func (*RollbackResponse) ProtoMessage() {} func (*RollbackResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{25} + return fileDescriptor_5c6ac9b241082464, []int{24} } func (m *RollbackResponse) XXX_Unmarshal(b []byte) error { @@ -2165,7 +2084,7 @@ func (m *PrepareRequest) Reset() { *m = PrepareRequest{} } func (m *PrepareRequest) String() string { return proto.CompactTextString(m) } func (*PrepareRequest) ProtoMessage() {} func (*PrepareRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{26} + return fileDescriptor_5c6ac9b241082464, []int{25} } func (m *PrepareRequest) XXX_Unmarshal(b []byte) error { @@ -2232,7 +2151,7 @@ func (m *PrepareResponse) Reset() { *m = PrepareResponse{} } func (m *PrepareResponse) String() string { return proto.CompactTextString(m) } func (*PrepareResponse) ProtoMessage() {} func (*PrepareResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{27} + return fileDescriptor_5c6ac9b241082464, []int{26} } func (m *PrepareResponse) XXX_Unmarshal(b []byte) error { @@ -2268,7 +2187,7 @@ func (m *CommitPreparedRequest) Reset() { *m = CommitPreparedRequest{} } func (m *CommitPreparedRequest) String() string { return proto.CompactTextString(m) } func (*CommitPreparedRequest) ProtoMessage() {} func (*CommitPreparedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{28} + return fileDescriptor_5c6ac9b241082464, []int{27} } func (m *CommitPreparedRequest) XXX_Unmarshal(b []byte) error { @@ -2328,7 +2247,7 @@ func (m *CommitPreparedResponse) Reset() { *m = CommitPreparedResponse{} func (m *CommitPreparedResponse) String() string { return proto.CompactTextString(m) } func (*CommitPreparedResponse) ProtoMessage() {} func (*CommitPreparedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{29} + return fileDescriptor_5c6ac9b241082464, []int{28} } func (m *CommitPreparedResponse) XXX_Unmarshal(b []byte) error { @@ -2365,7 +2284,7 @@ func (m *RollbackPreparedRequest) Reset() { *m = RollbackPreparedRequest func (m *RollbackPreparedRequest) String() string { return proto.CompactTextString(m) } func (*RollbackPreparedRequest) ProtoMessage() {} func (*RollbackPreparedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{30} + return fileDescriptor_5c6ac9b241082464, []int{29} } func (m *RollbackPreparedRequest) XXX_Unmarshal(b []byte) error { @@ -2432,7 +2351,7 @@ func (m *RollbackPreparedResponse) Reset() { *m = RollbackPreparedRespon func (m *RollbackPreparedResponse) String() string { return proto.CompactTextString(m) } func (*RollbackPreparedResponse) ProtoMessage() {} func (*RollbackPreparedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{31} + return fileDescriptor_5c6ac9b241082464, []int{30} } func (m *RollbackPreparedResponse) XXX_Unmarshal(b []byte) error { @@ -2469,7 +2388,7 @@ func (m *CreateTransactionRequest) Reset() { *m = CreateTransactionReque func (m *CreateTransactionRequest) String() string { return proto.CompactTextString(m) } func (*CreateTransactionRequest) ProtoMessage() {} func (*CreateTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{32} + return fileDescriptor_5c6ac9b241082464, []int{31} } func (m *CreateTransactionRequest) XXX_Unmarshal(b []byte) error { @@ -2536,7 +2455,7 @@ func (m *CreateTransactionResponse) Reset() { *m = CreateTransactionResp func (m *CreateTransactionResponse) String() string { return proto.CompactTextString(m) } func (*CreateTransactionResponse) ProtoMessage() {} func (*CreateTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{33} + return fileDescriptor_5c6ac9b241082464, []int{32} } func (m *CreateTransactionResponse) XXX_Unmarshal(b []byte) error { @@ -2573,7 +2492,7 @@ func (m *StartCommitRequest) Reset() { *m = StartCommitRequest{} } func (m *StartCommitRequest) String() string { return proto.CompactTextString(m) } func (*StartCommitRequest) ProtoMessage() {} func (*StartCommitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{34} + return fileDescriptor_5c6ac9b241082464, []int{33} } func (m *StartCommitRequest) XXX_Unmarshal(b []byte) error { @@ -2640,7 +2559,7 @@ func (m *StartCommitResponse) Reset() { *m = StartCommitResponse{} } func (m *StartCommitResponse) String() string { return proto.CompactTextString(m) } func (*StartCommitResponse) ProtoMessage() {} func (*StartCommitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{35} + return fileDescriptor_5c6ac9b241082464, []int{34} } func (m *StartCommitResponse) XXX_Unmarshal(b []byte) error { @@ -2677,7 +2596,7 @@ func (m *SetRollbackRequest) Reset() { *m = SetRollbackRequest{} } func (m *SetRollbackRequest) String() string { return proto.CompactTextString(m) } func (*SetRollbackRequest) ProtoMessage() {} func (*SetRollbackRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{36} + return fileDescriptor_5c6ac9b241082464, []int{35} } func (m *SetRollbackRequest) XXX_Unmarshal(b []byte) error { @@ -2744,7 +2663,7 @@ func (m *SetRollbackResponse) Reset() { *m = SetRollbackResponse{} } func (m *SetRollbackResponse) String() string { return proto.CompactTextString(m) } func (*SetRollbackResponse) ProtoMessage() {} func (*SetRollbackResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{37} + return fileDescriptor_5c6ac9b241082464, []int{36} } func (m *SetRollbackResponse) XXX_Unmarshal(b []byte) error { @@ -2780,7 +2699,7 @@ func (m *ConcludeTransactionRequest) Reset() { *m = ConcludeTransactionR func (m *ConcludeTransactionRequest) String() string { return proto.CompactTextString(m) } func (*ConcludeTransactionRequest) ProtoMessage() {} func (*ConcludeTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{38} + return fileDescriptor_5c6ac9b241082464, []int{37} } func (m *ConcludeTransactionRequest) XXX_Unmarshal(b []byte) error { @@ -2840,7 +2759,7 @@ func (m *ConcludeTransactionResponse) Reset() { *m = ConcludeTransaction func (m *ConcludeTransactionResponse) String() string { return proto.CompactTextString(m) } func (*ConcludeTransactionResponse) ProtoMessage() {} func (*ConcludeTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{39} + return fileDescriptor_5c6ac9b241082464, []int{38} } func (m *ConcludeTransactionResponse) XXX_Unmarshal(b []byte) error { @@ -2876,7 +2795,7 @@ func (m *ReadTransactionRequest) Reset() { *m = ReadTransactionRequest{} func (m *ReadTransactionRequest) String() string { return proto.CompactTextString(m) } func (*ReadTransactionRequest) ProtoMessage() {} func (*ReadTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{40} + return fileDescriptor_5c6ac9b241082464, []int{39} } func (m *ReadTransactionRequest) XXX_Unmarshal(b []byte) error { @@ -2937,7 +2856,7 @@ func (m *ReadTransactionResponse) Reset() { *m = ReadTransactionResponse func (m *ReadTransactionResponse) String() string { return proto.CompactTextString(m) } func (*ReadTransactionResponse) ProtoMessage() {} func (*ReadTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{41} + return fileDescriptor_5c6ac9b241082464, []int{40} } func (m *ReadTransactionResponse) XXX_Unmarshal(b []byte) error { @@ -2981,7 +2900,7 @@ func (m *BeginExecuteRequest) Reset() { *m = BeginExecuteRequest{} } func (m *BeginExecuteRequest) String() string { return proto.CompactTextString(m) } func (*BeginExecuteRequest) ProtoMessage() {} func (*BeginExecuteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{42} + return fileDescriptor_5c6ac9b241082464, []int{41} } func (m *BeginExecuteRequest) XXX_Unmarshal(b []byte) error { @@ -3055,7 +2974,7 @@ func (m *BeginExecuteResponse) Reset() { *m = BeginExecuteResponse{} } func (m *BeginExecuteResponse) String() string { return proto.CompactTextString(m) } func (*BeginExecuteResponse) ProtoMessage() {} func (*BeginExecuteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{43} + return fileDescriptor_5c6ac9b241082464, []int{42} } func (m *BeginExecuteResponse) XXX_Unmarshal(b []byte) error { @@ -3114,7 +3033,7 @@ func (m *BeginExecuteBatchRequest) Reset() { *m = BeginExecuteBatchReque func (m *BeginExecuteBatchRequest) String() string { return proto.CompactTextString(m) } func (*BeginExecuteBatchRequest) ProtoMessage() {} func (*BeginExecuteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{44} + return fileDescriptor_5c6ac9b241082464, []int{43} } func (m *BeginExecuteBatchRequest) XXX_Unmarshal(b []byte) error { @@ -3195,7 +3114,7 @@ func (m *BeginExecuteBatchResponse) Reset() { *m = BeginExecuteBatchResp func (m *BeginExecuteBatchResponse) String() string { return proto.CompactTextString(m) } func (*BeginExecuteBatchResponse) ProtoMessage() {} func (*BeginExecuteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{45} + return fileDescriptor_5c6ac9b241082464, []int{44} } func (m *BeginExecuteBatchResponse) XXX_Unmarshal(b []byte) error { @@ -3253,7 +3172,7 @@ func (m *MessageStreamRequest) Reset() { *m = MessageStreamRequest{} } func (m *MessageStreamRequest) String() string { return proto.CompactTextString(m) } func (*MessageStreamRequest) ProtoMessage() {} func (*MessageStreamRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{46} + return fileDescriptor_5c6ac9b241082464, []int{45} } func (m *MessageStreamRequest) XXX_Unmarshal(b []byte) error { @@ -3314,7 +3233,7 @@ func (m *MessageStreamResponse) Reset() { *m = MessageStreamResponse{} } func (m *MessageStreamResponse) String() string { return proto.CompactTextString(m) } func (*MessageStreamResponse) ProtoMessage() {} func (*MessageStreamResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{47} + return fileDescriptor_5c6ac9b241082464, []int{46} } func (m *MessageStreamResponse) XXX_Unmarshal(b []byte) error { @@ -3359,7 +3278,7 @@ func (m *MessageAckRequest) Reset() { *m = MessageAckRequest{} } func (m *MessageAckRequest) String() string { return proto.CompactTextString(m) } func (*MessageAckRequest) ProtoMessage() {} func (*MessageAckRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{48} + return fileDescriptor_5c6ac9b241082464, []int{47} } func (m *MessageAckRequest) XXX_Unmarshal(b []byte) error { @@ -3430,7 +3349,7 @@ func (m *MessageAckResponse) Reset() { *m = MessageAckResponse{} } func (m *MessageAckResponse) String() string { return proto.CompactTextString(m) } func (*MessageAckResponse) ProtoMessage() {} func (*MessageAckResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{49} + return fileDescriptor_5c6ac9b241082464, []int{48} } func (m *MessageAckResponse) XXX_Unmarshal(b []byte) error { @@ -3469,7 +3388,7 @@ func (m *StreamHealthRequest) Reset() { *m = StreamHealthRequest{} } func (m *StreamHealthRequest) String() string { return proto.CompactTextString(m) } func (*StreamHealthRequest) ProtoMessage() {} func (*StreamHealthRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{50} + return fileDescriptor_5c6ac9b241082464, []int{49} } func (m *StreamHealthRequest) XXX_Unmarshal(b []byte) error { @@ -3529,7 +3448,7 @@ func (m *RealtimeStats) Reset() { *m = RealtimeStats{} } func (m *RealtimeStats) String() string { return proto.CompactTextString(m) } func (*RealtimeStats) ProtoMessage() {} func (*RealtimeStats) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{51} + return fileDescriptor_5c6ac9b241082464, []int{50} } func (m *RealtimeStats) XXX_Unmarshal(b []byte) error { @@ -3618,7 +3537,7 @@ func (m *AggregateStats) Reset() { *m = AggregateStats{} } func (m *AggregateStats) String() string { return proto.CompactTextString(m) } func (*AggregateStats) ProtoMessage() {} func (*AggregateStats) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{52} + return fileDescriptor_5c6ac9b241082464, []int{51} } func (m *AggregateStats) XXX_Unmarshal(b []byte) error { @@ -3724,7 +3643,7 @@ func (m *StreamHealthResponse) Reset() { *m = StreamHealthResponse{} } func (m *StreamHealthResponse) String() string { return proto.CompactTextString(m) } func (*StreamHealthResponse) ProtoMessage() {} func (*StreamHealthResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{53} + return fileDescriptor_5c6ac9b241082464, []int{52} } func (m *StreamHealthResponse) XXX_Unmarshal(b []byte) error { @@ -3795,7 +3714,7 @@ func (m *TransactionMetadata) Reset() { *m = TransactionMetadata{} } func (m *TransactionMetadata) String() string { return proto.CompactTextString(m) } func (*TransactionMetadata) ProtoMessage() {} func (*TransactionMetadata) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{54} + return fileDescriptor_5c6ac9b241082464, []int{53} } func (m *TransactionMetadata) XXX_Unmarshal(b []byte) error { @@ -3863,7 +3782,6 @@ func init() { proto.RegisterType((*ExecuteOptions)(nil), "query.ExecuteOptions") proto.RegisterType((*Field)(nil), "query.Field") proto.RegisterType((*Row)(nil), "query.Row") - proto.RegisterType((*ResultExtras)(nil), "query.ResultExtras") proto.RegisterType((*QueryResult)(nil), "query.QueryResult") proto.RegisterType((*QueryWarning)(nil), "query.QueryWarning") proto.RegisterType((*StreamEvent)(nil), "query.StreamEvent") @@ -3915,195 +3833,192 @@ func init() { func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } var fileDescriptor_5c6ac9b241082464 = []byte{ - // 3039 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcd, 0x73, 0xdb, 0xd6, - 0xb5, 0x37, 0x48, 0x8a, 0x22, 0x0f, 0x45, 0xea, 0xea, 0x4a, 0xb2, 0x69, 0x39, 0x1f, 0x0a, 0x12, - 0x27, 0x7e, 0xca, 0x7b, 0xb2, 0x23, 0x3b, 0x7e, 0x7e, 0x49, 0xde, 0x7b, 0x86, 0x28, 0xc8, 0x61, - 0x4c, 0x82, 0xf4, 0x25, 0x68, 0xc7, 0x9e, 0xcc, 0x60, 0x20, 0xf2, 0x9a, 0xc2, 0x08, 0x24, 0x68, - 0x00, 0x92, 0xad, 0x9d, 0xdb, 0x34, 0xfd, 0xfe, 0x48, 0x3f, 0xd3, 0xb4, 0xd3, 0xb4, 0x33, 0x5d, - 0x74, 0xba, 0xe9, 0xdf, 0xd0, 0xe9, 0xa2, 0xcb, 0xee, 0xdb, 0x2e, 0xba, 0xea, 0x74, 0x97, 0xe9, - 0xaa, 0x8b, 0x2e, 0x3a, 0x9d, 0xfb, 0x01, 0x10, 0x94, 0xe8, 0x8f, 0xb8, 0xdd, 0xd8, 0xc9, 0xee, - 0x9e, 0x8f, 0x7b, 0xcf, 0x3d, 0xbf, 0x73, 0x70, 0xee, 0x01, 0x2e, 0xa0, 0x70, 0x6b, 0x97, 0xfa, - 0xfb, 0xab, 0x43, 0xdf, 0x0b, 0x3d, 0x3c, 0xc5, 0x89, 0xa5, 0x52, 0xe8, 0x0d, 0xbd, 0xae, 0x1d, - 0xda, 0x82, 0xbd, 0x54, 0xd8, 0x0b, 0xfd, 0x61, 0x47, 0x10, 0xea, 0x7b, 0x0a, 0x64, 0x4d, 0xdb, - 0xef, 0xd1, 0x10, 0x2f, 0x41, 0x6e, 0x87, 0xee, 0x07, 0x43, 0xbb, 0x43, 0xcb, 0xca, 0xb2, 0x72, - 0x2a, 0x4f, 0x62, 0x1a, 0x2f, 0xc0, 0x54, 0xb0, 0x6d, 0xfb, 0xdd, 0x72, 0x8a, 0x0b, 0x04, 0x81, - 0x5f, 0x85, 0x42, 0x68, 0x6f, 0xb9, 0x34, 0xb4, 0xc2, 0xfd, 0x21, 0x2d, 0xa7, 0x97, 0x95, 0x53, - 0xa5, 0xb5, 0x85, 0xd5, 0xd8, 0x9e, 0xc9, 0x85, 0xe6, 0xfe, 0x90, 0x12, 0x08, 0xe3, 0x31, 0xc6, - 0x90, 0xe9, 0x50, 0xd7, 0x2d, 0x67, 0xf8, 0x5a, 0x7c, 0xac, 0x6e, 0x40, 0xe9, 0xaa, 0x79, 0xc9, - 0x0e, 0x69, 0xc5, 0x76, 0x5d, 0xea, 0x57, 0x37, 0xd8, 0x76, 0x76, 0x03, 0xea, 0x0f, 0xec, 0x7e, - 0xbc, 0x9d, 0x88, 0xc6, 0x47, 0x21, 0xdb, 0xf3, 0xbd, 0xdd, 0x61, 0x50, 0x4e, 0x2d, 0xa7, 0x4f, - 0xe5, 0x89, 0xa4, 0xd4, 0x77, 0x00, 0xf4, 0x3d, 0x3a, 0x08, 0x4d, 0x6f, 0x87, 0x0e, 0xf0, 0x53, - 0x90, 0x0f, 0x9d, 0x3e, 0x0d, 0x42, 0xbb, 0x3f, 0xe4, 0x4b, 0xa4, 0xc9, 0x88, 0x71, 0x0f, 0x97, - 0x96, 0x20, 0x37, 0xf4, 0x02, 0x27, 0x74, 0xbc, 0x01, 0xf7, 0x27, 0x4f, 0x62, 0x5a, 0xfd, 0x3f, - 0x98, 0xba, 0x6a, 0xbb, 0xbb, 0x14, 0x3f, 0x0b, 0x19, 0xee, 0xb0, 0xc2, 0x1d, 0x2e, 0xac, 0x0a, - 0xd0, 0xb9, 0x9f, 0x5c, 0xc0, 0xd6, 0xde, 0x63, 0x9a, 0x7c, 0xed, 0x19, 0x22, 0x08, 0x75, 0x07, - 0x66, 0xd6, 0x9d, 0x41, 0xf7, 0xaa, 0xed, 0x3b, 0x0c, 0x8c, 0x47, 0x5c, 0x06, 0xbf, 0x00, 0x59, - 0x3e, 0x08, 0xca, 0xe9, 0xe5, 0xf4, 0xa9, 0xc2, 0xda, 0x8c, 0x9c, 0xc8, 0xf7, 0x46, 0xa4, 0x4c, - 0xfd, 0x8d, 0x02, 0xb0, 0xee, 0xed, 0x0e, 0xba, 0x57, 0x98, 0x10, 0x23, 0x48, 0x07, 0xb7, 0x5c, - 0x09, 0x24, 0x1b, 0xe2, 0xcb, 0x50, 0xda, 0x72, 0x06, 0x5d, 0x6b, 0x4f, 0x6e, 0x47, 0x60, 0x59, - 0x58, 0x7b, 0x41, 0x2e, 0x37, 0x9a, 0xbc, 0x9a, 0xdc, 0x75, 0xa0, 0x0f, 0x42, 0x7f, 0x9f, 0x14, - 0xb7, 0x92, 0xbc, 0xa5, 0x36, 0xe0, 0xc3, 0x4a, 0xcc, 0xe8, 0x0e, 0xdd, 0x8f, 0x8c, 0xee, 0xd0, - 0x7d, 0xfc, 0x1f, 0x49, 0x8f, 0x0a, 0x6b, 0xf3, 0x91, 0xad, 0xc4, 0x5c, 0xe9, 0xe6, 0x6b, 0xa9, - 0x0b, 0x8a, 0xfa, 0xd3, 0x2c, 0x94, 0xf4, 0x3b, 0xb4, 0xb3, 0x1b, 0xd2, 0xc6, 0x90, 0xc5, 0x20, - 0xc0, 0xab, 0x30, 0xef, 0x0c, 0x3a, 0xee, 0x6e, 0x97, 0x5a, 0x94, 0x85, 0xda, 0x0a, 0x59, 0xac, - 0xf9, 0x7a, 0x39, 0x32, 0x27, 0x45, 0x89, 0x24, 0xd0, 0x60, 0xbe, 0xe3, 0xf5, 0x87, 0xb6, 0x3f, - 0xae, 0x9f, 0xe6, 0xf6, 0xe7, 0xa4, 0xfd, 0x91, 0x3e, 0x99, 0x93, 0xda, 0x89, 0x25, 0xea, 0x30, - 0x2b, 0xd7, 0xed, 0x5a, 0x37, 0x1d, 0xea, 0x76, 0x03, 0x9e, 0xba, 0xa5, 0x18, 0xaa, 0xf1, 0x2d, - 0xae, 0x56, 0xa5, 0xf2, 0x26, 0xd7, 0x25, 0x25, 0x67, 0x8c, 0xc6, 0x2b, 0x30, 0xd7, 0x71, 0x1d, - 0xb6, 0x95, 0x9b, 0x0c, 0x62, 0xcb, 0xf7, 0x6e, 0x07, 0xe5, 0x29, 0xbe, 0xff, 0x59, 0x21, 0xd8, - 0x64, 0x7c, 0xe2, 0xdd, 0x0e, 0xf0, 0x6b, 0x90, 0xbb, 0xed, 0xf9, 0x3b, 0xae, 0x67, 0x77, 0xcb, - 0x59, 0x6e, 0xf3, 0x99, 0xc9, 0x36, 0xaf, 0x49, 0x2d, 0x12, 0xeb, 0xe3, 0x53, 0x80, 0x82, 0x5b, - 0xae, 0x15, 0x50, 0x97, 0x76, 0x42, 0xcb, 0x75, 0xfa, 0x4e, 0x58, 0xce, 0xf1, 0xa7, 0xa0, 0x14, - 0xdc, 0x72, 0x5b, 0x9c, 0x5d, 0x63, 0x5c, 0x6c, 0xc1, 0x62, 0xe8, 0xdb, 0x83, 0xc0, 0xee, 0xb0, - 0xc5, 0x2c, 0x27, 0xf0, 0x5c, 0x9b, 0x3f, 0x01, 0x79, 0x6e, 0x72, 0x65, 0xb2, 0x49, 0x73, 0x34, - 0xa5, 0x1a, 0xcd, 0x20, 0x0b, 0xe1, 0x04, 0x2e, 0x7e, 0x05, 0x16, 0x83, 0x1d, 0x67, 0x68, 0xf1, - 0x75, 0xac, 0xa1, 0x6b, 0x0f, 0xac, 0x8e, 0xdd, 0xd9, 0xa6, 0x65, 0xe0, 0x6e, 0x63, 0x26, 0xe4, - 0xa9, 0xd6, 0x74, 0xed, 0x41, 0x85, 0x49, 0xd4, 0xd7, 0xa1, 0x34, 0x8e, 0x23, 0x9e, 0x83, 0xa2, - 0x79, 0xbd, 0xa9, 0x5b, 0x9a, 0xb1, 0x61, 0x19, 0x5a, 0x5d, 0x47, 0x47, 0x70, 0x11, 0xf2, 0x9c, - 0xd5, 0x30, 0x6a, 0xd7, 0x91, 0x82, 0xa7, 0x21, 0xad, 0xd5, 0x6a, 0x28, 0xa5, 0x5e, 0x80, 0x5c, - 0x04, 0x08, 0x9e, 0x85, 0x42, 0xdb, 0x68, 0x35, 0xf5, 0x4a, 0x75, 0xb3, 0xaa, 0x6f, 0xa0, 0x23, - 0x38, 0x07, 0x99, 0x46, 0xcd, 0x6c, 0x22, 0x45, 0x8c, 0xb4, 0x26, 0x4a, 0xb1, 0x99, 0x1b, 0xeb, - 0x1a, 0x4a, 0xab, 0xbf, 0x50, 0x60, 0x61, 0x92, 0x63, 0xb8, 0x00, 0xd3, 0x1b, 0xfa, 0xa6, 0xd6, - 0xae, 0x99, 0xe8, 0x08, 0x9e, 0x87, 0x59, 0xa2, 0x37, 0x75, 0xcd, 0xd4, 0xd6, 0x6b, 0xba, 0x45, - 0x74, 0x6d, 0x03, 0x29, 0x18, 0x43, 0x89, 0x8d, 0xac, 0x4a, 0xa3, 0x5e, 0xaf, 0x9a, 0xa6, 0xbe, - 0x81, 0x52, 0x78, 0x01, 0x10, 0xe7, 0xb5, 0x8d, 0x11, 0x37, 0x8d, 0x11, 0xcc, 0xb4, 0x74, 0x52, - 0xd5, 0x6a, 0xd5, 0x1b, 0x6c, 0x01, 0x94, 0xc1, 0xcf, 0xc1, 0xd3, 0x95, 0x86, 0xd1, 0xaa, 0xb6, - 0x4c, 0xdd, 0x30, 0xad, 0x96, 0xa1, 0x35, 0x5b, 0x6f, 0x36, 0x4c, 0xbe, 0xb2, 0x70, 0x6e, 0x0a, - 0x97, 0x00, 0xb4, 0xb6, 0xd9, 0x10, 0xeb, 0xa0, 0xec, 0x5b, 0x99, 0x9c, 0x82, 0x52, 0xea, 0x07, - 0x29, 0x98, 0xe2, 0xf8, 0xb0, 0xaa, 0x9a, 0xa8, 0x95, 0x7c, 0x1c, 0x57, 0x98, 0xd4, 0x7d, 0x2a, - 0x0c, 0x2f, 0xcc, 0xb2, 0xd6, 0x09, 0x02, 0x9f, 0x80, 0xbc, 0xe7, 0xf7, 0x2c, 0x21, 0x11, 0x55, - 0x3a, 0xe7, 0xf9, 0x3d, 0x5e, 0xce, 0x59, 0x85, 0x64, 0xc5, 0x7d, 0xcb, 0x0e, 0x28, 0xcf, 0xda, - 0x3c, 0x89, 0x69, 0x7c, 0x1c, 0x98, 0x9e, 0xc5, 0xf7, 0x91, 0xe5, 0xb2, 0x69, 0xcf, 0xef, 0x19, - 0x6c, 0x2b, 0xcf, 0x43, 0xb1, 0xe3, 0xb9, 0xbb, 0xfd, 0x81, 0xe5, 0xd2, 0x41, 0x2f, 0xdc, 0x2e, - 0x4f, 0x2f, 0x2b, 0xa7, 0x8a, 0x64, 0x46, 0x30, 0x6b, 0x9c, 0x87, 0xcb, 0x30, 0xdd, 0xd9, 0xb6, - 0xfd, 0x80, 0x8a, 0x4c, 0x2d, 0x92, 0x88, 0xe4, 0x56, 0x69, 0xc7, 0xe9, 0xdb, 0x6e, 0xc0, 0xb3, - 0xb2, 0x48, 0x62, 0x9a, 0x39, 0x71, 0xd3, 0xb5, 0x7b, 0x01, 0xcf, 0xa6, 0x22, 0x11, 0x84, 0xfa, - 0xdf, 0x90, 0x26, 0xde, 0x6d, 0xb6, 0xa4, 0x30, 0x18, 0x94, 0x95, 0xe5, 0xf4, 0x29, 0x4c, 0x22, - 0x92, 0x1d, 0x22, 0xb2, 0x8e, 0x8a, 0xf2, 0x1a, 0x55, 0xce, 0x77, 0x60, 0x86, 0xd0, 0x60, 0xd7, - 0x0d, 0xf5, 0x3b, 0xa1, 0x6f, 0x07, 0x78, 0x0d, 0x0a, 0xc9, 0xca, 0xa1, 0xdc, 0xab, 0x72, 0x00, - 0x1d, 0x95, 0x8c, 0x32, 0x4c, 0xdf, 0xf4, 0x69, 0xb0, 0x4d, 0x7d, 0x59, 0x99, 0x22, 0x92, 0xd5, - 0xe5, 0x02, 0x4f, 0x75, 0x61, 0x83, 0x55, 0x73, 0x59, 0x53, 0x94, 0xb1, 0x6a, 0xce, 0x83, 0x4a, - 0xa4, 0x8c, 0xa1, 0xc7, 0xca, 0x84, 0x65, 0xdf, 0xbc, 0x49, 0x3b, 0x21, 0x15, 0x87, 0x56, 0x86, - 0xcc, 0x30, 0xa6, 0x26, 0x79, 0x2c, 0x6c, 0xce, 0x20, 0xa0, 0x7e, 0x68, 0x39, 0x5d, 0x1e, 0xd0, - 0x0c, 0xc9, 0x09, 0x46, 0xb5, 0x8b, 0x9f, 0x81, 0x0c, 0x2f, 0x34, 0x19, 0x6e, 0x05, 0xa4, 0x15, - 0xe2, 0xdd, 0x26, 0x9c, 0x8f, 0x5f, 0x86, 0x2c, 0xe5, 0xfe, 0xf2, 0xa0, 0x8e, 0x4a, 0x73, 0x12, - 0x0a, 0x22, 0x55, 0xd4, 0x37, 0x60, 0x86, 0xfb, 0x70, 0xcd, 0xf6, 0x07, 0xce, 0xa0, 0xc7, 0x4f, - 0x74, 0xaf, 0x2b, 0x72, 0xaf, 0x48, 0xf8, 0x98, 0x41, 0xd0, 0xa7, 0x41, 0x60, 0xf7, 0xa8, 0x3c, - 0x61, 0x23, 0x52, 0xfd, 0x59, 0x1a, 0x0a, 0xad, 0xd0, 0xa7, 0x76, 0x9f, 0xa3, 0x87, 0xdf, 0x00, - 0x08, 0x42, 0x3b, 0xa4, 0x7d, 0x3a, 0x08, 0x23, 0x18, 0x9e, 0x92, 0xe6, 0x13, 0x7a, 0xab, 0xad, - 0x48, 0x89, 0x24, 0xf4, 0x0f, 0x86, 0x27, 0xf5, 0x10, 0xe1, 0x59, 0xfa, 0x28, 0x05, 0xf9, 0x78, - 0x35, 0xac, 0x41, 0xae, 0x63, 0x87, 0xb4, 0xe7, 0xf9, 0xfb, 0xf2, 0x2c, 0x3e, 0x79, 0x3f, 0xeb, - 0xab, 0x15, 0xa9, 0x4c, 0xe2, 0x69, 0xf8, 0x69, 0x10, 0x0d, 0x8e, 0x48, 0x7d, 0xe1, 0x6f, 0x9e, - 0x73, 0x78, 0xf2, 0xbf, 0x06, 0x78, 0xe8, 0x3b, 0x7d, 0xdb, 0xdf, 0xb7, 0x76, 0xe8, 0x7e, 0x74, - 0x88, 0xa4, 0x27, 0x04, 0x1c, 0x49, 0xbd, 0xcb, 0x74, 0x5f, 0x96, 0xbd, 0x0b, 0xe3, 0x73, 0x65, - 0xca, 0x1e, 0x0e, 0x63, 0x62, 0x26, 0xef, 0x04, 0x82, 0xe8, 0xcc, 0x9f, 0xe2, 0xd9, 0xcd, 0x86, - 0xea, 0x4b, 0x90, 0x8b, 0x36, 0x8f, 0xf3, 0x30, 0xa5, 0xfb, 0xbe, 0xe7, 0xa3, 0x23, 0xbc, 0xfa, - 0xd5, 0x6b, 0xa2, 0x80, 0x6e, 0x6c, 0xb0, 0x02, 0xfa, 0xeb, 0x54, 0x7c, 0xf0, 0x12, 0x7a, 0x6b, - 0x97, 0x06, 0x21, 0xfe, 0x7f, 0x98, 0xa7, 0x3c, 0xd3, 0x9c, 0x3d, 0x6a, 0x75, 0x78, 0x97, 0xc6, - 0xf2, 0x4c, 0x3c, 0x0e, 0xb3, 0xab, 0xa2, 0xa9, 0x8c, 0xba, 0x37, 0x32, 0x17, 0xeb, 0x4a, 0x56, - 0x17, 0xeb, 0x30, 0xef, 0xf4, 0xfb, 0xb4, 0xeb, 0xd8, 0x61, 0x72, 0x01, 0x11, 0xb0, 0xc5, 0xa8, - 0x89, 0x19, 0x6b, 0x02, 0xc9, 0x5c, 0x3c, 0x23, 0x5e, 0xe6, 0x24, 0x64, 0x43, 0xde, 0xb0, 0xca, - 0x33, 0xbc, 0x18, 0x55, 0x35, 0xce, 0x24, 0x52, 0x88, 0x5f, 0x02, 0xd1, 0xfe, 0xf2, 0xfa, 0x35, - 0x4a, 0x88, 0x51, 0x57, 0x43, 0x84, 0x1c, 0x9f, 0x84, 0xd2, 0xd8, 0xe1, 0xd7, 0xe5, 0x80, 0xa5, - 0x49, 0x31, 0x79, 0x92, 0x75, 0xf1, 0x69, 0x98, 0xf6, 0xc4, 0xc1, 0xc7, 0x2b, 0xdb, 0x68, 0xc7, - 0xe3, 0xa7, 0x22, 0x89, 0xb4, 0xd4, 0xff, 0x85, 0xd9, 0x18, 0xc1, 0x60, 0xe8, 0x0d, 0x02, 0x8a, - 0x57, 0x20, 0xeb, 0xf3, 0xc7, 0x49, 0xa2, 0x86, 0xe5, 0x12, 0x89, 0x7a, 0x40, 0xa4, 0x86, 0xda, - 0x85, 0x59, 0xc1, 0xb9, 0xe6, 0x84, 0xdb, 0x3c, 0x50, 0xf8, 0x24, 0x4c, 0x51, 0x36, 0x38, 0x80, - 0x39, 0x69, 0x56, 0xb8, 0x9c, 0x08, 0x69, 0xc2, 0x4a, 0xea, 0x81, 0x56, 0xfe, 0x9a, 0x82, 0x79, - 0xb9, 0xcb, 0x75, 0x3b, 0xec, 0x6c, 0x3f, 0xa6, 0xc1, 0x7e, 0x19, 0xa6, 0x19, 0xdf, 0x89, 0x1f, - 0x8c, 0x09, 0xe1, 0x8e, 0x34, 0x58, 0xc0, 0xed, 0xc0, 0x4a, 0x44, 0x57, 0x36, 0x5f, 0x45, 0x3b, - 0x48, 0x9c, 0xfc, 0x13, 0xf2, 0x22, 0xfb, 0x80, 0xbc, 0x98, 0x7e, 0xa8, 0xbc, 0xd8, 0x80, 0x85, - 0x71, 0xc4, 0x65, 0x72, 0xfc, 0x27, 0x4c, 0x8b, 0xa0, 0x44, 0x25, 0x70, 0x52, 0xdc, 0x22, 0x15, - 0xf5, 0xb7, 0x29, 0x58, 0x90, 0xd5, 0xe9, 0xd3, 0xf1, 0x98, 0x26, 0x70, 0x9e, 0x7a, 0x18, 0x9c, - 0x1f, 0x32, 0x7e, 0x6a, 0x05, 0x16, 0x0f, 0xe0, 0xf8, 0x08, 0x0f, 0xeb, 0xc7, 0x0a, 0xcc, 0xac, - 0xd3, 0x9e, 0x33, 0x78, 0x4c, 0xa3, 0x90, 0x00, 0x37, 0xf3, 0x50, 0x49, 0x7c, 0x1e, 0x8a, 0xd2, - 0x5f, 0x89, 0xd6, 0x61, 0xb4, 0x95, 0x49, 0x68, 0xff, 0x59, 0x81, 0x62, 0xc5, 0xeb, 0xf7, 0x9d, - 0xf0, 0x31, 0x45, 0xea, 0xb0, 0x9f, 0x99, 0x49, 0x7e, 0x22, 0x28, 0x45, 0x6e, 0x0a, 0x80, 0xd4, - 0xbf, 0x28, 0x30, 0x4b, 0x3c, 0xd7, 0xdd, 0xb2, 0x3b, 0x3b, 0x4f, 0xb6, 0xef, 0x18, 0xd0, 0xc8, - 0x51, 0xe9, 0xfd, 0xdf, 0x15, 0x28, 0x35, 0x7d, 0xca, 0x5e, 0xac, 0x9f, 0x68, 0xe7, 0x59, 0x27, - 0xdc, 0x0d, 0x65, 0x0f, 0x91, 0x27, 0x7c, 0xac, 0xce, 0xc1, 0x6c, 0xec, 0xbb, 0xc4, 0xe3, 0x0f, - 0x0a, 0x2c, 0x8a, 0x04, 0x91, 0x92, 0xee, 0x63, 0x0a, 0x4b, 0xe4, 0x6f, 0x26, 0xe1, 0x6f, 0x19, - 0x8e, 0x1e, 0xf4, 0x4d, 0xba, 0xfd, 0x6e, 0x0a, 0x8e, 0x45, 0xb9, 0xf1, 0x98, 0x3b, 0xfe, 0x2f, - 0xe4, 0xc3, 0x12, 0x94, 0x0f, 0x83, 0x20, 0x11, 0x7a, 0x3f, 0x05, 0xe5, 0x8a, 0x4f, 0xed, 0x90, - 0x26, 0x7a, 0x91, 0x27, 0x27, 0x37, 0xf0, 0x2b, 0x30, 0x33, 0xb4, 0xfd, 0xd0, 0xe9, 0x38, 0x43, - 0x9b, 0xbd, 0xed, 0x4d, 0xf1, 0x56, 0xe7, 0xc0, 0x02, 0x63, 0x2a, 0xea, 0x09, 0x38, 0x3e, 0x01, - 0x11, 0x89, 0xd7, 0x3f, 0x14, 0xc0, 0xad, 0xd0, 0xf6, 0xc3, 0x4f, 0xc1, 0xa9, 0x32, 0x31, 0x99, - 0x16, 0x61, 0x7e, 0xcc, 0xff, 0x24, 0x2e, 0x34, 0xfc, 0x54, 0x9c, 0x38, 0xf7, 0xc4, 0x25, 0xe9, - 0xbf, 0xc4, 0xe5, 0x4f, 0x0a, 0x2c, 0x55, 0x3c, 0xf1, 0x61, 0xf1, 0x89, 0x7c, 0xc2, 0xd4, 0xa7, - 0xe1, 0xc4, 0x44, 0x07, 0x25, 0x00, 0x7f, 0x54, 0xe0, 0x28, 0xa1, 0x76, 0xf7, 0xc9, 0x74, 0xfe, - 0x0a, 0x1c, 0x3b, 0xe4, 0x9c, 0xec, 0x50, 0xcf, 0x43, 0xae, 0x4f, 0x43, 0xbb, 0x6b, 0x87, 0xb6, - 0x74, 0x69, 0x29, 0x5a, 0x77, 0xa4, 0x5d, 0x97, 0x1a, 0x24, 0xd6, 0x55, 0x3f, 0x4a, 0xc1, 0x3c, - 0xef, 0x75, 0x3f, 0x7b, 0xd1, 0x9a, 0xfc, 0x2e, 0xf0, 0xbe, 0x02, 0x0b, 0xe3, 0x00, 0xc5, 0xef, - 0x04, 0xff, 0xee, 0xef, 0x15, 0x13, 0x0a, 0x42, 0x7a, 0x52, 0x0b, 0xfa, 0xbb, 0x14, 0x94, 0x93, - 0x5b, 0xfa, 0xec, 0xdb, 0xc6, 0xf8, 0xb7, 0x8d, 0x4f, 0xfc, 0x31, 0xeb, 0x03, 0x05, 0x8e, 0x4f, - 0x00, 0xf4, 0x93, 0x05, 0x3a, 0xf1, 0x85, 0x23, 0xf5, 0xc0, 0x2f, 0x1c, 0x0f, 0x1b, 0xea, 0xdf, - 0x2b, 0xb0, 0x50, 0x17, 0x1f, 0x96, 0xc5, 0x7b, 0xfc, 0xe3, 0x5b, 0xcd, 0xf8, 0xb7, 0xe3, 0xcc, - 0xe8, 0xfa, 0x46, 0xad, 0xc0, 0xe2, 0x01, 0xd7, 0x1e, 0xe1, 0xdb, 0xc4, 0xdf, 0x14, 0x98, 0x93, - 0xab, 0x68, 0x8f, 0x6d, 0x23, 0x30, 0x01, 0x1d, 0xfc, 0x0c, 0xa4, 0x9d, 0x6e, 0xd4, 0x41, 0x8e, - 0x5f, 0x82, 0x33, 0x81, 0x7a, 0x11, 0x70, 0xd2, 0xef, 0x47, 0x80, 0x8e, 0xf7, 0x56, 0x0c, 0xf8, - 0x37, 0xa9, 0xed, 0x86, 0x51, 0x01, 0x51, 0x7f, 0x9e, 0x82, 0x22, 0x61, 0x1c, 0xa7, 0x4f, 0x5b, - 0xa1, 0x1d, 0x06, 0xf8, 0x39, 0x98, 0xd9, 0xe6, 0x2a, 0xd6, 0xe8, 0x39, 0xc8, 0x93, 0x82, 0xe0, - 0x89, 0x8f, 0xb7, 0x6b, 0xb0, 0x18, 0xd0, 0x8e, 0x37, 0xe8, 0x06, 0xd6, 0x16, 0xdd, 0x76, 0x06, - 0x5d, 0xab, 0x6f, 0x07, 0xa1, 0xbc, 0x1f, 0x2a, 0x92, 0x79, 0x29, 0x5c, 0xe7, 0xb2, 0x3a, 0x17, - 0xe1, 0x33, 0xb0, 0xb0, 0xe5, 0x0c, 0x5c, 0xaf, 0x67, 0x0d, 0x5d, 0x7b, 0x9f, 0xfa, 0x81, 0xd5, - 0xf1, 0x76, 0x07, 0x02, 0xaa, 0x29, 0x82, 0x85, 0xac, 0x29, 0x44, 0x15, 0x26, 0xc1, 0x37, 0x60, - 0x65, 0xa2, 0x15, 0xeb, 0xa6, 0xe3, 0x86, 0xd4, 0xa7, 0x5d, 0xcb, 0xa7, 0x43, 0xd7, 0xe9, 0x88, - 0xeb, 0x5d, 0xd1, 0x4c, 0xbd, 0x38, 0xc1, 0xf4, 0xa6, 0x54, 0x27, 0x23, 0x6d, 0x7c, 0x02, 0xf2, - 0x9d, 0xe1, 0xae, 0xb5, 0xcb, 0xaf, 0x74, 0x58, 0x59, 0x51, 0x48, 0xae, 0x33, 0xdc, 0x6d, 0x33, - 0x1a, 0x23, 0x48, 0xdf, 0x1a, 0x8a, 0x6a, 0xa2, 0x10, 0x36, 0x54, 0x3f, 0x56, 0xa0, 0xa4, 0xf5, - 0x7a, 0x3e, 0xed, 0xd9, 0xa1, 0x84, 0xe9, 0x0c, 0x2c, 0x08, 0x48, 0xf6, 0x2d, 0xf9, 0xdf, 0x88, - 0xf0, 0x47, 0x11, 0xfe, 0x48, 0x99, 0xf8, 0x6b, 0x44, 0xf8, 0x73, 0x0e, 0x8e, 0xee, 0x0e, 0x26, - 0xce, 0x49, 0xf1, 0x39, 0x0b, 0xb1, 0x34, 0x39, 0xeb, 0x7f, 0xe0, 0xf8, 0x64, 0x14, 0xfa, 0x8e, - 0xb8, 0xf9, 0x2f, 0x92, 0xa3, 0x13, 0x9c, 0xae, 0x3b, 0x83, 0xfb, 0x4c, 0xb5, 0xef, 0x70, 0xbc, - 0xee, 0x31, 0xd5, 0xbe, 0xa3, 0xfe, 0x32, 0xfe, 0x24, 0x1b, 0xa5, 0x4b, 0x5c, 0x1e, 0xa3, 0x1c, - 0x57, 0xee, 0x97, 0xe3, 0x65, 0x98, 0x0e, 0xa8, 0xbf, 0xe7, 0x0c, 0x7a, 0xd1, 0x9d, 0xa1, 0x24, - 0x71, 0x0b, 0x5e, 0x94, 0xbe, 0xd3, 0x3b, 0x21, 0xf5, 0x07, 0xb6, 0xeb, 0xee, 0x5b, 0xe2, 0xcd, - 0x71, 0x10, 0xd2, 0xae, 0x35, 0xfa, 0xcb, 0x45, 0x94, 0xc8, 0xe7, 0x85, 0xb6, 0x1e, 0x2b, 0x93, - 0x58, 0xd7, 0x8c, 0xff, 0x7f, 0x79, 0x1d, 0x4a, 0xbe, 0x4c, 0x62, 0x2b, 0x60, 0xe1, 0x91, 0x9d, - 0xc1, 0x42, 0x7c, 0xf1, 0x97, 0xc8, 0x70, 0x52, 0xf4, 0xc7, 0x12, 0xfe, 0x02, 0xcc, 0xc8, 0x1d, - 0xd9, 0xae, 0x63, 0x8f, 0x3a, 0x85, 0x03, 0xbf, 0xfe, 0x68, 0x4c, 0x48, 0xe4, 0x4f, 0x42, 0x9c, - 0x78, 0x2b, 0x93, 0xcb, 0xa2, 0x69, 0xf5, 0x57, 0x0a, 0xcc, 0x4f, 0x68, 0xbb, 0xe2, 0x9e, 0x4e, - 0x49, 0xbc, 0x32, 0xfe, 0x17, 0x4c, 0xf1, 0xeb, 0x3e, 0x79, 0x8b, 0x7d, 0xec, 0x70, 0xd7, 0xc6, - 0xaf, 0xe6, 0x88, 0xd0, 0x62, 0xcf, 0x22, 0xf7, 0xa9, 0xc3, 0xdf, 0x19, 0xa3, 0x53, 0xa3, 0xc0, - 0x78, 0xe2, 0x35, 0xf2, 0xf0, 0x4b, 0x68, 0xe6, 0x81, 0x2f, 0xa1, 0x2b, 0xdf, 0x49, 0x43, 0xbe, - 0xbe, 0xdf, 0xba, 0xe5, 0x6e, 0xba, 0x76, 0x8f, 0xdf, 0x9d, 0xd5, 0x9b, 0xe6, 0x75, 0x74, 0x04, - 0xcf, 0x41, 0xd1, 0x68, 0x98, 0x96, 0xd1, 0xae, 0xd5, 0xac, 0xcd, 0x9a, 0x76, 0x09, 0x29, 0x18, - 0xc1, 0x4c, 0x93, 0x54, 0xad, 0xcb, 0xfa, 0x75, 0xc1, 0x49, 0xe1, 0x79, 0x98, 0x6d, 0x1b, 0xd5, - 0x2b, 0x6d, 0x7d, 0xc4, 0xcc, 0xe0, 0x45, 0x98, 0xab, 0xb7, 0x6b, 0x66, 0xb5, 0x59, 0x4b, 0xb0, - 0x73, 0xb8, 0x08, 0xf9, 0xf5, 0x5a, 0x63, 0x5d, 0x90, 0x88, 0xad, 0xdf, 0x36, 0x5a, 0xd5, 0x4b, - 0x86, 0xbe, 0x21, 0x58, 0xcb, 0x8c, 0x75, 0x43, 0x27, 0x8d, 0xcd, 0x6a, 0x64, 0xf2, 0x22, 0x46, - 0x50, 0x58, 0xaf, 0x1a, 0x1a, 0x91, 0xab, 0xdc, 0x55, 0x70, 0x09, 0xf2, 0xba, 0xd1, 0xae, 0x4b, - 0x3a, 0x85, 0xcb, 0x30, 0xaf, 0xb5, 0xcd, 0x86, 0x55, 0x35, 0x2a, 0x44, 0xaf, 0xeb, 0x86, 0x29, - 0x25, 0x19, 0x3c, 0x0f, 0x25, 0xb3, 0x5a, 0xd7, 0x5b, 0xa6, 0x56, 0x6f, 0x4a, 0x26, 0xdb, 0x45, - 0xae, 0xa5, 0x47, 0x3a, 0x08, 0x2f, 0xc1, 0xa2, 0xd1, 0xb0, 0xe4, 0x0f, 0x10, 0xd6, 0x55, 0xad, - 0xd6, 0xd6, 0xa5, 0x6c, 0x19, 0x1f, 0x03, 0xdc, 0x30, 0xac, 0x76, 0x73, 0x43, 0x33, 0x75, 0xcb, - 0x68, 0x5c, 0x93, 0x82, 0x8b, 0xb8, 0x04, 0xb9, 0xd1, 0x0e, 0xee, 0x32, 0x14, 0x8a, 0x4d, 0x8d, - 0x98, 0x23, 0x67, 0xef, 0xde, 0x65, 0x60, 0xc1, 0x25, 0xd2, 0x68, 0x37, 0x47, 0x6a, 0x73, 0x50, - 0x90, 0x60, 0x49, 0x56, 0x86, 0xb1, 0xd6, 0xab, 0x46, 0x25, 0xde, 0xdf, 0xdd, 0xdc, 0x52, 0x0a, - 0x29, 0x2b, 0x3b, 0x90, 0xe1, 0xe1, 0xc8, 0x41, 0xc6, 0x68, 0x18, 0x3a, 0x3a, 0x82, 0x67, 0x01, - 0xaa, 0xad, 0xaa, 0x61, 0xea, 0x97, 0x88, 0x56, 0x63, 0x6e, 0x73, 0x46, 0x04, 0x20, 0xf3, 0x76, - 0x06, 0xa6, 0xab, 0xad, 0xcd, 0x5a, 0x43, 0x33, 0xa5, 0x9b, 0xd5, 0xd6, 0x95, 0x76, 0xc3, 0x64, - 0x42, 0x84, 0x0b, 0x90, 0xad, 0xb6, 0x4c, 0xfd, 0x6d, 0x93, 0xf9, 0xc5, 0x65, 0x02, 0x55, 0x74, - 0xf7, 0xe2, 0xca, 0x87, 0x69, 0xc8, 0xf0, 0xdf, 0xd7, 0x8a, 0x90, 0xe7, 0xd1, 0x36, 0xaf, 0x37, - 0x99, 0xc9, 0x3c, 0x64, 0xaa, 0x86, 0x79, 0x01, 0x7d, 0x2e, 0x85, 0x01, 0xa6, 0xda, 0x7c, 0xfc, - 0xf9, 0x2c, 0x1b, 0x57, 0x0d, 0xf3, 0x95, 0xf3, 0xe8, 0xdd, 0x14, 0x5b, 0xb6, 0x2d, 0x88, 0x2f, - 0x44, 0x82, 0xb5, 0x73, 0xe8, 0xbd, 0x58, 0xb0, 0x76, 0x0e, 0x7d, 0x31, 0x12, 0x9c, 0x5d, 0x43, - 0x5f, 0x8a, 0x05, 0x67, 0xd7, 0xd0, 0x97, 0x23, 0xc1, 0xf9, 0x73, 0xe8, 0x2b, 0xb1, 0xe0, 0xfc, - 0x39, 0xf4, 0xd5, 0x2c, 0xf3, 0x85, 0x7b, 0x72, 0x76, 0x0d, 0x7d, 0x2d, 0x17, 0x53, 0xe7, 0xcf, - 0xa1, 0xaf, 0xe7, 0x58, 0xfc, 0xe3, 0xa8, 0xa2, 0x6f, 0x20, 0xb6, 0x4d, 0x16, 0x20, 0xf4, 0x4d, - 0x3e, 0x64, 0x22, 0xf4, 0x2d, 0xc4, 0x7c, 0x64, 0x5c, 0x4e, 0xbe, 0xcf, 0x25, 0xd7, 0x75, 0x8d, - 0xa0, 0x6f, 0x67, 0xc5, 0xff, 0x2e, 0x95, 0x6a, 0x5d, 0xab, 0x21, 0xcc, 0x67, 0x30, 0x54, 0xbe, - 0x7b, 0x86, 0x0d, 0x59, 0x7a, 0xa2, 0xef, 0x35, 0x99, 0xc1, 0xab, 0x1a, 0xa9, 0xbc, 0xa9, 0x11, - 0xf4, 0xfd, 0x33, 0xcc, 0xe0, 0x55, 0x8d, 0x48, 0xbc, 0x7e, 0xd0, 0x64, 0x8a, 0x5c, 0xf4, 0xc1, - 0x19, 0xb6, 0x69, 0xc9, 0xff, 0x61, 0x13, 0xe7, 0x20, 0xbd, 0x5e, 0x35, 0xd1, 0x87, 0xdc, 0x1a, - 0x4b, 0x51, 0xf4, 0x23, 0xc4, 0x98, 0x2d, 0xdd, 0x44, 0x3f, 0x66, 0xcc, 0x29, 0xb3, 0xdd, 0xac, - 0xe9, 0xe8, 0x29, 0xb6, 0xb9, 0x4b, 0x7a, 0xa3, 0xae, 0x9b, 0xe4, 0x3a, 0xfa, 0x09, 0x57, 0x7f, - 0xab, 0xd5, 0x30, 0xd0, 0x47, 0x08, 0x97, 0x00, 0xf4, 0xb7, 0x9b, 0x44, 0x6f, 0xb5, 0xaa, 0x0d, - 0x03, 0x3d, 0xbb, 0xb2, 0x09, 0xe8, 0x60, 0x39, 0x60, 0x0e, 0xb4, 0x8d, 0xcb, 0x46, 0xe3, 0x9a, - 0x81, 0x8e, 0x30, 0xa2, 0x49, 0xf4, 0xa6, 0x46, 0x74, 0xa4, 0x60, 0x80, 0xac, 0xfc, 0x8b, 0x26, - 0x85, 0x67, 0x20, 0x47, 0x1a, 0xb5, 0xda, 0xba, 0x56, 0xb9, 0x8c, 0xd2, 0xeb, 0xaf, 0xc2, 0xac, - 0xe3, 0xad, 0xee, 0x39, 0x21, 0x0d, 0x02, 0xf1, 0x83, 0xe4, 0x0d, 0x55, 0x52, 0x8e, 0x77, 0x5a, - 0x8c, 0x4e, 0xf7, 0xbc, 0xd3, 0x7b, 0xe1, 0x69, 0x2e, 0x3d, 0xcd, 0x2b, 0xc6, 0x56, 0x96, 0x13, - 0x67, 0xff, 0x19, 0x00, 0x00, 0xff, 0xff, 0x59, 0xd0, 0x47, 0x8d, 0x7e, 0x29, 0x00, 0x00, + // 2983 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0x4b, 0x70, 0xdb, 0xd6, + 0xb9, 0x36, 0xf8, 0x12, 0xf9, 0x53, 0xa4, 0x8e, 0x8e, 0x24, 0x9b, 0x96, 0xf3, 0x50, 0x90, 0x38, + 0xd1, 0xd5, 0xbd, 0x57, 0x76, 0x64, 0xc7, 0xd7, 0x37, 0x49, 0x5b, 0x43, 0x14, 0xe4, 0xd0, 0x26, + 0x41, 0xfa, 0x10, 0xb4, 0x63, 0x4f, 0x67, 0x30, 0x10, 0x79, 0x4c, 0x61, 0x04, 0x02, 0x34, 0x00, + 0xc9, 0xd6, 0xce, 0x6d, 0x9a, 0xbe, 0x1f, 0xe9, 0x33, 0x4d, 0x33, 0xcd, 0x74, 0xa6, 0x8b, 0x4e, + 0x37, 0x5d, 0x77, 0xd9, 0xe9, 0xa2, 0xcb, 0xee, 0xdb, 0x2e, 0xba, 0xea, 0x74, 0x97, 0xe9, 0xaa, + 0x8b, 0x2e, 0x3a, 0x9d, 0xf3, 0x00, 0x48, 0x59, 0x8c, 0xed, 0xb8, 0xdd, 0xd8, 0xc9, 0xee, 0xfc, + 0x8f, 0xf3, 0xf8, 0xbe, 0xf3, 0xe3, 0x3f, 0x3f, 0x80, 0x03, 0xc5, 0x5b, 0xbb, 0x34, 0xd8, 0x5f, + 0x1d, 0x06, 0x7e, 0xe4, 0xe3, 0x2c, 0x17, 0x16, 0xcb, 0x91, 0x3f, 0xf4, 0x7b, 0x76, 0x64, 0x0b, + 0xf5, 0x62, 0x71, 0x2f, 0x0a, 0x86, 0x5d, 0x21, 0xa8, 0x6f, 0x2b, 0x90, 0x33, 0xed, 0xa0, 0x4f, + 0x23, 0xbc, 0x08, 0xf9, 0x1d, 0xba, 0x1f, 0x0e, 0xed, 0x2e, 0xad, 0x28, 0x4b, 0xca, 0x72, 0x81, + 0x24, 0x32, 0x9e, 0x87, 0x6c, 0xb8, 0x6d, 0x07, 0xbd, 0x4a, 0x8a, 0x1b, 0x84, 0x80, 0x5f, 0x81, + 0x62, 0x64, 0x6f, 0xb9, 0x34, 0xb2, 0xa2, 0xfd, 0x21, 0xad, 0xa4, 0x97, 0x94, 0xe5, 0xf2, 0xda, + 0xfc, 0x6a, 0x32, 0x9f, 0xc9, 0x8d, 0xe6, 0xfe, 0x90, 0x12, 0x88, 0x92, 0x36, 0xc6, 0x90, 0xe9, + 0x52, 0xd7, 0xad, 0x64, 0xf8, 0x58, 0xbc, 0xad, 0x6e, 0x40, 0xf9, 0xaa, 0x79, 0xd1, 0x8e, 0x68, + 0xd5, 0x76, 0x5d, 0x1a, 0xd4, 0x36, 0xd8, 0x72, 0x76, 0x43, 0x1a, 0x78, 0xf6, 0x20, 0x59, 0x4e, + 0x2c, 0xe3, 0xa3, 0x90, 0xeb, 0x07, 0xfe, 0xee, 0x30, 0xac, 0xa4, 0x96, 0xd2, 0xcb, 0x05, 0x22, + 0x25, 0xf5, 0xf3, 0x00, 0xfa, 0x1e, 0xf5, 0x22, 0xd3, 0xdf, 0xa1, 0x1e, 0x7e, 0x0a, 0x0a, 0x91, + 0x33, 0xa0, 0x61, 0x64, 0x0f, 0x86, 0x7c, 0x88, 0x34, 0x19, 0x29, 0x3e, 0x02, 0xd2, 0x22, 0xe4, + 0x87, 0x7e, 0xe8, 0x44, 0x8e, 0xef, 0x71, 0x3c, 0x05, 0x92, 0xc8, 0xea, 0x67, 0x21, 0x7b, 0xd5, + 0x76, 0x77, 0x29, 0x7e, 0x16, 0x32, 0x1c, 0xb0, 0xc2, 0x01, 0x17, 0x57, 0x05, 0xe9, 0x1c, 0x27, + 0x37, 0xb0, 0xb1, 0xf7, 0x98, 0x27, 0x1f, 0x7b, 0x9a, 0x08, 0x41, 0xdd, 0x81, 0xe9, 0x75, 0xc7, + 0xeb, 0x5d, 0xb5, 0x03, 0x87, 0x91, 0xf1, 0x88, 0xc3, 0xe0, 0x17, 0x20, 0xc7, 0x1b, 0x61, 0x25, + 0xbd, 0x94, 0x5e, 0x2e, 0xae, 0x4d, 0xcb, 0x8e, 0x7c, 0x6d, 0x44, 0xda, 0xd4, 0xdf, 0x2a, 0x00, + 0xeb, 0xfe, 0xae, 0xd7, 0xbb, 0xc2, 0x8c, 0x18, 0x41, 0x3a, 0xbc, 0xe5, 0x4a, 0x22, 0x59, 0x13, + 0x5f, 0x86, 0xf2, 0x96, 0xe3, 0xf5, 0xac, 0x3d, 0xb9, 0x1c, 0xc1, 0x65, 0x71, 0xed, 0x05, 0x39, + 0xdc, 0xa8, 0xf3, 0xea, 0xf8, 0xaa, 0x43, 0xdd, 0x8b, 0x82, 0x7d, 0x52, 0xda, 0x1a, 0xd7, 0x2d, + 0x76, 0x00, 0x1f, 0x76, 0x62, 0x93, 0xee, 0xd0, 0xfd, 0x78, 0xd2, 0x1d, 0xba, 0x8f, 0xff, 0x6b, + 0x1c, 0x51, 0x71, 0x6d, 0x2e, 0x9e, 0x6b, 0xac, 0xaf, 0x84, 0xf9, 0x6a, 0xea, 0xbc, 0xa2, 0xfe, + 0x3a, 0x0b, 0x65, 0xfd, 0x0e, 0xed, 0xee, 0x46, 0xb4, 0x39, 0x64, 0x7b, 0x10, 0xe2, 0x06, 0xcc, + 0x38, 0x5e, 0xd7, 0xdd, 0xed, 0xd1, 0x9e, 0x75, 0xd3, 0xa1, 0x6e, 0x2f, 0xe4, 0x71, 0x54, 0x4e, + 0xd6, 0x7d, 0xd0, 0x7f, 0xb5, 0x26, 0x9d, 0x37, 0xb9, 0x2f, 0x29, 0x3b, 0x07, 0x64, 0xbc, 0x02, + 0xb3, 0x5d, 0xd7, 0xa1, 0x5e, 0x64, 0xdd, 0x64, 0x78, 0xad, 0xc0, 0xbf, 0x1d, 0x56, 0xb2, 0x4b, + 0xca, 0x72, 0x9e, 0xcc, 0x08, 0xc3, 0x26, 0xd3, 0x13, 0xff, 0x76, 0x88, 0x5f, 0x85, 0xfc, 0x6d, + 0x3f, 0xd8, 0x71, 0x7d, 0xbb, 0x57, 0xc9, 0xf1, 0x39, 0x9f, 0x99, 0x3c, 0xe7, 0x35, 0xe9, 0x45, + 0x12, 0x7f, 0xbc, 0x0c, 0x28, 0xbc, 0xe5, 0x5a, 0x21, 0x75, 0x69, 0x37, 0xb2, 0x5c, 0x67, 0xe0, + 0x44, 0x95, 0x3c, 0x0f, 0xc9, 0x72, 0x78, 0xcb, 0x6d, 0x73, 0x75, 0x9d, 0x69, 0xb1, 0x05, 0x0b, + 0x51, 0x60, 0x7b, 0xa1, 0xdd, 0x65, 0x83, 0x59, 0x4e, 0xe8, 0xbb, 0x36, 0x0f, 0xc7, 0x02, 0x9f, + 0x72, 0x65, 0xf2, 0x94, 0xe6, 0xa8, 0x4b, 0x2d, 0xee, 0x41, 0xe6, 0xa3, 0x09, 0x5a, 0xfc, 0x32, + 0x2c, 0x84, 0x3b, 0xce, 0xd0, 0xe2, 0xe3, 0x58, 0x43, 0xd7, 0xf6, 0xac, 0xae, 0xdd, 0xdd, 0xa6, + 0x15, 0xe0, 0xb0, 0x31, 0x33, 0xf2, 0x7d, 0x6f, 0xb9, 0xb6, 0x57, 0x65, 0x16, 0xf5, 0x35, 0x28, + 0x1f, 0xe4, 0x11, 0xcf, 0x42, 0xc9, 0xbc, 0xde, 0xd2, 0x2d, 0xcd, 0xd8, 0xb0, 0x0c, 0xad, 0xa1, + 0xa3, 0x23, 0xb8, 0x04, 0x05, 0xae, 0x6a, 0x1a, 0xf5, 0xeb, 0x48, 0xc1, 0x53, 0x90, 0xd6, 0xea, + 0x75, 0x94, 0x52, 0xcf, 0x43, 0x3e, 0x26, 0x04, 0xcf, 0x40, 0xb1, 0x63, 0xb4, 0x5b, 0x7a, 0xb5, + 0xb6, 0x59, 0xd3, 0x37, 0xd0, 0x11, 0x9c, 0x87, 0x4c, 0xb3, 0x6e, 0xb6, 0x90, 0x22, 0x5a, 0x5a, + 0x0b, 0xa5, 0x58, 0xcf, 0x8d, 0x75, 0x0d, 0xa5, 0xd5, 0x5f, 0x28, 0x30, 0x3f, 0x09, 0x18, 0x2e, + 0xc2, 0xd4, 0x86, 0xbe, 0xa9, 0x75, 0xea, 0x26, 0x3a, 0x82, 0xe7, 0x60, 0x86, 0xe8, 0x2d, 0x5d, + 0x33, 0xb5, 0xf5, 0xba, 0x6e, 0x11, 0x5d, 0xdb, 0x40, 0x0a, 0xc6, 0x50, 0x66, 0x2d, 0xab, 0xda, + 0x6c, 0x34, 0x6a, 0xa6, 0xa9, 0x6f, 0xa0, 0x14, 0x9e, 0x07, 0xc4, 0x75, 0x1d, 0x63, 0xa4, 0x4d, + 0x63, 0x04, 0xd3, 0x6d, 0x9d, 0xd4, 0xb4, 0x7a, 0xed, 0x06, 0x1b, 0x00, 0x65, 0xf0, 0x73, 0xf0, + 0x74, 0xb5, 0x69, 0xb4, 0x6b, 0x6d, 0x53, 0x37, 0x4c, 0xab, 0x6d, 0x68, 0xad, 0xf6, 0x1b, 0x4d, + 0x93, 0x8f, 0x2c, 0xc0, 0x65, 0x71, 0x19, 0x40, 0xeb, 0x98, 0x4d, 0x31, 0x0e, 0xca, 0x5d, 0xca, + 0xe4, 0x15, 0x94, 0xba, 0x94, 0xc9, 0xa7, 0x50, 0xfa, 0x52, 0x26, 0x9f, 0x46, 0x19, 0xf5, 0xdd, + 0x14, 0x64, 0x39, 0x57, 0x2c, 0xdd, 0x8d, 0x25, 0x31, 0xde, 0x4e, 0x1e, 0xfd, 0xd4, 0x7d, 0x1e, + 0x7d, 0x9e, 0x31, 0x65, 0x12, 0x12, 0x02, 0x3e, 0x01, 0x05, 0x3f, 0xe8, 0x5b, 0xc2, 0x22, 0xd2, + 0x67, 0xde, 0x0f, 0xfa, 0x3c, 0xcf, 0xb2, 0xd4, 0xc5, 0xb2, 0xee, 0x96, 0x1d, 0x52, 0x1e, 0xc1, + 0x05, 0x92, 0xc8, 0xf8, 0x38, 0x30, 0x3f, 0x8b, 0xaf, 0x23, 0xc7, 0x6d, 0x53, 0x7e, 0xd0, 0x37, + 0xd8, 0x52, 0x9e, 0x87, 0x52, 0xd7, 0x77, 0x77, 0x07, 0x9e, 0xe5, 0x52, 0xaf, 0x1f, 0x6d, 0x57, + 0xa6, 0x96, 0x94, 0xe5, 0x12, 0x99, 0x16, 0xca, 0x3a, 0xd7, 0xe1, 0x0a, 0x4c, 0x75, 0xb7, 0xed, + 0x20, 0xa4, 0x22, 0x6a, 0x4b, 0x24, 0x16, 0xf9, 0xac, 0xb4, 0xeb, 0x0c, 0x6c, 0x37, 0xe4, 0x11, + 0x5a, 0x22, 0x89, 0xcc, 0x40, 0xdc, 0x74, 0xed, 0x7e, 0xc8, 0x23, 0xab, 0x44, 0x84, 0xa0, 0xfe, + 0x1f, 0xa4, 0x89, 0x7f, 0x9b, 0x0d, 0x29, 0x26, 0x0c, 0x2b, 0xca, 0x52, 0x7a, 0x19, 0x93, 0x58, + 0x64, 0xd9, 0x5d, 0x26, 0x38, 0x91, 0xf7, 0xe2, 0x94, 0xf6, 0xbe, 0x02, 0x45, 0x1e, 0x98, 0x84, + 0x86, 0xbb, 0x6e, 0xc4, 0x12, 0xa1, 0xcc, 0x00, 0xca, 0x81, 0x44, 0xc8, 0x69, 0x27, 0xd2, 0xc6, + 0xf0, 0xb1, 0x87, 0xda, 0xb2, 0x6f, 0xde, 0xa4, 0xdd, 0x88, 0x8a, 0x7c, 0x9f, 0x21, 0xd3, 0x4c, + 0xa9, 0x49, 0x1d, 0x23, 0xd6, 0xf1, 0x42, 0x1a, 0x44, 0x96, 0xd3, 0xe3, 0x94, 0x67, 0x48, 0x5e, + 0x28, 0x6a, 0x3d, 0xfc, 0x0c, 0x64, 0x78, 0x5a, 0xc8, 0xf0, 0x59, 0x40, 0xce, 0x42, 0xfc, 0xdb, + 0x84, 0xeb, 0x2f, 0x65, 0xf2, 0x59, 0x94, 0x53, 0x5f, 0x87, 0x69, 0xbe, 0xb8, 0x6b, 0x76, 0xe0, + 0x39, 0x5e, 0x9f, 0x9f, 0x72, 0x7e, 0x4f, 0x6c, 0x7b, 0x89, 0xf0, 0x36, 0xc3, 0x3c, 0xa0, 0x61, + 0x68, 0xf7, 0xa9, 0x3c, 0x75, 0x62, 0x51, 0xfd, 0x59, 0x1a, 0x8a, 0xed, 0x28, 0xa0, 0xf6, 0x80, + 0x1f, 0x60, 0xf8, 0x75, 0x80, 0x30, 0xb2, 0x23, 0x3a, 0xa0, 0x5e, 0x14, 0xe3, 0x7b, 0x4a, 0xce, + 0x3c, 0xe6, 0xb7, 0xda, 0x8e, 0x9d, 0xc8, 0x98, 0x3f, 0x5e, 0x83, 0x22, 0x65, 0x66, 0x2b, 0x62, + 0x07, 0xa1, 0x4c, 0xb6, 0xb3, 0x71, 0xe6, 0x48, 0x4e, 0x48, 0x02, 0x34, 0x69, 0x2f, 0x7e, 0x90, + 0x82, 0x42, 0x32, 0x1a, 0xd6, 0x20, 0xdf, 0xb5, 0x23, 0xda, 0xf7, 0x83, 0x7d, 0x79, 0x3e, 0x9d, + 0xbc, 0xdf, 0xec, 0xab, 0x55, 0xe9, 0x4c, 0x92, 0x6e, 0xf8, 0x69, 0x10, 0x87, 0xbe, 0x88, 0x3a, + 0x81, 0xb7, 0xc0, 0x35, 0x3c, 0xee, 0x5e, 0x05, 0x3c, 0x0c, 0x9c, 0x81, 0x1d, 0xec, 0x5b, 0x3b, + 0x74, 0x3f, 0xce, 0xe5, 0xe9, 0x09, 0x3b, 0x89, 0xa4, 0xdf, 0x65, 0xba, 0x2f, 0xb3, 0xcf, 0xf9, + 0x83, 0x7d, 0x65, 0xb4, 0x1c, 0xde, 0x9f, 0xb1, 0x9e, 0xfc, 0x74, 0x0c, 0xe3, 0x73, 0x30, 0xcb, + 0x03, 0x8b, 0x35, 0xd5, 0x97, 0x20, 0x1f, 0x2f, 0x1e, 0x17, 0x20, 0xab, 0x07, 0x81, 0x1f, 0xa0, + 0x23, 0x3c, 0x09, 0x35, 0xea, 0x22, 0x8f, 0x6d, 0x6c, 0xb0, 0x3c, 0xf6, 0x9b, 0x54, 0x72, 0x18, + 0x11, 0x7a, 0x6b, 0x97, 0x86, 0x11, 0xfe, 0x1c, 0xcc, 0x51, 0x1e, 0x42, 0xce, 0x1e, 0xb5, 0xba, + 0xbc, 0x72, 0x61, 0x01, 0xa4, 0x70, 0xbe, 0x67, 0x56, 0x45, 0xa1, 0x15, 0x57, 0x34, 0x64, 0x36, + 0xf1, 0x95, 0xaa, 0x1e, 0xd6, 0x61, 0xce, 0x19, 0x0c, 0x68, 0xcf, 0xb1, 0xa3, 0xf1, 0x01, 0xc4, + 0x86, 0x2d, 0xc4, 0x07, 0xfb, 0x81, 0xc2, 0x88, 0xcc, 0x26, 0x3d, 0x92, 0x61, 0x4e, 0x42, 0x2e, + 0xe2, 0x45, 0x1c, 0x8f, 0xdd, 0xe2, 0x5a, 0x29, 0x4e, 0x28, 0x5c, 0x49, 0xa4, 0x11, 0xbf, 0x04, + 0xa2, 0x24, 0xe4, 0xa9, 0x63, 0x14, 0x10, 0xa3, 0x93, 0x9e, 0x08, 0x3b, 0x3e, 0x09, 0xe5, 0x03, + 0x67, 0x50, 0x8f, 0x13, 0x96, 0x26, 0xa5, 0xf1, 0x03, 0xa5, 0x87, 0x4f, 0xc1, 0x94, 0x2f, 0xce, + 0x1f, 0x9e, 0x54, 0x46, 0x2b, 0x3e, 0x78, 0x38, 0x91, 0xd8, 0x4b, 0xfd, 0x0c, 0xcc, 0x24, 0x0c, + 0x86, 0x43, 0xdf, 0x0b, 0x29, 0x5e, 0x81, 0x5c, 0xc0, 0x1f, 0x67, 0xc9, 0x1a, 0x96, 0x43, 0x8c, + 0x3d, 0xe8, 0x44, 0x7a, 0xa8, 0x3d, 0x98, 0x11, 0x9a, 0x6b, 0x4e, 0xb4, 0xcd, 0x37, 0x0a, 0x9f, + 0x84, 0x2c, 0x65, 0x8d, 0x7b, 0x38, 0x27, 0xad, 0x2a, 0xb7, 0x13, 0x61, 0x1d, 0x9b, 0x25, 0xf5, + 0xc0, 0x59, 0xfe, 0x96, 0x82, 0x39, 0xb9, 0xca, 0x75, 0x3b, 0xea, 0x6e, 0x3f, 0xa6, 0x9b, 0xfd, + 0xdf, 0x30, 0xc5, 0xf4, 0x4e, 0xf2, 0x60, 0x4c, 0xd8, 0xee, 0xd8, 0x83, 0x6d, 0xb8, 0x1d, 0x5a, + 0x63, 0xbb, 0x2b, 0x6b, 0xa0, 0x92, 0x1d, 0x8e, 0x1d, 0xc0, 0x13, 0xe2, 0x22, 0xf7, 0x80, 0xb8, + 0x98, 0x7a, 0xa8, 0xb8, 0xd8, 0x80, 0xf9, 0x83, 0x8c, 0xcb, 0xe0, 0xf8, 0x1f, 0x98, 0x12, 0x9b, + 0x12, 0xa7, 0xc0, 0x49, 0xfb, 0x16, 0xbb, 0xa8, 0xbf, 0x4b, 0xc1, 0xbc, 0xcc, 0x4e, 0x9f, 0x8c, + 0xc7, 0x74, 0x8c, 0xe7, 0xec, 0xc3, 0xf0, 0xfc, 0x90, 0xfb, 0xa7, 0x56, 0x61, 0xe1, 0x1e, 0x1e, + 0x1f, 0xe1, 0x61, 0xfd, 0x50, 0x81, 0xe9, 0x75, 0xda, 0x77, 0xbc, 0xc7, 0x74, 0x17, 0xc6, 0xc8, + 0xcd, 0x3c, 0x54, 0x10, 0x9f, 0x83, 0x92, 0xc4, 0x2b, 0xd9, 0x3a, 0xcc, 0xb6, 0x32, 0x89, 0xed, + 0xbf, 0x28, 0x50, 0xaa, 0xfa, 0x83, 0x81, 0x13, 0x3d, 0xa6, 0x4c, 0x1d, 0xc6, 0x99, 0x99, 0x84, + 0x13, 0x41, 0x39, 0x86, 0x29, 0x08, 0x52, 0xff, 0xaa, 0xc0, 0x0c, 0xf1, 0x5d, 0x77, 0xcb, 0xee, + 0xee, 0x3c, 0xd9, 0xd8, 0x31, 0xa0, 0x11, 0x50, 0x89, 0xfe, 0x1f, 0x0a, 0x94, 0x5b, 0x01, 0x1d, + 0xda, 0x01, 0x7d, 0xa2, 0xc1, 0xb3, 0x4a, 0xb8, 0x17, 0xc9, 0x1a, 0xa2, 0x40, 0x78, 0x5b, 0x9d, + 0x85, 0x99, 0x04, 0xbb, 0xe4, 0xe3, 0x8f, 0x0a, 0x2c, 0x88, 0x00, 0x91, 0x96, 0xde, 0x63, 0x4a, + 0x4b, 0x8c, 0x37, 0x33, 0x86, 0xb7, 0x02, 0x47, 0xef, 0xc5, 0x26, 0x61, 0xbf, 0x95, 0x82, 0x63, + 0x71, 0x6c, 0x3c, 0xe6, 0xc0, 0xff, 0x8d, 0x78, 0x58, 0x84, 0xca, 0x61, 0x12, 0x24, 0x43, 0xef, + 0xa4, 0xa0, 0x52, 0x0d, 0xa8, 0x1d, 0xd1, 0xb1, 0x5a, 0xe4, 0xc9, 0x89, 0x0d, 0xfc, 0x32, 0x4c, + 0x0f, 0xed, 0x20, 0x72, 0xba, 0xce, 0xd0, 0x66, 0x6f, 0x7b, 0x59, 0x5e, 0xea, 0xdc, 0x33, 0xc0, + 0x01, 0x17, 0xf5, 0x04, 0x1c, 0x9f, 0xc0, 0x88, 0xe4, 0xeb, 0x9f, 0x0a, 0xe0, 0x76, 0x64, 0x07, + 0xd1, 0x27, 0xe0, 0x54, 0x99, 0x18, 0x4c, 0x0b, 0x30, 0x77, 0x00, 0xff, 0x38, 0x2f, 0x34, 0xfa, + 0x44, 0x9c, 0x38, 0x1f, 0xc9, 0xcb, 0x38, 0x7e, 0xc9, 0xcb, 0x9f, 0x15, 0x58, 0xac, 0xfa, 0xe2, + 0xfb, 0xde, 0x13, 0xf9, 0x84, 0xa9, 0x4f, 0xc3, 0x89, 0x89, 0x00, 0x25, 0x01, 0x7f, 0x52, 0xe0, + 0x28, 0xa1, 0x76, 0xef, 0xc9, 0x04, 0x7f, 0x05, 0x8e, 0x1d, 0x02, 0x27, 0x2b, 0xd4, 0x73, 0x90, + 0x1f, 0xd0, 0xc8, 0xee, 0xd9, 0x91, 0x2d, 0x21, 0x2d, 0xc6, 0xe3, 0x8e, 0xbc, 0x1b, 0xd2, 0x83, + 0x24, 0xbe, 0xea, 0x07, 0x29, 0x98, 0xe3, 0xb5, 0xee, 0xa7, 0x2f, 0x5a, 0x93, 0xdf, 0x05, 0xde, + 0x51, 0x60, 0xfe, 0x20, 0x41, 0xc9, 0x3b, 0xc1, 0x7f, 0xfa, 0x7b, 0xc5, 0x84, 0x84, 0x90, 0x9e, + 0x54, 0x82, 0xfe, 0x3e, 0x05, 0x95, 0xf1, 0x25, 0x7d, 0xfa, 0x6d, 0xe3, 0xe0, 0xb7, 0x8d, 0x8f, + 0xfd, 0x31, 0xeb, 0x5d, 0x05, 0x8e, 0x4f, 0x20, 0xf4, 0xe3, 0x6d, 0xf4, 0xd8, 0x17, 0x8e, 0xd4, + 0x03, 0xbf, 0x70, 0x3c, 0xec, 0x56, 0xff, 0x41, 0x81, 0xf9, 0x86, 0xf8, 0xb0, 0x2c, 0xde, 0xe3, + 0x1f, 0xdf, 0x6c, 0xc6, 0xbf, 0x1d, 0x67, 0x46, 0x7f, 0x4e, 0xd4, 0x2a, 0x2c, 0xdc, 0x03, 0xed, + 0x11, 0xbe, 0x4d, 0xfc, 0x5d, 0x81, 0x59, 0x39, 0x8a, 0xf6, 0xd8, 0x16, 0x02, 0x13, 0xd8, 0xc1, + 0xcf, 0x40, 0xda, 0xe9, 0xc5, 0x15, 0xe4, 0xc1, 0x1f, 0xc3, 0xcc, 0xa0, 0x5e, 0x00, 0x3c, 0x8e, + 0xfb, 0x11, 0xa8, 0xe3, 0xb5, 0x15, 0x23, 0xfe, 0x0d, 0x6a, 0xbb, 0x51, 0x9c, 0x40, 0xd4, 0x9f, + 0xa7, 0xa0, 0x44, 0x98, 0xc6, 0x19, 0xd0, 0x76, 0x64, 0x47, 0x21, 0x7e, 0x0e, 0xa6, 0xb7, 0xb9, + 0x8b, 0x35, 0x7a, 0x0e, 0x0a, 0xa4, 0x28, 0x74, 0xe2, 0xe3, 0xed, 0x1a, 0x2c, 0x84, 0xb4, 0xeb, + 0x7b, 0xbd, 0xd0, 0xda, 0xa2, 0xdb, 0x8e, 0xd7, 0xb3, 0x06, 0x76, 0x18, 0xd1, 0x80, 0x33, 0x56, + 0x22, 0x73, 0xd2, 0xb8, 0xce, 0x6d, 0x0d, 0x6e, 0xc2, 0xa7, 0x61, 0x7e, 0xcb, 0xf1, 0x5c, 0xbf, + 0x6f, 0x0d, 0x5d, 0x7b, 0x9f, 0x06, 0xa1, 0xd5, 0xf5, 0x77, 0x3d, 0x41, 0x55, 0x96, 0x60, 0x61, + 0x6b, 0x09, 0x53, 0x95, 0x59, 0xf0, 0x0d, 0x58, 0x99, 0x38, 0x8b, 0x75, 0xd3, 0x71, 0x23, 0x1a, + 0xd0, 0x9e, 0x15, 0xd0, 0xa1, 0xeb, 0x74, 0xc5, 0x5f, 0x56, 0x51, 0x4c, 0xbd, 0x38, 0x61, 0xea, + 0x4d, 0xe9, 0x4e, 0x46, 0xde, 0xf8, 0x04, 0x14, 0xba, 0xc3, 0x5d, 0x6b, 0x97, 0xff, 0xd2, 0x61, + 0x69, 0x45, 0x21, 0xf9, 0xee, 0x70, 0xb7, 0xc3, 0x64, 0x8c, 0x20, 0x7d, 0x6b, 0x28, 0xb2, 0x89, + 0x42, 0x58, 0x53, 0xfd, 0x50, 0x81, 0xb2, 0xd6, 0xef, 0x07, 0xb4, 0x6f, 0x47, 0x92, 0xa6, 0xd3, + 0x30, 0x2f, 0x28, 0xd9, 0xb7, 0xe4, 0x5d, 0x0a, 0x81, 0x47, 0x11, 0x78, 0xa4, 0x4d, 0xdc, 0xa4, + 0x10, 0x78, 0xce, 0xc2, 0xd1, 0x5d, 0x6f, 0x62, 0x9f, 0x14, 0xef, 0x33, 0x9f, 0x58, 0xc7, 0x7b, + 0xfd, 0x3f, 0x1c, 0x9f, 0xcc, 0xc2, 0xc0, 0x11, 0x37, 0x1d, 0x4a, 0xe4, 0xe8, 0x04, 0xd0, 0x0d, + 0xc7, 0xbb, 0x4f, 0x57, 0xfb, 0x0e, 0xe7, 0xeb, 0x23, 0xba, 0xda, 0x77, 0xd4, 0x5f, 0x26, 0x9f, + 0x64, 0xe3, 0x70, 0x49, 0xd2, 0x63, 0x1c, 0xe3, 0xca, 0xfd, 0x62, 0xbc, 0x02, 0x53, 0x21, 0x0d, + 0xf6, 0x1c, 0xaf, 0xcf, 0xc1, 0xe5, 0x49, 0x2c, 0xe2, 0x36, 0xbc, 0x28, 0xb1, 0xd3, 0x3b, 0x11, + 0x0d, 0x3c, 0xdb, 0x75, 0xf7, 0x2d, 0xf1, 0xe6, 0xe8, 0x45, 0xb4, 0x67, 0x8d, 0x6e, 0x7e, 0x88, + 0x14, 0xf9, 0xbc, 0xf0, 0xd6, 0x13, 0x67, 0x92, 0xf8, 0x9a, 0xc9, 0x9d, 0x90, 0xd7, 0xa0, 0x1c, + 0xc8, 0x20, 0xb6, 0x42, 0xb6, 0x3d, 0xb2, 0x32, 0x98, 0x8f, 0xff, 0x29, 0x8d, 0x47, 0x38, 0x29, + 0x05, 0x07, 0x02, 0xfe, 0x3c, 0x4c, 0xcb, 0x15, 0xd9, 0xae, 0x63, 0x8f, 0x2a, 0x85, 0x7b, 0xae, + 0xc3, 0x68, 0xcc, 0x48, 0xe4, 0xc5, 0x19, 0x2e, 0x5c, 0xca, 0xe4, 0x73, 0x68, 0x4a, 0xfd, 0x95, + 0x02, 0x73, 0x13, 0xca, 0xae, 0xa4, 0xa6, 0x53, 0xc6, 0x5e, 0x19, 0xff, 0x17, 0xb2, 0xfc, 0x77, + 0x9f, 0xfc, 0x81, 0x7c, 0xec, 0x70, 0xd5, 0xc6, 0x7f, 0xcd, 0x11, 0xe1, 0xc5, 0x9e, 0x45, 0x8e, + 0xa9, 0xcb, 0xdf, 0x19, 0xe3, 0x53, 0xa3, 0xc8, 0x74, 0xe2, 0x35, 0xf2, 0xf0, 0x4b, 0x68, 0xe6, + 0x81, 0x2f, 0xa1, 0x2b, 0xdf, 0x4b, 0x43, 0xa1, 0xb1, 0xdf, 0xbe, 0xe5, 0x6e, 0xba, 0x76, 0x9f, + 0xff, 0x3b, 0x6b, 0xb4, 0xcc, 0xeb, 0xe8, 0x08, 0x9e, 0x85, 0x92, 0xd1, 0x34, 0x2d, 0xa3, 0x53, + 0xaf, 0x5b, 0x9b, 0x75, 0xed, 0x22, 0x52, 0x30, 0x82, 0xe9, 0x16, 0xa9, 0x59, 0x97, 0xf5, 0xeb, + 0x42, 0x93, 0xc2, 0x73, 0x30, 0xd3, 0x31, 0x6a, 0x57, 0x3a, 0xfa, 0x48, 0x99, 0xc1, 0x0b, 0x30, + 0xdb, 0xe8, 0xd4, 0xcd, 0x5a, 0xab, 0x3e, 0xa6, 0xce, 0xe3, 0x12, 0x14, 0xd6, 0xeb, 0xcd, 0x75, + 0x21, 0x22, 0x36, 0x7e, 0xc7, 0x68, 0xd7, 0x2e, 0x1a, 0xfa, 0x86, 0x50, 0x2d, 0x31, 0xd5, 0x0d, + 0x9d, 0x34, 0x37, 0x6b, 0xf1, 0x94, 0x17, 0x30, 0x82, 0xe2, 0x7a, 0xcd, 0xd0, 0x88, 0x1c, 0xe5, + 0xae, 0x82, 0xcb, 0x50, 0xd0, 0x8d, 0x4e, 0x43, 0xca, 0x29, 0x5c, 0x81, 0x39, 0xad, 0x63, 0x36, + 0xad, 0x9a, 0x51, 0x25, 0x7a, 0x43, 0x37, 0x4c, 0x69, 0xc9, 0xe0, 0x39, 0x28, 0x9b, 0xb5, 0x86, + 0xde, 0x36, 0xb5, 0x46, 0x4b, 0x2a, 0xd9, 0x2a, 0xf2, 0x6d, 0x3d, 0xf6, 0x41, 0x78, 0x11, 0x16, + 0x8c, 0xa6, 0x25, 0xef, 0x21, 0x58, 0x57, 0xb5, 0x7a, 0x47, 0x97, 0xb6, 0x25, 0x7c, 0x0c, 0x70, + 0xd3, 0xb0, 0x3a, 0xad, 0x0d, 0xcd, 0xd4, 0x2d, 0xa3, 0x79, 0x4d, 0x1a, 0x2e, 0xe0, 0x32, 0xe4, + 0x47, 0x2b, 0xb8, 0xcb, 0x58, 0x28, 0xb5, 0x34, 0x62, 0x8e, 0xc0, 0xde, 0xbd, 0xcb, 0xc8, 0x82, + 0x8b, 0xa4, 0xd9, 0x69, 0x8d, 0xdc, 0x66, 0xa1, 0x28, 0xc9, 0x92, 0xaa, 0x0c, 0x53, 0xad, 0xd7, + 0x8c, 0x6a, 0xb2, 0xbe, 0xbb, 0xf9, 0xc5, 0x14, 0x52, 0x56, 0x76, 0x20, 0xc3, 0xb7, 0x23, 0x0f, + 0x19, 0xa3, 0x69, 0xe8, 0xe8, 0x08, 0x9e, 0x01, 0xa8, 0xb5, 0x6b, 0x86, 0xa9, 0x5f, 0x24, 0x5a, + 0x9d, 0xc1, 0xe6, 0x8a, 0x98, 0x40, 0x86, 0x76, 0x1a, 0xa6, 0x6a, 0xed, 0xcd, 0x7a, 0x53, 0x33, + 0x25, 0xcc, 0x5a, 0xfb, 0x4a, 0xa7, 0x69, 0x32, 0x23, 0xc2, 0x45, 0xc8, 0xd5, 0xda, 0xa6, 0xfe, + 0xa6, 0xc9, 0x70, 0x71, 0x9b, 0x60, 0x15, 0xdd, 0xbd, 0xb0, 0xf2, 0x5e, 0x1a, 0x32, 0xfc, 0x4a, + 0x57, 0x09, 0x0a, 0x7c, 0xb7, 0xcd, 0xeb, 0x2d, 0x36, 0x65, 0x01, 0x32, 0x35, 0xc3, 0x3c, 0x8f, + 0xbe, 0x90, 0xc2, 0x00, 0xd9, 0x0e, 0x6f, 0x7f, 0x31, 0xc7, 0xda, 0x35, 0xc3, 0x7c, 0xf9, 0x1c, + 0x7a, 0x2b, 0xc5, 0x86, 0xed, 0x08, 0xe1, 0x4b, 0xb1, 0x61, 0xed, 0x2c, 0x7a, 0x3b, 0x31, 0xac, + 0x9d, 0x45, 0x5f, 0x8e, 0x0d, 0x67, 0xd6, 0xd0, 0x57, 0x12, 0xc3, 0x99, 0x35, 0xf4, 0xd5, 0xd8, + 0x70, 0xee, 0x2c, 0xfa, 0x5a, 0x62, 0x38, 0x77, 0x16, 0x7d, 0x3d, 0xc7, 0xb0, 0x70, 0x24, 0x67, + 0xd6, 0xd0, 0x37, 0xf2, 0x89, 0x74, 0xee, 0x2c, 0xfa, 0x66, 0x9e, 0xed, 0x7f, 0xb2, 0xab, 0xe8, + 0x5b, 0x88, 0x2d, 0x93, 0x6d, 0x10, 0xfa, 0x36, 0x6f, 0x32, 0x13, 0xfa, 0x0e, 0x62, 0x18, 0x99, + 0x96, 0x8b, 0xef, 0x70, 0xcb, 0x75, 0x5d, 0x23, 0xe8, 0xbb, 0x39, 0x71, 0xed, 0xa4, 0x5a, 0x6b, + 0x68, 0x75, 0x84, 0x79, 0x0f, 0xc6, 0xca, 0xf7, 0x4f, 0xb3, 0x26, 0x0b, 0x4f, 0xf4, 0x83, 0x16, + 0x9b, 0xf0, 0xaa, 0x46, 0xaa, 0x6f, 0x68, 0x04, 0xfd, 0xf0, 0x34, 0x9b, 0xf0, 0xaa, 0x46, 0x24, + 0x5f, 0x3f, 0x6a, 0x31, 0x47, 0x6e, 0x7a, 0xf7, 0x34, 0x5b, 0xb4, 0xd4, 0xff, 0xb8, 0x85, 0xf3, + 0x90, 0x5e, 0xaf, 0x99, 0xe8, 0x3d, 0x3e, 0x1b, 0x0b, 0x51, 0xf4, 0x13, 0xc4, 0x94, 0x6d, 0xdd, + 0x44, 0xef, 0x33, 0x65, 0xd6, 0xec, 0xb4, 0xea, 0x3a, 0x7a, 0x8a, 0x2d, 0xee, 0xa2, 0xde, 0x6c, + 0xe8, 0x26, 0xb9, 0x8e, 0x7e, 0xca, 0xdd, 0x2f, 0xb5, 0x9b, 0x06, 0xfa, 0x00, 0xe1, 0x32, 0x80, + 0xfe, 0x66, 0x8b, 0xe8, 0xed, 0x76, 0xad, 0x69, 0xa0, 0x67, 0x57, 0x36, 0x01, 0xdd, 0x9b, 0x0e, + 0x18, 0x80, 0x8e, 0x71, 0xd9, 0x68, 0x5e, 0x33, 0xd0, 0x11, 0x26, 0xb4, 0x88, 0xde, 0xd2, 0x88, + 0x8e, 0x14, 0x0c, 0x90, 0x93, 0x97, 0x59, 0x52, 0x78, 0x1a, 0xf2, 0xa4, 0x59, 0xaf, 0xaf, 0x6b, + 0xd5, 0xcb, 0x28, 0xbd, 0xfe, 0x0a, 0xcc, 0x38, 0xfe, 0xea, 0x9e, 0x13, 0xd1, 0x30, 0x14, 0x97, + 0x06, 0x6f, 0xa8, 0x52, 0x72, 0xfc, 0x53, 0xa2, 0x75, 0xaa, 0xef, 0x9f, 0xda, 0x8b, 0x4e, 0x71, + 0xeb, 0x29, 0x9e, 0x31, 0xb6, 0x72, 0x5c, 0x38, 0xf3, 0xaf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x69, + 0x36, 0x21, 0xf2, 0x92, 0x28, 0x00, 0x00, } diff --git a/go/vt/vtgate/engine/ordered_aggregate.go b/go/vt/vtgate/engine/ordered_aggregate.go index 9ec98b35d22..0cf008afb73 100644 --- a/go/vt/vtgate/engine/ordered_aggregate.go +++ b/go/vt/vtgate/engine/ordered_aggregate.go @@ -156,7 +156,6 @@ func (oa *OrderedAggregate) execute(vcursor VCursor, bindVars map[string]*queryp out := &sqltypes.Result{ Fields: oa.convertFields(result.Fields), Rows: make([][]sqltypes.Value, 0, len(result.Rows)), - Extras: result.Extras, } // This code is similar to the one in StreamExecute. var current []sqltypes.Value diff --git a/go/vt/vtgate/grpcvtgateconn/suite_test.go b/go/vt/vtgate/grpcvtgateconn/suite_test.go index 150f39dffc4..d762191ad20 100644 --- a/go/vt/vtgate/grpcvtgateconn/suite_test.go +++ b/go/vt/vtgate/grpcvtgateconn/suite_test.go @@ -1275,7 +1275,6 @@ func testStreamExecute(t *testing.T, session *vtgateconn.VTGateSession) { wantResult := *execCase.result wantResult.RowsAffected = 0 wantResult.InsertID = 0 - wantResult.Extras = nil if !qr.Equal(&wantResult) { t.Errorf("Unexpected result from StreamExecute: got %+v want %+v", qr, wantResult) } @@ -1356,7 +1355,6 @@ func testStreamExecuteShards(t *testing.T, conn *vtgateconn.VTGateConn) { wantResult := *execCase.result wantResult.RowsAffected = 0 wantResult.InsertID = 0 - wantResult.Extras = nil if !qr.Equal(&wantResult) { t.Errorf("Unexpected result from StreamExecuteShards: got %+v want %+v", qr, wantResult) } @@ -1437,7 +1435,6 @@ func testStreamExecuteKeyRanges(t *testing.T, conn *vtgateconn.VTGateConn) { wantResult := *execCase.result wantResult.RowsAffected = 0 wantResult.InsertID = 0 - wantResult.Extras = nil if !qr.Equal(&wantResult) { t.Errorf("Unexpected result from StreamExecuteKeyRanges: got %+v want %+v", qr, wantResult) } @@ -1518,7 +1515,6 @@ func testStreamExecuteKeyspaceIds(t *testing.T, conn *vtgateconn.VTGateConn) { wantResult := *execCase.result wantResult.RowsAffected = 0 wantResult.InsertID = 0 - wantResult.Extras = nil if !qr.Equal(&wantResult) { t.Errorf("Unexpected result from StreamExecuteKeyspaceIds: got %+v want %+v", qr, wantResult) } @@ -1865,13 +1861,7 @@ var testCallerID = &vtrpcpb.CallerID{ } var testExecuteOptions = &querypb.ExecuteOptions{ - IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, - IncludeEventToken: true, - CompareEventToken: &querypb.EventToken{ - Timestamp: 135, - Shard: "shrd", - Position: "pstn", - }, + IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, } var execMap = map[string]struct { @@ -2196,15 +2186,6 @@ var execMap = map[string]struct { }, } -var extras = querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 123, - Shard: "sh", - Position: "po", - }, - Fresher: true, -} - var result1 = sqltypes.Result{ Fields: []*querypb.Field{ { @@ -2228,7 +2209,6 @@ var result1 = sqltypes.Result{ sqltypes.NewInt32(3), }, }, - Extras: &extras, } // streamResultFields is only the fields, sent as the first packet diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go index 76a54fde386..a1ad9c39cb7 100644 --- a/go/vt/vttablet/tabletconntest/fakequeryservice.go +++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go @@ -91,13 +91,7 @@ var TestVTGateCallerID = &querypb.VTGateCallerID{ // TestExecuteOptions is a test execute options. var TestExecuteOptions = &querypb.ExecuteOptions{ - IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, - IncludeEventToken: true, - CompareEventToken: &querypb.EventToken{ - Timestamp: 9876, - Shard: "ssss", - Position: "pppp", - }, + IncludedFields: querypb.ExecuteOptions_TYPE_ONLY, ClientFoundRows: true, } @@ -385,14 +379,6 @@ var ExecuteQueryResult = sqltypes.Result{ sqltypes.TestValue(sqltypes.Char, "row2 value2"), }, }, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 456321, - Shard: "test_shard", - Position: "test_position", - }, - Fresher: true, - }, } // Execute is part of the queryservice.QueryService interface @@ -529,14 +515,6 @@ var ExecuteBatchQueryResultList = []sqltypes.Result{ sqltypes.TestValue(sqltypes.Int8, "2"), }, }, - Extras: &querypb.ResultExtras{ - EventToken: &querypb.EventToken{ - Timestamp: 456322, - Shard: "test_shard2", - Position: "test_position2", - }, - Fresher: true, - }, }, { Fields: []*querypb.Field{ diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index 6377be170a1..0d1981f8246 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -365,7 +365,7 @@ func TestWaitForPosError(t *testing.T) { dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{}}}, nil) err = vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") - want = "unexpected result: &{[] 0 0 [[]] }" + want = "unexpected result: &{[] 0 0 [[]]}" if err == nil || err.Error() != want { t.Errorf("WaitForPos: %v, want %v", err, want) } @@ -376,7 +376,7 @@ func TestWaitForPosError(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1083"), }}}, nil) err = vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") - want = `unexpected result: &{[] 0 0 [[VARBINARY("MariaDB/0-1-1083")] [VARBINARY("MariaDB/0-1-1083")]] }` + want = `unexpected result: &{[] 0 0 [[VARBINARY("MariaDB/0-1-1083")] [VARBINARY("MariaDB/0-1-1083")]]}` if err == nil || err.Error() != want { t.Errorf("WaitForPos: %v, want %v", err, want) } diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index e15c428708b..68b5b9cba83 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -104,11 +104,7 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { case planbuilder.PlanSelectImpossible: if qre.plan.Fields != nil { return &sqltypes.Result{ - Fields: qre.plan.Fields, - RowsAffected: 0, - InsertID: 0, - Rows: nil, - Extras: nil, + Fields: qre.plan.Fields, }, nil } } diff --git a/go/vt/vttablet/tabletserver/replication_watcher.go b/go/vt/vttablet/tabletserver/replication_watcher.go index 3a65508615a..9d2d13b0f12 100644 --- a/go/vt/vttablet/tabletserver/replication_watcher.go +++ b/go/vt/vttablet/tabletserver/replication_watcher.go @@ -17,183 +17,80 @@ limitations under the License. package tabletserver import ( - "sync" "time" "golang.org/x/net/context" - "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/stats" - "vitess.io/vitess/go/vt/binlog" - "vitess.io/vitess/go/vt/binlog/eventtoken" - "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" ) +// VStreamer defines the functions of VStreamer +// that the replicationWatcher needs. +type VStreamer interface { + Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error +} + // ReplicationWatcher is a tabletserver service that watches the -// replication stream. It can tell you the current event token, -// and it will trigger schema reloads if a DDL is encountered. +// replication stream. It will trigger schema reloads if a DDL +// is encountered. type ReplicationWatcher struct { - dbconfigs *dbconfigs.DBConfigs - - // Life cycle management vars - isOpen bool - cancel context.CancelFunc - wg sync.WaitGroup - watchReplication bool - se *schema.Engine + vs VStreamer - mu sync.Mutex - eventToken *querypb.EventToken + cancel context.CancelFunc } -var replOnce sync.Once - // NewReplicationWatcher creates a new ReplicationWatcher. -func NewReplicationWatcher(se *schema.Engine, config tabletenv.TabletConfig) *ReplicationWatcher { - rpw := &ReplicationWatcher{ +func NewReplicationWatcher(vs VStreamer, config tabletenv.TabletConfig) *ReplicationWatcher { + return &ReplicationWatcher{ + vs: vs, watchReplication: config.WatchReplication, - se: se, } - replOnce.Do(func() { - stats.Publish("EventTokenPosition", stats.StringFunc(func() string { - if e := rpw.EventToken(); e != nil { - return e.Position - } - return "" - })) - stats.NewGaugeFunc( - "EventTokenTimestamp", - "Replication watcher event token timestamp", - func() int64 { - if e := rpw.EventToken(); e != nil { - return e.Timestamp - } - return 0 - }) - }) - return rpw -} - -// InitDBConfig must be called before Open. -func (rpw *ReplicationWatcher) InitDBConfig(dbcfgs *dbconfigs.DBConfigs) { - rpw.dbconfigs = dbcfgs } // Open starts the ReplicationWatcher service. func (rpw *ReplicationWatcher) Open() { - if rpw.isOpen || !rpw.watchReplication { + if rpw.cancel != nil || !rpw.watchReplication { return } + ctx, cancel := context.WithCancel(tabletenv.LocalContext()) rpw.cancel = cancel - rpw.wg.Add(1) - go rpw.Process(ctx, rpw.dbconfigs) - rpw.isOpen = true + go rpw.Process(ctx) } // Close stops the ReplicationWatcher service. func (rpw *ReplicationWatcher) Close() { - if !rpw.isOpen { + if rpw.cancel == nil { return } rpw.cancel() - rpw.wg.Wait() - rpw.isOpen = false + rpw.cancel = nil } // Process processes the replication stream. -func (rpw *ReplicationWatcher) Process(ctx context.Context, dbconfigs *dbconfigs.DBConfigs) { - defer func() { - tabletenv.LogError() - rpw.wg.Done() - }() +func (rpw *ReplicationWatcher) Process(ctx context.Context) { + defer tabletenv.LogError() - for { - log.Infof("Starting a binlog Streamer from current replication position to monitor binlogs") - streamer := binlog.NewStreamer(dbconfigs.DbaWithDB(), rpw.se, nil /*clientCharset*/, mysql.Position{}, 0 /*timestamp*/, func(eventToken *querypb.EventToken, statements []binlog.FullBinlogStatement) error { - // Save the event token. - rpw.mu.Lock() - rpw.eventToken = eventToken - rpw.mu.Unlock() - - // If it's a DDL, trigger a schema reload. - for _, statement := range statements { - if statement.Statement.Category != binlogdatapb.BinlogTransaction_Statement_BL_DDL { - continue - } - err := rpw.se.Reload(ctx) - log.Infof("Streamer triggered a schema reload, with result: %v", err) - return nil - } + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + for { + err := rpw.vs.Stream(ctx, "current", filter, func(events []*binlogdatapb.VEvent) error { return nil }) - - if err := streamer.Stream(ctx); err != nil { - log.Infof("Streamer stopped: %v", err) - } - select { case <-ctx.Done(): return case <-time.After(5 * time.Second): } + log.Infof("VStream ended: %v, retrying in 5 seconds", err) + time.Sleep(5 * time.Second) } } - -// ComputeExtras returns the requested ResultExtras based on the supplied options. -func (rpw *ReplicationWatcher) ComputeExtras(options *querypb.ExecuteOptions) *querypb.ResultExtras { - if options == nil { - // No options passed in. - return nil - } - - if !options.IncludeEventToken && options.CompareEventToken == nil { - // The flags that make extras exist are not there. - return nil - } - - et := rpw.EventToken() - if et == nil { - return nil - } - - var extras *querypb.ResultExtras - - // See if we need to fill in EventToken. - if options.IncludeEventToken { - extras = &querypb.ResultExtras{ - EventToken: et, - } - } - - // See if we need to compare. - if options.CompareEventToken != nil { - if eventtoken.Fresher(et, options.CompareEventToken) >= 0 { - // For a query, we are fresher if greater or equal - // to the provided compare_event_token. - if extras == nil { - extras = &querypb.ResultExtras{ - Fresher: true, - } - } else { - extras.Fresher = true - } - } - } - return extras -} - -// EventToken returns the current event token. -func (rpw *ReplicationWatcher) EventToken() *querypb.EventToken { - rpw.mu.Lock() - defer rpw.mu.Unlock() - return rpw.eventToken -} diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 259d055136d..9e3a443624e 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -228,7 +228,6 @@ func NewTabletServer(config tabletenv.TabletConfig, topoServer *topo.Server, ali tsv.hw = heartbeat.NewWriter(tsv, alias, config) tsv.hr = heartbeat.NewReader(tsv, config) tsv.txThrottler = txthrottler.CreateTxThrottlerFromTabletConfig(topoServer) - tsv.watcher = NewReplicationWatcher(tsv.se, config) // FIXME(alainjobart) could we move this to the Register method below? // So that vtcombo doesn't even call it once, on the first tablet. // And we can remove the tsOnce variable. @@ -252,6 +251,7 @@ func NewTabletServer(config tabletenv.TabletConfig, topoServer *topo.Server, ali }) // TODO(sougou): move this up once the stats naming problem is fixed. tsv.vstreamer = vstreamer.NewEngine(srvTopoServer, tsv.se) + tsv.watcher = NewReplicationWatcher(tsv.vstreamer, config) tsv.messager = messager.NewEngine(tsv, tsv.se, tsv.vstreamer, config) return tsv } @@ -339,7 +339,6 @@ func (tsv *TabletServer) InitDBConfig(target querypb.Target, dbcfgs *dbconfigs.D tsv.te.InitDBConfig(tsv.dbconfigs) tsv.hw.InitDBConfig(tsv.dbconfigs) tsv.hr.InitDBConfig(tsv.dbconfigs) - tsv.watcher.InitDBConfig(tsv.dbconfigs) tsv.vstreamer.InitDBConfig(tsv.dbconfigs.DbaWithDB()) return nil } @@ -505,6 +504,7 @@ func (tsv *TabletServer) fullStart() (err error) { } tsv.hr.Init(tsv.target) tsv.vstreamer.Open(tsv.target.Keyspace, tsv.alias.Cell) + tsv.watcher.Open() return tsv.serveNewType() } @@ -518,7 +518,6 @@ func (tsv *TabletServer) serveNewType() (err error) { if err := tsv.txThrottler.Open(tsv.target.Keyspace, tsv.target.Shard); err != nil { return err } - tsv.watcher.Close() tsv.messager.Open() tsv.hr.Close() tsv.hw.Open() @@ -562,6 +561,7 @@ func (tsv *TabletServer) StopService() { log.Infof("Executing complete shutdown.") tsv.waitForShutdown() + tsv.watcher.Close() tsv.vstreamer.Close() tsv.qe.Close() tsv.se.Close() @@ -589,6 +589,7 @@ func (tsv *TabletServer) waitForShutdown() { // It forcibly shuts down everything. func (tsv *TabletServer) closeAll() { tsv.messager.Close() + tsv.watcher.Close() tsv.vstreamer.Close() tsv.hr.Close() tsv.hw.Close() @@ -986,12 +987,10 @@ func (tsv *TabletServer) qreExecute(ctx context.Context, query string, comments tsv: tsv, tabletType: tabletType, } - extras := tsv.watcher.ComputeExtras(options) result, err = qre.Execute() if err != nil { return nil, err } - result.Extras = extras result = result.StripMetadata(sqltypes.IncludeFieldsOrDefault(options)) return result, nil diff --git a/proto/query.proto b/proto/query.proto index 4839fc31f39..ff4bf43b265 100644 --- a/proto/query.proto +++ b/proto/query.proto @@ -234,17 +234,11 @@ message BoundQuery { // ExecuteOptions is passed around for all Execute calls. message ExecuteOptions { - // This used to be exclude_field_names, which was replaced by + // 1 used to be exclude_field_names, which was replaced by // IncludedFields enum below - reserved 1; - - // If set, we will try to include an EventToken with the responses. - bool include_event_token = 2; - - // If set, the fresher field may be set as a result comparison to this token. - // This is a shortcut so the application doesn't need to care about - // comparing EventTokens. - EventToken compare_event_token = 3; + // 2 used to be include_event_token + // 3 used to be compare_event_token + reserved 1, 2, 3; enum IncludedFields { TYPE_AND_NAME = 0; @@ -345,18 +339,6 @@ message Row { bytes values = 2; } -// ResultExtras contains optional out-of-band information. Usually the -// extras are requested by adding ExecuteOptions flags. -message ResultExtras { - // event_token is populated if the include_event_token flag is set - // in ExecuteOptions. - EventToken event_token = 1; - - // If set, it means the data returned with this result is fresher - // than the compare_token passed in the ExecuteOptions. - bool fresher = 2; -} - // QueryResult is returned by Execute and ExecuteStream. // // As returned by Execute, len(fields) is always equal to len(row) @@ -367,11 +349,13 @@ message ResultExtras { // len(QueryResult[0].fields) is always equal to len(row) (for each // row in rows for each QueryResult in QueryResult[1:]). message QueryResult { + // This used to be ResultExtras. + reserved 5; + repeated Field fields = 1; uint64 rows_affected = 2; uint64 insert_id = 3; repeated Row rows = 4; - ResultExtras extras = 5; } // QueryWarning is used to convey out of band query execution warnings From 8647051ec265d1c77c3ce2f62de1b2d6e72ee054 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 11 Mar 2020 17:40:04 -0700 Subject: [PATCH 03/18] tabletserver: simplify some code Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/query_executor.go | 15 +---- .../tabletserver/replication_watcher.go | 1 + go/vt/vttablet/tabletserver/tabletserver.go | 62 ++++++++----------- 3 files changed, 29 insertions(+), 49 deletions(-) diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 68b5b9cba83..a7a695f3232 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -139,7 +139,7 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { case planbuilder.PlanSet: return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) case planbuilder.PlanPassSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: - return qre.execDirect(conn) + return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, false) default: // handled above: // planbuilder.PlanNextval @@ -522,19 +522,6 @@ func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) { }, nil } -// execDirect is for reads inside transactions. Always send to MySQL. -func (qre *QueryExecutor) execDirect(conn *TxConnection) (*sqltypes.Result, error) { - if qre.tsv.qe.enableQueryPlanFieldCaching && qre.plan.Fields != nil { - result, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, false) - if err != nil { - return nil, err - } - result.Fields = qre.plan.Fields - return result, nil - } - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, false) -} - // execSelect sends a query to mysql only if another identical query is not running. Otherwise, it waits and // reuses the result. If the plan is missng field info, it sends the query to mysql requesting full info. func (qre *QueryExecutor) execSelect() (*sqltypes.Result, error) { diff --git a/go/vt/vttablet/tabletserver/replication_watcher.go b/go/vt/vttablet/tabletserver/replication_watcher.go index 9d2d13b0f12..9d6966a9c03 100644 --- a/go/vt/vttablet/tabletserver/replication_watcher.go +++ b/go/vt/vttablet/tabletserver/replication_watcher.go @@ -82,6 +82,7 @@ func (rpw *ReplicationWatcher) Process(ctx context.Context) { } for { + // VStreamer will reload the schema when it encounters a DDL. err := rpw.vs.Stream(ctx, "current", filter, func(events []*binlogdatapb.VEvent) error { return nil }) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 9e3a443624e..b63b73b519d 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -192,14 +192,10 @@ type TabletServer struct { alias topodatapb.TabletAlias } -// RegisterFunction is a callback type to be called when we -// Register() a TabletServer -type RegisterFunction func(Controller) - // RegisterFunctions is a list of all the // RegisterFunction that will be called upon // Register() on a TabletServer -var RegisterFunctions []RegisterFunction +var RegisterFunctions []func(Controller) // NewServer creates a new TabletServer based on the command line flags. func NewServer(topoServer *topo.Server, alias topodatapb.TabletAlias) *TabletServer { @@ -717,12 +713,11 @@ func (tsv *TabletServer) Begin(ctx context.Context, target *querypb.Target, opti err = tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "Begin", "begin", nil, - target, options, true /* isBegin */, false, /* allowOnShutdown */ + target, options, false, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { startTime := time.Now() if tsv.txThrottler.Throttle() { - // TODO(erez): I think this should be RESOURCE_EXHAUSTED. - return vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "Transaction throttled") + return vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "Transaction throttled") } var beginSQL string transactionID, beginSQL, err = tsv.te.Begin(ctx, options) @@ -749,7 +744,7 @@ func (tsv *TabletServer) Commit(ctx context.Context, target *querypb.Target, tra return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "Commit", "commit", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { startTime := time.Now() logStats.TransactionID = transactionID @@ -775,7 +770,7 @@ func (tsv *TabletServer) Rollback(ctx context.Context, target *querypb.Target, t return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "Rollback", "rollback", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { defer tabletenv.QueryStats.Record("ROLLBACK", time.Now()) logStats.TransactionID = transactionID @@ -789,7 +784,7 @@ func (tsv *TabletServer) Prepare(ctx context.Context, target *querypb.Target, tr return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "Prepare", "prepare", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -806,7 +801,7 @@ func (tsv *TabletServer) CommitPrepared(ctx context.Context, target *querypb.Tar return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "CommitPrepared", "commit_prepared", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -823,7 +818,7 @@ func (tsv *TabletServer) RollbackPrepared(ctx context.Context, target *querypb.T return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "RollbackPrepared", "rollback_prepared", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -840,7 +835,7 @@ func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb. return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "CreateTransaction", "create_transaction", nil, - target, nil, true /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -858,7 +853,7 @@ func (tsv *TabletServer) StartCommit(ctx context.Context, target *querypb.Target return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "StartCommit", "start_commit", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -876,7 +871,7 @@ func (tsv *TabletServer) SetRollback(ctx context.Context, target *querypb.Target return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "SetRollback", "set_rollback", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -894,7 +889,7 @@ func (tsv *TabletServer) ConcludeTransaction(ctx context.Context, target *queryp return tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "ConcludeTransaction", "conclude_transaction", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -911,7 +906,7 @@ func (tsv *TabletServer) ReadTransaction(ctx context.Context, target *querypb.Ta err = tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "ReadTransaction", "read_transaction", nil, - target, nil, false /* isBegin */, true, /* allowOnShutdown */ + target, nil, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { txe := &TxExecutor{ ctx: ctx, @@ -935,7 +930,7 @@ func (tsv *TabletServer) Execute(ctx context.Context, target *querypb.Target, sq err = tsv.execRequest( ctx, tsv.QueryTimeout.Get(), "Execute", sql, bindVariables, - target, options, false /* isBegin */, allowOnShutdown, + target, options, allowOnShutdown, func(ctx context.Context, logStats *tabletenv.LogStats) error { if bindVariables == nil { bindVariables = make(map[string]*querypb.BindVariable) @@ -1003,7 +998,7 @@ func (tsv *TabletServer) StreamExecute(ctx context.Context, target *querypb.Targ return tsv.execRequest( ctx, 0, "StreamExecute", sql, bindVariables, - target, options, false /* isBegin */, false, /* allowOnShutdown */ + target, options, false, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { if bindVariables == nil { bindVariables = make(map[string]*querypb.BindVariable) @@ -1066,10 +1061,10 @@ func (tsv *TabletServer) ExecuteBatch(ctx context.Context, target *querypb.Targe // tsv.convertAndLogError. That's because the methods which returned "err", // e.g. tsv.Execute(), already called that function and therefore already // converted and logged the error. - if err = tsv.startRequest(ctx, target, false /* isBegin */, allowOnShutdown); err != nil { + if err = tsv.startRequest(ctx, target, allowOnShutdown); err != nil { return nil, err } - defer tsv.endRequest(false) + defer tsv.endRequest() defer tsv.handlePanicAndSendLogStats("batch", nil, nil) if options == nil { @@ -1159,7 +1154,7 @@ func (tsv *TabletServer) beginWaitForSameRangeTransactions(ctx context.Context, // -queryserver-config-txpool-timeout (defaults to 1s) to limit the waiting. ctx, tsv.QueryTimeout.Get(), "", "waitForSameRangeTransactions", nil, - target, options, true /* isBegin */, false, /* allowOnShutdown */ + target, options, false, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { k, table := tsv.computeTxSerializerKey(ctx, logStats, sql, bindVariables) if k == "" { @@ -1233,7 +1228,7 @@ func (tsv *TabletServer) MessageStream(ctx context.Context, target *querypb.Targ return tsv.execRequest( ctx, 0, "MessageStream", "stream", nil, - target, nil, false /* isBegin */, false, /* allowOnShutdown */ + target, nil, false, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { plan, err := tsv.qe.GetMessageStreamPlan(name) if err != nil { @@ -1285,10 +1280,10 @@ func (tsv *TabletServer) PurgeMessages(ctx context.Context, target *querypb.Targ } func (tsv *TabletServer) execDML(ctx context.Context, target *querypb.Target, queryGenerator func() (string, map[string]*querypb.BindVariable, error)) (count int64, err error) { - if err = tsv.startRequest(ctx, target, true /* isBegin */, false /* allowOnShutdown */); err != nil { + if err = tsv.startRequest(ctx, target, false /* allowOnShutdown */); err != nil { return 0, err } - defer tsv.endRequest(true) + defer tsv.endRequest() defer tsv.handlePanicAndSendLogStats("ack", nil, nil) query, bv, err := queryGenerator() @@ -1356,7 +1351,7 @@ func (tsv *TabletServer) VStreamResults(ctx context.Context, target *querypb.Tar func (tsv *TabletServer) execRequest( ctx context.Context, timeout time.Duration, requestName, sql string, bindVariables map[string]*querypb.BindVariable, - target *querypb.Target, options *querypb.ExecuteOptions, isBegin, allowOnShutdown bool, + target *querypb.Target, options *querypb.ExecuteOptions, allowOnShutdown bool, exec func(ctx context.Context, logStats *tabletenv.LogStats) error, ) (err error) { span, ctx := trace.NewSpan(ctx, "TabletServer."+requestName) @@ -1376,14 +1371,14 @@ func (tsv *TabletServer) execRequest( logStats.OriginalSQL = sql logStats.BindVariables = bindVariables defer tsv.handlePanicAndSendLogStats(sql, bindVariables, logStats) - if err = tsv.startRequest(ctx, target, isBegin, allowOnShutdown); err != nil { + if err = tsv.startRequest(ctx, target, allowOnShutdown); err != nil { return err } ctx, cancel := withTimeout(ctx, timeout, options) defer func() { cancel() - tsv.endRequest(isBegin) + tsv.endRequest() }() err = exec(ctx, logStats) @@ -1740,11 +1735,8 @@ func (tsv *TabletServer) Close(ctx context.Context) error { // the request (a waitgroup) as started. Every startRequest requires // one and only one corresponding endRequest. When the service shuts // down, StopService will wait on this waitgroup to ensure that there -// are no requests in flight. For begin requests, isBegin must be set -// to true, which increments an additional waitgroup. During state -// transitions, this waitgroup will be checked to make sure that no -// such statements are in-flight while we resolve the tx pool. -func (tsv *TabletServer) startRequest(ctx context.Context, target *querypb.Target, isBegin, allowOnShutdown bool) (err error) { +// are no requests in flight. +func (tsv *TabletServer) startRequest(ctx context.Context, target *querypb.Target, allowOnShutdown bool) (err error) { tsv.mu.Lock() defer tsv.mu.Unlock() if tsv.state == StateServing { @@ -1781,7 +1773,7 @@ ok: } // endRequest unregisters the current request (a waitgroup) as done. -func (tsv *TabletServer) endRequest(isBegin bool) { +func (tsv *TabletServer) endRequest() { tsv.requests.Done() } From cbd81a39906b78cf88bf4e0a6a133f6b0e9c7aaf Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Fri, 13 Mar 2020 17:42:45 -0700 Subject: [PATCH 04/18] deprecation: deprecate DML_PK Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/planbuilder/dml.go | 146 ++++-------------- .../tabletserver/planbuilder/query_gen.go | 18 +-- go/vt/vttablet/tabletserver/schema/schema.go | 13 +- 3 files changed, 52 insertions(+), 125 deletions(-) diff --git a/go/vt/vttablet/tabletserver/planbuilder/dml.go b/go/vt/vttablet/tabletserver/planbuilder/dml.go index 54be33c5486..16d0baacae8 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/dml.go +++ b/go/vt/vttablet/tabletserver/planbuilder/dml.go @@ -48,21 +48,16 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan plan.Reason = ReasonTable return plan, nil } - table, tableErr := plan.setTable(tableName, tables) - - // Updates aren't supported on topics - if tableErr == nil && table.IsTopic() { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "updates not allowed on topics") - } + table, _ := plan.setTable(tableName, tables) - // In passthrough dml mode, allow the operation even if the - // table is unknown in the schema. - if PassthroughDMLs { + // Drop to pass-through mode if table cannot be indentified. + if PassthroughDMLs || table == nil { return plan, nil } - if tableErr != nil { - return nil, tableErr + // Updates aren't supported on topics + if table.IsTopic() { + return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "updates not allowed on topics") } // Store the WHERE clause as string for the hot row protection (txserializer). @@ -70,32 +65,12 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan buf.Myprintf("%v", upd.Where) plan.WhereClause = buf.ParsedQuery() - if !table.HasPrimary() { - log.Warningf("no primary key for table %s", tableName) - plan.Reason = ReasonTableNoIndex + if hasPKConstraint(upd.Where, table) { return plan, nil } - plan.SecondaryPKValues, err = analyzeUpdateExpressions(upd.Exprs, table.Indexes[0]) - if err != nil { - if err == ErrTooComplex { - plan.Reason = ReasonPKChange - return plan, nil - } - return nil, err - } - plan.OuterQuery = GenerateUpdateOuterQuery(upd, aliased, nil) - if pkValues := analyzeWhere(upd.Where, table.Indexes[0]); pkValues != nil { - // Also, there should be no limit clause. - if upd.Limit == nil { - plan.PlanID = PlanDMLPK - plan.PKValues = pkValues - return plan, nil - } - } - plan.PlanID = PlanDMLSubquery plan.Subquery = GenerateUpdateSubquery(upd, table, aliased) return plan, nil @@ -111,31 +86,28 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan plan.Reason = ReasonMultiTable return plan, nil } + aliased, ok := del.TableExprs[0].(*sqlparser.AliasedTableExpr) if !ok { plan.Reason = ReasonMultiTable return plan, nil } + tableName := sqlparser.GetTableName(aliased.Expr) if tableName.IsEmpty() { plan.Reason = ReasonTable return plan, nil } - table, tableErr := plan.setTable(tableName, tables) + table, _ := plan.setTable(tableName, tables) - // Deletes aren't supported on topics - if tableErr == nil && table.IsTopic() { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "deletes not allowed on topics") - } - - // In passthrough dml mode, allow the operation even if the - // table is unknown in the schema. - if PassthroughDMLs { + // Drop to pass-through mode if table cannot be indentified. + if PassthroughDMLs || table == nil { return plan, nil } - if tableErr != nil { - return nil, tableErr + // Deletes aren't supported on topics + if table.IsTopic() { + return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "deletes not allowed on topics") } // Store the WHERE clause as string for the hot row protection (txserializer). @@ -143,23 +115,12 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan buf.Myprintf("%v", del.Where) plan.WhereClause = buf.ParsedQuery() - if !table.HasPrimary() { - log.Warningf("no primary key for table %s", tableName) - plan.Reason = ReasonTableNoIndex + if hasPKConstraint(del.Where, table) { return plan, nil } plan.OuterQuery = GenerateDeleteOuterQuery(del, aliased) - if pkValues := analyzeWhere(del.Where, table.Indexes[0]); pkValues != nil { - // Also, there should be no limit clause. - if del.Limit == nil { - plan.PlanID = PlanDMLPK - plan.PKValues = pkValues - return plan, nil - } - } - plan.PlanID = PlanDMLSubquery plan.Subquery = GenerateDeleteSubquery(del, table, aliased) return plan, nil @@ -172,27 +133,6 @@ func analyzeSet(set *sqlparser.Set) (plan *Plan) { } } -func analyzeUpdateExpressions(exprs sqlparser.UpdateExprs, pkIndex *schema.Index) (pkValues []sqltypes.PlanValue, err error) { - for _, expr := range exprs { - index := pkIndex.FindColumn(expr.Name.Name) - if index == -1 { - continue - } - if !sqlparser.IsValue(expr.Expr) { - return nil, ErrTooComplex - } - if pkValues == nil { - pkValues = make([]sqltypes.PlanValue, len(pkIndex.Columns)) - } - var err error - pkValues[index], err = sqlparser.NewPlanValue(expr.Expr) - if err != nil { - return nil, err - } - } - return pkValues, nil -} - func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ PlanID: PlanPassSelect, @@ -253,28 +193,22 @@ func analyzeFrom(tableExprs sqlparser.TableExprs) sqlparser.TableIdent { return sqlparser.GetTableName(node.Expr) } -func analyzeWhere(node *sqlparser.Where, pkIndex *schema.Index) []sqltypes.PlanValue { - if node == nil { - return nil +func hasPKConstraint(node *sqlparser.Where, table *schema.Table) bool { + if node == nil || len(table.PKColumns) == 0 { + return false } - conditions := analyzeBoolean(node.Expr) - if conditions == nil { - return nil - } - return getPKValues(conditions, pkIndex) + conditions := splitBool(node.Expr) + return matchPKs(conditions, table) } -func analyzeBoolean(node sqlparser.Expr) (conditions []*sqlparser.ComparisonExpr) { +func splitBool(node sqlparser.Expr) (conditions []*sqlparser.ComparisonExpr) { switch node := node.(type) { case *sqlparser.AndExpr: - left := analyzeBoolean(node.Left) - right := analyzeBoolean(node.Right) - if left == nil || right == nil { - return nil - } + left := splitBool(node.Left) + right := splitBool(node.Right) return append(left, right...) case *sqlparser.ParenExpr: - return analyzeBoolean(node.Expr) + return splitBool(node.Expr) case *sqlparser.ComparisonExpr: switch { case node.Operator == sqlparser.EqualStr: @@ -290,35 +224,21 @@ func analyzeBoolean(node sqlparser.Expr) (conditions []*sqlparser.ComparisonExpr return nil } -func getPKValues(conditions []*sqlparser.ComparisonExpr, pkIndex *schema.Index) []sqltypes.PlanValue { - pkValues := make([]sqltypes.PlanValue, len(pkIndex.Columns)) - inClauseSeen := false +func matchPKs(conditions []*sqlparser.ComparisonExpr, table *schema.Table) bool { + pkValues := make([]bool, len(table.PKColumns)) for _, condition := range conditions { - if condition.Operator == sqlparser.InStr { - if inClauseSeen { - return nil - } - inClauseSeen = true - } - index := pkIndex.FindColumn(condition.Left.(*sqlparser.ColName).Name) + index := table.FindPKColumn(condition.Left.(*sqlparser.ColName).Name) if index == -1 { - return nil - } - if !pkValues[index].IsNull() { - return nil - } - var err error - pkValues[index], err = sqlparser.NewPlanValue(condition.Right) - if err != nil { - return nil + continue } + pkValues[index] = true } for _, v := range pkValues { - if v.IsNull() { - return nil + if !v { + return false } } - return pkValues + return true } func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan *Plan, err error) { diff --git a/go/vt/vttablet/tabletserver/planbuilder/query_gen.go b/go/vt/vttablet/tabletserver/planbuilder/query_gen.go index 0418d2bb969..6dd37591ebf 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/query_gen.go +++ b/go/vt/vttablet/tabletserver/planbuilder/query_gen.go @@ -98,42 +98,38 @@ func GenerateDeleteOuterQuery(del *sqlparser.Delete, aliased *sqlparser.AliasedT // GenerateUpdateSubquery generates the subquery for updates. func GenerateUpdateSubquery(upd *sqlparser.Update, table *schema.Table, aliased *sqlparser.AliasedTableExpr) *sqlparser.ParsedQuery { return GenerateSubquery( - table.Indexes[0].Columns, + table, aliased, upd.Where, upd.OrderBy, upd.Limit, - true, ) } // GenerateDeleteSubquery generates the subquery for deletes. func GenerateDeleteSubquery(del *sqlparser.Delete, table *schema.Table, aliased *sqlparser.AliasedTableExpr) *sqlparser.ParsedQuery { return GenerateSubquery( - table.Indexes[0].Columns, + table, aliased, del.Where, del.OrderBy, del.Limit, - true, ) } // GenerateSubquery generates a subquery based on the input parameters. -func GenerateSubquery(columns []sqlparser.ColIdent, table *sqlparser.AliasedTableExpr, where *sqlparser.Where, order sqlparser.OrderBy, limit *sqlparser.Limit, forUpdate bool) *sqlparser.ParsedQuery { +func GenerateSubquery(table *schema.Table, tableName *sqlparser.AliasedTableExpr, where *sqlparser.Where, order sqlparser.OrderBy, limit *sqlparser.Limit) *sqlparser.ParsedQuery { buf := sqlparser.NewTrackedBuffer(nil) if limit == nil { limit = execLimit } buf.WriteString("select ") prefix := "" - for _, c := range columns { - buf.Myprintf("%s%v", prefix, c) + for _, colnum := range table.PKColumns { + buf.Myprintf("%s%v", prefix, table.Columns[colnum].Name) prefix = ", " } - buf.Myprintf(" from %v%v%v%v", table, where, order, limit) - if forUpdate { - buf.Myprintf(sqlparser.ForUpdateStr) - } + buf.Myprintf(" from %v%v%v%v", tableName, where, order, limit) + buf.Myprintf(sqlparser.ForUpdateStr) return buf.ParsedQuery() } diff --git a/go/vt/vttablet/tabletserver/schema/schema.go b/go/vt/vttablet/tabletserver/schema/schema.go index 3f6bce060a9..c943420d5a8 100644 --- a/go/vt/vttablet/tabletserver/schema/schema.go +++ b/go/vt/vttablet/tabletserver/schema/schema.go @@ -184,6 +184,17 @@ func (ta *Table) FindColumn(name sqlparser.ColIdent) int { return -1 } +// FindPKColumn finds a pk column in the table. It returns the index if found. +// Otherwise, it returns -1. +func (ta *Table) FindPKColumn(name sqlparser.ColIdent) int { + for i, colnum := range ta.PKColumns { + if ta.Columns[colnum].Name.Equal(name) { + return i + } + } + return -1 +} + // GetPKColumn returns the pk column specified by the index. func (ta *Table) GetPKColumn(index int) *TableColumn { return &ta.Columns[ta.PKColumns[index]] @@ -212,7 +223,7 @@ func (ta *Table) SetMysqlStats(tr, dl, il, df, mdl sqltypes.Value) { // HasPrimary returns true if the table has a primary key. func (ta *Table) HasPrimary() bool { - return len(ta.Indexes) != 0 && ta.Indexes[0].Name.EqualString("primary") + return len(ta.PKColumns) != 0 } // IsTopic returns true if TopicInfo is not nil. From 585fa2a99a11fd6b7da18d7b57ffc5e9a8299c69 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 14 Mar 2020 19:08:54 -0700 Subject: [PATCH 05/18] deprecation: implement DMLLimit Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/planbuilder/dml.go | 200 ++++-------------- .../vttablet/tabletserver/planbuilder/plan.go | 21 +- go/vt/vttablet/tabletserver/query_executor.go | 127 ++--------- .../tabletserver/query_executor_test.go | 133 ------------ go/vt/vttablet/tabletserver/tabletserver.go | 2 +- 5 files changed, 71 insertions(+), 412 deletions(-) diff --git a/go/vt/vttablet/tabletserver/planbuilder/dml.go b/go/vt/vttablet/tabletserver/planbuilder/dml.go index 16d0baacae8..205f980548e 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/dml.go +++ b/go/vt/vttablet/tabletserver/planbuilder/dml.go @@ -28,36 +28,8 @@ import ( func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassDML, - FullQuery: GenerateFullQuery(upd), - } - - if len(upd.TableExprs) > 1 { - plan.Reason = ReasonMultiTable - return plan, nil - } - - aliased, ok := upd.TableExprs[0].(*sqlparser.AliasedTableExpr) - if !ok { - plan.Reason = ReasonMultiTable - return plan, nil - } - - tableName := sqlparser.GetTableName(aliased.Expr) - if tableName.IsEmpty() { - plan.Reason = ReasonTable - return plan, nil - } - table, _ := plan.setTable(tableName, tables) - - // Drop to pass-through mode if table cannot be indentified. - if PassthroughDMLs || table == nil { - return plan, nil - } - - // Updates aren't supported on topics - if table.IsTopic() { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "updates not allowed on topics") + PlanID: PlanPassDML, + Table: lookupTable(upd.TableExprs, tables), } // Store the WHERE clause as string for the hot row protection (txserializer). @@ -65,49 +37,22 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan buf.Myprintf("%v", upd.Where) plan.WhereClause = buf.ParsedQuery() - if hasPKConstraint(upd.Where, table) { + if PassthroughDMLs || upd.Limit != nil { + plan.FullQuery = GenerateFullQuery(upd) return plan, nil } - plan.OuterQuery = GenerateUpdateOuterQuery(upd, aliased, nil) - - plan.PlanID = PlanDMLSubquery - plan.Subquery = GenerateUpdateSubquery(upd, table, aliased) + plan.PlanID = PlanDMLLimit + upd.Limit = execLimit + plan.FullQuery = GenerateFullQuery(upd) + upd.Limit = nil return plan, nil } func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassDML, - FullQuery: GenerateFullQuery(del), - } - - if len(del.TableExprs) > 1 { - plan.Reason = ReasonMultiTable - return plan, nil - } - - aliased, ok := del.TableExprs[0].(*sqlparser.AliasedTableExpr) - if !ok { - plan.Reason = ReasonMultiTable - return plan, nil - } - - tableName := sqlparser.GetTableName(aliased.Expr) - if tableName.IsEmpty() { - plan.Reason = ReasonTable - return plan, nil - } - table, _ := plan.setTable(tableName, tables) - - // Drop to pass-through mode if table cannot be indentified. - if PassthroughDMLs || table == nil { - return plan, nil - } - - // Deletes aren't supported on topics - if table.IsTopic() { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "deletes not allowed on topics") + PlanID: PlanPassDML, + Table: lookupTable(del.TableExprs, tables), } // Store the WHERE clause as string for the hot row protection (txserializer). @@ -115,14 +60,14 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan buf.Myprintf("%v", del.Where) plan.WhereClause = buf.ParsedQuery() - if hasPKConstraint(del.Where, table) { + if PassthroughDMLs || del.Limit != nil { + plan.FullQuery = GenerateFullQuery(del) return plan, nil } - - plan.OuterQuery = GenerateDeleteOuterQuery(del, aliased) - - plan.PlanID = PlanDMLSubquery - plan.Subquery = GenerateDeleteSubquery(del, table, aliased) + plan.PlanID = PlanDMLLimit + del.Limit = execLimit + plan.FullQuery = GenerateFullQuery(del) + del.Limit = nil return plan, nil } @@ -136,6 +81,7 @@ func analyzeSet(set *sqlparser.Set) (plan *Plan) { func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ PlanID: PlanPassSelect, + Table: lookupTable(sel.From, tables), FieldQuery: GenerateFieldQuery(sel), FullQuery: GenerateLimitQuery(sel), } @@ -143,20 +89,6 @@ func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan plan.PlanID = PlanSelectLock } - tableName := analyzeFrom(sel.From) - if tableName.IsEmpty() { - return plan, nil - } - table, err := plan.setTable(tableName, tables) - if err != nil { - return nil, err - } - - // Selects aren't supported on topics - if table.IsTopic() { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "selects not allowed on topics") - } - if sel.Where != nil { comp, ok := sel.Where.Expr.(*sqlparser.ComparisonExpr) if ok && comp.IsImpossible() { @@ -167,8 +99,8 @@ func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan // Check if it's a NEXT VALUE statement. if nextVal, ok := sel.SelectExprs[0].(sqlparser.Nextval); ok { - if table.Type != schema.Sequence { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s is not a sequence", tableName) + if plan.Table == nil || plan.Table.Type != schema.Sequence { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s is not a sequence", sqlparser.String(sel.From)) } plan.PlanID = PlanNextval v, err := sqlparser.NewPlanValue(nextVal.Expr) @@ -182,65 +114,6 @@ func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan return plan, nil } -func analyzeFrom(tableExprs sqlparser.TableExprs) sqlparser.TableIdent { - if len(tableExprs) > 1 { - return sqlparser.NewTableIdent("") - } - node, ok := tableExprs[0].(*sqlparser.AliasedTableExpr) - if !ok { - return sqlparser.NewTableIdent("") - } - return sqlparser.GetTableName(node.Expr) -} - -func hasPKConstraint(node *sqlparser.Where, table *schema.Table) bool { - if node == nil || len(table.PKColumns) == 0 { - return false - } - conditions := splitBool(node.Expr) - return matchPKs(conditions, table) -} - -func splitBool(node sqlparser.Expr) (conditions []*sqlparser.ComparisonExpr) { - switch node := node.(type) { - case *sqlparser.AndExpr: - left := splitBool(node.Left) - right := splitBool(node.Right) - return append(left, right...) - case *sqlparser.ParenExpr: - return splitBool(node.Expr) - case *sqlparser.ComparisonExpr: - switch { - case node.Operator == sqlparser.EqualStr: - if sqlparser.IsColName(node.Left) && sqlparser.IsValue(node.Right) { - return []*sqlparser.ComparisonExpr{node} - } - case node.Operator == sqlparser.InStr: - if sqlparser.IsColName(node.Left) && sqlparser.IsSimpleTuple(node.Right) { - return []*sqlparser.ComparisonExpr{node} - } - } - } - return nil -} - -func matchPKs(conditions []*sqlparser.ComparisonExpr, table *schema.Table) bool { - pkValues := make([]bool, len(table.PKColumns)) - for _, condition := range conditions { - index := table.FindPKColumn(condition.Left.(*sqlparser.ColName).Name) - if index == -1 { - continue - } - pkValues[index] = true - } - for _, v := range pkValues { - if !v { - return false - } - } - return true -} - func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ PlanID: PlanPassDML, @@ -256,14 +129,17 @@ func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan plan.Reason = ReasonTable return plan, nil } - table, tableErr := plan.setTable(tableName, tables) + plan.Table = tables[tableName.String()] + if plan.Table == nil { + return plan, nil + } switch { - case tableErr == nil && table.Type == schema.Message: + case plan.Table.Type == schema.Message: // message inserts need to continue being strict, even in passthrough dml mode, // because field defaults are set here - case tableErr == nil && table.IsTopic(): + case plan.Table.IsTopic(): plan.PlanID = PlanInsertTopic plan.Reason = ReasonTopic return plan, nil @@ -272,22 +148,19 @@ func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan // In passthrough dml mode, allow the operation even if the // table is unknown in the schema. return plan, nil - - case tableErr != nil: - return nil, tableErr } - if !table.HasPrimary() { + if !plan.Table.HasPrimary() { log.Warningf("no primary key for table %s", tableName) plan.Reason = ReasonTableNoIndex return plan, nil } - switch table.Type { + switch plan.Table.Type { case schema.NoType, schema.Sequence: // For now, allow sequence inserts. - return analyzeInsertNoType(ins, plan, table) + return analyzeInsertNoType(ins, plan, plan.Table) case schema.Message: - return analyzeInsertMessage(ins, plan, table) + return analyzeInsertMessage(ins, plan, plan.Table) } panic("unreachable") } @@ -572,6 +445,21 @@ func analyzeOnDupExpressions(ins *sqlparser.Insert, pkIndex *schema.Index) (pkVa return pkValues, true } +func lookupTable(tableExprs sqlparser.TableExprs, tables map[string]*schema.Table) *schema.Table { + if len(tableExprs) > 1 { + return nil + } + aliased, ok := tableExprs[0].(*sqlparser.AliasedTableExpr) + if !ok { + return nil + } + tableName := sqlparser.GetTableName(aliased.Expr) + if tableName.IsEmpty() { + return nil + } + return tables[tableName.String()] +} + // extractColumnValues extracts the values of a column into a PlanValue. func extractColumnValues(rowList sqlparser.Values, colnum int) (sqltypes.PlanValue, bool) { pv := sqltypes.PlanValue{Values: make([]sqltypes.PlanValue, len(rowList))} diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index 87e119f3c52..a682b7f9aa2 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -57,11 +57,8 @@ const ( // and is valid in all replication modes. // Otherwise is only allowed in row based replication mode PlanPassDML - // PlanDMLPK is an update or delete with an equality where clause(s) - // on primary key(s). - PlanDMLPK - // PlanDMLSubquery is an update or delete with a subselect statement - PlanDMLSubquery + // PlanDMLLimit is an update or delete with a limit. + PlanDMLLimit // PlanInsertPK is insert statement where the PK value is // supplied with the query. PlanInsertPK @@ -97,8 +94,7 @@ var planName = [NumPlans]string{ "SELECT_LOCK", "NEXTVAL", "PASS_DML", - "DML_PK", - "DML_SUBQUERY", + "DML_LIMIT", "INSERT_PK", "INSERT_SUBQUERY", "UPSERT_PK", @@ -240,13 +236,6 @@ func (plan *Plan) TableName() sqlparser.TableIdent { return tableName } -func (plan *Plan) setTable(tableName sqlparser.TableIdent, tables map[string]*schema.Table) (*schema.Table, error) { - if plan.Table = tables[tableName.String()]; plan.Table == nil { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "table %s not found in schema", tableName) - } - return plan.Table, nil -} - // Build builds a plan based on the schema. func Build(statement sqlparser.Statement, tables map[string]*schema.Table) (*Plan, error) { var plan *Plan @@ -314,9 +303,7 @@ func BuildStreaming(sql string, tables map[string]*schema.Table) (*Plan, error) if stmt.Lock != "" { return nil, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "select with lock not allowed for streaming") } - if tableName := analyzeFrom(stmt.From); !tableName.IsEmpty() { - plan.setTable(tableName, tables) - } + plan.Table = lookupTable(stmt.From, tables) case *sqlparser.OtherRead, *sqlparser.Show, *sqlparser.Union: // pass default: diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index a7a695f3232..c6491bfde5b 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -128,10 +128,8 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { return qre.execInsertMessage(conn) case planbuilder.PlanInsertSubquery: return qre.execInsertSubquery(conn) - case planbuilder.PlanDMLPK: - return qre.execDMLPK(conn) - case planbuilder.PlanDMLSubquery: - return qre.execDMLSubquery(conn) + case planbuilder.PlanDMLLimit: + return qre.execDMLLimit(conn) case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: return qre.execSQL(conn, qre.query, true) case planbuilder.PlanUpsertPK: @@ -174,9 +172,7 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { fallthrough case planbuilder.PlanInsertSubquery: fallthrough - case planbuilder.PlanDMLPK: - fallthrough - case planbuilder.PlanDMLSubquery: + case planbuilder.PlanDMLLimit: fallthrough case planbuilder.PlanUpsertPK: if !qre.tsv.qe.autoCommit.Get() { @@ -278,10 +274,8 @@ func (qre *QueryExecutor) execDmlAutoCommit() (reply *sqltypes.Result, err error return qre.execInsertMessage(conn) case planbuilder.PlanInsertSubquery: reply, err = qre.execInsertSubquery(conn) - case planbuilder.PlanDMLPK: - reply, err = qre.execDMLPK(conn) - case planbuilder.PlanDMLSubquery: - reply, err = qre.execDMLSubquery(conn) + case planbuilder.PlanDMLLimit: + reply, err = qre.execDMLLimit(conn) case planbuilder.PlanUpsertPK: reply, err = qre.execUpsertPK(conn) default: @@ -597,109 +591,34 @@ func (qre *QueryExecutor) execInsertPKRows(conn *TxConnection, extras map[string } func (qre *QueryExecutor) execUpsertPK(conn *TxConnection) (*sqltypes.Result, error) { - // For RBR, upserts are passed through. - if qre.tsv.qe.binlogFormat == connpool.BinlogFormatRow { - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) - } - - // For statement or mixed mode, we have to split into two ops. - pkRows, err := buildValueList(qre.plan.Table, qre.plan.PKValues, qre.bindVars) - if err != nil { - return nil, err - } - // We do not need to build the secondary list for the insert part. - // But the part that updates will build it if it gets executed, - // because it's the one that can change the primary keys. - bsc := buildStreamComment(qre.plan.Table, pkRows, nil) - result, err := qre.txFetch(conn, qre.plan.OuterQuery, qre.bindVars, nil, bsc, true, true) - if err == nil { - return result, nil - } - sqlErr, ok := err.(*mysql.SQLError) - if !ok { - return result, err - } - if sqlErr.Number() != mysql.ERDupEntry { - return nil, err - } - // If the error didn't match pk, just return the error without updating. - if !strings.Contains(sqlErr.Error(), "'PRIMARY'") { - return nil, err - } - // At this point, we know the insert failed due to a duplicate pk row. - // So, we just update the row. - result, err = qre.execDMLPKRows(conn, qre.plan.UpsertQuery, pkRows) - if err != nil { - return nil, err - } - // Follow MySQL convention. RowsAffected must be 2 if a row was updated. - if result.RowsAffected == 1 { - result.RowsAffected = 2 - } - return result, err + return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) } -func (qre *QueryExecutor) execDMLPK(conn *TxConnection) (*sqltypes.Result, error) { - pkRows, err := buildValueList(qre.plan.Table, qre.plan.PKValues, qre.bindVars) +func (qre *QueryExecutor) execDMLLimit(conn *TxConnection) (*sqltypes.Result, error) { + maxrows := qre.tsv.qe.maxResultSize.Get() + qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) + result, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) if err != nil { return nil, err } - return qre.execDMLPKRows(conn, qre.plan.OuterQuery, pkRows) -} - -func (qre *QueryExecutor) execDMLSubquery(conn *TxConnection) (*sqltypes.Result, error) { - innerResult, err := qre.txFetch(conn, qre.plan.Subquery, qre.bindVars, nil, "", true, false) - if err != nil { + if err := qre.verifyRowCount(int64(result.RowsAffected), maxrows); err != nil { return nil, err } - return qre.execDMLPKRows(conn, qre.plan.OuterQuery, innerResult.Rows) + return result, nil } -func (qre *QueryExecutor) execDMLPKRows(conn *TxConnection, query *sqlparser.ParsedQuery, pkRows [][]sqltypes.Value) (*sqltypes.Result, error) { - if len(pkRows) == 0 { - return &sqltypes.Result{RowsAffected: 0}, nil - } - secondaryList, err := buildSecondaryList(qre.plan.Table, pkRows, qre.plan.SecondaryPKValues, qre.bindVars) - if err != nil { - return nil, err +func (qre *QueryExecutor) verifyRowCount(count, maxrows int64) error { + if count > maxrows { + callerID := callerid.ImmediateCallerIDFromContext(qre.ctx) + return mysql.NewSQLError(mysql.ERVitessMaxRowsExceeded, mysql.SSUnknownSQLState, "caller id: %s: row count exceeded %d", callerID.Username, maxrows) } - - result := &sqltypes.Result{} - maxRows := int(qre.tsv.qe.maxDMLRows.Get()) - for i := 0; i < len(pkRows); i += maxRows { - end := i + maxRows - if end >= len(pkRows) { - end = len(pkRows) - } - pkRows := pkRows[i:end] - secondaryList := secondaryList - if secondaryList != nil { - secondaryList = secondaryList[i:end] - } - var bsc string - // Build comments only if we're not in RBR mode. - if qre.tsv.qe.binlogFormat != connpool.BinlogFormatRow { - bsc = buildStreamComment(qre.plan.Table, pkRows, secondaryList) - } - extras := map[string]sqlparser.Encodable{ - "#pk": &sqlparser.TupleEqualityList{ - Columns: qre.plan.Table.Indexes[0].Columns, - Rows: pkRows, - }, - } - r, err := qre.txFetch(conn, query, qre.bindVars, extras, bsc, true, true) - if err != nil { - return nil, err - } - - // UPDATEs can return InsertID when LAST_INSERT_ID(expr) is used. In - // this case it should be the same for all rows. - result.InsertID = r.InsertID - - // DMLs should all return RowsAffected. - result.RowsAffected += r.RowsAffected + warnThreshold := qre.tsv.qe.warnResultSize.Get() + if warnThreshold > 0 && count > warnThreshold { + callerID := callerid.ImmediateCallerIDFromContext(qre.ctx) + tabletenv.Warnings.Add("ResultsExceeded", 1) + log.Warningf("CallerID: %s row count %v exceeds warning threshold %v: %q", callerID.Username, count, warnThreshold, queryAsString(qre.plan.FullQuery.Query, qre.bindVars)) } - return result, nil + return nil } func (qre *QueryExecutor) execSet() (*sqltypes.Result, error) { @@ -820,8 +739,6 @@ func (qre *QueryExecutor) streamFetch(conn *connpool.DBConn, parsedQuery *sqlpar } func (qre *QueryExecutor) generateFinalSQL(parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, extras map[string]sqlparser.Encodable, buildStreamComment string) (string, string, error) { - bindVars["#maxLimit"] = sqltypes.Int64BindVariable(qre.getLimit(parsedQuery)) - var buf strings.Builder buf.WriteString(qre.marginComments.Leading) diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 782cda8a8c6..3053b65b3b6 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -725,139 +725,6 @@ func TestQueryExecutorPlanUpsertPkAutoCommit(t *testing.T) { } } -func TestQueryExecutorPlanDmlPk(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanDMLPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{"update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanDmlPkTransactionIsolation(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - db.AddQuery("set transaction isolation level SERIALIZABLE", &sqltypes.Result{}) - txid := newTransaction(tsv, &querypb.ExecuteOptions{ - TransactionIsolation: querypb.ExecuteOptions_SERIALIZABLE, - }) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanDMLPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{"update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanDmlPkRBR(t *testing.T) { - // RBR test is almost identical to the non-RBR test, except that - // the _stream comments are suppressed for RBR. - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set name = 2 where pk in (1)" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanDMLPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanDmlAutoCommit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanDMLPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanDmlAutoCommitTransactionIsolation(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - - qre.options = &querypb.ExecuteOptions{ - TransactionIsolation: querypb.ExecuteOptions_READ_UNCOMMITTED, - } - db.AddQuery("set transaction isolation level READ UNCOMMITTED", &sqltypes.Result{}) - - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanDMLPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - func TestQueryExecutorPlanDmlSubQuery(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index b63b73b519d..00eb4a8bced 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1188,7 +1188,7 @@ func (tsv *TabletServer) computeTxSerializerKey(ctx context.Context, logStats *t return "", "" } - if plan.PlanID != planbuilder.PlanDMLPK && plan.PlanID != planbuilder.PlanDMLSubquery { + if plan.PlanID != planbuilder.PlanPassDML && plan.PlanID != planbuilder.PlanDMLLimit { // Serialize only UPDATE or DELETE queries. return "", "" } From 4ee0b11ce287ce4eeba2e46e90b3a7550a01e1eb Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 14 Mar 2020 20:34:55 -0700 Subject: [PATCH 06/18] deprecation: delete lot of SBR code Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/codex.go | 169 ------ go/vt/vttablet/tabletserver/codex_test.go | 264 --------- .../vttablet/tabletserver/planbuilder/dml.go | 362 ++---------- .../vttablet/tabletserver/planbuilder/plan.go | 35 +- .../tabletserver/planbuilder/plan_test.go | 46 +- go/vt/vttablet/tabletserver/query_engine.go | 10 + go/vt/vttablet/tabletserver/query_executor.go | 207 ++----- .../tabletserver/query_executor_test.go | 541 ------------------ go/vt/vttablet/tabletserver/queryz.go | 10 +- go/vt/vttablet/tabletserver/queryz_test.go | 4 - .../vttablet/tabletserver/rules/rules_test.go | 10 +- 11 files changed, 154 insertions(+), 1504 deletions(-) delete mode 100644 go/vt/vttablet/tabletserver/codex.go delete mode 100644 go/vt/vttablet/tabletserver/codex_test.go diff --git a/go/vt/vttablet/tabletserver/codex.go b/go/vt/vttablet/tabletserver/codex.go deleted file mode 100644 index de3b6bf7dfd..00000000000 --- a/go/vt/vttablet/tabletserver/codex.go +++ /dev/null @@ -1,169 +0,0 @@ -/* -Copyright 2019 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package tabletserver - -import ( - "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" - - querypb "vitess.io/vitess/go/vt/proto/query" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" -) - -// buildValueList builds the set of PK reference rows used to drive the next query. -// It uses the PK values supplied in the original query and bind variables. -// The generated reference rows are validated for type match against the PK of the table. -func buildValueList(table *schema.Table, pkValues []sqltypes.PlanValue, bindVars map[string]*querypb.BindVariable) ([][]sqltypes.Value, error) { - rows, err := sqltypes.ResolveRows(pkValues, bindVars) - if err != nil { - return nil, err - } - // Iterate by columns. - for j := range pkValues { - typ := table.GetPKColumn(j).Type - for i := range rows { - rows[i][j], err = sqltypes.Cast(rows[i][j], typ) - if err != nil { - return nil, err - } - } - } - return rows, nil -} - -// buildSecondaryList is used for handling ON DUPLICATE DMLs, or those that change the PK. -func buildSecondaryList(table *schema.Table, pkList [][]sqltypes.Value, secondaryList []sqltypes.PlanValue, bindVars map[string]*querypb.BindVariable) ([][]sqltypes.Value, error) { - if secondaryList == nil { - return nil, nil - } - secondaryRows, err := sqltypes.ResolveRows(secondaryList, bindVars) - if err != nil { - return nil, err - } - rows := make([][]sqltypes.Value, len(pkList)) - for i, row := range pkList { - // If secondaryRows has only one row, then that - // row should be duplicated for every row in pkList. - // Otherwise, we use the individual values. - var changedValues []sqltypes.Value - if len(secondaryRows) == 1 { - changedValues = secondaryRows[0] - } else { - changedValues = secondaryRows[i] - } - rows[i] = make([]sqltypes.Value, len(row)) - for j, value := range row { - if changedValues[j].IsNull() { - rows[i][j] = value - } else { - rows[i][j] = changedValues[j] - } - } - } - return rows, nil -} - -// resolveNumber extracts a number from a bind variable or sql value. -func resolveNumber(pv sqltypes.PlanValue, bindVars map[string]*querypb.BindVariable) (int64, error) { - v, err := pv.ResolveValue(bindVars) - if err != nil { - return 0, err - } - ret, err := sqltypes.ToInt64(v) - if err != nil { - return 0, err - } - return ret, nil -} - -func validateRow(table *schema.Table, columnNumbers []int, row []sqltypes.Value) error { - if len(row) != len(columnNumbers) { - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "data inconsistency %d vs %d", len(row), len(columnNumbers)) - } - for j, value := range row { - if err := validateValue(&table.Columns[columnNumbers[j]], value); err != nil { - return err - } - } - - return nil -} - -func validateValue(col *schema.TableColumn, value sqltypes.Value) error { - if value.IsNull() { - return nil - } - if sqltypes.IsIntegral(col.Type) { - if !value.IsIntegral() { - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "type mismatch, expecting numeric type for %v for column: %v", value, col) - } - } else if col.Type == sqltypes.VarBinary { - if !value.IsQuoted() { - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "type mismatch, expecting string type for %v for column: %v", value, col) - } - } - return nil -} - -func buildStreamComment(table *schema.Table, pkValueList [][]sqltypes.Value, secondaryList [][]sqltypes.Value) string { - buf := sqlparser.NewTrackedBuffer(nil) - buf.Myprintf(" /* _stream %v (", table.Name) - // We assume the first index exists, and is the pk - for _, pkName := range table.Indexes[0].Columns { - buf.Myprintf("%v ", pkName) - } - buf.WriteString(")") - buildPKValueList(buf, table, pkValueList) - buildPKValueList(buf, table, secondaryList) - buf.WriteString("; */") - return buf.String() -} - -func buildPKValueList(buf *sqlparser.TrackedBuffer, table *schema.Table, pkValueList [][]sqltypes.Value) { - for _, pkValues := range pkValueList { - buf.WriteString(" (") - for _, pkValue := range pkValues { - pkValue.EncodeASCII(buf) - buf.WriteString(" ") - } - buf.WriteString(")") - } -} - -func applyFilterWithPKDefaults(table *schema.Table, columnNumbers []int, input []sqltypes.Value) (output []sqltypes.Value) { - output = make([]sqltypes.Value, len(columnNumbers)) - for colIndex, colPointer := range columnNumbers { - if colPointer >= 0 { - output[colIndex] = input[colPointer] - } else { - output[colIndex] = table.GetPKColumn(colIndex).Default - } - } - return output -} - -// unicoded returns a valid UTF-8 string that json won't reject -func unicoded(in string) (out string) { - for i, v := range in { - if v == 0xFFFD { - return in[:i] - } - } - return in -} diff --git a/go/vt/vttablet/tabletserver/codex_test.go b/go/vt/vttablet/tabletserver/codex_test.go deleted file mode 100644 index f8ee20f5c3f..00000000000 --- a/go/vt/vttablet/tabletserver/codex_test.go +++ /dev/null @@ -1,264 +0,0 @@ -/* -Copyright 2019 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package tabletserver - -import ( - "reflect" - "testing" - - "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" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" -) - -func TestCodexBuildValuesList(t *testing.T) { - table := createTable("Table", - []string{"pk1", "pk2", "col1"}, - []querypb.Type{sqltypes.Int64, sqltypes.VarBinary, sqltypes.Int32}, - []string{"pk1", "pk2"}) - bindVars := map[string]*querypb.BindVariable{ - "key": sqltypes.Int64BindVariable(10), - } - tcases := []struct { - pkValues []sqltypes.PlanValue - out [][]sqltypes.Value - err string - }{{ - pkValues: []sqltypes.PlanValue{{ - Key: "key", - }, { - Value: sqltypes.NewVarBinary("aa"), - }}, - out: [][]sqltypes.Value{ - {sqltypes.NewInt64(10), sqltypes.NewVarBinary("aa")}, - }, - }, { - pkValues: []sqltypes.PlanValue{{ - Key: "nokey", - }}, - err: "missing bind var nokey", - }, { - pkValues: []sqltypes.PlanValue{{ - Value: sqltypes.NewVarChar("aa"), - }}, - err: `strconv.ParseInt: parsing "aa": invalid syntax`, - }} - for _, tc := range tcases { - got, err := buildValueList(table, tc.pkValues, bindVars) - if tc.err != "" { - if err == nil || err.Error() != tc.err { - t.Errorf("buildValueList(%v) error: %v, want %s", tc.pkValues, err, tc.err) - } - continue - } - if err != nil { - t.Errorf("buildValueList(%v) error: %v", tc.pkValues, err) - } - if !reflect.DeepEqual(got, tc.out) { - t.Errorf("buildValueList(%v): %v, want %s", tc.pkValues, got, tc.out) - } - } -} - -func TestBuildSecondaryList(t *testing.T) { - table := createTable("Table", - []string{"pk1", "pk2", "col1"}, - []querypb.Type{sqltypes.Int64, sqltypes.VarBinary, sqltypes.Int32}, - []string{"pk1", "pk2"}) - bindVars := map[string]*querypb.BindVariable{ - "key": sqltypes.Int64BindVariable(10), - } - r := sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "pk1|pk2", - "int64|varchar", - ), - "1|aa", - "2|bb", - ) - pkList := r.Rows - tcases := []struct { - secondaryList []sqltypes.PlanValue - out [][]sqltypes.Value - err string - }{{ - secondaryList: nil, - out: nil, - }, { - secondaryList: []sqltypes.PlanValue{{}, { - Value: sqltypes.NewVarBinary("cc"), - }}, - out: [][]sqltypes.Value{ - {sqltypes.NewInt64(1), sqltypes.NewVarBinary("cc")}, - {sqltypes.NewInt64(2), sqltypes.NewVarBinary("cc")}, - }, - }, { - secondaryList: []sqltypes.PlanValue{{ - Key: "nokey", - }}, - err: "missing bind var nokey", - }} - for _, tc := range tcases { - got, err := buildSecondaryList(table, pkList, tc.secondaryList, bindVars) - if tc.err != "" { - if err == nil || err.Error() != tc.err { - t.Errorf("buildSecondaryList(%v) error: %v, want %s", tc.secondaryList, err, tc.err) - } - continue - } - if err != nil { - t.Errorf("buildSecondaryList(%v) error: %v", tc.secondaryList, err) - } - if !reflect.DeepEqual(got, tc.out) { - t.Errorf("buildSecondaryList(%v): %v, want %s", tc.secondaryList, got, tc.out) - } - } -} - -func TestResolveNumber(t *testing.T) { - bindVars := map[string]*querypb.BindVariable{ - "key": sqltypes.Int64BindVariable(10), - } - tcases := []struct { - pv sqltypes.PlanValue - out int64 - err string - }{{ - pv: sqltypes.PlanValue{Key: "key"}, - out: 10, - }, { - pv: sqltypes.PlanValue{Key: "nokey"}, - err: "missing bind var nokey", - }, { - pv: sqltypes.PlanValue{Value: sqltypes.NewVarChar("aa")}, - err: "could not parse value: 'aa'", - }} - for _, tc := range tcases { - got, err := resolveNumber(tc.pv, bindVars) - if tc.err != "" { - if err == nil || err.Error() != tc.err { - t.Errorf("resolveNumber(%v) error: %v, want %s", tc.pv, err, tc.err) - } - continue - } - if err != nil { - t.Errorf("resolveNumber(%v) error: %v", tc.pv, err) - } - if got != tc.out { - t.Errorf("resolveNumber(%v): %d, want %d", tc.pv, got, tc.out) - } - } -} - -func TestCodexBuildStreamComment(t *testing.T) { - pk1 := "pk1" - pk2 := "pk2" - table := createTable("Table", - []string{"pk1", "pk2", "col1"}, - []querypb.Type{sqltypes.Int64, sqltypes.VarBinary, sqltypes.Int32}, - []string{pk1, pk2}) - - // set pk2 = 'xyz' where pk1=1 and pk2 = 'abc' - bindVars := map[string]*querypb.BindVariable{} - pk1Val := sqltypes.NewInt64(1) - pk2Val := sqltypes.NewVarChar("abc") - pkValues := []sqltypes.PlanValue{{Value: pk1Val}, {Value: pk2Val}} - pkList, _ := buildValueList(table, pkValues, bindVars) - pk2SecVal := sqltypes.NewVarChar("xyz") - secondaryPKValues := []sqltypes.PlanValue{{}, {Value: pk2SecVal}} - secondaryList, _ := buildSecondaryList(table, pkList, secondaryPKValues, bindVars) - want := " /* _stream `Table` (pk1 pk2 ) (1 'YWJj' ) (1 'eHl6' ); */" - got := buildStreamComment(table, pkList, secondaryList) - if !reflect.DeepEqual(got, want) { - t.Fatalf("case 1 failed, got\n%s, want\n%s", got, want) - } -} - -func TestCodexValidateRow(t *testing.T) { - table := createTable("Table", - []string{"pk1", "pk2", "col1"}, - []querypb.Type{sqltypes.Int64, sqltypes.VarBinary, sqltypes.Int32}, - []string{"pk1", "pk2"}) - // #columns and #rows do not match - err := validateRow(table, []int{1}, []sqltypes.Value{}) - if code := vterrors.Code(err); code != vtrpcpb.Code_INVALID_ARGUMENT { - t.Errorf("validateRow: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) - } - // column 0 is int type but row is in string type - err = validateRow(table, []int{0}, []sqltypes.Value{sqltypes.NewVarBinary("str")}) - if code := vterrors.Code(err); code != vtrpcpb.Code_INVALID_ARGUMENT { - t.Errorf("validateRow: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) - } -} - -func TestCodexApplyFilterWithPKDefaults(t *testing.T) { - testUtils := newTestUtils() - table := createTable("Table", - []string{"pk1", "pk2", "col1"}, - []querypb.Type{sqltypes.Int64, sqltypes.VarBinary, sqltypes.Int32}, - []string{"pk1", "pk2"}) - output := applyFilterWithPKDefaults(table, []int{-1}, []sqltypes.Value{}) - if len(output) != 1 { - t.Fatalf("expect to only one output but got: %v", output) - } - val, err := sqltypes.ToInt64(output[0]) - if err != nil { - t.Fatalf("should not get an error, but got err: %v", err) - } - testUtils.checkEqual(t, int64(0), val) -} - -func TestCodexUnicoded(t *testing.T) { - testUtils := newTestUtils() - in := "test" - out := unicoded(in) - testUtils.checkEqual(t, in, out) - in = "tes\xFFFDt" - out = unicoded(in) - testUtils.checkEqual(t, "tes", out) -} - -func createTable(name string, colNames []string, colTypes []querypb.Type, pKeys []string) *schema.Table { - table := schema.NewTable(name) - for i, colName := range colNames { - colType := colTypes[i] - defaultVal := sqltypes.Value{} - if sqltypes.IsIntegral(colType) { - defaultVal = sqltypes.NewInt64(0) - } else if colType == sqltypes.VarBinary { - defaultVal = sqltypes.NewVarBinary("") - } - table.AddColumn(colName, colType, defaultVal, "") - } - setPK(table, pKeys) - return table -} - -func setPK(ta *schema.Table, colnames []string) error { - if len(ta.Indexes) != 0 { - panic("setPK must be called before adding other indexes") - } - pkIndex := ta.AddIndex("PRIMARY", true) - for _, colname := range colnames { - pkIndex.AddColumn(colname, 1) - } - ta.Done() - return nil -} diff --git a/go/vt/vttablet/tabletserver/planbuilder/dml.go b/go/vt/vttablet/tabletserver/planbuilder/dml.go index 205f980548e..155cc76e3bd 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/dml.go +++ b/go/vt/vttablet/tabletserver/planbuilder/dml.go @@ -17,8 +17,6 @@ limitations under the License. package planbuilder import ( - "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" @@ -26,6 +24,42 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) +func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan *Plan, err error) { + plan = &Plan{ + PlanID: PlanPassSelect, + Table: lookupTable(sel.From, tables), + FieldQuery: GenerateFieldQuery(sel), + FullQuery: GenerateLimitQuery(sel), + } + if sel.Lock != "" { + plan.PlanID = PlanSelectLock + } + + if sel.Where != nil { + comp, ok := sel.Where.Expr.(*sqlparser.ComparisonExpr) + if ok && comp.IsImpossible() { + plan.PlanID = PlanSelectImpossible + return plan, nil + } + } + + // Check if it's a NEXT VALUE statement. + if nextVal, ok := sel.SelectExprs[0].(sqlparser.Nextval); ok { + if plan.Table == nil || plan.Table.Type != schema.Sequence { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s is not a sequence", sqlparser.String(sel.From)) + } + plan.PlanID = PlanNextval + v, err := sqlparser.NewPlanValue(nextVal.Expr) + if err != nil { + return nil, err + } + plan.NextCount = v + plan.FieldQuery = nil + plan.FullQuery = nil + } + return plan, nil +} + func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ PlanID: PlanPassDML, @@ -71,64 +105,13 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan return plan, nil } -func analyzeSet(set *sqlparser.Set) (plan *Plan) { - return &Plan{ - PlanID: PlanSet, - FullQuery: GenerateFullQuery(set), - } -} - -func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan *Plan, err error) { - plan = &Plan{ - PlanID: PlanPassSelect, - Table: lookupTable(sel.From, tables), - FieldQuery: GenerateFieldQuery(sel), - FullQuery: GenerateLimitQuery(sel), - } - if sel.Lock != "" { - plan.PlanID = PlanSelectLock - } - - if sel.Where != nil { - comp, ok := sel.Where.Expr.(*sqlparser.ComparisonExpr) - if ok && comp.IsImpossible() { - plan.PlanID = PlanSelectImpossible - return plan, nil - } - } - - // Check if it's a NEXT VALUE statement. - if nextVal, ok := sel.SelectExprs[0].(sqlparser.Nextval); ok { - if plan.Table == nil || plan.Table.Type != schema.Sequence { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s is not a sequence", sqlparser.String(sel.From)) - } - plan.PlanID = PlanNextval - v, err := sqlparser.NewPlanValue(nextVal.Expr) - if err != nil { - return nil, err - } - plan.PKValues = []sqltypes.PlanValue{v} - plan.FieldQuery = nil - plan.FullQuery = nil - } - return plan, nil -} - func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ PlanID: PlanPassDML, FullQuery: GenerateFullQuery(ins), } - if ins.Action == sqlparser.ReplaceStr { - plan.Reason = ReasonReplace - return plan, nil - } tableName := sqlparser.GetTableName(ins.Table) - if tableName.IsEmpty() { - plan.Reason = ReasonTable - return plan, nil - } plan.Table = tables[tableName.String()] if plan.Table == nil { return plan, nil @@ -136,153 +119,11 @@ func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan switch { case plan.Table.Type == schema.Message: - // message inserts need to continue being strict, even in passthrough dml mode, - // because field defaults are set here - + return analyzeInsertMessage(ins, plan, plan.Table) case plan.Table.IsTopic(): plan.PlanID = PlanInsertTopic - plan.Reason = ReasonTopic - return plan, nil - - case PassthroughDMLs: - // In passthrough dml mode, allow the operation even if the - // table is unknown in the schema. - return plan, nil - } - - if !plan.Table.HasPrimary() { - log.Warningf("no primary key for table %s", tableName) - plan.Reason = ReasonTableNoIndex - return plan, nil - } - switch plan.Table.Type { - case schema.NoType, schema.Sequence: - // For now, allow sequence inserts. - return analyzeInsertNoType(ins, plan, plan.Table) - case schema.Message: - return analyzeInsertMessage(ins, plan, plan.Table) - } - panic("unreachable") -} - -func analyzeInsertNoType(ins *sqlparser.Insert, plan *Plan, table *schema.Table) (*Plan, error) { - // Populate column list from schema if it wasn't specified. - if len(ins.Columns) == 0 { - for _, col := range table.Columns { - ins.Columns = append(ins.Columns, col.Name) - } - } - pkColumnNumbers := getInsertPKColumns(ins.Columns, table) - - if sel, ok := ins.Rows.(sqlparser.SelectStatement); ok { - if ins.OnDup != nil { - // Upserts not allowed for subqueries. - // http://bugs.mysql.com/bug.php?id=58637 - plan.Reason = ReasonUpsertSubquery - return plan, nil - } - plan.PlanID = PlanInsertSubquery - plan.OuterQuery = GenerateInsertOuterQuery(ins) - plan.Subquery = GenerateLimitQuery(sel) - for _, col := range ins.Columns { - colIndex := table.FindColumn(col) - if colIndex == -1 { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "column %v not found in table %s", col, table.Name) - } - plan.ColumnNumbers = append(plan.ColumnNumbers, colIndex) - } - plan.SubqueryPKColumns = pkColumnNumbers - return plan, nil - } - - // If it's not a sqlparser.SelectStatement, it's Values. - rowList := ins.Rows.(sqlparser.Values) - for i := range rowList { - if len(rowList[i]) == 0 { - for _, col := range table.Columns { - expr, err := sqlparser.ExprFromValue(col.Default) - if err != nil { - return nil, vterrors.Wrap(err, "could not create default row for insert without row values") - } - rowList[i] = append(rowList[i], expr) - } - continue - } - if len(rowList[i]) != len(ins.Columns) { - return nil, vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "column count doesn't match value count") - } - } - plan.PKValues = getInsertPKValues(pkColumnNumbers, rowList, table) - if plan.PKValues == nil { - plan.Reason = ReasonComplexExpr - return plan, nil - } - - if ins.OnDup == nil { - plan.PlanID = PlanInsertPK - plan.OuterQuery = sqlparser.NewParsedQuery(ins) - return plan, nil - } - - // Compute secondary pk values if OnDup changes them. - var ok bool - plan.SecondaryPKValues, ok = analyzeOnDupExpressions(ins, table.Indexes[0]) - if !ok { - plan.Reason = ReasonPKChange - return plan, nil - } - - // If the table only has one unique key then it is safe to pass through - // a simple upsert unmodified even if there are multiple rows in the - // statement. The action is same as a regular insert except that we - // may have to publish possible PK changes by OnDup, which would be - // recorded in SecondaryPKValues. - if table.UniqueIndexes() <= 1 { - plan.PlanID = PlanInsertPK - plan.OuterQuery = sqlparser.NewParsedQuery(ins) return plan, nil } - - // Otherwise multiple rows are unsupported - if len(rowList) > 1 { - plan.Reason = ReasonUpsertMultiRow - return plan, nil - } - plan.PlanID = PlanUpsertPK - newins := *ins - newins.Ignore = "" - newins.OnDup = nil - plan.OuterQuery = sqlparser.NewParsedQuery(&newins) - tableAlias := &sqlparser.AliasedTableExpr{Expr: ins.Table} - upd := &sqlparser.Update{ - Comments: ins.Comments, - TableExprs: sqlparser.TableExprs{tableAlias}, - Exprs: sqlparser.UpdateExprs(ins.OnDup), - } - - // We need to replace 'values' expressions with the actual values they reference. - var formatErr error - plan.UpsertQuery = GenerateUpdateOuterQuery(upd, tableAlias, func(buf *sqlparser.TrackedBuffer, node sqlparser.SQLNode) { - if node, ok := node.(*sqlparser.ValuesFuncExpr); ok { - if !node.Name.Qualifier.IsEmpty() && node.Name.Qualifier != ins.Table { - formatErr = vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, - "could not find qualified column %v in table %v", - sqlparser.String(node.Name), sqlparser.String(ins.Table)) - return - } - colnum := ins.Columns.FindColumn(node.Name.Name) - if colnum == -1 { - formatErr = vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, "could not find column %v", node.Name) - return - } - buf.Myprintf("(%v)", rowList[0][colnum]) - return - } - node.Format(buf) - }) - if formatErr != nil { - return nil, formatErr - } return plan, nil } @@ -346,103 +187,16 @@ func analyzeInsertMessage(ins *sqlparser.Insert, plan *Plan, table *schema.Table return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s must be specified for message insert", col.String()) } - pkColumnNumbers := getInsertPKColumns(ins.Columns, table) - plan.PKValues = getInsertPKValues(pkColumnNumbers, rowList, table) - if plan.PKValues == nil { - // Dead code. The previous checks already catch this condition. - plan.Reason = ReasonComplexExpr - return plan, nil - } plan.PlanID = PlanInsertMessage - plan.OuterQuery = sqlparser.NewParsedQuery(ins) + plan.FullQuery = GenerateFullQuery(ins) return plan, nil } -func getInsertPKColumns(columns sqlparser.Columns, table *schema.Table) (pkColumnNumbers []int) { - pkIndex := table.Indexes[0] - pkColumnNumbers = make([]int, len(pkIndex.Columns)) - for i := range pkColumnNumbers { - pkColumnNumbers[i] = -1 - } - for i, column := range columns { - index := pkIndex.FindColumn(column) - if index == -1 { - continue - } - pkColumnNumbers[index] = i - } - return pkColumnNumbers -} - -func addVal(ins *sqlparser.Insert, col sqlparser.ColIdent, expr sqlparser.Expr) int { - ins.Columns = append(ins.Columns, col) - rows := ins.Rows.(sqlparser.Values) - for i := range rows { - rows[i] = append(rows[i], expr) - } - return len(ins.Columns) - 1 -} - -func copyVal(ins *sqlparser.Insert, col sqlparser.ColIdent, colIndex int) int { - ins.Columns = append(ins.Columns, col) - rows := ins.Rows.(sqlparser.Values) - for i := range rows { - rows[i] = append(rows[i], rows[i][colIndex]) - } - return len(ins.Columns) - 1 -} - -func getInsertPKValues(pkColumnNumbers []int, rowList sqlparser.Values, table *schema.Table) []sqltypes.PlanValue { - pkValues := make([]sqltypes.PlanValue, len(pkColumnNumbers)) - // We iterate by columns (j, i). - for j, columnNumber := range pkColumnNumbers { - if columnNumber == -1 { - // No value was specified. Use the default from the schema for all rows. - pkValues[j] = sqltypes.PlanValue{Value: table.GetPKColumn(j).Default} - continue - } - var ok bool - pkValues[j], ok = extractColumnValues(rowList, columnNumber) - if !ok { - return nil - } - } - return pkValues -} - -// analyzeOnDupExpressions analyzes the OnDup and returns the list for any pk value changes. -func analyzeOnDupExpressions(ins *sqlparser.Insert, pkIndex *schema.Index) (pkValues []sqltypes.PlanValue, ok bool) { - rowList := ins.Rows.(sqlparser.Values) - for _, expr := range ins.OnDup { - index := pkIndex.FindColumn(expr.Name.Name) - if index == -1 { - continue - } - - if pkValues == nil { - pkValues = make([]sqltypes.PlanValue, len(pkIndex.Columns)) - } - if vf, ok := expr.Expr.(*sqlparser.ValuesFuncExpr); ok { - if !vf.Name.Qualifier.IsEmpty() && vf.Name.Qualifier != ins.Table { - return nil, false - } - insertCol := ins.Columns.FindColumn(vf.Name.Name) - if insertCol == -1 { - return nil, false - } - pkValues[index], ok = extractColumnValues(rowList, insertCol) - if !ok { - return nil, false - } - continue - } - - pkValues[index], ok = extractSingleValue(expr.Expr) - if !ok { - return nil, false - } +func analyzeSet(set *sqlparser.Set) (plan *Plan) { + return &Plan{ + PlanID: PlanSet, + FullQuery: GenerateFullQuery(set), } - return pkValues, true } func lookupTable(tableExprs sqlparser.TableExprs, tables map[string]*schema.Table) *schema.Table { @@ -460,28 +214,20 @@ func lookupTable(tableExprs sqlparser.TableExprs, tables map[string]*schema.Tabl return tables[tableName.String()] } -// extractColumnValues extracts the values of a column into a PlanValue. -func extractColumnValues(rowList sqlparser.Values, colnum int) (sqltypes.PlanValue, bool) { - pv := sqltypes.PlanValue{Values: make([]sqltypes.PlanValue, len(rowList))} - for i := 0; i < len(rowList); i++ { - var ok bool - pv.Values[i], ok = extractSingleValue(rowList[i][colnum]) - if !ok { - return pv, false - } +func addVal(ins *sqlparser.Insert, col sqlparser.ColIdent, expr sqlparser.Expr) int { + ins.Columns = append(ins.Columns, col) + rows := ins.Rows.(sqlparser.Values) + for i := range rows { + rows[i] = append(rows[i], expr) } - return pv, true + return len(ins.Columns) - 1 } -func extractSingleValue(expr sqlparser.Expr) (sqltypes.PlanValue, bool) { - pv := sqltypes.PlanValue{} - if !sqlparser.IsNull(expr) && !sqlparser.IsValue(expr) { - return pv, false - } - var err error - pv, err = sqlparser.NewPlanValue(expr) - if err != nil { - return pv, false +func copyVal(ins *sqlparser.Insert, col sqlparser.ColIdent, colIndex int) int { + ins.Columns = append(ins.Columns, col) + rows := ins.Rows.(sqlparser.Values) + for i := range rows { + rows[i] = append(rows[i], rows[i][colIndex]) } - return pv, true + return len(ins.Columns) - 1 } diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index a682b7f9aa2..f2ab88c2c4a 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -59,13 +59,6 @@ const ( PlanPassDML // PlanDMLLimit is an update or delete with a limit. PlanDMLLimit - // PlanInsertPK is insert statement where the PK value is - // supplied with the query. - PlanInsertPK - // PlanInsertSubquery is same as PlanDMLSubquery but for inserts. - PlanInsertSubquery - // PlanUpsertPK is for insert ... on duplicate key constructs. - PlanUpsertPK // PlanInsertTopic is for inserting into message topics. PlanInsertTopic // PlanInsertMessage is for inserting into message tables. @@ -95,9 +88,6 @@ var planName = [NumPlans]string{ "NEXTVAL", "PASS_DML", "DML_LIMIT", - "INSERT_PK", - "INSERT_SUBQUERY", - "UPSERT_PK", "INSERT_TOPIC", "INSERT_MESSAGE", "SET", @@ -185,7 +175,6 @@ func (rt ReasonType) MarshalJSON() ([]byte, error) { // Plan is built for selects and DMLs. type Plan struct { PlanID PlanType - Reason ReasonType Table *schema.Table // NewName is the new name of the table. Set for DDLs which create or change the table. NewName sqlparser.TableIdent @@ -199,32 +188,12 @@ type Plan struct { // FullQuery will be set for all plans. FullQuery *sqlparser.ParsedQuery - // For PK plans, only OuterQuery is set. - // For SUBQUERY plans, Subquery is also set. - OuterQuery *sqlparser.ParsedQuery - Subquery *sqlparser.ParsedQuery - UpsertQuery *sqlparser.ParsedQuery - - // PlanInsertSubquery: columns to be inserted. - ColumnNumbers []int - - // PKValues is an sqltypes.Value if it's sourced - // from the query. If it's a bind var then it's - // a string including the ':' prefix(es). - // PlanDMLPK: where clause values. - // PlanInsertPK: values clause. - // PlanNextVal: increment. - PKValues []sqltypes.PlanValue - - // For update: set clause if pk is changing. - SecondaryPKValues []sqltypes.PlanValue + // NextCount stores the count for "select next". + NextCount sqltypes.PlanValue // WhereClause is set for DMLs. It is used by the hot row protection // to serialize e.g. UPDATEs going to the same row. WhereClause *sqlparser.ParsedQuery - - // For PlanInsertSubquery: pk columns in the subquery result. - SubqueryPKColumns []int } // TableName returns the table name for the plan. diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan_test.go b/go/vt/vttablet/tabletserver/planbuilder/plan_test.go index 020fffc9d0d..1026e6c99e8 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan_test.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan_test.go @@ -30,7 +30,6 @@ import ( "testing" "github.com/stretchr/testify/require" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/tableacl" "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" @@ -40,35 +39,24 @@ import ( // This is only for testing. func (p *Plan) MarshalJSON() ([]byte, error) { mplan := struct { - PlanID PlanType - Reason ReasonType `json:",omitempty"` - TableName sqlparser.TableIdent `json:",omitempty"` - Permissions []Permission `json:",omitempty"` - FieldQuery *sqlparser.ParsedQuery `json:",omitempty"` - FullQuery *sqlparser.ParsedQuery `json:",omitempty"` - OuterQuery *sqlparser.ParsedQuery `json:",omitempty"` - Subquery *sqlparser.ParsedQuery `json:",omitempty"` - UpsertQuery *sqlparser.ParsedQuery `json:",omitempty"` - ColumnNumbers []int `json:",omitempty"` - PKValues []sqltypes.PlanValue `json:",omitempty"` - SecondaryPKValues []sqltypes.PlanValue `json:",omitempty"` - WhereClause *sqlparser.ParsedQuery `json:",omitempty"` - SubqueryPKColumns []int `json:",omitempty"` + PlanID PlanType + TableName sqlparser.TableIdent `json:",omitempty"` + Permissions []Permission `json:",omitempty"` + FieldQuery *sqlparser.ParsedQuery `json:",omitempty"` + FullQuery *sqlparser.ParsedQuery `json:",omitempty"` + NextCount string `json:",omitempty"` + WhereClause *sqlparser.ParsedQuery `json:",omitempty"` }{ - PlanID: p.PlanID, - Reason: p.Reason, - TableName: p.TableName(), - Permissions: p.Permissions, - FieldQuery: p.FieldQuery, - FullQuery: p.FullQuery, - OuterQuery: p.OuterQuery, - Subquery: p.Subquery, - UpsertQuery: p.UpsertQuery, - ColumnNumbers: p.ColumnNumbers, - PKValues: p.PKValues, - SecondaryPKValues: p.SecondaryPKValues, - WhereClause: p.WhereClause, - SubqueryPKColumns: p.SubqueryPKColumns, + PlanID: p.PlanID, + TableName: p.TableName(), + Permissions: p.Permissions, + FieldQuery: p.FieldQuery, + FullQuery: p.FullQuery, + WhereClause: p.WhereClause, + } + if !p.NextCount.IsNull() { + b, _ := p.NextCount.MarshalJSON() + mplan.NextCount = string(b) } return json.Marshal(&mplan) } diff --git a/go/vt/vttablet/tabletserver/query_engine.go b/go/vt/vttablet/tabletserver/query_engine.go index 16d82967940..fe468b67d07 100644 --- a/go/vt/vttablet/tabletserver/query_engine.go +++ b/go/vt/vttablet/tabletserver/query_engine.go @@ -619,3 +619,13 @@ func (qe *QueryEngine) handleHTTPConsolidations(response http.ResponseWriter, re response.Write([]byte(fmt.Sprintf("%v: %s\n", v.Count, query))) } } + +// unicoded returns a valid UTF-8 string that json won't reject +func unicoded(in string) (out string) { + for i, v := range in { + if v == 0xFFFD { + return in[:i] + } + } + return in +} diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index c6491bfde5b..949d533619b 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -39,7 +39,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" querypb "vitess.io/vitess/go/vt/proto/query" - "vitess.io/vitess/go/vt/proto/topodata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) @@ -54,7 +54,7 @@ type QueryExecutor struct { ctx context.Context logStats *tabletenv.LogStats tsv *TabletServer - tabletType topodata.TabletType + tabletType topodatapb.TabletType } var sequenceFields = []*querypb.Field{ @@ -117,32 +117,26 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { } defer conn.Recycle() switch qre.plan.PlanID { - case planbuilder.PlanPassDML: - if !qre.tsv.qe.allowUnsafeDMLs && (qre.tsv.qe.binlogFormat != connpool.BinlogFormatRow) { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: cannot identify primary key of statement") - } - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) - case planbuilder.PlanInsertPK: - return qre.execInsertPK(conn) + case planbuilder.PlanPassDML, planbuilder.PlanSet: + return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) case planbuilder.PlanInsertMessage: return qre.execInsertMessage(conn) - case planbuilder.PlanInsertSubquery: - return qre.execInsertSubquery(conn) case planbuilder.PlanDMLLimit: return qre.execDMLLimit(conn) case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: - return qre.execSQL(conn, qre.query, true) - case planbuilder.PlanUpsertPK: - return qre.execUpsertPK(conn) - case planbuilder.PlanSet: - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) + return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, false) case planbuilder.PlanPassSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, false) + maxrows := qre.getSelectLimit() + qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) + qr, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, false) + if err != nil { + return nil, err + } + if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { + return nil, err + } + return qr, nil default: - // handled above: - // planbuilder.PlanNextval - // planbuilder.PlanDDL - // not valid for Execute: // planbuilder.PlanSelectStream // planbuilder.PlanMessageStream: @@ -151,39 +145,26 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { } else { switch qre.plan.PlanID { case planbuilder.PlanPassSelect, planbuilder.PlanSelectImpossible: - return qre.execSelect() + maxrows := qre.getSelectLimit() + qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) + qr, err := qre.execSelect() + if err != nil { + return nil, err + } + if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { + return nil, err + } + return qr, nil case planbuilder.PlanSelectLock: return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) - case planbuilder.PlanSet: - return qre.execSet() - case planbuilder.PlanOtherRead: - conn, connErr := qre.getConn() - if connErr != nil { - return nil, connErr - } - defer conn.Recycle() - return qre.execSQL(conn, qre.query, true) - - case planbuilder.PlanPassDML: - fallthrough - case planbuilder.PlanInsertPK: - fallthrough - case planbuilder.PlanInsertMessage: - fallthrough - case planbuilder.PlanInsertSubquery: - fallthrough - case planbuilder.PlanDMLLimit: - fallthrough - case planbuilder.PlanUpsertPK: + case planbuilder.PlanSet, planbuilder.PlanOtherRead: + return qre.execOther() + case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage, planbuilder.PlanDMLLimit: if !qre.tsv.qe.autoCommit.Get() { return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) } return qre.execDmlAutoCommit() default: - // handled above: - // planbuilder.PlanNextval - // planbuilder.PlanDDL - // not valid for Execute: // planbuilder.PlanSelectStream // planbuilder.PlanMessageStream: @@ -228,7 +209,7 @@ func (qre *QueryExecutor) Stream(callback func(*sqltypes.Result) error) error { qre.tsv.qe.streamQList.Add(qd) defer qre.tsv.qe.streamQList.Remove(qd) - return qre.streamFetch(conn, qre.plan.FullQuery, qre.bindVars, "", callback) + return qre.streamFetch(conn, qre.plan.FullQuery, qre.bindVars, callback) } // MessageStream streams messages from a message table. @@ -264,20 +245,11 @@ func (qre *QueryExecutor) execDmlAutoCommit() (reply *sqltypes.Result, err error return qre.execAsTransaction(func(conn *TxConnection) (reply *sqltypes.Result, err error) { switch qre.plan.PlanID { case planbuilder.PlanPassDML: - if !qre.tsv.qe.allowUnsafeDMLs && (qre.tsv.qe.binlogFormat != connpool.BinlogFormatRow) { - return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, "unsupported: cannot identify primary key of statement") - } - reply, err = qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) - case planbuilder.PlanInsertPK: - reply, err = qre.execInsertPK(conn) + reply, err = qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) case planbuilder.PlanInsertMessage: return qre.execInsertMessage(conn) - case planbuilder.PlanInsertSubquery: - reply, err = qre.execInsertSubquery(conn) case planbuilder.PlanDMLLimit: reply, err = qre.execDMLLimit(conn) - case planbuilder.PlanUpsertPK: - reply, err = qre.execUpsertPK(conn) default: return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unsupported query: %s", qre.query) } @@ -398,7 +370,7 @@ func (qre *QueryExecutor) execDDL() (*sqltypes.Result, error) { sql := qre.query var err error if qre.plan.FullQuery != nil { - sql, _, err = qre.generateFinalSQL(qre.plan.FullQuery, qre.bindVars, nil, "") + sql, _, err = qre.generateFinalSQL(qre.plan.FullQuery, qre.bindVars) if err != nil { return nil, err } @@ -444,7 +416,7 @@ func (qre *QueryExecutor) execDDL() (*sqltypes.Result, error) { } func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) { - inc, err := resolveNumber(qre.plan.PKValues[0], qre.bindVars) + inc, err := resolveNumber(qre.plan.NextCount, qre.bindVars) if err != nil { return nil, err } @@ -534,70 +506,18 @@ func (qre *QueryExecutor) execSelect() (*sqltypes.Result, error) { return nil, err } defer conn.Recycle() - return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars, "", true) -} - -func (qre *QueryExecutor) execInsertPK(conn *TxConnection) (*sqltypes.Result, error) { - pkRows, err := buildValueList(qre.plan.Table, qre.plan.PKValues, qre.bindVars) - if err != nil { - return nil, err - } - return qre.execInsertPKRows(conn, nil, pkRows) + return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars) } func (qre *QueryExecutor) execInsertMessage(conn *TxConnection) (*sqltypes.Result, error) { qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano()) - return qre.execInsertPK(conn) -} - -func (qre *QueryExecutor) execInsertSubquery(conn *TxConnection) (*sqltypes.Result, error) { - innerResult, err := qre.txFetch(conn, qre.plan.Subquery, qre.bindVars, nil, "", true, false) - if err != nil { - return nil, err - } - innerRows := innerResult.Rows - if len(innerRows) == 0 { - return &sqltypes.Result{RowsAffected: 0}, nil - } - if len(qre.plan.ColumnNumbers) != len(innerRows[0]) { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "Subquery length does not match column list") - } - pkRows := make([][]sqltypes.Value, len(innerRows)) - for i, innerRow := range innerRows { - pkRows[i] = applyFilterWithPKDefaults(qre.plan.Table, qre.plan.SubqueryPKColumns, innerRow) - } - // Validating first row is sufficient - if err := validateRow(qre.plan.Table, qre.plan.Table.PKColumns, pkRows[0]); err != nil { - return nil, err - } - - extras := map[string]sqlparser.Encodable{ - "#values": sqlparser.InsertValues(innerRows), - } - return qre.execInsertPKRows(conn, extras, pkRows) -} - -func (qre *QueryExecutor) execInsertPKRows(conn *TxConnection, extras map[string]sqlparser.Encodable, pkRows [][]sqltypes.Value) (*sqltypes.Result, error) { - var bsc string - // Build comments only if we're not in RBR mode. - if qre.tsv.qe.binlogFormat != connpool.BinlogFormatRow { - secondaryList, err := buildSecondaryList(qre.plan.Table, pkRows, qre.plan.SecondaryPKValues, qre.bindVars) - if err != nil { - return nil, err - } - bsc = buildStreamComment(qre.plan.Table, pkRows, secondaryList) - } - return qre.txFetch(conn, qre.plan.OuterQuery, qre.bindVars, extras, bsc, true, true) -} - -func (qre *QueryExecutor) execUpsertPK(conn *TxConnection) (*sqltypes.Result, error) { - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) + return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) } func (qre *QueryExecutor) execDMLLimit(conn *TxConnection) (*sqltypes.Result, error) { maxrows := qre.tsv.qe.maxResultSize.Get() qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) - result, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, nil, "", true, true) + result, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) if err != nil { return nil, err } @@ -616,18 +536,18 @@ func (qre *QueryExecutor) verifyRowCount(count, maxrows int64) error { if warnThreshold > 0 && count > warnThreshold { callerID := callerid.ImmediateCallerIDFromContext(qre.ctx) tabletenv.Warnings.Add("ResultsExceeded", 1) - log.Warningf("CallerID: %s row count %v exceeds warning threshold %v: %q", callerID.Username, count, warnThreshold, queryAsString(qre.plan.FullQuery.Query, qre.bindVars)) + log.Warningf("caller id: %s row count %v exceeds warning threshold %v: %q", callerID.Username, count, warnThreshold, queryAsString(qre.plan.FullQuery.Query, qre.bindVars)) } return nil } -func (qre *QueryExecutor) execSet() (*sqltypes.Result, error) { +func (qre *QueryExecutor) execOther() (*sqltypes.Result, error) { conn, err := qre.getConn() if err != nil { return nil, err } defer conn.Recycle() - return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars, "", false) + return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars) } func (qre *QueryExecutor) getConn() (*connpool.DBConn, error) { @@ -663,12 +583,12 @@ func (qre *QueryExecutor) getStreamConn() (*connpool.DBConn, error) { } func (qre *QueryExecutor) qFetch(logStats *tabletenv.LogStats, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - sql, sqlWithoutComments, err := qre.generateFinalSQL(parsedQuery, bindVars, nil, "") + sql, sqlWithoutComments, err := qre.generateFinalSQL(parsedQuery, bindVars) if err != nil { return nil, err } // Check tablet type. - if qre.tsv.qe.enableConsolidator || (qre.tsv.qe.enableConsolidatorReplicas && qre.tabletType != topodata.TabletType_MASTER) { + if qre.tsv.qe.enableConsolidator || (qre.tsv.qe.enableConsolidatorReplicas && qre.tabletType != topodatapb.TabletType_MASTER) { q, original := qre.tsv.qe.consolidator.Create(string(sqlWithoutComments)) if original { defer q.Broadcast() @@ -704,12 +624,12 @@ func (qre *QueryExecutor) qFetch(logStats *tabletenv.LogStats, parsedQuery *sqlp } // txFetch fetches from a TxConnection. -func (qre *QueryExecutor) txFetch(conn *TxConnection, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, extras map[string]sqlparser.Encodable, buildStreamComment string, wantfields, record bool) (*sqltypes.Result, error) { - sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars, extras, buildStreamComment) +func (qre *QueryExecutor) txFetch(conn *TxConnection, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, record bool) (*sqltypes.Result, error) { + sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars) if err != nil { return nil, err } - qr, err := qre.execSQL(conn, sql, wantfields) + qr, err := qre.execSQL(conn, sql, true) if err != nil { return nil, err } @@ -721,48 +641,45 @@ func (qre *QueryExecutor) txFetch(conn *TxConnection, parsedQuery *sqlparser.Par } // dbConnFetch fetches from a connpool.DBConn. -func (qre *QueryExecutor) dbConnFetch(conn *connpool.DBConn, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, buildStreamComment string, wantfields bool) (*sqltypes.Result, error) { - sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars, nil, buildStreamComment) +func (qre *QueryExecutor) dbConnFetch(conn *connpool.DBConn, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars) if err != nil { return nil, err } - return qre.execSQL(conn, sql, wantfields) + return qre.execSQL(conn, sql, true) } // streamFetch performs a streaming fetch. -func (qre *QueryExecutor) streamFetch(conn *connpool.DBConn, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, buildStreamComment string, callback func(*sqltypes.Result) error) error { - sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars, nil, buildStreamComment) +func (qre *QueryExecutor) streamFetch(conn *connpool.DBConn, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, callback func(*sqltypes.Result) error) error { + sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars) if err != nil { return err } return qre.execStreamSQL(conn, sql, callback) } -func (qre *QueryExecutor) generateFinalSQL(parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, extras map[string]sqlparser.Encodable, buildStreamComment string) (string, string, error) { +func (qre *QueryExecutor) generateFinalSQL(parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable) (string, string, error) { var buf strings.Builder buf.WriteString(qre.marginComments.Leading) - query, err := parsedQuery.GenerateQuery(bindVars, extras) + query, err := parsedQuery.GenerateQuery(bindVars, nil) if err != nil { return "", "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%s", err) } buf.WriteString(query) - if buildStreamComment != "" { - buf.WriteString(buildStreamComment) - } withoutComments := buf.String() buf.WriteString(qre.marginComments.Trailing) fullSQL := buf.String() return fullSQL, withoutComments, nil } -func (qre *QueryExecutor) getLimit(query *sqlparser.ParsedQuery) int64 { +func (qre *QueryExecutor) getSelectLimit() int64 { maxRows := qre.tsv.qe.maxResultSize.Get() sqlLimit := qre.options.GetSqlSelectLimit() - if sqlLimit > 0 && sqlLimit < maxRows && strings.HasPrefix(sqlparser.StripLeadingComments(query.Query), "select") { + if sqlLimit > 0 && sqlLimit < maxRows { return sqlLimit } - return maxRows + 1 + return maxRows } // poolConn is an abstraction for reusing code in execSQL. @@ -775,14 +692,7 @@ func (qre *QueryExecutor) execSQL(conn poolConn, sql string, wantfields bool) (* defer span.Finish() defer qre.logStats.AddRewrittenSQL(sql, time.Now()) - res, err := conn.Exec(ctx, sql, int(qre.tsv.qe.maxResultSize.Get()), wantfields) - warnThreshold := qre.tsv.qe.warnResultSize.Get() - if res != nil && warnThreshold > 0 && int64(len(res.Rows)) > warnThreshold { - callerID := callerid.ImmediateCallerIDFromContext(qre.ctx) - tabletenv.Warnings.Add("ResultsExceeded", 1) - log.Warningf("CallerID: %s Results returned (%v) exceeds warning threshold (%v): %q", callerID.Username, len(res.Rows), warnThreshold, sql) - } - return res, err + return conn.Exec(ctx, sql, int(qre.tsv.qe.maxResultSize.Get()), wantfields) } func (qre *QueryExecutor) execStreamSQL(conn *connpool.DBConn, sql string, callback func(*sqltypes.Result) error) error { @@ -802,3 +712,12 @@ func (qre *QueryExecutor) execStreamSQL(conn *connpool.DBConn, sql string, callb } return nil } + +// resolveNumber extracts a number from a bind variable or sql value. +func resolveNumber(pv sqltypes.PlanValue, bindVars map[string]*querypb.BindVariable) (int64, error) { + v, err := pv.ResolveValue(bindVars) + if err != nil { + return 0, err + } + return sqltypes.ToInt64(v) +} diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 3053b65b3b6..ba6ce759293 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -267,26 +267,6 @@ func TestQueryExecutorPlanPassDmlReplaceInto(t *testing.T) { testCommitHelper(t, tsv, qre) } -func TestQueryExecutorPlanInsertPk(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - db.AddQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) - want := &sqltypes.Result{} - query := "insert into test_table(pk) values(1)" - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanInsertPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - func TestQueryExecutorPlanInsertMessage(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() @@ -307,527 +287,6 @@ func TestQueryExecutorPlanInsertMessage(t *testing.T) { } } -func TestQueryExecutorPlanInsertSubQueryAutoCommmit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "insert into test_table(pk) select pk from test_table where pk = 2" - want := &sqltypes.Result{} - db.AddQuery(query, want) - selectQuery := "select pk from test_table where pk = 2 limit 10001" - db.AddQuery(selectQuery, &sqltypes.Result{ - Fields: []*querypb.Field{{ - Name: "pk", - Type: sqltypes.Int32, - }}, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(2)}, - }, - }) - - insertQuery := "insert into test_table(pk) values (2) /* _stream test_table (pk ) (2 ); */" - - db.AddQuery(insertQuery, &sqltypes.Result{}) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanInsertSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanInsertSubQuery(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "insert into test_table(pk) select pk from test_table where pk = 2" - want := &sqltypes.Result{} - db.AddQuery(query, want) - selectQuery := "select pk from test_table where pk = 2 limit 10001" - db.AddQuery(selectQuery, &sqltypes.Result{ - Fields: []*querypb.Field{{ - Name: "pk", - Type: sqltypes.Int32, - }}, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(2)}, - }, - }) - - insertQuery := "insert into test_table(pk) values (2) /* _stream test_table (pk ) (2 ); */" - - db.AddQuery(insertQuery, &sqltypes.Result{}) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{"insert into test_table(pk) values (2) /* _stream test_table (pk ) (2 ); */"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanInsertSubQueryRBR(t *testing.T) { - // RBR test is almost identical to the non-RBR test, except that - // the _stream comments are suppressed for RBR. - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "insert into test_table(pk) select pk from test_table where pk = 2" - want := &sqltypes.Result{} - db.AddQuery(query, want) - selectQuery := "select pk from test_table where pk = 2 limit 10001" - db.AddQuery(selectQuery, &sqltypes.Result{ - Fields: []*querypb.Field{{ - Name: "pk", - Type: sqltypes.Int32, - }}, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(2)}, - }, - }) - - insertQuery := "insert into test_table(pk) values (2)" - - db.AddQuery(insertQuery, &sqltypes.Result{}) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{"insert into test_table(pk) values (2)"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanUpsertPk(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - db.AddQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) - want := &sqltypes.Result{} - query := "insert into test_table(pk) values(1) on duplicate key update val=1" - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanUpsertPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{"insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - db.AddRejectedQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", errRejected) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query, txid) - defer testCommitHelper(t, tsv, qre) - _, err = qre.Execute() - wantErr := "rejected" - if err == nil || !strings.Contains(err.Error(), wantErr) { - t.Errorf("qre.Execute() = %v, want %v", err, wantErr) - } - if gotqueries = fetchRecordedQueries(qre); gotqueries != nil { - t.Errorf("queries: %v, want nil", gotqueries) - } - - db.AddRejectedQuery( - "insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", - mysql.NewSQLError(mysql.ERDupEntry, mysql.SSDupKey, "err"), - ) - db.AddQuery("update test_table(pk) set val = 1 where pk in (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query, txid) - defer testCommitHelper(t, tsv, qre) - _, err = qre.Execute() - wantErr = "err (errno 1062) (sqlstate 23000)" - if err == nil || !strings.Contains(err.Error(), wantErr) { - t.Errorf("qre.Execute() = %v, want %v", err, wantErr) - } - if gotqueries = fetchRecordedQueries(qre); gotqueries != nil { - t.Errorf("queries: %v, want nil", gotqueries) - } - - db.AddRejectedQuery( - "insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", - mysql.NewSQLError(mysql.ERDupEntry, mysql.SSDupKey, "ERROR 1062 (23000): Duplicate entry '2' for key 'PRIMARY'"), - ) - db.AddQuery( - "update test_table set val = 1 where pk in (1) /* _stream test_table (pk ) (1 ); */", - &sqltypes.Result{RowsAffected: 1}, - ) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query, txid) - defer testCommitHelper(t, tsv, qre) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - want = &sqltypes.Result{ - RowsAffected: 2, - } - if !reflect.DeepEqual(got, want) { - t.Errorf("got: %v, want: %v", got, want) - } - wantqueries = []string{"update test_table set val = 1 where pk in (1) /* _stream test_table (pk ) (1 ); */"} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - // Test pk change. - db.AddRejectedQuery( - "insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", - mysql.NewSQLError(mysql.ERDupEntry, mysql.SSDupKey, "ERROR 1062 (23000): Duplicate entry '2' for key 'PRIMARY'"), - ) - db.AddQuery( - "update test_table set pk = 2 where pk in (1) /* _stream test_table (pk ) (1 ) (2 ); */", - &sqltypes.Result{RowsAffected: 1}, - ) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, "insert into test_table(pk) values (1) on duplicate key update pk=2", txid) - defer testCommitHelper(t, tsv, qre) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - want = &sqltypes.Result{ - RowsAffected: 2, - } - if !reflect.DeepEqual(got, want) { - t.Errorf("got: %v, want: %v", got, want) - } - wantqueries = []string{"update test_table set pk = 2 where pk in (1) /* _stream test_table (pk ) (1 ) (2 ); */"} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanUpsertPkSingleUnique(t *testing.T) { - db := setUpQueryExecutorTestWithOneUniqueKey(t) - defer db.Close() - query := "insert into test_table(pk) values (1) on duplicate key update val = 1 /* _stream test_table (pk ) (1 ); */" - db.AddQuery(query, &sqltypes.Result{}) - db.AddRejectedQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", errRejected) - want := &sqltypes.Result{} - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query[0:strings.Index(query, " /*")], txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - // PK changed by upsert. - query = "insert into test_table(pk) values (1), (2), (3) on duplicate key update pk = 5 /* _stream test_table (pk ) (1 ) (2 ) (3 ) (5 ) (5 ) (5 ); */" - db.AddQuery(query, &sqltypes.Result{}) - want = &sqltypes.Result{} - ctx = context.Background() - tsv = newTestTabletServer(ctx, noFlags, db) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query[0:strings.Index(query, " /*")], txid) - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertPK, qre.plan.PlanID) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries = []string{query} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - // PK changed by using values. - query = "insert into test_table(pk, name) values (1, 4), (2, 5), (3, 6) on duplicate key update pk = values(name) /* _stream test_table (pk ) (1 ) (2 ) (3 ) (4 ) (5 ) (6 ); */" - db.AddQuery(query, &sqltypes.Result{}) - want = &sqltypes.Result{} - ctx = context.Background() - tsv = newTestTabletServer(ctx, noFlags, db) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query[0:strings.Index(query, " /*")], txid) - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertPK, qre.plan.PlanID) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries = []string{query} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - query = "insert into test_table(pk) values (1), (2), (3) on duplicate key update val = 5 /* _stream test_table (pk ) (1 ) (2 ) (3 ); */" - db.AddQuery(query, &sqltypes.Result{}) - want = &sqltypes.Result{} - ctx = context.Background() - tsv = newTestTabletServer(ctx, noFlags, db) - txid = newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, query[0:strings.Index(query, " /*")], txid) - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanInsertPK, qre.plan.PlanID) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries = []string{query} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanUpsertPkRBR(t *testing.T) { - // For UPSERT, the query just becomes a pass-through in RBR mode. - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "insert into test_table(pk) values (1) on duplicate key update val = 1" - db.AddQuery(query, &sqltypes.Result{}) - want := &sqltypes.Result{} - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanUpsertPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanUpsertPkAutoCommit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - db.AddQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) - want := &sqltypes.Result{} - query := "insert into test_table(pk) values(1) on duplicate key update val=1" - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanUpsertPK, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - - db.AddRejectedQuery("insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", errRejected) - _, err = qre.Execute() - wantErr := "rejected" - if err == nil || !strings.Contains(err.Error(), wantErr) { - t.Fatalf("qre.Execute() = %v, want %v", err, wantErr) - } - - db.AddRejectedQuery( - "insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", - mysql.NewSQLError(mysql.ERDupEntry, mysql.SSDupKey, "err"), - ) - db.AddQuery("update test_table set val = 1 where pk in (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) - _, err = qre.Execute() - wantErr = "err (errno 1062) (sqlstate 23000)" - if err == nil || !strings.Contains(err.Error(), wantErr) { - t.Fatalf("qre.Execute() = %v, want %v", err, wantErr) - } - - db.AddRejectedQuery( - "insert into test_table(pk) values (1) /* _stream test_table (pk ) (1 ); */", - mysql.NewSQLError(mysql.ERDupEntry, mysql.SSDupKey, "ERROR 1062 (23000): Duplicate entry '2' for key 'PRIMARY'"), - ) - db.AddQuery( - "update test_table set val = 1 where pk in (1) /* _stream test_table (pk ) (1 ); */", - &sqltypes.Result{RowsAffected: 1}, - ) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - want = &sqltypes.Result{ - RowsAffected: 2, - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanDmlSubQuery(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set addr = 3 where name = 1" - expandedQuery := "select pk from test_table where name = 1 limit 10001 for update" - want := &sqltypes.Result{} - db.AddQuery(query, want) - db.AddQuery(expandedQuery, &sqltypes.Result{ - Fields: []*querypb.Field{ - {Type: sqltypes.Int32}, - }, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(2)}, - }, - }) - updateQuery := "update test_table set addr = 3 where pk in (2) /* _stream test_table (pk ) (2 ); */" - db.AddQuery(updateQuery, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanDMLSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{updateQuery} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanDmlSubQueryRBR(t *testing.T) { - // RBR test is almost identical to the non-RBR test, except that - // the _stream comments are suppressed for RBR. - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set addr = 3 where name = 1" - expandedQuery := "select pk from test_table where name = 1 limit 10001 for update" - want := &sqltypes.Result{} - db.AddQuery(query, want) - db.AddQuery(expandedQuery, &sqltypes.Result{ - Fields: []*querypb.Field{ - {Type: sqltypes.Int32}, - }, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(2)}, - }, - }) - updateQuery := "update test_table set addr = 3 where pk in (2)" - db.AddQuery(updateQuery, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanDMLSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{updateQuery} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } -} - -func TestQueryExecutorPlanDmlSubQueryAutoCommit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set addr = 3 where name = 1" - expandedQuery := "select pk from test_table where name = 1 limit 10001 for update" - want := &sqltypes.Result{} - db.AddQuery(query, want) - db.AddQuery(expandedQuery, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanDMLSubquery, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - func TestQueryExecutorPlanOtherWithinATransaction(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() diff --git a/go/vt/vttablet/tabletserver/queryz.go b/go/vt/vttablet/tabletserver/queryz.go index dea2b1dfd0a..ef37e77c5cf 100644 --- a/go/vt/vttablet/tabletserver/queryz.go +++ b/go/vt/vttablet/tabletserver/queryz.go @@ -36,7 +36,6 @@ var ( Query Table Plan - Reason Count Time MySQL Time @@ -54,7 +53,6 @@ var ( {{.Query}} {{.Table}} {{.Plan}} - {{.Reason}} {{.Count}} {{.Time}} {{.MysqlTime}} @@ -74,7 +72,6 @@ type queryzRow struct { Query string Table string Plan planbuilder.PlanType - Reason planbuilder.ReasonType Count int64 tm time.Duration mysqlTime time.Duration @@ -150,10 +147,9 @@ func queryzHandler(qe *QueryEngine, w http.ResponseWriter, r *http.Request) { continue } Value := &queryzRow{ - Query: logz.Wrappable(sqlparser.TruncateForUI(v)), - Table: plan.TableName().String(), - Plan: plan.PlanID, - Reason: plan.Reason, + Query: logz.Wrappable(sqlparser.TruncateForUI(v)), + Table: plan.TableName().String(), + Plan: plan.PlanID, } Value.Count, Value.tm, Value.mysqlTime, Value.Rows, Value.Errors = plan.Stats() var timepq time.Duration diff --git a/go/vt/vttablet/tabletserver/queryz_test.go b/go/vt/vttablet/tabletserver/queryz_test.go index f639ed7f061..78f5256a199 100644 --- a/go/vt/vttablet/tabletserver/queryz_test.go +++ b/go/vt/vttablet/tabletserver/queryz_test.go @@ -41,7 +41,6 @@ func TestQueryzHandler(t *testing.T) { Plan: &planbuilder.Plan{ Table: &schema.Table{Name: sqlparser.NewTableIdent("test_table")}, PlanID: planbuilder.PlanPassSelect, - Reason: planbuilder.ReasonTable, }, } plan1.AddStats(10, 2*time.Second, 1*time.Second, 2, 0) @@ -51,7 +50,6 @@ func TestQueryzHandler(t *testing.T) { Plan: &planbuilder.Plan{ Table: &schema.Table{Name: sqlparser.NewTableIdent("test_table")}, PlanID: planbuilder.PlanDDL, - Reason: planbuilder.ReasonDefault, }, } plan2.AddStats(1, 2*time.Millisecond, 1*time.Millisecond, 1, 0) @@ -61,7 +59,6 @@ func TestQueryzHandler(t *testing.T) { Plan: &planbuilder.Plan{ Table: &schema.Table{Name: sqlparser.NewTableIdent("")}, PlanID: planbuilder.PlanOtherRead, - Reason: planbuilder.ReasonDefault, }, } plan3.AddStats(1, 75*time.Millisecond, 50*time.Millisecond, 1, 0) @@ -72,7 +69,6 @@ func TestQueryzHandler(t *testing.T) { Plan: &planbuilder.Plan{ Table: &schema.Table{Name: sqlparser.NewTableIdent("")}, PlanID: planbuilder.PlanOtherRead, - Reason: planbuilder.ReasonDefault, }, } plan4.AddStats(1, 1*time.Millisecond, 1*time.Millisecond, 1, 0) diff --git a/go/vt/vttablet/tabletserver/rules/rules_test.go b/go/vt/vttablet/tabletserver/rules/rules_test.go index 31eb8f2cf1c..309d93fb9fa 100644 --- a/go/vt/vttablet/tabletserver/rules/rules_test.go +++ b/go/vt/vttablet/tabletserver/rules/rules_test.go @@ -185,7 +185,7 @@ func TestFilterByPlan(t *testing.T) { t.Errorf("qrs1:\n%s, want\n%s", got, want) } - qrs1 = qrs.FilterByPlan("select", planbuilder.PlanInsertPK, "a") + qrs1 = qrs.FilterByPlan("select", planbuilder.PlanPassDML, "a") want = compacted(`[{ "Description":"rule 3", "Name":"r3", @@ -201,7 +201,7 @@ func TestFilterByPlan(t *testing.T) { t.Errorf("qrs1:\n%s, want\n%s", got, want) } - qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanInsertPK, "a") + qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanPassDML, "a") if qrs1.rules != nil { t.Errorf("want nil, got non-nil") } @@ -220,7 +220,7 @@ func TestFilterByPlan(t *testing.T) { qr5 := NewQueryRule("rule 5", "r5", QRFail) qrs.Add(qr5) - qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanInsertPK, "a") + qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanPassDML, "a") want = compacted(`[{ "Description":"rule 5", "Name":"r5", @@ -258,12 +258,12 @@ func TestQueryRule(t *testing.T) { } qr.AddPlanCond(planbuilder.PlanPassSelect) - qr.AddPlanCond(planbuilder.PlanInsertPK) + qr.AddPlanCond(planbuilder.PlanPassDML) if qr.plans[0] != planbuilder.PlanPassSelect { t.Errorf("want PASS_SELECT, got %s", qr.plans[0].String()) } - if qr.plans[1] != planbuilder.PlanInsertPK { + if qr.plans[1] != planbuilder.PlanPassDML { t.Errorf("want INSERT_PK, got %s", qr.plans[1].String()) } From 11e4978fc19b88dd275684071dfbf039f9f2e600 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 15 Mar 2020 16:22:05 -0700 Subject: [PATCH 07/18] deprecation: refactor code for readability Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/query_executor.go | 188 ++++++++---------- 1 file changed, 82 insertions(+), 106 deletions(-) diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 949d533619b..264efe34ace 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -116,61 +116,89 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { return nil, err } defer conn.Recycle() - switch qre.plan.PlanID { - case planbuilder.PlanPassDML, planbuilder.PlanSet: - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) - case planbuilder.PlanInsertMessage: - return qre.execInsertMessage(conn) - case planbuilder.PlanDMLLimit: - return qre.execDMLLimit(conn) - case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, false) - case planbuilder.PlanPassSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: - maxrows := qre.getSelectLimit() - qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) - qr, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, false) - if err != nil { - return nil, err - } - if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { - return nil, err - } - return qr, nil - default: - // not valid for Execute: - // planbuilder.PlanSelectStream - // planbuilder.PlanMessageStream: - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) + return qre.txConnExec(conn) + } + + switch qre.plan.PlanID { + case planbuilder.PlanPassSelect, planbuilder.PlanSelectImpossible: + maxrows := qre.getSelectLimit() + qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) + qr, err := qre.execSelect() + if err != nil { + return nil, err } - } else { - switch qre.plan.PlanID { - case planbuilder.PlanPassSelect, planbuilder.PlanSelectImpossible: - maxrows := qre.getSelectLimit() - qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) - qr, err := qre.execSelect() - if err != nil { - return nil, err - } - if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { - return nil, err - } - return qr, nil - case planbuilder.PlanSelectLock: + if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { + return nil, err + } + return qr, nil + case planbuilder.PlanSelectLock: + return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) + case planbuilder.PlanSet, planbuilder.PlanOtherRead: + return qre.execOther() + case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage, planbuilder.PlanDMLLimit: + if !qre.tsv.qe.autoCommit.Get() { return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) - case planbuilder.PlanSet, planbuilder.PlanOtherRead: - return qre.execOther() - case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage, planbuilder.PlanDMLLimit: - if !qre.tsv.qe.autoCommit.Get() { - return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) - } - return qre.execDmlAutoCommit() - default: - // not valid for Execute: - // planbuilder.PlanSelectStream - // planbuilder.PlanMessageStream: - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) } + return qre.execAsTransaction(qre.txConnExec) + } + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) +} + +func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { + conn, beginSQL, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) + if err != nil { + return nil, err + } + defer qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) + if beginSQL != "" { + qre.logStats.AddRewrittenSQL(beginSQL, time.Now()) + } + + reply, err = f(conn) + + start := time.Now() + if err != nil { + qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) + qre.logStats.AddRewrittenSQL("rollback", start) + return nil, err + } + commitSQL, err := qre.tsv.te.txPool.LocalCommit(qre.ctx, conn) + + // As above LocalCommit is a no-op for autocommmit so don't log anything. + if commitSQL != "" { + qre.logStats.AddRewrittenSQL(commitSQL, start) + } + + if err != nil { + return nil, err + } + return reply, nil +} + +func (qre *QueryExecutor) txConnExec(conn *TxConnection) (*sqltypes.Result, error) { + switch qre.plan.PlanID { + case planbuilder.PlanPassDML, planbuilder.PlanSet: + return qre.txFetch(conn, true) + case planbuilder.PlanInsertMessage: + qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano()) + return qre.txFetch(conn, true) + case planbuilder.PlanDMLLimit: + return qre.execDMLLimit(conn) + case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: + return qre.txFetch(conn, false) + case planbuilder.PlanPassSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: + maxrows := qre.getSelectLimit() + qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) + qr, err := qre.txFetch(conn, false) + if err != nil { + return nil, err + } + if err := qre.verifyRowCount(int64(len(qr.Rows)), maxrows); err != nil { + return nil, err + } + return qr, nil } + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) } // Stream performs a streaming query execution. @@ -241,53 +269,6 @@ func (qre *QueryExecutor) MessageStream(callback func(*sqltypes.Result) error) e return nil } -func (qre *QueryExecutor) execDmlAutoCommit() (reply *sqltypes.Result, err error) { - return qre.execAsTransaction(func(conn *TxConnection) (reply *sqltypes.Result, err error) { - switch qre.plan.PlanID { - case planbuilder.PlanPassDML: - reply, err = qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) - case planbuilder.PlanInsertMessage: - return qre.execInsertMessage(conn) - case planbuilder.PlanDMLLimit: - reply, err = qre.execDMLLimit(conn) - default: - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unsupported query: %s", qre.query) - } - return reply, err - }) -} - -func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { - conn, beginSQL, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) - if err != nil { - return nil, err - } - defer qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) - if beginSQL != "" { - qre.logStats.AddRewrittenSQL(beginSQL, time.Now()) - } - - reply, err = f(conn) - - start := time.Now() - if err != nil { - qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) - qre.logStats.AddRewrittenSQL("rollback", start) - return nil, err - } - commitSQL, err := qre.tsv.te.txPool.LocalCommit(qre.ctx, conn) - - // As above LocalCommit is a no-op for autocommmit so don't log anything. - if commitSQL != "" { - qre.logStats.AddRewrittenSQL(commitSQL, start) - } - - if err != nil { - return nil, err - } - return reply, nil -} - // checkPermissions returns an error if the query does not pass all checks // (query blacklisting, table ACL). func (qre *QueryExecutor) checkPermissions() error { @@ -509,15 +490,10 @@ func (qre *QueryExecutor) execSelect() (*sqltypes.Result, error) { return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars) } -func (qre *QueryExecutor) execInsertMessage(conn *TxConnection) (*sqltypes.Result, error) { - qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano()) - return qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) -} - func (qre *QueryExecutor) execDMLLimit(conn *TxConnection) (*sqltypes.Result, error) { maxrows := qre.tsv.qe.maxResultSize.Get() qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) - result, err := qre.txFetch(conn, qre.plan.FullQuery, qre.bindVars, true) + result, err := qre.txFetch(conn, true) if err != nil { return nil, err } @@ -624,8 +600,8 @@ func (qre *QueryExecutor) qFetch(logStats *tabletenv.LogStats, parsedQuery *sqlp } // txFetch fetches from a TxConnection. -func (qre *QueryExecutor) txFetch(conn *TxConnection, parsedQuery *sqlparser.ParsedQuery, bindVars map[string]*querypb.BindVariable, record bool) (*sqltypes.Result, error) { - sql, _, err := qre.generateFinalSQL(parsedQuery, bindVars) +func (qre *QueryExecutor) txFetch(conn *TxConnection, record bool) (*sqltypes.Result, error) { + sql, _, err := qre.generateFinalSQL(qre.plan.FullQuery, qre.bindVars) if err != nil { return nil, err } From f5a5dc54afc07b9a1f18f0d095fee2f2c4744061 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 15 Mar 2020 16:31:45 -0700 Subject: [PATCH 08/18] deprecation: support direct autocommit Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/endtoend/framework/server.go | 1 - go/vt/vttablet/tabletserver/query_engine.go | 2 -- go/vt/vttablet/tabletserver/query_executor.go | 21 ++++++++++++------- .../vttablet/tabletserver/tabletenv/config.go | 5 ++--- go/vt/vttablet/tabletserver/tabletserver.go | 7 ------- .../tabletserver/tabletserver_flaky_test.go | 5 ----- 6 files changed, 15 insertions(+), 26 deletions(-) diff --git a/go/vt/vttablet/endtoend/framework/server.go b/go/vt/vttablet/endtoend/framework/server.go index 9d235ea946b..f51d26eae03 100644 --- a/go/vt/vttablet/endtoend/framework/server.go +++ b/go/vt/vttablet/endtoend/framework/server.go @@ -65,7 +65,6 @@ func StartServer(connParams, connAppDebugParams mysql.ConnParams, dbName string) dbcfgs := dbconfigs.NewTestDBConfigs(connParams, connAppDebugParams, dbName) config := tabletenv.DefaultQsConfig - config.EnableAutoCommit = true config.StrictTableACL = true config.TwoPCEnable = true config.TwoPCAbandonAge = 1 diff --git a/go/vt/vttablet/tabletserver/query_engine.go b/go/vt/vttablet/tabletserver/query_engine.go index fe468b67d07..7329621f32f 100644 --- a/go/vt/vttablet/tabletserver/query_engine.go +++ b/go/vt/vttablet/tabletserver/query_engine.go @@ -150,7 +150,6 @@ type QueryEngine struct { queryPoolWaiters sync2.AtomicInt64 queryPoolWaiterCap sync2.AtomicInt64 binlogFormat connpool.BinlogFormat - autoCommit sync2.AtomicBool maxResultSize sync2.AtomicInt64 warnResultSize sync2.AtomicInt64 maxDMLRows sync2.AtomicInt64 @@ -217,7 +216,6 @@ func NewQueryEngine(checker connpool.MySQLChecker, se *schema.Engine, config tab config.HotRowProtectionConcurrentTransactions) qe.streamQList = NewQueryList() - qe.autoCommit.Set(config.EnableAutoCommit) qe.strictTableACL = config.StrictTableACL qe.enableTableACLDryRun = config.EnableTableACLDryRun diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 264efe34ace..69ba5062f55 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -135,16 +135,21 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) case planbuilder.PlanSet, planbuilder.PlanOtherRead: return qre.execOther() - case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage, planbuilder.PlanDMLLimit: - if !qre.tsv.qe.autoCommit.Get() { - return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) - } - return qre.execAsTransaction(qre.txConnExec) + case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage: + return qre.execAsTransaction(true /* autocommit */, qre.txConnExec) + case planbuilder.PlanDMLLimit: + return qre.execAsTransaction(false /* autocommit */, qre.txConnExec) } return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) } -func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { +func (qre *QueryExecutor) execAsTransaction(autocommit bool, f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { + if autocommit { + if qre.options == nil { + qre.options = &querypb.ExecuteOptions{} + } + qre.options.TransactionIsolation = querypb.ExecuteOptions_AUTOCOMMIT + } conn, beginSQL, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) if err != nil { return nil, err @@ -385,7 +390,7 @@ func (qre *QueryExecutor) execDDL() (*sqltypes.Result, error) { return result, nil } - result, err := qre.execAsTransaction(func(conn *TxConnection) (*sqltypes.Result, error) { + result, err := qre.execAsTransaction(true /* autocommit */, func(conn *TxConnection) (*sqltypes.Result, error) { return qre.execSQL(conn, sql, true) }) @@ -410,7 +415,7 @@ func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) { t.SequenceInfo.Lock() defer t.SequenceInfo.Unlock() if t.SequenceInfo.NextVal == 0 || t.SequenceInfo.NextVal+inc > t.SequenceInfo.LastVal { - _, err := qre.execAsTransaction(func(conn *TxConnection) (*sqltypes.Result, error) { + _, err := qre.execAsTransaction(false /* autocommit */, func(conn *TxConnection) (*sqltypes.Result, error) { query := fmt.Sprintf("select next_id, cache from %s where id = 0 for update", sqlparser.String(tableName)) qr, err := qre.execSQL(conn, query, false) if err != nil { diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go index 454ae05c29f..1034c7a8ccd 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config.go @@ -45,6 +45,7 @@ var ( // Placeholder for deprecated variable. // TODO(sougou): deprecate the flag after release 7.0. deprecatedMessagePoolPrefillParallelism int + deprecatedAutocommit bool ) func init() { @@ -82,7 +83,7 @@ func init() { flag.BoolVar(&Config.TerseErrors, "queryserver-config-terse-errors", DefaultQsConfig.TerseErrors, "prevent bind vars from escaping in returned errors") flag.StringVar(&Config.PoolNamePrefix, "pool-name-prefix", DefaultQsConfig.PoolNamePrefix, "pool name prefix, vttablet has several pools and each of them has a name. This config specifies the prefix of these pool names") flag.BoolVar(&Config.WatchReplication, "watch_replication_stream", false, "When enabled, vttablet will stream the MySQL replication stream from the local server, and use it to support the include_event_token ExecuteOptions.") - flag.BoolVar(&Config.EnableAutoCommit, "enable-autocommit", DefaultQsConfig.EnableAutoCommit, "if the flag is on, a DML outsides a transaction will be auto committed. This flag is deprecated and is unsafe. Instead, use the VTGate provided autocommit feature.") + flag.BoolVar(&deprecatedAutocommit, "enable-autocommit", true, "This flag is deprecated. Autocommit is always allowed.") flag.BoolVar(&Config.TwoPCEnable, "twopc_enable", DefaultQsConfig.TwoPCEnable, "if the flag is on, 2pc is enabled. Other 2pc flags must be supplied.") flag.StringVar(&Config.TwoPCCoordinatorAddress, "twopc_coordinator_address", DefaultQsConfig.TwoPCCoordinatorAddress, "address of the (VTGate) process(es) that will be used to notify of abandoned transactions.") flag.Float64Var(&Config.TwoPCAbandonAge, "twopc_abandon_age", DefaultQsConfig.TwoPCAbandonAge, "time in seconds. Any unresolved transaction older than this time will be sent to the coordinator to be resolved.") @@ -160,7 +161,6 @@ type TabletConfig struct { TxPoolWaiterCap int StrictTableACL bool TerseErrors bool - EnableAutoCommit bool EnableTableACLDryRun bool PoolNamePrefix string TableACLExemptACL string @@ -237,7 +237,6 @@ var DefaultQsConfig = TabletConfig{ StreamBufferSize: 32 * 1024, StrictTableACL: false, TerseErrors: false, - EnableAutoCommit: false, EnableTableACLDryRun: false, PoolNamePrefix: "", TableACLExemptACL: "", diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 00eb4a8bced..27f5db98624 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1077,7 +1077,6 @@ func (tsv *TabletServer) ExecuteBatch(ctx context.Context, target *querypb.Targe // Setting ExecuteOptions_AUTOCOMMIT will get a connection out of the // pool without actually begin/commit the transaction. if (options.TransactionIsolation == querypb.ExecuteOptions_DEFAULT) && - tsv.qe.autoCommit.Get() && asTransaction && tsv.qe.passthroughDMLs.Get() { options.TransactionIsolation = querypb.ExecuteOptions_AUTOCOMMIT @@ -1885,12 +1884,6 @@ func (tsv *TabletServer) QueryPlanCacheCap() int { return int(tsv.qe.QueryPlanCacheCap()) } -// SetAutoCommit sets autocommit on or off. -// This function should only be used for testing. -func (tsv *TabletServer) SetAutoCommit(auto bool) { - tsv.qe.autoCommit.Set(auto) -} - // SetMaxResultSize changes the max result size to the specified value. // This function should only be used for testing. func (tsv *TabletServer) SetMaxResultSize(val int) { diff --git a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go index 90ed7bf7274..7730c747b8a 100644 --- a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go +++ b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go @@ -2572,11 +2572,6 @@ func TestConfigChanges(t *testing.T) { t.Errorf("tsv.qe.QueryPlanCacheCap: %d, want %d", val, newSize) } - tsv.SetAutoCommit(true) - if val := tsv.qe.autoCommit.Get(); !val { - t.Errorf("tsv.qe.autoCommit.Get: %v, want true", val) - } - tsv.SetMaxResultSize(newSize) if val := tsv.MaxResultSize(); val != newSize { t.Errorf("MaxResultSize: %d, want %d", val, newSize) From 0794095eba95943eb10d72d9d9860494c1584178 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 15 Mar 2020 20:14:08 -0700 Subject: [PATCH 09/18] deprecation: tests for planbuilder Signed-off-by: Sugu Sougoumarane --- .../planbuilder/{dml.go => builder.go} | 28 +- .../vttablet/tabletserver/planbuilder/plan.go | 124 +- .../tabletserver/planbuilder/query_gen.go | 69 - .../planbuilder/testdata/exec_cases.txt | 2047 +++-------------- .../planbuilder/testdata/stream_cases.txt | 10 +- go/vt/vttablet/tabletserver/query_executor.go | 12 +- .../tabletserver/query_executor_test.go | 36 +- go/vt/vttablet/tabletserver/querylogz_test.go | 2 +- go/vt/vttablet/tabletserver/queryz_test.go | 2 +- go/vt/vttablet/tabletserver/rules/map_test.go | 6 +- .../vttablet/tabletserver/rules/rules_test.go | 28 +- go/vt/vttablet/tabletserver/tabletserver.go | 4 +- 12 files changed, 405 insertions(+), 1963 deletions(-) rename go/vt/vttablet/tabletserver/planbuilder/{dml.go => builder.go} (93%) diff --git a/go/vt/vttablet/tabletserver/planbuilder/dml.go b/go/vt/vttablet/tabletserver/planbuilder/builder.go similarity index 93% rename from go/vt/vttablet/tabletserver/planbuilder/dml.go rename to go/vt/vttablet/tabletserver/planbuilder/builder.go index 155cc76e3bd..74fb0e4e42d 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/dml.go +++ b/go/vt/vttablet/tabletserver/planbuilder/builder.go @@ -26,7 +26,7 @@ import ( func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassSelect, + PlanID: PlanSelect, Table: lookupTable(sel.From, tables), FieldQuery: GenerateFieldQuery(sel), FullQuery: GenerateLimitQuery(sel), @@ -62,21 +62,23 @@ func analyzeSelect(sel *sqlparser.Select, tables map[string]*schema.Table) (plan func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassDML, + PlanID: PlanUpdate, Table: lookupTable(upd.TableExprs, tables), } // Store the WHERE clause as string for the hot row protection (txserializer). - buf := sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("%v", upd.Where) - plan.WhereClause = buf.ParsedQuery() + if upd.Where != nil { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("%v", upd.Where) + plan.WhereClause = buf.ParsedQuery() + } if PassthroughDMLs || upd.Limit != nil { plan.FullQuery = GenerateFullQuery(upd) return plan, nil } - plan.PlanID = PlanDMLLimit + plan.PlanID = PlanUpdateLimit upd.Limit = execLimit plan.FullQuery = GenerateFullQuery(upd) upd.Limit = nil @@ -85,20 +87,22 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassDML, + PlanID: PlanDelete, Table: lookupTable(del.TableExprs, tables), } // Store the WHERE clause as string for the hot row protection (txserializer). - buf := sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("%v", del.Where) - plan.WhereClause = buf.ParsedQuery() + if del.Where != nil { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("%v", del.Where) + plan.WhereClause = buf.ParsedQuery() + } if PassthroughDMLs || del.Limit != nil { plan.FullQuery = GenerateFullQuery(del) return plan, nil } - plan.PlanID = PlanDMLLimit + plan.PlanID = PlanDeleteLimit del.Limit = execLimit plan.FullQuery = GenerateFullQuery(del) del.Limit = nil @@ -107,7 +111,7 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan *Plan, err error) { plan = &Plan{ - PlanID: PlanPassDML, + PlanID: PlanInsert, FullQuery: GenerateFullQuery(ins), } diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index f2ab88c2c4a..deabf41f49a 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -18,7 +18,6 @@ package planbuilder import ( "encoding/json" - "fmt" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" @@ -30,11 +29,9 @@ import ( ) var ( - // ErrTooComplex indicates given sql query is too complex. - ErrTooComplex = vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "Complex") - execLimit = &sqlparser.Limit{Rowcount: sqlparser.NewValArg([]byte(":#maxLimit"))} + execLimit = &sqlparser.Limit{Rowcount: sqlparser.NewValArg([]byte(":#maxLimit"))} - // PassthroughDMLs will return PlanPassDML for all update or delete statements + // PassthroughDMLs will return plans that pass-through the DMLs without changing them. PassthroughDMLs = false ) @@ -43,60 +40,47 @@ var ( // PlanType indicates a query plan type. type PlanType int +// The following are PlanType values. const ( - // PlanPassSelect is pass through select statements. This is the - // default plan for select statements. - PlanPassSelect PlanType = iota - // PlanSelectLock is for a select that locks. + PlanSelect PlanType = iota PlanSelectLock - // PlanNextval is for NEXTVAL. PlanNextval - // PlanPassDML is pass through update & delete statements. This is - // the default plan for update and delete statements. - // If PassthroughDMLs is true, then it is used for all DML statements - // and is valid in all replication modes. - // Otherwise is only allowed in row based replication mode - PlanPassDML - // PlanDMLLimit is an update or delete with a limit. - PlanDMLLimit - // PlanInsertTopic is for inserting into message topics. + PlanSelectImpossible + PlanInsert PlanInsertTopic - // PlanInsertMessage is for inserting into message tables. PlanInsertMessage - // PlanSet is for SET statements. - PlanSet - // PlanDDL is for DDL statements. + PlanUpdate + PlanUpdateLimit + PlanDelete + PlanDeleteLimit PlanDDL - // PlanSelectStream is used for streaming queries. - PlanSelectStream - // PlanOtherRead is for SHOW, DESCRIBE & EXPLAIN statements. + PlanSet PlanOtherRead - // PlanOtherAdmin is for REPAIR, OPTIMIZE and TRUNCATE statements. PlanOtherAdmin - // PlanMessageStream is used for streaming messages. + PlanSelectStream PlanMessageStream - // PlanSelectImpossible is used for where or having clauses that can never be true. - PlanSelectImpossible - // NumPlans stores the total number of plans NumPlans ) // Must exactly match order of plan constants. var planName = [NumPlans]string{ - "PASS_SELECT", - "SELECT_LOCK", - "NEXTVAL", - "PASS_DML", - "DML_LIMIT", - "INSERT_TOPIC", - "INSERT_MESSAGE", - "SET", + "Select", + "SelectLock", + "Nextval", + "SelectImpossible", + "Insert", + "InsertTopic", + "InsertMessage", + "Update", + "UpdateLimit", + "Delete", + "DeleteLimit", "DDL", - "SELECT_STREAM", - "OTHER_READ", - "OTHER_ADMIN", - "MESSAGE_STREAM", - "SELECT_IMPOSSIBLE", + "Set", + "OtherRead", + "OtherAdmin", + "SelectStream", + "MessageStream", } func (pt PlanType) String() string { @@ -118,7 +102,7 @@ func PlanByName(s string) (pt PlanType, ok bool) { // IsSelect returns true if PlanType is about a select query. func (pt PlanType) IsSelect() bool { - return pt == PlanPassSelect || pt == PlanSelectLock || pt == PlanSelectImpossible + return pt == PlanSelect || pt == PlanSelectLock || pt == PlanSelectImpossible } // MarshalJSON returns a json string for PlanType. @@ -128,56 +112,10 @@ func (pt PlanType) MarshalJSON() ([]byte, error) { //_______________________________________________ -// ReasonType indicates why a query plan fails to build -type ReasonType int - -// Reason codes give a hint about why a certain plan was chosen. -const ( - ReasonDefault ReasonType = iota - ReasonTable - ReasonTableNoIndex - ReasonPKChange - ReasonComplexExpr - ReasonUpsertSubquery - ReasonUpsertMultiRow - ReasonReplace - ReasonMultiTable - ReasonTopic - NumReasons -) - -// Must exactly match order of reason constants. -var reasonName = [NumReasons]string{ - "DEFAULT", - "TABLE", - "TABLE_NOINDEX", - "PK_CHANGE", - "COMPLEX_EXPR", - "UPSERT_SUBQUERY", - "UPSERT_MULTI_ROW", - "REPLACE", - "MULTI_TABLE", - "TOPIC", -} - -// String returns a string representation of a ReasonType. -func (rt ReasonType) String() string { - return reasonName[rt] -} - -// MarshalJSON returns a json string for ReasonType. -func (rt ReasonType) MarshalJSON() ([]byte, error) { - return ([]byte)(fmt.Sprintf("\"%s\"", rt.String())), nil -} - -//_______________________________________________ - -// Plan is built for selects and DMLs. +// Plan contains the parameters for executing a request. type Plan struct { PlanID PlanType Table *schema.Table - // NewName is the new name of the table. Set for DDLs which create or change the table. - NewName sqlparser.TableIdent // Permissions stores the permissions for the tables accessed in the query. Permissions []Permission @@ -217,7 +155,7 @@ func Build(statement sqlparser.Statement, tables map[string]*schema.Table) (*Pla switch stmt := statement.(type) { case *sqlparser.Union: plan, err = &Plan{ - PlanID: PlanPassSelect, + PlanID: PlanSelect, FieldQuery: GenerateFieldQuery(stmt), FullQuery: GenerateLimitQuery(stmt), }, nil diff --git a/go/vt/vttablet/tabletserver/planbuilder/query_gen.go b/go/vt/vttablet/tabletserver/planbuilder/query_gen.go index 6dd37591ebf..8001ace3447 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/query_gen.go +++ b/go/vt/vttablet/tabletserver/planbuilder/query_gen.go @@ -18,7 +18,6 @@ package planbuilder import ( "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" ) // GenerateFullQuery generates the full query from the ast. @@ -65,71 +64,3 @@ func GenerateLimitQuery(selStmt sqlparser.SelectStatement) *sqlparser.ParsedQuer buf.Myprintf("%v", selStmt) return buf.ParsedQuery() } - -// GenerateInsertOuterQuery generates the outer query for inserts. -func GenerateInsertOuterQuery(ins *sqlparser.Insert) *sqlparser.ParsedQuery { - buf := sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("%s %v%sinto %v%v values %a", - ins.Action, - ins.Comments, - ins.Ignore, - ins.Table, - ins.Columns, - ":#values", - ) - return buf.ParsedQuery() -} - -// GenerateUpdateOuterQuery generates the outer query for updates. -// If there is no custom formatting needed, formatter can be nil. -func GenerateUpdateOuterQuery(upd *sqlparser.Update, aliased *sqlparser.AliasedTableExpr, formatter sqlparser.NodeFormatter) *sqlparser.ParsedQuery { - buf := sqlparser.NewTrackedBuffer(formatter) - buf.Myprintf("update %v%v set %v where %a%v", upd.Comments, aliased.RemoveHints(), upd.Exprs, ":#pk", upd.OrderBy) - return buf.ParsedQuery() -} - -// GenerateDeleteOuterQuery generates the outer query for deletes. -func GenerateDeleteOuterQuery(del *sqlparser.Delete, aliased *sqlparser.AliasedTableExpr) *sqlparser.ParsedQuery { - buf := sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("delete %vfrom %v where %a%v", del.Comments, aliased.RemoveHints(), ":#pk", del.OrderBy) - return buf.ParsedQuery() -} - -// GenerateUpdateSubquery generates the subquery for updates. -func GenerateUpdateSubquery(upd *sqlparser.Update, table *schema.Table, aliased *sqlparser.AliasedTableExpr) *sqlparser.ParsedQuery { - return GenerateSubquery( - table, - aliased, - upd.Where, - upd.OrderBy, - upd.Limit, - ) -} - -// GenerateDeleteSubquery generates the subquery for deletes. -func GenerateDeleteSubquery(del *sqlparser.Delete, table *schema.Table, aliased *sqlparser.AliasedTableExpr) *sqlparser.ParsedQuery { - return GenerateSubquery( - table, - aliased, - del.Where, - del.OrderBy, - del.Limit, - ) -} - -// GenerateSubquery generates a subquery based on the input parameters. -func GenerateSubquery(table *schema.Table, tableName *sqlparser.AliasedTableExpr, where *sqlparser.Where, order sqlparser.OrderBy, limit *sqlparser.Limit) *sqlparser.ParsedQuery { - buf := sqlparser.NewTrackedBuffer(nil) - if limit == nil { - limit = execLimit - } - buf.WriteString("select ") - prefix := "" - for _, colnum := range table.PKColumns { - buf.Myprintf("%s%v", prefix, table.Columns[colnum].Name) - prefix = ", " - } - buf.Myprintf(" from %v%v%v%v", tableName, where, order, limit) - buf.Myprintf(sqlparser.ForUpdateStr) - return buf.ParsedQuery() -} diff --git a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt index d26044b6c73..3397ff6ee84 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt +++ b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt @@ -1,7 +1,7 @@ # union "select * from a union select * from b" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "", "Permissions": [ { @@ -20,7 +20,7 @@ # union with limit "select * from a union select * from b limit 10" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "", "Permissions": [ { @@ -36,10 +36,10 @@ "FullQuery": "select * from a union select * from b limit 10" } -# distinct -"select distinct * from a" +# with no where clause +"select * from a" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "a", "Permissions": [ { @@ -48,28 +48,13 @@ } ], "FieldQuery": "select * from a where 1 != 1", - "FullQuery": "select distinct * from a limit :#maxLimit" -} - -# group by -"select * from a group by b" -{ - "PlanID": "PASS_SELECT", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - } - ], - "FieldQuery": "select * from a where 1 != 1 group by b", - "FullQuery": "select * from a group by b limit :#maxLimit" + "FullQuery": "select * from a limit :#maxLimit" } -# having -"select * from a having b=1" +# select with a regular where clause +"select * from a where id=1" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "a", "Permissions": [ { @@ -78,13 +63,13 @@ } ], "FieldQuery": "select * from a where 1 != 1", - "FullQuery": "select * from a having b = 1 limit :#maxLimit" + "FullQuery": "select * from a where id = 1 limit :#maxLimit" } -# limit +# select with limit "select * from a limit 5" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "a", "Permissions": [ { @@ -99,7 +84,7 @@ # limit with offset arg "select * from a limit 10, 5" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "a", "Permissions": [ { @@ -111,10 +96,10 @@ "FullQuery": "select * from a limit 10, 5" } -# limit with offset keyword -"select * from a limit 5 offset 10" +# select impossible +"select * from a where 1 != 1" { - "PlanID": "PASS_SELECT", + "PlanID": "SelectImpossible", "TableName": "a", "Permissions": [ { @@ -123,100 +108,13 @@ } ], "FieldQuery": "select * from a where 1 != 1", - "FullQuery": "select * from a limit 10, 5" -} - -# cross-db -"select * from a.b" -{ - "PlanID": "PASS_SELECT", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 0 - } - ], - "FieldQuery": "select * from a.b where 1 != 1", - "FullQuery": "select * from a.b limit :#maxLimit" -} - -# multi-table -"select * from a,b" -{ - "PlanID": "PASS_SELECT", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - }, - { - "TableName": "b", - "Role": 0 - } - ], - "FieldQuery": "select * from a, b where 1 != 1", - "FullQuery": "select * from a, b limit :#maxLimit" -} - -# multi-table (join) -"select * from a join b" -{ - "PlanID": "PASS_SELECT", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - }, - { - "TableName": "b", - "Role": 0 - } - ], - "FieldQuery": "select * from a join b where 1 != 1", - "FullQuery": "select * from a join b limit :#maxLimit" -} - -# multi-table (right join) -"select * from a right join b on c = d" -{ - "PlanID": "PASS_SELECT", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - }, - { - "TableName": "b", - "Role": 0 - } - ], - "FieldQuery": "select * from a right join b on c = d where 1 != 1", - "FullQuery": "select * from a right join b on c = d limit :#maxLimit" -} - -# Parenthesized table -"select * from (b)" -{ - "PlanID": "PASS_SELECT", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 0 - } - ], - "FieldQuery": "select * from (b) where 1 != 1", - "FullQuery": "select * from (b) limit :#maxLimit" + "FullQuery": "select * from a where 1 != 1 limit :#maxLimit" } # bind in select list "select :bv from a" { - "PlanID": "PASS_SELECT", + "PlanID": "Select", "TableName": "a", "Permissions": [ { @@ -227,55 +125,10 @@ "FullQuery": "select :bv from a limit :#maxLimit" } -# simple -"select eid from a" -{ - "PlanID": "PASS_SELECT", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - } - ], - "FieldQuery": "select eid from a where 1 != 1", - "FullQuery": "select eid from a limit :#maxLimit" -} - -# as -"select eid as foo from a" -{ - "PlanID": "PASS_SELECT", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - } - ], - "FieldQuery": "select eid as foo from a where 1 != 1", - "FullQuery": "select eid as foo from a limit :#maxLimit" -} - -# * -"select * from a" -{ - "PlanID": "PASS_SELECT", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 0 - } - ], - "FieldQuery": "select * from a where 1 != 1", - "FullQuery": "select * from a limit :#maxLimit" -} - -# c.eid -"select c.eid from a as c" +# bind in select list and impossible +"select :bv from a where 1 != 1" { - "PlanID": "PASS_SELECT", + "PlanID": "SelectImpossible", "TableName": "a", "Permissions": [ { @@ -283,1370 +136,158 @@ "Role": 0 } ], - "FieldQuery": "select c.eid from a as c where 1 != 1", - "FullQuery": "select c.eid from a as c limit :#maxLimit" + "FullQuery": "select :bv from a where 1 != 1 limit :#maxLimit" } -# for update -"select eid from a for update" +# single value sequence +"select next value from seq" { - "PlanID": "SELECT_LOCK", - "TableName": "a", + "PlanID": "Nextval", + "TableName": "seq", "Permissions": [ { - "TableName": "a", + "TableName": "seq", "Role": 0 } ], - "FieldQuery": "select eid from a where 1 != 1", - "FullQuery": "select eid from a limit :#maxLimit for update" + "NextCount": "1" } -# lock in share mode -"select eid from a lock in share mode" +# sequence with number +"select next 10 values from seq" { - "PlanID": "SELECT_LOCK", - "TableName": "a", + "PlanID": "Nextval", + "TableName": "seq", "Permissions": [ { - "TableName": "a", + "TableName": "seq", "Role": 0 } ], - "FieldQuery": "select eid from a where 1 != 1", - "FullQuery": "select eid from a limit :#maxLimit lock in share mode" -} - -# insert cross-db -"insert into b.a (eid, id) values (1, :a)" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into b.a(eid, id) values (1, :a)" -} - -# insert cross-db -options:PassthroughDMLs -"insert into b.a (eid, id) values (1, :a)" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into b.a(eid, id) values (1, :a)" -} - -# insert with bind value -"insert into a (eid, id) values (1, :a)" -{ - "PlanID": "INSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, :a)", - "OuterQuery": "insert into a(eid, id) values (1, :a)", - "PKValues": [[1], [":a"]] -} - -# insert with bind value -options:PassthroughDMLs -"insert into a (eid, id) values (1, :a)" -{ - "PlanID": "PASS_DML", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, :a)" -} - -# default number -"insert into a (id) values (1)" -{ - "PlanID": "INSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(id) values (1)", - "OuterQuery": "insert into a(id) values (1)", - "PKValues": [0, [1]] -} - -# default string -"insert into d(id) values(1)" -{ - "PlanID": "INSERT_PK", - "TableName": "d", - "Permissions": [ - { - "TableName": "d", - "Role": 1 - } - ], - "FullQuery": "insert into d(id) values (1)", - "OuterQuery": "insert into d(id) values (1)", - "PKValues": ["0"] -} - -# default without INTO -"insert d(id) values(1)" -{ - "PlanID": "INSERT_PK", - "TableName": "d", - "Permissions": [ - { - "TableName": "d", - "Role": 1 - } - ], - "FullQuery": "insert into d(id) values (1)", - "OuterQuery": "insert into d(id) values (1)", - "PKValues": ["0"] -} - -# mismatch -"insert into a (eid, id) values (1)" -"column count doesn't match value count" - -# negative number -"insert into a (eid, id) values (-1, 2)" -{ - "PlanID": "INSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (-1, 2)", - "OuterQuery": "insert into a(eid, id) values (-1, 2)", - "PKValues": [[-1], [2]] -} - -# positive number -"insert into a (eid, id) values (+1, 2)" -{ - "PlanID": "INSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2)", - "OuterQuery": "insert into a(eid, id) values (1, 2)", - "PKValues": [[1], [2]] -} - -# non-trivial unary -"insert into a (eid, id) values (~1, 2)" -{ - "PlanID": "PASS_DML", - "Reason": "COMPLEX_EXPR", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (~1, 2)" + "NextCount": "10" } -# complex -"insert into a (eid, id) values (1+1, 2)" -{ - "PlanID": "PASS_DML", - "Reason": "COMPLEX_EXPR", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1 + 1, 2)" -} -# complex -"insert into a (eid, id) values (0x04, 2)" +# sequence with bindvar +"select next :a values from seq" { - "PlanID": "PASS_DML", - "Reason": "COMPLEX_EXPR", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (0x04, 2)" -} - -# no index -"insert into c (eid, id) values (1, 2)" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE_NOINDEX", - "TableName": "c", - "Permissions": [ - { - "TableName": "c", - "Role": 1 - } - ], - "FullQuery": "insert into c(eid, id) values (1, 2)" -} - -# no column list -"insert into a values (1, 2, 'name', 'foo', 'camelcase')" -{ - "PlanID": "INSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a values (1, 2, 'name', 'foo', 'camelcase')", - "OuterQuery": "insert into a(eid, id, name, foo, CamelCase) values (1, 2, 'name', 'foo', 'camelcase')", - "PKValues": [[1], [2]] -} - -# upsert multiple unique index -"insert into a (eid, id) values (1, 2) on duplicate key update name = func(a)" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update name = func(a)", - "OuterQuery": "insert into a(eid, id) values (1, 2)", - "UpsertQuery": "update a set name = func(a) where :#pk", - "PKValues": [[1], [2]] -} - -# upsert multiple unique index -options:PassthroughDMLs -"insert into a (eid, id) values (1, 2) on duplicate key update name = func(a)" -{ - "PlanID": "PASS_DML", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update name = func(a)" -} - -# upsert single unique index -"insert into b (eid, id) values (1, 2) on duplicate key update name = func(a)" -{ - "PlanID": "INSERT_PK", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "insert into b(eid, id) values (1, 2) on duplicate key update name = func(a)", - "OuterQuery": "insert into b(eid, id) values (1, 2) on duplicate key update name = func(a)", - "PKValues": [[1], [2]] -} - -# upsert pk change -"insert into a (eid, id) values (1, 2) on duplicate key update eid = 2" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update eid = 2", - "OuterQuery": "insert into a(eid, id) values (1, 2)", - "UpsertQuery": "update a set eid = 2 where :#pk", - "PKValues": [[1], [2]], - "SecondaryPKValues": [2, null] -} - -# upsert with values() func -"insert into a (eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(name)" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(name)", - "OuterQuery": "insert into a(eid, id, name) values (1, 2, 'foo')", - "UpsertQuery": "update a set name = ('foo') where :#pk", - "PKValues": [[1], [2]] -} - -# upsert with values() func and qualified column -"insert into a (eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(a.name)" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(a.name)", - "OuterQuery": "insert into a(eid, id, name) values (1, 2, 'foo')", - "UpsertQuery": "update a set name = ('foo') where :#pk", - "PKValues": [[1], [2]] -} - -# upsert with values() func and qualified column with improper table -"insert into a (eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(c.name)" -"could not find qualified column c.name in table a" - -# upsert with values() func and qualified column and qualified table -"insert into b.a (eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(b.a.name)" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into b.a(eid, id, name) values (1, 2, 'foo') on duplicate key update name = values(b.a.name)" -} - -# upsert with values() inside another func -"insert into a (eid, id, name) values (1, 2, 'foo') on duplicate key update name = concat(values(name), 'foo')" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 'foo') on duplicate key update name = concat(values(name), 'foo')", - "OuterQuery": "insert into a(eid, id, name) values (1, 2, 'foo')", - "UpsertQuery": "update a set name = concat(('foo'), 'foo') where :#pk", - "PKValues": [[1], [2]] -} - -# upsert with values() and simple expression -"insert into a (eid, id, name) values (1, 2, 3) on duplicate key update name = values(name) + 5" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 3) on duplicate key update name = values(name) + 5", - "OuterQuery": "insert into a(eid, id, name) values (1, 2, 3)", - "UpsertQuery": "update a set name = (3) + 5 where :#pk", - "PKValues": [[1], [2]] -} - -# upsert with bindvars in values() -"insert into a (eid, id, name) values (1, :id, :name) on duplicate key update name = values(name), id = values(id)" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, :id, :name) on duplicate key update name = values(name), id = values(id)", - "OuterQuery": "insert into a(eid, id, name) values (1, :id, :name)", - "UpsertQuery": "update a set name = (:name), id = (:id) where :#pk", - "PKValues": [[1], [":id"]], - "SecondaryPKValues": [null, [":id"]] -} - -# complex upsert with values() -"insert into a (eid, id) values (1+1, 2) on duplicate key update eid = values(eid) + 1" -{ - "PlanID": "PASS_DML", - "Reason": "COMPLEX_EXPR", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1 + 1, 2) on duplicate key update eid = values(eid) + 1" -} - -# upsert where values for pk references a complex expression -"insert into a (eid, id, name) values (1, 2, 1+1) on duplicate key update eid = values(name)" -{ - "PlanID": "PASS_DML", - "Reason": "PK_CHANGE", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 1 + 1) on duplicate key update eid = values(name)", - "PKValues": [[1],[2]] -} - -# upsert where values for pk references a non-existent column -"insert into a (eid, id) values (1, 2) on duplicate key update eid = values(name)" -{ - "PlanID": "PASS_DML", - "Reason": "PK_CHANGE", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update eid = values(name)", - "PKValues": [[1],[2]] -} - -# upsert with mismatch values() -"insert into a (eid, id) values (1, 2) on duplicate key update eid = values(eid), name = values(name)" -"could not find column &{ name { }}" - -# upsert pk change, with values() func -"insert into a (eid, id) values (1, 2) on duplicate key update eid = values(eid), id = values(id)" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update eid = values(eid), id = values(id)", - "OuterQuery": "insert into a(eid, id) values (1, 2)", - "UpsertQuery": "update a set eid = (1), id = (2) where :#pk", - "PKValues": [[1], [2]], - "SecondaryPKValues": [[1], [2]] -} - -# upsert pk change, with mixed expr and values() func -"insert into a (eid, id, name) values (1, 2, 'foo') on duplicate key update eid = 2, id = values(id), name = func()" -{ - "PlanID": "UPSERT_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(eid, id, name) values (1, 2, 'foo') on duplicate key update eid = 2, id = values(id), name = func()", - "OuterQuery": "insert into a(eid, id, name) values (1, 2, 'foo')", - "UpsertQuery": "update a set eid = 2, id = (2), name = func() where :#pk", - "PKValues": [[1], [2]], - "SecondaryPKValues": [2, [2]] -} - -# upsert complex pk change -"insert into a (id, eid) values (1, 2) on duplicate key update eid = func(a)" -{ - "PlanID": "PASS_DML", - "Reason": "PK_CHANGE", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(id, eid) values (1, 2) on duplicate key update eid = func(a)", - "PKValues": [[2], [1]] -} - -# upsert multi-row with multiple unique indexes -"insert into a (id, eid) values (1, 2), (2, 3) on duplicate key update name = func(a)" -{ - "PlanID": "PASS_DML", - "Reason": "UPSERT_MULTI_ROW", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "insert into a(id, eid) values (1, 2), (2, 3) on duplicate key update name = func(a)", - "PKValues": [[2,3],[1,2]] -} - -# upsert multi-row with 0-1 unique indexes -"insert into b (id, eid) values (1, 2), (2, 3) on duplicate key update name = func(a)" -{ - "PlanID": "INSERT_PK", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "insert into b(id, eid) values (1, 2), (2, 3) on duplicate key update name = func(a)", - "OuterQuery": "insert into b(id, eid) values (1, 2), (2, 3) on duplicate key update name = func(a)", - "PKValues": [[2,3],[1,2]] -} - -# upsert multi-row with 0-1 unique indexes and pk change -"insert into b (id, eid) values (1, 2), (2, 3) on duplicate key update id = 1" -{ - "PlanID": "INSERT_PK", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "insert into b(id, eid) values (1, 2), (2, 3) on duplicate key update id = 1", - "OuterQuery": "insert into b(id, eid) values (1, 2), (2, 3) on duplicate key update id = 1", - "PKValues": [[2,3],[1,2]], - "SecondaryPKValues": [null, 1] -} - -# upsert multi-row with 0-1 unique indexes and pk change, using values, and a different column source -"insert into b (id, eid) values (1, 2), (3, 4) on duplicate key update id = values(eid)" -{ - "PlanID": "INSERT_PK", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "insert into b(id, eid) values (1, 2), (3, 4) on duplicate key update id = values(eid)", - "OuterQuery": "insert into b(id, eid) values (1, 2), (3, 4) on duplicate key update id = values(eid)", - "PKValues": [[2,4],[1,3]], - "SecondaryPKValues": [null, [2, 4]] -} - -# upsert subquery -"insert into b (id, eid) select * from a on duplicate key update name = func(a)" -{ - "PlanID": "PASS_DML", - "Reason": "UPSERT_SUBQUERY", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "a", - "Role": 0 - } - ], - "FullQuery": "insert into b(id, eid) select * from a on duplicate key update name = func(a)" -} - -# subquery -"insert into b (eid, id) select * from a" -{ - "PlanID": "INSERT_SUBQUERY", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "a", - "Role": 0 - } - ], - "FullQuery": "insert into b(eid, id) select * from a", - "OuterQuery": "insert into b(eid, id) values :#values", - "Subquery": "select * from a limit :#maxLimit", - "ColumnNumbers": [0, 1], - "SubqueryPKColumns": [0, 1] -} - -# subquery with no column list -"insert into b select * from a" -{ - "PlanID": "INSERT_SUBQUERY", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "a", - "Role": 0 - } - ], - "FullQuery": "insert into b select * from a", - "OuterQuery": "insert into b(eid, id) values :#values", - "Subquery": "select * from a limit :#maxLimit", - "ColumnNumbers": [0, 1], - "SubqueryPKColumns": [0, 1] -} - -# multi-row -"insert into b (eid, id) values (1, 2), (3, 4)" -{ - "PlanID": "INSERT_PK", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "insert into b(eid, id) values (1, 2), (3, 4)", - "OuterQuery": "insert into b(eid, id) values (1, 2), (3, 4)", - "PKValues": [[1, 3], [2, 4]] -} - -# topic insert with time_scheduled specified -"insert into test_topic(time_scheduled, id, message) values(1, 2, 'aa')" -{ - "PlanID": "INSERT_TOPIC", - "Reason": "TOPIC", - "TableName": "test_topic", - "Permissions": [ - { - "TableName": "test_topic", - "Role": 1 - } - ], - "FullQuery": "insert into test_topic(time_scheduled, id, message) values (1, 2, 'aa')" -} - -# topic update -"update test_topic set time_next = 1 where id = 1" -"updates not allowed on topics" - -# topic delete -"delete from test_topic where id = 1" -"deletes not allowed on topics" - -# message insert with time_scheduled specified -"insert into msg(time_scheduled, id, message) values(1, 2, 'aa')" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(time_scheduled, id, message) values (1, 2, 'aa')", - "OuterQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0)", - "PKValues": [[1], [2]] -} - -# message insert with time_scheduled specified with PassthroughDMLs -options:PassthroughDMLs -"insert into msg(time_scheduled, id, message) values(1, 2, 'aa')" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(time_scheduled, id, message) values (1, 2, 'aa')", - "OuterQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0)", - "PKValues": [[1], [2]] -} - -# message insert with no time_schedule -"insert into msg(id, message) values(2, 'aa')" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(id, message) values (2, 'aa')", - "OuterQuery": "insert into msg(id, message, time_scheduled, time_next, time_created, epoch) values (2, 'aa', :#time_now, :#time_now, :#time_now, 0)", - "PKValues": [[":#time_now"], [2]] -} - -# message insert with no time_schedule with PassthroughDMLs -options:PassthroughDMLs -"insert into msg(id, message) values(2, 'aa')" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(id, message) values (2, 'aa')", - "OuterQuery": "insert into msg(id, message, time_scheduled, time_next, time_created, epoch) values (2, 'aa', :#time_now, :#time_now, :#time_now, 0)", - "PKValues": [[":#time_now"], [2]] -} - -# message multi-value insert -"insert into msg(time_scheduled, id, message) values(1, 2, 'aa'), (3, 4, 'bb')" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(time_scheduled, id, message) values (1, 2, 'aa'), (3, 4, 'bb')", - "OuterQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0), (3, 4, 'bb', 3, :#time_now, 0)", - "PKValues": [ - [ - 1, - 3 - ], - [ - 2, - 4 - ] - ] -} - -# message multi-value upsert -"insert into msg(time_scheduled, id, message) values(1, 2, 'aa'), (3, 4, 'bb') on duplicate key update message = values(message)" -{ - "PlanID": "INSERT_MESSAGE", - "TableName": "msg", - "Permissions": [ - { - "TableName": "msg", - "Role": 1 - } - ], - "FullQuery": "insert into msg(time_scheduled, id, message) values (1, 2, 'aa'), (3, 4, 'bb') on duplicate key update message = values(message)", - "OuterQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0), (3, 4, 'bb', 3, :#time_now, 0) on duplicate key update message = values(message)", - "PKValues": [ - [ - 1, - 3 - ], - [ - 2, - 4 - ] - ] -} - -# message insert subquery -"insert into msg(time_scheduled, id, message) select * from a" -"subquery not allowed for message table: msg" - -# message insert without column list -"insert into msg values(1)" -"column list must be specified for message table insert: msg" - -# message column count mismatch -"insert into msg(id, message) values(1)" -"column count doesn't match value count" - -# message insert time_next -"insert into msg(id, message, time_next) values(2, 'aa', 3)" -"time_next must not be specified for message insert" - -# message insert time_created -"insert into msg(id, message, time_created) values(2, 'aa', 3)" -"time_created must not be specified for message insert" - -# message insert epoch -"insert into msg(id, message, epoch) values(2, 'aa', 3)" -"epoch must not be specified for message insert" - -# message insert time_acked -"insert into msg(id, message, time_acked) values(2, 'aa', 3)" -"time_acked must not be specified for message insert" - -# message insert id missing -"insert into msg(message) values('aa')" -"id must be specified for message insert" - -# multi-row -"replace into b (eid, id) values (1, 2), (3, 4)" -{ - "PlanID": "PASS_DML", - "Reason":"REPLACE", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "replace into b(eid, id) values (1, 2), (3, 4)" -} - -# multi-row -options:PassthroughDMLs -"replace into b (eid, id) values (1, 2), (3, 4)" -{ - "PlanID": "PASS_DML", - "Reason": "REPLACE", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "replace into b(eid, id) values (1, 2), (3, 4)" -} - -# single-row with set -"replace into b set eid = 1, id = 2" -{ - "PlanID": "PASS_DML", - "Reason":"REPLACE", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "replace into b(eid, id) values (1, 2)" -} - -# replace subquery -"replace into b (eid, id) select * from a" -{ - "PlanID": "PASS_DML", - "Reason":"REPLACE", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "a", - "Role": 0 - } - ], - "FullQuery": "replace into b(eid, id) select * from a" -} - -# subquery with no column list -"replace into b select * from a" -{ - "PlanID": "PASS_DML", - "Reason":"REPLACE", - "TableName": "", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "a", - "Role": 0 - } - ], - "FullQuery": "replace into b select * from a" -} - -# update limit with pk -"update d set foo='foo' where name in ('a', 'b') limit 1" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "d", - "Permissions": [ - { - "TableName": "d", - "Role": 1 - } - ], - "FullQuery": "update d set foo = 'foo' where name in ('a', 'b') limit 1", - "OuterQuery": "update d set foo = 'foo' where :#pk", - "Subquery": "select name from d where name in ('a', 'b') limit 1 for update", - "WhereClause": "where name in ('a', 'b')" -} - -# update limit with pk -options:PassthroughDMLs -"update d set foo='foo' where name in ('a', 'b') limit 1" -{ - "PlanID": "PASS_DML", - "TableName": "d", - "Permissions": [ - { - "TableName": "d", - "Role": 1 - } - ], - "FullQuery": "update d set foo = 'foo' where name in ('a', 'b') limit 1" -} - -# update cross-db -"update b.a set name='foo' where eid=1 and id=1" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update b.a set name = 'foo' where eid = 1 and id = 1" -} - -# update cross-db -options:PassthroughDMLs -"update b.a set name='foo' where eid=1 and id=1" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update b.a set name = 'foo' where eid = 1 and id = 1" -} - -# update unknown table -"update bogus set name='foo' where id=1" -"table bogus not found in schema" - -# update unknown table -options:PassthroughDMLs -"update bogus set name='foo' where id=1" -{ - "PlanID": "PASS_DML", - "TableName": "", - "Permissions": [ - { - "TableName": "bogus", - "Role": 1 - } - ], - "FullQuery": "update bogus set name = 'foo' where id = 1" -} - -# multi-table update -"update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" -{ - "PlanID": "PASS_DML", - "Reason": "MULTI_TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - }, - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" -} - -# multi-table update -"update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" -{ - "PlanID": "PASS_DML", - "Reason": "MULTI_TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - }, - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" -} - -# multi-table update -options:PassthroughDMLs -"update a join b on a.id = b.id set a.name = 'foo' where b.var = 'test'" -{ - "PlanID": "PASS_DML", - "Reason": "MULTI_TABLE", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - }, - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update a join b on a.id = b.id set a.name = 'foo' where b.var = 'test'" -} - -# pk changed -"update b set eid=1" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update b set eid = 1", - "OuterQuery": "update b set eid = 1 where :#pk", - "Subquery": "select eid, id from b limit :#maxLimit for update", - "SecondaryPKValues": [1, null], - "WhereClause": "" -} - -# type mismatch -"update b set eid=18446744073709551616" -"strconv.ParseUint: parsing "18446744073709551616": value out of range" - -# complex pk change -"update b set eid=foo()" -{ - "PlanID": "PASS_DML", - "Reason": "PK_CHANGE", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update b set eid = foo()", - "WhereClause": "" -} - -# complex pk change -options:PassthroughDMLs -"update b set eid=foo()" -{ - "PlanID": "PASS_DML", - "TableName": "b", - "Permissions": [ - { - "TableName": "b", - "Role": 1 - } - ], - "FullQuery": "update b set eid = foo()" -} - -# update subquery -"update a set name='foo'" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo'", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a limit :#maxLimit for update", - "WhereClause": "" -} - -# update complex where clause -"update a set name='foo' where eid+1=1" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid + 1 = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a where eid + 1 = 1 limit :#maxLimit for update", - "WhereClause": "where eid + 1 = 1" -} - -# pk -"update a set name='foo' where eid=1 and id=1" -{ - "PlanID": "DML_PK", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid = 1 and id = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "PKValues": [1, 1], - "WhereClause": "where eid = 1 and id = 1" -} - -# partial pk -"update a set name='foo' where eid=1" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a where eid = 1 limit :#maxLimit for update", - "WhereClause": "where eid = 1" -} - -# bad pk -"update a set name='foo' where eid=1.0 and id=1" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid = 1.0 and id = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a where eid = 1.0 and id = 1 limit :#maxLimit for update", - "WhereClause": "where eid = 1.0 and id = 1" -} - -# partial pk with limit -"update a set name='foo' where eid=1 limit 10" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid = 1 limit 10", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a where eid = 1 limit 10 for update", - "WhereClause": "where eid = 1" -} - -# non-pk -"update a set name='foo' where eid=1 and name='foo'" -{ - "PlanID": "DML_SUBQUERY", - "TableName": "a", - "Permissions": [ - { - "TableName": "a", - "Role": 1 - } - ], - "FullQuery": "update a set name = 'foo' where eid = 1 and name = 'foo'", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a where eid = 1 and name = 'foo' limit :#maxLimit for update", - "WhereClause": "where eid = 1 and name = 'foo'" -} - -# no index -"update c set eid=1" -{ - "PlanID": "PASS_DML", - "Reason": "TABLE_NOINDEX", - "TableName": "c", + "PlanID": "Nextval", + "TableName": "seq", "Permissions": [ { - "TableName": "c", - "Role": 1 + "TableName": "seq", + "Role": 0 } ], - "FullQuery": "update c set eid = 1", - "WhereClause": "" + "NextCount": "\":a\"" } -# complex expression in where -"update a set name='foo' where eid+1=1 and id=1" +# squence with bad value +"select next 12345667852342342342323423423 values from seq" +"strconv.ParseUint: parsing "12345667852342342342323423423": value out of range" + +# nextval on non-sequence table +"select next value from a" +"a is not a sequence" + +# nextval on non-existent table +"select next value from id" +"id is not a sequence" + +# for update +"select eid from a for update" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "SelectLock", + "TableName": "a", "Permissions": [ { "TableName": "a", - "Role": 1 + "Role": 0 } ], - "FullQuery":"update a set name = 'foo' where eid + 1 = 1 and id = 1", - "OuterQuery":"update a set name = 'foo' where :#pk", - "Subquery":"select eid, id from a where eid + 1 = 1 and id = 1 limit :#maxLimit for update", - "WhereClause": "where eid + 1 = 1 and id = 1" + "FieldQuery": "select eid from a where 1 != 1", + "FullQuery": "select eid from a limit :#maxLimit for update" } -# parenthesized expressions in where -"update a set name='foo' where (eid=1) and id=1" +# lock in share mode +"select eid from a lock in share mode" { - "PlanID": "DML_PK", + "PlanID": "SelectLock", "TableName": "a", "Permissions": [ { "TableName": "a", - "Role": 1 + "Role": 0 } ], - "FullQuery": "update a set name = 'foo' where (eid = 1) and id = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "PKValues": [1, 1], - "WhereClause": "where (eid = 1) and id = 1" + "FieldQuery": "select eid from a where 1 != 1", + "FullQuery": "select eid from a limit :#maxLimit lock in share mode" } -# in clause expression in where -"update a set name='foo' where eid in (1, 2) and id=1" +# normal insert +"insert into a(eid, id) values (1, 2)" { - "PlanID":"DML_PK", - "TableName":"a", + "PlanID": "Insert", + "TableName": "a", "Permissions": [ { "TableName": "a", "Role": 1 } ], - "FullQuery":"update a set name = 'foo' where eid in (1, 2) and id = 1", - "OuterQuery":"update a set name = 'foo' where :#pk", - "PKValues":[[1,2],1], - "WhereClause": "where eid in (1, 2) and id = 1" + "FullQuery": "insert into a(eid, id) values (1, 2)" } -# 'like' expression in where -"update d set foo='foo' where name like 'a%'" +# insert cross-db +"insert into b.a (eid, id) values (1, 2)" { - "PlanID": "DML_SUBQUERY", - "TableName": "d", + "PlanID": "Insert", + "TableName": "", "Permissions": [ { - "TableName": "d", + "TableName": "a", "Role": 1 } ], - "FullQuery": "update d set foo = 'foo' where name like 'a%'", - "OuterQuery": "update d set foo = 'foo' where :#pk", - "Subquery": "select name from d where name like 'a%' limit :#maxLimit for update", - "WhereClause": "where name like 'a%'" + "FullQuery": "insert into b.a(eid, id) values (1, 2)" } -# double in clause -"update a set name='foo' where eid in (1, 2) and id in (1, 2)" +# insert with bind value +"insert into a (eid, id) values (1, :a)" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "Insert", + "TableName": "a", "Permissions": [ { "TableName": "a", "Role": 1 } ], - "FullQuery":"update a set name = 'foo' where eid in (1, 2) and id in (1, 2)", - "OuterQuery":"update a set name = 'foo' where :#pk", - "Subquery":"select eid, id from a where eid in (1, 2) and id in (1, 2) limit :#maxLimit for update", - "WhereClause": "where eid in (1, 2) and id in (1, 2)" + "FullQuery": "insert into a(eid, id) values (1, :a)" } -# double use of pk -"update a set name='foo' where eid=1 and eid=2" +# insert with subquery +"insert into b (eid, id) select * from a" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "Insert", + "TableName": "b", "Permissions": [ { - "TableName": "a", + "TableName": "b", "Role": 1 + }, + { + "TableName": "a", + "Role": 0 } ], - "FullQuery":"update a set name = 'foo' where eid = 1 and eid = 2", - "OuterQuery":"update a set name = 'foo' where :#pk", - "Subquery":"select eid, id from a where eid = 1 and eid = 2 limit :#maxLimit for update", - "WhereClause": "where eid = 1 and eid = 2" + "FullQuery": "insert into b(eid, id) select * from a" } -# partial pk with order by -"update a set name = 'foo' where eid=1 order by id desc" +# upsert +"insert into a (eid, id) values (1, 2) on duplicate key update name = func(a)" { - "PlanID": "DML_SUBQUERY", + "PlanID": "Insert", "TableName": "a", "Permissions": [ { @@ -1654,181 +295,189 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "update a set name = 'foo' where eid = 1 order by id desc", - "OuterQuery": "update a set name = 'foo' where :#pk order by id desc", - "Subquery": "select eid, id from a where eid = 1 order by id desc limit :#maxLimit for update", - "WhereClause": "where eid = 1" + "FullQuery": "insert into a(eid, id) values (1, 2) on duplicate key update name = func(a)" } -# update with index hint -# note that you won't find a corresponding test for delete because the grammar doesn't allow it. -"update a use index(b) set name = 'foo' where eid=1" +# topic insert with time_scheduled specified +"insert into test_topic(time_scheduled, id, message) values(1, 2, 'aa')" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "InsertTopic", + "TableName": "test_topic", "Permissions": [ { - "TableName": "a", + "TableName": "test_topic", "Role": 1 } ], - "FullQuery": "update a use index (b) set name = 'foo' where eid = 1", - "OuterQuery": "update a set name = 'foo' where :#pk", - "Subquery": "select eid, id from a use index (b) where eid = 1 limit :#maxLimit for update", - "WhereClause": "where eid = 1" + "FullQuery": "insert into test_topic(time_scheduled, id, message) values (1, 2, 'aa')" } -# delete limit with pk -"delete from d where name in ('a', 'b') limit 1" +# message insert with time_scheduled specified +"insert into msg(time_scheduled, id, message) values(1, 2, 'aa')" { - "PlanID": "DML_SUBQUERY", - "TableName": "d", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "d", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from d where name in ('a', 'b') limit 1", - "OuterQuery": "delete from d where :#pk", - "Subquery": "select name from d where name in ('a', 'b') limit 1 for update", - "WhereClause": "where name in ('a', 'b')" + "FullQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0)" } -# delete limit with pk +# message insert with time_scheduled specified with PassthroughDMLs options:PassthroughDMLs -"delete from d where name in ('a', 'b') limit 1" +"insert into msg(time_scheduled, id, message) values(1, 2, 'aa')" { - "PlanID": "PASS_DML", - "TableName": "d", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "d", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from d where name in ('a', 'b') limit 1" + "FullQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0)" } -# delete cross-db -"delete from b.a where eid=1 and id=1" +# message insert with no time_schedule +"insert into msg(id, message) values(2, 'aa')" { - "PlanID": "PASS_DML", - "Reason": "TABLE", - "TableName": "", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "a", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from b.a where eid = 1 and id = 1" + "FullQuery": "insert into msg(id, message, time_scheduled, time_next, time_created, epoch) values (2, 'aa', :#time_now, :#time_now, :#time_now, 0)" } -# delete with no where clause -"delete from a" +# message insert with no time_schedule with PassthroughDMLs +options:PassthroughDMLs +"insert into msg(id, message) values(2, 'aa')" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "a", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from a", - "OuterQuery": "delete from a where :#pk", - "Subquery": "select eid, id from a limit :#maxLimit for update", - "WhereClause": "" + "FullQuery": "insert into msg(id, message, time_scheduled, time_next, time_created, epoch) values (2, 'aa', :#time_now, :#time_now, :#time_now, 0)" } -# delete complex where clause -"delete from a where eid+1=1" +# message multi-value insert +"insert into msg(time_scheduled, id, message) values(1, 2, 'aa'), (3, 4, 'bb')" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "a", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from a where eid + 1 = 1", - "OuterQuery": "delete from a where :#pk", - "Subquery": "select eid, id from a where eid + 1 = 1 limit :#maxLimit for update", - "WhereClause": "where eid + 1 = 1" + "FullQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0), (3, 4, 'bb', 3, :#time_now, 0)" } -# pk -"delete from a where eid=1 and id=1" +# message multi-value upsert +"insert into msg(time_scheduled, id, message) values(1, 2, 'aa'), (3, 4, 'bb') on duplicate key update message = values(message)" { - "PlanID": "DML_PK", - "TableName": "a", + "PlanID": "InsertMessage", + "TableName": "msg", "Permissions": [ { - "TableName": "a", + "TableName": "msg", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1 and id = 1", - "OuterQuery": "delete from a where :#pk", - "PKValues": [1, 1], - "WhereClause": "where eid = 1 and id = 1" + "FullQuery": "insert into msg(time_scheduled, id, message, time_next, time_created, epoch) values (1, 2, 'aa', 1, :#time_now, 0), (3, 4, 'bb', 3, :#time_now, 0) on duplicate key update message = values(message)" } -# pk -options:PassthroughDMLs -"delete from a where eid=1 and id=1" +# message insert subquery +"insert into msg(time_scheduled, id, message) select * from a" +"subquery not allowed for message table: msg" + +# message insert without column list +"insert into msg values(1)" +"column list must be specified for message table insert: msg" + +# message column count mismatch +"insert into msg(id, message) values(1)" +"column count doesn't match value count" + +# message insert time_next +"insert into msg(id, message, time_next) values(2, 'aa', 3)" +"time_next must not be specified for message insert" + +# message insert time_created +"insert into msg(id, message, time_created) values(2, 'aa', 3)" +"time_created must not be specified for message insert" + +# message insert epoch +"insert into msg(id, message, epoch) values(2, 'aa', 3)" +"epoch must not be specified for message insert" + +# message insert time_acked +"insert into msg(id, message, time_acked) values(2, 'aa', 3)" +"time_acked must not be specified for message insert" + +# message insert id missing +"insert into msg(message) values('aa')" +"id must be specified for message insert" + +# replace +"replace into b (eid, id) values (1, 2), (3, 4)" { - "PlanID": "PASS_DML", - "TableName": "a", + "PlanID": "Insert", + "TableName": "b", "Permissions": [ { - "TableName": "a", + "TableName": "b", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1 and id = 1" + "FullQuery": "replace into b(eid, id) values (1, 2), (3, 4)" } -# partial pk -"delete from a where eid=1" +# update with no where clause +"update d set foo='foo'" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "UpdateLimit", + "TableName": "d", "Permissions": [ { - "TableName": "a", + "TableName": "d", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1", - "OuterQuery": "delete from a where :#pk", - "Subquery": "select eid, id from a where eid = 1 limit :#maxLimit for update", - "WhereClause": "where eid = 1" + "FullQuery": "update d set foo = 'foo' limit :#maxLimit" } -# partial pk with order by -"delete from a where eid=1 order by id desc" +# normal update +"update d set foo='foo' where name in ('a', 'b')" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "UpdateLimit", + "TableName": "d", "Permissions": [ { - "TableName": "a", + "TableName": "d", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1 order by id desc", - "OuterQuery": "delete from a where :#pk order by id desc", - "Subquery": "select eid, id from a where eid = 1 order by id desc limit :#maxLimit for update", - "WhereClause": "where eid = 1" + "FullQuery": "update d set foo = 'foo' where name in ('a', 'b') limit :#maxLimit", + "WhereClause": "where name in ('a', 'b')" } -# 'like' expression -"delete from d where name like 'a%'" +# normal update +options:PassthroughDMLs +"update d set foo='foo' where name in ('a', 'b')" { - "PlanID": "DML_SUBQUERY", + "PlanID": "Update", "TableName": "d", "Permissions": [ { @@ -1836,279 +485,232 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "delete from d where name like 'a%'", - "OuterQuery": "delete from d where :#pk", - "Subquery": "select name from d where name like 'a%' limit :#maxLimit for update", - "WhereClause": "where name like 'a%'" + "FullQuery": "update d set foo = 'foo' where name in ('a', 'b')", + "WhereClause": "where name in ('a', 'b')" } -# bad pk value delete -"delete from a where eid=1.0 and id=1" +# cross-db update +"update a.b set foo='foo' where name in ('a', 'b')" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "UpdateLimit", + "TableName": "", "Permissions": [ { - "TableName": "a", + "TableName": "b", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1.0 and id = 1", - "OuterQuery": "delete from a where :#pk", - "Subquery": "select eid, id from a where eid = 1.0 and id = 1 limit :#maxLimit for update", - "WhereClause": "where eid = 1.0 and id = 1" + "FullQuery": "update a.b set foo = 'foo' where name in ('a', 'b') limit :#maxLimit", + "WhereClause": "where name in ('a', 'b')" } -# non-pk -"delete from a where eid=1 and name='foo'" +# update unknown table +"update bogus set name='foo' where id=1" { - "PlanID": "DML_SUBQUERY", - "TableName": "a", + "PlanID": "UpdateLimit", + "TableName": "", "Permissions": [ { - "TableName": "a", + "TableName": "bogus", "Role": 1 } ], - "FullQuery": "delete from a where eid = 1 and name = 'foo'", - "OuterQuery": "delete from a where :#pk", - "Subquery": "select eid, id from a where eid = 1 and name = 'foo' limit :#maxLimit for update", - "WhereClause": "where eid = 1 and name = 'foo'" + "FullQuery": "update bogus set name = 'foo' where id = 1 limit :#maxLimit", + "WhereClause": "where id = 1" } -# no index -"delete from c" +# update unknown table +options:PassthroughDMLs +"update bogus set name='foo' where id=1" { - "PlanID": "PASS_DML", - "Reason": "TABLE_NOINDEX", - "TableName": "c", + "PlanID": "Update", + "TableName": "", "Permissions": [ { - "TableName": "c", + "TableName": "bogus", "Role": 1 } ], - "FullQuery": "delete from c", - "WhereClause": "" + "FullQuery": "update bogus set name = 'foo' where id = 1", + "WhereClause": "where id = 1" } -# delete complex expression in where -"delete from a where eid+1=1 and id=1" +# multi-table update +"update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "UpdateLimit", + "TableName": "", "Permissions": [ { "TableName": "a", "Role": 1 - } - ], - "FullQuery":"delete from a where eid + 1 = 1 and id = 1", - "OuterQuery":"delete from a where :#pk", - "Subquery":"select eid, id from a where eid + 1 = 1 and id = 1 limit :#maxLimit for update", - "WhereClause": "where eid + 1 = 1 and id = 1" -} - -# parenthesized expressions in where -"delete from a where (eid=1) and id=1" -{ - "PlanID": "DML_PK", - "TableName": "a", - "Permissions": [ + }, { - "TableName": "a", + "TableName": "b", "Role": 1 } ], - "FullQuery": "delete from a where (eid = 1) and id = 1", - "OuterQuery": "delete from a where :#pk", - "PKValues": [1, 1], - "WhereClause": "where (eid = 1) and id = 1" + "FullQuery": "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test' limit :#maxLimit", + "WhereClause": "where a.id = b.id and b.var = 'test'" } -# delete in clause expression in where -"delete from a where eid in (1, 2) and id=1" +# multi-table update +options:PassthroughDMLs +"update a join b on a.id = b.id set a.name = 'foo' where b.var = 'test'" { - "PlanID":"DML_PK", - "TableName":"a", + "PlanID": "Update", + "TableName": "", "Permissions": [ { "TableName": "a", "Role": 1 + }, + { + "TableName": "b", + "Role": 1 } ], - "FullQuery":"delete from a where eid in (1, 2) and id = 1", - "OuterQuery":"delete from a where :#pk", - "PKValues":[[1,2],1], - "WhereClause": "where eid in (1, 2) and id = 1" + "FullQuery": "update a join b on a.id = b.id set a.name = 'foo' where b.var = 'test'", + "WhereClause": "where b.var = 'test'" } -# delete double in clause -"delete from a where eid in (1, 2) and id in (1, 2)" + +# update with limit +"update a set name='foo' limit 1" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "Update", + "TableName": "a", "Permissions": [ { "TableName": "a", "Role": 1 } ], - "FullQuery":"delete from a where eid in (1, 2) and id in (1, 2)", - "OuterQuery":"delete from a where :#pk", - "Subquery":"select eid, id from a where eid in (1, 2) and id in (1, 2) limit :#maxLimit for update", - "WhereClause": "where eid in (1, 2) and id in (1, 2)" + "FullQuery": "update a set name = 'foo' limit 1" } -# delete double use of pk -"delete from a where eid=1 and eid=2" +# update with limit +options:PassthroughDMLs +"update a set name='foo' limit 1" { - "PlanID":"DML_SUBQUERY", - "TableName":"a", + "PlanID": "Update", + "TableName": "a", "Permissions": [ { "TableName": "a", "Role": 1 } ], - "FullQuery":"delete from a where eid = 1 and eid = 2", - "OuterQuery":"delete from a where :#pk", - "Subquery":"select eid, id from a where eid = 1 and eid = 2 limit :#maxLimit for update", - "WhereClause": "where eid = 1 and eid = 2" + "FullQuery": "update a set name = 'foo' limit 1" } -# multi-table delete -"delete a, b from a, b, c where a.id = b.id and b.id = c.id and c.name = 'foo'" +# delete with no where clause +"delete from a" { - "PlanID": "PASS_DML", - "Reason": "MULTI_TABLE", - "TableName": "", + "PlanID": "DeleteLimit", + "TableName": "a", "Permissions": [ { "TableName": "a", "Role": 1 - }, - { - "TableName": "b", - "Role": 1 - }, - { - "TableName": "c", - "Role": 1 } ], - "FullQuery": "delete a, b from a, b, c where a.id = b.id and b.id = c.id and c.name = 'foo'" + "FullQuery": "delete from a limit :#maxLimit" } -# multi-table delete with join -"delete a from a join b on a.id = b.id where a.name = 'foo'" +# normal delete +"delete from d where name in ('a', 'b')" { - "PlanID": "PASS_DML", - "Reason": "MULTI_TABLE", - "TableName": "", + "PlanID": "DeleteLimit", + "TableName": "d", "Permissions": [ { - "TableName": "a", - "Role": 1 - }, - { - "TableName": "b", + "TableName": "d", "Role": 1 } ], - "FullQuery": "delete a from a join b on a.id = b.id where a.name = 'foo'" + "FullQuery": "delete from d where name in ('a', 'b') limit :#maxLimit", + "WhereClause": "where name in ('a', 'b')" } -# single value sequence -"select next value from seq" +# normal delete +options:PassthroughDMLs +"delete from d where name in ('a', 'b')" { - "PlanID": "NEXTVAL", - "TableName": "seq", + "PlanID": "Delete", + "TableName": "d", "Permissions": [ { - "TableName": "seq", - "Role": 0 + "TableName": "d", + "Role": 1 } ], - "PKValues":[1] + "FullQuery": "delete from d where name in ('a', 'b')", + "WhereClause": "where name in ('a', 'b')" } -# sequence with number -"select next 10 values from seq" +# delete unknown table +"delete from bogus" { - "PlanID": "NEXTVAL", - "TableName": "seq", + "PlanID": "DeleteLimit", + "TableName": "", "Permissions": [ { - "TableName": "seq", - "Role": 0 + "TableName": "bogus", + "Role": 1 } ], - "PKValues":[10] + "FullQuery": "delete from bogus limit :#maxLimit" } -# sequence with bindvar -"select next :a values from seq" +# delete unknown table +options:PassthroughDMLs +"delete from bogus" { - "PlanID": "NEXTVAL", - "TableName": "seq", + "PlanID": "Delete", + "TableName": "", "Permissions": [ { - "TableName": "seq", - "Role": 0 + "TableName": "bogus", + "Role": 1 } ], - "PKValues":[":a"] + "FullQuery": "delete from bogus" } -# insert no values autoinc -"insert into auto values ()" +# delete with limit +"delete from a limit 10" { - "PlanID": "INSERT_PK", - "TableName": "auto", + "PlanID": "Delete", + "TableName": "a", "Permissions": [ { - "TableName": "auto", + "TableName": "a", "Role": 1 } ], - "FullQuery": "insert into auto values ()", - "OuterQuery": "insert into auto(id) values (null)", - "PKValues":[ - [null] - ] + "FullQuery": "delete from a limit 10" } -# insert no values defaults -"insert into with_defaults values ()" +# delete with limit +options:PassthroughDMLs +"delete from a limit 10" { - "PlanID": "INSERT_PK", - "TableName": "with_defaults", + "PlanID": "Delete", + "TableName": "a", "Permissions": [ { - "TableName": "with_defaults", + "TableName": "a", "Role": 1 } ], - "FullQuery": "insert into with_defaults values ()", - "OuterQuery": "insert into with_defaults(aid, bid, cid) values (3, -2, null)", - "PKValues":[ - 3 - ] + "FullQuery": "delete from a limit 10" } -# nextval on non-sequence table -"select next value from a" -"a is not a sequence" - -# nextval on non-existent table -"select next value from id" -"table id not found in schema" - # int "set a=1" { - "PlanID": "SET", + "PlanID": "Set", "TableName": "", "FullQuery": "set a = 1" } @@ -2116,7 +718,7 @@ options:PassthroughDMLs # float "set a=1.2" { - "PlanID": "SET", + "PlanID": "Set", "TableName": "", "FullQuery": "set a = 1.2" } @@ -2124,7 +726,7 @@ options:PassthroughDMLs # string "set a='b'" { - "PlanID": "SET", + "PlanID": "Set", "TableName": "", "FullQuery": "set a = 'b'" } @@ -2132,7 +734,7 @@ options:PassthroughDMLs # multi "set a=1, b=2" { - "PlanID": "SET", + "PlanID": "Set", "TableName": "", "FullQuery": "set a = 1, b = 2" } @@ -2295,58 +897,38 @@ options:PassthroughDMLs # show "show a" { - "PlanID": "OTHER_READ", + "PlanID": "OtherRead", "TableName": "" } # describe "describe a" { - "PlanID": "OTHER_READ", + "PlanID": "OtherRead", "TableName": "" } # explain "explain a" { - "PlanID": "OTHER_READ", + "PlanID": "OtherRead", "TableName": "" } # repair "repair a" { - "PlanID": "OTHER_ADMIN", + "PlanID": "OtherAdmin", "TableName": "" } # optimize "optimize a" { - "PlanID": "OTHER_ADMIN", + "PlanID": "OtherAdmin", "TableName": "" } -# table not found select -"select * from aaaa" -"table aaaa not found in schema" - -# table not found update -"update aaaa set a=1" -"table aaaa not found in schema" - -# table not found update -"delete from aaaa" -"table aaaa not found in schema" - -# table not found update -"insert into aaaa values(1)" -"table aaaa not found in schema" - -# column not found insert with subquery -"insert into a(missing) select * from b" -"column missing not found in table a" - # syntax error "syntax error" "syntax error at position 7 near 'syntax'" @@ -2354,18 +936,3 @@ options:PassthroughDMLs # named locks are unsafe with server-side connection pooling "select get_lock('foo') from dual" "get_lock() not allowed" - -# select DISTINCT ((1,2),(1,2)) from dual; -"select DISTINCT ((1,2),(1,2)) from dual" -{ - "PlanID": "PASS_SELECT", - "TableName": "dual", - "Permissions": [ - { - "TableName": "dual", - "Role": 0 - } - ], - "FieldQuery": "select ((1, 2), (1, 2)) from dual where 1 != 1", - "FullQuery": "select distinct ((1, 2), (1, 2)) from dual limit :#maxLimit" -} diff --git a/go/vt/vttablet/tabletserver/planbuilder/testdata/stream_cases.txt b/go/vt/vttablet/tabletserver/planbuilder/testdata/stream_cases.txt index fb32bfd3dc1..1a5cce0146c 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/testdata/stream_cases.txt +++ b/go/vt/vttablet/tabletserver/planbuilder/testdata/stream_cases.txt @@ -1,7 +1,7 @@ # select "select * from a" { - "PlanID": "SELECT_STREAM", + "PlanID": "SelectStream", "TableName": "a", "Permissions":[{"TableName":"a","Role":0}], "FullQuery": "select * from a" @@ -10,7 +10,7 @@ # select join "select * from a join b" { - "PlanID": "SELECT_STREAM", + "PlanID": "SelectStream", "TableName": "", "Permissions":[{"TableName":"a","Role":0},{"TableName":"b","Role":0}], "FullQuery": "select * from a join b" @@ -23,7 +23,7 @@ # union "select * from a union select * from b" { - "PlanID": "SELECT_STREAM", + "PlanID": "SelectStream", "TableName": "", "Permissions":[{"TableName":"a","Role":0},{"TableName":"b","Role":0}], "FullQuery": "select * from a union select * from b" @@ -32,7 +32,7 @@ # show "show tables" { - "PlanID": "SELECT_STREAM", + "PlanID": "SelectStream", "TableName": "", "FullQuery": "show tables" } @@ -40,7 +40,7 @@ # other "desc foo" { - "PlanID": "SELECT_STREAM", + "PlanID": "SelectStream", "TableName": "", "FullQuery": "otherread" } diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 69ba5062f55..2958648f628 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -120,7 +120,7 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { } switch qre.plan.PlanID { - case planbuilder.PlanPassSelect, planbuilder.PlanSelectImpossible: + case planbuilder.PlanSelect, planbuilder.PlanSelectImpossible: maxrows := qre.getSelectLimit() qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) qr, err := qre.execSelect() @@ -135,9 +135,9 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) case planbuilder.PlanSet, planbuilder.PlanOtherRead: return qre.execOther() - case planbuilder.PlanPassDML, planbuilder.PlanInsertMessage: + case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanInsertMessage: return qre.execAsTransaction(true /* autocommit */, qre.txConnExec) - case planbuilder.PlanDMLLimit: + case planbuilder.PlanUpdateLimit, planbuilder.PlanDeleteLimit: return qre.execAsTransaction(false /* autocommit */, qre.txConnExec) } return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) @@ -182,16 +182,16 @@ func (qre *QueryExecutor) execAsTransaction(autocommit bool, f func(conn *TxConn func (qre *QueryExecutor) txConnExec(conn *TxConnection) (*sqltypes.Result, error) { switch qre.plan.PlanID { - case planbuilder.PlanPassDML, planbuilder.PlanSet: + case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanSet: return qre.txFetch(conn, true) case planbuilder.PlanInsertMessage: qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano()) return qre.txFetch(conn, true) - case planbuilder.PlanDMLLimit: + case planbuilder.PlanUpdateLimit, planbuilder.PlanDeleteLimit: return qre.execDMLLimit(conn) case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: return qre.txFetch(conn, false) - case planbuilder.PlanPassSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: + case planbuilder.PlanSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: maxrows := qre.getSelectLimit() qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) qr, err := qre.txFetch(conn, false) diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index ba6ce759293..1563a7e4aff 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -80,7 +80,7 @@ func TestQueryExecutorPlanPassDmlRBR(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanPassDML, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -121,7 +121,7 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) - checkPlanID(t, planbuilder.PlanPassDML, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -172,7 +172,7 @@ func TestQueryExecutorPlanPassDmlAutoCommitRBR(t *testing.T) { defer tsv.StopService() qre := newTestQueryExecutor(ctx, tsv, query, 0) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanPassDML, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -205,7 +205,7 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { tsv.qe.binlogFormat = connpool.BinlogFormatRow qre := newTestQueryExecutor(ctx, tsv, query, 0) - checkPlanID(t, planbuilder.PlanPassDML, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -244,7 +244,7 @@ func TestQueryExecutorPlanPassDmlReplaceInto(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanPassDML, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanInsert, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -338,7 +338,7 @@ func TestQueryExecutorPlanPassSelectWithInATransaction(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, txid) defer tsv.StopService() defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -389,7 +389,7 @@ func TestQueryExecutorPlanPassSelect(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -441,7 +441,7 @@ func TestQueryExecutorPlanPassSelectSqlSelectLimit(t *testing.T) { SqlSelectLimit: 20, } defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -743,7 +743,7 @@ func TestQueryExecutorTableAcl(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("got: %v, want nil", err) @@ -787,7 +787,7 @@ func TestQueryExecutorTableAclNoPermission(t *testing.T) { // without enabling Config.StrictTableAcl tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("got: %v, want nil", err) @@ -801,7 +801,7 @@ func TestQueryExecutorTableAclNoPermission(t *testing.T) { tsv = newTestTabletServer(ctx, enableStrictTableACL, db) qre = newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) // query should fail because current user do not have read permissions _, err = qre.Execute() if err == nil { @@ -898,7 +898,7 @@ func TestQueryExecutorTableAclExemptACL(t *testing.T) { tsv := newTestTabletServer(ctx, enableStrictTableACL, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) // query should fail because current user do not have read permissions _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_PERMISSION_DENIED { @@ -965,7 +965,7 @@ func TestQueryExecutorTableAclDryRun(t *testing.T) { tableACLStatsKey := strings.Join([]string{ "test_table", "group02", - planbuilder.PlanPassSelect.String(), + planbuilder.PlanSelect.String(), username, }, ".") // enable Config.StrictTableAcl @@ -973,7 +973,7 @@ func TestQueryExecutorTableAclDryRun(t *testing.T) { tsv.qe.enableTableACLDryRun = true qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) beforeCount := tabletenv.TableaclPseudoDenied.Counts()[tableACLStatsKey] // query should fail because current user do not have read permissions _, err := qre.Execute() @@ -1008,7 +1008,7 @@ func TestQueryExecutorBlacklistQRFail(t *testing.T) { alterRule.SetIPCond(bannedAddr) alterRule.SetUserCond(bannedUser) alterRule.SetQueryCond("select.*") - alterRule.AddPlanCond(planbuilder.PlanPassSelect) + alterRule.AddPlanCond(planbuilder.PlanSelect) alterRule.AddTableCond("test_table") rulesName := "blacklistedRulesQRFail" @@ -1032,7 +1032,7 @@ func TestQueryExecutorBlacklistQRFail(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) // execute should fail because query has been blacklisted _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_INVALID_ARGUMENT { @@ -1062,7 +1062,7 @@ func TestQueryExecutorBlacklistQRRetry(t *testing.T) { alterRule.SetIPCond(bannedAddr) alterRule.SetUserCond(bannedUser) alterRule.SetQueryCond("select.*") - alterRule.AddPlanCond(planbuilder.PlanPassSelect) + alterRule.AddPlanCond(planbuilder.PlanSelect) alterRule.AddTableCond("test_table") rulesName := "blacklistedRulesQRRetry" @@ -1086,7 +1086,7 @@ func TestQueryExecutorBlacklistQRRetry(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanPassSelect, qre.plan.PlanID) + checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_FAILED_PRECONDITION { t.Fatalf("tsv.qe.queryRuleSources.SetRules: %v, want %v", code, vtrpcpb.Code_FAILED_PRECONDITION) diff --git a/go/vt/vttablet/tabletserver/querylogz_test.go b/go/vt/vttablet/tabletserver/querylogz_test.go index e829b23a144..793562864bd 100644 --- a/go/vt/vttablet/tabletserver/querylogz_test.go +++ b/go/vt/vttablet/tabletserver/querylogz_test.go @@ -47,7 +47,7 @@ func TestQuerylogzHandlerInvalidLogStats(t *testing.T) { func TestQuerylogzHandler(t *testing.T) { req, _ := http.NewRequest("GET", "/querylogz?timeout=10&limit=1", nil) logStats := tabletenv.NewLogStats(context.Background(), "Execute") - logStats.PlanType = planbuilder.PlanPassSelect.String() + logStats.PlanType = planbuilder.PlanSelect.String() logStats.OriginalSQL = "select name from test_table limit 1000" logStats.RowsAffected = 1000 logStats.NumberOfQueries = 1 diff --git a/go/vt/vttablet/tabletserver/queryz_test.go b/go/vt/vttablet/tabletserver/queryz_test.go index 78f5256a199..9d1f8fc44f3 100644 --- a/go/vt/vttablet/tabletserver/queryz_test.go +++ b/go/vt/vttablet/tabletserver/queryz_test.go @@ -40,7 +40,7 @@ func TestQueryzHandler(t *testing.T) { plan1 := &TabletPlan{ Plan: &planbuilder.Plan{ Table: &schema.Table{Name: sqlparser.NewTableIdent("test_table")}, - PlanID: planbuilder.PlanPassSelect, + PlanID: planbuilder.PlanSelect, }, } plan1.AddStats(10, 2*time.Second, 1*time.Second, 2, 0) diff --git a/go/vt/vttablet/tabletserver/rules/map_test.go b/go/vt/vttablet/tabletserver/rules/map_test.go index ffb533cf5b8..ffe80b1156b 100644 --- a/go/vt/vttablet/tabletserver/rules/map_test.go +++ b/go/vt/vttablet/tabletserver/rules/map_test.go @@ -174,7 +174,7 @@ func TestMapFilterByPlan(t *testing.T) { qri.SetRules(customQueryRules, otherRules) // Test filter by blacklist rule - qrs = qri.FilterByPlan("select * from bannedtable2", planbuilder.PlanPassSelect, "bannedtable2") + qrs = qri.FilterByPlan("select * from bannedtable2", planbuilder.PlanSelect, "bannedtable2") if l := len(qrs.rules); l != 1 { t.Errorf("Select from bannedtable matches %d rules, but we expect %d", l, 1) } @@ -183,7 +183,7 @@ func TestMapFilterByPlan(t *testing.T) { } // Test filter by custom rule - qrs = qri.FilterByPlan("select cid from t_customer limit 10", planbuilder.PlanPassSelect, "t_customer") + qrs = qri.FilterByPlan("select cid from t_customer limit 10", planbuilder.PlanSelect, "t_customer") if l := len(qrs.rules); l != 1 { t.Errorf("Select from t_customer matches %d rules, but we expect %d", l, 1) } @@ -197,7 +197,7 @@ func TestMapFilterByPlan(t *testing.T) { qr.AddBindVarCond("bindvar1", true, false, QRNoOp, nil) otherRules.Add(qr) qri.SetRules(customQueryRules, otherRules) - qrs = qri.FilterByPlan("select * from bannedtable2", planbuilder.PlanPassSelect, "bannedtable2") + qrs = qri.FilterByPlan("select * from bannedtable2", planbuilder.PlanSelect, "bannedtable2") if l := len(qrs.rules); l != 2 { t.Errorf("Insert into bannedtable2 matches %d rules: %v, but we expect %d rules to be matched", l, qrs.rules, 2) } diff --git a/go/vt/vttablet/tabletserver/rules/rules_test.go b/go/vt/vttablet/tabletserver/rules/rules_test.go index 309d93fb9fa..09377833e4a 100644 --- a/go/vt/vttablet/tabletserver/rules/rules_test.go +++ b/go/vt/vttablet/tabletserver/rules/rules_test.go @@ -81,7 +81,7 @@ func TestQueryRules(t *testing.T) { func TestCopy(t *testing.T) { qrs1 := New() qr1 := NewQueryRule("rule 1", "r1", QRFail) - qr1.AddPlanCond(planbuilder.PlanPassSelect) + qr1.AddPlanCond(planbuilder.PlanSelect) qr1.AddTableCond("aa") qr1.AddBindVarCond("a", true, false, QRNoOp, nil) @@ -107,11 +107,11 @@ func TestFilterByPlan(t *testing.T) { qr1 := NewQueryRule("rule 1", "r1", QRFail) qr1.SetIPCond("123") qr1.SetQueryCond("select") - qr1.AddPlanCond(planbuilder.PlanPassSelect) + qr1.AddPlanCond(planbuilder.PlanSelect) qr1.AddBindVarCond("a", true, false, QRNoOp, nil) qr2 := NewQueryRule("rule 2", "r2", QRFail) - qr2.AddPlanCond(planbuilder.PlanPassSelect) + qr2.AddPlanCond(planbuilder.PlanSelect) qr2.AddPlanCond(planbuilder.PlanSelectLock) qr2.AddBindVarCond("a", true, false, QRNoOp, nil) @@ -128,7 +128,7 @@ func TestFilterByPlan(t *testing.T) { qrs.Add(qr3) qrs.Add(qr4) - qrs1 := qrs.FilterByPlan("select", planbuilder.PlanPassSelect, "a") + qrs1 := qrs.FilterByPlan("select", planbuilder.PlanSelect, "a") want := compacted(`[{ "Description":"rule 1", "Name":"r1", @@ -163,7 +163,7 @@ func TestFilterByPlan(t *testing.T) { t.Errorf("qrs1:\n%s, want\n%s", got, want) } - qrs1 = qrs.FilterByPlan("insert", planbuilder.PlanPassSelect, "a") + qrs1 = qrs.FilterByPlan("insert", planbuilder.PlanSelect, "a") want = compacted(`[{ "Description":"rule 2", "Name":"r2", @@ -185,7 +185,7 @@ func TestFilterByPlan(t *testing.T) { t.Errorf("qrs1:\n%s, want\n%s", got, want) } - qrs1 = qrs.FilterByPlan("select", planbuilder.PlanPassDML, "a") + qrs1 = qrs.FilterByPlan("select", planbuilder.PlanInsert, "a") want = compacted(`[{ "Description":"rule 3", "Name":"r3", @@ -201,12 +201,12 @@ func TestFilterByPlan(t *testing.T) { t.Errorf("qrs1:\n%s, want\n%s", got, want) } - qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanPassDML, "a") + qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanInsert, "a") if qrs1.rules != nil { t.Errorf("want nil, got non-nil") } - qrs1 = qrs.FilterByPlan("table", planbuilder.PlanPassDML, "b") + qrs1 = qrs.FilterByPlan("table", planbuilder.PlanInsert, "b") want = compacted(`[{ "Description":"rule 4", "Name":"r4", @@ -220,7 +220,7 @@ func TestFilterByPlan(t *testing.T) { qr5 := NewQueryRule("rule 5", "r5", QRFail) qrs.Add(qr5) - qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanPassDML, "a") + qrs1 = qrs.FilterByPlan("sel", planbuilder.PlanInsert, "a") want = compacted(`[{ "Description":"rule 5", "Name":"r5", @@ -232,7 +232,7 @@ func TestFilterByPlan(t *testing.T) { } qrsnil1 := New() - if qrsnil2 := qrsnil1.FilterByPlan("", planbuilder.PlanPassSelect, "a"); qrsnil2.rules != nil { + if qrsnil2 := qrsnil1.FilterByPlan("", planbuilder.PlanSelect, "a"); qrsnil2.rules != nil { t.Errorf("want nil, got non-nil") } } @@ -257,13 +257,13 @@ func TestQueryRule(t *testing.T) { t.Errorf("want error") } - qr.AddPlanCond(planbuilder.PlanPassSelect) - qr.AddPlanCond(planbuilder.PlanPassDML) + qr.AddPlanCond(planbuilder.PlanSelect) + qr.AddPlanCond(planbuilder.PlanInsert) - if qr.plans[0] != planbuilder.PlanPassSelect { + if qr.plans[0] != planbuilder.PlanSelect { t.Errorf("want PASS_SELECT, got %s", qr.plans[0].String()) } - if qr.plans[1] != planbuilder.PlanPassDML { + if qr.plans[1] != planbuilder.PlanInsert { t.Errorf("want INSERT_PK, got %s", qr.plans[1].String()) } diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 27f5db98624..a9a753d17eb 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1187,7 +1187,9 @@ func (tsv *TabletServer) computeTxSerializerKey(ctx context.Context, logStats *t return "", "" } - if plan.PlanID != planbuilder.PlanPassDML && plan.PlanID != planbuilder.PlanDMLLimit { + switch plan.PlanID { + case planbuilder.PlanUpdate, planbuilder.PlanUpdateLimit, + planbuilder.PlanDelete, planbuilder.PlanDeleteLimit: // Serialize only UPDATE or DELETE queries. return "", "" } From c6af1148c280c53281138178a07b8c2c4c72d393 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Mon, 16 Mar 2020 13:25:55 -0700 Subject: [PATCH 10/18] deprecation: improve DDL pass-through Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/planbuilder/ddl.go | 54 ------------------- .../vttablet/tabletserver/planbuilder/plan.go | 2 +- .../tabletserver/planbuilder/plan_test.go | 12 ----- .../planbuilder/testdata/exec_cases.txt | 27 ---------- go/vt/vttablet/tabletserver/query_executor.go | 44 +++------------ go/vt/vttablet/tabletserver/tx_pool.go | 3 ++ 6 files changed, 11 insertions(+), 131 deletions(-) delete mode 100644 go/vt/vttablet/tabletserver/planbuilder/ddl.go diff --git a/go/vt/vttablet/tabletserver/planbuilder/ddl.go b/go/vt/vttablet/tabletserver/planbuilder/ddl.go deleted file mode 100644 index 68b0449db19..00000000000 --- a/go/vt/vttablet/tabletserver/planbuilder/ddl.go +++ /dev/null @@ -1,54 +0,0 @@ -/* -Copyright 2019 The Vitess Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package planbuilder - -import ( - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" -) - -// DDLPlan provides a plan for DDLs. -type DDLPlan struct { - Action string -} - -// DDLParse parses a DDL and produces a DDLPlan. -func DDLParse(sql string) (plan *DDLPlan) { - statement, err := sqlparser.Parse(sql) - if err != nil { - return &DDLPlan{Action: ""} - } - stmt, ok := statement.(*sqlparser.DDL) - if !ok { - return &DDLPlan{Action: ""} - } - return &DDLPlan{ - Action: stmt.Action, - } -} - -func analyzeDDL(ddl *sqlparser.DDL, tables map[string]*schema.Table) *Plan { - // TODO(sougou): Add support for sequences. - plan := &Plan{ - PlanID: PlanDDL, - } - // this can become a whitelist of fully supported ddl actions as support grows - if ddl.PartitionSpec != nil { - plan.FullQuery = GenerateFullQuery(ddl) - } - return plan -} diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index deabf41f49a..a9b662a79c8 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -170,7 +170,7 @@ func Build(statement sqlparser.Statement, tables map[string]*schema.Table) (*Pla case *sqlparser.Set: plan, err = analyzeSet(stmt), nil case *sqlparser.DDL: - plan, err = analyzeDDL(stmt, tables), nil + plan = &Plan{PlanID: PlanDDL} case *sqlparser.Show: plan, err = &Plan{PlanID: PlanOtherRead}, nil case *sqlparser.OtherRead: diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan_test.go b/go/vt/vttablet/tabletserver/planbuilder/plan_test.go index 1026e6c99e8..7b4c7adea91 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan_test.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan_test.go @@ -161,18 +161,6 @@ func TestStreamPlan(t *testing.T) { } } -func TestDDLPlan(t *testing.T) { - for tcase := range iterateExecFile("ddl_cases.txt") { - plan := DDLParse(tcase.input) - expected := make(map[string]interface{}) - err := json.Unmarshal([]byte(tcase.output), &expected) - if err != nil { - t.Fatalf("Error marshalling %v", plan) - } - matchString(t, tcase.lineno, expected["Action"], plan.Action) - } -} - func TestMessageStreamingPlan(t *testing.T) { testSchema := loadSchema("schema_test.json") plan, err := BuildMessageStreaming("msg", testSchema) diff --git a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt index 3397ff6ee84..32a47b833f9 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt +++ b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt @@ -867,33 +867,6 @@ options:PassthroughDMLs ] } -# reorganize partition with bind -"alter table a reorganize partition b into (partition c values less than (:bv), partition d values less than (maxvalue))" -{ - "PlanID": "DDL", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 2 - } - ], - "FullQuery": "alter table a reorganize partition b into (partition c values less than (:bv), partition d values less than (maxvalue))" -} - -# partition -"alter table a partition by range (id) (partition p0 values less than (10), partition p1 values less than (maxvalue))" -{ - "PlanID": "DDL", - "TableName": "", - "Permissions": [ - { - "TableName": "a", - "Role": 2 - } - ] -} - # show "show a" { diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 2958648f628..66d3451f242 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -97,8 +97,6 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { } switch qre.plan.PlanID { - case planbuilder.PlanDDL: - return qre.execDDL() case planbuilder.PlanNextval: return qre.execNextval() case planbuilder.PlanSelectImpossible: @@ -202,6 +200,8 @@ func (qre *QueryExecutor) txConnExec(conn *TxConnection) (*sqltypes.Result, erro return nil, err } return qr, nil + case planbuilder.PlanDDL: + return qre.execDDL(conn) } return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) } @@ -352,20 +352,7 @@ func (qre *QueryExecutor) checkAccess(authorized *tableacl.ACLResult, tableName return nil } -func (qre *QueryExecutor) execDDL() (*sqltypes.Result, error) { - sql := qre.query - var err error - if qre.plan.FullQuery != nil { - sql, _, err = qre.generateFinalSQL(qre.plan.FullQuery, qre.bindVars) - if err != nil { - return nil, err - } - } - ddlPlan := planbuilder.DDLParse(sql) - if ddlPlan.Action == "" { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "DDL is not understood") - } - +func (qre *QueryExecutor) execDDL(conn *TxConnection) (*sqltypes.Result, error) { defer func() { err := qre.tsv.se.Reload(qre.ctx) if err != nil { @@ -373,31 +360,14 @@ func (qre *QueryExecutor) execDDL() (*sqltypes.Result, error) { } }() - if qre.transactionID != 0 { - conn, err := qre.tsv.te.txPool.Get(qre.transactionID, "DDL begin again") - if err != nil { - return nil, err - } - defer conn.Recycle() - result, err := qre.execSQL(conn, sql, true) - if err != nil { - return nil, err - } - err = conn.BeginAgain(qre.ctx) - if err != nil { - return nil, err - } - return result, nil + result, err := qre.execSQL(conn, qre.query, true) + if err != nil { + return nil, err } - - result, err := qre.execAsTransaction(true /* autocommit */, func(conn *TxConnection) (*sqltypes.Result, error) { - return qre.execSQL(conn, sql, true) - }) - + err = conn.BeginAgain(qre.ctx) if err != nil { return nil, err } - return result, nil } diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go index 1bb5723bc9d..4d597e0108c 100644 --- a/go/vt/vttablet/tabletserver/tx_pool.go +++ b/go/vt/vttablet/tabletserver/tx_pool.go @@ -458,6 +458,9 @@ func (txc *TxConnection) Exec(ctx context.Context, query string, maxrows int, wa // BeginAgain commits the existing transaction and begins a new one func (txc *TxConnection) BeginAgain(ctx context.Context) error { + if txc.Autocommit { + return nil + } if _, err := txc.DBConn.Exec(ctx, "commit", 1, false); err != nil { return err } From a4e107d0a59d59b4e690484550c72f787c9de84e Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 17 Mar 2020 17:53:50 -0700 Subject: [PATCH 11/18] deprecation: tabletserver tests WIP Signed-off-by: Sugu Sougoumarane --- go/mysql/fakesqldb/server.go | 7 - go/vt/vttablet/tabletserver/query_engine.go | 4 - go/vt/vttablet/tabletserver/query_executor.go | 6 +- .../tabletserver/query_executor_test.go | 244 ++++++++++++------ .../tabletserver/schema/load_table.go | 15 -- .../tabletserver/schema/load_table_test.go | 2 - go/vt/vttablet/tabletserver/schema/schema.go | 6 - .../vttablet/tabletserver/tabletenv/config.go | 3 +- go/vt/vttablet/tabletserver/tabletserver.go | 11 +- .../tabletserver/tabletserver_flaky_test.go | 1 - 10 files changed, 175 insertions(+), 124 deletions(-) diff --git a/go/mysql/fakesqldb/server.go b/go/mysql/fakesqldb/server.go index dd29a495a0e..f7c4f904742 100644 --- a/go/mysql/fakesqldb/server.go +++ b/go/mysql/fakesqldb/server.go @@ -287,10 +287,6 @@ func (db *DB) NewConnection(c *mysql.Conn) { db.mu.Lock() defer db.mu.Unlock() - if db.t != nil { - db.t.Logf("NewConnection(%v): client %v", db.name, c.ConnectionID) - } - if db.isConnFail { panic(fmt.Errorf("simulating a connection failure")) } @@ -336,9 +332,6 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R return callback(&sqltypes.Result{}) } - if db.t != nil { - db.t.Logf("ComQuery(%v): client %v: %v", db.name, c.ConnectionID, query) - } if db.orderMatters { result, err := db.comQueryOrdered(query) if err != nil { diff --git a/go/vt/vttablet/tabletserver/query_engine.go b/go/vt/vttablet/tabletserver/query_engine.go index 7329621f32f..b996cdbf572 100644 --- a/go/vt/vttablet/tabletserver/query_engine.go +++ b/go/vt/vttablet/tabletserver/query_engine.go @@ -153,8 +153,6 @@ type QueryEngine struct { maxResultSize sync2.AtomicInt64 warnResultSize sync2.AtomicInt64 maxDMLRows sync2.AtomicInt64 - passthroughDMLs sync2.AtomicBool - allowUnsafeDMLs bool streamBufferSize sync2.AtomicInt64 // tableaclExemptCount count the number of accesses allowed // based on membership in the superuser ACL @@ -239,8 +237,6 @@ func NewQueryEngine(checker connpool.MySQLChecker, se *schema.Engine, config tab qe.maxDMLRows = sync2.NewAtomicInt64(int64(config.MaxDMLRows)) qe.streamBufferSize = sync2.NewAtomicInt64(int64(config.StreamBufferSize)) - qe.passthroughDMLs = sync2.NewAtomicBool(config.PassthroughDMLs) - qe.allowUnsafeDMLs = config.AllowUnsafeDMLs planbuilder.PassthroughDMLs = config.PassthroughDMLs qe.accessCheckerLogger = logutil.NewThrottledLogger("accessChecker", 1*time.Second) diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 66d3451f242..ee62b338bce 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -131,9 +131,9 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { return qr, nil case planbuilder.PlanSelectLock: return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s disallowed outside transaction", qre.plan.PlanID.String()) - case planbuilder.PlanSet, planbuilder.PlanOtherRead: + case planbuilder.PlanSet, planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: return qre.execOther() - case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanInsertMessage: + case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanInsertMessage, planbuilder.PlanDDL: return qre.execAsTransaction(true /* autocommit */, qre.txConnExec) case planbuilder.PlanUpdateLimit, planbuilder.PlanDeleteLimit: return qre.execAsTransaction(false /* autocommit */, qre.txConnExec) @@ -498,7 +498,7 @@ func (qre *QueryExecutor) execOther() (*sqltypes.Result, error) { return nil, err } defer conn.Recycle() - return qre.dbConnFetch(conn, qre.plan.FullQuery, qre.bindVars) + return qre.execSQL(conn, qre.query, true) } func (qre *QueryExecutor) getConn() (*connpool.DBConn, error) { diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 1563a7e4aff..312e70b0627 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -24,6 +24,8 @@ import ( "strings" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "golang.org/x/net/context" "vitess.io/vitess/go/mysql" @@ -47,23 +49,150 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -func TestQueryExecutorPlanDDL(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "alter table test_table add zipcode int" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - checkPlanID(t, planbuilder.PlanDDL, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) +func TestQueryExecutorPlans(t *testing.T) { + type dbResponse struct { + query string + result *sqltypes.Result } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) + + dmlResult := &sqltypes.Result{ + RowsAffected: 1, + } + fields := sqltypes.MakeTestFields("a|b", "int64|varchar") + fieldResult := sqltypes.MakeTestResult(fields) + selectResult := sqltypes.MakeTestResult(fields, "1|aaa") + + testcases := []struct { + input string + passThrough bool + dbResponses []dbResponse + resultWant *sqltypes.Result + planWant string + logWant string + }{{ + input: "select * from t where 1 != 1", + dbResponses: []dbResponse{{ + query: "select * from t where 1 != 1", + result: fieldResult, + }}, + resultWant: fieldResult, + planWant: "SelectImpossible", + logWant: "select * from t where 1 != 1", + }, { + input: "select * from t", + dbResponses: []dbResponse{{ + query: "select * from t where 1 != 1", + result: fieldResult, + }, { + query: "select * from t limit 10001", + result: selectResult, + }}, + resultWant: selectResult, + planWant: "Select", + logWant: "select * from t where 1 != 1; select * from t limit 10001", + }, { + input: "set a=1", + dbResponses: []dbResponse{{ + query: "set a=1", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "Set", + logWant: "set a=1", + }, { + input: "show engines", + dbResponses: []dbResponse{{ + query: "show engines", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "OtherRead", + logWant: "show engines", + }, { + input: "repair t", + dbResponses: []dbResponse{{ + query: "repair t", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "OtherAdmin", + logWant: "repair t", + }, { + input: "insert into test_table(a) values(1)", + dbResponses: []dbResponse{{ + query: "insert into test_table(a) values (1)", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "Insert", + logWant: "insert into test_table(a) values (1)", + }, { + input: "update test_table set a=1", + dbResponses: []dbResponse{{ + query: "update test_table set a = 1 limit 10001", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "UpdateLimit", + logWant: "begin; update test_table set a = 1 limit 10001; commit", + }, { + input: "update test_table set a=1", + passThrough: true, + dbResponses: []dbResponse{{ + query: "update test_table set a = 1", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "Update", + logWant: "update test_table set a = 1", + }, { + input: "delete from test_table", + dbResponses: []dbResponse{{ + query: "delete from test_table limit 10001", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "DeleteLimit", + logWant: "begin; delete from test_table limit 10001; commit", + }, { + input: "delete from test_table", + passThrough: true, + dbResponses: []dbResponse{{ + query: "delete from test_table", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "Delete", + logWant: "delete from test_table", + }, { + input: "alter table test_table add zipcode int", + dbResponses: []dbResponse{{ + query: "alter table test_table add zipcode int", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "DDL", + logWant: "alter table test_table add zipcode int", + }} + for _, tcase := range testcases { + func() { + db := setUpQueryExecutorTest(t) + defer db.Close() + for _, dbr := range tcase.dbResponses { + db.AddQuery(dbr.query, dbr.result) + } + ctx := context.Background() + tsv := newTestTabletServer(ctx, noFlags, db) + defer tsv.StopService() + + tsv.SetPassthroughDMLs(tcase.passThrough) + qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0) + got, err := qre.Execute() + require.NoError(t, err, tcase.input) + assert.Equal(t, tcase.resultWant, got, tcase.input) + assert.Equal(t, tcase.planWant, qre.logStats.PlanType, tcase.input) + assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input) + }() } } @@ -80,7 +209,7 @@ func TestQueryExecutorPlanPassDmlRBR(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -121,7 +250,7 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) - checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -141,23 +270,6 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - - tsv.SetAllowUnsafeDMLs(true) - got, err = qre.Execute() - - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries = []string{query, query} - gotqueries = fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - testCommitHelper(t, tsv, qre) } func TestQueryExecutorPlanPassDmlAutoCommitRBR(t *testing.T) { @@ -172,7 +284,7 @@ func TestQueryExecutorPlanPassDmlAutoCommitRBR(t *testing.T) { defer tsv.StopService() qre := newTestQueryExecutor(ctx, tsv, query, 0) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -205,7 +317,7 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { tsv.qe.binlogFormat = connpool.BinlogFormatRow qre := newTestQueryExecutor(ctx, tsv, query, 0) - checkPlanID(t, planbuilder.PlanUpdate, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -220,15 +332,6 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - - tsv.SetAllowUnsafeDMLs(true) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } } func TestQueryExecutorPlanPassDmlReplaceInto(t *testing.T) { @@ -244,7 +347,7 @@ func TestQueryExecutorPlanPassDmlReplaceInto(t *testing.T) { txid := newTransaction(tsv, nil) qre := newTestQueryExecutor(ctx, tsv, query, txid) tsv.qe.binlogFormat = connpool.BinlogFormatRow - checkPlanID(t, planbuilder.PlanInsert, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanInsert, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -277,7 +380,7 @@ func TestQueryExecutorPlanInsertMessage(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanInsertMessage, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanInsertMessage, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -301,7 +404,7 @@ func TestQueryExecutorPlanOtherWithinATransaction(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, txid) defer tsv.StopService() defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanOtherRead, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanOtherRead, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -338,7 +441,7 @@ func TestQueryExecutorPlanPassSelectWithInATransaction(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, txid) defer tsv.StopService() defer testCommitHelper(t, tsv, qre) - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -367,7 +470,7 @@ func TestQueryExecutorPlanPassSelectWithLockOutsideATransaction(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelectLock, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelectLock, qre.plan.PlanID) _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_FAILED_PRECONDITION { t.Fatalf("qre.Execute: %v, want %v", code, vtrpcpb.Code_FAILED_PRECONDITION) @@ -389,7 +492,7 @@ func TestQueryExecutorPlanPassSelect(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -414,7 +517,7 @@ func TestQueryExecutorPlanSelectImpossible(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelectImpossible, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelectImpossible, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -441,7 +544,7 @@ func TestQueryExecutorPlanPassSelectSqlSelectLimit(t *testing.T) { SqlSelectLimit: 20, } defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -460,7 +563,7 @@ func TestQueryExecutorPlanSet(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() qre := newTestQueryExecutor(ctx, tsv, setQuery, 0) - checkPlanID(t, planbuilder.PlanSet, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSet, qre.plan.PlanID) // Query will be delegated to MySQL and both Fields and Rows should be // empty arrays in this case. want := &sqltypes.Result{} @@ -503,7 +606,7 @@ func TestQueryExecutorPlanOther(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanOtherRead, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanOtherRead, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("got: %v, want nil", err) @@ -534,7 +637,7 @@ func TestQueryExecutorPlanNextval(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() qre := newTestQueryExecutor(ctx, tsv, "select next value from seq", 0) - checkPlanID(t, planbuilder.PlanNextval, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanNextval, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) @@ -743,7 +846,7 @@ func TestQueryExecutorTableAcl(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("got: %v, want nil", err) @@ -787,7 +890,7 @@ func TestQueryExecutorTableAclNoPermission(t *testing.T) { // without enabling Config.StrictTableAcl tsv := newTestTabletServer(ctx, noFlags, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) got, err := qre.Execute() if err != nil { t.Fatalf("got: %v, want nil", err) @@ -801,7 +904,7 @@ func TestQueryExecutorTableAclNoPermission(t *testing.T) { tsv = newTestTabletServer(ctx, enableStrictTableACL, db) qre = newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) // query should fail because current user do not have read permissions _, err = qre.Execute() if err == nil { @@ -838,7 +941,7 @@ func TestQueryExecutorTableAclDualTableExempt(t *testing.T) { query := "select * from test_table where 1 != 1" qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelectImpossible, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelectImpossible, qre.plan.PlanID) // query should fail because nobody has read access to test_table _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_PERMISSION_DENIED { @@ -898,7 +1001,7 @@ func TestQueryExecutorTableAclExemptACL(t *testing.T) { tsv := newTestTabletServer(ctx, enableStrictTableACL, db) qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) // query should fail because current user do not have read permissions _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_PERMISSION_DENIED { @@ -973,7 +1076,7 @@ func TestQueryExecutorTableAclDryRun(t *testing.T) { tsv.qe.enableTableACLDryRun = true qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) beforeCount := tabletenv.TableaclPseudoDenied.Counts()[tableACLStatsKey] // query should fail because current user do not have read permissions _, err := qre.Execute() @@ -1032,7 +1135,7 @@ func TestQueryExecutorBlacklistQRFail(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) // execute should fail because query has been blacklisted _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_INVALID_ARGUMENT { @@ -1086,7 +1189,7 @@ func TestQueryExecutorBlacklistQRRetry(t *testing.T) { qre := newTestQueryExecutor(ctx, tsv, query, 0) defer tsv.StopService() - checkPlanID(t, planbuilder.PlanSelect, qre.plan.PlanID) + assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) _, err := qre.Execute() if code := vterrors.Code(err); code != vtrpcpb.Code_FAILED_PRECONDITION { t.Fatalf("tsv.qe.queryRuleSources.SetRules: %v, want %v", code, vtrpcpb.Code_FAILED_PRECONDITION) @@ -1114,7 +1217,6 @@ func newTestTabletServer(ctx context.Context, flags executorFlags, db *fakesqldb } else { config.TransactionCap = 100 } - config.EnableAutoCommit = true if flags&enableStrictTableACL > 0 { config.StrictTableACL = true } else { @@ -1205,16 +1307,6 @@ func getTestTableFields() []*querypb.Field { } } -func checkPlanID( - t *testing.T, - expectedPlanID planbuilder.PlanType, - actualPlanID planbuilder.PlanType) { - if expectedPlanID != actualPlanID { - t.Fatalf("expect to get PlanID: %s, but got %s", - expectedPlanID.String(), actualPlanID.String()) - } -} - func getQueryExecutorSupportedQueries(testTableHasMultipleUniqueKeys bool) map[string]*sqltypes.Result { return map[string]*sqltypes.Result{ // queries for twopc diff --git a/go/vt/vttablet/tabletserver/schema/load_table.go b/go/vt/vttablet/tabletserver/schema/load_table.go index b530dcb09da..d8a4a0a7353 100644 --- a/go/vt/vttablet/tabletserver/schema/load_table.go +++ b/go/vt/vttablet/tabletserver/schema/load_table.go @@ -198,21 +198,6 @@ func loadMessageInfo(ta *Table, comment string) error { } } - // Store the position of the id column in the PK - // list. This is required to handle arbitrary updates. - // In such cases, we have to be able to identify the - // affected id and invalidate the message cache. - ta.MessageInfo.IDPKIndex = -1 - for i, j := range ta.PKColumns { - if ta.Columns[j].Name.EqualString("id") { - ta.MessageInfo.IDPKIndex = i - break - } - } - if ta.MessageInfo.IDPKIndex == -1 { - return fmt.Errorf("id column is not part of the primary key for message table: %s", ta.Name.String()) - } - // Load user-defined columns. Any "unrecognized" column is user-defined. for _, c := range ta.Columns { if _, ok := findCols[c.Name.Lowered()]; ok { diff --git a/go/vt/vttablet/tabletserver/schema/load_table_test.go b/go/vt/vttablet/tabletserver/schema/load_table_test.go index d29f1c3b148..5b9fb6ecda9 100644 --- a/go/vt/vttablet/tabletserver/schema/load_table_test.go +++ b/go/vt/vttablet/tabletserver/schema/load_table_test.go @@ -138,7 +138,6 @@ func TestLoadTableMessage(t *testing.T) { Name: sqlparser.NewTableIdent("test_table"), Type: Message, MessageInfo: &MessageInfo{ - IDPKIndex: 1, Fields: []*querypb.Field{{ Name: "id", Type: sqltypes.Int64, @@ -214,7 +213,6 @@ func TestLoadTableMessageTopic(t *testing.T) { Name: sqlparser.NewTableIdent("test_table"), Type: Message, MessageInfo: &MessageInfo{ - IDPKIndex: 1, Fields: []*querypb.Field{{ Name: "id", Type: sqltypes.Int64, diff --git a/go/vt/vttablet/tabletserver/schema/schema.go b/go/vt/vttablet/tabletserver/schema/schema.go index c943420d5a8..b6e7b7bbc02 100644 --- a/go/vt/vttablet/tabletserver/schema/schema.go +++ b/go/vt/vttablet/tabletserver/schema/schema.go @@ -99,12 +99,6 @@ type TopicInfo struct { // MessageInfo contains info specific to message tables. type MessageInfo struct { - // IDPKIndex is the index of the ID column - // in PKvalues. This is used to extract the ID - // value for message tables to discard items - // from the cache. - IDPKIndex int - // Fields stores the field info to be // returned for subscribers. Fields []*querypb.Field diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go index 1034c7a8ccd..b6894f1f438 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/config.go +++ b/go/vt/vttablet/tabletserver/tabletenv/config.go @@ -46,6 +46,7 @@ var ( // TODO(sougou): deprecate the flag after release 7.0. deprecatedMessagePoolPrefillParallelism int deprecatedAutocommit bool + deprecateAllowUnsafeDMLs bool ) func init() { @@ -65,7 +66,7 @@ func init() { flag.IntVar(&Config.WarnResultSize, "queryserver-config-warn-result-size", DefaultQsConfig.WarnResultSize, "query server result size warning threshold, warn if number of rows returned from vttablet for non-streaming queries exceeds this") flag.IntVar(&Config.MaxDMLRows, "queryserver-config-max-dml-rows", DefaultQsConfig.MaxDMLRows, "query server max dml rows per statement, maximum number of rows allowed to return at a time for an update or delete with either 1) an equality where clauses on primary keys, or 2) a subselect statement. For update and delete statements in above two categories, vttablet will split the original query into multiple small queries based on this configuration value. ") flag.BoolVar(&Config.PassthroughDMLs, "queryserver-config-passthrough-dmls", DefaultQsConfig.PassthroughDMLs, "query server pass through all dml statements without rewriting") - flag.BoolVar(&Config.AllowUnsafeDMLs, "queryserver-config-allowunsafe-dmls", DefaultQsConfig.AllowUnsafeDMLs, "query server allow unsafe dml statements") + flag.BoolVar(&deprecateAllowUnsafeDMLs, "queryserver-config-allowunsafe-dmls", false, "deprecated") flag.IntVar(&Config.StreamBufferSize, "queryserver-config-stream-buffer-size", DefaultQsConfig.StreamBufferSize, "query server stream buffer size, the maximum number of bytes sent from vttablet for each stream call. It's recommended to keep this value in sync with vtgate's stream_buffer_size.") flag.IntVar(&Config.QueryPlanCacheSize, "queryserver-config-query-cache-size", DefaultQsConfig.QueryPlanCacheSize, "query server query cache size, maximum number of queries to be cached. vttablet analyzes every incoming query and generate a query plan, these plans are being cached in a lru cache. This config controls the capacity of the lru cache.") diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index a9a753d17eb..4ddbb1f232c 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1078,7 +1078,7 @@ func (tsv *TabletServer) ExecuteBatch(ctx context.Context, target *querypb.Targe // pool without actually begin/commit the transaction. if (options.TransactionIsolation == querypb.ExecuteOptions_DEFAULT) && asTransaction && - tsv.qe.passthroughDMLs.Get() { + planbuilder.PassthroughDMLs { options.TransactionIsolation = querypb.ExecuteOptions_AUTOCOMMIT } @@ -1922,14 +1922,7 @@ func (tsv *TabletServer) MaxDMLRows() int { // SetPassthroughDMLs changes the setting to pass through all DMLs // It should only be used for testing func (tsv *TabletServer) SetPassthroughDMLs(val bool) { - planbuilder.PassthroughDMLs = true - tsv.qe.passthroughDMLs.Set(val) -} - -// SetAllowUnsafeDMLs changes the setting to allow unsafe DML statements -// in SBR mode. It should be used only on initialization or for testing. -func (tsv *TabletServer) SetAllowUnsafeDMLs(val bool) { - tsv.qe.allowUnsafeDMLs = val + planbuilder.PassthroughDMLs = val } // SetQueryPoolTimeout changes the timeout to get a connection from the diff --git a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go index 7730c747b8a..2a38e4acb05 100644 --- a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go +++ b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go @@ -1283,7 +1283,6 @@ func TestTabletServerExecuteBatchSqlSucceedInTransaction(t *testing.T) { db.AddRejectedQuery(sql, errRejected) config := testUtils.newQueryServiceConfig() - config.EnableAutoCommit = true tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) dbcfgs := testUtils.newDBConfigs(db) target := querypb.Target{TabletType: topodatapb.TabletType_MASTER} From e5c87fc7ba66ce5859c051296349fdd0da2be955 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 17 Mar 2020 23:19:38 -0700 Subject: [PATCH 12/18] deprecation: tabletserver unit tests pass Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/query_executor.go | 6 +- .../tabletserver/query_executor_test.go | 500 ++++-------------- go/vt/vttablet/tabletserver/querylogz_test.go | 7 +- go/vt/vttablet/tabletserver/queryz_test.go | 10 +- go/vt/vttablet/tabletserver/tabletserver.go | 3 +- .../tabletserver/tabletserver_flaky_test.go | 86 +-- .../vttablet/tabletserver/tx_executor_test.go | 4 +- 7 files changed, 157 insertions(+), 459 deletions(-) diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index ee62b338bce..4a1818509f7 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -180,15 +180,15 @@ func (qre *QueryExecutor) execAsTransaction(autocommit bool, f func(conn *TxConn func (qre *QueryExecutor) txConnExec(conn *TxConnection) (*sqltypes.Result, error) { switch qre.plan.PlanID { - case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanSet: + case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete: return qre.txFetch(conn, true) case planbuilder.PlanInsertMessage: qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano()) return qre.txFetch(conn, true) case planbuilder.PlanUpdateLimit, planbuilder.PlanDeleteLimit: return qre.execDMLLimit(conn) - case planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: - return qre.txFetch(conn, false) + case planbuilder.PlanSet, planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: + return qre.execSQL(conn, qre.query, true) case planbuilder.PlanSelect, planbuilder.PlanSelectLock, planbuilder.PlanSelectImpossible: maxrows := qre.getSelectLimit() qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1) diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 312e70b0627..3091078cf92 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -38,7 +38,6 @@ import ( "vitess.io/vitess/go/vt/tableacl/simpleacl" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool" "vitess.io/vitess/go/vt/vttablet/tabletserver/planbuilder" "vitess.io/vitess/go/vt/vttablet/tabletserver/rules" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" @@ -62,34 +61,53 @@ func TestQueryExecutorPlans(t *testing.T) { fieldResult := sqltypes.MakeTestResult(fields) selectResult := sqltypes.MakeTestResult(fields, "1|aaa") + // The queries are run both in and outside a transaction. testcases := []struct { - input string + // input is the input query. + input string + // passThrough specifies if planbuilder.PassthroughDML must be set. passThrough bool + // dbResponses specifes the list of queries and responses to add to the fake db. dbResponses []dbResponse - resultWant *sqltypes.Result - planWant string - logWant string + // resultWant is the result we want. + resultWant *sqltypes.Result + // planWant is the PlanType we want to see built. + planWant string + // logWant is the log of queries we expect to be executed. + logWant string + // inTxWant is the query log we expect if we're in a transation. + // If empty, then we should expect the same as logWant. + inTxWant string }{{ - input: "select * from t where 1 != 1", + input: "select * from t", dbResponses: []dbResponse{{ query: "select * from t where 1 != 1", result: fieldResult, + }, { + query: "select * from t limit 10001", + result: selectResult, }}, - resultWant: fieldResult, - planWant: "SelectImpossible", - logWant: "select * from t where 1 != 1", + resultWant: selectResult, + planWant: "Select", + logWant: "select * from t where 1 != 1; select * from t limit 10001", + // Because the fields would have been cached before, the field query will + // not get re-executed. + inTxWant: "select * from t limit 10001", }, { - input: "select * from t", + input: "select * from t limit 1", dbResponses: []dbResponse{{ query: "select * from t where 1 != 1", result: fieldResult, }, { - query: "select * from t limit 10001", + query: "select * from t limit 1", result: selectResult, }}, resultWant: selectResult, planWant: "Select", - logWant: "select * from t where 1 != 1; select * from t limit 10001", + logWant: "select * from t where 1 != 1; select * from t limit 1", + // Because the fields would have been cached before, the field query will + // not get re-executed. + inTxWant: "select * from t limit 1", }, { input: "set a=1", dbResponses: []dbResponse{{ @@ -126,6 +144,15 @@ func TestQueryExecutorPlans(t *testing.T) { resultWant: dmlResult, planWant: "Insert", logWant: "insert into test_table(a) values (1)", + }, { + input: "replace into test_table(a) values(1)", + dbResponses: []dbResponse{{ + query: "replace into test_table(a) values (1)", + result: dmlResult, + }}, + resultWant: dmlResult, + planWant: "Insert", + logWant: "replace into test_table(a) values (1)", }, { input: "update test_table set a=1", dbResponses: []dbResponse{{ @@ -134,7 +161,10 @@ func TestQueryExecutorPlans(t *testing.T) { }}, resultWant: dmlResult, planWant: "UpdateLimit", - logWant: "begin; update test_table set a = 1 limit 10001; commit", + // The UpdateLimit query will not use autocommit because + // it needs to roll back on failure. + logWant: "begin; update test_table set a = 1 limit 10001; commit", + inTxWant: "update test_table set a = 1 limit 10001", }, { input: "update test_table set a=1", passThrough: true, @@ -153,7 +183,10 @@ func TestQueryExecutorPlans(t *testing.T) { }}, resultWant: dmlResult, planWant: "DeleteLimit", - logWant: "begin; delete from test_table limit 10001; commit", + // The DeleteLimit query will not use autocommit because + // it needs to roll back on failure. + logWant: "begin; delete from test_table limit 10001; commit", + inTxWant: "delete from test_table limit 10001", }, { input: "delete from test_table", passThrough: true, @@ -186,188 +219,92 @@ func TestQueryExecutorPlans(t *testing.T) { defer tsv.StopService() tsv.SetPassthroughDMLs(tcase.passThrough) + + // Test outside a transaction. qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0) got, err := qre.Execute() require.NoError(t, err, tcase.input) assert.Equal(t, tcase.resultWant, got, tcase.input) assert.Equal(t, tcase.planWant, qre.logStats.PlanType, tcase.input) assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input) - }() - } -} - -func TestQueryExecutorPlanPassDmlRBR(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set pk = foo()" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - // RBR mode - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - - // Statement mode - tsv.qe.binlogFormat = connpool.BinlogFormatStatement - _, err = qre.Execute() - if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { - t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) - } - testCommitHelper(t, tsv, qre) -} - -func TestQueryExecutorPassthroughDml(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set pk = foo()" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - // RBR mode - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - - tsv.SetPassthroughDMLs(true) - defer tsv.SetPassthroughDMLs(false) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } + // Test inside a transaction. + txid, err := tsv.Begin(ctx, &tsv.target, nil) + require.NoError(t, err) + defer tsv.Commit(ctx, &tsv.target, txid) - // Statement mode also works when allowUnsafeDMLs is true - tsv.qe.binlogFormat = connpool.BinlogFormatStatement - _, err = qre.Execute() - if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { - t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) + qre = newTestQueryExecutor(ctx, tsv, tcase.input, txid) + got, err = qre.Execute() + require.NoError(t, err, tcase.input) + assert.Equal(t, tcase.resultWant, got, "in tx: %v", tcase.input) + assert.Equal(t, tcase.planWant, qre.logStats.PlanType, "in tx: %v", tcase.input) + want := tcase.logWant + if tcase.inTxWant != "" { + want = tcase.inTxWant + } + assert.Equal(t, want, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input) + }() } } -func TestQueryExecutorPlanPassDmlAutoCommitRBR(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set pk = foo()" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - // RBR mode - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - qre := newTestQueryExecutor(ctx, tsv, query, 0) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - - // Statement mode - tsv.qe.binlogFormat = connpool.BinlogFormatStatement - _, err = qre.Execute() - if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { - t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) +// TestQueryExecutorSelectImpossible is separate because it's a special case +// because the "in transaction" case is a no-op. +func TestQueryExecutorSelectImpossible(t *testing.T) { + type dbResponse struct { + query string + result *sqltypes.Result } -} - -func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "update test_table set pk = foo()" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - // RBR mode - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - - tsv.SetPassthroughDMLs(true) - defer tsv.SetPassthroughDMLs(false) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - qre := newTestQueryExecutor(ctx, tsv, query, 0) - assert.Equal(t, planbuilder.PlanUpdate, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } + fields := sqltypes.MakeTestFields("a|b", "int64|varchar") + fieldResult := sqltypes.MakeTestResult(fields) - // Statement mode - tsv.qe.binlogFormat = connpool.BinlogFormatStatement - _, err = qre.Execute() - if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { - t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) - } -} + testcases := []struct { + input string + dbResponses []dbResponse + resultWant *sqltypes.Result + planWant string + logWant string + inTxWant string + }{{ + input: "select * from t where 1 != 1", + dbResponses: []dbResponse{{ + query: "select * from t where 1 != 1", + result: fieldResult, + }}, + resultWant: fieldResult, + planWant: "SelectImpossible", + logWant: "select * from t where 1 != 1", + inTxWant: "", + }} + for _, tcase := range testcases { + func() { + db := setUpQueryExecutorTest(t) + defer db.Close() + for _, dbr := range tcase.dbResponses { + db.AddQuery(dbr.query, dbr.result) + } + ctx := context.Background() + tsv := newTestTabletServer(ctx, noFlags, db) + defer tsv.StopService() -func TestQueryExecutorPlanPassDmlReplaceInto(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "replace into test_table values (1)" - want := &sqltypes.Result{} - db.AddQuery(query, want) - ctx := context.Background() - // RBR mode - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - tsv.qe.binlogFormat = connpool.BinlogFormatRow - assert.Equal(t, planbuilder.PlanInsert, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - wantqueries := []string{query} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } + qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0) + got, err := qre.Execute() + require.NoError(t, err, tcase.input) + assert.Equal(t, tcase.resultWant, got, tcase.input) + assert.Equal(t, tcase.planWant, qre.logStats.PlanType, tcase.input) + assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input) + txid, err := tsv.Begin(ctx, &tsv.target, nil) + require.NoError(t, err) + defer tsv.Commit(ctx, &tsv.target, txid) - // Statement mode - tsv.qe.binlogFormat = connpool.BinlogFormatStatement - _, err = qre.Execute() - if code := vterrors.Code(err); code != vtrpcpb.Code_UNIMPLEMENTED { - t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) + qre = newTestQueryExecutor(ctx, tsv, tcase.input, txid) + got, err = qre.Execute() + require.NoError(t, err, tcase.input) + assert.Equal(t, tcase.resultWant, got, "in tx: %v", tcase.input) + assert.Equal(t, tcase.planWant, qre.logStats.PlanType, "in tx: %v", tcase.input) + assert.Equal(t, tcase.inTxWant, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input) + }() } - testCommitHelper(t, tsv, qre) } func TestQueryExecutorPlanInsertMessage(t *testing.T) { @@ -390,70 +327,6 @@ func TestQueryExecutorPlanInsertMessage(t *testing.T) { } } -func TestQueryExecutorPlanOtherWithinATransaction(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "show test_table" - want := &sqltypes.Result{ - Fields: getTestTableFields(), - } - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - assert.Equal(t, planbuilder.PlanOtherRead, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - if gotqueries := fetchRecordedQueries(qre); gotqueries != nil { - t.Errorf("queries: %v, want nil", gotqueries) - } -} - -func TestQueryExecutorPlanPassSelectWithInATransaction(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - fields := []*querypb.Field{ - {Name: "addr", Type: sqltypes.Int32}, - } - query := "select addr from test_table where pk = 1 limit 1000" - want := &sqltypes.Result{ - Fields: fields, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - {sqltypes.NewInt32(123)}, - }, - } - db.AddQuery(query, want) - db.AddQuery("select addr from test_table where 1 != 1", &sqltypes.Result{ - Fields: fields, - }) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - txid := newTransaction(tsv, nil) - qre := newTestQueryExecutor(ctx, tsv, query, txid) - defer tsv.StopService() - defer testCommitHelper(t, tsv, qre) - assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } - if gotqueries := fetchRecordedQueries(qre); gotqueries != nil { - t.Errorf("queries: %v, want nil", gotqueries) - } -} - func TestQueryExecutorPlanPassSelectWithLockOutsideATransaction(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() @@ -477,145 +350,6 @@ func TestQueryExecutorPlanPassSelectWithLockOutsideATransaction(t *testing.T) { } } -func TestQueryExecutorPlanPassSelect(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "select * from test_table limit 1000" - want := &sqltypes.Result{ - Fields: getTestTableFields(), - } - db.AddQuery(query, want) - db.AddQuery("select * from test_table where 1 != 1", &sqltypes.Result{ - Fields: getTestTableFields(), - }) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanSelectImpossible(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "select * from test_table where 1 != 1" - want := &sqltypes.Result{ - Fields: getTestTableFields(), - } - db.AddQuery(query, want) - db.AddQuery("select * from test_table where 1 != 1", &sqltypes.Result{ - Fields: getTestTableFields(), - }) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - assert.Equal(t, planbuilder.PlanSelectImpossible, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanPassSelectSqlSelectLimit(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "select * from test_table" - expandedQuery := "select * from test_table limit 20" - want := &sqltypes.Result{ - Fields: getTestTableFields(), - } - db.AddQuery(query, want) - db.AddQuery(expandedQuery, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - qre.options = &querypb.ExecuteOptions{ - SqlSelectLimit: 20, - } - defer tsv.StopService() - assert.Equal(t, planbuilder.PlanSelect, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("got: %v, want: %v", got, want) - } -} - -func TestQueryExecutorPlanSet(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - setQuery := "set unknown_key = 1" - db.AddQuery(setQuery, &sqltypes.Result{}) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - defer tsv.StopService() - qre := newTestQueryExecutor(ctx, tsv, setQuery, 0) - assert.Equal(t, planbuilder.PlanSet, qre.plan.PlanID) - // Query will be delegated to MySQL and both Fields and Rows should be - // empty arrays in this case. - want := &sqltypes.Result{} - got, err := qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("qre.Execute() = %v, want: %v", got, want) - } - - // Test inside transaction. - txid := newTransaction(tsv, nil) - qre = newTestQueryExecutor(ctx, tsv, setQuery, txid) - got, err = qre.Execute() - if err != nil { - t.Fatalf("qre.Execute() = %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("qre.Execute() = %v, want: %v", got, want) - } - wantqueries := []string{"set unknown_key = 1"} - gotqueries := fetchRecordedQueries(qre) - if !reflect.DeepEqual(gotqueries, wantqueries) { - t.Errorf("queries: %v, want %v", gotqueries, wantqueries) - } - testCommitHelper(t, tsv, qre) - tsv.StopService() -} - -func TestQueryExecutorPlanOther(t *testing.T) { - db := setUpQueryExecutorTest(t) - defer db.Close() - query := "show test_table" - want := &sqltypes.Result{ - Fields: getTestTableFields(), - } - db.AddQuery(query, want) - ctx := context.Background() - tsv := newTestTabletServer(ctx, noFlags, db) - qre := newTestQueryExecutor(ctx, tsv, query, 0) - defer tsv.StopService() - assert.Equal(t, planbuilder.PlanOtherRead, qre.plan.PlanID) - got, err := qre.Execute() - if err != nil { - t.Fatalf("got: %v, want nil", err) - } - if !reflect.DeepEqual(got, want) { - t.Fatalf("qre.Execute() = %v, want: %v", got, want) - } -} - func TestQueryExecutorPlanNextval(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() @@ -803,7 +537,7 @@ func TestQueryExecutorMessageStreamACL(t *testing.T) { return io.EOF }) - want := `table acl error: "u2" [] cannot run MESSAGE_STREAM on table "msg"` + want := `table acl error: "u2" [] cannot run MessageStream on table "msg"` if err == nil || err.Error() != want { t.Errorf("qre.MessageStream(msg) error: %v, want %s", err, want) } diff --git a/go/vt/vttablet/tabletserver/querylogz_test.go b/go/vt/vttablet/tabletserver/querylogz_test.go index 793562864bd..249f1962f95 100644 --- a/go/vt/vttablet/tabletserver/querylogz_test.go +++ b/go/vt/vttablet/tabletserver/querylogz_test.go @@ -73,7 +73,7 @@ func TestQuerylogzHandler(t *testing.T) { `0.001`, `0.001`, `1e-08`, - `PASS_SELECT`, + `Select`, `select name from test_table limit 1000`, `1`, `none`, @@ -103,7 +103,7 @@ func TestQuerylogzHandler(t *testing.T) { `0.02`, `0.001`, `1e-08`, - `PASS_SELECT`, + `Select`, `select name from test_table limit 1000`, `1`, `none`, @@ -133,7 +133,7 @@ func TestQuerylogzHandler(t *testing.T) { `0.5`, `0.001`, `1e-08`, - `PASS_SELECT`, + `Select`, `select name from test_table limit 1000`, `1`, `none`, @@ -162,6 +162,7 @@ func TestQuerylogzHandler(t *testing.T) { } func checkQuerylogzHasStats(t *testing.T, pattern []string, logStats *tabletenv.LogStats, page []byte) { + t.Helper() matcher := regexp.MustCompile(strings.Join(pattern, `\s*`)) if !matcher.Match(page) { t.Fatalf("querylogz page does not contain stats: %v, pattern: %v, page: %s", logStats, pattern, string(page)) diff --git a/go/vt/vttablet/tabletserver/queryz_test.go b/go/vt/vttablet/tabletserver/queryz_test.go index 9d1f8fc44f3..cfcc26a45e5 100644 --- a/go/vt/vttablet/tabletserver/queryz_test.go +++ b/go/vt/vttablet/tabletserver/queryz_test.go @@ -85,8 +85,7 @@ func TestQueryzHandler(t *testing.T) { ``, `select name from test_table`, `test_table`, - `PASS_SELECT`, - `TABLE`, + `Select`, `10`, `2.000000`, `1.000000`, @@ -103,7 +102,6 @@ func TestQueryzHandler(t *testing.T) { `insert into test_table values 1`, `test_table`, `DDL`, - `DEFAULT`, `1`, `0.002000`, `0.001000`, @@ -119,8 +117,7 @@ func TestQueryzHandler(t *testing.T) { ``, `show tables`, ``, - `OTHER_READ`, - `DEFAULT`, + `OtherRead`, `1`, `0.075000`, `0.050000`, @@ -136,8 +133,7 @@ func TestQueryzHandler(t *testing.T) { ``, `insert into test_table values .* \[TRUNCATED\][^<]*`, ``, - `OTHER_READ`, - `DEFAULT`, + `OtherRead`, `1`, `0.001000`, `0.001000`, diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 4ddbb1f232c..0aaa4d86b76 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1188,9 +1188,10 @@ func (tsv *TabletServer) computeTxSerializerKey(ctx context.Context, logStats *t } switch plan.PlanID { + // Serialize only UPDATE or DELETE queries. case planbuilder.PlanUpdate, planbuilder.PlanUpdateLimit, planbuilder.PlanDelete, planbuilder.PlanDeleteLimit: - // Serialize only UPDATE or DELETE queries. + default: return "", "" } diff --git a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go index 2a38e4acb05..faa2937d95b 100644 --- a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go +++ b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go @@ -627,7 +627,7 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) { sqltypes.NewVarBinary("dtid0"), sqltypes.NewInt64(RedoStatePrepared), sqltypes.NewVarBinary(""), - sqltypes.NewVarBinary("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"), + sqltypes.NewVarBinary("update test_table set name = 2 where pk = 1 limit 10001"), }}, }) turnOnTxEngine() @@ -635,7 +635,7 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) { t.Errorf("len(tsv.te.preparedPool.conns): %d, want 1", len(tsv.te.preparedPool.conns)) } got := tsv.te.preparedPool.conns["dtid0"].Queries - want := []string{"update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"} + want := []string{"update test_table set name = 2 where pk = 1 limit 10001"} if !reflect.DeepEqual(got, want) { t.Errorf("Prepared queries: %v, want %v", got, want) } @@ -662,7 +662,7 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) { sqltypes.NewVarBinary("a:b:10"), sqltypes.NewInt64(RedoStatePrepared), sqltypes.NewVarBinary(""), - sqltypes.NewVarBinary("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"), + sqltypes.NewVarBinary("update test_table set name = 2 where pk = 1 limit 10001"), }, { sqltypes.NewVarBinary("a:b:20"), sqltypes.NewInt64(RedoStateFailed), @@ -675,7 +675,7 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) { t.Errorf("len(tsv.te.preparedPool.conns): %d, want 1", len(tsv.te.preparedPool.conns)) } got = tsv.te.preparedPool.conns["a:b:10"].Queries - want = []string{"update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */"} + want = []string{"update test_table set name = 2 where pk = 1 limit 10001"} if !reflect.DeepEqual(got, want) { t.Errorf("Prepared queries: %v, want %v", got, want) } @@ -1098,10 +1098,10 @@ func TestTabletServerExecuteBatch(t *testing.T) { testUtils := newTestUtils() sql := "insert into test_table values (1, 2, 'addr', 'name')" sqlResult := &sqltypes.Result{} - expanedSQL := "insert into test_table(pk, name, addr, name_string) values (1, 2, 'addr', 'name') /* _stream test_table (pk ) (1 ); */" + expandedSQL := "insert into test_table(pk, name, addr, name_string) values (1, 2, 'addr', 'name') /* _stream test_table (pk ) (1 ); */" db.AddQuery(sql, sqlResult) - db.AddQuery(expanedSQL, sqlResult) + db.AddQuery(expandedSQL, sqlResult) config := testUtils.newQueryServiceConfig() tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) dbcfgs := testUtils.newDBConfigs(db) @@ -1231,14 +1231,14 @@ func TestTabletServerExecuteBatchSqlExecFailInTransaction(t *testing.T) { testUtils := newTestUtils() sql := "insert into test_table values (1, 2)" sqlResult := &sqltypes.Result{} - expanedSQL := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */" + expandedSQL := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */" db.AddQuery(sql, sqlResult) - db.AddQuery(expanedSQL, sqlResult) + db.AddQuery(expandedSQL, sqlResult) // make this query fail db.AddRejectedQuery(sql, errRejected) - db.AddRejectedQuery(expanedSQL, errRejected) + db.AddRejectedQuery(expandedSQL, errRejected) config := testUtils.newQueryServiceConfig() tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) @@ -1268,40 +1268,6 @@ func TestTabletServerExecuteBatchSqlExecFailInTransaction(t *testing.T) { } } -func TestTabletServerExecuteBatchSqlSucceedInTransaction(t *testing.T) { - db := setUpTabletServerTest(t) - defer db.Close() - testUtils := newTestUtils() - sql := "insert into test_table values (1, 2, 'addr', 'name')" - sqlResult := &sqltypes.Result{} - expanedSQL := "insert into test_table(pk, name, addr, name_string) values (1, 2, 'addr', 'name') /* _stream test_table (pk ) (1 ); */" - - db.AddQuery(sql, sqlResult) - db.AddQuery(expanedSQL, sqlResult) - - // cause execution error for this particular sql query - db.AddRejectedQuery(sql, errRejected) - - config := testUtils.newQueryServiceConfig() - tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) - dbcfgs := testUtils.newDBConfigs(db) - target := querypb.Target{TabletType: topodatapb.TabletType_MASTER} - err := tsv.StartService(target, dbcfgs) - if err != nil { - t.Fatalf("StartService failed: %v", err) - } - defer tsv.StopService() - ctx := context.Background() - if _, err := tsv.ExecuteBatch(ctx, &target, []*querypb.BoundQuery{ - { - Sql: sql, - BindVariables: nil, - }, - }, false, 0, nil); err != nil { - t.Fatal(err) - } -} - func TestTabletServerExecuteBatchCallCommitWithoutABegin(t *testing.T) { db := setUpTabletServerTest(t) defer db.Close() @@ -1332,10 +1298,10 @@ func TestExecuteBatchNestedTransaction(t *testing.T) { testUtils := newTestUtils() sql := "insert into test_table values (1, 2)" sqlResult := &sqltypes.Result{} - expanedSQL := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */" + expandedSQL := "insert into test_table values (1, 2) /* _stream test_table (pk ) (1 ); */" db.AddQuery(sql, sqlResult) - db.AddQuery(expanedSQL, sqlResult) + db.AddQuery(expandedSQL, sqlResult) config := testUtils.newQueryServiceConfig() tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) dbcfgs := testUtils.newDBConfigs(db) @@ -1421,7 +1387,7 @@ func TestSerializeTransactionsSameRow(t *testing.T) { // Make sure that tx3 could finish while tx2 could not. tx3Finished := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) if err := waitForTxSerializationPendingQueries(tsv, "test_table where pk = 1 and name = 1", 2); err != nil { @@ -1546,7 +1512,7 @@ func TestSerializeTransactionsSameRow_ExecuteBatchAsTransaction(t *testing.T) { // Make sure that tx2 and tx3 start only after tx1 is running its Execute(). tx1Started := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) if err := waitForTxSerializationPendingQueries(tsv, "test_table where pk = 1 and name = 1", 2); err != nil { @@ -1663,7 +1629,7 @@ func TestSerializeTransactionsSameRow_ConcurrentTransactions(t *testing.T) { tx1Started := make(chan struct{}) allQueriesPending := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) <-allQueriesPending @@ -1802,7 +1768,7 @@ func TestSerializeTransactionsSameRow_TooManyPendingRequests(t *testing.T) { // Signal when tx2 is done. tx2Failed := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) <-tx2Failed @@ -1891,7 +1857,7 @@ func TestSerializeTransactionsSameRow_TooManyPendingRequests_ExecuteBatchAsTrans // Signal when tx2 is done. tx2Failed := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) <-tx2Failed @@ -1988,7 +1954,7 @@ func TestSerializeTransactionsSameRow_RequestCanceled(t *testing.T) { // Signal when tx2 is done. tx2Done := make(chan struct{}) - db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */", + db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001", func() { close(tx1Started) // Keep blocking until tx2 was canceled. @@ -2117,9 +2083,9 @@ func TestMessageAck(t *testing.T) { } _, err = tsv.MessageAck(ctx, &target, "msg", ids) - want = "query: 'select time_scheduled, id from msg where id in ('1', '2') and time_acked is null limit 10001 for update' is not supported on fakesqldb" + want = "query: 'update msg set time_acked" if err == nil || !strings.Contains(err.Error(), want) { - t.Errorf("tsv.MessageAck(invalid): %v, want %s", err, want) + t.Errorf("tsv.MessageAck(invalid):\n%v, want\n%s", err, want) } db.AddQuery( @@ -2158,7 +2124,7 @@ func TestRescheduleMessages(t *testing.T) { } _, err = tsv.PostponeMessages(ctx, &target, "msg", []string{"1", "2"}) - want = "query: 'select time_scheduled, id from msg where id in ('1', '2') and time_acked is null limit 10001 for update' is not supported" + want = "query: 'update msg set time_next" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("tsv.PostponeMessages(invalid):\n%v, want\n%s", err, want) } @@ -2199,7 +2165,7 @@ func TestPurgeMessages(t *testing.T) { } _, err = tsv.PurgeMessages(ctx, &target, "msg", 0) - want = "query: 'select time_scheduled, id from msg where time_scheduled < 0 and time_acked is not null limit 500 for update' is not supported" + want = "query: 'delete from msg where time_scheduled" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("tsv.PurgeMessages(invalid):\n%v, want\n%s", err, want) } @@ -2218,7 +2184,7 @@ func TestPurgeMessages(t *testing.T) { }}, }, ) - db.AddQuery("delete from msg where (time_scheduled = 1 and id = 1) /* _stream msg (time_scheduled id ) (1 1 ); */", &sqltypes.Result{RowsAffected: 1}) + db.AddQuery("delete from msg where time_scheduled < 3 and time_acked is not null limit 500", &sqltypes.Result{RowsAffected: 1}) count, err := tsv.PurgeMessages(ctx, &target, "msg", 3) require.NoError(t, err) if count != 1 { @@ -2616,17 +2582,17 @@ func checkTabletServerState(t *testing.T, tsv *TabletServer, expectState int64) func getSupportedQueries() map[string]*sqltypes.Result { return map[string]*sqltypes.Result{ // Queries for how row protection test (txserializer). - "update test_table set name_string = 'tx1' where pk in (1) /* _stream test_table (pk ) (1 ); */": { + "update test_table set name_string = 'tx1' where pk = 1 and name = 1 limit 10001": { RowsAffected: 1, }, - "update test_table set name_string = 'tx2' where pk in (1) /* _stream test_table (pk ) (1 ); */": { + "update test_table set name_string = 'tx2' where pk = 1 and name = 1 limit 10001": { RowsAffected: 1, }, - "update test_table set name_string = 'tx3' where pk in (1) /* _stream test_table (pk ) (1 ); */": { + "update test_table set name_string = 'tx3' where pk = 1 and name = 1 limit 10001": { RowsAffected: 1, }, // tx3, but with different primary key. - "update test_table set name_string = 'tx3' where pk in (2) /* _stream test_table (pk ) (2 ); */": { + "update test_table set name_string = 'tx3' where pk = 2 and name = 1 limit 10001": { RowsAffected: 1, }, // Complex WHERE clause requires SELECT of primary key first. diff --git a/go/vt/vttablet/tabletserver/tx_executor_test.go b/go/vt/vttablet/tabletserver/tx_executor_test.go index d4c3fef16b3..bc4bcb64b0a 100644 --- a/go/vt/vttablet/tabletserver/tx_executor_test.go +++ b/go/vt/vttablet/tabletserver/tx_executor_test.go @@ -529,7 +529,7 @@ func newTestTxExecutor(t *testing.T) (txe *TxExecutor, tsv *TabletServer, db *fa db.AddQueryPattern("insert into `_vt`\\.redo_statement.*", &sqltypes.Result{}) db.AddQuery("delete from `_vt`.redo_state where dtid = 'aa'", &sqltypes.Result{}) db.AddQuery("delete from `_vt`.redo_statement where dtid = 'aa'", &sqltypes.Result{}) - db.AddQuery("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) + db.AddQuery("update test_table set name = 2 where pk = 1 limit 10001", &sqltypes.Result{}) return &TxExecutor{ ctx: ctx, logStats: logStats, @@ -547,7 +547,7 @@ func newShortAgeExecutor(t *testing.T) (txe *TxExecutor, tsv *TabletServer, db * db.AddQueryPattern("insert into `_vt`\\.redo_statement.*", &sqltypes.Result{}) db.AddQuery("delete from `_vt`.redo_state where dtid = 'aa'", &sqltypes.Result{}) db.AddQuery("delete from `_vt`.redo_statement where dtid = 'aa'", &sqltypes.Result{}) - db.AddQuery("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */", &sqltypes.Result{}) + db.AddQuery("update test_table set name = 2 where pk = 1 limit 10001", &sqltypes.Result{}) return &TxExecutor{ ctx: ctx, logStats: logStats, From a1e533d9aec1b5267d8e590a9ec95a16d7e9431a Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 18 Mar 2020 13:11:40 -0700 Subject: [PATCH 13/18] deprecation: no need to check binlog format Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/connpool/dbconn.go | 42 +++++-------------- go/vt/vttablet/tabletserver/query_engine.go | 5 ++- .../tabletserver/query_executor_test.go | 12 ------ .../tabletserver/tabletserver_flaky_test.go | 20 --------- 4 files changed, 14 insertions(+), 65 deletions(-) diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn.go b/go/vt/vttablet/tabletserver/connpool/dbconn.go index d5f9a6c20d3..ef39b8e193b 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn.go @@ -227,65 +227,45 @@ var ( getModeSQL = "select @@global.sql_mode" getAutocommit = "select @@autocommit" getAutoIsNull = "select @@sql_auto_is_null" - showBinlog = "show variables like 'binlog_format'" ) // VerifyMode is a helper method to verify mysql is running with // sql_mode = STRICT_TRANS_TABLES or STRICT_ALL_TABLES and autocommit=ON. -// It also returns the current binlog format. -func (dbc *DBConn) VerifyMode(strictTransTables bool) (BinlogFormat, error) { +func (dbc *DBConn) VerifyMode(strictTransTables bool) error { if strictTransTables { qr, err := dbc.conn.ExecuteFetch(getModeSQL, 2, false) if err != nil { - return 0, vterrors.Wrap(err, "could not verify mode") + return vterrors.Wrap(err, "could not verify mode") } if len(qr.Rows) != 1 { - return 0, fmt.Errorf("incorrect rowcount received for %s: %d", getModeSQL, len(qr.Rows)) + return fmt.Errorf("incorrect rowcount received for %s: %d", getModeSQL, len(qr.Rows)) } sqlMode := qr.Rows[0][0].ToString() if !(strings.Contains(sqlMode, "STRICT_TRANS_TABLES") || strings.Contains(sqlMode, "STRICT_ALL_TABLES")) { - return 0, fmt.Errorf("require sql_mode to be STRICT_TRANS_TABLES or STRICT_ALL_TABLES: got '%s'", qr.Rows[0][0].ToString()) + return fmt.Errorf("require sql_mode to be STRICT_TRANS_TABLES or STRICT_ALL_TABLES: got '%s'", qr.Rows[0][0].ToString()) } } qr, err := dbc.conn.ExecuteFetch(getAutocommit, 2, false) if err != nil { - return 0, vterrors.Wrap(err, "could not verify mode") + return vterrors.Wrap(err, "could not verify mode") } if len(qr.Rows) != 1 { - return 0, fmt.Errorf("incorrect rowcount received for %s: %d", getAutocommit, len(qr.Rows)) + return fmt.Errorf("incorrect rowcount received for %s: %d", getAutocommit, len(qr.Rows)) } if !strings.Contains(qr.Rows[0][0].ToString(), "1") { - return 0, fmt.Errorf("require autocommit to be 1: got %s", qr.Rows[0][0].ToString()) + return fmt.Errorf("require autocommit to be 1: got %s", qr.Rows[0][0].ToString()) } qr, err = dbc.conn.ExecuteFetch(getAutoIsNull, 2, false) if err != nil { - return 0, vterrors.Wrap(err, "could not verify mode") + return vterrors.Wrap(err, "could not verify mode") } if len(qr.Rows) != 1 { - return 0, fmt.Errorf("incorrect rowcount received for %s: %d", getAutoIsNull, len(qr.Rows)) + return fmt.Errorf("incorrect rowcount received for %s: %d", getAutoIsNull, len(qr.Rows)) } if !strings.Contains(qr.Rows[0][0].ToString(), "0") { - return 0, fmt.Errorf("require sql_auto_is_null to be 0: got %s", qr.Rows[0][0].ToString()) + return fmt.Errorf("require sql_auto_is_null to be 0: got %s", qr.Rows[0][0].ToString()) } - qr, err = dbc.conn.ExecuteFetch(showBinlog, 10, false) - if err != nil { - return 0, vterrors.Wrap(err, "could not fetch binlog format") - } - if len(qr.Rows) != 1 { - return 0, fmt.Errorf("incorrect rowcount received for %s: %d", showBinlog, len(qr.Rows)) - } - if len(qr.Rows[0]) != 2 { - return 0, fmt.Errorf("incorrect column count received for %s: %d", showBinlog, len(qr.Rows[0])) - } - switch qr.Rows[0][1].ToString() { - case "STATEMENT": - return BinlogFormatStatement, nil - case "ROW": - return BinlogFormatRow, nil - case "MIXED": - return BinlogFormatMixed, nil - } - return 0, fmt.Errorf("unexpected binlog format for %s: %s", showBinlog, qr.Rows[0][1].ToString()) + return nil } // Close closes the DBConn. diff --git a/go/vt/vttablet/tabletserver/query_engine.go b/go/vt/vttablet/tabletserver/query_engine.go index b996cdbf572..303ecf6657c 100644 --- a/go/vt/vttablet/tabletserver/query_engine.go +++ b/go/vt/vttablet/tabletserver/query_engine.go @@ -149,7 +149,6 @@ type QueryEngine struct { connTimeout sync2.AtomicDuration queryPoolWaiters sync2.AtomicInt64 queryPoolWaiterCap sync2.AtomicInt64 - binlogFormat connpool.BinlogFormat maxResultSize sync2.AtomicInt64 warnResultSize sync2.AtomicInt64 maxDMLRows sync2.AtomicInt64 @@ -292,7 +291,9 @@ func (qe *QueryEngine) Open() error { qe.conns.Close() return err } - qe.binlogFormat, err = conn.VerifyMode(qe.strictTransTables) + err = conn.VerifyMode(qe.strictTransTables) + // Recycle needs to happen before error check. + // Otherwise, qe.conns.Close will hang. conn.Recycle() if err != nil { diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 3091078cf92..aa2e38ceffc 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -1105,18 +1105,6 @@ func getQueryExecutorSupportedQueries(testTableHasMultipleUniqueKeys bool) map[s {sqltypes.NewVarBinary("fakedb server")}, }, }, - "show variables like 'binlog_format'": { - Fields: []*querypb.Field{{ - Type: sqltypes.VarChar, - }, { - Type: sqltypes.VarChar, - }}, - RowsAffected: 1, - Rows: [][]sqltypes.Value{{ - sqltypes.NewVarBinary("binlog_format"), - sqltypes.NewVarBinary("STATEMENT"), - }}, - }, mysql.BaseShowTables: { Fields: mysql.BaseShowTablesFields, RowsAffected: 3, diff --git a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go index faa2937d95b..4d2cc6a0b8b 100644 --- a/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go +++ b/go/vt/vttablet/tabletserver/tabletserver_flaky_test.go @@ -2595,26 +2595,6 @@ func getSupportedQueries() map[string]*sqltypes.Result { "update test_table set name_string = 'tx3' where pk = 2 and name = 1 limit 10001": { RowsAffected: 1, }, - // Complex WHERE clause requires SELECT of primary key first. - "select pk from test_table where pk = 1 and name = 1 limit 10001 for update": { - Fields: []*querypb.Field{ - {Type: sqltypes.Int64}, - }, - RowsAffected: 1, - Rows: [][]sqltypes.Value{{ - sqltypes.NewVarBinary("1"), - }}, - }, - // Complex WHERE clause requires SELECT of primary key first. - "select pk from test_table where pk = 2 and name = 1 limit 10001 for update": { - Fields: []*querypb.Field{ - {Type: sqltypes.Int64}, - }, - RowsAffected: 1, - Rows: [][]sqltypes.Value{{ - sqltypes.NewVarBinary("2"), - }}, - }, // queries for twopc sqlTurnoffBinlog: {}, fmt.Sprintf(sqlCreateSidecarDB, "`_vt`"): {}, From d5a97a3942e07dbf7e3c4ce4d5f0bcf0e0b15c16 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 18 Mar 2020 13:23:50 -0700 Subject: [PATCH 14/18] deprecation: fix other tests under tabletserver Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/rules/rules_test.go | 2 +- .../tabletserver/schema/load_table_test.go | 38 ------------------- go/vt/vttablet/tabletserver/schema/schema.go | 2 +- 3 files changed, 2 insertions(+), 40 deletions(-) diff --git a/go/vt/vttablet/tabletserver/rules/rules_test.go b/go/vt/vttablet/tabletserver/rules/rules_test.go index 09377833e4a..3a25ce74fca 100644 --- a/go/vt/vttablet/tabletserver/rules/rules_test.go +++ b/go/vt/vttablet/tabletserver/rules/rules_test.go @@ -532,7 +532,7 @@ func TestImport(t *testing.T) { "RequestIP": "123.123.123", "User": "user", "Query": "query", - "Plans": ["PASS_SELECT", "INSERT_PK"], + "Plans": ["Select", "Insert"], "TableNames":["a", "b"], "BindVarConds": [{ "Name": "bvname1", diff --git a/go/vt/vttablet/tabletserver/schema/load_table_test.go b/go/vt/vttablet/tabletserver/schema/load_table_test.go index 5b9fb6ecda9..ce96c828f2c 100644 --- a/go/vt/vttablet/tabletserver/schema/load_table_test.go +++ b/go/vt/vttablet/tabletserver/schema/load_table_test.go @@ -170,25 +170,6 @@ func TestLoadTableMessage(t *testing.T) { t.Errorf("newTestLoadTable: %v, want %s", err, wanterr) } - // id column must be part of primary key. - for query, result := range getMessageTableQueries() { - db.AddQuery(query, result) - } - db.AddQuery( - "show index from test_table", - &sqltypes.Result{ - Fields: mysql.ShowIndexFromTableFields, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - mysql.ShowIndexFromTableRow("test_table", true, "PRIMARY", 1, "time_scheduled", false), - }, - }) - _, err = newTestLoadTable("USER_TABLE", "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30", db) - wanterr = "id column is not part of the primary key for message table: test_table" - if err == nil || err.Error() != wanterr { - t.Errorf("newTestLoadTable: %v, want %s", err, wanterr) - } - for query, result := range getTestLoadTableQueries() { db.AddQuery(query, result) } @@ -246,25 +227,6 @@ func TestLoadTableMessageTopic(t *testing.T) { t.Errorf("newTestLoadTable: %v, want %s", err, wanterr) } - // id column must be part of primary key. - for query, result := range getMessageTableQueries() { - db.AddQuery(query, result) - } - db.AddQuery( - "show index from test_table", - &sqltypes.Result{ - Fields: mysql.ShowIndexFromTableFields, - RowsAffected: 1, - Rows: [][]sqltypes.Value{ - mysql.ShowIndexFromTableRow("test_table", true, "PRIMARY", 1, "time_scheduled", false), - }, - }) - _, err = newTestLoadTable("USER_TABLE", "vitess_message,vt_topic=test_topic,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30", db) - wanterr = "id column is not part of the primary key for message table: test_table" - if err == nil || err.Error() != wanterr { - t.Errorf("newTestLoadTable: %v, want %s", err, wanterr) - } - for query, result := range getTestLoadTableQueries() { db.AddQuery(query, result) } diff --git a/go/vt/vttablet/tabletserver/schema/schema.go b/go/vt/vttablet/tabletserver/schema/schema.go index b6e7b7bbc02..13b2d7282a0 100644 --- a/go/vt/vttablet/tabletserver/schema/schema.go +++ b/go/vt/vttablet/tabletserver/schema/schema.go @@ -140,7 +140,7 @@ func NewTable(name string) *Table { // Done must be called after columns and indexes are added to // the table. It will build additional metadata like PKColumns. func (ta *Table) Done() { - if !ta.HasPrimary() { + if len(ta.Indexes) == 0 { return } pkIndex := ta.Indexes[0] From 8e728d4e443baf702b64ed694d01b5a275269d55 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 18 Mar 2020 19:25:20 -0700 Subject: [PATCH 15/18] deprecation: fix tabletserver endtoend tests Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/endtoend/compatibility_test.go | 84 ---------- go/vt/vttablet/endtoend/config_test.go | 100 ------------ go/vt/vttablet/endtoend/misc_test.go | 22 +-- go/vt/vttablet/endtoend/queries_test.go | 148 +++++------------- go/vt/vttablet/endtoend/transaction_test.go | 29 ++-- 5 files changed, 58 insertions(+), 325 deletions(-) diff --git a/go/vt/vttablet/endtoend/compatibility_test.go b/go/vt/vttablet/endtoend/compatibility_test.go index e5837eda5b2..fa4180659aa 100644 --- a/go/vt/vttablet/endtoend/compatibility_test.go +++ b/go/vt/vttablet/endtoend/compatibility_test.go @@ -657,90 +657,6 @@ func TestNull(t *testing.T) { } } -func TestTypeLimits(t *testing.T) { - client := framework.NewClient() - defer func() { - for _, cleanup := range []string{ - "delete from vitess_ints", - "delete from vitess_fracts", - "delete from vitess_strings", - } { - _, err := client.Execute(cleanup, nil) - if err != nil { - t.Error(err) - } - } - }() - - for _, query := range []string{ - "insert into vitess_ints(tiny, medium) values(1, -129)", - "insert into vitess_fracts(id, num) values(1, 1)", - "insert into vitess_strings(vb) values('a')", - } { - _, err := client.Execute(query, nil) - if err != nil { - t.Fatal(err) - } - } - - mismatchCases := []struct { - query string - bv map[string]*querypb.BindVariable - out string - }{{ - query: "insert into vitess_ints(tiny) values('str')", - bv: nil, - out: "strconv.ParseInt", - }, { - query: "insert into vitess_ints(tiny) values(:str)", - bv: map[string]*querypb.BindVariable{"str": sqltypes.StringBindVariable("str")}, - out: "strconv.ParseInt", - }, { - query: "insert into vitess_ints(tiny) values(1.2)", - bv: nil, - out: "Duplicate entry '1' for key 'PRIMARY'", - }, { - query: "insert into vitess_ints(tiny) values(:fl)", - bv: map[string]*querypb.BindVariable{"fl": sqltypes.Float64BindVariable(1.2)}, - out: "invalid syntax", - }, { - query: "insert into vitess_strings(vb) select tiny from vitess_ints", - bv: nil, - out: "type mismatch", - }, { - query: "insert into vitess_ints(tiny) select num from vitess_fracts", - bv: nil, - out: "type mismatch", - }, { - query: "insert into vitess_ints(tiny) select vb from vitess_strings", - bv: nil, - out: "type mismatch", - }} - for _, tcase := range mismatchCases { - _, err := client.Execute(tcase.query, tcase.bv) - if err == nil || !strings.Contains(err.Error(), tcase.out) { - t.Errorf("Error(%s): %v, want %s", tcase.query, err, tcase.out) - } - } - - want := "Out of range" - for _, query := range []string{ - "insert into vitess_ints(tiny) values(-129)", - "insert into vitess_ints(tiny) select medium from vitess_ints", - } { - _, err := client.Execute(query, nil) - if err == nil || !strings.HasPrefix(err.Error(), want) { - t.Errorf("Error(%s): %v, want %s", query, err, want) - } - } - - want = "Data too long" - _, err := client.Execute("insert into vitess_strings(vb) values('12345678901234567')", nil) - if err == nil || !strings.HasPrefix(err.Error(), want) { - t.Errorf("Error: %v, want %s", err, want) - } -} - func TestJSONType(t *testing.T) { // JSON is supported only after mysql57. client := framework.NewClient() diff --git a/go/vt/vttablet/endtoend/config_test.go b/go/vt/vttablet/endtoend/config_test.go index 507ae7ea2b6..b83000074e2 100644 --- a/go/vt/vttablet/endtoend/config_test.go +++ b/go/vt/vttablet/endtoend/config_test.go @@ -360,106 +360,6 @@ func TestWarnResultSize(t *testing.T) { } } -func TestMaxDMLRows(t *testing.T) { - client := framework.NewClient() - _, err := client.Execute( - "insert into vitess_a(eid, id, name, foo) values "+ - "(3, 1, '', ''), (3, 2, '', ''), (3, 3, '', '')", - nil, - ) - if err != nil { - t.Error(err) - return - } - catcher := framework.NewQueryCatcher() - defer catcher.Close() - - // Verify all three rows are updated in a single DML. - _, err = client.Execute("update vitess_a set foo='fghi' where eid = 3", nil) - if err != nil { - t.Error(err) - return - } - queryInfo, err := catcher.Next() - if err != nil { - t.Error(err) - return - } - want := "begin; " + - "select eid, id from vitess_a where eid = 3 limit 10001 for update; " + - "update vitess_a set foo = 'fghi' where " + - "(eid = 3 and id = 1) or (eid = 3 and id = 2) or (eid = 3 and id = 3); " + - "commit" - if queryInfo.RewrittenSQL() != want { - t.Errorf("Query info: \n%s, want \n%s", queryInfo.RewrittenSQL(), want) - } - - // Verify that rows get split, and if pk changes, those values are also - // split correctly. - defer framework.Server.SetMaxDMLRows(framework.Server.MaxDMLRows()) - framework.Server.SetMaxDMLRows(2) - _, err = client.Execute("update vitess_a set eid=2 where eid = 3", nil) - if err != nil { - t.Error(err) - return - } - queryInfo, err = catcher.Next() - if err != nil { - t.Error(err) - return - } - want = "begin; " + - "select eid, id from vitess_a where eid = 3 limit 10001 for update; " + - "update vitess_a set eid = 2 where " + - "(eid = 3 and id = 1) or (eid = 3 and id = 2); " + - "update vitess_a set eid = 2 where (eid = 3 and id = 3); " + - "commit" - if queryInfo.RewrittenSQL() != want { - t.Errorf("Query info: \n%s, want \n%s", queryInfo.RewrittenSQL(), want) - } - - // Verify that a normal update is split correctly. - _, err = client.Execute("update vitess_a set foo='fghi' where eid = 2", nil) - if err != nil { - t.Error(err) - return - } - queryInfo, err = catcher.Next() - if err != nil { - t.Error(err) - return - } - want = "begin; " + - "select eid, id from vitess_a where eid = 2 limit 10001 for update; " + - "update vitess_a set foo = 'fghi' where (eid = 2 and id = 1) or " + - "(eid = 2 and id = 2); " + - "update vitess_a set foo = 'fghi' where (eid = 2 and id = 3); " + - "commit" - if queryInfo.RewrittenSQL() != want { - t.Errorf("Query info: \n%s, want \n%s", queryInfo.RewrittenSQL(), want) - } - - // Verufy that a delete is split correctly. - _, err = client.Execute("delete from vitess_a where eid = 2", nil) - if err != nil { - t.Error(err) - return - } - queryInfo, err = catcher.Next() - if err != nil { - t.Error(err) - return - } - want = "begin; " + - "select eid, id from vitess_a where eid = 2 limit 10001 for update; " + - "delete from vitess_a where (eid = 2 and id = 1) or (eid = 2 and id = 2); " + - "delete from vitess_a where (eid = 2 and id = 3); " + - "commit" - if queryInfo.RewrittenSQL() != want { - t.Errorf("Query info: \n%s, want \n%s", queryInfo.RewrittenSQL(), want) - } -} - func TestQueryTimeout(t *testing.T) { vstart := framework.DebugVars() defer framework.Server.QueryTimeout.Set(framework.Server.QueryTimeout.Get()) diff --git a/go/vt/vttablet/endtoend/misc_test.go b/go/vt/vttablet/endtoend/misc_test.go index eac102015d5..39634869519 100644 --- a/go/vt/vttablet/endtoend/misc_test.go +++ b/go/vt/vttablet/endtoend/misc_test.go @@ -52,7 +52,7 @@ func TestSimpleRead(t *testing.T) { if err := compareIntDiff(vend, "Queries/TotalCount", vstart, 1); err != nil { t.Error(err) } - if err := compareIntDiff(vend, "Queries/Histograms/PASS_SELECT/Count", vstart, 1); err != nil { + if err := compareIntDiff(vend, "Queries/Histograms/Select/Count", vstart, 1); err != nil { t.Error(err) } } @@ -479,7 +479,7 @@ func TestQueryStats(t *testing.T) { want := framework.QueryStat{ Query: query, Table: "vitess_a", - Plan: "PASS_SELECT", + Plan: "Select", QueryCount: 1, RowCount: 2, ErrorCount: 0, @@ -497,7 +497,7 @@ func TestQueryStats(t *testing.T) { want = framework.QueryStat{ Query: query, Table: "vitess_a", - Plan: "PASS_SELECT", + Plan: "Select", QueryCount: 1, RowCount: 0, ErrorCount: 1, @@ -506,13 +506,13 @@ func TestQueryStats(t *testing.T) { t.Errorf("stat: %+v, want %+v", stat, want) } vend := framework.DebugVars() - if err := compareIntDiff(vend, "QueryCounts/vitess_a.PASS_SELECT", vstart, 2); err != nil { + if err := compareIntDiff(vend, "QueryCounts/vitess_a.Select", vstart, 2); err != nil { t.Error(err) } - if err := compareIntDiff(vend, "QueryRowCounts/vitess_a.PASS_SELECT", vstart, 2); err != nil { + if err := compareIntDiff(vend, "QueryRowCounts/vitess_a.Select", vstart, 2); err != nil { t.Error(err) } - if err := compareIntDiff(vend, "QueryErrorCounts/vitess_a.PASS_SELECT", vstart, 1); err != nil { + if err := compareIntDiff(vend, "QueryErrorCounts/vitess_a.Select", vstart, 1); err != nil { t.Error(err) } @@ -607,7 +607,7 @@ func TestLogTruncation(t *testing.T) { "insert into vitess_test values(123, null, :data, null)", map[string]*querypb.BindVariable{"data": sqltypes.StringBindVariable("THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED")}, ) - wantLog := `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null)", BindVars: {#maxLimit: "type:INT64 value:\"10001\" "data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` + wantLog := `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null)", BindVars: {data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` wantErr := wantLog if err == nil { t.Errorf("query unexpectedly succeeded") @@ -626,8 +626,8 @@ func TestLogTruncation(t *testing.T) { "insert into vitess_test values(123, null, :data, null)", map[string]*querypb.BindVariable{"data": sqltypes.StringBindVariable("THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED")}, ) - wantLog = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess [TRUNCATED]", BindVars: {#maxLim [TRUNCATED]` - wantErr = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null)", BindVars: {#maxLimit: "type:INT64 value:\"10001\" "data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` + wantLog = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess [TRUNCATED]", BindVars: {data: " [TRUNCATED]` + wantErr = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null)", BindVars: {data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` if err == nil { t.Errorf("query unexpectedly succeeded") } @@ -644,8 +644,8 @@ func TestLogTruncation(t *testing.T) { "insert into vitess_test values(123, null, :data, null) /* KEEP ME */", map[string]*querypb.BindVariable{"data": sqltypes.StringBindVariable("THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED")}, ) - wantLog = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess [TRUNCATED] /* KEEP ME */", BindVars: {#maxLim [TRUNCATED]` - wantErr = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null) /* KEEP ME */", BindVars: {#maxLimit: "type:INT64 value:\"10001\" "data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` + wantLog = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess [TRUNCATED] /* KEEP ME */", BindVars: {data: " [TRUNCATED]` + wantErr = `Data too long for column 'charval' at row 1 (errno 1406) (sqlstate 22001) (CallerID: dev): Sql: "insert into vitess_test values(123, null, :data, null) /* KEEP ME */", BindVars: {data: "type:VARCHAR value:\"THIS IS A LONG LONG LONG LONG QUERY STRING THAT SHOULD BE SHORTENED\" "}` if err == nil { t.Errorf("query unexpectedly succeeded") } diff --git a/go/vt/vttablet/endtoend/queries_test.go b/go/vt/vttablet/endtoend/queries_test.go index d9287359925..1cd00a795dc 100644 --- a/go/vt/vttablet/endtoend/queries_test.go +++ b/go/vt/vttablet/endtoend/queries_test.go @@ -33,7 +33,7 @@ RowsAffected mismatch: 2, want 1 Rewritten mismatch: '["select eid, id from vitess_a where 1 != 1 union select eid, id from vitess_b where 1 != 1" "select /* fail */ eid, id from vitess_a union select eid, id from vitess_b limit 10001"]' does not match '["select eid id from vitess_a where 1 != 1 union select eid, id from vitess_b where 1 != 1" "select /* fail */ eid, id from vitess_a union select eid, id from vitess_b"]' -Plan mismatch: PASS_SELECT, want aa` +Plan mismatch: Select, want aa` func TestTheFramework(t *testing.T) { client := framework.NewClient() @@ -59,7 +59,8 @@ func TestTheFramework(t *testing.T) { } } -// TODO(sougou): break this up into smaller parts. +// Most of these tests are not really needed because the queries are mostly pass-through. +// They're left as is because they still demonstrate the variety of constructs being supported. func TestQueries(t *testing.T) { client := framework.NewClient() @@ -478,7 +479,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert /* simple */ into vitess_a values (2, 1, 'aaaa', 'bbbb')", Rewritten: []string{ - "insert /* simple */ into vitess_a(eid, id, name, foo) values (2, 1, 'aaaa', 'bbbb')", + "insert /* simple */ into vitess_a values (2, 1, 'aaaa', 'bbbb')", }, RowsAffected: 1, }, @@ -501,7 +502,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert /* simple */ ignore into vitess_a values (2, 1, 'aaaa', 'bbbb')", Rewritten: []string{ - "insert /* simple */ ignore into vitess_a(eid, id, name, foo) values (2, 1, 'aaaa', 'bbbb')", + "insert /* simple */ ignore into vitess_a values (2, 1, 'aaaa', 'bbbb')", }, RowsAffected: 1, }, @@ -516,7 +517,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert /* simple */ ignore into vitess_a values (2, 1, 'cccc', 'cccc')", Rewritten: []string{ - "insert /* simple */ ignore into vitess_a(eid, id, name, foo) values (2, 1, 'cccc', 'cccc')", + "insert /* simple */ ignore into vitess_a values (2, 1, 'cccc', 'cccc')", }, }, framework.TestQuery("commit"), @@ -759,8 +760,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert /* subquery */ into vitess_a(eid, name, foo) select eid, name, foo from vitess_c", Rewritten: []string{ - "select eid, name, foo from vitess_c limit 10001", - "insert /* subquery */ into vitess_a(eid, name, foo) values (10, 'abcd', '20'), (11, 'bcde', '30')", + "insert /* subquery */ into vitess_a(eid, name, foo) select eid, name, foo from vitess_c", }, RowsAffected: 2, }, @@ -777,8 +777,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert into vitess_e(id, name, foo) select eid, name, foo from vitess_c", Rewritten: []string{ - "select eid, name, foo from vitess_c limit 10001", - "insert into vitess_e(id, name, foo) values (10, 'abcd', '20'), (11, 'bcde', '30')", + "insert into vitess_e(id, name, foo) select eid, name, foo from vitess_c", }, RowsAffected: 2, }, @@ -796,22 +795,6 @@ func TestQueries(t *testing.T) { framework.TestQuery("commit"), }, }, - &framework.MultiCase{ - Name: "reorganize partition with bindvar", - Cases: []framework.Testable{ - framework.TestQuery("begin"), - &framework.TestCase{ - Query: "alter table vitess_part reorganize partition p1 into (partition p2 values less than (:bv), partition p3 values less than (maxvalue))", - BindVars: map[string]*querypb.BindVariable{ - "bv": sqltypes.Int64BindVariable(1000), - }, - Rewritten: []string{ - "alter table vitess_part reorganize partition p1 into (partition p2 values less than (1000), partition p3 values less than (maxvalue))", - }, - }, - framework.TestQuery("commit"), - }, - }, &framework.MultiCase{ Name: "multi-value", Cases: []framework.Testable{ @@ -995,7 +978,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid = 1 and id = 1", Rewritten: []string{ - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1)", + "update /* pk */ vitess_a set foo = 'bar' where eid = 1 and id = 1 limit 10001", }, RowsAffected: 1, }, @@ -1018,7 +1001,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid = 1 and id in (1, 2)", Rewritten: []string{ - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1) or (eid = 1 and id = 2)", + "update /* pk */ vitess_a set foo = 'bar' where eid = 1 and id in (1, 2) limit 10001", }, RowsAffected: 2, }, @@ -1042,8 +1025,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid in (1) and id in (1, 2)", Rewritten: []string{ - "select eid, id from vitess_a where eid in (1) and id in (1, 2) limit 10001 for update", - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1) or (eid = 1 and id = 2)", + "update /* pk */ vitess_a set foo = 'bar' where eid in (1) and id in (1, 2) limit 10001", }, RowsAffected: 2, }, @@ -1067,8 +1049,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid in (1, 2) and id in (1, 2)", Rewritten: []string{ - "select eid, id from vitess_a where eid in (1, 2) and id in (1, 2) limit 10001 for update", - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1) or (eid = 1 and id = 2)", + "update /* pk */ vitess_a set foo = 'bar' where eid in (1, 2) and id in (1, 2) limit 10001", }, RowsAffected: 2, }, @@ -1092,7 +1073,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_a set eid = 2 where eid = 1 and id = 1", Rewritten: []string{ - "update vitess_a set eid = 2 where (eid = 1 and id = 1)", + "update vitess_a set eid = 2 where eid = 1 and id = 1 limit 10001", }, RowsAffected: 1, }, @@ -1115,8 +1096,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where id = 1", Rewritten: []string{ - "select eid, id from vitess_a where id = 1 limit 10001 for update", - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1)", + "update /* pk */ vitess_a set foo = 'bar' where id = 1 limit 10001", }, RowsAffected: 1, }, @@ -1139,8 +1119,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid = 1 limit 1", Rewritten: []string{ - "select eid, id from vitess_a where eid = 1 limit 1 for update", - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 1)", + "update /* pk */ vitess_a set foo = 'bar' where eid = 1 limit 1", }, RowsAffected: 1, }, @@ -1163,8 +1142,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update /* pk */ vitess_a set foo='bar' where eid = 1 order by id desc limit 1", Rewritten: []string{ - "select eid, id from vitess_a where eid = 1 order by id desc limit 1 for update", - "update /* pk */ vitess_a set foo = 'bar' where (eid = 1 and id = 2) order by id desc", + "update /* pk */ vitess_a set foo = 'bar' where eid = 1 order by id desc limit 1", }, RowsAffected: 1, }, @@ -1187,8 +1165,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_a set foo='bar'", Rewritten: []string{ - "select eid, id from vitess_a limit 10001 for update", - "update vitess_a set foo = 'bar' where (eid = 1 and id = 1) or (eid = 1 and id = 2)", + "update vitess_a set foo = 'bar' limit 10001", }, RowsAffected: 2, }, @@ -1216,7 +1193,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_f set id=2 where vb='a'", Rewritten: []string{ - "update vitess_f set id = 2 where vb in ('a')", + "update vitess_f set id = 2 where vb = 'a' limit 10001", }, RowsAffected: 1, }, @@ -1243,7 +1220,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_f set id=3 where vb in ('a', 'b')", Rewritten: []string{ - "update vitess_f set id = 3 where vb in ('a', 'b')", + "update vitess_f set id = 3 where vb in ('a', 'b') limit 10001", }, RowsAffected: 2, }, @@ -1270,8 +1247,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_f set id=4 where id >= 0", Rewritten: []string{ - "select vb from vitess_f where id >= 0 limit 10001 for update", - "update vitess_f set id = 4 where vb in ('a', 'b')", + "update vitess_f set id = 4 where id >= 0 limit 10001", }, RowsAffected: 2, }, @@ -1298,7 +1274,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_f set id=4 where id < 0", Rewritten: []string{ - "select vb from vitess_f where id < 0 limit 10001 for update", + "update vitess_f set id = 4 where id < 0 limit 10001", }, }, framework.TestQuery("commit"), @@ -1322,7 +1298,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete /* pk */ from vitess_a where eid = 2 and id = 1", Rewritten: []string{ - "delete /* pk */ from vitess_a where (eid = 2 and id = 1)", + "delete /* pk */ from vitess_a where eid = 2 and id = 1 limit 10001", }, RowsAffected: 1, }, @@ -1340,7 +1316,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete /* pk */ from vitess_a where eid = 2 and id in (1, 2)", Rewritten: []string{ - "delete /* pk */ from vitess_a where (eid = 2 and id = 1) or (eid = 2 and id = 2)", + "delete /* pk */ from vitess_a where eid = 2 and id in (1, 2) limit 10001", }, RowsAffected: 1, }, @@ -1358,8 +1334,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete /* pk */ from vitess_a where eid in (2) and id in (1, 2)", Rewritten: []string{ - "select eid, id from vitess_a where eid in (2) and id in (1, 2) limit 10001 for update", - "delete /* pk */ from vitess_a where (eid = 2 and id = 1)", + "delete /* pk */ from vitess_a where eid in (2) and id in (1, 2) limit 10001", }, RowsAffected: 1, }, @@ -1377,8 +1352,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete /* pk */ from vitess_a where eid in (2, 3) and id in (1, 2)", Rewritten: []string{ - "select eid, id from vitess_a where eid in (2, 3) and id in (1, 2) limit 10001 for update", - "delete /* pk */ from vitess_a where (eid = 2 and id = 1)", + "delete /* pk */ from vitess_a where eid in (2, 3) and id in (1, 2) limit 10001", }, RowsAffected: 1, }, @@ -1396,8 +1370,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete from vitess_a where eid = 1+1 and id = 1", Rewritten: []string{ - "select eid, id from vitess_a where eid = 1 + 1 and id = 1 limit 10001 for update", - "delete from vitess_a where (eid = 2 and id = 1)", + "delete from vitess_a where eid = 1 + 1 and id = 1 limit 10001", }, RowsAffected: 1, }, @@ -1415,8 +1388,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete from vitess_a where eid = 2", Rewritten: []string{ - "select eid, id from vitess_a where eid = 2 limit 10001 for update", - "delete from vitess_a where (eid = 2 and id = 1)", + "delete from vitess_a where eid = 2 limit 10001", }, RowsAffected: 1, }, @@ -1434,8 +1406,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete from vitess_a where eid = 2 limit 1", Rewritten: []string{ - "select eid, id from vitess_a where eid = 2 limit 1 for update", - "delete from vitess_a where (eid = 2 and id = 1)", + "delete from vitess_a where eid = 2 limit 1", }, RowsAffected: 1, }, @@ -1454,8 +1425,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "delete from vitess_a where eid = 2 order by id desc", Rewritten: []string{ - "select eid, id from vitess_a where eid = 2 order by id desc limit 10001 for update", - "delete from vitess_a where (eid = 2 and id = 2) or (eid = 2 and id = 1) order by id desc", + "delete from vitess_a where eid = 2 order by id desc limit 10001", }, RowsAffected: 2, }, @@ -1485,7 +1455,7 @@ func TestQueries(t *testing.T) { "mediumu": sqltypes.Int64BindVariable(16777215), }, Rewritten: []string{ - "insert into vitess_ints(tiny, tinyu, small, smallu, medium, mediumu, normal, normalu, big, bigu, y) values (-128, 255, -32768, 65535, -8388608, 16777215, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 2012)", + "insert into vitess_ints values (-128, 255, -32768, 65535, -8388608, 16777215, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 2012)", }, }, framework.TestQuery("commit"), @@ -1512,8 +1482,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert into vitess_ints select 2, tinyu, small, smallu, medium, mediumu, normal, normalu, big, bigu, y from vitess_ints", Rewritten: []string{ - "select 2, tinyu, small, smallu, medium, mediumu, normal, normalu, big, bigu, y from vitess_ints limit 10001", - "insert into vitess_ints(tiny, tinyu, small, smallu, medium, mediumu, normal, normalu, big, bigu, y) values (2, 255, -32768, 65535, -8388608, 16777215, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 2012)", + "insert into vitess_ints select 2, tinyu, small, smallu, medium, mediumu, normal, normalu, big, bigu, y from vitess_ints", }, }, framework.TestQuery("commit"), @@ -1536,7 +1505,7 @@ func TestQueries(t *testing.T) { "deci": sqltypes.StringBindVariable("1.99"), }, Rewritten: []string{ - "insert into vitess_fracts(id, deci, num, f, d) values (1, '1.99', '2.99', 3.99, 4.99)", + "insert into vitess_fracts values (1, '1.99', '2.99', 3.99, 4.99)", }, }, framework.TestQuery("commit"), @@ -1563,8 +1532,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert into vitess_fracts select 2, deci, num, f, d from vitess_fracts", Rewritten: []string{ - "select 2, deci, num, f, d from vitess_fracts limit 10001", - "insert into vitess_fracts(id, deci, num, f, d) values (2, 1.99, 2.99, 3.99, 4.99)", + "insert into vitess_fracts select 2, deci, num, f, d from vitess_fracts", }, }, framework.TestQuery("commit"), @@ -1592,7 +1560,7 @@ func TestQueries(t *testing.T) { "c": sqltypes.StringBindVariable("b"), }, Rewritten: []string{ - "insert into vitess_strings(vb, c, vc, b, tb, bl, ttx, tx, en, s) values ('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')", + "insert into vitess_strings values ('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')", }, }, framework.TestQuery("commit"), @@ -1619,8 +1587,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "insert into vitess_strings select 'b', c, vc, b, tb, bl, ttx, tx, en, s from vitess_strings", Rewritten: []string{ - "select 'b', c, vc, b, tb, bl, ttx, tx, en, s from vitess_strings limit 10001", - "insert into vitess_strings(vb, c, vc, b, tb, bl, ttx, tx, en, s) values ('b', 'b', 'c', 'd\\0\\0\\0', 'e', 'f', 'g', 'h', 'a', 'a,b')", + "insert into vitess_strings select 'b', c, vc, b, tb, bl, ttx, tx, en, s from vitess_strings", }, }, framework.TestQuery("commit"), @@ -1643,7 +1610,7 @@ func TestQueries(t *testing.T) { "d": sqltypes.StringBindVariable("2012-01-01"), }, Rewritten: []string{ - "insert into vitess_misc(id, b, d, dt, t, g) values (1, '\x01', '2012-01-01', '2012-01-01 15:45:45', '15:45:45', point(1, 2))", + "insert into vitess_misc values (1, '\x01', '2012-01-01', '2012-01-01 15:45:45', '15:45:45', point(1, 2))", }, }, framework.TestQuery("commit"), @@ -1671,8 +1638,7 @@ func TestQueries(t *testing.T) { // Skip geometry test. The binary representation is non-trivial to represent as go string. Query: "insert into vitess_misc(id, b, d, dt, t) select 2, b, d, dt, t from vitess_misc", Rewritten: []string{ - "select 2, b, d, dt, t from vitess_misc limit 10001", - "insert into vitess_misc(id, b, d, dt, t) values (2, b'00000001', '2012-01-01', '2012-01-01 15:45:45', '15:45:45')", + "insert into vitess_misc(id, b, d, dt, t) select 2, b, d, dt, t from vitess_misc", }, }, framework.TestQuery("commit"), @@ -1752,8 +1718,7 @@ func TestQueries(t *testing.T) { &framework.TestCase{ Query: "update vitess_bool set sval = 'test' where bval is false or ival = 23", Rewritten: []string{ - "select auto from vitess_bool where bval is false or ival = 23 limit 10001 for update", - "update vitess_bool set sval = 'test' where auto in (1, 2, 6, 7, 8, 9)", + "update vitess_bool set sval = 'test' where bval is false or ival = 23 limit 10001", }, RowsAffected: 6, }, @@ -1821,40 +1786,3 @@ func TestQueries(t *testing.T) { } } } - -func TestBitDefault(t *testing.T) { - // Default values for bit fields that are PKs are not supported - // Does not make sense to use a bit field as PK - client := framework.NewClient() - - expectedError := "bit default value: Execute failed: could not create default row for insert without row values: cannot convert value BIT(\"\\x05\") to AST (CallerID: dev)" - testCases := []framework.Testable{ - &framework.MultiCase{ - Name: "bit default value", - Cases: []framework.Testable{ - framework.TestQuery("begin"), - &framework.TestCase{ - Query: "insert into vitess_bit_default values()", - Rewritten: []string{ - "insert into vitess_bit_default(id) values ('\x05')", - }, - RowsAffected: 1, - }, - framework.TestQuery("commit"), - &framework.TestCase{ - Query: "select hex(id) from vitess_bit_default", - Result: [][]string{ - {"5"}, - }, - RowsAffected: 1, - }, - }, - }, - } - for _, tcase := range testCases { - err := tcase.Test("", client) - if err == nil || err.Error() != expectedError { - t.Errorf("TestBitDefault result: \n%q\nexpecting\n%q", err.Error(), expectedError) - } - } -} diff --git a/go/vt/vttablet/endtoend/transaction_test.go b/go/vt/vttablet/endtoend/transaction_test.go index eecb7d74526..78e811bb574 100644 --- a/go/vt/vttablet/endtoend/transaction_test.go +++ b/go/vt/vttablet/endtoend/transaction_test.go @@ -119,13 +119,13 @@ func TestCommit(t *testing.T) { tag: "Queries/Histograms/COMMIT/Count", diff: 1, }, { - tag: "Queries/Histograms/INSERT_PK/Count", + tag: "Queries/Histograms/Insert/Count", diff: 1, }, { - tag: "Queries/Histograms/DML_PK/Count", + tag: "Queries/Histograms/DeleteLimit/Count", diff: 1, }, { - tag: "Queries/Histograms/PASS_SELECT/Count", + tag: "Queries/Histograms/Select/Count", diff: 2, }} vend := framework.DebugVars() @@ -164,7 +164,7 @@ func TestRollback(t *testing.T) { t.Error(err) return } - want := []string{"insert into vitess_test(intval, floatval, charval, binval) values (4, null, null, null)"} + want := []string{"insert into vitess_test values (4, null, null, null)"} if !reflect.DeepEqual(tx.Queries, want) { t.Errorf("queries: %v, want %v", tx.Queries, want) } @@ -197,7 +197,7 @@ func TestRollback(t *testing.T) { tag: "Queries/Histograms/ROLLBACK/Count", diff: 1, }, { - tag: "Queries/Histograms/INSERT_PK/Count", + tag: "Queries/Histograms/Insert/Count", diff: 1, }} vend := framework.DebugVars() @@ -282,13 +282,13 @@ func TestAutoCommit(t *testing.T) { tag: "Queries/Histograms/COMMIT/Count", diff: 0, }, { - tag: "Queries/Histograms/INSERT_PK/Count", + tag: "Queries/Histograms/Insert/Count", diff: 1, }, { - tag: "Queries/Histograms/DML_PK/Count", + tag: "Queries/Histograms/DeleteLimit/Count", diff: 1, }, { - tag: "Queries/Histograms/PASS_SELECT/Count", + tag: "Queries/Histograms/Select/Count", diff: 2, }} vend := framework.DebugVars() @@ -303,17 +303,6 @@ func TestAutoCommit(t *testing.T) { } } -func TestAutoCommitOff(t *testing.T) { - framework.Server.SetAutoCommit(false) - defer framework.Server.SetAutoCommit(true) - - _, err := framework.NewClient().Execute("insert into vitess_test values(4, null, null, null)", nil) - want := "INSERT_PK disallowed outside transaction" - if err == nil || !strings.HasPrefix(err.Error(), want) { - t.Errorf("%v, must start with %s", err, want) - } -} - func TestTxPoolSize(t *testing.T) { vstart := framework.DebugVars() @@ -398,7 +387,7 @@ func TestForUpdate(t *testing.T) { client := framework.NewClient() query := fmt.Sprintf("select * from vitess_test where intval=2 %s", mode) _, err := client.Execute(query, nil) - want := "SELECT_LOCK disallowed outside transaction" + want := "SelectLock disallowed outside transaction" if err == nil || !strings.HasPrefix(err.Error(), want) { t.Errorf("%v, must have prefix %s", err, want) } From e69b85c53f7902992808251a193747a8b744e994 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 18 Mar 2020 22:24:04 -0700 Subject: [PATCH 16/18] deprecation: fix other e2e tests Signed-off-by: Sugu Sougoumarane --- .../encrypted_transport_test.go | 6 +-- go/test/endtoend/messaging/message_test.go | 3 +- .../endtoend/mysqlserver/mysql_server_test.go | 2 +- .../endtoend/preparestmt/stmt_methods_test.go | 2 +- .../multi-output/deletesharded-output.txt | 38 +++++++----------- .../testdata/multi-output/options-output.txt | 4 +- .../multi-output/unsharded-output.txt | 8 ++-- .../multi-output/updatesharded-output.txt | 39 ++++++++----------- .../twopc-output/deletesharded-output.txt | 9 ++--- go/vt/vtexplain/vtexplain_flaky_test.go | 2 +- .../tabletserver/planbuilder/builder.go | 10 ++++- .../planbuilder/testdata/exec_cases.txt | 36 +++++++++++++---- 12 files changed, 83 insertions(+), 76 deletions(-) diff --git a/go/test/endtoend/encryption/encryptedtransport/encrypted_transport_test.go b/go/test/endtoend/encryption/encryptedtransport/encrypted_transport_test.go index 3bca70c58eb..9e38feb1689 100644 --- a/go/test/endtoend/encryption/encryptedtransport/encrypted_transport_test.go +++ b/go/test/endtoend/encryption/encryptedtransport/encrypted_transport_test.go @@ -194,7 +194,7 @@ func TestSecureTransport(t *testing.T) { err = vterrors.FromVTRPC(qr.Error) require.Error(t, err) assert.Contains(t, err.Error(), "table acl error") - assert.Contains(t, err.Error(), "cannot run PASS_SELECT on table") + assert.Contains(t, err.Error(), "cannot run Select on table") // now restart vtgate in the mode where we don't use SSL // for client connections, but we copy effective caller id @@ -218,7 +218,7 @@ func TestSecureTransport(t *testing.T) { err = vterrors.FromVTRPC(qr.Error) require.Error(t, err) assert.Contains(t, err.Error(), "table acl error") - assert.Contains(t, err.Error(), "cannot run PASS_SELECT on table") + assert.Contains(t, err.Error(), "cannot run Select on table") // 'vtgate client 1' is authorized to access vt_insert_test callerID := &vtrpc.CallerID{ @@ -238,7 +238,7 @@ func TestSecureTransport(t *testing.T) { err = vterrors.FromVTRPC(qr.Error) require.Error(t, err) assert.Contains(t, err.Error(), "table acl error") - assert.Contains(t, err.Error(), "cannot run PASS_SELECT on table") + assert.Contains(t, err.Error(), "cannot run Select on table") clusterInstance.Teardown() } diff --git a/go/test/endtoend/messaging/message_test.go b/go/test/endtoend/messaging/message_test.go index 7bcdf056a19..667b77709a2 100644 --- a/go/test/endtoend/messaging/message_test.go +++ b/go/test/endtoend/messaging/message_test.go @@ -387,7 +387,8 @@ func TestMessageTopic(t *testing.T) { // this should fail because the topic doesn't exist. Any other outcome fails the test _, err = conn.ExecuteFetch("insert into test_topic(id, message) values(4, 'msg4'), (5, 'msg5'), (6, 'msg6')", 1, false) - want := "table test_topic not found in schema" + // 1146: table doesn't exist. + want := "errno 1146" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("non-topic insert err: %v, must contain %v", err, want) } diff --git a/go/test/endtoend/mysqlserver/mysql_server_test.go b/go/test/endtoend/mysqlserver/mysql_server_test.go index 656e0b509f5..192f6912fb1 100644 --- a/go/test/endtoend/mysqlserver/mysql_server_test.go +++ b/go/test/endtoend/mysqlserver/mysql_server_test.go @@ -194,7 +194,7 @@ func TestSelectWithUnauthorizedUser(t *testing.T) { _, err = conn.ExecuteFetch("SELECT * from vt_insert_test limit 1", 1, false) require.NotNilf(t, err, "error expected, got nil") assert.Contains(t, err.Error(), "table acl error") - assert.Contains(t, err.Error(), "cannot run PASS_SELECT on table") + assert.Contains(t, err.Error(), "cannot run Select on table") } func connectDB(t *testing.T, vtParams mysql.ConnParams, params ...string) *sql.DB { diff --git a/go/test/endtoend/preparestmt/stmt_methods_test.go b/go/test/endtoend/preparestmt/stmt_methods_test.go index d631b904068..7d4b84e9922 100644 --- a/go/test/endtoend/preparestmt/stmt_methods_test.go +++ b/go/test/endtoend/preparestmt/stmt_methods_test.go @@ -180,5 +180,5 @@ func TestWrongTableName(t *testing.T) { defer cluster.PanicHandler(t) dbo := Connect(t) defer dbo.Close() - execWithError(t, dbo, []uint16{1105}, "select * from teseting_table;") + execWithError(t, dbo, []uint16{1146}, "select * from teseting_table;") } diff --git a/go/vt/vtexplain/testdata/multi-output/deletesharded-output.txt b/go/vt/vtexplain/testdata/multi-output/deletesharded-output.txt index 554453ec54a..ed6ca3710c8 100644 --- a/go/vt/vtexplain/testdata/multi-output/deletesharded-output.txt +++ b/go/vt/vtexplain/testdata/multi-output/deletesharded-output.txt @@ -2,15 +2,14 @@ delete from music_extra where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: delete from music_extra where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: delete from music_extra where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 1 ks_sharded/-40: commit ---------------------------------------------------------------------- delete from music_extra where id=1 and extra='abc' 1 ks_sharded/-40: begin -1 ks_sharded/-40: select id from music_extra where id = 1 and extra = 'abc' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -1 ks_sharded/-40: delete from music_extra where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: delete from music_extra where id = 1 and extra = 'abc' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 1 ks_sharded/-40: commit ---------------------------------------------------------------------- @@ -19,8 +18,8 @@ delete from user where id=1 1 ks_sharded/-40: begin 1 ks_sharded/-40: select id, name from user where id = 1 limit 10001 for update 2 ks_sharded/40-80: begin -2 ks_sharded/40-80: delete from name_user_map where (name = 'name_val_2' and user_id = 1) /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ -3 ks_sharded/-40: delete from user where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +2 ks_sharded/40-80: delete from name_user_map where name = 'name_val_2' and user_id = 1 limit 10001 /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ +3 ks_sharded/-40: delete from user where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 4 ks_sharded/-40: commit 5 ks_sharded/40-80: commit @@ -32,9 +31,8 @@ delete from user where name='billy' 2 ks_sharded/-40: begin 2 ks_sharded/-40: select id, name from user where name = 'billy' limit 10001 for update 3 ks_sharded/40-80: begin -3 ks_sharded/40-80: delete from name_user_map where (name = 'name_val_2' and user_id = 1) /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ -4 ks_sharded/-40: select id from user where name = 'billy' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -4 ks_sharded/-40: delete from user where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +3 ks_sharded/40-80: delete from name_user_map where name = 'name_val_2' and user_id = 1 limit 10001 /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ +4 ks_sharded/-40: delete from user where name = 'billy' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 5 ks_sharded/c0-: commit 6 ks_sharded/-40: commit 7 ks_sharded/40-80: commit @@ -43,40 +41,32 @@ delete from user where name='billy' delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra='abc' 1 ks_sharded/-40: begin -1 ks_sharded/-40: select id from music_extra where extra = 'abc' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/-40: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/-40: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/-40: commit 1 ks_sharded/40-80: begin -1 ks_sharded/40-80: select id from music_extra where extra = 'abc' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/40-80: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/40-80: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/40-80: commit 1 ks_sharded/80-c0: begin -1 ks_sharded/80-c0: select id from music_extra where extra = 'abc' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/80-c0: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/80-c0: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/80-c0: commit 1 ks_sharded/c0-: begin -1 ks_sharded/c0-: select id from music_extra where extra = 'abc' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/c0-: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/c0-: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/c0-: commit ---------------------------------------------------------------------- delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from `ks_sharded[-]`.music_extra where extra='abc' LIMIT 10 1 ks_sharded/-40: begin -1 ks_sharded/-40: select id from music_extra where extra = 'abc' limit 10 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/-40: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/-40: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/-40: commit 1 ks_sharded/40-80: begin -1 ks_sharded/40-80: select id from music_extra where extra = 'abc' limit 10 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/40-80: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/40-80: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/40-80: commit 1 ks_sharded/80-c0: begin -1 ks_sharded/80-c0: select id from music_extra where extra = 'abc' limit 10 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/80-c0: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/80-c0: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/80-c0: commit 1 ks_sharded/c0-: begin -1 ks_sharded/c0-: select id from music_extra where extra = 'abc' limit 10 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/c0-: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where id in (1)/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/c0-: delete /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ from music_extra where extra = 'abc' limit 10/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/c0-: commit ---------------------------------------------------------------------- diff --git a/go/vt/vtexplain/testdata/multi-output/options-output.txt b/go/vt/vtexplain/testdata/multi-output/options-output.txt index 5e763a2e749..fa6f2610656 100644 --- a/go/vt/vtexplain/testdata/multi-output/options-output.txt +++ b/go/vt/vtexplain/testdata/multi-output/options-output.txt @@ -17,9 +17,9 @@ select * from user where id in (1,2,3,4,5,6,7,8) insert into user (id, name) values (2, 'bob') 1 ks_sharded/c0-: begin -1 ks_sharded/c0-: insert into name_user_map(name, user_id) values ('bob', 2) /* _stream name_user_map (name user_id ) ('Ym9i' 2 ); */ /* vtgate:: keyspace_id:da8a82595aa28154c17717955ffeed8b */ +1 ks_sharded/c0-: insert into name_user_map(name, user_id) values ('bob', 2) /* vtgate:: keyspace_id:da8a82595aa28154c17717955ffeed8b */ 2 ks_sharded/-40: begin -2 ks_sharded/-40: insert into user(id, name) values (2, 'bob') /* _stream user (id ) (2 ); */ /* vtgate:: keyspace_id:06e7ea22ce92708f */ +2 ks_sharded/-40: insert into user(id, name) values (2, 'bob') /* vtgate:: keyspace_id:06e7ea22ce92708f */ 3 ks_sharded/c0-: commit 4 ks_sharded/-40: commit diff --git a/go/vt/vtexplain/testdata/multi-output/unsharded-output.txt b/go/vt/vtexplain/testdata/multi-output/unsharded-output.txt index b3b3560a58e..98bdad579b9 100644 --- a/go/vt/vtexplain/testdata/multi-output/unsharded-output.txt +++ b/go/vt/vtexplain/testdata/multi-output/unsharded-output.txt @@ -14,23 +14,21 @@ insert into t1 (id,intval,floatval) values (1,2,3.14) update t1 set intval = 10 1 ks_unsharded/-: begin -1 ks_unsharded/-: select id from t1 limit 10001 for update -1 ks_unsharded/-: update t1 set intval = 10 where id in (1) +1 ks_unsharded/-: update t1 set intval = 10 limit 10001 1 ks_unsharded/-: commit ---------------------------------------------------------------------- update t1 set floatval = 9.99 1 ks_unsharded/-: begin -1 ks_unsharded/-: select id from t1 limit 10001 for update -1 ks_unsharded/-: update t1 set floatval = 9.99 where id in (1) +1 ks_unsharded/-: update t1 set floatval = 9.99 limit 10001 1 ks_unsharded/-: commit ---------------------------------------------------------------------- delete from t1 where id = 100 1 ks_unsharded/-: begin -1 ks_unsharded/-: delete from t1 where id in (100) +1 ks_unsharded/-: delete from t1 where id = 100 limit 10001 1 ks_unsharded/-: commit ---------------------------------------------------------------------- diff --git a/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt b/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt index 7f143656dbd..95b03cc4327 100644 --- a/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt +++ b/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt @@ -2,7 +2,7 @@ update user set nickname='alice' where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: update user set nickname = 'alice' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: update user set nickname = 'alice' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 1 ks_sharded/-40: commit ---------------------------------------------------------------------- @@ -11,8 +11,7 @@ update user set nickname='alice' where name='alice' 1 ks_sharded/40-80: begin 1 ks_sharded/40-80: select user_id from name_user_map where name = 'alice' limit 10001 2 ks_sharded/-40: begin -2 ks_sharded/-40: select id from user where name = 'alice' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -2 ks_sharded/-40: update user set nickname = 'alice' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +2 ks_sharded/-40: update user set nickname = 'alice' where name = 'alice' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 3 ks_sharded/40-80: commit 4 ks_sharded/-40: commit @@ -20,7 +19,7 @@ update user set nickname='alice' where name='alice' update user set pet='fido' where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: update user set pet = 'fido' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: update user set pet = 'fido' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 1 ks_sharded/-40: commit ---------------------------------------------------------------------- @@ -29,10 +28,10 @@ update user set name='alicia' where id=1 1 ks_sharded/-40: begin 1 ks_sharded/-40: select id, name from user where id = 1 limit 10001 for update 2 ks_sharded/40-80: begin -2 ks_sharded/40-80: delete from name_user_map where (name = 'name_val_2' and user_id = 1) /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ +2 ks_sharded/40-80: delete from name_user_map where name = 'name_val_2' and user_id = 1 limit 10001 /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ 3 ks_sharded/c0-: begin 3 ks_sharded/c0-: insert into name_user_map(name, user_id) values ('alicia', 1) /* vtgate:: keyspace_id:e2821261367fbee90bb5cf72955146c6 */ -4 ks_sharded/-40: update user set name = 'alicia' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +4 ks_sharded/-40: update user set name = 'alicia' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 5 ks_sharded/-40: commit 6 ks_sharded/40-80: commit 7 ks_sharded/c0-: commit @@ -44,11 +43,10 @@ update user set name='alicia' where name='alice' 1 ks_sharded/40-80: select user_id from name_user_map where name = 'alice' limit 10001 2 ks_sharded/-40: begin 2 ks_sharded/-40: select id, name from user where name = 'alice' limit 10001 for update -3 ks_sharded/40-80: delete from name_user_map where (name = 'name_val_2' and user_id = 1) /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ +3 ks_sharded/40-80: delete from name_user_map where name = 'name_val_2' and user_id = 1 limit 10001 /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ 4 ks_sharded/c0-: begin 4 ks_sharded/c0-: insert into name_user_map(name, user_id) values ('alicia', 1) /* vtgate:: keyspace_id:e2821261367fbee90bb5cf72955146c6 */ -5 ks_sharded/-40: select id from user where name = 'alice' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -5 ks_sharded/-40: update user set name = 'alicia' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +5 ks_sharded/-40: update user set name = 'alicia' where name = 'alice' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 6 ks_sharded/40-80: commit 7 ks_sharded/-40: commit 8 ks_sharded/c0-: commit @@ -57,20 +55,16 @@ update user set name='alicia' where name='alice' update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname=1 where nickname != '' 1 ks_sharded/-40: begin -1 ks_sharded/-40: select name from name_info where nickname != '' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/-40: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where name in ('name_val_1')/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/-40: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where nickname != '' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/-40: commit 1 ks_sharded/40-80: begin -1 ks_sharded/40-80: select name from name_info where nickname != '' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/40-80: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where name in ('name_val_1')/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/40-80: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where nickname != '' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/40-80: commit 1 ks_sharded/80-c0: begin -1 ks_sharded/80-c0: select name from name_info where nickname != '' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/80-c0: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where name in ('name_val_1')/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/80-c0: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where nickname != '' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/80-c0: commit 1 ks_sharded/c0-: begin -1 ks_sharded/c0-: select name from name_info where nickname != '' limit 10001 for update/* vtgate:: filtered_replication_unfriendly */ -1 ks_sharded/c0-: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where name in ('name_val_1')/* vtgate:: filtered_replication_unfriendly */ +1 ks_sharded/c0-: update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname = 1 where nickname != '' limit 10001/* vtgate:: filtered_replication_unfriendly */ 1 ks_sharded/c0-: commit ---------------------------------------------------------------------- @@ -79,8 +73,7 @@ update user set pet='rover' where name='alice' 1 ks_sharded/40-80: begin 1 ks_sharded/40-80: select user_id from name_user_map where name = 'alice' limit 10001 2 ks_sharded/-40: begin -2 ks_sharded/-40: select id from user where name = 'alice' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -2 ks_sharded/-40: update user set pet = 'rover' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +2 ks_sharded/-40: update user set pet = 'rover' where name = 'alice' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 3 ks_sharded/40-80: commit 4 ks_sharded/-40: commit @@ -92,12 +85,12 @@ begin update user set nickname='alice' where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: update user set nickname = 'alice' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: update user set nickname = 'alice' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ ---------------------------------------------------------------------- update user set nickname='bob' where id=1 -2 ks_sharded/-40: update user set nickname = 'bob' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +2 ks_sharded/-40: update user set nickname = 'bob' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ ---------------------------------------------------------------------- commit @@ -112,13 +105,13 @@ begin update user set nickname='alice' where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: update user set nickname = 'alice' where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: update user set nickname = 'alice' where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ ---------------------------------------------------------------------- update user set nickname='bob' where id=3 2 ks_sharded/40-80: begin -2 ks_sharded/40-80: update user set nickname = 'bob' where id in (3) /* vtgate:: keyspace_id:4eb190c9a2fa169c */ +2 ks_sharded/40-80: update user set nickname = 'bob' where id = 3 limit 10001 /* vtgate:: keyspace_id:4eb190c9a2fa169c */ ---------------------------------------------------------------------- commit diff --git a/go/vt/vtexplain/testdata/twopc-output/deletesharded-output.txt b/go/vt/vtexplain/testdata/twopc-output/deletesharded-output.txt index 9270d59f3ac..f811b5dfd18 100644 --- a/go/vt/vtexplain/testdata/twopc-output/deletesharded-output.txt +++ b/go/vt/vtexplain/testdata/twopc-output/deletesharded-output.txt @@ -2,15 +2,14 @@ delete from music_extra where id=1 1 ks_sharded/-40: begin -1 ks_sharded/-40: delete from music_extra where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: delete from music_extra where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 2 ks_sharded/-40: commit ---------------------------------------------------------------------- delete from music_extra where id=1 and extra='abc' 1 ks_sharded/-40: begin -1 ks_sharded/-40: select id from music_extra where id = 1 and extra = 'abc' limit 10001 for update /* vtgate:: keyspace_id:166b40b44aba4bd6 */ -1 ks_sharded/-40: delete from music_extra where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +1 ks_sharded/-40: delete from music_extra where id = 1 and extra = 'abc' limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 2 ks_sharded/-40: commit ---------------------------------------------------------------------- @@ -19,8 +18,8 @@ delete from user where id=1 1 ks_sharded/-40: begin 1 ks_sharded/-40: select name from user where id = 1 limit 10001 for update 2 ks_sharded/80-c0: begin -2 ks_sharded/80-c0: delete from name_user_map where (name = 'name_val_1' and user_id = 1) /* vtgate:: keyspace_id:a6e89b54b129c33051b76db219595660 */ -3 ks_sharded/-40: delete from user where id in (1) /* vtgate:: keyspace_id:166b40b44aba4bd6 */ +2 ks_sharded/80-c0: delete from name_user_map where name = 'name_val_2' and user_id = 1 limit 10001 /* vtgate:: keyspace_id:73004f940e97faf0a1b54ec5586a090e */ +3 ks_sharded/-40: delete from user where id = 1 limit 10001 /* vtgate:: keyspace_id:166b40b44aba4bd6 */ 4 ks_sharded/-40: begin 4 ks_sharded/-40: insert into `_vt`.dt_state(dtid, state, time_created) values ('ks_sharded:-40:1515392388787015722', 1, 1515392388898391433) 4 ks_sharded/-40: insert into `_vt`.dt_participant(dtid, id, keyspace, shard) values ('ks_sharded:-40:1515392388787015722', 1, 'ks_sharded', '80-c0') diff --git a/go/vt/vtexplain/vtexplain_flaky_test.go b/go/vt/vtexplain/vtexplain_flaky_test.go index ef35da06890..8cd492c52d6 100644 --- a/go/vt/vtexplain/vtexplain_flaky_test.go +++ b/go/vt/vtexplain/vtexplain_flaky_test.go @@ -153,7 +153,7 @@ func TestErrors(t *testing.T) { { SQL: "SELECT * FROM table_not_in_schema", - Err: "target: ks_unsharded.-.master, used tablet: explainCell-0 (ks_unsharded/-): table table_not_in_schema not found in schema", + Err: "target: ks_unsharded.-.master, used tablet: explainCell-0 (ks_unsharded/-): unknown error: unable to resolve table name table_not_in_schema", }, } diff --git a/go/vt/vttablet/tabletserver/planbuilder/builder.go b/go/vt/vttablet/tabletserver/planbuilder/builder.go index 74fb0e4e42d..d2c5f9479a6 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/builder.go +++ b/go/vt/vttablet/tabletserver/planbuilder/builder.go @@ -73,7 +73,10 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan plan.WhereClause = buf.ParsedQuery() } - if PassthroughDMLs || upd.Limit != nil { + // If plan.Table==nil, it's likely a multi-table statement. + // MySQL doesn't allow limit clauses for multi-table dmls. + // If there's an explicity Limit, honor it. + if PassthroughDMLs || plan.Table == nil || upd.Limit != nil { plan.FullQuery = GenerateFullQuery(upd) return plan, nil } @@ -98,7 +101,10 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan plan.WhereClause = buf.ParsedQuery() } - if PassthroughDMLs || del.Limit != nil { + // If plan.Table==nil, it's likely a multi-table statement. + // MySQL doesn't allow limit clauses for multi-table dmls. + // If there's an explicity Limit, honor it. + if PassthroughDMLs || plan.Table == nil || del.Limit != nil { plan.FullQuery = GenerateFullQuery(del) return plan, nil } diff --git a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt index 32a47b833f9..045715451a9 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt +++ b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt @@ -492,7 +492,7 @@ options:PassthroughDMLs # cross-db update "update a.b set foo='foo' where name in ('a', 'b')" { - "PlanID": "UpdateLimit", + "PlanID": "Update", "TableName": "", "Permissions": [ { @@ -500,14 +500,14 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "update a.b set foo = 'foo' where name in ('a', 'b') limit :#maxLimit", + "FullQuery": "update a.b set foo = 'foo' where name in ('a', 'b')", "WhereClause": "where name in ('a', 'b')" } # update unknown table "update bogus set name='foo' where id=1" { - "PlanID": "UpdateLimit", + "PlanID": "Update", "TableName": "", "Permissions": [ { @@ -515,7 +515,7 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "update bogus set name = 'foo' where id = 1 limit :#maxLimit", + "FullQuery": "update bogus set name = 'foo' where id = 1", "WhereClause": "where id = 1" } @@ -538,7 +538,7 @@ options:PassthroughDMLs # multi-table update "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'" { - "PlanID": "UpdateLimit", + "PlanID": "Update", "TableName": "", "Permissions": [ { @@ -550,7 +550,7 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test' limit :#maxLimit", + "FullQuery": "update a, b set a.name = 'foo' where a.id = b.id and b.var = 'test'", "WhereClause": "where a.id = b.id and b.var = 'test'" } @@ -652,7 +652,7 @@ options:PassthroughDMLs # delete unknown table "delete from bogus" { - "PlanID": "DeleteLimit", + "PlanID": "Delete", "TableName": "", "Permissions": [ { @@ -660,7 +660,7 @@ options:PassthroughDMLs "Role": 1 } ], - "FullQuery": "delete from bogus limit :#maxLimit" + "FullQuery": "delete from bogus" } # delete unknown table @@ -678,6 +678,26 @@ options:PassthroughDMLs "FullQuery": "delete from bogus" } +# multi-table delete +"delete a, b from a, b where id = 1" +{ + "PlanID": "Delete", + "TableName": "", + "Permissions": [ + { + "TableName": "a", + "Role": 1 + }, + { + "TableName": "b", + "Role": 1 + } + ], + "FullQuery": "delete a, b from a, b where id = 1", + "WhereClause": "where id = 1" +} + + # delete with limit "delete from a limit 10" { From ebd6524e8b6f949cb81165da7b66694fece1b2e5 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Thu, 19 Mar 2020 22:14:02 -0700 Subject: [PATCH 17/18] deprecation: rollback on limit failure Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletserver/query_executor.go | 54 +++++++----- .../tabletserver/query_executor_test.go | 87 +++++++++++++++++++ go/vt/vttablet/tabletserver/tx_pool.go | 45 +++++++--- 3 files changed, 151 insertions(+), 35 deletions(-) diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 4a1818509f7..8dc5deece83 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -134,45 +134,51 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { case planbuilder.PlanSet, planbuilder.PlanOtherRead, planbuilder.PlanOtherAdmin: return qre.execOther() case planbuilder.PlanInsert, planbuilder.PlanUpdate, planbuilder.PlanDelete, planbuilder.PlanInsertMessage, planbuilder.PlanDDL: - return qre.execAsTransaction(true /* autocommit */, qre.txConnExec) + return qre.execAutocommit(qre.txConnExec) case planbuilder.PlanUpdateLimit, planbuilder.PlanDeleteLimit: - return qre.execAsTransaction(false /* autocommit */, qre.txConnExec) + return qre.execAsTransaction(qre.txConnExec) } return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s unexpected plan type", qre.plan.PlanID.String()) } -func (qre *QueryExecutor) execAsTransaction(autocommit bool, f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { - if autocommit { - if qre.options == nil { - qre.options = &querypb.ExecuteOptions{} - } - qre.options.TransactionIsolation = querypb.ExecuteOptions_AUTOCOMMIT +func (qre *QueryExecutor) execAutocommit(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { + if qre.options == nil { + qre.options = &querypb.ExecuteOptions{} } - conn, beginSQL, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) + qre.options.TransactionIsolation = querypb.ExecuteOptions_AUTOCOMMIT + conn, _, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) if err != nil { return nil, err } defer qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) - if beginSQL != "" { - qre.logStats.AddRewrittenSQL(beginSQL, time.Now()) - } - reply, err = f(conn) + return f(conn) +} - start := time.Now() +func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { + conn, beginSQL, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) if err != nil { - qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) - qre.logStats.AddRewrittenSQL("rollback", start) return nil, err } - commitSQL, err := qre.tsv.te.txPool.LocalCommit(qre.ctx, conn) + defer qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) + qre.logStats.AddRewrittenSQL(beginSQL, time.Now()) - // As above LocalCommit is a no-op for autocommmit so don't log anything. - if commitSQL != "" { - qre.logStats.AddRewrittenSQL(commitSQL, start) + reply, err = f(conn) + if err != nil { + // dbConn is nil, it means the transaction was aborted. + // If so, we should not relog the rollback. + // TODO(sougou): these txPool functions should take the logstats + // and log any statements they issue. This needs to be done as + // a separate refactor because it impacts lot of code. + if conn.dbConn != nil { + defer qre.logStats.AddRewrittenSQL("rollback", time.Now()) + qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) + } + return nil, err } - if err != nil { + defer qre.logStats.AddRewrittenSQL("commit", time.Now()) + if _, err := qre.tsv.te.txPool.LocalCommit(qre.ctx, conn); err != nil { return nil, err } return reply, nil @@ -228,7 +234,7 @@ func (qre *QueryExecutor) Stream(callback func(*sqltypes.Result) error) error { return err } defer txConn.Recycle() - conn = txConn.DBConn + conn = txConn.dbConn } else { dbConn, err := qre.getStreamConn() if err != nil { @@ -385,7 +391,7 @@ func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) { t.SequenceInfo.Lock() defer t.SequenceInfo.Unlock() if t.SequenceInfo.NextVal == 0 || t.SequenceInfo.NextVal+inc > t.SequenceInfo.LastVal { - _, err := qre.execAsTransaction(false /* autocommit */, func(conn *TxConnection) (*sqltypes.Result, error) { + _, err := qre.execAsTransaction(func(conn *TxConnection) (*sqltypes.Result, error) { query := fmt.Sprintf("select next_id, cache from %s where id = 0 for update", sqlparser.String(tableName)) qr, err := qre.execSQL(conn, query, false) if err != nil { @@ -473,6 +479,8 @@ func (qre *QueryExecutor) execDMLLimit(conn *TxConnection) (*sqltypes.Result, er return nil, err } if err := qre.verifyRowCount(int64(result.RowsAffected), maxrows); err != nil { + defer qre.logStats.AddRewrittenSQL("rollback", time.Now()) + qre.tsv.te.txPool.LocalConclude(qre.ctx, conn) return nil, err } return result, nil diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index aa2e38ceffc..9fce5aa53f3 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -307,6 +307,89 @@ func TestQueryExecutorSelectImpossible(t *testing.T) { } } +func TestQueryExecutorDMLLimitFailure(t *testing.T) { + type dbResponse struct { + query string + result *sqltypes.Result + } + + dmlResult := &sqltypes.Result{ + RowsAffected: 3, + } + + // The queries are run both in and outside a transaction. + testcases := []struct { + input string + passThrough bool + dbResponses []dbResponse + planWant string + logWant string + inTxWant string + }{{ + input: "update test_table set a=1", + dbResponses: []dbResponse{{ + query: "update test_table set a = 1 limit 3", + result: dmlResult, + }}, + logWant: "begin; update test_table set a = 1 limit 3; rollback", + inTxWant: "update test_table set a = 1 limit 3; rollback", + }, { + input: "delete from test_table", + dbResponses: []dbResponse{{ + query: "delete from test_table limit 3", + result: dmlResult, + }}, + logWant: "begin; delete from test_table limit 3; rollback", + inTxWant: "delete from test_table limit 3; rollback", + }} + for _, tcase := range testcases { + func() { + db := setUpQueryExecutorTest(t) + defer db.Close() + for _, dbr := range tcase.dbResponses { + db.AddQuery(dbr.query, dbr.result) + } + ctx := callerid.NewContext(context.Background(), callerid.NewEffectiveCallerID("a", "b", "c"), callerid.NewImmediateCallerID("d")) + tsv := newTestTabletServer(ctx, smallResultSize, db) + defer tsv.StopService() + + tsv.SetPassthroughDMLs(tcase.passThrough) + + // Test outside a transaction. + qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0) + _, err := qre.Execute() + wantErr := "caller id: d: row count exceeded 2 (errno 10001) (sqlstate HY000)" + if err == nil || err.Error() != wantErr { + t.Errorf("Execute(%v): %v, want %v", tcase.input, err, wantErr) + } + assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input) + + // Test inside a transaction. + txid, err := tsv.Begin(ctx, &tsv.target, nil) + require.NoError(t, err) + defer tsv.Commit(ctx, &tsv.target, txid) + + qre = newTestQueryExecutor(ctx, tsv, tcase.input, txid) + _, err = qre.Execute() + if err == nil || err.Error() != wantErr { + t.Errorf("Execute(%v): %v, want %v", tcase.input, err, wantErr) + } + want := tcase.logWant + if tcase.inTxWant != "" { + want = tcase.inTxWant + } + assert.Equal(t, want, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input) + + qre = newTestQueryExecutor(ctx, tsv, "update test_table set a=1", txid) + _, err = qre.Execute() + notxError := "ended at" + if err == nil || !strings.Contains(err.Error(), notxError) { + t.Errorf("Execute(%v): %v, must contain %v", tcase.input, err, notxError) + } + }() + } +} + func TestQueryExecutorPlanInsertMessage(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() @@ -938,6 +1021,7 @@ const ( smallTxPool noTwopc shortTwopcAge + smallResultSize ) // newTestQueryExecutor uses a package level variable testTabletServer defined in tabletserver_test.go @@ -967,6 +1051,9 @@ func newTestTabletServer(ctx context.Context, flags executorFlags, db *fakesqldb } else { config.TwoPCAbandonAge = 10 } + if flags&smallResultSize > 0 { + config.MaxResultSize = 2 + } tsv := NewTabletServer(config, memorytopo.NewServer(""), topodatapb.TabletAlias{}) testUtils := newTestUtils() dbconfigs := testUtils.newDBConfigs(db) diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go index 4d597e0108c..f5312e76b35 100644 --- a/go/vt/vttablet/tabletserver/tx_pool.go +++ b/go/vt/vttablet/tabletserver/tx_pool.go @@ -357,14 +357,19 @@ func (axp *TxPool) LocalCommit(ctx context.Context, conn *TxConnection) (string, // LocalConclude concludes a transaction started by LocalBegin. // If the transaction was not previously concluded, it's rolled back. func (axp *TxPool) LocalConclude(ctx context.Context, conn *TxConnection) { + if conn.dbConn == nil { + return + } span, ctx := trace.NewSpan(ctx, "TxPool.LocalConclude") defer span.Finish() - if conn.DBConn != nil { - _ = axp.localRollback(ctx, conn) - } + _ = axp.localRollback(ctx, conn) } func (axp *TxPool) localRollback(ctx context.Context, conn *TxConnection) error { + if conn.Autocommit { + conn.conclude(TxCommit, "returned to pool") + return nil + } defer conn.conclude(TxRollback, "transaction rolled back") if _, err := conn.Exec(ctx, "rollback", 1, false); err != nil { conn.Close() @@ -413,7 +418,7 @@ func (axp *TxPool) SetPoolTimeout(timeout time.Duration) { // the tx pool correctly. It also does not retry statements if there // are failures. type TxConnection struct { - *connpool.DBConn + dbConn *connpool.DBConn TransactionID int64 pool *TxPool StartTime time.Time @@ -428,7 +433,7 @@ type TxConnection struct { func newTxConnection(conn *connpool.DBConn, transactionID int64, pool *TxPool, immediate *querypb.VTGateCallerID, effective *vtrpcpb.CallerID, autocommit bool) *TxConnection { return &TxConnection{ - DBConn: conn, + dbConn: conn, TransactionID: transactionID, pool: pool, StartTime: time.Now(), @@ -438,9 +443,19 @@ func newTxConnection(conn *connpool.DBConn, transactionID int64, pool *TxPool, i } } +// Close closes the connection. +func (txc *TxConnection) Close() { + if txc.dbConn != nil { + txc.dbConn.Close() + } +} + // Exec executes the statement for the current transaction. func (txc *TxConnection) Exec(ctx context.Context, query string, maxrows int, wantfields bool) (*sqltypes.Result, error) { - r, err := txc.DBConn.ExecOnce(ctx, query, maxrows, wantfields) + if txc.dbConn == nil { + return nil, vterrors.Errorf(vtrpcpb.Code_ABORTED, "transaction was aborted: %v", txc.Conclusion) + } + r, err := txc.dbConn.ExecOnce(ctx, query, maxrows, wantfields) if err != nil { if mysql.IsConnErr(err) { select { @@ -458,13 +473,13 @@ func (txc *TxConnection) Exec(ctx context.Context, query string, maxrows int, wa // BeginAgain commits the existing transaction and begins a new one func (txc *TxConnection) BeginAgain(ctx context.Context) error { - if txc.Autocommit { + if txc.dbConn == nil || txc.Autocommit { return nil } - if _, err := txc.DBConn.Exec(ctx, "commit", 1, false); err != nil { + if _, err := txc.dbConn.Exec(ctx, "commit", 1, false); err != nil { return err } - if _, err := txc.DBConn.Exec(ctx, "begin", 1, false); err != nil { + if _, err := txc.dbConn.Exec(ctx, "begin", 1, false); err != nil { return err } return nil @@ -473,7 +488,10 @@ func (txc *TxConnection) BeginAgain(ctx context.Context) error { // Recycle returns the connection to the pool. The transaction remains // active. func (txc *TxConnection) Recycle() { - if txc.IsClosed() { + if txc.dbConn == nil { + return + } + if txc.dbConn.IsClosed() { txc.conclude(TxClose, "closed") } else { txc.pool.activePool.Put(txc.TransactionID) @@ -486,9 +504,12 @@ func (txc *TxConnection) RecordQuery(query string) { } func (txc *TxConnection) conclude(conclusion, reason string) { + if txc.dbConn == nil { + return + } txc.pool.activePool.Unregister(txc.TransactionID, reason) - txc.DBConn.Recycle() - txc.DBConn = nil + txc.dbConn.Recycle() + txc.dbConn = nil txc.pool.limiter.Release(txc.ImmediateCallerID, txc.EffectiveCallerID) txc.log(conclusion) } From ac2567fc76196d5c91497e5bba5bbc4d7660ea4d Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Fri, 20 Mar 2020 09:45:02 -0700 Subject: [PATCH 18/18] deprecation: address review comments Signed-off-by: Sugu Sougoumarane --- .../vttablet/tabletserver/planbuilder/plan.go | 5 ++ go/vt/vttablet/tabletserver/query_executor.go | 2 + .../tabletserver/query_executor_test.go | 72 ++++++++++++++----- 3 files changed, 62 insertions(+), 17 deletions(-) diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index a9b662a79c8..227523f8820 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -55,9 +55,12 @@ const ( PlanDeleteLimit PlanDDL PlanSet + // PlanOtherRead is for statements like show, etc. PlanOtherRead + // PlanOtherAdmin is for statements like repair, lock table, etc. PlanOtherAdmin PlanSelectStream + // PlanMessageStream is for "stream" statements. PlanMessageStream NumPlans ) @@ -170,6 +173,8 @@ func Build(statement sqlparser.Statement, tables map[string]*schema.Table) (*Pla case *sqlparser.Set: plan, err = analyzeSet(stmt), nil case *sqlparser.DDL: + // DDLs and other statements below don't get fully parsed. + // We have to use the original query at the time of execution. plan = &Plan{PlanID: PlanDDL} case *sqlparser.Show: plan, err = &Plan{PlanID: PlanOtherRead}, nil diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 8dc5deece83..88ce78e3c9b 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -100,6 +100,8 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) { case planbuilder.PlanNextval: return qre.execNextval() case planbuilder.PlanSelectImpossible: + // If the fields did not get cached, we have send the query + // to mysql, which you can see below. if qre.plan.Fields != nil { return &sqltypes.Result{ Fields: qre.plan.Fields, diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 9fce5aa53f3..8a7e0c758e2 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -307,7 +307,7 @@ func TestQueryExecutorSelectImpossible(t *testing.T) { } } -func TestQueryExecutorDMLLimitFailure(t *testing.T) { +func TestQueryExecutorLimitFailure(t *testing.T) { type dbResponse struct { query string result *sqltypes.Result @@ -316,31 +316,66 @@ func TestQueryExecutorDMLLimitFailure(t *testing.T) { dmlResult := &sqltypes.Result{ RowsAffected: 3, } + fields := sqltypes.MakeTestFields("a|b", "int64|varchar") + fieldResult := sqltypes.MakeTestResult(fields) + selectResult := sqltypes.MakeTestResult(fields, "1|aaa", "2|bbb", "3|ccc") // The queries are run both in and outside a transaction. testcases := []struct { - input string - passThrough bool - dbResponses []dbResponse - planWant string - logWant string - inTxWant string + input string + dbResponses []dbResponse + err string + logWant string + inTxWant string + testRollback bool }{{ + input: "select * from t", + dbResponses: []dbResponse{{ + query: "select * from t where 1 != 1", + result: fieldResult, + }, { + query: "select * from t limit 3", + result: selectResult, + }}, + err: "count exceeded", + logWant: "select * from t where 1 != 1; select * from t limit 3", + // Because the fields would have been cached before, the field query will + // not get re-executed. + inTxWant: "select * from t limit 3", + }, { input: "update test_table set a=1", dbResponses: []dbResponse{{ query: "update test_table set a = 1 limit 3", result: dmlResult, }}, - logWant: "begin; update test_table set a = 1 limit 3; rollback", - inTxWant: "update test_table set a = 1 limit 3; rollback", + err: "count exceeded", + logWant: "begin; update test_table set a = 1 limit 3; rollback", + inTxWant: "update test_table set a = 1 limit 3; rollback", + testRollback: true, }, { input: "delete from test_table", dbResponses: []dbResponse{{ query: "delete from test_table limit 3", result: dmlResult, }}, - logWant: "begin; delete from test_table limit 3; rollback", - inTxWant: "delete from test_table limit 3; rollback", + err: "count exceeded", + logWant: "begin; delete from test_table limit 3; rollback", + inTxWant: "delete from test_table limit 3; rollback", + testRollback: true, + }, { + // There should be no rollback on normal failures. + input: "update test_table set a=1", + dbResponses: nil, + err: "not supported", + logWant: "begin; update test_table set a = 1 limit 3; rollback", + inTxWant: "update test_table set a = 1 limit 3", + }, { + // There should be no rollback on normal failures. + input: "delete from test_table", + dbResponses: nil, + err: "not supported", + logWant: "begin; delete from test_table limit 3; rollback", + inTxWant: "delete from test_table limit 3", }} for _, tcase := range testcases { func() { @@ -353,14 +388,13 @@ func TestQueryExecutorDMLLimitFailure(t *testing.T) { tsv := newTestTabletServer(ctx, smallResultSize, db) defer tsv.StopService() - tsv.SetPassthroughDMLs(tcase.passThrough) + tsv.SetPassthroughDMLs(false) // Test outside a transaction. qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0) _, err := qre.Execute() - wantErr := "caller id: d: row count exceeded 2 (errno 10001) (sqlstate HY000)" - if err == nil || err.Error() != wantErr { - t.Errorf("Execute(%v): %v, want %v", tcase.input, err, wantErr) + if err == nil || !strings.Contains(err.Error(), tcase.err) { + t.Errorf("Execute(%v): %v, must contain %v", tcase.input, err, tcase.err) } assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input) @@ -371,8 +405,8 @@ func TestQueryExecutorDMLLimitFailure(t *testing.T) { qre = newTestQueryExecutor(ctx, tsv, tcase.input, txid) _, err = qre.Execute() - if err == nil || err.Error() != wantErr { - t.Errorf("Execute(%v): %v, want %v", tcase.input, err, wantErr) + if err == nil || !strings.Contains(err.Error(), tcase.err) { + t.Errorf("Execute(%v): %v, must contain %v", tcase.input, err, tcase.err) } want := tcase.logWant if tcase.inTxWant != "" { @@ -380,6 +414,10 @@ func TestQueryExecutorDMLLimitFailure(t *testing.T) { } assert.Equal(t, want, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input) + if !tcase.testRollback { + return + } + // Ensure transaction was rolled back. qre = newTestQueryExecutor(ctx, tsv, "update test_table set a=1", txid) _, err = qre.Execute() notxError := "ended at"