From 8f0bc14aafc563c7e23c566170f9b4f81a2db8e0 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 3 Jan 2018 11:46:10 -0800 Subject: [PATCH 01/72] add an error log if creating the app connection fails --- go/vt/vttablet/tabletserver/tabletserver.go | 1 + 1 file changed, 1 insertion(+) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index f5add2a1eb7..3d3c3572ffc 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -440,6 +440,7 @@ func (tsv *TabletServer) decideAction(tabletType topodatapb.TabletType, serving func (tsv *TabletServer) fullStart() (err error) { c, err := dbconnpool.NewDBConnection(&tsv.dbconfigs.App, tabletenv.MySQLStats) if err != nil { + log.Errorf("error creating db app connection: %v", err) return err } c.Close() From f786a06df288bd096c78677591afff2708a4234b Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Mon, 8 Jan 2018 09:25:45 -0800 Subject: [PATCH 02/72] add a reusable mysqlproxy module Modeling after the vtgate mysql server implementation, add a simple module that implements the mysql server protocol handlers and proxies all requests to the appropriate methods in a queryservice backend. This includes optional support for query normalization. --- go/vt/mysqlproxy/mysqlproxy.go | 156 +++++++++++++++++++++++++++++++++ 1 file changed, 156 insertions(+) create mode 100644 go/vt/mysqlproxy/mysqlproxy.go diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go new file mode 100644 index 00000000000..9503b572c6a --- /dev/null +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -0,0 +1,156 @@ +/* +Copyright 2017 Google Inc. + +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 mysqlproxy is a basic module that proxies a mysql server +// session to appropriate calls in a queryservice back end, with optional +// query normalization. +package mysqlproxy + +import ( + "context" + "fmt" + + log "github.com/golang/glog" + + "github.com/youtube/vitess/go/sqltypes" + "github.com/youtube/vitess/go/vt/sqlparser" + "github.com/youtube/vitess/go/vt/vttablet/queryservice" + + querypb "github.com/youtube/vitess/go/vt/proto/query" +) + +// ProxySession holds session state for the proxy +type ProxySession struct { + TransactionID int64 + TargetString string + Options *querypb.ExecuteOptions + Autocommit bool +} + +// Proxy wraps the standalone query service +type Proxy struct { + target *querypb.Target + qs queryservice.QueryService + normalize bool +} + +// NewProxy creates a new proxy +func NewProxy(target *querypb.Target, qs queryservice.QueryService, normalize bool) *Proxy { + return &Proxy{ + target: target, + qs: qs, + normalize: normalize, + } +} + +// Execute runs the given sql query in the specified session +func (mp *Proxy) Execute(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*ProxySession, *sqltypes.Result, error) { + var err error + result := &sqltypes.Result{} + + switch sqlparser.Preview(sql) { + case sqlparser.StmtBegin: + err = mp.doBegin(ctx, session) + case sqlparser.StmtCommit: + err = mp.doCommit(ctx, session) + case sqlparser.StmtRollback: + err = mp.doRollback(ctx, session) + case sqlparser.StmtSet: + result, err = mp.doSet(ctx, session, sql, bindVariables) + default: + result, err = mp.doExecute(ctx, session, sql, bindVariables) + } + + if err != nil { + return nil, nil, err + } + + return session, result, nil +} + +// Rollback rolls back the session +func (mp *Proxy) Rollback(ctx context.Context, session *ProxySession) error { + return mp.doRollback(ctx, session) +} + +func (mp *Proxy) doBegin(ctx context.Context, session *ProxySession) error { + txID, err := mp.qs.Begin(ctx, mp.target, session.Options) + if err != nil { + return err + } + session.TransactionID = txID + return nil +} + +func (mp *Proxy) doCommit(ctx context.Context, session *ProxySession) error { + if session.TransactionID == 0 { + return fmt.Errorf("commit: no open transaction") + + } + err := mp.qs.Commit(ctx, mp.target, session.TransactionID) + session.TransactionID = 0 + return err +} + +// Rollback rolls back the session +func (mp *Proxy) doRollback(ctx context.Context, session *ProxySession) error { + if session.TransactionID != 0 { + err := mp.qs.Rollback(ctx, mp.target, session.TransactionID) + session.TransactionID = 0 + return err + } + return nil +} + +// Set is currently ignored +func (mp *Proxy) doSet(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + vals, charset, err := sqlparser.ExtractSetValues(sql) + if err != nil { + return nil, err + } + if len(vals) > 0 && charset != "" { + return nil, err + } + + switch charset { + case "", "utf8", "utf8mb4", "latin1", "default": + break + default: + return nil, fmt.Errorf("unexpected value for charset: %v", charset) + } + + for k, v := range vals { + log.Warningf("Ignored inapplicable SET %v = %v", k, v) + } + + return &sqltypes.Result{}, nil +} + +// doExecute runs the given query +func (mp *Proxy) doExecute(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + if mp.normalize { + query, comments := sqlparser.SplitTrailingComments(sql) + stmt, err := sqlparser.Parse(query) + if err != nil { + return nil, err + } + sqlparser.Normalize(stmt, bindVariables, "vtp") + normalized := sqlparser.String(stmt) + sql = normalized + comments + } + + return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) +} From 5e7254ee614dac092de3c319fffb0d577ce639e4 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Mon, 8 Jan 2018 13:19:08 -0800 Subject: [PATCH 03/72] add vtqueryserver Add a binary to wrap a standalone queryservice with a grpc service interface and a mysql server implementation using the mysqlproxy module to direct to the queryservice. --- go/cmd/vtqueryserver/index.go | 31 +++ go/cmd/vtqueryserver/plugin_auth_static.go | 28 +++ .../vtqueryserver/plugin_grpcqueryservice.go | 34 +++ go/cmd/vtqueryserver/vtqueryserver.go | 71 ++++++ go/vt/vtqueryserver/plugin_mysql_server.go | 236 ++++++++++++++++++ .../vtqueryserver/plugin_mysql_server_test.go | 157 ++++++++++++ go/vt/vtqueryserver/status.go | 90 +++++++ go/vt/vtqueryserver/vtqueryserver.go | 78 ++++++ 8 files changed, 725 insertions(+) create mode 100644 go/cmd/vtqueryserver/index.go create mode 100644 go/cmd/vtqueryserver/plugin_auth_static.go create mode 100644 go/cmd/vtqueryserver/plugin_grpcqueryservice.go create mode 100644 go/cmd/vtqueryserver/vtqueryserver.go create mode 100644 go/vt/vtqueryserver/plugin_mysql_server.go create mode 100644 go/vt/vtqueryserver/plugin_mysql_server_test.go create mode 100644 go/vt/vtqueryserver/status.go create mode 100644 go/vt/vtqueryserver/vtqueryserver.go diff --git a/go/cmd/vtqueryserver/index.go b/go/cmd/vtqueryserver/index.go new file mode 100644 index 00000000000..72b2637abf0 --- /dev/null +++ b/go/cmd/vtqueryserver/index.go @@ -0,0 +1,31 @@ +/* +Copyright 2017 Google Inc. + +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 main + +import ( + "net/http" +) + +// This is a separate file so it can be selectively included/excluded from +// builds to opt in/out of the redirect. + +func init() { + // Anything unrecognized gets redirected to the status page. + http.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { + http.Redirect(w, r, "/debug/status", http.StatusFound) + }) +} diff --git a/go/cmd/vtqueryserver/plugin_auth_static.go b/go/cmd/vtqueryserver/plugin_auth_static.go new file mode 100644 index 00000000000..f95882338c7 --- /dev/null +++ b/go/cmd/vtqueryserver/plugin_auth_static.go @@ -0,0 +1,28 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 main + +// This plugin imports staticauthserver to register the flat-file implementation of AuthServer. + +import ( + "github.com/youtube/vitess/go/mysql" + "github.com/youtube/vitess/go/vt/vtqueryserver" +) + +func init() { + vtqueryserver.RegisterPluginInitializer(func() { mysql.InitAuthServerStatic() }) +} diff --git a/go/cmd/vtqueryserver/plugin_grpcqueryservice.go b/go/cmd/vtqueryserver/plugin_grpcqueryservice.go new file mode 100644 index 00000000000..0580c5310b3 --- /dev/null +++ b/go/cmd/vtqueryserver/plugin_grpcqueryservice.go @@ -0,0 +1,34 @@ +/* +Copyright 2017 Google Inc. + +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 main + +import ( + "github.com/youtube/vitess/go/vt/servenv" + "github.com/youtube/vitess/go/vt/vttablet/grpcqueryservice" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver" +) + +// Imports and register the gRPC queryservice server + +func init() { + tabletserver.RegisterFunctions = append(tabletserver.RegisterFunctions, func(qsc tabletserver.Controller) { + if servenv.GRPCCheckServiceMap("queryservice") { + grpcqueryservice.Register(servenv.GRPCServer, qsc.QueryService()) + } + }) + +} diff --git a/go/cmd/vtqueryserver/vtqueryserver.go b/go/cmd/vtqueryserver/vtqueryserver.go new file mode 100644 index 00000000000..eed1ae3a77d --- /dev/null +++ b/go/cmd/vtqueryserver/vtqueryserver.go @@ -0,0 +1,71 @@ +/* +Copyright 2017 Google Inc. + +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 main + +import ( + "flag" + "os" + + log "github.com/golang/glog" + "github.com/youtube/vitess/go/vt/dbconfigs" + "github.com/youtube/vitess/go/vt/servenv" + "github.com/youtube/vitess/go/vt/vtqueryserver" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" +) + +var ( + mysqlSocketFile = flag.String("mysql-socket-file", "", "path to unix socket file to connect to mysql") +) + +func init() { + servenv.RegisterDefaultFlags() +} + +func main() { + dbconfigFlags := dbconfigs.AppConfig | dbconfigs.AppDebugConfig + dbconfigs.RegisterFlags(dbconfigFlags) + flag.Parse() + + if *servenv.Version { + servenv.AppVersion.Print() + os.Exit(0) + } + + if len(flag.Args()) > 0 { + flag.Usage() + log.Exit("vtqueryserver doesn't take any positional arguments") + } + if err := tabletenv.VerifyConfig(); err != nil { + log.Exitf("invalid config: %v", err) + } + + tabletenv.Init() + + servenv.Init() + + dbcfgs, err := dbconfigs.Init(*mysqlSocketFile, dbconfigFlags) + if err != nil { + log.Fatal(err) + } + + err = vtqueryserver.Init(dbcfgs) + if err != nil { + log.Exitf("error initializing proxy: %v", err) + } + + servenv.RunDefault() +} diff --git a/go/vt/vtqueryserver/plugin_mysql_server.go b/go/vt/vtqueryserver/plugin_mysql_server.go new file mode 100644 index 00000000000..41d39af98ab --- /dev/null +++ b/go/vt/vtqueryserver/plugin_mysql_server.go @@ -0,0 +1,236 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 vtqueryserver + +import ( + "flag" + "fmt" + "net" + "os" + "syscall" + + log "github.com/golang/glog" + "golang.org/x/net/context" + + "github.com/youtube/vitess/go/mysql" + "github.com/youtube/vitess/go/sqltypes" + "github.com/youtube/vitess/go/vt/callerid" + "github.com/youtube/vitess/go/vt/mysqlproxy" + "github.com/youtube/vitess/go/vt/servenv" + "github.com/youtube/vitess/go/vt/vttls" + + querypb "github.com/youtube/vitess/go/vt/proto/query" +) + +var ( + mysqlServerPort = flag.Int("mysqlproxy_server_port", -1, "If set, also listen for MySQL binary protocol connections on this port.") + mysqlServerBindAddress = flag.String("mysqlproxy_server_bind_address", "", "Binds on this address when listening to MySQL binary protocol. Useful to restrict listening to 'localhost' only for instance.") + mysqlServerSocketPath = flag.String("mysqlproxy_server_socket_path", "", "This option specifies the Unix socket file to use when listening for local connections. By default it will be empty and it won't listen to a unix socket") + mysqlAuthServerImpl = flag.String("mysql_auth_server_impl", "static", "Which auth server implementation to use.") + mysqlAllowClearTextWithoutTLS = flag.Bool("mysql_allow_clear_text_without_tls", false, "If set, the server will allow the use of a clear text password over non-SSL connections.") + + mysqlSslCert = flag.String("mysqlproxy_server_ssl_cert", "", "Path to the ssl cert for mysql server plugin SSL") + mysqlSslKey = flag.String("mysqlproxy_server_ssl_key", "", "Path to ssl key for mysql server plugin SSL") + mysqlSslCa = flag.String("mysqlproxy_server_ssl_ca", "", "Path to ssl CA for mysql server plugin SSL. If specified, server will require and validate client certs.") + + mysqlSlowConnectWarnThreshold = flag.Duration("mysqlproxy_slow_connect_warn_threshold", 0, "Warn if it takes more than the given threshold for a mysql connection to establish") +) + +// proxyHandler implements the Listener interface. +// It stores the Session in the ClientData of a Connection, if a transaction +// is in progress. +type proxyHandler struct { + mp *mysqlproxy.Proxy +} + +func newProxyHandler(mp *mysqlproxy.Proxy) *proxyHandler { + return &proxyHandler{ + mp: mp, + } +} + +func (mh *proxyHandler) NewConnection(c *mysql.Conn) { +} + +func (mh *proxyHandler) ConnectionClosed(c *mysql.Conn) { + // Rollback if there is an ongoing transaction. Ignore error. + ctx := context.Background() + session, _ := c.ClientData.(*mysqlproxy.ProxySession) + if session != nil && session.TransactionID != 0 { + _ = mh.mp.Rollback(ctx, session) + } +} + +func (mh *proxyHandler) ComQuery(c *mysql.Conn, query string, callback func(*sqltypes.Result) error) error { + // FIXME(alainjobart): Add some kind of timeout to the context. + ctx := context.Background() + + // Fill in the ImmediateCallerID with the UserData returned by + // the AuthServer plugin for that user. If nothing was + // returned, use the User. This lets the plugin map a MySQL + // user used for authentication to a Vitess User used for + // Table ACLs and Vitess authentication in general. + im := c.UserData.Get() + ef := callerid.NewEffectiveCallerID( + c.User, /* principal: who */ + c.RemoteAddr().String(), /* component: running client process */ + "mysqlproxy MySQL Connector" /* subcomponent: part of the client */) + ctx = callerid.NewContext(ctx, ef, im) + + session, _ := c.ClientData.(*mysqlproxy.ProxySession) + if session == nil { + session = &mysqlproxy.ProxySession{ + Options: &querypb.ExecuteOptions{ + IncludedFields: querypb.ExecuteOptions_ALL, + }, + Autocommit: true, + } + if c.Capabilities&mysql.CapabilityClientFoundRows != 0 { + session.Options.ClientFoundRows = true + } + } + if c.SchemaName != "" { + session.TargetString = c.SchemaName + } + session, result, err := mh.mp.Execute(ctx, session, query, make(map[string]*querypb.BindVariable)) + c.ClientData = session + err = mysql.NewSQLErrorFromError(err) + if err != nil { + return err + } + + return callback(result) +} + +var mysqlListener *mysql.Listener +var mysqlUnixListener *mysql.Listener + +// initiMySQLProtocol starts the mysql protocol. +// It should be called only once in a process. +func initMySQLProtocol() { + log.Infof("initializing mysql protocol") + + // Flag is not set, just return. + if *mysqlServerPort < 0 && *mysqlServerSocketPath == "" { + return + } + + // If no mysqlproxy was created, just return. + if mysqlProxy == nil { + log.Fatalf("mysqlProxy not initialized") + return + } + + // Initialize registered AuthServer implementations (or other plugins) + for _, initFn := range pluginInitializers { + initFn() + } + authServer := mysql.GetAuthServer(*mysqlAuthServerImpl) + + // Create a Listener. + var err error + mh := newProxyHandler(mysqlProxy) + if *mysqlServerPort >= 0 { + mysqlListener, err = mysql.NewListener("tcp", net.JoinHostPort(*mysqlServerBindAddress, fmt.Sprintf("%v", *mysqlServerPort)), authServer, mh) + if err != nil { + log.Exitf("mysql.NewListener failed: %v", err) + } + if *mysqlSslCert != "" && *mysqlSslKey != "" { + mysqlListener.TLSConfig, err = vttls.ServerConfig(*mysqlSslCert, *mysqlSslKey, *mysqlSslCa) + if err != nil { + log.Exitf("grpcutils.TLSServerConfig failed: %v", err) + return + } + } + mysqlListener.AllowClearTextWithoutTLS = *mysqlAllowClearTextWithoutTLS + + // Check for the connection threshold + if *mysqlSlowConnectWarnThreshold != 0 { + log.Infof("setting mysql slow connection threshold to %v", mysqlSlowConnectWarnThreshold) + mysqlListener.SlowConnectWarnThreshold = *mysqlSlowConnectWarnThreshold + } + // Start listening for tcp + go mysqlListener.Accept() + } + + if *mysqlServerSocketPath != "" { + // Let's create this unix socket with permissions to all users. In this way, + // clients can connect to mysqlproxy mysql server without being mysqlproxy user + oldMask := syscall.Umask(000) + mysqlUnixListener, err = newMysqlUnixSocket(*mysqlServerSocketPath, authServer, mh) + _ = syscall.Umask(oldMask) + if err != nil { + log.Exitf("mysql.NewListener failed: %v", err) + return + } + // Listen for unix socket + go mysqlUnixListener.Accept() + } +} + +// newMysqlUnixSocket creates a new unix socket mysql listener. If a socket file already exists, attempts +// to clean it up. +func newMysqlUnixSocket(address string, authServer mysql.AuthServer, handler mysql.Handler) (*mysql.Listener, error) { + listener, err := mysql.NewListener("unix", address, authServer, handler) + switch err := err.(type) { + case nil: + return listener, nil + case *net.OpError: + log.Warningf("Found existent socket when trying to create new unix mysql listener: %s, attempting to clean up", address) + // err.Op should never be different from listen, just being extra careful + // in case in the future other errors are returned here + if err.Op != "listen" { + return nil, err + } + _, dialErr := net.Dial("unix", address) + if dialErr == nil { + log.Errorf("Existent socket '%s' is still accepting connections, aborting", address) + return nil, err + } + removeFileErr := os.Remove(address) + if removeFileErr != nil { + log.Errorf("Couldn't remove existent socket file: %s", address) + return nil, err + } + listener, listenerErr := mysql.NewListener("unix", address, authServer, handler) + return listener, listenerErr + default: + return nil, err + } +} + +func init() { + servenv.OnRun(initMySQLProtocol) + + servenv.OnTerm(func() { + if mysqlListener != nil { + mysqlListener.Close() + mysqlListener = nil + } + if mysqlUnixListener != nil { + mysqlUnixListener.Close() + mysqlUnixListener = nil + } + }) +} + +var pluginInitializers []func() + +// RegisterPluginInitializer lets plugins register themselves to be init'ed at servenv.OnRun-time +func RegisterPluginInitializer(initializer func()) { + pluginInitializers = append(pluginInitializers, initializer) +} diff --git a/go/vt/vtqueryserver/plugin_mysql_server_test.go b/go/vt/vtqueryserver/plugin_mysql_server_test.go new file mode 100644 index 00000000000..487aa51c36a --- /dev/null +++ b/go/vt/vtqueryserver/plugin_mysql_server_test.go @@ -0,0 +1,157 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 vtqueryserver + +import ( + "io/ioutil" + "os" + "strings" + "testing" + + "golang.org/x/net/context" + + "github.com/youtube/vitess/go/mysql" + "github.com/youtube/vitess/go/sqltypes" +) + +type testHandler struct { + lastConn *mysql.Conn +} + +func (th *testHandler) NewConnection(c *mysql.Conn) { + th.lastConn = c +} + +func (th *testHandler) ConnectionClosed(c *mysql.Conn) { +} + +func (th *testHandler) ComQuery(c *mysql.Conn, q string, callback func(*sqltypes.Result) error) error { + return nil +} + +func TestConnectionUnixSocket(t *testing.T) { + th := &testHandler{} + + authServer := mysql.NewAuthServerStatic() + + authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ + { + Password: "password1", + UserData: "userData1", + SourceHost: "localhost", + }, + } + + // Use tmp file to reserve a path, remove it immediately, we only care about + // name in this context + unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") + if err != nil { + t.Fatalf("Failed to create temp file") + } + os.Remove(unixSocket.Name()) + + l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) + if err != nil { + t.Fatalf("NewUnixSocket failed: %v", err) + } + defer l.Close() + go l.Accept() + + params := &mysql.ConnParams{ + UnixSocket: unixSocket.Name(), + Uname: "user1", + Pass: "password1", + } + + c, err := mysql.Connect(context.Background(), params) + if err != nil { + t.Errorf("Should be able to connect to server but found error: %v", err) + } + c.Close() +} + +func TestConnectionStaleUnixSocket(t *testing.T) { + th := &testHandler{} + + authServer := mysql.NewAuthServerStatic() + + authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ + { + Password: "password1", + UserData: "userData1", + SourceHost: "localhost", + }, + } + + // First let's create a file. In this way, we simulate + // having a stale socket on disk that needs to be cleaned up. + unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") + if err != nil { + t.Fatalf("Failed to create temp file") + } + + l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) + if err != nil { + t.Fatalf("NewListener failed: %v", err) + } + defer l.Close() + go l.Accept() + + params := &mysql.ConnParams{ + UnixSocket: unixSocket.Name(), + Uname: "user1", + Pass: "password1", + } + + c, err := mysql.Connect(context.Background(), params) + if err != nil { + t.Errorf("Should be able to connect to server but found error: %v", err) + } + c.Close() +} + +func TestConnectionRespectsExistingUnixSocket(t *testing.T) { + th := &testHandler{} + + authServer := mysql.NewAuthServerStatic() + + authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ + { + Password: "password1", + UserData: "userData1", + SourceHost: "localhost", + }, + } + + unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") + if err != nil { + t.Fatalf("Failed to create temp file") + } + os.Remove(unixSocket.Name()) + + l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) + if err != nil { + t.Errorf("NewListener failed: %v", err) + } + defer l.Close() + go l.Accept() + _, err = newMysqlUnixSocket(unixSocket.Name(), authServer, th) + want := "listen unix" + if err == nil || !strings.HasPrefix(err.Error(), want) { + t.Errorf("Error: %v, want prefix %s", err, want) + } +} diff --git a/go/vt/vtqueryserver/status.go b/go/vt/vtqueryserver/status.go new file mode 100644 index 00000000000..04606919d9e --- /dev/null +++ b/go/vt/vtqueryserver/status.go @@ -0,0 +1,90 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 vtqueryserver + +import ( + "github.com/youtube/vitess/go/vt/servenv" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver" +) + +var ( + // proxyTemplate contains the style sheet and the tablet itself. + proxyTemplate = ` + + + + + + + + +
+ Target Keyspace: {{.Target.Keyspace}}
+
+ Schema
+ Schema Query Plans
+ Schema Query Stats
+ Schema Table Stats
+
+ Query Stats
+ Streaming Query Stats
+ Consolidations
+ Current Query Log
+ Current Transaction Log
+ In-flight 2PC Transactions
+
+ Query Service Health Check
+ Current Stream Queries
+
+` +) + +// For use by plugins which wish to avoid racing when registering status page parts. +var onStatusRegistered func() + +func addStatusParts(qsc tabletserver.Controller) { + servenv.AddStatusPart("Target", proxyTemplate, func() interface{} { + return map[string]interface{}{ + "Target": target, + } + }) + qsc.AddStatusPart() + if onStatusRegistered != nil { + onStatusRegistered() + } +} diff --git a/go/vt/vtqueryserver/vtqueryserver.go b/go/vt/vtqueryserver/vtqueryserver.go new file mode 100644 index 00000000000..71d8a708b4b --- /dev/null +++ b/go/vt/vtqueryserver/vtqueryserver.go @@ -0,0 +1,78 @@ +/* +Copyright 2017 Google Inc. + +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 vtqueryserver is a standalone version of the tablet server that +// only implements the queryservice interface without any of the topology, +// replication management, or other features of the full vttablet. +package vtqueryserver + +import ( + "flag" + + log "github.com/golang/glog" + + "github.com/youtube/vitess/go/vt/dbconfigs" + "github.com/youtube/vitess/go/vt/mysqlproxy" + "github.com/youtube/vitess/go/vt/servenv" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" + + querypb "github.com/youtube/vitess/go/vt/proto/query" + topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" +) + +var ( + mysqlProxy *mysqlproxy.Proxy + target = querypb.Target{ + TabletType: topodatapb.TabletType_MASTER, + Keyspace: "", + } + + targetKeyspace = flag.String("target", "", "Target database name") + normalizeQueries = flag.Bool("normalize_queries", true, "Rewrite queries with bind vars. Turn this off if the app itself sends normalized queries with bind vars.") +) + +// Init initializes the proxy +func Init(dbcfgs *dbconfigs.DBConfigs) error { + target.Keyspace = *targetKeyspace + log.Infof("initalizing vtqueryserver.Proxy for target %s", target.Keyspace) + + // force autocommit to be enabled + tabletenv.Config.EnableAutoCommit = true + + // creates and registers the query service + qs := tabletserver.NewTabletServerWithNilTopoServer(tabletenv.Config) + + mysqlProxy = mysqlproxy.NewProxy(&target, qs, *normalizeQueries) + + servenv.OnRun(func() { + qs.Register() + addStatusParts(qs) + }) + + servenv.OnClose(func() { + // We now leave the queryservice running during lameduck, + // so stop it in OnClose(), after lameduck is over. + qs.StopService() + }) + + err := qs.StartService(target, *dbcfgs) + if err != nil { + return err + } + + return nil +} From bb33739fa25259144da4aa3fabff90ec6af8cc29 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 23 Jan 2018 17:55:23 -0800 Subject: [PATCH 04/72] resolve upstream conflict --- go/vt/mysqlproxy/mysqlproxy.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go index 9503b572c6a..ff9d1c7da96 100644 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -117,7 +117,7 @@ func (mp *Proxy) doRollback(ctx context.Context, session *ProxySession) error { // Set is currently ignored func (mp *Proxy) doSet(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - vals, charset, err := sqlparser.ExtractSetValues(sql) + vals, charset, _, err := sqlparser.ExtractSetValues(sql) if err != nil { return nil, err } From e1fdd3e969ae5165a8ce5187388f08d09f15bd15 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Fri, 26 Jan 2018 12:20:34 -0800 Subject: [PATCH 05/72] refactor vtqueryserver startup to enable testing --- go/vt/vtqueryserver/plugin_mysql_server.go | 26 +++++++++++++--------- go/vt/vtqueryserver/vtqueryserver.go | 24 +++++++++++++------- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/go/vt/vtqueryserver/plugin_mysql_server.go b/go/vt/vtqueryserver/plugin_mysql_server.go index 41d39af98ab..aa183989e43 100644 --- a/go/vt/vtqueryserver/plugin_mysql_server.go +++ b/go/vt/vtqueryserver/plugin_mysql_server.go @@ -165,6 +165,7 @@ func initMySQLProtocol() { } // Start listening for tcp go mysqlListener.Accept() + log.Infof("listening on %s:%d", *mysqlServerBindAddress, *mysqlServerPort) } if *mysqlServerSocketPath != "" { @@ -213,19 +214,22 @@ func newMysqlUnixSocket(address string, authServer mysql.AuthServer, handler mys } } +func shutdownMySQLProtocol() { + log.Infof("shutting down mysql protocol") + if mysqlListener != nil { + mysqlListener.Close() + mysqlListener = nil + } + + if mysqlUnixListener != nil { + mysqlUnixListener.Close() + mysqlUnixListener = nil + } +} + func init() { servenv.OnRun(initMySQLProtocol) - - servenv.OnTerm(func() { - if mysqlListener != nil { - mysqlListener.Close() - mysqlListener = nil - } - if mysqlUnixListener != nil { - mysqlUnixListener.Close() - mysqlUnixListener = nil - } - }) + servenv.OnTerm(shutdownMySQLProtocol) } var pluginInitializers []func() diff --git a/go/vt/vtqueryserver/vtqueryserver.go b/go/vt/vtqueryserver/vtqueryserver.go index 71d8a708b4b..8ac90a4b931 100644 --- a/go/vt/vtqueryserver/vtqueryserver.go +++ b/go/vt/vtqueryserver/vtqueryserver.go @@ -45,8 +45,7 @@ var ( normalizeQueries = flag.Bool("normalize_queries", true, "Rewrite queries with bind vars. Turn this off if the app itself sends normalized queries with bind vars.") ) -// Init initializes the proxy -func Init(dbcfgs *dbconfigs.DBConfigs) error { +func initProxy(dbcfgs *dbconfigs.DBConfigs) (*tabletserver.TabletServer, error) { target.Keyspace = *targetKeyspace log.Infof("initalizing vtqueryserver.Proxy for target %s", target.Keyspace) @@ -55,9 +54,23 @@ func Init(dbcfgs *dbconfigs.DBConfigs) error { // creates and registers the query service qs := tabletserver.NewTabletServerWithNilTopoServer(tabletenv.Config) - mysqlProxy = mysqlproxy.NewProxy(&target, qs, *normalizeQueries) + err := qs.StartService(target, *dbcfgs) + if err != nil { + return nil, err + } + + return qs, nil +} + +// Init initializes the proxy +func Init(dbcfgs *dbconfigs.DBConfigs) error { + qs, err := initProxy(dbcfgs) + if err != nil { + return err + } + servenv.OnRun(func() { qs.Register() addStatusParts(qs) @@ -69,10 +82,5 @@ func Init(dbcfgs *dbconfigs.DBConfigs) error { qs.StopService() }) - err := qs.StartService(target, *dbcfgs) - if err != nil { - return err - } - return nil } From a75a754e345eff0c2668061c0b1f04a0bdf1556b Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Fri, 26 Jan 2018 12:20:48 -0800 Subject: [PATCH 06/72] add initial end to end test for vtqueryserver --- go/vt/vtqueryserver/endtoend_test.go | 213 +++++++++++++++++++++++++++ 1 file changed, 213 insertions(+) create mode 100644 go/vt/vtqueryserver/endtoend_test.go diff --git a/go/vt/vtqueryserver/endtoend_test.go b/go/vt/vtqueryserver/endtoend_test.go new file mode 100644 index 00000000000..263d7ff3d1e --- /dev/null +++ b/go/vt/vtqueryserver/endtoend_test.go @@ -0,0 +1,213 @@ +/* +Copyright 2017 Google Inc. + +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 vtqueryserver + +import ( + "context" + "flag" + "fmt" + "os" + "path" + "strings" + "testing" + + "github.com/youtube/vitess/go/mysql" + "github.com/youtube/vitess/go/vt/dbconfigs" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" + "github.com/youtube/vitess/go/vt/vttest" + + vttestpb "github.com/youtube/vitess/go/vt/proto/vttest" +) + +var ( + mysqlConnParams mysql.ConnParams + proxyConnParams mysql.ConnParams +) + +func TestMain(m *testing.M) { + flag.Parse() // Do not remove this comment, import into google3 depends on it + tabletenv.Init() + + exitCode := func() int { + // Launch MySQL. + // We need a Keyspace in the topology, so the DbName is set. + // We need a Shard too, so the database 'vttest' is created. + cfg := vttest.Config{ + Topology: &vttestpb.VTTestTopology{ + Keyspaces: []*vttestpb.Keyspace{ + { + Name: "vttest", + Shards: []*vttestpb.Shard{ + { + Name: "0", + DbNameOverride: "vttest", + }, + }, + }, + }, + }, + OnlyMySQL: true, + } + if err := cfg.InitSchemas("vttest", testSchema, nil); err != nil { + fmt.Fprintf(os.Stderr, "InitSchemas failed: %v\n", err) + return 1 + } + defer os.RemoveAll(cfg.SchemaDir) + cluster := vttest.LocalCluster{ + Config: cfg, + } + if err := cluster.Setup(); err != nil { + fmt.Fprintf(os.Stderr, "could not launch mysql: %v\n", err) + return 1 + } + defer cluster.TearDown() + + mysqlConnParams = cluster.MySQLConnParams() + + proxySock := path.Join(cluster.Env.Directory(), "mysqlproxy.sock") + + proxyConnParams.UnixSocket = proxySock + proxyConnParams.Uname = "proxy" + proxyConnParams.Pass = "letmein" + + *mysqlServerSocketPath = proxyConnParams.UnixSocket + *mysqlAuthServerImpl = "none" + + dbcfgs := dbconfigs.DBConfigs{ + App: mysqlConnParams, + } + qs, err := initProxy(&dbcfgs) + if err != nil { + fmt.Fprintf(os.Stderr, "could not start proxy: %v\n", err) + return 1 + } + defer qs.StopService() + + initMySQLProtocol() + defer shutdownMySQLProtocol() + + return m.Run() + }() + os.Exit(exitCode) +} + +var testSchema = ` +create table test(id int, val varchar(256), primary key(id)); +create table valtest(intval int default 0, floatval float default null, charval varchar(256) default null, binval varbinary(256) default null, primary key(intval)); +` + +func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { + t.Helper() + + result, err := conn.ExecuteFetch(sql, 1000, true) + if err != nil { + t.Fatalf("error: %v", err) + } + + if len(result.Rows) != expectedRows { + t.Fatalf("expected %d rows but got %d", expectedRows, len(result.Rows)) + } +} + +func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { + t.Helper() + + result, err := conn.ExecuteFetch(sql, 1000, true) + if err != nil { + t.Fatalf("error: %v", err) + } + + if int(result.RowsAffected) != expectedRows { + t.Fatalf("expected %d rows affected but got %d", expectedRows, result.RowsAffected) + } +} + +func TestQueries(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + // Try a simple query case. + testFetch(t, conn, "select * from test", 0) + + // Try a simple error case. + _, err = conn.ExecuteFetch("select * from aa", 1000, true) + if err == nil || !strings.Contains(err.Error(), "table aa not found in schema") { + t.Fatalf("expected error but got: %v", err) + } +} + +func TestAutocommitDMLs(t *testing.T) { + ctx := context.Background() + + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + conn2, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 1) + + testDML(t, conn, "delete from test", 1) + + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 0) +} + +func TestTransactions(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + conn2, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + testDML(t, conn, "begin", 0) + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 0) + testDML(t, conn, "commit", 0) + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 1) + + testDML(t, conn, "begin", 0) + testDML(t, conn, "delete from test", 1) + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 1) + testDML(t, conn, "rollback", 0) + + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 1) + + testDML(t, conn2, "begin", 0) + testDML(t, conn2, "delete from test", 1) + testDML(t, conn2, "commit", 0) + + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 0) +} From 45ad9265220c2bda979b3c3d37d8cb9d09632273 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Sat, 27 Jan 2018 08:29:52 -0800 Subject: [PATCH 07/72] rework the proxy implementation of autocommit to match mysql Following the example of vtgate, add support to set autocommit on or off in the session, update the execution path to use ExecuteBatch for single round-trip autocommit semantics, and add tests to ensure the behavior matches mysql. --- go/vt/mysqlproxy/mysqlproxy.go | 70 ++++++++++++++++++++++++++-- go/vt/vtqueryserver/endtoend_test.go | 44 +++++++++++++++-- go/vt/vtqueryserver/vtqueryserver.go | 3 -- 3 files changed, 106 insertions(+), 11 deletions(-) diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go index ff9d1c7da96..23e5e4b7e5c 100644 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -70,8 +70,10 @@ func (mp *Proxy) Execute(ctx context.Context, session *ProxySession, sql string, err = mp.doRollback(ctx, session) case sqlparser.StmtSet: result, err = mp.doSet(ctx, session, sql, bindVariables) + case sqlparser.StmtSelect: + result, err = mp.doSelect(ctx, session, sql, bindVariables) default: - result, err = mp.doExecute(ctx, session, sql, bindVariables) + result, err = mp.doExecuteDML(ctx, session, sql, bindVariables) } if err != nil { @@ -133,14 +135,35 @@ func (mp *Proxy) doSet(ctx context.Context, session *ProxySession, sql string, b } for k, v := range vals { - log.Warningf("Ignored inapplicable SET %v = %v", k, v) + switch k { + case "autocommit": + val, ok := v.(int64) + if !ok { + return nil, fmt.Errorf("unexpected value type for autocommit: %T", v) + } + switch val { + case 0: + session.Autocommit = false + case 1: + if session.TransactionID != 0 { + if err := mp.doCommit(ctx, session); err != nil { + return nil, err + } + } + session.Autocommit = true + default: + return nil, fmt.Errorf("unexpected value for autocommit: %d", val) + } + default: + log.Warningf("Ignored inapplicable SET %v = %v", k, v) + } } return &sqltypes.Result{}, nil } -// doExecute runs the given query -func (mp *Proxy) doExecute(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { +// doSelect runs the given select +func (mp *Proxy) doSelect(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { if mp.normalize { query, comments := sqlparser.SplitTrailingComments(sql) stmt, err := sqlparser.Parse(query) @@ -154,3 +177,42 @@ func (mp *Proxy) doExecute(ctx context.Context, session *ProxySession, sql strin return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) } + +// doExecuteDML runs the given query handling autocommit semantics +func (mp *Proxy) doExecuteDML(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + if mp.normalize { + query, comments := sqlparser.SplitTrailingComments(sql) + stmt, err := sqlparser.Parse(query) + if err != nil { + return nil, err + } + sqlparser.Normalize(stmt, bindVariables, "vtp") + normalized := sqlparser.String(stmt) + sql = normalized + comments + } + + if session.TransactionID != 0 { + return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) + + } else if session.Autocommit { + queries := []*querypb.BoundQuery{{ + Sql: sql, + BindVariables: bindVariables, + }} + + // This is a stopgap until there is a better way to do autocommit + results, err := mp.qs.ExecuteBatch(ctx, mp.target, queries, true /* asTransaction */, 0, session.Options) + if err != nil { + return nil, err + } + return &results[0], nil + + } else { + result, txnID, err := mp.qs.BeginExecute(ctx, mp.target, sql, bindVariables, session.Options) + if err != nil { + return nil, err + } + session.TransactionID = txnID + return result, nil + } +} diff --git a/go/vt/vtqueryserver/endtoend_test.go b/go/vt/vtqueryserver/endtoend_test.go index 263d7ff3d1e..e25b79afc8b 100644 --- a/go/vt/vtqueryserver/endtoend_test.go +++ b/go/vt/vtqueryserver/endtoend_test.go @@ -114,11 +114,11 @@ func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { result, err := conn.ExecuteFetch(sql, 1000, true) if err != nil { - t.Fatalf("error: %v", err) + t.Errorf("error: %v", err) } if len(result.Rows) != expectedRows { - t.Fatalf("expected %d rows but got %d", expectedRows, len(result.Rows)) + t.Errorf("expected %d rows but got %d", expectedRows, len(result.Rows)) } } @@ -127,11 +127,11 @@ func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { result, err := conn.ExecuteFetch(sql, 1000, true) if err != nil { - t.Fatalf("error: %v", err) + t.Errorf("error: %v", err) } if int(result.RowsAffected) != expectedRows { - t.Fatalf("expected %d rows affected but got %d", expectedRows, result.RowsAffected) + t.Errorf("expected %d rows affected but got %d", expectedRows, result.RowsAffected) } } @@ -211,3 +211,39 @@ func TestTransactions(t *testing.T) { testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 0) } + +func TestNoAutocommit(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + conn2, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + testDML(t, conn, "set autocommit=0", 0) + + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 0) + testDML(t, conn, "commit", 0) + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 1) + + testDML(t, conn, "delete from test", 1) + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 1) + testDML(t, conn, "rollback", 0) + + testFetch(t, conn, "select * from test", 1) + testFetch(t, conn2, "select * from test", 1) + + testDML(t, conn2, "set autocommit=0", 0) + testDML(t, conn2, "delete from test", 1) + testDML(t, conn2, "commit", 0) + + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 0) +} diff --git a/go/vt/vtqueryserver/vtqueryserver.go b/go/vt/vtqueryserver/vtqueryserver.go index 8ac90a4b931..07750e552b1 100644 --- a/go/vt/vtqueryserver/vtqueryserver.go +++ b/go/vt/vtqueryserver/vtqueryserver.go @@ -49,9 +49,6 @@ func initProxy(dbcfgs *dbconfigs.DBConfigs) (*tabletserver.TabletServer, error) target.Keyspace = *targetKeyspace log.Infof("initalizing vtqueryserver.Proxy for target %s", target.Keyspace) - // force autocommit to be enabled - tabletenv.Config.EnableAutoCommit = true - // creates and registers the query service qs := tabletserver.NewTabletServerWithNilTopoServer(tabletenv.Config) mysqlProxy = mysqlproxy.NewProxy(&target, qs, *normalizeQueries) From 2b3048f58486967949a090816145326cd95bca6e Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Sat, 27 Jan 2018 10:10:08 -0800 Subject: [PATCH 08/72] add accessors to set PassthroughDMLs and AllowUnsafeDMLs --- .../vttablet/tabletserver/query_executor_test.go | 16 ++++++---------- go/vt/vttablet/tabletserver/tabletserver.go | 13 +++++++++++++ 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index 031a54e3575..8b1fb8b83af 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -108,8 +108,6 @@ func TestQueryExecutorPlanPassDmlRBR(t *testing.T) { func TestQueryExecutorPassthroughDml(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() - planbuilder.PassthroughDMLs = true - defer func() { planbuilder.PassthroughDMLs = false }() query := "update test_table set pk = foo()" want := &sqltypes.Result{} db.AddQuery(query, want) @@ -118,9 +116,8 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() - planbuilder.PassthroughDMLs = true - defer func() { planbuilder.PassthroughDMLs = false }() - tsv.qe.passthroughDMLs.Set(true) + tsv.SetPassthroughDMLs(true) + defer tsv.SetPassthroughDMLs(false) tsv.qe.binlogFormat = connpool.BinlogFormatRow txid := newTransaction(tsv, nil) @@ -147,7 +144,7 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - tsv.qe.allowUnsafeDMLs = true + tsv.SetAllowUnsafeDMLs(true) got, err = qre.Execute() if !reflect.DeepEqual(got, want) { t.Fatalf("got: %v, want: %v", got, want) @@ -201,9 +198,8 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() - planbuilder.PassthroughDMLs = true - defer func() { planbuilder.PassthroughDMLs = false }() - tsv.qe.passthroughDMLs.Set(true) + tsv.SetPassthroughDMLs(true) + defer tsv.SetPassthroughDMLs(false) tsv.qe.binlogFormat = connpool.BinlogFormatRow qre := newTestQueryExecutor(ctx, tsv, query, 0) @@ -223,7 +219,7 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - tsv.qe.allowUnsafeDMLs = true + tsv.SetAllowUnsafeDMLs(true) got, err = qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 3d3c3572ffc..942075fd8df 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1976,6 +1976,19 @@ func (tsv *TabletServer) MaxDMLRows() int { return int(tsv.qe.maxDMLRows.Get()) } +// 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 +} + // queryAsString prints a readable version of query+bind variables, // and also truncates data if it's too long func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable) string { From 0ea8013db075f027e994596a351202c8eb389a3e Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Sat, 27 Jan 2018 10:11:13 -0800 Subject: [PATCH 09/72] add vtqueryserver tests for passthrough DMLs --- go/vt/vtqueryserver/endtoend_test.go | 102 ++++++++++++++++++++------- 1 file changed, 76 insertions(+), 26 deletions(-) diff --git a/go/vt/vtqueryserver/endtoend_test.go b/go/vt/vtqueryserver/endtoend_test.go index e25b79afc8b..b855559a7e7 100644 --- a/go/vt/vtqueryserver/endtoend_test.go +++ b/go/vt/vtqueryserver/endtoend_test.go @@ -26,6 +26,7 @@ import ( "github.com/youtube/vitess/go/mysql" "github.com/youtube/vitess/go/vt/dbconfigs" + "github.com/youtube/vitess/go/vt/vttablet/tabletserver" "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" "github.com/youtube/vitess/go/vt/vttest" @@ -33,6 +34,7 @@ import ( ) var ( + queryServer *tabletserver.TabletServer mysqlConnParams mysql.ConnParams proxyConnParams mysql.ConnParams ) @@ -89,12 +91,14 @@ func TestMain(m *testing.M) { dbcfgs := dbconfigs.DBConfigs{ App: mysqlConnParams, } - qs, err := initProxy(&dbcfgs) + + var err error + queryServer, err = initProxy(&dbcfgs) if err != nil { fmt.Fprintf(os.Stderr, "could not start proxy: %v\n", err) return 1 } - defer qs.StopService() + defer queryServer.StopService() initMySQLProtocol() defer shutdownMySQLProtocol() @@ -112,7 +116,7 @@ create table valtest(intval int default 0, floatval float default null, charval func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { t.Helper() - result, err := conn.ExecuteFetch(sql, 1000, true) + result, err := conn.ExecuteFetch(sql, 1000, false) if err != nil { t.Errorf("error: %v", err) } @@ -122,16 +126,22 @@ func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { } } -func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { +func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedNumQueries int64, expectedRowsAffected uint64) { t.Helper() - result, err := conn.ExecuteFetch(sql, 1000, true) + numQueries := tabletenv.MySQLStats.Count() + result, err := conn.ExecuteFetch(sql, 1000, false) if err != nil { t.Errorf("error: %v", err) } + numQueries = tabletenv.MySQLStats.Count() - numQueries + + if numQueries != expectedNumQueries { + t.Errorf("expected %d mysql queries but got %d", expectedNumQueries, numQueries) + } - if int(result.RowsAffected) != expectedRows { - t.Errorf("expected %d rows affected but got %d", expectedRows, result.RowsAffected) + if result.RowsAffected != expectedRowsAffected { + t.Errorf("expected %d rows affected but got %d", expectedRowsAffected, result.RowsAffected) } } @@ -165,12 +175,52 @@ func TestAutocommitDMLs(t *testing.T) { t.Fatal(err) } - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 3, 1) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "delete from test", 1) + testDML(t, conn, "delete from test", 4, 1) + + testFetch(t, conn, "select * from test", 0) + testFetch(t, conn2, "select * from test", 0) +} + +func TestPassthroughDMLs(t *testing.T) { + ctx := context.Background() + + queryServer.SetPassthroughDMLs(true) + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + conn2, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 3, 1) + testDML(t, conn, "insert into test (id, val) values(2, 'hello')", 3, 1) + testDML(t, conn, "insert into test (id, val) values(3, 'hello')", 3, 1) + + // Subquery DMLs are errors in passthrough mode with SBR, unless + // SetAllowUnsafeDMLs is set + _, err = conn.ExecuteFetch("update test set val='goodbye'", 1000, true) + if err == nil || !strings.Contains(err.Error(), "cannot identify primary key of statement") { + t.Fatalf("expected error but got: %v", err) + } + + queryServer.SetAllowUnsafeDMLs(true) + + // This is 3 queries in passthrough mode and not 4 queries as it would + // be in non-passthrough mode + testDML(t, conn, "update test set val='goodbye'", 3, 3) + + testFetch(t, conn, "select * from test where val='goodbye'", 3) + testFetch(t, conn2, "select * from test where val='goodbye'", 3) + + testDML(t, conn, "delete from test", 4, 3) testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 0) @@ -187,26 +237,26 @@ func TestTransactions(t *testing.T) { t.Fatal(err) } - testDML(t, conn, "begin", 0) - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + testDML(t, conn, "begin", 1, 0) + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1, 1) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 0) - testDML(t, conn, "commit", 0) + testDML(t, conn, "commit", 1, 0) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "begin", 0) - testDML(t, conn, "delete from test", 1) + testDML(t, conn, "begin", 1, 0) + testDML(t, conn, "delete from test", 2, 1) testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "rollback", 0) + testDML(t, conn, "rollback", 1, 0) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn2, "begin", 0) - testDML(t, conn2, "delete from test", 1) - testDML(t, conn2, "commit", 0) + testDML(t, conn2, "begin", 1, 0) + testDML(t, conn2, "delete from test", 2, 1) + testDML(t, conn2, "commit", 1, 0) testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 0) @@ -223,26 +273,26 @@ func TestNoAutocommit(t *testing.T) { t.Fatal(err) } - testDML(t, conn, "set autocommit=0", 0) + testFetch(t, conn, "set autocommit=0", 0) - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1) + testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 2, 1) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 0) - testDML(t, conn, "commit", 0) + testDML(t, conn, "commit", 1, 0) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "delete from test", 1) + testDML(t, conn, "delete from test", 3, 1) testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "rollback", 0) + testDML(t, conn, "rollback", 1, 0) testFetch(t, conn, "select * from test", 1) testFetch(t, conn2, "select * from test", 1) - testDML(t, conn2, "set autocommit=0", 0) - testDML(t, conn2, "delete from test", 1) - testDML(t, conn2, "commit", 0) + testFetch(t, conn2, "set autocommit=0", 0) + testDML(t, conn2, "delete from test", 3, 1) + testDML(t, conn2, "commit", 1, 0) testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 0) From bc7ffa9894a7d6b4a49f8a07ae941eb490d4c065 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Sat, 27 Jan 2018 10:11:48 -0800 Subject: [PATCH 10/72] add vtqueryserver flag for allow_unsafe_dmls --- go/vt/vtqueryserver/vtqueryserver.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/go/vt/vtqueryserver/vtqueryserver.go b/go/vt/vtqueryserver/vtqueryserver.go index 07750e552b1..a048d0cfeda 100644 --- a/go/vt/vtqueryserver/vtqueryserver.go +++ b/go/vt/vtqueryserver/vtqueryserver.go @@ -43,6 +43,7 @@ var ( targetKeyspace = flag.String("target", "", "Target database name") normalizeQueries = flag.Bool("normalize_queries", true, "Rewrite queries with bind vars. Turn this off if the app itself sends normalized queries with bind vars.") + allowUnsafeDMLs = flag.Bool("allow_unsafe_dmls", false, "Allow passthrough DML statements when running with statement-based replication") ) func initProxy(dbcfgs *dbconfigs.DBConfigs) (*tabletserver.TabletServer, error) { @@ -51,6 +52,7 @@ func initProxy(dbcfgs *dbconfigs.DBConfigs) (*tabletserver.TabletServer, error) // creates and registers the query service qs := tabletserver.NewTabletServerWithNilTopoServer(tabletenv.Config) + qs.SetAllowUnsafeDMLs(*allowUnsafeDMLs) mysqlProxy = mysqlproxy.NewProxy(&target, qs, *normalizeQueries) err := qs.StartService(target, *dbcfgs) From 4a44d25ff4e324887886da7d5708a004782d983b Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Mon, 29 Jan 2018 14:59:29 -0800 Subject: [PATCH 11/72] add proxy support for other statements This allows EXPLAIN and other such statements to go through without query normalization. --- go/vt/mysqlproxy/mysqlproxy.go | 20 ++++++++++++++------ go/vt/vtqueryserver/endtoend_test.go | 17 ++++++++++++++++- 2 files changed, 30 insertions(+), 7 deletions(-) diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go index 23e5e4b7e5c..e89471d7039 100644 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -70,10 +70,12 @@ func (mp *Proxy) Execute(ctx context.Context, session *ProxySession, sql string, err = mp.doRollback(ctx, session) case sqlparser.StmtSet: result, err = mp.doSet(ctx, session, sql, bindVariables) + case sqlparser.StmtInsert, sqlparser.StmtUpdate, sqlparser.StmtDelete, sqlparser.StmtReplace: + result, err = mp.executeDML(ctx, session, sql, bindVariables) case sqlparser.StmtSelect: - result, err = mp.doSelect(ctx, session, sql, bindVariables) + result, err = mp.executeSelect(ctx, session, sql, bindVariables) default: - result, err = mp.doExecuteDML(ctx, session, sql, bindVariables) + result, err = mp.executeOther(ctx, session, sql, bindVariables) } if err != nil { @@ -162,12 +164,13 @@ func (mp *Proxy) doSet(ctx context.Context, session *ProxySession, sql string, b return &sqltypes.Result{}, nil } -// doSelect runs the given select -func (mp *Proxy) doSelect(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { +// executeSelect runs the given select statement +func (mp *Proxy) executeSelect(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { if mp.normalize { query, comments := sqlparser.SplitTrailingComments(sql) stmt, err := sqlparser.Parse(query) if err != nil { + fmt.Printf("YYY parse error %s\n", query) return nil, err } sqlparser.Normalize(stmt, bindVariables, "vtp") @@ -178,8 +181,8 @@ func (mp *Proxy) doSelect(ctx context.Context, session *ProxySession, sql string return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) } -// doExecuteDML runs the given query handling autocommit semantics -func (mp *Proxy) doExecuteDML(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { +// executeDML runs the given query handling autocommit semantics +func (mp *Proxy) executeDML(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { if mp.normalize { query, comments := sqlparser.SplitTrailingComments(sql) stmt, err := sqlparser.Parse(query) @@ -216,3 +219,8 @@ func (mp *Proxy) doExecuteDML(ctx context.Context, session *ProxySession, sql st return result, nil } } + +// executeOther runs the given other statement bypassing the normalizer +func (mp *Proxy) executeOther(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { + return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) +} diff --git a/go/vt/vtqueryserver/endtoend_test.go b/go/vt/vtqueryserver/endtoend_test.go index b855559a7e7..b04a0bb8a58 100644 --- a/go/vt/vtqueryserver/endtoend_test.go +++ b/go/vt/vtqueryserver/endtoend_test.go @@ -25,6 +25,7 @@ import ( "testing" "github.com/youtube/vitess/go/mysql" + "github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/vt/dbconfigs" "github.com/youtube/vitess/go/vt/vttablet/tabletserver" "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" @@ -113,7 +114,7 @@ create table test(id int, val varchar(256), primary key(id)); create table valtest(intval int default 0, floatval float default null, charval varchar(256) default null, binval varbinary(256) default null, primary key(intval)); ` -func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { +func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) *sqltypes.Result { t.Helper() result, err := conn.ExecuteFetch(sql, 1000, false) @@ -124,6 +125,8 @@ func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) { if len(result.Rows) != expectedRows { t.Errorf("expected %d rows but got %d", expectedRows, len(result.Rows)) } + + return result } func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedNumQueries int64, expectedRowsAffected uint64) { @@ -297,3 +300,15 @@ func TestNoAutocommit(t *testing.T) { testFetch(t, conn, "select * from test", 0) testFetch(t, conn2, "select * from test", 0) } + +func TestOther(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &proxyConnParams) + if err != nil { + t.Fatal(err) + } + + testFetch(t, conn, "explain select * from test", 1) + testFetch(t, conn, "select table_name, table_rows from information_schema.tables where table_name='test'", 1) + +} From b6703a71a7608fbf8597e1ee434ad47d4e86b38d Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 31 Jan 2018 06:57:46 -0800 Subject: [PATCH 12/72] remove debugging printf --- go/vt/mysqlproxy/mysqlproxy.go | 1 - 1 file changed, 1 deletion(-) diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go index e89471d7039..e3eebea702e 100644 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -170,7 +170,6 @@ func (mp *Proxy) executeSelect(ctx context.Context, session *ProxySession, sql s query, comments := sqlparser.SplitTrailingComments(sql) stmt, err := sqlparser.Parse(query) if err != nil { - fmt.Printf("YYY parse error %s\n", query) return nil, err } sqlparser.Normalize(stmt, bindVariables, "vtp") From c45710b82b5f14e673afa8247b0f8edbe3a80906 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 6 Feb 2018 12:15:56 -0800 Subject: [PATCH 13/72] add support for `USING ` to CREATE TABLE --- go/vt/sqlparser/ast.go | 4 + go/vt/sqlparser/parse_test.go | 15 + go/vt/sqlparser/sql.go | 1385 +++++++++++++++++---------------- go/vt/sqlparser/sql.y | 12 +- 4 files changed, 718 insertions(+), 698 deletions(-) diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go index 4ed6d6d689b..d9edde9b5c4 100644 --- a/go/vt/sqlparser/ast.go +++ b/go/vt/sqlparser/ast.go @@ -982,6 +982,7 @@ func (ct *ColumnType) WalkSubtree(visit Visit) error { type IndexDefinition struct { Info *IndexInfo Columns []*IndexColumn + Using ColIdent } // Format formats the node. @@ -998,6 +999,9 @@ func (idx *IndexDefinition) Format(buf *TrackedBuffer) { } } buf.Myprintf(")") + if !idx.Using.IsEmpty() { + buf.Myprintf(" USING %v", idx.Using) + } } // WalkSubtree walks the nodes of the subtree. diff --git a/go/vt/sqlparser/parse_test.go b/go/vt/sqlparser/parse_test.go index bda1920747f..492946980d8 100644 --- a/go/vt/sqlparser/parse_test.go +++ b/go/vt/sqlparser/parse_test.go @@ -1496,6 +1496,21 @@ func TestCreateTable(t *testing.T) { " key by_full_name (full_name)\n" + ")", + // test that indexes support USING + "create table t (\n" + + " id int auto_increment,\n" + + " username varchar,\n" + + " email varchar,\n" + + " full_name varchar,\n" + + " status_nonkeyword varchar,\n" + + " primary key (id) USING BTREE,\n" + + " unique key by_username (username) USING HASH,\n" + + " unique by_username2 (username) USING OTHER,\n" + + " unique index by_username3 (username) USING XYZ,\n" + + " index by_status (status_nonkeyword) USING PDQ,\n" + + " key by_full_name (full_name) USING OTHER\n" + + ")", + // multi-column indexes "create table t (\n" + " id int auto_increment,\n" + diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index a0d21ed47c1..9d712fb7e9b 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -553,238 +553,302 @@ var yyExca = [...]int{ -1, 1015, 5, 22, -2, 446, - -1, 1061, + -1, 1062, 5, 23, -2, 447, } const yyPrivate = 57344 -const yyLast = 8494 +const yyLast = 8507 var yyAct = [...]int{ - 380, 38, 1053, 565, 379, 885, 353, 908, 198, 664, - 624, 886, 641, 677, 882, 943, 973, 173, 432, 429, - 44, 736, 743, 782, 430, 3, 746, 856, 240, 848, - 242, 640, 637, 797, 760, 818, 713, 650, 238, 38, - 167, 785, 342, 402, 408, 213, 348, 178, 201, 434, - 603, 351, 193, 189, 745, 418, 182, 673, 43, 1084, - 207, 1075, 1081, 172, 1070, 1079, 1074, 1069, 956, 1008, - 217, 187, 1030, 814, 188, 168, 169, 170, 171, 913, - 914, 915, 340, 657, 985, 233, 665, 48, 916, 1003, - 693, 1056, 531, 530, 540, 541, 533, 534, 535, 536, - 537, 538, 539, 532, 691, 1036, 542, 1001, 50, 51, - 52, 53, 334, 335, 1078, 223, 1076, 1054, 531, 530, - 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, - 838, 697, 542, 604, 224, 625, 627, 219, 658, 135, - 690, 136, 492, 136, 497, 220, 235, 1028, 237, 138, - 139, 140, 239, 239, 239, 239, 849, 239, 239, 796, - 795, 794, 748, 215, 239, 577, 835, 234, 236, 147, - 137, 1046, 837, 652, 554, 555, 993, 970, 860, 38, - 801, 564, 450, 228, 819, 542, 517, 652, 687, 692, - 685, 331, 332, 333, 431, 336, 337, 404, 520, 761, - 449, 808, 339, 405, 958, 720, 207, 626, 444, 695, - 698, 494, 1057, 665, 917, 214, 153, 239, 922, 718, - 719, 717, 239, 232, 532, 518, 1049, 542, 1068, 446, - 812, 239, 239, 239, 239, 239, 239, 239, 239, 689, - 163, 520, 1029, 1027, 944, 410, 535, 536, 537, 538, - 539, 532, 654, 688, 542, 491, 226, 655, 989, 504, - 496, 502, 836, 651, 834, 988, 946, 406, 923, 505, - 506, 507, 508, 509, 510, 511, 512, 651, 694, 761, - 867, 872, 649, 648, 41, 827, 519, 518, 826, 696, - 148, 522, 815, 948, 716, 952, 150, 947, 1063, 945, - 211, 156, 152, 520, 950, 533, 534, 535, 536, 537, - 538, 539, 532, 949, 1039, 542, 592, 593, 951, 953, - 987, 866, 154, 865, 825, 158, 63, 521, 519, 518, - 145, 921, 737, 145, 738, 239, 239, 345, 403, 519, - 518, 1065, 341, 519, 518, 520, 910, 809, 149, 739, - 551, 553, 145, 145, 1019, 341, 520, 490, 145, 230, - 520, 225, 519, 518, 841, 842, 843, 151, 157, 159, - 160, 161, 162, 513, 514, 165, 164, 214, 563, 520, - 341, 567, 568, 569, 570, 571, 572, 573, 1032, 576, + 380, 38, 565, 1054, 379, 885, 198, 908, 624, 677, + 664, 680, 353, 886, 641, 973, 432, 173, 882, 429, + 736, 782, 743, 943, 44, 856, 848, 240, 430, 3, + 746, 640, 637, 797, 242, 713, 650, 785, 238, 38, + 167, 408, 818, 342, 745, 760, 402, 178, 201, 348, + 213, 351, 193, 418, 673, 434, 182, 43, 603, 1085, + 1076, 1082, 1071, 1080, 189, 1075, 207, 172, 1070, 956, + 1008, 187, 217, 1031, 577, 168, 169, 170, 171, 814, + 913, 914, 915, 340, 658, 233, 657, 48, 188, 916, + 985, 665, 1037, 1003, 1001, 334, 335, 1079, 1077, 693, + 223, 1055, 748, 838, 604, 625, 627, 224, 50, 51, + 52, 53, 219, 691, 652, 135, 136, 136, 1057, 531, + 530, 540, 541, 533, 534, 535, 536, 537, 538, 539, + 532, 1029, 497, 542, 138, 139, 140, 492, 922, 796, + 697, 795, 214, 794, 215, 819, 235, 220, 237, 690, + 147, 137, 239, 239, 239, 239, 835, 239, 239, 652, + 554, 555, 837, 1047, 239, 520, 993, 234, 236, 970, + 535, 536, 537, 538, 539, 532, 860, 626, 542, 38, + 801, 564, 532, 450, 228, 542, 542, 808, 923, 517, + 449, 331, 332, 333, 431, 336, 337, 687, 692, 685, + 665, 404, 339, 958, 651, 761, 444, 405, 518, 649, + 648, 494, 207, 519, 518, 917, 153, 239, 695, 698, + 960, 812, 239, 232, 520, 1050, 1030, 1028, 446, 1069, + 520, 239, 239, 239, 239, 239, 239, 239, 239, 1058, + 163, 226, 720, 761, 944, 872, 654, 410, 689, 651, + 989, 655, 836, 1064, 834, 491, 718, 719, 717, 988, + 496, 502, 688, 504, 519, 518, 946, 406, 827, 505, + 506, 507, 508, 509, 510, 511, 512, 345, 403, 41, + 737, 520, 738, 706, 708, 709, 826, 694, 707, 716, + 148, 815, 1040, 948, 987, 952, 150, 947, 696, 945, + 211, 156, 152, 825, 950, 533, 534, 535, 536, 537, + 538, 539, 532, 949, 921, 542, 592, 593, 951, 953, + 1066, 341, 154, 1019, 341, 158, 63, 841, 842, 843, + 145, 1019, 1020, 145, 910, 239, 239, 540, 541, 533, + 534, 535, 536, 537, 538, 539, 532, 809, 149, 542, + 551, 553, 145, 145, 982, 981, 902, 341, 145, 969, + 341, 341, 519, 518, 854, 341, 867, 151, 157, 159, + 160, 161, 162, 513, 514, 165, 164, 522, 563, 520, + 739, 567, 568, 569, 570, 571, 572, 573, 490, 576, 578, 578, 578, 578, 578, 578, 578, 578, 586, 587, - 588, 589, 1031, 556, 557, 558, 559, 560, 561, 562, - 706, 708, 709, 1019, 1020, 707, 193, 193, 193, 193, - 193, 207, 594, 606, 519, 518, 982, 981, 902, 341, - 883, 960, 431, 443, 628, 918, 145, 443, 145, 355, - 193, 520, 145, 969, 341, 854, 341, 751, 145, 965, - 207, 415, 63, 63, 63, 63, 623, 63, 63, 631, - 928, 927, 881, 596, 63, 925, 924, 751, 341, 45, - 608, 609, 783, 611, 666, 667, 668, 415, 341, 636, - 642, 638, 619, 452, 451, 629, 634, 145, 595, 630, - 607, 192, 633, 610, 145, 145, 145, 783, 645, 926, - 19, 63, 239, 679, 414, 854, 441, 19, 854, 854, - 523, 802, 621, 622, 415, 590, 41, 63, 179, 145, - 659, 145, 63, 777, 145, 678, 778, 145, 415, 145, - 714, 63, 63, 63, 63, 63, 63, 63, 63, 443, - 701, 896, 805, 566, 675, 676, 41, 442, 681, 440, - 575, 674, 669, 41, 38, 55, 579, 580, 581, 582, - 583, 584, 585, 19, 41, 786, 787, 912, 567, 134, - 883, 712, 828, 789, 721, 722, 723, 724, 725, 726, - 727, 728, 729, 730, 731, 732, 733, 734, 735, 1014, - 500, 338, 623, 750, 618, 638, 424, 425, 660, 661, - 662, 663, 17, 616, 780, 781, 740, 741, 617, 41, - 765, 614, 635, 670, 671, 672, 615, 602, 758, 792, - 186, 791, 613, 753, 754, 612, 1077, 757, 779, 183, - 184, 1073, 840, 192, 768, 63, 63, 769, 702, 145, - 1072, 764, 774, 766, 767, 409, 773, 820, 752, 177, - 343, 448, 790, 803, 799, 800, 775, 231, 811, 407, - 1051, 763, 344, 368, 367, 370, 371, 372, 373, 642, - 816, 817, 369, 374, 1050, 1012, 806, 963, 991, 683, - 499, 703, 704, 239, 710, 711, 428, 807, 540, 541, - 533, 534, 535, 536, 537, 538, 539, 532, 180, 181, - 542, 239, 821, 822, 823, 409, 793, 772, 63, 174, - 1042, 175, 45, 145, 1041, 771, 145, 145, 145, 145, - 145, 830, 714, 1011, 783, 831, 1043, 986, 145, 516, - 566, 47, 145, 755, 756, 49, 145, 439, 42, 839, - 145, 145, 420, 423, 424, 425, 421, 1, 422, 426, - 686, 861, 63, 403, 844, 1052, 907, 647, 639, 212, - 54, 646, 824, 1026, 984, 845, 846, 847, 653, 813, - 656, 911, 1048, 810, 455, 456, 454, 458, 457, 888, - 453, 38, 155, 887, 207, 199, 884, 427, 445, 855, - 552, 680, 56, 145, 871, 898, 899, 900, 145, 853, - 833, 145, 63, 893, 832, 889, 684, 218, 550, 770, - 200, 890, 591, 869, 401, 892, 1040, 906, 1010, 870, - 903, 574, 851, 759, 354, 705, 852, 366, 363, 365, - 904, 642, 364, 642, 905, 597, 863, 864, 776, 524, - 868, 352, 346, 191, 411, 874, 419, 875, 876, 877, - 878, 417, 919, 920, 416, 192, 192, 192, 192, 192, - 196, 788, 784, 63, 190, 880, 1007, 1055, 601, 20, - 46, 192, 185, 16, 931, 933, 15, 63, 14, 192, - 901, 938, 193, 955, 942, 939, 13, 24, 954, 750, - 957, 638, 12, 941, 11, 10, 935, 936, 9, 964, - 8, 962, 7, 6, 961, 5, 4, 176, 18, 377, - 2, 972, 0, 0, 862, 975, 976, 977, 803, 0, - 978, 980, 0, 0, 873, 0, 0, 0, 0, 63, - 239, 0, 0, 0, 642, 61, 937, 0, 0, 0, - 0, 63, 0, 0, 0, 0, 894, 0, 0, 895, - 0, 0, 897, 0, 0, 0, 0, 1006, 0, 0, - 0, 0, 999, 208, 0, 715, 888, 0, 990, 1016, - 887, 966, 967, 968, 0, 971, 0, 1013, 0, 0, - 0, 992, 63, 63, 0, 0, 0, 0, 0, 1024, - 1025, 0, 1033, 1015, 0, 0, 0, 0, 0, 0, - 0, 63, 420, 423, 424, 425, 421, 1035, 422, 426, - 0, 0, 786, 787, 0, 888, 0, 38, 0, 887, - 1044, 0, 904, 0, 0, 994, 995, 0, 0, 0, - 0, 0, 0, 0, 0, 959, 0, 1004, 1005, 0, - 0, 1045, 0, 0, 0, 0, 1059, 0, 983, 63, - 207, 566, 1060, 1037, 0, 0, 0, 0, 1021, 1022, - 1023, 241, 241, 241, 241, 0, 241, 241, 1071, 0, - 0, 145, 0, 241, 0, 0, 0, 0, 0, 1080, - 0, 63, 63, 996, 997, 0, 998, 0, 0, 1000, - 1038, 1002, 0, 0, 0, 194, 0, 0, 0, 0, - 0, 0, 63, 63, 0, 63, 63, 0, 0, 208, - 241, 530, 540, 541, 533, 534, 535, 536, 537, 538, - 539, 532, 0, 1009, 542, 142, 241, 1082, 1061, 145, - 0, 241, 0, 145, 341, 1064, 0, 0, 1067, 63, - 241, 241, 241, 241, 241, 241, 241, 241, 197, 0, - 0, 0, 0, 216, 0, 0, 0, 715, 63, 0, - 0, 0, 378, 0, 0, 1085, 1086, 0, 0, 0, - 531, 530, 540, 541, 533, 534, 535, 536, 537, 538, - 539, 532, 145, 0, 542, 0, 0, 0, 0, 0, - 0, 0, 143, 0, 0, 166, 0, 63, 0, 63, - 63, 63, 145, 63, 1058, 566, 63, 0, 0, 0, - 0, 0, 0, 0, 143, 143, 209, 0, 0, 0, - 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 63, 221, 0, 222, 0, 0, 0, 227, 0, 526, - 0, 529, 0, 229, 241, 241, 0, 543, 544, 545, - 546, 547, 548, 549, 0, 527, 528, 525, 531, 530, - 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, - 63, 63, 542, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 413, 63, 0, 0, 0, 0, 0, 0, - 0, 438, 0, 0, 63, 0, 0, 0, 143, 0, - 143, 0, 0, 0, 143, 0, 0, 0, 0, 0, - 143, 0, 0, 0, 493, 0, 495, 598, 0, 498, - 63, 192, 501, 0, 208, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 19, 39, 21, 22, - 0, 0, 0, 934, 0, 0, 0, 63, 0, 143, - 0, 0, 0, 208, 33, 63, 143, 436, 143, 23, - 241, 241, 209, 531, 530, 540, 541, 533, 534, 535, - 536, 537, 538, 539, 532, 850, 0, 542, 32, 0, - 0, 143, 41, 143, 0, 0, 143, 0, 0, 143, - 0, 503, 0, 0, 0, 531, 530, 540, 541, 533, - 534, 535, 536, 537, 538, 539, 532, 0, 0, 542, - 0, 241, 531, 530, 540, 541, 533, 534, 535, 536, - 537, 538, 539, 532, 0, 0, 542, 0, 0, 0, - 0, 0, 0, 0, 515, 0, 0, 0, 0, 0, - 0, 25, 26, 28, 27, 30, 0, 0, 0, 0, - 0, 0, 0, 0, 31, 34, 35, 0, 0, 36, - 37, 29, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 742, 0, 241, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 762, 0, 0, 0, + 588, 589, 928, 927, 925, 924, 556, 557, 558, 559, + 560, 561, 562, 521, 519, 518, 193, 193, 193, 193, + 193, 606, 594, 751, 341, 1033, 866, 207, 865, 519, + 518, 520, 431, 230, 628, 225, 145, 214, 145, 1032, + 193, 918, 145, 443, 519, 518, 520, 751, 145, 45, + 523, 783, 63, 63, 63, 63, 207, 63, 63, 631, + 623, 520, 965, 596, 63, 579, 580, 581, 582, 583, + 584, 585, 415, 341, 19, 666, 667, 668, 636, 642, + 619, 608, 609, 566, 611, 638, 634, 145, 854, 595, + 575, 629, 633, 415, 145, 145, 145, 630, 415, 679, + 1014, 63, 239, 645, 607, 452, 451, 610, 441, 883, + 881, 926, 443, 621, 622, 783, 854, 63, 802, 145, + 41, 145, 63, 19, 145, 590, 414, 145, 41, 145, + 714, 63, 63, 63, 63, 63, 63, 63, 63, 659, + 701, 675, 676, 678, 660, 661, 662, 663, 179, 442, + 415, 440, 635, 854, 38, 896, 681, 443, 134, 670, + 671, 672, 368, 367, 370, 371, 372, 373, 567, 41, + 805, 369, 374, 674, 712, 786, 787, 721, 722, 723, + 724, 725, 726, 727, 728, 729, 730, 731, 732, 733, + 734, 735, 750, 669, 41, 55, 623, 19, 912, 638, + 883, 828, 789, 500, 780, 781, 740, 741, 338, 186, + 765, 616, 792, 753, 754, 758, 617, 757, 17, 602, + 777, 703, 704, 778, 710, 711, 618, 791, 424, 425, + 613, 764, 779, 766, 767, 63, 63, 768, 769, 145, + 614, 612, 1078, 41, 1074, 615, 775, 840, 790, 752, + 183, 184, 799, 800, 702, 803, 1073, 420, 423, 424, + 425, 421, 763, 422, 426, 177, 774, 773, 642, 820, + 566, 816, 817, 755, 756, 343, 409, 448, 420, 423, + 424, 425, 421, 239, 422, 426, 807, 344, 786, 787, + 407, 231, 811, 403, 1052, 1051, 355, 1012, 806, 963, + 991, 239, 683, 499, 428, 180, 181, 793, 63, 821, + 822, 823, 409, 145, 174, 772, 145, 145, 145, 145, + 145, 830, 714, 771, 1043, 831, 175, 45, 145, 1042, + 1011, 783, 145, 1044, 986, 516, 145, 47, 49, 839, + 145, 145, 439, 42, 1, 686, 1053, 907, 192, 647, + 639, 861, 63, 844, 530, 540, 541, 533, 534, 535, + 536, 537, 538, 539, 532, 212, 54, 542, 845, 846, + 847, 646, 824, 1027, 984, 653, 813, 656, 911, 888, + 1049, 38, 810, 887, 884, 455, 456, 454, 458, 853, + 207, 457, 453, 145, 155, 898, 899, 900, 145, 199, + 427, 145, 63, 869, 445, 871, 855, 893, 56, 889, + 833, 832, 684, 218, 550, 892, 770, 200, 890, 906, + 591, 401, 1041, 851, 903, 1010, 870, 852, 574, 904, + 642, 759, 642, 354, 905, 705, 366, 863, 864, 363, + 365, 868, 364, 597, 776, 524, 874, 352, 875, 876, + 877, 878, 377, 346, 862, 191, 411, 419, 417, 919, + 920, 416, 196, 63, 873, 788, 784, 190, 880, 1007, + 1056, 601, 933, 20, 931, 46, 185, 63, 61, 16, + 15, 901, 193, 939, 942, 938, 894, 954, 750, 895, + 192, 955, 897, 941, 957, 638, 14, 13, 24, 935, + 936, 962, 961, 964, 12, 11, 208, 10, 9, 8, + 7, 972, 6, 5, 975, 976, 977, 4, 978, 176, + 803, 980, 18, 2, 0, 0, 0, 0, 0, 63, + 239, 0, 0, 642, 0, 0, 0, 937, 0, 0, + 0, 63, 531, 530, 540, 541, 533, 534, 535, 536, + 537, 538, 539, 532, 0, 0, 542, 1006, 999, 0, + 0, 0, 0, 0, 0, 0, 888, 0, 990, 1016, + 887, 0, 966, 967, 968, 959, 971, 0, 0, 1013, + 849, 0, 63, 63, 992, 0, 0, 1024, 0, 0, + 1026, 566, 1034, 1025, 983, 0, 0, 1015, 0, 0, + 0, 63, 0, 0, 241, 241, 241, 241, 1036, 241, + 241, 0, 0, 0, 0, 888, 241, 38, 0, 887, + 1045, 904, 0, 0, 0, 0, 994, 995, 0, 996, + 997, 0, 998, 0, 0, 1000, 0, 1002, 1004, 1005, + 0, 0, 0, 0, 0, 1046, 1060, 552, 0, 63, + 0, 1061, 208, 241, 0, 0, 1038, 207, 0, 1021, + 1022, 1023, 0, 1009, 0, 0, 0, 0, 0, 241, + 1072, 145, 0, 0, 241, 0, 0, 0, 0, 0, + 1081, 63, 63, 241, 241, 241, 241, 241, 241, 241, + 241, 1039, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 63, 63, 0, 63, 63, 0, 0, 0, + 0, 194, 192, 192, 192, 192, 192, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 192, 145, + 1062, 1083, 0, 145, 0, 0, 192, 1065, 0, 63, + 1068, 142, 0, 0, 0, 1059, 566, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 63, 0, + 0, 0, 0, 0, 197, 0, 0, 1086, 1087, 216, + 0, 0, 0, 341, 378, 0, 0, 0, 0, 0, + 0, 0, 145, 0, 0, 0, 0, 241, 241, 0, + 0, 0, 0, 0, 0, 0, 0, 63, 0, 63, + 63, 63, 145, 63, 143, 0, 63, 166, 0, 531, + 530, 540, 541, 533, 534, 535, 536, 537, 538, 539, + 532, 0, 715, 542, 0, 0, 143, 143, 209, 0, + 63, 0, 143, 0, 531, 530, 540, 541, 533, 534, + 535, 536, 537, 538, 539, 532, 0, 221, 542, 222, + 0, 0, 0, 227, 0, 0, 0, 0, 0, 229, + 598, 0, 0, 0, 0, 0, 0, 208, 0, 0, + 63, 63, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 63, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 63, 0, 208, 0, 413, 0, + 0, 0, 0, 241, 241, 0, 0, 438, 0, 0, + 143, 0, 143, 0, 0, 0, 143, 0, 0, 0, + 63, 0, 143, 0, 0, 0, 0, 0, 0, 0, + 493, 0, 495, 0, 0, 498, 0, 0, 501, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 63, 0, + 0, 0, 0, 0, 241, 0, 63, 0, 0, 0, + 0, 143, 0, 0, 19, 39, 21, 22, 143, 436, + 143, 0, 0, 0, 209, 0, 0, 0, 0, 0, + 0, 0, 33, 0, 0, 0, 0, 23, 0, 0, + 0, 0, 0, 143, 0, 143, 0, 0, 143, 0, + 0, 143, 0, 503, 0, 0, 32, 0, 0, 0, + 41, 0, 0, 0, 715, 742, 0, 241, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 762, + 0, 0, 0, 0, 0, 526, 0, 529, 0, 0, + 0, 0, 0, 543, 544, 545, 546, 547, 548, 549, + 515, 527, 528, 525, 531, 530, 540, 541, 533, 534, + 535, 536, 537, 538, 539, 532, 0, 0, 542, 25, + 26, 28, 27, 30, 934, 0, 0, 0, 0, 0, + 0, 798, 31, 34, 35, 0, 0, 36, 37, 29, + 0, 0, 0, 241, 531, 530, 540, 541, 533, 534, + 535, 536, 537, 538, 539, 532, 0, 0, 542, 0, + 0, 0, 0, 143, 0, 0, 850, 0, 0, 0, + 0, 0, 0, 0, 605, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 829, 241, 531, 530, 540, 541, + 533, 534, 535, 536, 537, 538, 539, 532, 0, 0, + 542, 0, 632, 241, 0, 0, 0, 0, 0, 40, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 143, 0, 0, 0, 0, 0, 0, 605, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 192, 0, + 0, 0, 0, 0, 0, 0, 0, 143, 0, 209, + 143, 143, 143, 143, 143, 0, 0, 0, 0, 0, + 0, 858, 620, 0, 682, 0, 143, 0, 0, 699, + 436, 0, 700, 0, 143, 143, 0, 0, 209, 0, + 0, 0, 0, 0, 0, 503, 0, 0, 0, 0, + 208, 0, 0, 891, 798, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 241, 241, 0, 241, 909, 0, + 0, 0, 0, 0, 0, 0, 0, 143, 0, 0, + 0, 0, 143, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 40, 0, 0, 0, 0, 632, 0, 798, 0, + 0, 932, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 858, 0, 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 143, 0, 209, 143, 143, - 143, 143, 143, 461, 0, 0, 0, 0, 682, 0, - 620, 829, 241, 699, 143, 0, 700, 0, 436, 0, - 0, 0, 143, 143, 0, 473, 209, 0, 0, 0, - 241, 0, 0, 503, 0, 0, 0, 0, 0, 0, - 0, 478, 479, 480, 481, 482, 483, 484, 0, 485, - 486, 487, 488, 489, 474, 475, 476, 477, 459, 460, - 0, 0, 462, 0, 463, 464, 465, 466, 467, 468, - 469, 470, 471, 472, 0, 143, 0, 0, 858, 0, - 143, 0, 0, 143, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 749, 503, + 0, 0, 0, 749, 749, 461, 0, 749, 0, 974, + 0, 974, 974, 974, 0, 979, 0, 0, 241, 0, + 0, 749, 749, 749, 749, 0, 0, 473, 0, 0, + 0, 0, 0, 0, 0, 0, 749, 0, 0, 0, + 0, 0, 241, 478, 479, 480, 481, 482, 483, 484, + 0, 485, 486, 487, 488, 489, 474, 475, 476, 477, + 459, 460, 0, 0, 462, 0, 463, 464, 465, 466, + 467, 468, 469, 470, 471, 472, 0, 0, 0, 0, + 0, 0, 1017, 1018, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 909, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 208, 0, 0, - 891, 798, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 241, 241, 0, 241, 909, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 749, 503, 0, 0, - 0, 749, 749, 0, 0, 749, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 932, 749, - 749, 749, 749, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 749, 0, 0, 858, 0, 0, - 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 974, 0, 974, 974, - 974, 0, 979, 0, 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 241, + 0, 0, 1048, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 879, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 208, 0, 0, + 1063, 0, 0, 0, 0, 0, 0, 0, 1067, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 749, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 749, 0, 0, 0, 0, 0, 0, + 929, 0, 0, 0, 930, 143, 0, 0, 0, 0, + 0, 0, 209, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 879, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1017, - 1018, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 909, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 241, 0, 0, 0, 749, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 749, 0, 0, 929, 0, 0, 0, 930, 1047, - 0, 0, 0, 143, 0, 0, 0, 0, 0, 0, - 209, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 208, 0, 0, 1062, 0, 0, 0, - 0, 0, 0, 0, 1066, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 143, 0, 0, 0, 143, 0, 0, 0, 0, + 0, 0, 0, 143, 0, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 749, 0, 0, - 0, 0, 0, 503, 749, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 749, + 0, 0, 0, 0, 0, 503, 749, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 143, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 436, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 436, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 320, 310, 283, 322, 261, 275, 330, + 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, + 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, + 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, + 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, + 0, 0, 0, 62, 0, 643, 644, 0, 0, 0, + 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, + 0, 247, 250, 329, 317, 267, 268, 804, 0, 0, + 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, + 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, + 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, + 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, + 321, 101, 313, 263, 271, 73, 269, 106, 99, 209, + 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, + 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, + 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, + 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, + 91, 89, 84, 78, 0, 246, 0, 109, 120, 133, + 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, + 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, + 252, 0, 0, 311, 295, 64, 0, 86, 131, 103, + 79, 121, 320, 310, 283, 322, 261, 275, 330, 276, + 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, + 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, + 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, + 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, + 0, 0, 62, 0, 643, 644, 0, 0, 0, 0, + 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, + 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, + 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, + 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, + 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, + 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, + 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, + 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, + 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, + 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, + 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, + 89, 84, 78, 0, 246, 0, 109, 120, 133, 259, + 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, + 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, + 0, 0, 311, 295, 64, 0, 86, 131, 103, 79, + 121, 320, 310, 283, 322, 261, 275, 330, 276, 277, + 304, 249, 291, 98, 273, 0, 264, 244, 270, 245, + 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, + 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, + 282, 305, 254, 298, 323, 274, 302, 324, 0, 0, + 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, + 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, + 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, + 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, + 0, 0, 1035, 0, 265, 0, 297, 0, 0, 0, + 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, + 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, + 313, 263, 271, 73, 269, 106, 99, 0, 296, 100, + 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, + 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, + 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, + 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, + 84, 78, 0, 246, 0, 109, 120, 133, 259, 316, + 127, 128, 129, 130, 95, 71, 81, 107, 257, 258, + 255, 256, 292, 293, 325, 326, 327, 308, 252, 0, + 0, 311, 295, 64, 0, 86, 131, 103, 79, 121, 320, 310, 283, 322, 261, 275, 330, 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, - 305, 254, 298, 323, 274, 302, 324, 0, 0, 0, - 62, 0, 643, 644, 0, 0, 0, 0, 0, 72, + 305, 254, 298, 323, 274, 302, 324, 41, 0, 0, + 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, - 329, 317, 267, 268, 804, 0, 0, 0, 0, 0, + 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, - 263, 271, 73, 269, 106, 99, 209, 296, 100, 105, + 263, 271, 73, 269, 106, 99, 0, 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, @@ -797,12 +861,12 @@ var yyAct = [...]int{ 291, 98, 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, - 254, 298, 323, 274, 302, 324, 0, 0, 0, 62, - 0, 643, 644, 0, 0, 0, 0, 0, 72, 0, + 254, 298, 323, 274, 302, 324, 0, 0, 0, 210, + 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, 0, 0, - 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, + 940, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, 100, 105, 88, @@ -819,10 +883,10 @@ var yyAct = [...]int{ 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, 0, 0, 62, 0, - 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, + 447, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, - 290, 306, 280, 0, 0, 0, 0, 0, 0, 1034, + 290, 306, 280, 0, 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, @@ -839,7 +903,7 @@ var yyAct = [...]int{ 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, - 323, 274, 302, 324, 41, 0, 0, 62, 0, 0, + 323, 274, 302, 324, 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, @@ -864,7 +928,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, - 280, 0, 0, 0, 0, 0, 0, 940, 0, 265, + 280, 0, 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, 73, 269, @@ -881,7 +945,7 @@ var yyAct = [...]int{ 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, 323, 274, - 302, 324, 0, 0, 0, 62, 0, 447, 0, 0, + 302, 324, 0, 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, @@ -897,78 +961,35 @@ var yyAct = [...]int{ 120, 133, 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, 0, 0, 311, 295, 64, 0, 86, - 131, 103, 79, 121, 320, 310, 283, 322, 261, 275, - 330, 276, 277, 304, 249, 291, 98, 273, 0, 264, - 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, - 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, - 314, 289, 309, 282, 305, 254, 298, 323, 274, 302, - 324, 0, 0, 0, 62, 0, 0, 0, 0, 0, - 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, - 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, - 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, - 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, - 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, - 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, - 278, 321, 101, 313, 263, 271, 73, 269, 106, 99, - 0, 296, 100, 105, 88, 113, 102, 119, 125, 126, - 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, - 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, - 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, - 116, 91, 89, 84, 78, 0, 246, 0, 109, 120, - 133, 259, 316, 127, 128, 129, 130, 95, 71, 81, - 107, 257, 258, 255, 256, 292, 293, 325, 326, 327, - 308, 252, 0, 0, 311, 295, 64, 0, 86, 131, - 103, 79, 121, 320, 310, 283, 322, 261, 275, 330, - 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, - 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, - 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, - 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, - 0, 0, 0, 210, 0, 0, 0, 0, 0, 0, - 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, - 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, - 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, - 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, - 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, - 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, - 321, 101, 313, 263, 271, 73, 269, 106, 99, 0, - 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, - 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, - 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, - 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, - 91, 89, 84, 78, 0, 246, 0, 109, 120, 133, - 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, - 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, - 252, 0, 0, 311, 295, 64, 0, 86, 131, 103, - 79, 121, 320, 310, 283, 322, 261, 275, 330, 276, - 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, - 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, - 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, - 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, - 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, - 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, - 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, - 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, - 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, - 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, - 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, - 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, + 131, 103, 79, 121, 98, 0, 0, 744, 0, 350, + 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, + 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, + 382, 0, 0, 0, 0, 0, 0, 0, 0, 41, + 0, 0, 210, 368, 367, 370, 371, 372, 373, 0, + 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, + 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 358, 359, 747, 0, 0, 0, 399, 0, 360, + 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, + 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, - 89, 84, 78, 0, 246, 0, 109, 120, 133, 259, - 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, - 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, - 0, 0, 311, 295, 64, 0, 86, 131, 103, 79, - 121, 98, 0, 0, 744, 0, 350, 0, 0, 0, + 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, + 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, + 398, 395, 396, 393, 394, 392, 391, 390, 400, 383, + 384, 386, 0, 385, 64, 0, 86, 131, 103, 79, + 121, 98, 0, 0, 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, - 0, 0, 0, 0, 0, 0, 41, 0, 0, 210, + 0, 0, 0, 0, 0, 0, 41, 0, 341, 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 358, 359, - 747, 0, 0, 0, 399, 0, 360, 0, 0, 356, + 0, 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, @@ -983,11 +1004,11 @@ var yyAct = [...]int{ 0, 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, 0, - 0, 0, 0, 41, 0, 341, 210, 368, 367, 370, + 0, 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 358, 359, 0, 0, 0, + 0, 0, 0, 0, 0, 358, 359, 747, 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, 0, @@ -998,28 +1019,28 @@ var yyAct = [...]int{ 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, 392, - 391, 390, 400, 383, 384, 386, 0, 385, 64, 0, - 86, 131, 103, 79, 121, 98, 0, 0, 0, 0, - 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, - 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, - 381, 382, 0, 0, 0, 0, 0, 0, 0, 0, - 41, 0, 0, 210, 368, 367, 370, 371, 372, 373, - 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, - 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, + 391, 390, 400, 383, 384, 386, 19, 385, 64, 0, + 86, 131, 103, 79, 121, 0, 0, 98, 0, 0, + 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, + 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, + 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, + 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, + 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 358, 359, 747, 0, 0, 0, 399, 0, - 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, - 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, - 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, - 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, - 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, - 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, - 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, - 389, 398, 395, 396, 393, 394, 392, 391, 390, 400, - 383, 384, 386, 19, 385, 64, 0, 86, 131, 103, - 79, 121, 0, 0, 98, 0, 0, 0, 0, 350, + 0, 0, 0, 0, 358, 359, 0, 0, 0, 0, + 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, + 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, + 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, + 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, + 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, + 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, + 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, + 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, + 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, + 390, 400, 383, 384, 386, 0, 385, 64, 0, 86, + 131, 103, 79, 121, 98, 0, 0, 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, 0, 0, 41, @@ -1038,111 +1059,55 @@ var yyAct = [...]int{ 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, 390, 400, 383, - 384, 386, 0, 385, 64, 0, 86, 131, 103, 79, - 121, 98, 0, 0, 0, 0, 350, 0, 0, 0, - 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, - 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, - 0, 0, 0, 0, 0, 0, 41, 0, 0, 210, - 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, - 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, + 384, 386, 98, 385, 64, 0, 86, 131, 103, 79, + 121, 77, 0, 0, 0, 0, 85, 388, 87, 0, + 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, + 0, 0, 0, 0, 0, 0, 0, 41, 0, 0, + 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, + 369, 374, 375, 376, 0, 0, 0, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 358, 359, - 0, 0, 0, 0, 399, 0, 360, 0, 0, 356, - 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, - 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, - 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, - 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, - 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, - 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, - 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, - 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, - 393, 394, 392, 391, 390, 400, 383, 384, 386, 98, - 385, 64, 0, 86, 131, 103, 79, 121, 77, 0, - 0, 0, 0, 85, 388, 87, 0, 0, 108, 94, - 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, - 0, 0, 0, 0, 41, 0, 0, 210, 368, 367, - 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, - 376, 0, 0, 0, 361, 0, 387, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 358, 359, 0, 0, - 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, - 0, 106, 99, 0, 1083, 100, 105, 88, 113, 102, - 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, - 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, - 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, - 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, - 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, - 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, - 392, 391, 390, 400, 383, 384, 386, 98, 385, 64, - 0, 86, 131, 103, 79, 121, 77, 0, 0, 0, - 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, - 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, - 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, - 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, - 0, 0, 361, 0, 387, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 358, 359, 0, 0, 0, 0, - 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, - 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, - 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, - 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, - 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, - 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, - 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, - 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, - 390, 400, 383, 384, 386, 98, 385, 64, 0, 86, - 131, 103, 79, 121, 77, 0, 0, 0, 0, 85, - 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, - 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 531, 530, - 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, - 0, 0, 542, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, - 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, - 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, - 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, - 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, - 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, - 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 64, 0, 86, 131, 103, - 79, 121, 98, 0, 0, 0, 857, 0, 0, 0, - 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, - 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 62, 0, 859, 0, 0, 0, 0, 0, 0, 72, - 0, 0, 0, 0, 519, 518, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 520, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, - 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 358, + 359, 0, 0, 0, 0, 399, 0, 360, 0, 0, + 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, + 0, 0, 73, 0, 106, 99, 0, 1084, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, - 78, 0, 0, 0, 109, 120, 133, 0, 98, 127, - 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, + 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, + 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, + 396, 393, 394, 392, 391, 390, 400, 383, 384, 386, + 98, 385, 64, 0, 86, 131, 103, 79, 121, 77, + 0, 0, 0, 0, 85, 388, 87, 0, 0, 108, + 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, + 0, 0, 0, 0, 0, 41, 0, 0, 210, 368, + 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, + 375, 376, 0, 0, 0, 361, 0, 387, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 358, 359, 0, + 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, + 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, + 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, + 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, + 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, + 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, + 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, + 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, + 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, + 394, 392, 391, 390, 400, 383, 384, 386, 98, 385, + 64, 0, 86, 131, 103, 79, 121, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, - 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, - 58, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 59, 0, 57, - 0, 0, 0, 60, 101, 0, 0, 0, 73, 0, + 0, 531, 530, 540, 541, 533, 534, 535, 536, 537, + 538, 539, 532, 0, 0, 542, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, + 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, @@ -1151,14 +1116,14 @@ var yyAct = [...]int{ 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 64, 0, - 86, 131, 103, 79, 121, 98, 0, 0, 0, 435, + 86, 131, 103, 79, 121, 98, 0, 0, 0, 857, 0, 0, 0, 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 144, 0, 437, 0, 0, 0, 0, - 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 62, 0, 859, 0, 0, 0, 0, + 0, 0, 72, 0, 0, 0, 0, 519, 518, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 520, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, @@ -1168,68 +1133,68 @@ var yyAct = [...]int{ 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, - 0, 0, 19, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 98, 0, 64, 0, 86, 131, 103, - 79, 121, 77, 0, 0, 0, 0, 85, 0, 87, - 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 41, 0, - 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, - 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, - 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, - 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, - 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, - 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, - 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, - 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, - 127, 128, 129, 130, 95, 71, 81, 107, 0, 0, - 19, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 98, 0, 64, 0, 86, 131, 103, 79, 121, + 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, - 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 41, 0, 0, 144, + 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, + 79, 121, 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, + 0, 0, 0, 58, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, + 59, 0, 57, 0, 0, 0, 60, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, - 0, 0, 0, 109, 120, 133, 0, 98, 127, 128, - 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, - 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, - 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, - 0, 0, 0, 0, 0, 62, 0, 0, 599, 0, - 0, 600, 0, 0, 72, 0, 0, 0, 0, 0, + 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, + 129, 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 64, 0, 86, 131, 103, 79, 121, 98, 0, + 0, 0, 435, 0, 0, 0, 0, 77, 0, 0, + 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 144, 0, 437, 0, + 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, - 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, - 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, - 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, - 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, - 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, - 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, - 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 64, 0, 86, - 131, 103, 79, 121, 98, 0, 0, 0, 435, 0, - 0, 0, 0, 77, 0, 0, 0, 0, 85, 0, - 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 144, 0, 437, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, + 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, + 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, + 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, + 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, + 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, + 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, + 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, + 71, 81, 107, 0, 0, 19, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 98, 0, 64, 0, + 86, 131, 103, 79, 121, 77, 0, 0, 0, 0, + 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 41, 0, 0, 62, 0, 0, 0, 0, 0, + 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, + 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, + 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, + 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, + 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, + 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, + 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, + 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, + 107, 0, 0, 19, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 98, 0, 64, 0, 86, 131, + 103, 79, 121, 77, 0, 0, 0, 0, 85, 0, + 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 41, + 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1237,7 +1202,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, - 433, 105, 88, 113, 102, 119, 125, 126, 111, 124, + 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, @@ -1245,8 +1210,8 @@ var yyAct = [...]int{ 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, - 121, 0, 0, 0, 0, 41, 0, 0, 144, 0, - 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, + 121, 0, 0, 0, 0, 0, 0, 0, 62, 0, + 0, 599, 0, 0, 600, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1258,63 +1223,14 @@ var yyAct = [...]int{ 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, - 0, 0, 109, 120, 133, 0, 98, 127, 128, 129, - 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, - 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, - 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, - 0, 0, 0, 0, 62, 0, 859, 0, 0, 0, - 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, + 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, + 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, - 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, - 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, - 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, - 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, - 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, - 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, - 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, - 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, - 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, - 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, - 144, 0, 437, 0, 0, 0, 0, 0, 0, 72, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, - 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, - 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, - 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, - 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, - 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, - 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, - 128, 129, 130, 95, 71, 81, 107, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 98, 64, 0, 86, 131, 103, 79, 121, 412, - 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, - 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 144, - 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, - 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, - 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, - 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, - 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, - 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, - 195, 0, 0, 109, 120, 133, 0, 98, 127, 128, - 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, + 64, 0, 86, 131, 103, 79, 121, 98, 0, 0, + 0, 435, 0, 0, 0, 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, - 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, - 0, 0, 0, 0, 0, 144, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 144, 0, 437, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1322,7 +1238,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, + 99, 0, 0, 433, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, @@ -1330,14 +1246,14 @@ var yyAct = [...]int{ 120, 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, - 131, 103, 79, 121, 0, 0, 0, 0, 0, 0, + 131, 103, 79, 121, 0, 0, 0, 0, 41, 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 141, 0, 146, 0, 0, 0, 0, 101, + 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, @@ -1347,7 +1263,7 @@ var yyAct = [...]int{ 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, 121, - 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 62, 0, 859, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1364,7 +1280,7 @@ var yyAct = [...]int{ 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, 0, - 0, 0, 0, 210, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 144, 0, 437, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1377,52 +1293,137 @@ var yyAct = [...]int{ 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, - 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, - 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, - 79, 121, 0, 0, 0, 0, 0, 0, 0, 144, - 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, + 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 98, 64, 0, 86, 131, 103, + 79, 121, 412, 77, 0, 0, 0, 0, 85, 0, + 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, + 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, - 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, - 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, - 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, - 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, - 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, - 0, 0, 0, 109, 120, 133, 0, 98, 127, 128, - 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, - 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, - 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, - 0, 0, 0, 0, 0, 210, 0, 0, 0, 0, - 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, + 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, + 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, + 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, + 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, + 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, + 89, 84, 78, 195, 0, 0, 109, 120, 133, 0, + 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, + 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, + 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, + 121, 0, 0, 0, 0, 0, 0, 0, 144, 0, + 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, - 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, - 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, - 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, - 69, 123, 68, 205, 122, 96, 112, 118, 93, 90, - 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, - 120, 133, 0, 0, 127, 128, 129, 130, 206, 204, - 203, 202, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 64, 0, 86, - 131, 103, 79, 121, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, + 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, + 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, + 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, + 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, + 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, + 0, 0, 109, 120, 133, 0, 98, 127, 128, 129, + 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, + 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, + 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, + 0, 0, 0, 0, 144, 0, 0, 0, 0, 0, + 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 141, 0, 146, 0, 0, + 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, + 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, + 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, + 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, + 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, + 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, + 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, + 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, + 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, + 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, + 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, + 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, + 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, + 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, + 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, + 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, + 78, 0, 0, 0, 109, 120, 133, 0, 98, 127, + 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, + 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, + 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, + 0, 0, 0, 0, 0, 0, 210, 0, 0, 0, + 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, + 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, + 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, + 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, + 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, + 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, + 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, + 109, 120, 133, 0, 98, 127, 128, 129, 130, 95, + 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, + 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, + 86, 131, 103, 79, 121, 0, 0, 0, 0, 0, + 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, + 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, + 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, + 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, + 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, + 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, + 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, + 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, + 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, + 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, + 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, + 121, 0, 0, 0, 0, 0, 0, 0, 210, 0, + 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, + 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, + 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, + 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, + 115, 74, 132, 69, 123, 68, 205, 122, 96, 112, + 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, + 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, + 130, 206, 204, 203, 202, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 64, 0, 86, 131, 103, 79, 121, } var yyPact = [...]int{ - 1330, -1000, -156, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 698, 726, -1000, - -1000, -1000, -1000, -1000, 503, 5661, 21, 54, 33, 7616, - 53, 185, 8114, -1000, -1000, -1000, -1000, -1000, 501, -1000, - -1000, -1000, -1000, -1000, 693, 696, 512, 679, 591, -1000, - 23, 6763, 7450, 8280, -1000, 322, 46, 8114, -131, 16, + 1358, -1000, -157, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 713, 732, -1000, + -1000, -1000, -1000, -1000, 543, 5674, -3, 35, 18, 7629, + 34, 185, 8127, -1000, -1000, -1000, -1000, -1000, 517, -1000, + -1000, -1000, -1000, -1000, 698, 711, 542, 686, 612, -1000, + -4, 6776, 7463, 8293, -1000, 382, 27, 8127, -129, -9, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1430,17 +1431,17 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 29, 8114, -1000, 8114, 13, 306, - 13, 8114, -1000, 75, -1000, -1000, -1000, 8114, 304, 628, - 30, 2702, 2702, 2702, 2702, -29, 2702, 2702, 541, -1000, - -1000, -1000, -1000, 2702, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 326, 632, 4734, 4734, 698, -1000, 501, -1000, - -1000, -1000, 625, -1000, -1000, 182, 7284, 475, 702, -1000, - -1000, -1000, 665, 6234, 6597, 8114, 496, -1000, 384, 7948, - 3120, -1000, -1000, -1000, -1000, 622, -1000, 121, -1000, 74, - -1000, -1000, 430, -1000, 1478, 302, 2702, 24, 8114, 140, - 8114, 2702, 25, 8114, 658, 540, 8114, -1000, 3747, -1000, - 2702, 2702, 2702, 2702, 2702, 2702, 2702, 2702, -1000, -1000, + -1000, -1000, -1000, -1000, 31, 8127, -1000, 8127, -14, 380, + -14, 8127, -1000, 76, -1000, -1000, -1000, 8127, 378, 662, + 30, 2715, 2715, 2715, 2715, -46, 2715, 2715, 558, -1000, + -1000, -1000, -1000, 2715, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 307, 657, 4747, 4747, 713, -1000, 517, -1000, + -1000, -1000, 656, -1000, -1000, 184, 7297, 497, 617, -1000, + -1000, -1000, 683, 6247, 6610, 8127, 498, -1000, 390, 7961, + 3133, -1000, -1000, -1000, -1000, 648, -1000, 111, -1000, 75, + -1000, -1000, 452, -1000, 1640, 333, 2715, 19, 8127, 140, + 8127, 2715, 13, 8127, 681, 553, 8127, -1000, 3760, -1000, + 2715, 2715, 2715, 2715, 2715, 2715, 2715, 2715, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1450,102 +1451,102 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 2702, 2702, -1000, -1000, 8114, -1000, - -1000, -1000, -1000, 721, 97, 274, -1000, 4734, 1168, 464, - 464, -1000, -1000, 65, -1000, -1000, 5110, 5110, 5110, 5110, - 5110, 5110, 5110, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 464, 73, -1000, - 4537, 464, 464, 464, 464, 464, 464, 4734, 464, 464, - 464, 464, 464, 464, 464, 464, 464, 464, 464, 464, - 464, 462, -1000, 293, 693, 326, 591, 6400, 576, -1000, - -1000, 0, 8114, -1000, 7948, 6763, 6763, 6763, 6763, 6763, - -1000, 585, 582, -1000, 571, 563, 554, 8114, -1000, 424, - 326, 6234, 87, 464, -1000, 7095, -1000, -1000, 0, 6763, - 8114, -1000, -1000, 7948, 384, -1000, -1000, -1000, -1000, 4734, - 3538, 2284, 160, 186, -97, -1000, -1000, 468, -1000, 468, - 468, 468, 468, -77, -77, -77, -77, -1000, -1000, -1000, - -1000, -1000, 500, -1000, 468, 468, 468, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 499, 499, 499, 473, 473, - 497, -1000, 8114, -1000, 657, 76, -1000, 8114, -1000, -1000, - 8114, 2702, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 602, 4734, 4734, - 344, 4734, 4734, 112, 5110, 232, 132, 5110, 5110, 5110, - 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, - 5110, 5110, 277, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 294, -1000, 501, 607, 607, 81, 81, 81, 81, - 81, 81, 5298, 3944, 3538, 414, 217, 4537, 4338, 4338, - 4734, 4734, 4338, 685, 124, 217, 7782, -1000, 326, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 4338, 4338, 4338, 4338, - 4734, -1000, -1000, -1000, 632, -1000, 685, 697, -1000, 614, - 610, 4338, -1000, 494, 464, -1000, 486, 702, 515, 523, - 962, -1000, -1000, -1000, -1000, 581, -1000, 579, -1000, -1000, - -1000, -1000, -1000, 326, -1000, 44, 43, 42, 7782, -1000, - 713, 461, -1000, -1000, -1000, 217, -1000, 72, -1000, 458, - 2075, -1000, -1000, -1000, -1000, -1000, -1000, 490, 649, 146, - 292, -1000, -1000, 630, -1000, 164, -108, -1000, -1000, 234, - -77, -77, -1000, -1000, 80, 618, 80, 80, 80, 267, - -1000, -1000, -1000, -1000, 230, -1000, -1000, -1000, 227, -1000, - 522, 7782, 2702, -1000, -1000, 145, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -3, -1000, - 2702, -1000, 595, 112, 155, -1000, -1000, 298, -1000, -1000, - 217, 217, 1322, -1000, -1000, -1000, -1000, 232, 5110, 5110, - 5110, 28, 1322, 1305, 596, 1020, 81, 150, 150, 123, - 123, 123, 123, 123, 211, 211, -1000, -1000, -1000, 326, - -1000, -1000, -1000, 326, 4338, 456, -1000, -1000, 5495, 70, - 464, 4734, -1000, 392, 392, 270, 259, 392, 4338, 204, - -1000, 4734, 326, -1000, 392, 326, 392, 392, -1000, -1000, - 8114, -1000, -1000, -1000, -1000, 452, 520, 7948, 464, -1000, - 6046, 7782, 698, 4734, -1000, -1000, 4734, 489, -1000, 4734, - -1000, -1000, -1000, -1000, 464, 464, 464, 375, -1000, 698, - -1000, 3329, 2284, -1000, 2284, 7782, -1000, 291, -1000, -1000, - 517, 22, -1000, -1000, -1000, 381, 80, 80, -1000, 276, - 163, -1000, -1000, -1000, 412, -1000, 446, 407, 8114, -1000, - -1000, -1000, 8114, -1000, -1000, -1000, -1000, -1000, 7782, -1000, - -1000, -1000, -1000, -1000, -1000, 28, 1322, 1273, -1000, 5110, - 5110, -1000, -1000, 392, 4338, -1000, -1000, 6929, -1000, -1000, - 2911, 4338, 217, -1000, -1000, 139, 277, 139, -139, 455, - 126, -1000, 4734, 355, -1000, -1000, -1000, -1000, -1000, -1000, - 713, 6763, -1000, 651, 380, 396, -1000, -1000, 4141, 326, - 390, 69, 375, 693, 217, 217, 7782, 217, 7782, 7782, - 7782, 5858, 7782, 693, -1000, 2075, -1000, 373, -1000, 468, - -1000, -93, 719, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 263, 207, -1000, 200, 2702, - -1000, -1000, 653, -1000, 5110, 1322, 1322, -1000, -1000, -1000, - -1000, 68, 326, 326, 468, 468, -1000, 468, 473, -1000, - 468, -41, 468, -59, 326, 326, 464, -136, -1000, 217, - 4734, 711, 398, 648, -1000, 464, -1000, -1000, 557, 7782, - 7782, -1000, -1000, 360, -1000, 301, 301, 301, 87, -1000, - -1000, -1000, 7782, -1000, 120, -1000, -120, -1000, 348, 334, - -1000, 464, 1322, 2493, -1000, -1000, -1000, 50, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 5110, 326, 257, 217, - 701, 695, 718, -1000, 464, -1000, 501, 63, -1000, 7782, - -1000, -1000, -1000, -1000, -1000, -1000, 162, 647, -1000, 633, - -1000, -1000, -1000, -16, -1000, -1000, -1000, 2, -1000, -1000, - -1000, 4734, 4734, 7948, 396, 326, 7782, -1000, -1000, 241, - -1000, -1000, 288, -1000, 7782, 326, 20, -147, 217, 394, - 384, -1000, -1000, -1000, -1000, -16, 608, -1000, 594, -143, - -151, -1000, -19, -1000, 589, -1000, -22, -145, 464, -149, - 4922, -153, 1080, 326, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 2715, 2715, -1000, -1000, 8127, -1000, + -1000, -1000, -1000, 727, 100, 360, -1000, 4747, 1364, 476, + 476, -1000, -1000, 51, -1000, -1000, 5123, 5123, 5123, 5123, + 5123, 5123, 5123, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 476, 73, -1000, + 4550, 476, 476, 476, 476, 476, 476, 4747, 476, 476, + 476, 476, 476, 476, 476, 476, 476, 476, 476, 476, + 476, 472, -1000, 293, 698, 307, 612, 6413, 578, -1000, + -1000, -29, 8127, -1000, 7961, 6776, 6776, 6776, 6776, 6776, + -1000, 601, 590, -1000, 600, 571, 586, 8127, -1000, 419, + 307, 6247, 57, 476, -1000, 7108, -1000, -1000, -29, 6776, + 8127, -1000, -1000, 7961, 390, -1000, -1000, -1000, -1000, 4747, + 3551, 2297, 87, 180, -94, -1000, -1000, 487, -1000, 487, + 487, 487, 487, -72, -72, -72, -72, -1000, -1000, -1000, + -1000, -1000, 541, -1000, 487, 487, 487, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 521, 521, 521, 491, 491, + 505, -1000, 8127, -1000, 680, 85, -1000, 8127, -1000, -1000, + 8127, 2715, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 618, 4747, 4747, + 217, 4747, 4747, 79, 5123, 227, 169, 5123, 5123, 5123, + 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, + 5123, 5123, 225, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 325, -1000, 517, 506, 506, 82, 82, 82, 82, + 82, 82, 5311, 3957, 3551, 370, 195, 4550, 4351, 4351, + 4747, 4747, 4351, 692, 130, 195, 7795, -1000, 307, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 4351, 4351, 4351, 4351, + 4747, -1000, -1000, -1000, 657, -1000, 692, 705, -1000, 635, + 634, 4351, -1000, 591, 476, -1000, 504, 617, 525, 552, + 638, -1000, -1000, -1000, -1000, 587, -1000, 572, -1000, -1000, + -1000, -1000, -1000, 307, -1000, 26, 24, 22, 7795, -1000, + 720, 440, -1000, -1000, -1000, 195, -1000, 72, -1000, 465, + 2088, -1000, -1000, -1000, -1000, -1000, -1000, 518, 671, 132, + 292, -1000, -1000, 664, -1000, 155, -102, -1000, -1000, 233, + -72, -72, -1000, -1000, 41, 640, 41, 41, 41, 246, + -1000, -1000, -1000, -1000, 228, -1000, -1000, -1000, 210, -1000, + 551, 7795, 2715, -1000, -1000, 135, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -30, -1000, + 2715, -1000, 610, 79, 138, -1000, -1000, 261, -1000, -1000, + 195, 195, 1144, -1000, -1000, -1000, -1000, 227, 5123, 5123, + 5123, 852, 1144, 1446, 245, 663, 82, 74, 74, 81, + 81, 81, 81, 81, 211, 211, -1000, -1000, -1000, 307, + -1000, -1000, -1000, 307, 4351, 463, -1000, -1000, 5508, 68, + 476, 4747, -1000, 311, 311, 375, 345, 311, 4351, 168, + -1000, 4747, 307, -1000, 311, 307, 311, 311, -1000, -1000, + 8127, -1000, -1000, -1000, -1000, 500, 550, 7961, 476, -1000, + 6059, 7795, 713, 4747, -1000, -1000, 4747, 503, -1000, 4747, + -1000, -1000, -1000, -1000, 476, 476, 476, 303, -1000, 713, + -1000, 3342, 2297, -1000, 2297, 7795, -1000, 279, -1000, -1000, + 548, 23, -1000, -1000, -1000, 387, 41, 41, -1000, 259, + 83, -1000, -1000, -1000, 351, -1000, 458, 349, 8127, -1000, + -1000, -1000, 8127, -1000, -1000, -1000, -1000, -1000, 7795, -1000, + -1000, -1000, -1000, -1000, -1000, 852, 1144, 1404, -1000, 5123, + 5123, -1000, -1000, 311, 4351, -1000, -1000, 6942, -1000, -1000, + 2924, 4351, 195, -1000, -1000, 139, 225, 139, -138, 435, + 125, -1000, 4747, 144, -1000, -1000, -1000, -1000, -1000, -1000, + 720, 6776, -1000, 673, 459, 409, -1000, -1000, 4154, 307, + 306, 61, 303, 698, 195, 195, 7795, 195, 7795, 7795, + 7795, 5871, 7795, 698, -1000, 2088, -1000, 301, -1000, 487, + -1000, -87, 726, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 237, 201, -1000, 192, 2715, + -1000, -1000, 675, -1000, 5123, 1144, 1144, -1000, -1000, -1000, + -1000, 58, 307, 307, 487, 487, -1000, 487, 491, -1000, + 487, -54, 487, -55, 307, 307, 476, -135, -1000, 195, + 4747, 718, 445, 670, -1000, 476, -1000, -1000, 468, 7795, + 7795, -1000, -1000, 278, -1000, 270, 270, 270, 57, -1000, + -1000, 505, 7795, -1000, 104, -1000, -119, -1000, 385, 371, + -1000, 476, 1144, 2506, -1000, -1000, -1000, 37, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 5123, 307, 235, 195, + 716, 709, 725, -1000, 476, -1000, 517, 55, -1000, 7795, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 161, 668, -1000, + 667, -1000, -1000, -1000, -32, -1000, -1000, -1000, 29, -1000, + -1000, -1000, 4747, 4747, 7961, 409, 307, 7795, -1000, -1000, + 196, -1000, -1000, 267, -1000, 7795, 307, 21, -149, 195, + 394, 390, -1000, -1000, -1000, -1000, -32, 624, -1000, 607, + -144, -152, -1000, -37, -1000, 605, -1000, -39, -147, 476, + -150, 4935, -153, 1119, 307, -1000, -1000, -1000, } var yyPgo = [...]int{ - 0, 910, 24, 602, 908, 907, 906, 905, 903, 902, - 900, 898, 895, 894, 892, 887, 886, 878, 876, 873, - 87, 872, 870, 869, 44, 868, 56, 867, 866, 29, - 54, 22, 26, 162, 865, 19, 74, 53, 864, 41, - 862, 861, 860, 854, 55, 851, 846, 1095, 844, 843, - 10, 23, 842, 841, 839, 838, 51, 46, 835, 832, - 829, 828, 827, 825, 36, 3, 5, 4, 11, 824, - 439, 6, 823, 34, 821, 819, 818, 816, 20, 814, - 43, 812, 17, 42, 811, 16, 50, 33, 14, 8, - 810, 48, 809, 569, 808, 115, 807, 806, 804, 800, - 792, 791, 30, 300, 909, 28, 27, 789, 788, 1162, - 32, 49, 18, 787, 40, 38, 21, 785, 782, 15, - 780, 778, 777, 776, 775, 774, 138, 773, 772, 771, - 9, 35, 770, 769, 57, 13, 768, 764, 763, 762, - 45, 761, 37, 760, 759, 758, 31, 12, 757, 7, - 756, 755, 2, 750, 747, 738, 0, 82, 737, 735, - 165, + 0, 923, 28, 618, 922, 919, 917, 913, 912, 910, + 909, 908, 907, 905, 904, 898, 897, 896, 880, 879, + 87, 876, 875, 873, 41, 871, 56, 870, 869, 26, + 44, 22, 30, 102, 868, 19, 88, 64, 867, 37, + 866, 865, 862, 861, 53, 858, 857, 1111, 856, 855, + 8, 21, 853, 847, 845, 844, 51, 49, 843, 842, + 840, 839, 836, 835, 35, 2, 5, 4, 13, 833, + 696, 12, 831, 45, 828, 826, 825, 822, 24, 821, + 46, 820, 17, 43, 818, 15, 58, 33, 18, 6, + 817, 48, 816, 558, 814, 100, 813, 812, 811, 810, + 808, 34, 300, 852, 27, 25, 806, 804, 11, 1174, + 32, 55, 16, 800, 40, 38, 20, 799, 794, 23, + 792, 791, 788, 787, 786, 785, 84, 782, 780, 778, + 10, 42, 777, 776, 54, 9, 775, 774, 773, 772, + 50, 771, 36, 766, 765, 750, 31, 14, 749, 7, + 747, 746, 3, 745, 744, 743, 0, 83, 742, 738, + 74, } var yyR1 = [...]int{ @@ -1553,7 +1554,7 @@ var yyR1 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 3, 4, 4, 5, 5, 6, 6, 23, 23, 7, 8, 8, 158, 158, 42, 42, 86, 86, 9, - 9, 9, 90, 90, 90, 108, 108, 117, 117, 10, + 9, 9, 90, 90, 90, 107, 107, 117, 117, 10, 10, 10, 10, 15, 143, 144, 144, 144, 140, 120, 120, 120, 123, 123, 121, 121, 121, 121, 121, 121, 121, 122, 122, 122, 122, 122, 124, 124, 124, 124, @@ -1571,7 +1572,7 @@ var yyR1 = [...]int{ 17, 17, 118, 118, 118, 18, 18, 19, 19, 19, 19, 19, 159, 20, 21, 21, 22, 22, 22, 26, 26, 26, 24, 24, 25, 25, 31, 31, 30, 30, - 32, 32, 32, 32, 107, 107, 107, 106, 106, 34, + 32, 32, 32, 32, 106, 106, 106, 105, 105, 34, 34, 35, 35, 36, 36, 37, 37, 37, 49, 49, 85, 85, 87, 87, 38, 38, 38, 38, 39, 39, 40, 40, 41, 41, 113, 113, 112, 112, 112, 111, @@ -1598,23 +1599,23 @@ var yyR1 = [...]int{ 89, 89, 91, 92, 92, 95, 95, 96, 96, 93, 93, 97, 97, 97, 97, 97, 97, 97, 97, 97, 97, 97, 98, 98, 98, 99, 99, 100, 100, 100, - 101, 101, 104, 104, 105, 105, 109, 109, 110, 110, - 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, - 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, + 108, 108, 103, 103, 104, 104, 109, 109, 110, 110, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, + 101, 101, 101, 101, 101, 101, 101, 101, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, - 102, 102, 102, 102, 102, 102, 102, 102, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, - 103, 103, 103, 103, 103, 103, 103, 103, 103, 156, + 102, 102, 102, 102, 102, 102, 102, 102, 102, 156, 157, 114, 115, 115, 115, } var yyR2 = [...]int{ @@ -1632,7 +1633,7 @@ var yyR2 = [...]int{ 5, 0, 3, 5, 0, 1, 0, 1, 0, 1, 2, 0, 2, 2, 2, 2, 2, 0, 3, 0, 1, 0, 3, 3, 0, 2, 0, 2, 1, 2, - 1, 0, 2, 4, 2, 3, 2, 2, 1, 1, + 1, 0, 2, 5, 2, 3, 2, 2, 1, 1, 1, 3, 2, 0, 1, 3, 1, 2, 3, 1, 1, 1, 6, 7, 7, 4, 5, 7, 1, 3, 8, 8, 5, 4, 6, 5, 3, 2, 3, 4, @@ -1695,7 +1696,7 @@ var yyChk = [...]int{ 115, 124, 48, 24, 125, 126, 129, 130, -156, 7, 201, 52, -155, 214, -78, 14, -22, 5, -20, -159, -20, -20, -20, -20, -143, 52, -100, 118, 69, 116, - 122, -104, 55, -103, 207, 149, 143, 170, 162, 160, + 122, -103, 55, -102, 207, 149, 143, 170, 162, 160, 163, 189, 64, 127, 158, 154, 152, 26, 175, 212, 153, 190, 147, 148, 174, 31, 209, 33, 135, 173, 169, 172, 146, 168, 37, 188, 165, 155, 17, 130, @@ -1703,17 +1704,17 @@ var yyChk = [...]int{ 145, 141, 166, 136, 156, 157, 171, 144, 167, 138, 180, 213, 164, 161, 142, 139, 140, 184, 185, 186, 187, 210, 159, 181, -93, 118, 120, 116, 116, 117, - 118, 116, -47, -109, 55, -103, 118, 116, 105, 163, + 118, 116, -47, -109, 55, -102, 118, 116, 105, 163, 111, 182, 117, 31, 137, -118, 116, 183, 140, 184, 185, 186, 187, 55, 191, 190, -109, -114, -114, -114, -114, -114, -2, -82, 16, 15, -5, -3, -156, 6, 19, 20, -26, 38, 39, -21, -93, -35, -36, -37, -38, -49, -70, -156, -47, 10, -42, -47, -89, -117, - -90, -91, 191, 190, 189, 163, 188, -71, -104, -109, - 55, -103, -144, -140, 55, 117, -47, 201, -96, 121, + -90, -91, 191, 190, 189, 163, 188, -71, -103, -109, + 55, -102, -144, -140, 55, 117, -47, 201, -96, 121, 116, -47, -47, -95, 121, 55, -95, -47, 108, -47, 55, 29, 193, 55, 137, 116, 138, 118, -115, -156, - -105, -104, -102, 70, 21, 23, 177, 73, 105, 15, + -104, -103, -101, 70, 21, 23, 177, 73, 105, 15, 74, 104, 202, 111, 46, 194, 195, 192, 193, 182, 28, 9, 24, 125, 20, 98, 113, 77, 78, 128, 22, 126, 68, 18, 49, 10, 12, 13, 121, 120, @@ -1726,20 +1727,20 @@ var yyChk = [...]int{ -157, 54, -83, 18, 30, -33, -52, 71, -57, 28, 22, -56, -53, -71, -69, -70, 105, 106, 94, 95, 102, 72, 107, -61, -59, -60, -62, 57, 56, 65, - 58, 59, 60, 61, 66, 67, 68, -104, -109, -67, + 58, 59, 60, 61, 66, 67, 68, -103, -109, -67, -156, 42, 43, 202, 203, 206, 204, 74, 32, 192, 200, 199, 198, 196, 197, 194, 195, 121, 193, 100, 201, -79, -80, -33, -78, -2, -20, 34, -24, 20, 63, -48, 25, -47, 29, 53, -43, -45, -44, -46, 40, 44, 46, 41, 42, 43, 47, -113, 21, -35, -2, -156, -112, 133, -111, 21, -109, 57, -47, -158, - 53, 10, 51, 53, -89, -108, -105, 57, 29, 79, + 53, 10, 51, 53, -89, -107, -104, 57, 29, 79, 108, 54, 53, -120, -123, -125, -124, -121, -122, 160, 161, 105, 164, 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, 127, 156, 157, 158, 159, 143, 144, 145, 146, 147, 148, 149, 151, 152, 153, 154, 155, 55, -115, 118, -47, 71, -47, -115, 119, -47, 22, - 50, -47, -110, -109, -102, -115, -115, -115, -115, -115, + 50, -47, -110, -109, -101, -115, -115, -115, -115, -115, -115, -115, -115, -115, -115, -47, 8, 89, 70, 69, 86, 53, 17, -33, -54, 89, 71, 87, 88, 73, 91, 90, 101, 94, 95, 96, 97, 98, 99, 100, @@ -1748,56 +1749,56 @@ var yyChk = [...]int{ -57, -57, -57, -156, 108, -65, -33, -156, -156, -156, -156, -156, -156, -156, -74, -33, -156, -160, -156, -160, -160, -160, -160, -160, -160, -160, -156, -156, -156, -156, - 53, -81, 23, 24, -82, -157, -26, -58, -104, 58, + 53, -81, 23, 24, -82, -157, -26, -58, -103, 58, 61, -25, 41, -86, 133, -47, -89, -36, -37, -37, -36, -37, 40, 40, 40, 45, 40, 45, 40, -44, -109, -157, -157, -2, -50, 48, 120, 49, -156, -111, - -86, -35, -47, -91, -114, -33, -105, -110, -102, -145, - -146, -147, -105, 57, 58, -140, -141, -148, 123, 122, + -86, -35, -47, -91, -114, -33, -104, -110, -101, -145, + -146, -147, -104, 57, 58, -140, -141, -148, 123, 122, -142, 117, 27, -136, 66, 71, -132, 180, -126, 52, -126, -126, -126, -126, -130, 163, -130, -130, -130, 52, -126, -126, -126, -134, 52, -134, -134, -135, 52, -135, - -101, 51, -47, 22, -97, 114, -153, 112, 177, 163, + -108, 51, -47, 22, -97, 114, -153, 112, 177, 163, 64, 28, 113, 14, 202, 133, 213, 55, 134, -47, -47, -115, 36, -33, -33, -63, 66, 71, 67, 68, -33, -33, -57, -64, -67, -70, 62, 89, 87, 88, 73, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -116, 55, 57, 55, - -56, -56, -104, -31, 20, -30, -32, 96, -33, -109, - -105, 53, -157, -30, -30, -33, -33, -30, -24, -72, - -73, 75, -104, -157, -30, -31, -30, -30, -80, -83, + -56, -56, -103, -31, 20, -30, -32, 96, -33, -109, + -104, 53, -157, -30, -30, -33, -33, -30, -24, -72, + -73, 75, -103, -157, -30, -31, -30, -30, -80, -83, -92, 18, 10, 32, 32, -30, -55, 29, 32, -2, -156, -156, -51, 11, -40, -39, 50, 51, -41, 50, - -39, 40, 40, -157, 117, 117, 117, -87, -104, -51, + -39, 40, 40, -157, 117, 117, 117, -87, -103, -51, -51, 108, 53, -147, 79, 52, 27, -142, 55, 55, -127, 28, 66, -133, 181, 58, -130, -130, -131, 104, - 29, -131, -131, -131, -139, 57, 58, 58, 50, -104, + 29, -131, -131, -131, -139, 57, 58, 58, 50, -103, -115, -114, -98, -99, 119, 21, 117, 27, 133, -115, 37, 66, 67, 68, -64, -57, -57, -57, -29, 128, - 70, -157, -157, -30, 53, -107, -106, 21, -104, 57, + 70, -157, -157, -30, 53, -106, -105, 21, -103, 57, 108, -156, -33, -157, -157, 53, 51, 21, -157, -30, -75, -73, 77, -33, -157, -157, -157, -157, -157, -47, -34, 10, -88, 50, -89, -66, -68, -67, -156, -2, - -84, -104, -87, -78, -33, -33, 52, -33, -156, -156, - -156, -157, 53, -78, -105, -146, -147, -150, -149, -104, + -84, -103, -87, -78, -33, -33, 52, -33, -156, -156, + -156, -157, 53, -78, -104, -146, -147, -150, -149, -103, 55, -129, 50, 57, 58, 59, 66, 192, 54, -131, -131, 55, 55, 105, 54, 53, 53, 54, 53, -47, - -47, -114, -104, -29, 70, -57, -57, -157, -32, -106, + -47, -114, -103, -29, 70, -57, -57, -157, -32, -105, 96, -110, -31, -119, 105, 160, 127, 158, 154, 174, 165, 179, 156, 180, -116, -119, 207, -78, 78, -33, 76, -51, -35, 26, -88, 53, -157, -157, -157, 53, - 108, -157, -82, -85, -104, -85, -85, -85, -112, -104, + 108, -157, -82, -85, -103, -85, -85, -85, -112, -103, -82, 54, 53, -126, -137, 177, 8, 57, 58, 58, -115, 25, -57, 108, -157, -157, -126, -126, -126, -135, -126, 148, -126, 148, -157, -157, -156, -28, 205, -33, - -76, 12, 27, -68, 32, -2, -156, -104, -104, 53, - 54, -157, -157, -157, -50, -149, -138, 123, 27, 122, - 192, 54, 54, -156, 96, -130, 55, -57, -157, 57, - -77, 13, 15, 8, -66, -2, 108, -104, -128, 64, - 27, 27, -151, -152, 133, -27, 89, 210, -33, -65, - -89, -157, -104, 57, -157, 53, -104, -157, 208, 47, - 211, -152, 32, 37, 209, 212, 135, 37, 136, 210, - -156, 211, -57, 132, 212, -157, -157, + -76, 12, 27, -68, 32, -2, -156, -103, -103, 53, + 54, -157, -157, -157, -50, -108, -149, -138, 123, 27, + 122, 192, 54, 54, -156, 96, -130, 55, -57, -157, + 57, -77, 13, 15, 8, -66, -2, 108, -103, -128, + 64, 27, 27, -151, -152, 133, -27, 89, 210, -33, + -65, -89, -157, -103, 57, -157, 53, -103, -157, 208, + 47, 211, -152, 32, 37, 209, 212, 135, 37, 136, + 210, -156, 211, -57, 132, 212, -157, -157, } var yyDef = [...]int{ @@ -1899,17 +1900,17 @@ var yyDef = [...]int{ 97, 395, 97, 398, 0, 0, 0, 402, 364, 408, 0, 423, 230, 0, 27, 0, 457, -2, 0, 0, 0, 38, 30, 0, 240, 0, 0, 0, 275, 243, - 31, 133, 0, 142, 126, 120, 0, 96, 0, 0, + 31, 490, 0, 142, 126, 120, 0, 96, 0, 0, 50, 0, 300, 0, 356, 359, 387, 121, 391, 392, 394, 396, 397, 399, 361, 360, 0, 0, 0, 406, 425, 0, 0, 455, 0, -2, 0, 450, 449, 0, - 249, 276, 277, 278, 239, 141, 131, 0, 128, 130, - 118, 100, 103, 0, 223, 388, 389, 380, 363, 403, - 21, 0, 0, 0, 445, 22, 0, 241, 58, 0, - 127, 129, 0, 158, 0, 0, 0, 0, 426, 424, - 453, -2, 451, 132, 157, 0, 0, 362, 0, 0, - 0, 159, 0, 381, 0, 384, 0, 382, 0, 0, - 0, 0, 0, 0, 383, 160, 161, + 249, 276, 277, 278, 239, 133, 141, 131, 0, 128, + 130, 118, 100, 103, 0, 223, 388, 389, 380, 363, + 403, 21, 0, 0, 0, 445, 22, 0, 241, 58, + 0, 127, 129, 0, 158, 0, 0, 0, 0, 426, + 424, 453, -2, 451, 132, 157, 0, 0, 362, 0, + 0, 0, 159, 0, 381, 0, 384, 0, 382, 0, + 0, 0, 0, 0, 0, 383, 160, 161, } var yyTok1 = [...]int{ @@ -3023,10 +3024,10 @@ yydefault: yyVAL.optVal = NewStrVal(yyDollar[2].bytes) } case 133: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] //line sql.y:864 { - yyVAL.indexDefinition = &IndexDefinition{Info: yyDollar[1].indexInfo, Columns: yyDollar[3].indexColumns} + yyVAL.indexDefinition = &IndexDefinition{Info: yyDollar[1].indexInfo, Columns: yyDollar[3].indexColumns, Using: yyDollar[5].colIdent} } case 134: yyDollar = yyS[yypt-2 : yypt+1] @@ -5141,13 +5142,13 @@ yydefault: yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:2459 { - yyVAL.empty = struct{}{} + yyVAL.colIdent = ColIdent{} } case 491: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2461 { - yyVAL.empty = struct{}{} + yyVAL.colIdent = yyDollar[2].colIdent } case 492: yyDollar = yyS[yypt-1 : yypt+1] diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index 72fedb7ee3e..bf224694911 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -238,9 +238,9 @@ func forceEOF(yylex interface{}) { %type for_from %type ignore_opt default_opt %type exists_opt -%type not_exists_opt non_rename_operation to_opt index_opt constraint_opt using_opt +%type not_exists_opt non_rename_operation to_opt index_opt constraint_opt %type reserved_keyword non_reserved_keyword -%type sql_id reserved_sql_id col_alias as_ci_opt charset_value +%type sql_id reserved_sql_id col_alias as_ci_opt charset_value using_opt %type table_id reserved_table_id table_alias as_opt_id %type as_opt %type force_eof ddl_force_eof @@ -860,9 +860,9 @@ column_comment_opt: } index_definition: - index_info '(' index_column_list ')' + index_info '(' index_column_list ')' using_opt { - $$ = &IndexDefinition{Info: $1, Columns: $3} + $$ = &IndexDefinition{Info: $1, Columns: $3, Using: $5} } index_info: @@ -2456,9 +2456,9 @@ constraint_opt: { $$ = struct{}{} } using_opt: - { $$ = struct{}{} } + { $$ = ColIdent{} } | USING sql_id - { $$ = struct{}{} } + { $$ = $2 } sql_id: ID From 30c6a69c043544353c4de7b2c041a7cfb9ebcf16 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 31 Jan 2018 13:22:19 -0800 Subject: [PATCH 14/72] Add support for trailing comments in begin/commit/rollback statements --- go/vt/sqlparser/analyzer.go | 3 ++- go/vt/sqlparser/analyzer_test.go | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index 97bd8aeaa80..062f64394a1 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -52,6 +52,7 @@ const ( // textual comparison to identify the statement type. func Preview(sql string) int { trimmed := StripLeadingComments(sql) + trimmedNoComments, _ := SplitTrailingComments(trimmed) firstWord := trimmed if end := strings.IndexFunc(trimmed, unicode.IsSpace); end != -1 { @@ -72,7 +73,7 @@ func Preview(sql string) int { case "delete": return StmtDelete } - switch strings.ToLower(trimmed) { + switch strings.ToLower(trimmedNoComments) { case "begin", "start transaction": return StmtBegin case "commit": diff --git a/go/vt/sqlparser/analyzer_test.go b/go/vt/sqlparser/analyzer_test.go index c8f6ec0ed3d..c50955d2d47 100644 --- a/go/vt/sqlparser/analyzer_test.go +++ b/go/vt/sqlparser/analyzer_test.go @@ -45,9 +45,13 @@ func TestPreview(t *testing.T) { {"\n\t begin ", StmtBegin}, {"... begin ", StmtUnknown}, {"begin ...", StmtUnknown}, + {"begin /* ... */", StmtBegin}, + {"begin /* ... *//*test*/", StmtBegin}, {"start transaction", StmtBegin}, {"commit", StmtCommit}, + {"commit /*...*/", StmtCommit}, {"rollback", StmtRollback}, + {"rollback /*...*/", StmtRollback}, {"create", StmtDDL}, {"alter", StmtDDL}, {"rename", StmtDDL}, From 12124b0917425a65bffe79dce29949f526d6413e Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 31 Jan 2018 20:29:59 -0800 Subject: [PATCH 15/72] nit - only get trimmed comments when needed --- go/vt/sqlparser/analyzer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index 062f64394a1..f0a92d4dd23 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -52,7 +52,6 @@ const ( // textual comparison to identify the statement type. func Preview(sql string) int { trimmed := StripLeadingComments(sql) - trimmedNoComments, _ := SplitTrailingComments(trimmed) firstWord := trimmed if end := strings.IndexFunc(trimmed, unicode.IsSpace); end != -1 { @@ -73,6 +72,7 @@ func Preview(sql string) int { case "delete": return StmtDelete } + trimmedNoComments, _ := SplitTrailingComments(trimmed) switch strings.ToLower(trimmedNoComments) { case "begin", "start transaction": return StmtBegin From 028ab884bda543c21f8ee6f0a1397c9f6f40a113 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 31 Jan 2018 20:49:28 -0800 Subject: [PATCH 16/72] Added comment per code review --- go/vt/sqlparser/analyzer.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index f0a92d4dd23..52a369e5a7a 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -72,6 +72,11 @@ func Preview(sql string) int { case "delete": return StmtDelete } + // For the following statements, is not enough to just check + // loweredFirstWord. This is because they are not statements + // in the grammar and we are relying in Preview to parse them. + // For instance, we don't want: "BEGIN ..." to be parsed + // as StmtBegin. trimmedNoComments, _ := SplitTrailingComments(trimmed) switch strings.ToLower(trimmedNoComments) { case "begin", "start transaction": From 8ad6f9aa485a446dce1c899ce39d2b5cb19100dd Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 31 Jan 2018 20:50:23 -0800 Subject: [PATCH 17/72] nit - improve comment --- go/vt/sqlparser/analyzer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index 52a369e5a7a..5e2fe28aada 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -72,7 +72,7 @@ func Preview(sql string) int { case "delete": return StmtDelete } - // For the following statements, is not enough to just check + // For the following statements is not enough to just check // loweredFirstWord. This is because they are not statements // in the grammar and we are relying in Preview to parse them. // For instance, we don't want: "BEGIN ..." to be parsed From 5284aeeed51670b1ca263ab0ec34543a4b7e4bce Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 31 Jan 2018 21:06:54 -0800 Subject: [PATCH 18/72] Rewrote comment to make it clearer --- go/vt/sqlparser/analyzer.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index 5e2fe28aada..e76e6ead0e2 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -72,10 +72,10 @@ func Preview(sql string) int { case "delete": return StmtDelete } - // For the following statements is not enough to just check - // loweredFirstWord. This is because they are not statements - // in the grammar and we are relying in Preview to parse them. - // For instance, we don't want: "BEGIN ..." to be parsed + // For the following statements it is not sufficient to rely + // on loweredFirstWord. This is because they are not statements + // in the grammar and we are relying on Preview to parse them. + // For instance, we don't want: "BEGIN JUNK" to be parsed // as StmtBegin. trimmedNoComments, _ := SplitTrailingComments(trimmed) switch strings.ToLower(trimmedNoComments) { From 71d46627e1a7e2f3b4c5242710d52c0882a28ef5 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Fri, 2 Feb 2018 16:50:07 -0800 Subject: [PATCH 19/72] Fix for show vindexes, not picking the correct stat Co-authored-by: Michael Demmer --- go/vt/vtgate/executor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index a72ce18cab6..ca3067b6882 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -574,7 +574,7 @@ func (e *Executor) handleShow(ctx context.Context, safeSession *SafeSession, sql s.Cell, s.Target.Keyspace, s.Target.Shard, - ts.Tablet.Type.String(), + ts.Target.TabletType.String(), state, topoproto.TabletAliasString(ts.Tablet.Alias), ts.Tablet.Hostname, From 1faab9954384b395dba7268eb6bd9bec6efd9072 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 14 Feb 2018 20:39:09 -0800 Subject: [PATCH 20/72] Revert "Merge pull request #71 from tinyspeck/slack-sync-upstream/2018-02-07" This reverts commit 713273a5326f13df275e429464f6b707099d3fbd, reversing changes made to edc17cdd9eadd8c77299b0ccad26f488125bcc50. --- go/vt/sqlparser/analyzer.go | 8 +- go/vt/sqlparser/analyzer_test.go | 4 - go/vt/sqlparser/ast.go | 4 - go/vt/sqlparser/parse_test.go | 15 - go/vt/sqlparser/sql.go | 1385 +++++++++++++++--------------- go/vt/sqlparser/sql.y | 12 +- 6 files changed, 699 insertions(+), 729 deletions(-) diff --git a/go/vt/sqlparser/analyzer.go b/go/vt/sqlparser/analyzer.go index e76e6ead0e2..97bd8aeaa80 100644 --- a/go/vt/sqlparser/analyzer.go +++ b/go/vt/sqlparser/analyzer.go @@ -72,13 +72,7 @@ func Preview(sql string) int { case "delete": return StmtDelete } - // For the following statements it is not sufficient to rely - // on loweredFirstWord. This is because they are not statements - // in the grammar and we are relying on Preview to parse them. - // For instance, we don't want: "BEGIN JUNK" to be parsed - // as StmtBegin. - trimmedNoComments, _ := SplitTrailingComments(trimmed) - switch strings.ToLower(trimmedNoComments) { + switch strings.ToLower(trimmed) { case "begin", "start transaction": return StmtBegin case "commit": diff --git a/go/vt/sqlparser/analyzer_test.go b/go/vt/sqlparser/analyzer_test.go index c50955d2d47..c8f6ec0ed3d 100644 --- a/go/vt/sqlparser/analyzer_test.go +++ b/go/vt/sqlparser/analyzer_test.go @@ -45,13 +45,9 @@ func TestPreview(t *testing.T) { {"\n\t begin ", StmtBegin}, {"... begin ", StmtUnknown}, {"begin ...", StmtUnknown}, - {"begin /* ... */", StmtBegin}, - {"begin /* ... *//*test*/", StmtBegin}, {"start transaction", StmtBegin}, {"commit", StmtCommit}, - {"commit /*...*/", StmtCommit}, {"rollback", StmtRollback}, - {"rollback /*...*/", StmtRollback}, {"create", StmtDDL}, {"alter", StmtDDL}, {"rename", StmtDDL}, diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go index d9edde9b5c4..4ed6d6d689b 100644 --- a/go/vt/sqlparser/ast.go +++ b/go/vt/sqlparser/ast.go @@ -982,7 +982,6 @@ func (ct *ColumnType) WalkSubtree(visit Visit) error { type IndexDefinition struct { Info *IndexInfo Columns []*IndexColumn - Using ColIdent } // Format formats the node. @@ -999,9 +998,6 @@ func (idx *IndexDefinition) Format(buf *TrackedBuffer) { } } buf.Myprintf(")") - if !idx.Using.IsEmpty() { - buf.Myprintf(" USING %v", idx.Using) - } } // WalkSubtree walks the nodes of the subtree. diff --git a/go/vt/sqlparser/parse_test.go b/go/vt/sqlparser/parse_test.go index 492946980d8..bda1920747f 100644 --- a/go/vt/sqlparser/parse_test.go +++ b/go/vt/sqlparser/parse_test.go @@ -1496,21 +1496,6 @@ func TestCreateTable(t *testing.T) { " key by_full_name (full_name)\n" + ")", - // test that indexes support USING - "create table t (\n" + - " id int auto_increment,\n" + - " username varchar,\n" + - " email varchar,\n" + - " full_name varchar,\n" + - " status_nonkeyword varchar,\n" + - " primary key (id) USING BTREE,\n" + - " unique key by_username (username) USING HASH,\n" + - " unique by_username2 (username) USING OTHER,\n" + - " unique index by_username3 (username) USING XYZ,\n" + - " index by_status (status_nonkeyword) USING PDQ,\n" + - " key by_full_name (full_name) USING OTHER\n" + - ")", - // multi-column indexes "create table t (\n" + " id int auto_increment,\n" + diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index 9d712fb7e9b..a0d21ed47c1 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -553,302 +553,238 @@ var yyExca = [...]int{ -1, 1015, 5, 22, -2, 446, - -1, 1062, + -1, 1061, 5, 23, -2, 447, } const yyPrivate = 57344 -const yyLast = 8507 +const yyLast = 8494 var yyAct = [...]int{ - 380, 38, 565, 1054, 379, 885, 198, 908, 624, 677, - 664, 680, 353, 886, 641, 973, 432, 173, 882, 429, - 736, 782, 743, 943, 44, 856, 848, 240, 430, 3, - 746, 640, 637, 797, 242, 713, 650, 785, 238, 38, - 167, 408, 818, 342, 745, 760, 402, 178, 201, 348, - 213, 351, 193, 418, 673, 434, 182, 43, 603, 1085, - 1076, 1082, 1071, 1080, 189, 1075, 207, 172, 1070, 956, - 1008, 187, 217, 1031, 577, 168, 169, 170, 171, 814, - 913, 914, 915, 340, 658, 233, 657, 48, 188, 916, - 985, 665, 1037, 1003, 1001, 334, 335, 1079, 1077, 693, - 223, 1055, 748, 838, 604, 625, 627, 224, 50, 51, - 52, 53, 219, 691, 652, 135, 136, 136, 1057, 531, - 530, 540, 541, 533, 534, 535, 536, 537, 538, 539, - 532, 1029, 497, 542, 138, 139, 140, 492, 922, 796, - 697, 795, 214, 794, 215, 819, 235, 220, 237, 690, - 147, 137, 239, 239, 239, 239, 835, 239, 239, 652, - 554, 555, 837, 1047, 239, 520, 993, 234, 236, 970, - 535, 536, 537, 538, 539, 532, 860, 626, 542, 38, - 801, 564, 532, 450, 228, 542, 542, 808, 923, 517, - 449, 331, 332, 333, 431, 336, 337, 687, 692, 685, - 665, 404, 339, 958, 651, 761, 444, 405, 518, 649, - 648, 494, 207, 519, 518, 917, 153, 239, 695, 698, - 960, 812, 239, 232, 520, 1050, 1030, 1028, 446, 1069, - 520, 239, 239, 239, 239, 239, 239, 239, 239, 1058, - 163, 226, 720, 761, 944, 872, 654, 410, 689, 651, - 989, 655, 836, 1064, 834, 491, 718, 719, 717, 988, - 496, 502, 688, 504, 519, 518, 946, 406, 827, 505, - 506, 507, 508, 509, 510, 511, 512, 345, 403, 41, - 737, 520, 738, 706, 708, 709, 826, 694, 707, 716, - 148, 815, 1040, 948, 987, 952, 150, 947, 696, 945, - 211, 156, 152, 825, 950, 533, 534, 535, 536, 537, - 538, 539, 532, 949, 921, 542, 592, 593, 951, 953, - 1066, 341, 154, 1019, 341, 158, 63, 841, 842, 843, - 145, 1019, 1020, 145, 910, 239, 239, 540, 541, 533, - 534, 535, 536, 537, 538, 539, 532, 809, 149, 542, - 551, 553, 145, 145, 982, 981, 902, 341, 145, 969, - 341, 341, 519, 518, 854, 341, 867, 151, 157, 159, - 160, 161, 162, 513, 514, 165, 164, 522, 563, 520, - 739, 567, 568, 569, 570, 571, 572, 573, 490, 576, + 380, 38, 1053, 565, 379, 885, 353, 908, 198, 664, + 624, 886, 641, 677, 882, 943, 973, 173, 432, 429, + 44, 736, 743, 782, 430, 3, 746, 856, 240, 848, + 242, 640, 637, 797, 760, 818, 713, 650, 238, 38, + 167, 785, 342, 402, 408, 213, 348, 178, 201, 434, + 603, 351, 193, 189, 745, 418, 182, 673, 43, 1084, + 207, 1075, 1081, 172, 1070, 1079, 1074, 1069, 956, 1008, + 217, 187, 1030, 814, 188, 168, 169, 170, 171, 913, + 914, 915, 340, 657, 985, 233, 665, 48, 916, 1003, + 693, 1056, 531, 530, 540, 541, 533, 534, 535, 536, + 537, 538, 539, 532, 691, 1036, 542, 1001, 50, 51, + 52, 53, 334, 335, 1078, 223, 1076, 1054, 531, 530, + 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, + 838, 697, 542, 604, 224, 625, 627, 219, 658, 135, + 690, 136, 492, 136, 497, 220, 235, 1028, 237, 138, + 139, 140, 239, 239, 239, 239, 849, 239, 239, 796, + 795, 794, 748, 215, 239, 577, 835, 234, 236, 147, + 137, 1046, 837, 652, 554, 555, 993, 970, 860, 38, + 801, 564, 450, 228, 819, 542, 517, 652, 687, 692, + 685, 331, 332, 333, 431, 336, 337, 404, 520, 761, + 449, 808, 339, 405, 958, 720, 207, 626, 444, 695, + 698, 494, 1057, 665, 917, 214, 153, 239, 922, 718, + 719, 717, 239, 232, 532, 518, 1049, 542, 1068, 446, + 812, 239, 239, 239, 239, 239, 239, 239, 239, 689, + 163, 520, 1029, 1027, 944, 410, 535, 536, 537, 538, + 539, 532, 654, 688, 542, 491, 226, 655, 989, 504, + 496, 502, 836, 651, 834, 988, 946, 406, 923, 505, + 506, 507, 508, 509, 510, 511, 512, 651, 694, 761, + 867, 872, 649, 648, 41, 827, 519, 518, 826, 696, + 148, 522, 815, 948, 716, 952, 150, 947, 1063, 945, + 211, 156, 152, 520, 950, 533, 534, 535, 536, 537, + 538, 539, 532, 949, 1039, 542, 592, 593, 951, 953, + 987, 866, 154, 865, 825, 158, 63, 521, 519, 518, + 145, 921, 737, 145, 738, 239, 239, 345, 403, 519, + 518, 1065, 341, 519, 518, 520, 910, 809, 149, 739, + 551, 553, 145, 145, 1019, 341, 520, 490, 145, 230, + 520, 225, 519, 518, 841, 842, 843, 151, 157, 159, + 160, 161, 162, 513, 514, 165, 164, 214, 563, 520, + 341, 567, 568, 569, 570, 571, 572, 573, 1032, 576, 578, 578, 578, 578, 578, 578, 578, 578, 586, 587, - 588, 589, 928, 927, 925, 924, 556, 557, 558, 559, - 560, 561, 562, 521, 519, 518, 193, 193, 193, 193, - 193, 606, 594, 751, 341, 1033, 866, 207, 865, 519, - 518, 520, 431, 230, 628, 225, 145, 214, 145, 1032, - 193, 918, 145, 443, 519, 518, 520, 751, 145, 45, - 523, 783, 63, 63, 63, 63, 207, 63, 63, 631, - 623, 520, 965, 596, 63, 579, 580, 581, 582, 583, - 584, 585, 415, 341, 19, 666, 667, 668, 636, 642, - 619, 608, 609, 566, 611, 638, 634, 145, 854, 595, - 575, 629, 633, 415, 145, 145, 145, 630, 415, 679, - 1014, 63, 239, 645, 607, 452, 451, 610, 441, 883, - 881, 926, 443, 621, 622, 783, 854, 63, 802, 145, - 41, 145, 63, 19, 145, 590, 414, 145, 41, 145, - 714, 63, 63, 63, 63, 63, 63, 63, 63, 659, - 701, 675, 676, 678, 660, 661, 662, 663, 179, 442, - 415, 440, 635, 854, 38, 896, 681, 443, 134, 670, - 671, 672, 368, 367, 370, 371, 372, 373, 567, 41, - 805, 369, 374, 674, 712, 786, 787, 721, 722, 723, - 724, 725, 726, 727, 728, 729, 730, 731, 732, 733, - 734, 735, 750, 669, 41, 55, 623, 19, 912, 638, - 883, 828, 789, 500, 780, 781, 740, 741, 338, 186, - 765, 616, 792, 753, 754, 758, 617, 757, 17, 602, - 777, 703, 704, 778, 710, 711, 618, 791, 424, 425, - 613, 764, 779, 766, 767, 63, 63, 768, 769, 145, - 614, 612, 1078, 41, 1074, 615, 775, 840, 790, 752, - 183, 184, 799, 800, 702, 803, 1073, 420, 423, 424, - 425, 421, 763, 422, 426, 177, 774, 773, 642, 820, - 566, 816, 817, 755, 756, 343, 409, 448, 420, 423, - 424, 425, 421, 239, 422, 426, 807, 344, 786, 787, - 407, 231, 811, 403, 1052, 1051, 355, 1012, 806, 963, - 991, 239, 683, 499, 428, 180, 181, 793, 63, 821, - 822, 823, 409, 145, 174, 772, 145, 145, 145, 145, - 145, 830, 714, 771, 1043, 831, 175, 45, 145, 1042, - 1011, 783, 145, 1044, 986, 516, 145, 47, 49, 839, - 145, 145, 439, 42, 1, 686, 1053, 907, 192, 647, - 639, 861, 63, 844, 530, 540, 541, 533, 534, 535, - 536, 537, 538, 539, 532, 212, 54, 542, 845, 846, - 847, 646, 824, 1027, 984, 653, 813, 656, 911, 888, - 1049, 38, 810, 887, 884, 455, 456, 454, 458, 853, - 207, 457, 453, 145, 155, 898, 899, 900, 145, 199, - 427, 145, 63, 869, 445, 871, 855, 893, 56, 889, - 833, 832, 684, 218, 550, 892, 770, 200, 890, 906, - 591, 401, 1041, 851, 903, 1010, 870, 852, 574, 904, - 642, 759, 642, 354, 905, 705, 366, 863, 864, 363, - 365, 868, 364, 597, 776, 524, 874, 352, 875, 876, - 877, 878, 377, 346, 862, 191, 411, 419, 417, 919, - 920, 416, 196, 63, 873, 788, 784, 190, 880, 1007, - 1056, 601, 933, 20, 931, 46, 185, 63, 61, 16, - 15, 901, 193, 939, 942, 938, 894, 954, 750, 895, - 192, 955, 897, 941, 957, 638, 14, 13, 24, 935, - 936, 962, 961, 964, 12, 11, 208, 10, 9, 8, - 7, 972, 6, 5, 975, 976, 977, 4, 978, 176, - 803, 980, 18, 2, 0, 0, 0, 0, 0, 63, - 239, 0, 0, 642, 0, 0, 0, 937, 0, 0, - 0, 63, 531, 530, 540, 541, 533, 534, 535, 536, - 537, 538, 539, 532, 0, 0, 542, 1006, 999, 0, - 0, 0, 0, 0, 0, 0, 888, 0, 990, 1016, - 887, 0, 966, 967, 968, 959, 971, 0, 0, 1013, - 849, 0, 63, 63, 992, 0, 0, 1024, 0, 0, - 1026, 566, 1034, 1025, 983, 0, 0, 1015, 0, 0, - 0, 63, 0, 0, 241, 241, 241, 241, 1036, 241, - 241, 0, 0, 0, 0, 888, 241, 38, 0, 887, - 1045, 904, 0, 0, 0, 0, 994, 995, 0, 996, - 997, 0, 998, 0, 0, 1000, 0, 1002, 1004, 1005, - 0, 0, 0, 0, 0, 1046, 1060, 552, 0, 63, - 0, 1061, 208, 241, 0, 0, 1038, 207, 0, 1021, - 1022, 1023, 0, 1009, 0, 0, 0, 0, 0, 241, - 1072, 145, 0, 0, 241, 0, 0, 0, 0, 0, - 1081, 63, 63, 241, 241, 241, 241, 241, 241, 241, - 241, 1039, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 63, 63, 0, 63, 63, 0, 0, 0, - 0, 194, 192, 192, 192, 192, 192, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 192, 145, - 1062, 1083, 0, 145, 0, 0, 192, 1065, 0, 63, - 1068, 142, 0, 0, 0, 1059, 566, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 63, 0, - 0, 0, 0, 0, 197, 0, 0, 1086, 1087, 216, - 0, 0, 0, 341, 378, 0, 0, 0, 0, 0, - 0, 0, 145, 0, 0, 0, 0, 241, 241, 0, - 0, 0, 0, 0, 0, 0, 0, 63, 0, 63, - 63, 63, 145, 63, 143, 0, 63, 166, 0, 531, - 530, 540, 541, 533, 534, 535, 536, 537, 538, 539, - 532, 0, 715, 542, 0, 0, 143, 143, 209, 0, - 63, 0, 143, 0, 531, 530, 540, 541, 533, 534, - 535, 536, 537, 538, 539, 532, 0, 221, 542, 222, - 0, 0, 0, 227, 0, 0, 0, 0, 0, 229, - 598, 0, 0, 0, 0, 0, 0, 208, 0, 0, - 63, 63, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 63, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 63, 0, 208, 0, 413, 0, - 0, 0, 0, 241, 241, 0, 0, 438, 0, 0, - 143, 0, 143, 0, 0, 0, 143, 0, 0, 0, - 63, 0, 143, 0, 0, 0, 0, 0, 0, 0, - 493, 0, 495, 0, 0, 498, 0, 0, 501, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 63, 0, - 0, 0, 0, 0, 241, 0, 63, 0, 0, 0, - 0, 143, 0, 0, 19, 39, 21, 22, 143, 436, - 143, 0, 0, 0, 209, 0, 0, 0, 0, 0, - 0, 0, 33, 0, 0, 0, 0, 23, 0, 0, - 0, 0, 0, 143, 0, 143, 0, 0, 143, 0, - 0, 143, 0, 503, 0, 0, 32, 0, 0, 0, - 41, 0, 0, 0, 715, 742, 0, 241, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 762, - 0, 0, 0, 0, 0, 526, 0, 529, 0, 0, - 0, 0, 0, 543, 544, 545, 546, 547, 548, 549, - 515, 527, 528, 525, 531, 530, 540, 541, 533, 534, - 535, 536, 537, 538, 539, 532, 0, 0, 542, 25, - 26, 28, 27, 30, 934, 0, 0, 0, 0, 0, - 0, 798, 31, 34, 35, 0, 0, 36, 37, 29, - 0, 0, 0, 241, 531, 530, 540, 541, 533, 534, - 535, 536, 537, 538, 539, 532, 0, 0, 542, 0, - 0, 0, 0, 143, 0, 0, 850, 0, 0, 0, - 0, 0, 0, 0, 605, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 829, 241, 531, 530, 540, 541, - 533, 534, 535, 536, 537, 538, 539, 532, 0, 0, - 542, 0, 632, 241, 0, 0, 0, 0, 0, 40, + 588, 589, 1031, 556, 557, 558, 559, 560, 561, 562, + 706, 708, 709, 1019, 1020, 707, 193, 193, 193, 193, + 193, 207, 594, 606, 519, 518, 982, 981, 902, 341, + 883, 960, 431, 443, 628, 918, 145, 443, 145, 355, + 193, 520, 145, 969, 341, 854, 341, 751, 145, 965, + 207, 415, 63, 63, 63, 63, 623, 63, 63, 631, + 928, 927, 881, 596, 63, 925, 924, 751, 341, 45, + 608, 609, 783, 611, 666, 667, 668, 415, 341, 636, + 642, 638, 619, 452, 451, 629, 634, 145, 595, 630, + 607, 192, 633, 610, 145, 145, 145, 783, 645, 926, + 19, 63, 239, 679, 414, 854, 441, 19, 854, 854, + 523, 802, 621, 622, 415, 590, 41, 63, 179, 145, + 659, 145, 63, 777, 145, 678, 778, 145, 415, 145, + 714, 63, 63, 63, 63, 63, 63, 63, 63, 443, + 701, 896, 805, 566, 675, 676, 41, 442, 681, 440, + 575, 674, 669, 41, 38, 55, 579, 580, 581, 582, + 583, 584, 585, 19, 41, 786, 787, 912, 567, 134, + 883, 712, 828, 789, 721, 722, 723, 724, 725, 726, + 727, 728, 729, 730, 731, 732, 733, 734, 735, 1014, + 500, 338, 623, 750, 618, 638, 424, 425, 660, 661, + 662, 663, 17, 616, 780, 781, 740, 741, 617, 41, + 765, 614, 635, 670, 671, 672, 615, 602, 758, 792, + 186, 791, 613, 753, 754, 612, 1077, 757, 779, 183, + 184, 1073, 840, 192, 768, 63, 63, 769, 702, 145, + 1072, 764, 774, 766, 767, 409, 773, 820, 752, 177, + 343, 448, 790, 803, 799, 800, 775, 231, 811, 407, + 1051, 763, 344, 368, 367, 370, 371, 372, 373, 642, + 816, 817, 369, 374, 1050, 1012, 806, 963, 991, 683, + 499, 703, 704, 239, 710, 711, 428, 807, 540, 541, + 533, 534, 535, 536, 537, 538, 539, 532, 180, 181, + 542, 239, 821, 822, 823, 409, 793, 772, 63, 174, + 1042, 175, 45, 145, 1041, 771, 145, 145, 145, 145, + 145, 830, 714, 1011, 783, 831, 1043, 986, 145, 516, + 566, 47, 145, 755, 756, 49, 145, 439, 42, 839, + 145, 145, 420, 423, 424, 425, 421, 1, 422, 426, + 686, 861, 63, 403, 844, 1052, 907, 647, 639, 212, + 54, 646, 824, 1026, 984, 845, 846, 847, 653, 813, + 656, 911, 1048, 810, 455, 456, 454, 458, 457, 888, + 453, 38, 155, 887, 207, 199, 884, 427, 445, 855, + 552, 680, 56, 145, 871, 898, 899, 900, 145, 853, + 833, 145, 63, 893, 832, 889, 684, 218, 550, 770, + 200, 890, 591, 869, 401, 892, 1040, 906, 1010, 870, + 903, 574, 851, 759, 354, 705, 852, 366, 363, 365, + 904, 642, 364, 642, 905, 597, 863, 864, 776, 524, + 868, 352, 346, 191, 411, 874, 419, 875, 876, 877, + 878, 417, 919, 920, 416, 192, 192, 192, 192, 192, + 196, 788, 784, 63, 190, 880, 1007, 1055, 601, 20, + 46, 192, 185, 16, 931, 933, 15, 63, 14, 192, + 901, 938, 193, 955, 942, 939, 13, 24, 954, 750, + 957, 638, 12, 941, 11, 10, 935, 936, 9, 964, + 8, 962, 7, 6, 961, 5, 4, 176, 18, 377, + 2, 972, 0, 0, 862, 975, 976, 977, 803, 0, + 978, 980, 0, 0, 873, 0, 0, 0, 0, 63, + 239, 0, 0, 0, 642, 61, 937, 0, 0, 0, + 0, 63, 0, 0, 0, 0, 894, 0, 0, 895, + 0, 0, 897, 0, 0, 0, 0, 1006, 0, 0, + 0, 0, 999, 208, 0, 715, 888, 0, 990, 1016, + 887, 966, 967, 968, 0, 971, 0, 1013, 0, 0, + 0, 992, 63, 63, 0, 0, 0, 0, 0, 1024, + 1025, 0, 1033, 1015, 0, 0, 0, 0, 0, 0, + 0, 63, 420, 423, 424, 425, 421, 1035, 422, 426, + 0, 0, 786, 787, 0, 888, 0, 38, 0, 887, + 1044, 0, 904, 0, 0, 994, 995, 0, 0, 0, + 0, 0, 0, 0, 0, 959, 0, 1004, 1005, 0, + 0, 1045, 0, 0, 0, 0, 1059, 0, 983, 63, + 207, 566, 1060, 1037, 0, 0, 0, 0, 1021, 1022, + 1023, 241, 241, 241, 241, 0, 241, 241, 1071, 0, + 0, 145, 0, 241, 0, 0, 0, 0, 0, 1080, + 0, 63, 63, 996, 997, 0, 998, 0, 0, 1000, + 1038, 1002, 0, 0, 0, 194, 0, 0, 0, 0, + 0, 0, 63, 63, 0, 63, 63, 0, 0, 208, + 241, 530, 540, 541, 533, 534, 535, 536, 537, 538, + 539, 532, 0, 1009, 542, 142, 241, 1082, 1061, 145, + 0, 241, 0, 145, 341, 1064, 0, 0, 1067, 63, + 241, 241, 241, 241, 241, 241, 241, 241, 197, 0, + 0, 0, 0, 216, 0, 0, 0, 715, 63, 0, + 0, 0, 378, 0, 0, 1085, 1086, 0, 0, 0, + 531, 530, 540, 541, 533, 534, 535, 536, 537, 538, + 539, 532, 145, 0, 542, 0, 0, 0, 0, 0, + 0, 0, 143, 0, 0, 166, 0, 63, 0, 63, + 63, 63, 145, 63, 1058, 566, 63, 0, 0, 0, + 0, 0, 0, 0, 143, 143, 209, 0, 0, 0, + 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 63, 221, 0, 222, 0, 0, 0, 227, 0, 526, + 0, 529, 0, 229, 241, 241, 0, 543, 544, 545, + 546, 547, 548, 549, 0, 527, 528, 525, 531, 530, + 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, + 63, 63, 542, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 413, 63, 0, 0, 0, 0, 0, 0, + 0, 438, 0, 0, 63, 0, 0, 0, 143, 0, + 143, 0, 0, 0, 143, 0, 0, 0, 0, 0, + 143, 0, 0, 0, 493, 0, 495, 598, 0, 498, + 63, 192, 501, 0, 208, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 19, 39, 21, 22, + 0, 0, 0, 934, 0, 0, 0, 63, 0, 143, + 0, 0, 0, 208, 33, 63, 143, 436, 143, 23, + 241, 241, 209, 531, 530, 540, 541, 533, 534, 535, + 536, 537, 538, 539, 532, 850, 0, 542, 32, 0, + 0, 143, 41, 143, 0, 0, 143, 0, 0, 143, + 0, 503, 0, 0, 0, 531, 530, 540, 541, 533, + 534, 535, 536, 537, 538, 539, 532, 0, 0, 542, + 0, 241, 531, 530, 540, 541, 533, 534, 535, 536, + 537, 538, 539, 532, 0, 0, 542, 0, 0, 0, + 0, 0, 0, 0, 515, 0, 0, 0, 0, 0, + 0, 25, 26, 28, 27, 30, 0, 0, 0, 0, + 0, 0, 0, 0, 31, 34, 35, 0, 0, 36, + 37, 29, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 742, 0, 241, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 762, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 192, 0, - 0, 0, 0, 0, 0, 0, 0, 143, 0, 209, - 143, 143, 143, 143, 143, 0, 0, 0, 0, 0, - 0, 858, 620, 0, 682, 0, 143, 0, 0, 699, - 436, 0, 700, 0, 143, 143, 0, 0, 209, 0, - 0, 0, 0, 0, 0, 503, 0, 0, 0, 0, - 208, 0, 0, 891, 798, 0, 0, 0, 0, 0, + 0, 143, 0, 0, 0, 0, 0, 0, 605, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 241, 241, 0, 241, 909, 0, - 0, 0, 0, 0, 0, 0, 0, 143, 0, 0, - 0, 0, 143, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 932, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 40, 0, 0, 0, 0, 632, 0, 798, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 858, 0, 0, 241, 0, 0, 0, 0, 0, 0, + 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 749, 503, - 0, 0, 0, 749, 749, 461, 0, 749, 0, 974, - 0, 974, 974, 974, 0, 979, 0, 0, 241, 0, - 0, 749, 749, 749, 749, 0, 0, 473, 0, 0, - 0, 0, 0, 0, 0, 0, 749, 0, 0, 0, - 0, 0, 241, 478, 479, 480, 481, 482, 483, 484, - 0, 485, 486, 487, 488, 489, 474, 475, 476, 477, - 459, 460, 0, 0, 462, 0, 463, 464, 465, 466, - 467, 468, 469, 470, 471, 472, 0, 0, 0, 0, - 0, 0, 1017, 1018, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 909, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 241, 0, 0, 0, + 0, 0, 0, 0, 0, 143, 0, 209, 143, 143, + 143, 143, 143, 461, 0, 0, 0, 0, 682, 0, + 620, 829, 241, 699, 143, 0, 700, 0, 436, 0, + 0, 0, 143, 143, 0, 473, 209, 0, 0, 0, + 241, 0, 0, 503, 0, 0, 0, 0, 0, 0, + 0, 478, 479, 480, 481, 482, 483, 484, 0, 485, + 486, 487, 488, 489, 474, 475, 476, 477, 459, 460, + 0, 0, 462, 0, 463, 464, 465, 466, 467, 468, + 469, 470, 471, 472, 0, 143, 0, 0, 858, 0, + 143, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1048, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 879, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 208, 0, 0, - 1063, 0, 0, 0, 0, 0, 0, 0, 1067, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 749, + 891, 798, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 749, 0, 0, 0, 0, 0, 0, - 929, 0, 0, 0, 930, 143, 0, 0, 0, 0, - 0, 0, 209, 0, 0, 0, 0, 0, 0, 0, + 0, 241, 241, 0, 241, 909, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 749, 503, 0, 0, + 0, 749, 749, 0, 0, 749, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 932, 749, + 749, 749, 749, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 749, 0, 0, 858, 0, 0, + 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 974, 0, 974, 974, + 974, 0, 979, 0, 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 143, 0, 0, 0, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 749, - 0, 0, 0, 0, 0, 503, 749, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 879, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1017, + 1018, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 909, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 241, 0, 0, 0, 749, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 143, 0, 0, 0, + 0, 749, 0, 0, 929, 0, 0, 0, 930, 1047, + 0, 0, 0, 143, 0, 0, 0, 0, 0, 0, + 209, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 208, 0, 0, 1062, 0, 0, 0, + 0, 0, 0, 0, 1066, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 436, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 320, 310, 283, 322, 261, 275, 330, - 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, - 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, - 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, - 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, - 0, 0, 0, 62, 0, 643, 644, 0, 0, 0, - 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, - 0, 247, 250, 329, 317, 267, 268, 804, 0, 0, - 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, - 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, - 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, - 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, - 321, 101, 313, 263, 271, 73, 269, 106, 99, 209, - 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, - 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, - 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, - 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, - 91, 89, 84, 78, 0, 246, 0, 109, 120, 133, - 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, - 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, - 252, 0, 0, 311, 295, 64, 0, 86, 131, 103, - 79, 121, 320, 310, 283, 322, 261, 275, 330, 276, - 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, - 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, - 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, - 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, - 0, 0, 62, 0, 643, 644, 0, 0, 0, 0, - 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, - 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, - 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, - 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, - 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, - 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, - 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, - 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, - 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, - 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, - 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, - 89, 84, 78, 0, 246, 0, 109, 120, 133, 259, - 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, - 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, - 0, 0, 311, 295, 64, 0, 86, 131, 103, 79, - 121, 320, 310, 283, 322, 261, 275, 330, 276, 277, - 304, 249, 291, 98, 273, 0, 264, 244, 270, 245, - 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, - 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, - 282, 305, 254, 298, 323, 274, 302, 324, 0, 0, - 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, - 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, - 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, - 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, - 0, 0, 1035, 0, 265, 0, 297, 0, 0, 0, - 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, - 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, - 313, 263, 271, 73, 269, 106, 99, 0, 296, 100, - 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, - 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, - 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, - 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, - 84, 78, 0, 246, 0, 109, 120, 133, 259, 316, - 127, 128, 129, 130, 95, 71, 81, 107, 257, 258, - 255, 256, 292, 293, 325, 326, 327, 308, 252, 0, - 0, 311, 295, 64, 0, 86, 131, 103, 79, 121, + 0, 143, 0, 0, 0, 143, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 749, 0, 0, + 0, 0, 0, 503, 749, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 143, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 436, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 320, 310, 283, 322, 261, 275, 330, 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, - 305, 254, 298, 323, 274, 302, 324, 41, 0, 0, - 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, + 305, 254, 298, 323, 274, 302, 324, 0, 0, 0, + 62, 0, 643, 644, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, - 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, + 329, 317, 267, 268, 804, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, - 263, 271, 73, 269, 106, 99, 0, 296, 100, 105, + 263, 271, 73, 269, 106, 99, 209, 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, @@ -861,12 +797,12 @@ var yyAct = [...]int{ 291, 98, 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, - 254, 298, 323, 274, 302, 324, 0, 0, 0, 210, - 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, + 254, 298, 323, 274, 302, 324, 0, 0, 0, 62, + 0, 643, 644, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, 0, 0, 0, - 940, 0, 265, 0, 297, 0, 0, 0, 251, 248, + 0, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, 100, 105, 88, @@ -883,10 +819,10 @@ var yyAct = [...]int{ 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, 0, 0, 62, 0, - 447, 0, 0, 0, 0, 0, 0, 72, 0, 301, + 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, - 290, 306, 280, 0, 0, 0, 0, 0, 0, 0, + 290, 306, 280, 0, 0, 0, 0, 0, 0, 1034, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, @@ -903,7 +839,7 @@ var yyAct = [...]int{ 273, 0, 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, - 323, 274, 302, 324, 0, 0, 0, 62, 0, 0, + 323, 274, 302, 324, 41, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, @@ -928,7 +864,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, - 280, 0, 0, 0, 0, 0, 0, 0, 0, 265, + 280, 0, 0, 0, 0, 0, 0, 940, 0, 265, 0, 297, 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, 101, 313, 263, 271, 73, 269, @@ -945,7 +881,7 @@ var yyAct = [...]int{ 264, 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, 309, 282, 305, 254, 298, 323, 274, - 302, 324, 0, 0, 0, 144, 0, 0, 0, 0, + 302, 324, 0, 0, 0, 62, 0, 447, 0, 0, 0, 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, 0, 0, 0, 286, 290, 306, 280, @@ -961,35 +897,78 @@ var yyAct = [...]int{ 120, 133, 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, 0, 0, 311, 295, 64, 0, 86, - 131, 103, 79, 121, 98, 0, 0, 744, 0, 350, - 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, - 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, - 382, 0, 0, 0, 0, 0, 0, 0, 0, 41, - 0, 0, 210, 368, 367, 370, 371, 372, 373, 0, - 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, - 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 358, 359, 747, 0, 0, 0, 399, 0, 360, - 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, - 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, + 131, 103, 79, 121, 320, 310, 283, 322, 261, 275, + 330, 276, 277, 304, 249, 291, 98, 273, 0, 264, + 244, 270, 245, 262, 285, 77, 288, 260, 312, 294, + 85, 328, 87, 299, 0, 108, 94, 0, 0, 287, + 314, 289, 309, 282, 305, 254, 298, 323, 274, 302, + 324, 0, 0, 0, 62, 0, 0, 0, 0, 0, + 0, 0, 0, 72, 0, 301, 319, 272, 303, 243, + 300, 0, 247, 250, 329, 317, 267, 268, 0, 0, + 0, 0, 0, 0, 0, 286, 290, 306, 280, 0, + 0, 0, 0, 0, 0, 0, 0, 265, 0, 297, + 0, 0, 0, 251, 248, 0, 284, 0, 0, 0, + 253, 0, 266, 307, 0, 315, 281, 146, 318, 279, + 278, 321, 101, 313, 263, 271, 73, 269, 106, 99, + 0, 296, 100, 105, 88, 113, 102, 119, 125, 126, + 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, + 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, + 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, + 116, 91, 89, 84, 78, 0, 246, 0, 109, 120, + 133, 259, 316, 127, 128, 129, 130, 95, 71, 81, + 107, 257, 258, 255, 256, 292, 293, 325, 326, 327, + 308, 252, 0, 0, 311, 295, 64, 0, 86, 131, + 103, 79, 121, 320, 310, 283, 322, 261, 275, 330, + 276, 277, 304, 249, 291, 98, 273, 0, 264, 244, + 270, 245, 262, 285, 77, 288, 260, 312, 294, 85, + 328, 87, 299, 0, 108, 94, 0, 0, 287, 314, + 289, 309, 282, 305, 254, 298, 323, 274, 302, 324, + 0, 0, 0, 210, 0, 0, 0, 0, 0, 0, + 0, 0, 72, 0, 301, 319, 272, 303, 243, 300, + 0, 247, 250, 329, 317, 267, 268, 0, 0, 0, + 0, 0, 0, 0, 286, 290, 306, 280, 0, 0, + 0, 0, 0, 0, 0, 0, 265, 0, 297, 0, + 0, 0, 251, 248, 0, 284, 0, 0, 0, 253, + 0, 266, 307, 0, 315, 281, 146, 318, 279, 278, + 321, 101, 313, 263, 271, 73, 269, 106, 99, 0, + 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, + 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, + 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, + 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, + 91, 89, 84, 78, 0, 246, 0, 109, 120, 133, + 259, 316, 127, 128, 129, 130, 95, 71, 81, 107, + 257, 258, 255, 256, 292, 293, 325, 326, 327, 308, + 252, 0, 0, 311, 295, 64, 0, 86, 131, 103, + 79, 121, 320, 310, 283, 322, 261, 275, 330, 276, + 277, 304, 249, 291, 98, 273, 0, 264, 244, 270, + 245, 262, 285, 77, 288, 260, 312, 294, 85, 328, + 87, 299, 0, 108, 94, 0, 0, 287, 314, 289, + 309, 282, 305, 254, 298, 323, 274, 302, 324, 0, + 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, + 0, 72, 0, 301, 319, 272, 303, 243, 300, 0, + 247, 250, 329, 317, 267, 268, 0, 0, 0, 0, + 0, 0, 0, 286, 290, 306, 280, 0, 0, 0, + 0, 0, 0, 0, 0, 265, 0, 297, 0, 0, + 0, 251, 248, 0, 284, 0, 0, 0, 253, 0, + 266, 307, 0, 315, 281, 146, 318, 279, 278, 321, + 101, 313, 263, 271, 73, 269, 106, 99, 0, 296, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, - 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, - 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, - 398, 395, 396, 393, 394, 392, 391, 390, 400, 383, - 384, 386, 0, 385, 64, 0, 86, 131, 103, 79, - 121, 98, 0, 0, 0, 0, 350, 0, 0, 0, + 89, 84, 78, 0, 246, 0, 109, 120, 133, 259, + 316, 127, 128, 129, 130, 95, 71, 81, 107, 257, + 258, 255, 256, 292, 293, 325, 326, 327, 308, 252, + 0, 0, 311, 295, 64, 0, 86, 131, 103, 79, + 121, 98, 0, 0, 744, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, - 0, 0, 0, 0, 0, 0, 41, 0, 341, 210, + 0, 0, 0, 0, 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 358, 359, - 0, 0, 0, 0, 399, 0, 360, 0, 0, 356, + 747, 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, @@ -1004,11 +983,11 @@ var yyAct = [...]int{ 0, 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, 0, - 0, 0, 0, 41, 0, 0, 210, 368, 367, 370, + 0, 0, 0, 41, 0, 341, 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 358, 359, 747, 0, 0, + 0, 0, 0, 0, 0, 358, 359, 0, 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, 0, @@ -1019,28 +998,28 @@ var yyAct = [...]int{ 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, 392, - 391, 390, 400, 383, 384, 386, 19, 385, 64, 0, - 86, 131, 103, 79, 121, 0, 0, 98, 0, 0, - 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, - 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, - 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, - 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, - 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, - 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, + 391, 390, 400, 383, 384, 386, 0, 385, 64, 0, + 86, 131, 103, 79, 121, 98, 0, 0, 0, 0, + 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, + 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, + 381, 382, 0, 0, 0, 0, 0, 0, 0, 0, + 41, 0, 0, 210, 368, 367, 370, 371, 372, 373, + 0, 0, 72, 369, 374, 375, 376, 0, 0, 347, + 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 358, 359, 0, 0, 0, 0, - 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, - 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, - 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, - 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, - 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, - 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, - 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, - 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, - 390, 400, 383, 384, 386, 0, 385, 64, 0, 86, - 131, 103, 79, 121, 98, 0, 0, 0, 0, 350, + 0, 0, 358, 359, 747, 0, 0, 0, 399, 0, + 360, 0, 0, 356, 357, 362, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 146, 0, 0, 397, + 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, + 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, + 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, + 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, + 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, + 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, + 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, + 389, 398, 395, 396, 393, 394, 392, 391, 390, 400, + 383, 384, 386, 19, 385, 64, 0, 86, 131, 103, + 79, 121, 0, 0, 98, 0, 0, 0, 0, 350, 0, 0, 0, 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, 0, 0, 41, @@ -1059,55 +1038,111 @@ var yyAct = [...]int{ 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, 390, 400, 383, - 384, 386, 98, 385, 64, 0, 86, 131, 103, 79, - 121, 77, 0, 0, 0, 0, 85, 388, 87, 0, - 0, 108, 94, 0, 0, 0, 0, 381, 382, 0, - 0, 0, 0, 0, 0, 0, 0, 41, 0, 0, - 210, 368, 367, 370, 371, 372, 373, 0, 0, 72, - 369, 374, 375, 376, 0, 0, 0, 361, 0, 387, + 384, 386, 0, 385, 64, 0, 86, 131, 103, 79, + 121, 98, 0, 0, 0, 0, 350, 0, 0, 0, + 77, 0, 349, 0, 0, 85, 388, 87, 0, 0, + 108, 94, 0, 0, 0, 0, 381, 382, 0, 0, + 0, 0, 0, 0, 0, 0, 41, 0, 0, 210, + 368, 367, 370, 371, 372, 373, 0, 0, 72, 369, + 374, 375, 376, 0, 0, 347, 361, 0, 387, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 358, - 359, 0, 0, 0, 0, 399, 0, 360, 0, 0, - 356, 357, 362, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 146, 0, 0, 397, 0, 101, 0, - 0, 0, 73, 0, 106, 99, 0, 1084, 100, 105, + 0, 0, 0, 0, 0, 0, 0, 0, 358, 359, + 0, 0, 0, 0, 399, 0, 360, 0, 0, 356, + 357, 362, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 146, 0, 0, 397, 0, 101, 0, 0, + 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, + 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, + 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, + 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, + 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, + 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, + 129, 130, 95, 71, 81, 107, 389, 398, 395, 396, + 393, 394, 392, 391, 390, 400, 383, 384, 386, 98, + 385, 64, 0, 86, 131, 103, 79, 121, 77, 0, + 0, 0, 0, 85, 388, 87, 0, 0, 108, 94, + 0, 0, 0, 0, 381, 382, 0, 0, 0, 0, + 0, 0, 0, 0, 41, 0, 0, 210, 368, 367, + 370, 371, 372, 373, 0, 0, 72, 369, 374, 375, + 376, 0, 0, 0, 361, 0, 387, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 358, 359, 0, 0, + 0, 0, 399, 0, 360, 0, 0, 356, 357, 362, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 146, 0, 0, 397, 0, 101, 0, 0, 0, 73, + 0, 106, 99, 0, 1083, 100, 105, 88, 113, 102, + 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, + 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, + 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, + 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, + 0, 109, 120, 133, 0, 0, 127, 128, 129, 130, + 95, 71, 81, 107, 389, 398, 395, 396, 393, 394, + 392, 391, 390, 400, 383, 384, 386, 98, 385, 64, + 0, 86, 131, 103, 79, 121, 77, 0, 0, 0, + 0, 85, 388, 87, 0, 0, 108, 94, 0, 0, + 0, 0, 381, 382, 0, 0, 0, 0, 0, 0, + 0, 0, 41, 0, 0, 210, 368, 367, 370, 371, + 372, 373, 0, 0, 72, 369, 374, 375, 376, 0, + 0, 0, 361, 0, 387, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 358, 359, 0, 0, 0, 0, + 399, 0, 360, 0, 0, 356, 357, 362, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, + 0, 397, 0, 101, 0, 0, 0, 73, 0, 106, + 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, + 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, + 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, + 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, + 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, + 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, + 81, 107, 389, 398, 395, 396, 393, 394, 392, 391, + 390, 400, 383, 384, 386, 98, 385, 64, 0, 86, + 131, 103, 79, 121, 77, 0, 0, 0, 0, 85, + 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, + 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 531, 530, + 540, 541, 533, 534, 535, 536, 537, 538, 539, 532, + 0, 0, 542, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, + 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, + 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, + 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, + 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, + 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, + 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, + 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 64, 0, 86, 131, 103, + 79, 121, 98, 0, 0, 0, 857, 0, 0, 0, + 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, + 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 62, 0, 859, 0, 0, 0, 0, 0, 0, 72, + 0, 0, 0, 0, 519, 518, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 520, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, + 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, - 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, - 128, 129, 130, 95, 71, 81, 107, 389, 398, 395, - 396, 393, 394, 392, 391, 390, 400, 383, 384, 386, - 98, 385, 64, 0, 86, 131, 103, 79, 121, 77, - 0, 0, 0, 0, 85, 388, 87, 0, 0, 108, - 94, 0, 0, 0, 0, 381, 382, 0, 0, 0, - 0, 0, 0, 0, 0, 41, 0, 0, 210, 368, - 367, 370, 371, 372, 373, 0, 0, 72, 369, 374, - 375, 376, 0, 0, 0, 361, 0, 387, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 358, 359, 0, - 0, 0, 0, 399, 0, 360, 0, 0, 356, 357, - 362, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 146, 0, 0, 397, 0, 101, 0, 0, 0, - 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, - 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, - 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, - 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, - 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, - 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, - 130, 95, 71, 81, 107, 389, 398, 395, 396, 393, - 394, 392, 391, 390, 400, 383, 384, 386, 98, 385, - 64, 0, 86, 131, 103, 79, 121, 77, 0, 0, + 78, 0, 0, 0, 109, 120, 133, 0, 98, 127, + 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, + 58, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 531, 530, 540, 541, 533, 534, 535, 536, 537, - 538, 539, 532, 0, 0, 542, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, - 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 59, 0, 57, + 0, 0, 0, 60, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, @@ -1116,14 +1151,14 @@ var yyAct = [...]int{ 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 64, 0, - 86, 131, 103, 79, 121, 98, 0, 0, 0, 857, + 86, 131, 103, 79, 121, 98, 0, 0, 0, 435, 0, 0, 0, 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 62, 0, 859, 0, 0, 0, 0, - 0, 0, 72, 0, 0, 0, 0, 519, 518, 0, + 0, 0, 0, 144, 0, 437, 0, 0, 0, 0, + 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 520, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, @@ -1133,68 +1168,68 @@ var yyAct = [...]int{ 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, + 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, + 0, 0, 19, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 98, 0, 64, 0, 86, 131, 103, + 79, 121, 77, 0, 0, 0, 0, 85, 0, 87, + 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 41, 0, + 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, + 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, + 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, + 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, + 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, + 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, + 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, + 84, 78, 0, 0, 0, 109, 120, 133, 0, 0, + 127, 128, 129, 130, 95, 71, 81, 107, 0, 0, + 19, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 98, 0, 64, 0, 86, 131, 103, 79, 121, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, - 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, - 79, 121, 0, 0, 0, 0, 0, 0, 0, 62, + 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 41, 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, - 0, 0, 0, 58, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 59, 0, 57, 0, 0, 0, 60, 101, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, - 0, 0, 0, 109, 120, 133, 0, 0, 127, 128, - 129, 130, 95, 71, 81, 107, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 64, 0, 86, 131, 103, 79, 121, 98, 0, - 0, 0, 435, 0, 0, 0, 0, 77, 0, 0, - 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 144, 0, 437, 0, - 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, - 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, - 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, - 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, - 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, - 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, - 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, - 109, 120, 133, 0, 0, 127, 128, 129, 130, 95, - 71, 81, 107, 0, 0, 19, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 98, 0, 64, 0, - 86, 131, 103, 79, 121, 77, 0, 0, 0, 0, - 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 41, 0, 0, 62, 0, 0, 0, 0, 0, - 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 109, 120, 133, 0, 98, 127, 128, + 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, + 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, + 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, + 0, 0, 0, 0, 0, 62, 0, 0, 599, 0, + 0, 600, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, - 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, - 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, - 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, - 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, - 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, - 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, - 133, 0, 0, 127, 128, 129, 130, 95, 71, 81, - 107, 0, 0, 19, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 98, 0, 64, 0, 86, 131, - 103, 79, 121, 77, 0, 0, 0, 0, 85, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, + 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, + 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, + 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, + 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, + 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, + 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, + 120, 133, 0, 0, 127, 128, 129, 130, 95, 71, + 81, 107, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 64, 0, 86, + 131, 103, 79, 121, 98, 0, 0, 0, 435, 0, + 0, 0, 0, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 41, - 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 144, 0, 437, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1202,7 +1237,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, - 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, + 433, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, @@ -1210,8 +1245,8 @@ var yyAct = [...]int{ 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, - 121, 0, 0, 0, 0, 0, 0, 0, 62, 0, - 0, 599, 0, 0, 600, 0, 0, 72, 0, 0, + 121, 0, 0, 0, 0, 41, 0, 0, 144, 0, + 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1223,14 +1258,63 @@ var yyAct = [...]int{ 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, - 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, - 130, 95, 71, 81, 107, 0, 0, 0, 0, 0, + 0, 0, 109, 120, 133, 0, 98, 127, 128, 129, + 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, + 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, + 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, + 0, 0, 0, 0, 62, 0, 859, 0, 0, 0, + 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 64, 0, 86, 131, 103, 79, 121, 98, 0, 0, - 0, 435, 0, 0, 0, 0, 77, 0, 0, 0, - 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 144, 0, 437, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, + 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, + 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, + 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, + 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, + 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, + 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, + 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, + 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, + 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, + 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, + 144, 0, 437, 0, 0, 0, 0, 0, 0, 72, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, + 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, + 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, + 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, + 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, + 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, + 78, 0, 0, 0, 109, 120, 133, 0, 0, 127, + 128, 129, 130, 95, 71, 81, 107, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 98, 64, 0, 86, 131, 103, 79, 121, 412, + 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, + 108, 94, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 144, + 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, + 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, + 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, + 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, + 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, + 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, + 195, 0, 0, 109, 120, 133, 0, 98, 127, 128, + 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, + 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, + 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, + 0, 0, 0, 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1238,7 +1322,7 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, - 99, 0, 0, 433, 105, 88, 113, 102, 119, 125, + 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, @@ -1246,14 +1330,14 @@ var yyAct = [...]int{ 120, 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, - 131, 103, 79, 121, 0, 0, 0, 0, 41, 0, + 131, 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 146, 0, 0, 0, 0, 101, + 0, 0, 141, 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, 80, @@ -1263,7 +1347,7 @@ var yyAct = [...]int{ 127, 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, 121, - 0, 0, 0, 0, 0, 0, 0, 62, 0, 859, + 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1280,7 +1364,7 @@ var yyAct = [...]int{ 95, 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, 0, - 0, 0, 0, 144, 0, 437, 0, 0, 0, 0, + 0, 0, 0, 210, 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1293,137 +1377,52 @@ var yyAct = [...]int{ 76, 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, 133, - 0, 0, 127, 128, 129, 130, 95, 71, 81, 107, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 98, 64, 0, 86, 131, 103, - 79, 121, 412, 77, 0, 0, 0, 0, 85, 0, - 87, 0, 0, 108, 94, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, - 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, - 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, - 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, - 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, - 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, - 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, - 89, 84, 78, 195, 0, 0, 109, 120, 133, 0, - 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, - 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, - 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, - 121, 0, 0, 0, 0, 0, 0, 0, 144, 0, - 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, - 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, - 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, - 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, - 115, 74, 132, 69, 123, 68, 70, 122, 96, 112, - 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, - 0, 0, 109, 120, 133, 0, 98, 127, 128, 129, - 130, 95, 71, 81, 107, 77, 0, 0, 0, 0, - 85, 0, 87, 0, 0, 108, 94, 0, 0, 0, - 64, 0, 86, 131, 103, 79, 121, 0, 0, 0, - 0, 0, 0, 0, 144, 0, 0, 0, 0, 0, - 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 141, 0, 146, 0, 0, - 0, 0, 101, 0, 0, 0, 73, 0, 106, 99, - 0, 0, 100, 105, 88, 113, 102, 119, 125, 126, - 111, 124, 66, 117, 110, 92, 82, 83, 65, 0, - 104, 76, 80, 75, 97, 114, 115, 74, 132, 69, - 123, 68, 70, 122, 96, 112, 118, 93, 90, 67, - 116, 91, 89, 84, 78, 0, 0, 0, 109, 120, - 133, 0, 98, 127, 128, 129, 130, 95, 71, 81, - 107, 77, 0, 0, 0, 0, 85, 0, 87, 0, - 0, 108, 94, 0, 0, 0, 64, 0, 86, 131, - 103, 79, 121, 0, 0, 0, 0, 0, 0, 0, - 62, 0, 0, 0, 0, 0, 0, 0, 0, 72, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 146, 0, 0, 0, 0, 101, 0, - 0, 0, 73, 0, 106, 99, 0, 0, 100, 105, - 88, 113, 102, 119, 125, 126, 111, 124, 66, 117, - 110, 92, 82, 83, 65, 0, 104, 76, 80, 75, - 97, 114, 115, 74, 132, 69, 123, 68, 70, 122, - 96, 112, 118, 93, 90, 67, 116, 91, 89, 84, - 78, 0, 0, 0, 109, 120, 133, 0, 98, 127, - 128, 129, 130, 95, 71, 81, 107, 77, 0, 0, - 0, 0, 85, 0, 87, 0, 0, 108, 94, 0, - 0, 0, 64, 0, 86, 131, 103, 79, 121, 0, - 0, 0, 0, 0, 0, 0, 210, 0, 0, 0, - 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 146, - 0, 0, 0, 0, 101, 0, 0, 0, 73, 0, - 106, 99, 0, 0, 100, 105, 88, 113, 102, 119, - 125, 126, 111, 124, 66, 117, 110, 92, 82, 83, - 65, 0, 104, 76, 80, 75, 97, 114, 115, 74, - 132, 69, 123, 68, 70, 122, 96, 112, 118, 93, - 90, 67, 116, 91, 89, 84, 78, 0, 0, 0, - 109, 120, 133, 0, 98, 127, 128, 129, 130, 95, - 71, 81, 107, 77, 0, 0, 0, 0, 85, 0, - 87, 0, 0, 108, 94, 0, 0, 0, 64, 0, - 86, 131, 103, 79, 121, 0, 0, 0, 0, 0, - 0, 0, 144, 0, 0, 0, 0, 0, 0, 0, - 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 98, 127, 128, 129, 130, 95, 71, 81, 107, + 77, 0, 0, 0, 0, 85, 0, 87, 0, 0, + 108, 94, 0, 0, 0, 64, 0, 86, 131, 103, + 79, 121, 0, 0, 0, 0, 0, 0, 0, 144, + 0, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 146, 0, 0, 0, 0, - 101, 0, 0, 0, 73, 0, 106, 99, 0, 0, - 100, 105, 88, 113, 102, 119, 125, 126, 111, 124, - 66, 117, 110, 92, 82, 83, 65, 0, 104, 76, - 80, 75, 97, 114, 115, 74, 132, 69, 123, 68, - 70, 122, 96, 112, 118, 93, 90, 67, 116, 91, - 89, 84, 78, 0, 0, 0, 109, 120, 133, 0, - 98, 127, 128, 129, 130, 95, 71, 81, 107, 77, - 0, 0, 0, 0, 85, 0, 87, 0, 0, 108, - 94, 0, 0, 0, 64, 0, 86, 131, 103, 79, - 121, 0, 0, 0, 0, 0, 0, 0, 210, 0, - 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 146, 0, 0, 0, 0, 101, 0, 0, + 0, 73, 0, 106, 99, 0, 0, 100, 105, 88, + 113, 102, 119, 125, 126, 111, 124, 66, 117, 110, + 92, 82, 83, 65, 0, 104, 76, 80, 75, 97, + 114, 115, 74, 132, 69, 123, 68, 70, 122, 96, + 112, 118, 93, 90, 67, 116, 91, 89, 84, 78, + 0, 0, 0, 109, 120, 133, 0, 98, 127, 128, + 129, 130, 95, 71, 81, 107, 77, 0, 0, 0, + 0, 85, 0, 87, 0, 0, 108, 94, 0, 0, + 0, 64, 0, 86, 131, 103, 79, 121, 0, 0, + 0, 0, 0, 0, 0, 210, 0, 0, 0, 0, + 0, 0, 0, 0, 72, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 146, 0, 0, 0, 0, 101, 0, 0, 0, - 73, 0, 106, 99, 0, 0, 100, 105, 88, 113, - 102, 119, 125, 126, 111, 124, 66, 117, 110, 92, - 82, 83, 65, 0, 104, 76, 80, 75, 97, 114, - 115, 74, 132, 69, 123, 68, 205, 122, 96, 112, - 118, 93, 90, 67, 116, 91, 89, 84, 78, 0, - 0, 0, 109, 120, 133, 0, 0, 127, 128, 129, - 130, 206, 204, 203, 202, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 64, 0, 86, 131, 103, 79, 121, + 0, 0, 0, 0, 0, 0, 0, 0, 146, 0, + 0, 0, 0, 101, 0, 0, 0, 73, 0, 106, + 99, 0, 0, 100, 105, 88, 113, 102, 119, 125, + 126, 111, 124, 66, 117, 110, 92, 82, 83, 65, + 0, 104, 76, 80, 75, 97, 114, 115, 74, 132, + 69, 123, 68, 205, 122, 96, 112, 118, 93, 90, + 67, 116, 91, 89, 84, 78, 0, 0, 0, 109, + 120, 133, 0, 0, 127, 128, 129, 130, 206, 204, + 203, 202, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 64, 0, 86, + 131, 103, 79, 121, } var yyPact = [...]int{ - 1358, -1000, -157, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 713, 732, -1000, - -1000, -1000, -1000, -1000, 543, 5674, -3, 35, 18, 7629, - 34, 185, 8127, -1000, -1000, -1000, -1000, -1000, 517, -1000, - -1000, -1000, -1000, -1000, 698, 711, 542, 686, 612, -1000, - -4, 6776, 7463, 8293, -1000, 382, 27, 8127, -129, -9, + 1330, -1000, -156, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 698, 726, -1000, + -1000, -1000, -1000, -1000, 503, 5661, 21, 54, 33, 7616, + 53, 185, 8114, -1000, -1000, -1000, -1000, -1000, 501, -1000, + -1000, -1000, -1000, -1000, 693, 696, 512, 679, 591, -1000, + 23, 6763, 7450, 8280, -1000, 322, 46, 8114, -131, 16, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1431,17 +1430,17 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 31, 8127, -1000, 8127, -14, 380, - -14, 8127, -1000, 76, -1000, -1000, -1000, 8127, 378, 662, - 30, 2715, 2715, 2715, 2715, -46, 2715, 2715, 558, -1000, - -1000, -1000, -1000, 2715, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 307, 657, 4747, 4747, 713, -1000, 517, -1000, - -1000, -1000, 656, -1000, -1000, 184, 7297, 497, 617, -1000, - -1000, -1000, 683, 6247, 6610, 8127, 498, -1000, 390, 7961, - 3133, -1000, -1000, -1000, -1000, 648, -1000, 111, -1000, 75, - -1000, -1000, 452, -1000, 1640, 333, 2715, 19, 8127, 140, - 8127, 2715, 13, 8127, 681, 553, 8127, -1000, 3760, -1000, - 2715, 2715, 2715, 2715, 2715, 2715, 2715, 2715, -1000, -1000, + -1000, -1000, -1000, -1000, 29, 8114, -1000, 8114, 13, 306, + 13, 8114, -1000, 75, -1000, -1000, -1000, 8114, 304, 628, + 30, 2702, 2702, 2702, 2702, -29, 2702, 2702, 541, -1000, + -1000, -1000, -1000, 2702, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 326, 632, 4734, 4734, 698, -1000, 501, -1000, + -1000, -1000, 625, -1000, -1000, 182, 7284, 475, 702, -1000, + -1000, -1000, 665, 6234, 6597, 8114, 496, -1000, 384, 7948, + 3120, -1000, -1000, -1000, -1000, 622, -1000, 121, -1000, 74, + -1000, -1000, 430, -1000, 1478, 302, 2702, 24, 8114, 140, + 8114, 2702, 25, 8114, 658, 540, 8114, -1000, 3747, -1000, + 2702, 2702, 2702, 2702, 2702, 2702, 2702, 2702, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1451,102 +1450,102 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 2715, 2715, -1000, -1000, 8127, -1000, - -1000, -1000, -1000, 727, 100, 360, -1000, 4747, 1364, 476, - 476, -1000, -1000, 51, -1000, -1000, 5123, 5123, 5123, 5123, - 5123, 5123, 5123, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 476, 73, -1000, - 4550, 476, 476, 476, 476, 476, 476, 4747, 476, 476, - 476, 476, 476, 476, 476, 476, 476, 476, 476, 476, - 476, 472, -1000, 293, 698, 307, 612, 6413, 578, -1000, - -1000, -29, 8127, -1000, 7961, 6776, 6776, 6776, 6776, 6776, - -1000, 601, 590, -1000, 600, 571, 586, 8127, -1000, 419, - 307, 6247, 57, 476, -1000, 7108, -1000, -1000, -29, 6776, - 8127, -1000, -1000, 7961, 390, -1000, -1000, -1000, -1000, 4747, - 3551, 2297, 87, 180, -94, -1000, -1000, 487, -1000, 487, - 487, 487, 487, -72, -72, -72, -72, -1000, -1000, -1000, - -1000, -1000, 541, -1000, 487, 487, 487, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 521, 521, 521, 491, 491, - 505, -1000, 8127, -1000, 680, 85, -1000, 8127, -1000, -1000, - 8127, 2715, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 618, 4747, 4747, - 217, 4747, 4747, 79, 5123, 227, 169, 5123, 5123, 5123, - 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, 5123, - 5123, 5123, 225, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 325, -1000, 517, 506, 506, 82, 82, 82, 82, - 82, 82, 5311, 3957, 3551, 370, 195, 4550, 4351, 4351, - 4747, 4747, 4351, 692, 130, 195, 7795, -1000, 307, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 4351, 4351, 4351, 4351, - 4747, -1000, -1000, -1000, 657, -1000, 692, 705, -1000, 635, - 634, 4351, -1000, 591, 476, -1000, 504, 617, 525, 552, - 638, -1000, -1000, -1000, -1000, 587, -1000, 572, -1000, -1000, - -1000, -1000, -1000, 307, -1000, 26, 24, 22, 7795, -1000, - 720, 440, -1000, -1000, -1000, 195, -1000, 72, -1000, 465, - 2088, -1000, -1000, -1000, -1000, -1000, -1000, 518, 671, 132, - 292, -1000, -1000, 664, -1000, 155, -102, -1000, -1000, 233, - -72, -72, -1000, -1000, 41, 640, 41, 41, 41, 246, - -1000, -1000, -1000, -1000, 228, -1000, -1000, -1000, 210, -1000, - 551, 7795, 2715, -1000, -1000, 135, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -30, -1000, - 2715, -1000, 610, 79, 138, -1000, -1000, 261, -1000, -1000, - 195, 195, 1144, -1000, -1000, -1000, -1000, 227, 5123, 5123, - 5123, 852, 1144, 1446, 245, 663, 82, 74, 74, 81, - 81, 81, 81, 81, 211, 211, -1000, -1000, -1000, 307, - -1000, -1000, -1000, 307, 4351, 463, -1000, -1000, 5508, 68, - 476, 4747, -1000, 311, 311, 375, 345, 311, 4351, 168, - -1000, 4747, 307, -1000, 311, 307, 311, 311, -1000, -1000, - 8127, -1000, -1000, -1000, -1000, 500, 550, 7961, 476, -1000, - 6059, 7795, 713, 4747, -1000, -1000, 4747, 503, -1000, 4747, - -1000, -1000, -1000, -1000, 476, 476, 476, 303, -1000, 713, - -1000, 3342, 2297, -1000, 2297, 7795, -1000, 279, -1000, -1000, - 548, 23, -1000, -1000, -1000, 387, 41, 41, -1000, 259, - 83, -1000, -1000, -1000, 351, -1000, 458, 349, 8127, -1000, - -1000, -1000, 8127, -1000, -1000, -1000, -1000, -1000, 7795, -1000, - -1000, -1000, -1000, -1000, -1000, 852, 1144, 1404, -1000, 5123, - 5123, -1000, -1000, 311, 4351, -1000, -1000, 6942, -1000, -1000, - 2924, 4351, 195, -1000, -1000, 139, 225, 139, -138, 435, - 125, -1000, 4747, 144, -1000, -1000, -1000, -1000, -1000, -1000, - 720, 6776, -1000, 673, 459, 409, -1000, -1000, 4154, 307, - 306, 61, 303, 698, 195, 195, 7795, 195, 7795, 7795, - 7795, 5871, 7795, 698, -1000, 2088, -1000, 301, -1000, 487, - -1000, -87, 726, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 237, 201, -1000, 192, 2715, - -1000, -1000, 675, -1000, 5123, 1144, 1144, -1000, -1000, -1000, - -1000, 58, 307, 307, 487, 487, -1000, 487, 491, -1000, - 487, -54, 487, -55, 307, 307, 476, -135, -1000, 195, - 4747, 718, 445, 670, -1000, 476, -1000, -1000, 468, 7795, - 7795, -1000, -1000, 278, -1000, 270, 270, 270, 57, -1000, - -1000, 505, 7795, -1000, 104, -1000, -119, -1000, 385, 371, - -1000, 476, 1144, 2506, -1000, -1000, -1000, 37, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 5123, 307, 235, 195, - 716, 709, 725, -1000, 476, -1000, 517, 55, -1000, 7795, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 161, 668, -1000, - 667, -1000, -1000, -1000, -32, -1000, -1000, -1000, 29, -1000, - -1000, -1000, 4747, 4747, 7961, 409, 307, 7795, -1000, -1000, - 196, -1000, -1000, 267, -1000, 7795, 307, 21, -149, 195, - 394, 390, -1000, -1000, -1000, -1000, -32, 624, -1000, 607, - -144, -152, -1000, -37, -1000, 605, -1000, -39, -147, 476, - -150, 4935, -153, 1119, 307, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 2702, 2702, -1000, -1000, 8114, -1000, + -1000, -1000, -1000, 721, 97, 274, -1000, 4734, 1168, 464, + 464, -1000, -1000, 65, -1000, -1000, 5110, 5110, 5110, 5110, + 5110, 5110, 5110, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 464, 73, -1000, + 4537, 464, 464, 464, 464, 464, 464, 4734, 464, 464, + 464, 464, 464, 464, 464, 464, 464, 464, 464, 464, + 464, 462, -1000, 293, 693, 326, 591, 6400, 576, -1000, + -1000, 0, 8114, -1000, 7948, 6763, 6763, 6763, 6763, 6763, + -1000, 585, 582, -1000, 571, 563, 554, 8114, -1000, 424, + 326, 6234, 87, 464, -1000, 7095, -1000, -1000, 0, 6763, + 8114, -1000, -1000, 7948, 384, -1000, -1000, -1000, -1000, 4734, + 3538, 2284, 160, 186, -97, -1000, -1000, 468, -1000, 468, + 468, 468, 468, -77, -77, -77, -77, -1000, -1000, -1000, + -1000, -1000, 500, -1000, 468, 468, 468, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 499, 499, 499, 473, 473, + 497, -1000, 8114, -1000, 657, 76, -1000, 8114, -1000, -1000, + 8114, 2702, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 602, 4734, 4734, + 344, 4734, 4734, 112, 5110, 232, 132, 5110, 5110, 5110, + 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, 5110, + 5110, 5110, 277, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 294, -1000, 501, 607, 607, 81, 81, 81, 81, + 81, 81, 5298, 3944, 3538, 414, 217, 4537, 4338, 4338, + 4734, 4734, 4338, 685, 124, 217, 7782, -1000, 326, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 4338, 4338, 4338, 4338, + 4734, -1000, -1000, -1000, 632, -1000, 685, 697, -1000, 614, + 610, 4338, -1000, 494, 464, -1000, 486, 702, 515, 523, + 962, -1000, -1000, -1000, -1000, 581, -1000, 579, -1000, -1000, + -1000, -1000, -1000, 326, -1000, 44, 43, 42, 7782, -1000, + 713, 461, -1000, -1000, -1000, 217, -1000, 72, -1000, 458, + 2075, -1000, -1000, -1000, -1000, -1000, -1000, 490, 649, 146, + 292, -1000, -1000, 630, -1000, 164, -108, -1000, -1000, 234, + -77, -77, -1000, -1000, 80, 618, 80, 80, 80, 267, + -1000, -1000, -1000, -1000, 230, -1000, -1000, -1000, 227, -1000, + 522, 7782, 2702, -1000, -1000, 145, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -3, -1000, + 2702, -1000, 595, 112, 155, -1000, -1000, 298, -1000, -1000, + 217, 217, 1322, -1000, -1000, -1000, -1000, 232, 5110, 5110, + 5110, 28, 1322, 1305, 596, 1020, 81, 150, 150, 123, + 123, 123, 123, 123, 211, 211, -1000, -1000, -1000, 326, + -1000, -1000, -1000, 326, 4338, 456, -1000, -1000, 5495, 70, + 464, 4734, -1000, 392, 392, 270, 259, 392, 4338, 204, + -1000, 4734, 326, -1000, 392, 326, 392, 392, -1000, -1000, + 8114, -1000, -1000, -1000, -1000, 452, 520, 7948, 464, -1000, + 6046, 7782, 698, 4734, -1000, -1000, 4734, 489, -1000, 4734, + -1000, -1000, -1000, -1000, 464, 464, 464, 375, -1000, 698, + -1000, 3329, 2284, -1000, 2284, 7782, -1000, 291, -1000, -1000, + 517, 22, -1000, -1000, -1000, 381, 80, 80, -1000, 276, + 163, -1000, -1000, -1000, 412, -1000, 446, 407, 8114, -1000, + -1000, -1000, 8114, -1000, -1000, -1000, -1000, -1000, 7782, -1000, + -1000, -1000, -1000, -1000, -1000, 28, 1322, 1273, -1000, 5110, + 5110, -1000, -1000, 392, 4338, -1000, -1000, 6929, -1000, -1000, + 2911, 4338, 217, -1000, -1000, 139, 277, 139, -139, 455, + 126, -1000, 4734, 355, -1000, -1000, -1000, -1000, -1000, -1000, + 713, 6763, -1000, 651, 380, 396, -1000, -1000, 4141, 326, + 390, 69, 375, 693, 217, 217, 7782, 217, 7782, 7782, + 7782, 5858, 7782, 693, -1000, 2075, -1000, 373, -1000, 468, + -1000, -93, 719, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 263, 207, -1000, 200, 2702, + -1000, -1000, 653, -1000, 5110, 1322, 1322, -1000, -1000, -1000, + -1000, 68, 326, 326, 468, 468, -1000, 468, 473, -1000, + 468, -41, 468, -59, 326, 326, 464, -136, -1000, 217, + 4734, 711, 398, 648, -1000, 464, -1000, -1000, 557, 7782, + 7782, -1000, -1000, 360, -1000, 301, 301, 301, 87, -1000, + -1000, -1000, 7782, -1000, 120, -1000, -120, -1000, 348, 334, + -1000, 464, 1322, 2493, -1000, -1000, -1000, 50, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 5110, 326, 257, 217, + 701, 695, 718, -1000, 464, -1000, 501, 63, -1000, 7782, + -1000, -1000, -1000, -1000, -1000, -1000, 162, 647, -1000, 633, + -1000, -1000, -1000, -16, -1000, -1000, -1000, 2, -1000, -1000, + -1000, 4734, 4734, 7948, 396, 326, 7782, -1000, -1000, 241, + -1000, -1000, 288, -1000, 7782, 326, 20, -147, 217, 394, + 384, -1000, -1000, -1000, -1000, -16, 608, -1000, 594, -143, + -151, -1000, -19, -1000, 589, -1000, -22, -145, 464, -149, + 4922, -153, 1080, 326, -1000, -1000, -1000, } var yyPgo = [...]int{ - 0, 923, 28, 618, 922, 919, 917, 913, 912, 910, - 909, 908, 907, 905, 904, 898, 897, 896, 880, 879, - 87, 876, 875, 873, 41, 871, 56, 870, 869, 26, - 44, 22, 30, 102, 868, 19, 88, 64, 867, 37, - 866, 865, 862, 861, 53, 858, 857, 1111, 856, 855, - 8, 21, 853, 847, 845, 844, 51, 49, 843, 842, - 840, 839, 836, 835, 35, 2, 5, 4, 13, 833, - 696, 12, 831, 45, 828, 826, 825, 822, 24, 821, - 46, 820, 17, 43, 818, 15, 58, 33, 18, 6, - 817, 48, 816, 558, 814, 100, 813, 812, 811, 810, - 808, 34, 300, 852, 27, 25, 806, 804, 11, 1174, - 32, 55, 16, 800, 40, 38, 20, 799, 794, 23, - 792, 791, 788, 787, 786, 785, 84, 782, 780, 778, - 10, 42, 777, 776, 54, 9, 775, 774, 773, 772, - 50, 771, 36, 766, 765, 750, 31, 14, 749, 7, - 747, 746, 3, 745, 744, 743, 0, 83, 742, 738, - 74, + 0, 910, 24, 602, 908, 907, 906, 905, 903, 902, + 900, 898, 895, 894, 892, 887, 886, 878, 876, 873, + 87, 872, 870, 869, 44, 868, 56, 867, 866, 29, + 54, 22, 26, 162, 865, 19, 74, 53, 864, 41, + 862, 861, 860, 854, 55, 851, 846, 1095, 844, 843, + 10, 23, 842, 841, 839, 838, 51, 46, 835, 832, + 829, 828, 827, 825, 36, 3, 5, 4, 11, 824, + 439, 6, 823, 34, 821, 819, 818, 816, 20, 814, + 43, 812, 17, 42, 811, 16, 50, 33, 14, 8, + 810, 48, 809, 569, 808, 115, 807, 806, 804, 800, + 792, 791, 30, 300, 909, 28, 27, 789, 788, 1162, + 32, 49, 18, 787, 40, 38, 21, 785, 782, 15, + 780, 778, 777, 776, 775, 774, 138, 773, 772, 771, + 9, 35, 770, 769, 57, 13, 768, 764, 763, 762, + 45, 761, 37, 760, 759, 758, 31, 12, 757, 7, + 756, 755, 2, 750, 747, 738, 0, 82, 737, 735, + 165, } var yyR1 = [...]int{ @@ -1554,7 +1553,7 @@ var yyR1 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 3, 4, 4, 5, 5, 6, 6, 23, 23, 7, 8, 8, 158, 158, 42, 42, 86, 86, 9, - 9, 9, 90, 90, 90, 107, 107, 117, 117, 10, + 9, 9, 90, 90, 90, 108, 108, 117, 117, 10, 10, 10, 10, 15, 143, 144, 144, 144, 140, 120, 120, 120, 123, 123, 121, 121, 121, 121, 121, 121, 121, 122, 122, 122, 122, 122, 124, 124, 124, 124, @@ -1572,7 +1571,7 @@ var yyR1 = [...]int{ 17, 17, 118, 118, 118, 18, 18, 19, 19, 19, 19, 19, 159, 20, 21, 21, 22, 22, 22, 26, 26, 26, 24, 24, 25, 25, 31, 31, 30, 30, - 32, 32, 32, 32, 106, 106, 106, 105, 105, 34, + 32, 32, 32, 32, 107, 107, 107, 106, 106, 34, 34, 35, 35, 36, 36, 37, 37, 37, 49, 49, 85, 85, 87, 87, 38, 38, 38, 38, 39, 39, 40, 40, 41, 41, 113, 113, 112, 112, 112, 111, @@ -1599,23 +1598,23 @@ var yyR1 = [...]int{ 89, 89, 91, 92, 92, 95, 95, 96, 96, 93, 93, 97, 97, 97, 97, 97, 97, 97, 97, 97, 97, 97, 98, 98, 98, 99, 99, 100, 100, 100, - 108, 108, 103, 103, 104, 104, 109, 109, 110, 110, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 101, 101, - 101, 101, 101, 101, 101, 101, 101, 101, 102, 102, + 101, 101, 104, 104, 105, 105, 109, 109, 110, 110, + 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, + 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, 102, - 102, 102, 102, 102, 102, 102, 102, 102, 102, 156, + 102, 102, 102, 102, 102, 102, 102, 102, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 103, + 103, 103, 103, 103, 103, 103, 103, 103, 103, 156, 157, 114, 115, 115, 115, } var yyR2 = [...]int{ @@ -1633,7 +1632,7 @@ var yyR2 = [...]int{ 5, 0, 3, 5, 0, 1, 0, 1, 0, 1, 2, 0, 2, 2, 2, 2, 2, 0, 3, 0, 1, 0, 3, 3, 0, 2, 0, 2, 1, 2, - 1, 0, 2, 5, 2, 3, 2, 2, 1, 1, + 1, 0, 2, 4, 2, 3, 2, 2, 1, 1, 1, 3, 2, 0, 1, 3, 1, 2, 3, 1, 1, 1, 6, 7, 7, 4, 5, 7, 1, 3, 8, 8, 5, 4, 6, 5, 3, 2, 3, 4, @@ -1696,7 +1695,7 @@ var yyChk = [...]int{ 115, 124, 48, 24, 125, 126, 129, 130, -156, 7, 201, 52, -155, 214, -78, 14, -22, 5, -20, -159, -20, -20, -20, -20, -143, 52, -100, 118, 69, 116, - 122, -103, 55, -102, 207, 149, 143, 170, 162, 160, + 122, -104, 55, -103, 207, 149, 143, 170, 162, 160, 163, 189, 64, 127, 158, 154, 152, 26, 175, 212, 153, 190, 147, 148, 174, 31, 209, 33, 135, 173, 169, 172, 146, 168, 37, 188, 165, 155, 17, 130, @@ -1704,17 +1703,17 @@ var yyChk = [...]int{ 145, 141, 166, 136, 156, 157, 171, 144, 167, 138, 180, 213, 164, 161, 142, 139, 140, 184, 185, 186, 187, 210, 159, 181, -93, 118, 120, 116, 116, 117, - 118, 116, -47, -109, 55, -102, 118, 116, 105, 163, + 118, 116, -47, -109, 55, -103, 118, 116, 105, 163, 111, 182, 117, 31, 137, -118, 116, 183, 140, 184, 185, 186, 187, 55, 191, 190, -109, -114, -114, -114, -114, -114, -2, -82, 16, 15, -5, -3, -156, 6, 19, 20, -26, 38, 39, -21, -93, -35, -36, -37, -38, -49, -70, -156, -47, 10, -42, -47, -89, -117, - -90, -91, 191, 190, 189, 163, 188, -71, -103, -109, - 55, -102, -144, -140, 55, 117, -47, 201, -96, 121, + -90, -91, 191, 190, 189, 163, 188, -71, -104, -109, + 55, -103, -144, -140, 55, 117, -47, 201, -96, 121, 116, -47, -47, -95, 121, 55, -95, -47, 108, -47, 55, 29, 193, 55, 137, 116, 138, 118, -115, -156, - -104, -103, -101, 70, 21, 23, 177, 73, 105, 15, + -105, -104, -102, 70, 21, 23, 177, 73, 105, 15, 74, 104, 202, 111, 46, 194, 195, 192, 193, 182, 28, 9, 24, 125, 20, 98, 113, 77, 78, 128, 22, 126, 68, 18, 49, 10, 12, 13, 121, 120, @@ -1727,20 +1726,20 @@ var yyChk = [...]int{ -157, 54, -83, 18, 30, -33, -52, 71, -57, 28, 22, -56, -53, -71, -69, -70, 105, 106, 94, 95, 102, 72, 107, -61, -59, -60, -62, 57, 56, 65, - 58, 59, 60, 61, 66, 67, 68, -103, -109, -67, + 58, 59, 60, 61, 66, 67, 68, -104, -109, -67, -156, 42, 43, 202, 203, 206, 204, 74, 32, 192, 200, 199, 198, 196, 197, 194, 195, 121, 193, 100, 201, -79, -80, -33, -78, -2, -20, 34, -24, 20, 63, -48, 25, -47, 29, 53, -43, -45, -44, -46, 40, 44, 46, 41, 42, 43, 47, -113, 21, -35, -2, -156, -112, 133, -111, 21, -109, 57, -47, -158, - 53, 10, 51, 53, -89, -107, -104, 57, 29, 79, + 53, 10, 51, 53, -89, -108, -105, 57, 29, 79, 108, 54, 53, -120, -123, -125, -124, -121, -122, 160, 161, 105, 164, 166, 167, 168, 169, 170, 171, 172, 173, 174, 175, 127, 156, 157, 158, 159, 143, 144, 145, 146, 147, 148, 149, 151, 152, 153, 154, 155, 55, -115, 118, -47, 71, -47, -115, 119, -47, 22, - 50, -47, -110, -109, -101, -115, -115, -115, -115, -115, + 50, -47, -110, -109, -102, -115, -115, -115, -115, -115, -115, -115, -115, -115, -115, -47, 8, 89, 70, 69, 86, 53, 17, -33, -54, 89, 71, 87, 88, 73, 91, 90, 101, 94, 95, 96, 97, 98, 99, 100, @@ -1749,56 +1748,56 @@ var yyChk = [...]int{ -57, -57, -57, -156, 108, -65, -33, -156, -156, -156, -156, -156, -156, -156, -74, -33, -156, -160, -156, -160, -160, -160, -160, -160, -160, -160, -156, -156, -156, -156, - 53, -81, 23, 24, -82, -157, -26, -58, -103, 58, + 53, -81, 23, 24, -82, -157, -26, -58, -104, 58, 61, -25, 41, -86, 133, -47, -89, -36, -37, -37, -36, -37, 40, 40, 40, 45, 40, 45, 40, -44, -109, -157, -157, -2, -50, 48, 120, 49, -156, -111, - -86, -35, -47, -91, -114, -33, -104, -110, -101, -145, - -146, -147, -104, 57, 58, -140, -141, -148, 123, 122, + -86, -35, -47, -91, -114, -33, -105, -110, -102, -145, + -146, -147, -105, 57, 58, -140, -141, -148, 123, 122, -142, 117, 27, -136, 66, 71, -132, 180, -126, 52, -126, -126, -126, -126, -130, 163, -130, -130, -130, 52, -126, -126, -126, -134, 52, -134, -134, -135, 52, -135, - -108, 51, -47, 22, -97, 114, -153, 112, 177, 163, + -101, 51, -47, 22, -97, 114, -153, 112, 177, 163, 64, 28, 113, 14, 202, 133, 213, 55, 134, -47, -47, -115, 36, -33, -33, -63, 66, 71, 67, 68, -33, -33, -57, -64, -67, -70, 62, 89, 87, 88, 73, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -57, -116, 55, 57, 55, - -56, -56, -103, -31, 20, -30, -32, 96, -33, -109, - -104, 53, -157, -30, -30, -33, -33, -30, -24, -72, - -73, 75, -103, -157, -30, -31, -30, -30, -80, -83, + -56, -56, -104, -31, 20, -30, -32, 96, -33, -109, + -105, 53, -157, -30, -30, -33, -33, -30, -24, -72, + -73, 75, -104, -157, -30, -31, -30, -30, -80, -83, -92, 18, 10, 32, 32, -30, -55, 29, 32, -2, -156, -156, -51, 11, -40, -39, 50, 51, -41, 50, - -39, 40, 40, -157, 117, 117, 117, -87, -103, -51, + -39, 40, 40, -157, 117, 117, 117, -87, -104, -51, -51, 108, 53, -147, 79, 52, 27, -142, 55, 55, -127, 28, 66, -133, 181, 58, -130, -130, -131, 104, - 29, -131, -131, -131, -139, 57, 58, 58, 50, -103, + 29, -131, -131, -131, -139, 57, 58, 58, 50, -104, -115, -114, -98, -99, 119, 21, 117, 27, 133, -115, 37, 66, 67, 68, -64, -57, -57, -57, -29, 128, - 70, -157, -157, -30, 53, -106, -105, 21, -103, 57, + 70, -157, -157, -30, 53, -107, -106, 21, -104, 57, 108, -156, -33, -157, -157, 53, 51, 21, -157, -30, -75, -73, 77, -33, -157, -157, -157, -157, -157, -47, -34, 10, -88, 50, -89, -66, -68, -67, -156, -2, - -84, -103, -87, -78, -33, -33, 52, -33, -156, -156, - -156, -157, 53, -78, -104, -146, -147, -150, -149, -103, + -84, -104, -87, -78, -33, -33, 52, -33, -156, -156, + -156, -157, 53, -78, -105, -146, -147, -150, -149, -104, 55, -129, 50, 57, 58, 59, 66, 192, 54, -131, -131, 55, 55, 105, 54, 53, 53, 54, 53, -47, - -47, -114, -103, -29, 70, -57, -57, -157, -32, -105, + -47, -114, -104, -29, 70, -57, -57, -157, -32, -106, 96, -110, -31, -119, 105, 160, 127, 158, 154, 174, 165, 179, 156, 180, -116, -119, 207, -78, 78, -33, 76, -51, -35, 26, -88, 53, -157, -157, -157, 53, - 108, -157, -82, -85, -103, -85, -85, -85, -112, -103, + 108, -157, -82, -85, -104, -85, -85, -85, -112, -104, -82, 54, 53, -126, -137, 177, 8, 57, 58, 58, -115, 25, -57, 108, -157, -157, -126, -126, -126, -135, -126, 148, -126, 148, -157, -157, -156, -28, 205, -33, - -76, 12, 27, -68, 32, -2, -156, -103, -103, 53, - 54, -157, -157, -157, -50, -108, -149, -138, 123, 27, - 122, 192, 54, 54, -156, 96, -130, 55, -57, -157, - 57, -77, 13, 15, 8, -66, -2, 108, -103, -128, - 64, 27, 27, -151, -152, 133, -27, 89, 210, -33, - -65, -89, -157, -103, 57, -157, 53, -103, -157, 208, - 47, 211, -152, 32, 37, 209, 212, 135, 37, 136, - 210, -156, 211, -57, 132, 212, -157, -157, + -76, 12, 27, -68, 32, -2, -156, -104, -104, 53, + 54, -157, -157, -157, -50, -149, -138, 123, 27, 122, + 192, 54, 54, -156, 96, -130, 55, -57, -157, 57, + -77, 13, 15, 8, -66, -2, 108, -104, -128, 64, + 27, 27, -151, -152, 133, -27, 89, 210, -33, -65, + -89, -157, -104, 57, -157, 53, -104, -157, 208, 47, + 211, -152, 32, 37, 209, 212, 135, 37, 136, 210, + -156, 211, -57, 132, 212, -157, -157, } var yyDef = [...]int{ @@ -1900,17 +1899,17 @@ var yyDef = [...]int{ 97, 395, 97, 398, 0, 0, 0, 402, 364, 408, 0, 423, 230, 0, 27, 0, 457, -2, 0, 0, 0, 38, 30, 0, 240, 0, 0, 0, 275, 243, - 31, 490, 0, 142, 126, 120, 0, 96, 0, 0, + 31, 133, 0, 142, 126, 120, 0, 96, 0, 0, 50, 0, 300, 0, 356, 359, 387, 121, 391, 392, 394, 396, 397, 399, 361, 360, 0, 0, 0, 406, 425, 0, 0, 455, 0, -2, 0, 450, 449, 0, - 249, 276, 277, 278, 239, 133, 141, 131, 0, 128, - 130, 118, 100, 103, 0, 223, 388, 389, 380, 363, - 403, 21, 0, 0, 0, 445, 22, 0, 241, 58, - 0, 127, 129, 0, 158, 0, 0, 0, 0, 426, - 424, 453, -2, 451, 132, 157, 0, 0, 362, 0, - 0, 0, 159, 0, 381, 0, 384, 0, 382, 0, - 0, 0, 0, 0, 0, 383, 160, 161, + 249, 276, 277, 278, 239, 141, 131, 0, 128, 130, + 118, 100, 103, 0, 223, 388, 389, 380, 363, 403, + 21, 0, 0, 0, 445, 22, 0, 241, 58, 0, + 127, 129, 0, 158, 0, 0, 0, 0, 426, 424, + 453, -2, 451, 132, 157, 0, 0, 362, 0, 0, + 0, 159, 0, 381, 0, 384, 0, 382, 0, 0, + 0, 0, 0, 0, 383, 160, 161, } var yyTok1 = [...]int{ @@ -3024,10 +3023,10 @@ yydefault: yyVAL.optVal = NewStrVal(yyDollar[2].bytes) } case 133: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:864 { - yyVAL.indexDefinition = &IndexDefinition{Info: yyDollar[1].indexInfo, Columns: yyDollar[3].indexColumns, Using: yyDollar[5].colIdent} + yyVAL.indexDefinition = &IndexDefinition{Info: yyDollar[1].indexInfo, Columns: yyDollar[3].indexColumns} } case 134: yyDollar = yyS[yypt-2 : yypt+1] @@ -5142,13 +5141,13 @@ yydefault: yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:2459 { - yyVAL.colIdent = ColIdent{} + yyVAL.empty = struct{}{} } case 491: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2461 { - yyVAL.colIdent = yyDollar[2].colIdent + yyVAL.empty = struct{}{} } case 492: yyDollar = yyS[yypt-1 : yypt+1] diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index bf224694911..72fedb7ee3e 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -238,9 +238,9 @@ func forceEOF(yylex interface{}) { %type for_from %type ignore_opt default_opt %type exists_opt -%type not_exists_opt non_rename_operation to_opt index_opt constraint_opt +%type not_exists_opt non_rename_operation to_opt index_opt constraint_opt using_opt %type reserved_keyword non_reserved_keyword -%type sql_id reserved_sql_id col_alias as_ci_opt charset_value using_opt +%type sql_id reserved_sql_id col_alias as_ci_opt charset_value %type table_id reserved_table_id table_alias as_opt_id %type as_opt %type force_eof ddl_force_eof @@ -860,9 +860,9 @@ column_comment_opt: } index_definition: - index_info '(' index_column_list ')' using_opt + index_info '(' index_column_list ')' { - $$ = &IndexDefinition{Info: $1, Columns: $3, Using: $5} + $$ = &IndexDefinition{Info: $1, Columns: $3} } index_info: @@ -2456,9 +2456,9 @@ constraint_opt: { $$ = struct{}{} } using_opt: - { $$ = ColIdent{} } + { $$ = struct{}{} } | USING sql_id - { $$ = $2 } + { $$ = struct{}{} } sql_id: ID From 087bf3922b4433338597db8d2732e0c459f29124 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 14 Feb 2018 20:39:47 -0800 Subject: [PATCH 21/72] Revert "Merge pull request #72 from tinyspeck/slack-sync-upstream/2018-02-07" This reverts commit 0ce5f561ab0985d1c33ef87ece04a1cc3ef765b9, reversing changes made to 713273a5326f13df275e429464f6b707099d3fbd. --- go/vt/vtgate/executor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index ca3067b6882..a72ce18cab6 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -574,7 +574,7 @@ func (e *Executor) handleShow(ctx context.Context, safeSession *SafeSession, sql s.Cell, s.Target.Keyspace, s.Target.Shard, - ts.Target.TabletType.String(), + ts.Tablet.Type.String(), state, topoproto.TabletAliasString(ts.Tablet.Alias), ts.Tablet.Hostname, From ac980b086162929b3b783663b4171b44cf0de178 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 14 Feb 2018 20:42:23 -0800 Subject: [PATCH 22/72] Revert "Merge pull request #68 from tinyspeck/vtqueryserver-remove-stray-comment" This reverts commit 80d72ac28be8e1839716b7d5dc26d0522c0991d8, reversing changes made to 732edb1ad2ec7877d038944188fc7eb80ebe3d23. --- go/vt/mysqlproxy/mysqlproxy.go | 1 + 1 file changed, 1 insertion(+) diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go index e3eebea702e..e89471d7039 100644 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ b/go/vt/mysqlproxy/mysqlproxy.go @@ -170,6 +170,7 @@ func (mp *Proxy) executeSelect(ctx context.Context, session *ProxySession, sql s query, comments := sqlparser.SplitTrailingComments(sql) stmt, err := sqlparser.Parse(query) if err != nil { + fmt.Printf("YYY parse error %s\n", query) return nil, err } sqlparser.Normalize(stmt, bindVariables, "vtp") From 6a6c28f2a02fe1635790656bc5965d3d83ca8d20 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 14 Feb 2018 20:42:39 -0800 Subject: [PATCH 23/72] Revert "Merge branch 'vtqueryserver' into slack-sync-upstream-2018-01-29" This reverts commit d48e84a993a9cddc59af53e167018dbb06afb640, reversing changes made to 83866b34b70f7767dc28902a08d2228e8d49b1a5. --- go/cmd/vtqueryserver/index.go | 31 -- go/cmd/vtqueryserver/plugin_auth_static.go | 28 -- .../vtqueryserver/plugin_grpcqueryservice.go | 34 -- go/cmd/vtqueryserver/vtqueryserver.go | 71 ---- go/vt/mysqlproxy/mysqlproxy.go | 226 ------------- go/vt/vtqueryserver/endtoend_test.go | 314 ------------------ go/vt/vtqueryserver/plugin_mysql_server.go | 240 ------------- .../vtqueryserver/plugin_mysql_server_test.go | 157 --------- go/vt/vtqueryserver/status.go | 90 ----- go/vt/vtqueryserver/vtqueryserver.go | 85 ----- .../tabletserver/query_executor_test.go | 16 +- go/vt/vttablet/tabletserver/tabletserver.go | 14 - 12 files changed, 10 insertions(+), 1296 deletions(-) delete mode 100644 go/cmd/vtqueryserver/index.go delete mode 100644 go/cmd/vtqueryserver/plugin_auth_static.go delete mode 100644 go/cmd/vtqueryserver/plugin_grpcqueryservice.go delete mode 100644 go/cmd/vtqueryserver/vtqueryserver.go delete mode 100644 go/vt/mysqlproxy/mysqlproxy.go delete mode 100644 go/vt/vtqueryserver/endtoend_test.go delete mode 100644 go/vt/vtqueryserver/plugin_mysql_server.go delete mode 100644 go/vt/vtqueryserver/plugin_mysql_server_test.go delete mode 100644 go/vt/vtqueryserver/status.go delete mode 100644 go/vt/vtqueryserver/vtqueryserver.go diff --git a/go/cmd/vtqueryserver/index.go b/go/cmd/vtqueryserver/index.go deleted file mode 100644 index 72b2637abf0..00000000000 --- a/go/cmd/vtqueryserver/index.go +++ /dev/null @@ -1,31 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 main - -import ( - "net/http" -) - -// This is a separate file so it can be selectively included/excluded from -// builds to opt in/out of the redirect. - -func init() { - // Anything unrecognized gets redirected to the status page. - http.HandleFunc("/", func(w http.ResponseWriter, r *http.Request) { - http.Redirect(w, r, "/debug/status", http.StatusFound) - }) -} diff --git a/go/cmd/vtqueryserver/plugin_auth_static.go b/go/cmd/vtqueryserver/plugin_auth_static.go deleted file mode 100644 index f95882338c7..00000000000 --- a/go/cmd/vtqueryserver/plugin_auth_static.go +++ /dev/null @@ -1,28 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 agreedto 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 main - -// This plugin imports staticauthserver to register the flat-file implementation of AuthServer. - -import ( - "github.com/youtube/vitess/go/mysql" - "github.com/youtube/vitess/go/vt/vtqueryserver" -) - -func init() { - vtqueryserver.RegisterPluginInitializer(func() { mysql.InitAuthServerStatic() }) -} diff --git a/go/cmd/vtqueryserver/plugin_grpcqueryservice.go b/go/cmd/vtqueryserver/plugin_grpcqueryservice.go deleted file mode 100644 index 0580c5310b3..00000000000 --- a/go/cmd/vtqueryserver/plugin_grpcqueryservice.go +++ /dev/null @@ -1,34 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 main - -import ( - "github.com/youtube/vitess/go/vt/servenv" - "github.com/youtube/vitess/go/vt/vttablet/grpcqueryservice" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver" -) - -// Imports and register the gRPC queryservice server - -func init() { - tabletserver.RegisterFunctions = append(tabletserver.RegisterFunctions, func(qsc tabletserver.Controller) { - if servenv.GRPCCheckServiceMap("queryservice") { - grpcqueryservice.Register(servenv.GRPCServer, qsc.QueryService()) - } - }) - -} diff --git a/go/cmd/vtqueryserver/vtqueryserver.go b/go/cmd/vtqueryserver/vtqueryserver.go deleted file mode 100644 index eed1ae3a77d..00000000000 --- a/go/cmd/vtqueryserver/vtqueryserver.go +++ /dev/null @@ -1,71 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 main - -import ( - "flag" - "os" - - log "github.com/golang/glog" - "github.com/youtube/vitess/go/vt/dbconfigs" - "github.com/youtube/vitess/go/vt/servenv" - "github.com/youtube/vitess/go/vt/vtqueryserver" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" -) - -var ( - mysqlSocketFile = flag.String("mysql-socket-file", "", "path to unix socket file to connect to mysql") -) - -func init() { - servenv.RegisterDefaultFlags() -} - -func main() { - dbconfigFlags := dbconfigs.AppConfig | dbconfigs.AppDebugConfig - dbconfigs.RegisterFlags(dbconfigFlags) - flag.Parse() - - if *servenv.Version { - servenv.AppVersion.Print() - os.Exit(0) - } - - if len(flag.Args()) > 0 { - flag.Usage() - log.Exit("vtqueryserver doesn't take any positional arguments") - } - if err := tabletenv.VerifyConfig(); err != nil { - log.Exitf("invalid config: %v", err) - } - - tabletenv.Init() - - servenv.Init() - - dbcfgs, err := dbconfigs.Init(*mysqlSocketFile, dbconfigFlags) - if err != nil { - log.Fatal(err) - } - - err = vtqueryserver.Init(dbcfgs) - if err != nil { - log.Exitf("error initializing proxy: %v", err) - } - - servenv.RunDefault() -} diff --git a/go/vt/mysqlproxy/mysqlproxy.go b/go/vt/mysqlproxy/mysqlproxy.go deleted file mode 100644 index e89471d7039..00000000000 --- a/go/vt/mysqlproxy/mysqlproxy.go +++ /dev/null @@ -1,226 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 mysqlproxy is a basic module that proxies a mysql server -// session to appropriate calls in a queryservice back end, with optional -// query normalization. -package mysqlproxy - -import ( - "context" - "fmt" - - log "github.com/golang/glog" - - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/vt/sqlparser" - "github.com/youtube/vitess/go/vt/vttablet/queryservice" - - querypb "github.com/youtube/vitess/go/vt/proto/query" -) - -// ProxySession holds session state for the proxy -type ProxySession struct { - TransactionID int64 - TargetString string - Options *querypb.ExecuteOptions - Autocommit bool -} - -// Proxy wraps the standalone query service -type Proxy struct { - target *querypb.Target - qs queryservice.QueryService - normalize bool -} - -// NewProxy creates a new proxy -func NewProxy(target *querypb.Target, qs queryservice.QueryService, normalize bool) *Proxy { - return &Proxy{ - target: target, - qs: qs, - normalize: normalize, - } -} - -// Execute runs the given sql query in the specified session -func (mp *Proxy) Execute(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*ProxySession, *sqltypes.Result, error) { - var err error - result := &sqltypes.Result{} - - switch sqlparser.Preview(sql) { - case sqlparser.StmtBegin: - err = mp.doBegin(ctx, session) - case sqlparser.StmtCommit: - err = mp.doCommit(ctx, session) - case sqlparser.StmtRollback: - err = mp.doRollback(ctx, session) - case sqlparser.StmtSet: - result, err = mp.doSet(ctx, session, sql, bindVariables) - case sqlparser.StmtInsert, sqlparser.StmtUpdate, sqlparser.StmtDelete, sqlparser.StmtReplace: - result, err = mp.executeDML(ctx, session, sql, bindVariables) - case sqlparser.StmtSelect: - result, err = mp.executeSelect(ctx, session, sql, bindVariables) - default: - result, err = mp.executeOther(ctx, session, sql, bindVariables) - } - - if err != nil { - return nil, nil, err - } - - return session, result, nil -} - -// Rollback rolls back the session -func (mp *Proxy) Rollback(ctx context.Context, session *ProxySession) error { - return mp.doRollback(ctx, session) -} - -func (mp *Proxy) doBegin(ctx context.Context, session *ProxySession) error { - txID, err := mp.qs.Begin(ctx, mp.target, session.Options) - if err != nil { - return err - } - session.TransactionID = txID - return nil -} - -func (mp *Proxy) doCommit(ctx context.Context, session *ProxySession) error { - if session.TransactionID == 0 { - return fmt.Errorf("commit: no open transaction") - - } - err := mp.qs.Commit(ctx, mp.target, session.TransactionID) - session.TransactionID = 0 - return err -} - -// Rollback rolls back the session -func (mp *Proxy) doRollback(ctx context.Context, session *ProxySession) error { - if session.TransactionID != 0 { - err := mp.qs.Rollback(ctx, mp.target, session.TransactionID) - session.TransactionID = 0 - return err - } - return nil -} - -// Set is currently ignored -func (mp *Proxy) doSet(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - vals, charset, _, err := sqlparser.ExtractSetValues(sql) - if err != nil { - return nil, err - } - if len(vals) > 0 && charset != "" { - return nil, err - } - - switch charset { - case "", "utf8", "utf8mb4", "latin1", "default": - break - default: - return nil, fmt.Errorf("unexpected value for charset: %v", charset) - } - - for k, v := range vals { - switch k { - case "autocommit": - val, ok := v.(int64) - if !ok { - return nil, fmt.Errorf("unexpected value type for autocommit: %T", v) - } - switch val { - case 0: - session.Autocommit = false - case 1: - if session.TransactionID != 0 { - if err := mp.doCommit(ctx, session); err != nil { - return nil, err - } - } - session.Autocommit = true - default: - return nil, fmt.Errorf("unexpected value for autocommit: %d", val) - } - default: - log.Warningf("Ignored inapplicable SET %v = %v", k, v) - } - } - - return &sqltypes.Result{}, nil -} - -// executeSelect runs the given select statement -func (mp *Proxy) executeSelect(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - if mp.normalize { - query, comments := sqlparser.SplitTrailingComments(sql) - stmt, err := sqlparser.Parse(query) - if err != nil { - fmt.Printf("YYY parse error %s\n", query) - return nil, err - } - sqlparser.Normalize(stmt, bindVariables, "vtp") - normalized := sqlparser.String(stmt) - sql = normalized + comments - } - - return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) -} - -// executeDML runs the given query handling autocommit semantics -func (mp *Proxy) executeDML(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - if mp.normalize { - query, comments := sqlparser.SplitTrailingComments(sql) - stmt, err := sqlparser.Parse(query) - if err != nil { - return nil, err - } - sqlparser.Normalize(stmt, bindVariables, "vtp") - normalized := sqlparser.String(stmt) - sql = normalized + comments - } - - if session.TransactionID != 0 { - return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) - - } else if session.Autocommit { - queries := []*querypb.BoundQuery{{ - Sql: sql, - BindVariables: bindVariables, - }} - - // This is a stopgap until there is a better way to do autocommit - results, err := mp.qs.ExecuteBatch(ctx, mp.target, queries, true /* asTransaction */, 0, session.Options) - if err != nil { - return nil, err - } - return &results[0], nil - - } else { - result, txnID, err := mp.qs.BeginExecute(ctx, mp.target, sql, bindVariables, session.Options) - if err != nil { - return nil, err - } - session.TransactionID = txnID - return result, nil - } -} - -// executeOther runs the given other statement bypassing the normalizer -func (mp *Proxy) executeOther(ctx context.Context, session *ProxySession, sql string, bindVariables map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - return mp.qs.Execute(ctx, mp.target, sql, bindVariables, session.TransactionID, session.Options) -} diff --git a/go/vt/vtqueryserver/endtoend_test.go b/go/vt/vtqueryserver/endtoend_test.go deleted file mode 100644 index b04a0bb8a58..00000000000 --- a/go/vt/vtqueryserver/endtoend_test.go +++ /dev/null @@ -1,314 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 vtqueryserver - -import ( - "context" - "flag" - "fmt" - "os" - "path" - "strings" - "testing" - - "github.com/youtube/vitess/go/mysql" - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/vt/dbconfigs" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" - "github.com/youtube/vitess/go/vt/vttest" - - vttestpb "github.com/youtube/vitess/go/vt/proto/vttest" -) - -var ( - queryServer *tabletserver.TabletServer - mysqlConnParams mysql.ConnParams - proxyConnParams mysql.ConnParams -) - -func TestMain(m *testing.M) { - flag.Parse() // Do not remove this comment, import into google3 depends on it - tabletenv.Init() - - exitCode := func() int { - // Launch MySQL. - // We need a Keyspace in the topology, so the DbName is set. - // We need a Shard too, so the database 'vttest' is created. - cfg := vttest.Config{ - Topology: &vttestpb.VTTestTopology{ - Keyspaces: []*vttestpb.Keyspace{ - { - Name: "vttest", - Shards: []*vttestpb.Shard{ - { - Name: "0", - DbNameOverride: "vttest", - }, - }, - }, - }, - }, - OnlyMySQL: true, - } - if err := cfg.InitSchemas("vttest", testSchema, nil); err != nil { - fmt.Fprintf(os.Stderr, "InitSchemas failed: %v\n", err) - return 1 - } - defer os.RemoveAll(cfg.SchemaDir) - cluster := vttest.LocalCluster{ - Config: cfg, - } - if err := cluster.Setup(); err != nil { - fmt.Fprintf(os.Stderr, "could not launch mysql: %v\n", err) - return 1 - } - defer cluster.TearDown() - - mysqlConnParams = cluster.MySQLConnParams() - - proxySock := path.Join(cluster.Env.Directory(), "mysqlproxy.sock") - - proxyConnParams.UnixSocket = proxySock - proxyConnParams.Uname = "proxy" - proxyConnParams.Pass = "letmein" - - *mysqlServerSocketPath = proxyConnParams.UnixSocket - *mysqlAuthServerImpl = "none" - - dbcfgs := dbconfigs.DBConfigs{ - App: mysqlConnParams, - } - - var err error - queryServer, err = initProxy(&dbcfgs) - if err != nil { - fmt.Fprintf(os.Stderr, "could not start proxy: %v\n", err) - return 1 - } - defer queryServer.StopService() - - initMySQLProtocol() - defer shutdownMySQLProtocol() - - return m.Run() - }() - os.Exit(exitCode) -} - -var testSchema = ` -create table test(id int, val varchar(256), primary key(id)); -create table valtest(intval int default 0, floatval float default null, charval varchar(256) default null, binval varbinary(256) default null, primary key(intval)); -` - -func testFetch(t *testing.T, conn *mysql.Conn, sql string, expectedRows int) *sqltypes.Result { - t.Helper() - - result, err := conn.ExecuteFetch(sql, 1000, false) - if err != nil { - t.Errorf("error: %v", err) - } - - if len(result.Rows) != expectedRows { - t.Errorf("expected %d rows but got %d", expectedRows, len(result.Rows)) - } - - return result -} - -func testDML(t *testing.T, conn *mysql.Conn, sql string, expectedNumQueries int64, expectedRowsAffected uint64) { - t.Helper() - - numQueries := tabletenv.MySQLStats.Count() - result, err := conn.ExecuteFetch(sql, 1000, false) - if err != nil { - t.Errorf("error: %v", err) - } - numQueries = tabletenv.MySQLStats.Count() - numQueries - - if numQueries != expectedNumQueries { - t.Errorf("expected %d mysql queries but got %d", expectedNumQueries, numQueries) - } - - if result.RowsAffected != expectedRowsAffected { - t.Errorf("expected %d rows affected but got %d", expectedRowsAffected, result.RowsAffected) - } -} - -func TestQueries(t *testing.T) { - ctx := context.Background() - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - // Try a simple query case. - testFetch(t, conn, "select * from test", 0) - - // Try a simple error case. - _, err = conn.ExecuteFetch("select * from aa", 1000, true) - if err == nil || !strings.Contains(err.Error(), "table aa not found in schema") { - t.Fatalf("expected error but got: %v", err) - } -} - -func TestAutocommitDMLs(t *testing.T) { - ctx := context.Background() - - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - conn2, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 3, 1) - - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 1) - - testDML(t, conn, "delete from test", 4, 1) - - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 0) -} - -func TestPassthroughDMLs(t *testing.T) { - ctx := context.Background() - - queryServer.SetPassthroughDMLs(true) - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - conn2, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 3, 1) - testDML(t, conn, "insert into test (id, val) values(2, 'hello')", 3, 1) - testDML(t, conn, "insert into test (id, val) values(3, 'hello')", 3, 1) - - // Subquery DMLs are errors in passthrough mode with SBR, unless - // SetAllowUnsafeDMLs is set - _, err = conn.ExecuteFetch("update test set val='goodbye'", 1000, true) - if err == nil || !strings.Contains(err.Error(), "cannot identify primary key of statement") { - t.Fatalf("expected error but got: %v", err) - } - - queryServer.SetAllowUnsafeDMLs(true) - - // This is 3 queries in passthrough mode and not 4 queries as it would - // be in non-passthrough mode - testDML(t, conn, "update test set val='goodbye'", 3, 3) - - testFetch(t, conn, "select * from test where val='goodbye'", 3) - testFetch(t, conn2, "select * from test where val='goodbye'", 3) - - testDML(t, conn, "delete from test", 4, 3) - - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 0) -} - -func TestTransactions(t *testing.T) { - ctx := context.Background() - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - conn2, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - testDML(t, conn, "begin", 1, 0) - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 1, 1) - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 0) - testDML(t, conn, "commit", 1, 0) - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 1) - - testDML(t, conn, "begin", 1, 0) - testDML(t, conn, "delete from test", 2, 1) - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "rollback", 1, 0) - - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 1) - - testDML(t, conn2, "begin", 1, 0) - testDML(t, conn2, "delete from test", 2, 1) - testDML(t, conn2, "commit", 1, 0) - - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 0) -} - -func TestNoAutocommit(t *testing.T) { - ctx := context.Background() - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - conn2, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - testFetch(t, conn, "set autocommit=0", 0) - - testDML(t, conn, "insert into test (id, val) values(1, 'hello')", 2, 1) - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 0) - testDML(t, conn, "commit", 1, 0) - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 1) - - testDML(t, conn, "delete from test", 3, 1) - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 1) - testDML(t, conn, "rollback", 1, 0) - - testFetch(t, conn, "select * from test", 1) - testFetch(t, conn2, "select * from test", 1) - - testFetch(t, conn2, "set autocommit=0", 0) - testDML(t, conn2, "delete from test", 3, 1) - testDML(t, conn2, "commit", 1, 0) - - testFetch(t, conn, "select * from test", 0) - testFetch(t, conn2, "select * from test", 0) -} - -func TestOther(t *testing.T) { - ctx := context.Background() - conn, err := mysql.Connect(ctx, &proxyConnParams) - if err != nil { - t.Fatal(err) - } - - testFetch(t, conn, "explain select * from test", 1) - testFetch(t, conn, "select table_name, table_rows from information_schema.tables where table_name='test'", 1) - -} diff --git a/go/vt/vtqueryserver/plugin_mysql_server.go b/go/vt/vtqueryserver/plugin_mysql_server.go deleted file mode 100644 index aa183989e43..00000000000 --- a/go/vt/vtqueryserver/plugin_mysql_server.go +++ /dev/null @@ -1,240 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 agreedto 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 vtqueryserver - -import ( - "flag" - "fmt" - "net" - "os" - "syscall" - - log "github.com/golang/glog" - "golang.org/x/net/context" - - "github.com/youtube/vitess/go/mysql" - "github.com/youtube/vitess/go/sqltypes" - "github.com/youtube/vitess/go/vt/callerid" - "github.com/youtube/vitess/go/vt/mysqlproxy" - "github.com/youtube/vitess/go/vt/servenv" - "github.com/youtube/vitess/go/vt/vttls" - - querypb "github.com/youtube/vitess/go/vt/proto/query" -) - -var ( - mysqlServerPort = flag.Int("mysqlproxy_server_port", -1, "If set, also listen for MySQL binary protocol connections on this port.") - mysqlServerBindAddress = flag.String("mysqlproxy_server_bind_address", "", "Binds on this address when listening to MySQL binary protocol. Useful to restrict listening to 'localhost' only for instance.") - mysqlServerSocketPath = flag.String("mysqlproxy_server_socket_path", "", "This option specifies the Unix socket file to use when listening for local connections. By default it will be empty and it won't listen to a unix socket") - mysqlAuthServerImpl = flag.String("mysql_auth_server_impl", "static", "Which auth server implementation to use.") - mysqlAllowClearTextWithoutTLS = flag.Bool("mysql_allow_clear_text_without_tls", false, "If set, the server will allow the use of a clear text password over non-SSL connections.") - - mysqlSslCert = flag.String("mysqlproxy_server_ssl_cert", "", "Path to the ssl cert for mysql server plugin SSL") - mysqlSslKey = flag.String("mysqlproxy_server_ssl_key", "", "Path to ssl key for mysql server plugin SSL") - mysqlSslCa = flag.String("mysqlproxy_server_ssl_ca", "", "Path to ssl CA for mysql server plugin SSL. If specified, server will require and validate client certs.") - - mysqlSlowConnectWarnThreshold = flag.Duration("mysqlproxy_slow_connect_warn_threshold", 0, "Warn if it takes more than the given threshold for a mysql connection to establish") -) - -// proxyHandler implements the Listener interface. -// It stores the Session in the ClientData of a Connection, if a transaction -// is in progress. -type proxyHandler struct { - mp *mysqlproxy.Proxy -} - -func newProxyHandler(mp *mysqlproxy.Proxy) *proxyHandler { - return &proxyHandler{ - mp: mp, - } -} - -func (mh *proxyHandler) NewConnection(c *mysql.Conn) { -} - -func (mh *proxyHandler) ConnectionClosed(c *mysql.Conn) { - // Rollback if there is an ongoing transaction. Ignore error. - ctx := context.Background() - session, _ := c.ClientData.(*mysqlproxy.ProxySession) - if session != nil && session.TransactionID != 0 { - _ = mh.mp.Rollback(ctx, session) - } -} - -func (mh *proxyHandler) ComQuery(c *mysql.Conn, query string, callback func(*sqltypes.Result) error) error { - // FIXME(alainjobart): Add some kind of timeout to the context. - ctx := context.Background() - - // Fill in the ImmediateCallerID with the UserData returned by - // the AuthServer plugin for that user. If nothing was - // returned, use the User. This lets the plugin map a MySQL - // user used for authentication to a Vitess User used for - // Table ACLs and Vitess authentication in general. - im := c.UserData.Get() - ef := callerid.NewEffectiveCallerID( - c.User, /* principal: who */ - c.RemoteAddr().String(), /* component: running client process */ - "mysqlproxy MySQL Connector" /* subcomponent: part of the client */) - ctx = callerid.NewContext(ctx, ef, im) - - session, _ := c.ClientData.(*mysqlproxy.ProxySession) - if session == nil { - session = &mysqlproxy.ProxySession{ - Options: &querypb.ExecuteOptions{ - IncludedFields: querypb.ExecuteOptions_ALL, - }, - Autocommit: true, - } - if c.Capabilities&mysql.CapabilityClientFoundRows != 0 { - session.Options.ClientFoundRows = true - } - } - if c.SchemaName != "" { - session.TargetString = c.SchemaName - } - session, result, err := mh.mp.Execute(ctx, session, query, make(map[string]*querypb.BindVariable)) - c.ClientData = session - err = mysql.NewSQLErrorFromError(err) - if err != nil { - return err - } - - return callback(result) -} - -var mysqlListener *mysql.Listener -var mysqlUnixListener *mysql.Listener - -// initiMySQLProtocol starts the mysql protocol. -// It should be called only once in a process. -func initMySQLProtocol() { - log.Infof("initializing mysql protocol") - - // Flag is not set, just return. - if *mysqlServerPort < 0 && *mysqlServerSocketPath == "" { - return - } - - // If no mysqlproxy was created, just return. - if mysqlProxy == nil { - log.Fatalf("mysqlProxy not initialized") - return - } - - // Initialize registered AuthServer implementations (or other plugins) - for _, initFn := range pluginInitializers { - initFn() - } - authServer := mysql.GetAuthServer(*mysqlAuthServerImpl) - - // Create a Listener. - var err error - mh := newProxyHandler(mysqlProxy) - if *mysqlServerPort >= 0 { - mysqlListener, err = mysql.NewListener("tcp", net.JoinHostPort(*mysqlServerBindAddress, fmt.Sprintf("%v", *mysqlServerPort)), authServer, mh) - if err != nil { - log.Exitf("mysql.NewListener failed: %v", err) - } - if *mysqlSslCert != "" && *mysqlSslKey != "" { - mysqlListener.TLSConfig, err = vttls.ServerConfig(*mysqlSslCert, *mysqlSslKey, *mysqlSslCa) - if err != nil { - log.Exitf("grpcutils.TLSServerConfig failed: %v", err) - return - } - } - mysqlListener.AllowClearTextWithoutTLS = *mysqlAllowClearTextWithoutTLS - - // Check for the connection threshold - if *mysqlSlowConnectWarnThreshold != 0 { - log.Infof("setting mysql slow connection threshold to %v", mysqlSlowConnectWarnThreshold) - mysqlListener.SlowConnectWarnThreshold = *mysqlSlowConnectWarnThreshold - } - // Start listening for tcp - go mysqlListener.Accept() - log.Infof("listening on %s:%d", *mysqlServerBindAddress, *mysqlServerPort) - } - - if *mysqlServerSocketPath != "" { - // Let's create this unix socket with permissions to all users. In this way, - // clients can connect to mysqlproxy mysql server without being mysqlproxy user - oldMask := syscall.Umask(000) - mysqlUnixListener, err = newMysqlUnixSocket(*mysqlServerSocketPath, authServer, mh) - _ = syscall.Umask(oldMask) - if err != nil { - log.Exitf("mysql.NewListener failed: %v", err) - return - } - // Listen for unix socket - go mysqlUnixListener.Accept() - } -} - -// newMysqlUnixSocket creates a new unix socket mysql listener. If a socket file already exists, attempts -// to clean it up. -func newMysqlUnixSocket(address string, authServer mysql.AuthServer, handler mysql.Handler) (*mysql.Listener, error) { - listener, err := mysql.NewListener("unix", address, authServer, handler) - switch err := err.(type) { - case nil: - return listener, nil - case *net.OpError: - log.Warningf("Found existent socket when trying to create new unix mysql listener: %s, attempting to clean up", address) - // err.Op should never be different from listen, just being extra careful - // in case in the future other errors are returned here - if err.Op != "listen" { - return nil, err - } - _, dialErr := net.Dial("unix", address) - if dialErr == nil { - log.Errorf("Existent socket '%s' is still accepting connections, aborting", address) - return nil, err - } - removeFileErr := os.Remove(address) - if removeFileErr != nil { - log.Errorf("Couldn't remove existent socket file: %s", address) - return nil, err - } - listener, listenerErr := mysql.NewListener("unix", address, authServer, handler) - return listener, listenerErr - default: - return nil, err - } -} - -func shutdownMySQLProtocol() { - log.Infof("shutting down mysql protocol") - if mysqlListener != nil { - mysqlListener.Close() - mysqlListener = nil - } - - if mysqlUnixListener != nil { - mysqlUnixListener.Close() - mysqlUnixListener = nil - } -} - -func init() { - servenv.OnRun(initMySQLProtocol) - servenv.OnTerm(shutdownMySQLProtocol) -} - -var pluginInitializers []func() - -// RegisterPluginInitializer lets plugins register themselves to be init'ed at servenv.OnRun-time -func RegisterPluginInitializer(initializer func()) { - pluginInitializers = append(pluginInitializers, initializer) -} diff --git a/go/vt/vtqueryserver/plugin_mysql_server_test.go b/go/vt/vtqueryserver/plugin_mysql_server_test.go deleted file mode 100644 index 487aa51c36a..00000000000 --- a/go/vt/vtqueryserver/plugin_mysql_server_test.go +++ /dev/null @@ -1,157 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 agreedto 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 vtqueryserver - -import ( - "io/ioutil" - "os" - "strings" - "testing" - - "golang.org/x/net/context" - - "github.com/youtube/vitess/go/mysql" - "github.com/youtube/vitess/go/sqltypes" -) - -type testHandler struct { - lastConn *mysql.Conn -} - -func (th *testHandler) NewConnection(c *mysql.Conn) { - th.lastConn = c -} - -func (th *testHandler) ConnectionClosed(c *mysql.Conn) { -} - -func (th *testHandler) ComQuery(c *mysql.Conn, q string, callback func(*sqltypes.Result) error) error { - return nil -} - -func TestConnectionUnixSocket(t *testing.T) { - th := &testHandler{} - - authServer := mysql.NewAuthServerStatic() - - authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ - { - Password: "password1", - UserData: "userData1", - SourceHost: "localhost", - }, - } - - // Use tmp file to reserve a path, remove it immediately, we only care about - // name in this context - unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") - if err != nil { - t.Fatalf("Failed to create temp file") - } - os.Remove(unixSocket.Name()) - - l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) - if err != nil { - t.Fatalf("NewUnixSocket failed: %v", err) - } - defer l.Close() - go l.Accept() - - params := &mysql.ConnParams{ - UnixSocket: unixSocket.Name(), - Uname: "user1", - Pass: "password1", - } - - c, err := mysql.Connect(context.Background(), params) - if err != nil { - t.Errorf("Should be able to connect to server but found error: %v", err) - } - c.Close() -} - -func TestConnectionStaleUnixSocket(t *testing.T) { - th := &testHandler{} - - authServer := mysql.NewAuthServerStatic() - - authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ - { - Password: "password1", - UserData: "userData1", - SourceHost: "localhost", - }, - } - - // First let's create a file. In this way, we simulate - // having a stale socket on disk that needs to be cleaned up. - unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") - if err != nil { - t.Fatalf("Failed to create temp file") - } - - l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) - if err != nil { - t.Fatalf("NewListener failed: %v", err) - } - defer l.Close() - go l.Accept() - - params := &mysql.ConnParams{ - UnixSocket: unixSocket.Name(), - Uname: "user1", - Pass: "password1", - } - - c, err := mysql.Connect(context.Background(), params) - if err != nil { - t.Errorf("Should be able to connect to server but found error: %v", err) - } - c.Close() -} - -func TestConnectionRespectsExistingUnixSocket(t *testing.T) { - th := &testHandler{} - - authServer := mysql.NewAuthServerStatic() - - authServer.Entries["user1"] = []*mysql.AuthServerStaticEntry{ - { - Password: "password1", - UserData: "userData1", - SourceHost: "localhost", - }, - } - - unixSocket, err := ioutil.TempFile("", "mysql_vitess_test.sock") - if err != nil { - t.Fatalf("Failed to create temp file") - } - os.Remove(unixSocket.Name()) - - l, err := newMysqlUnixSocket(unixSocket.Name(), authServer, th) - if err != nil { - t.Errorf("NewListener failed: %v", err) - } - defer l.Close() - go l.Accept() - _, err = newMysqlUnixSocket(unixSocket.Name(), authServer, th) - want := "listen unix" - if err == nil || !strings.HasPrefix(err.Error(), want) { - t.Errorf("Error: %v, want prefix %s", err, want) - } -} diff --git a/go/vt/vtqueryserver/status.go b/go/vt/vtqueryserver/status.go deleted file mode 100644 index 04606919d9e..00000000000 --- a/go/vt/vtqueryserver/status.go +++ /dev/null @@ -1,90 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 agreedto 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 vtqueryserver - -import ( - "github.com/youtube/vitess/go/vt/servenv" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver" -) - -var ( - // proxyTemplate contains the style sheet and the tablet itself. - proxyTemplate = ` - - - - - - - - -
- Target Keyspace: {{.Target.Keyspace}}
-
- Schema
- Schema Query Plans
- Schema Query Stats
- Schema Table Stats
-
- Query Stats
- Streaming Query Stats
- Consolidations
- Current Query Log
- Current Transaction Log
- In-flight 2PC Transactions
-
- Query Service Health Check
- Current Stream Queries
-
-` -) - -// For use by plugins which wish to avoid racing when registering status page parts. -var onStatusRegistered func() - -func addStatusParts(qsc tabletserver.Controller) { - servenv.AddStatusPart("Target", proxyTemplate, func() interface{} { - return map[string]interface{}{ - "Target": target, - } - }) - qsc.AddStatusPart() - if onStatusRegistered != nil { - onStatusRegistered() - } -} diff --git a/go/vt/vtqueryserver/vtqueryserver.go b/go/vt/vtqueryserver/vtqueryserver.go deleted file mode 100644 index a048d0cfeda..00000000000 --- a/go/vt/vtqueryserver/vtqueryserver.go +++ /dev/null @@ -1,85 +0,0 @@ -/* -Copyright 2017 Google Inc. - -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 vtqueryserver is a standalone version of the tablet server that -// only implements the queryservice interface without any of the topology, -// replication management, or other features of the full vttablet. -package vtqueryserver - -import ( - "flag" - - log "github.com/golang/glog" - - "github.com/youtube/vitess/go/vt/dbconfigs" - "github.com/youtube/vitess/go/vt/mysqlproxy" - "github.com/youtube/vitess/go/vt/servenv" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver" - "github.com/youtube/vitess/go/vt/vttablet/tabletserver/tabletenv" - - querypb "github.com/youtube/vitess/go/vt/proto/query" - topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" -) - -var ( - mysqlProxy *mysqlproxy.Proxy - target = querypb.Target{ - TabletType: topodatapb.TabletType_MASTER, - Keyspace: "", - } - - targetKeyspace = flag.String("target", "", "Target database name") - normalizeQueries = flag.Bool("normalize_queries", true, "Rewrite queries with bind vars. Turn this off if the app itself sends normalized queries with bind vars.") - allowUnsafeDMLs = flag.Bool("allow_unsafe_dmls", false, "Allow passthrough DML statements when running with statement-based replication") -) - -func initProxy(dbcfgs *dbconfigs.DBConfigs) (*tabletserver.TabletServer, error) { - target.Keyspace = *targetKeyspace - log.Infof("initalizing vtqueryserver.Proxy for target %s", target.Keyspace) - - // creates and registers the query service - qs := tabletserver.NewTabletServerWithNilTopoServer(tabletenv.Config) - qs.SetAllowUnsafeDMLs(*allowUnsafeDMLs) - mysqlProxy = mysqlproxy.NewProxy(&target, qs, *normalizeQueries) - - err := qs.StartService(target, *dbcfgs) - if err != nil { - return nil, err - } - - return qs, nil -} - -// Init initializes the proxy -func Init(dbcfgs *dbconfigs.DBConfigs) error { - qs, err := initProxy(dbcfgs) - if err != nil { - return err - } - - servenv.OnRun(func() { - qs.Register() - addStatusParts(qs) - }) - - servenv.OnClose(func() { - // We now leave the queryservice running during lameduck, - // so stop it in OnClose(), after lameduck is over. - qs.StopService() - }) - - return nil -} diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go index c6e766c165d..548ff74a061 100644 --- a/go/vt/vttablet/tabletserver/query_executor_test.go +++ b/go/vt/vttablet/tabletserver/query_executor_test.go @@ -108,6 +108,8 @@ func TestQueryExecutorPlanPassDmlRBR(t *testing.T) { func TestQueryExecutorPassthroughDml(t *testing.T) { db := setUpQueryExecutorTest(t) defer db.Close() + planbuilder.PassthroughDMLs = true + defer func() { planbuilder.PassthroughDMLs = false }() query := "update test_table set pk = foo()" want := &sqltypes.Result{} db.AddQuery(query, want) @@ -116,8 +118,9 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() - tsv.SetPassthroughDMLs(true) - defer tsv.SetPassthroughDMLs(false) + planbuilder.PassthroughDMLs = true + defer func() { planbuilder.PassthroughDMLs = false }() + tsv.qe.passthroughDMLs.Set(true) tsv.qe.binlogFormat = connpool.BinlogFormatRow txid := newTransaction(tsv, nil) @@ -144,7 +147,7 @@ func TestQueryExecutorPassthroughDml(t *testing.T) { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - tsv.SetAllowUnsafeDMLs(true) + tsv.qe.allowUnsafeDMLs = true got, err = qre.Execute() if !reflect.DeepEqual(got, want) { t.Fatalf("got: %v, want: %v", got, want) @@ -198,8 +201,9 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { tsv := newTestTabletServer(ctx, noFlags, db) defer tsv.StopService() - tsv.SetPassthroughDMLs(true) - defer tsv.SetPassthroughDMLs(false) + planbuilder.PassthroughDMLs = true + defer func() { planbuilder.PassthroughDMLs = false }() + tsv.qe.passthroughDMLs.Set(true) tsv.qe.binlogFormat = connpool.BinlogFormatRow qre := newTestQueryExecutor(ctx, tsv, query, 0) @@ -219,7 +223,7 @@ func TestQueryExecutorPassthroughDmlAutoCommit(t *testing.T) { t.Errorf("qre.Execute: %v, want %v", code, vtrpcpb.Code_INVALID_ARGUMENT) } - tsv.SetAllowUnsafeDMLs(true) + tsv.qe.allowUnsafeDMLs = true got, err = qre.Execute() if err != nil { t.Fatalf("qre.Execute() = %v, want nil", err) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 942075fd8df..f5add2a1eb7 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -440,7 +440,6 @@ func (tsv *TabletServer) decideAction(tabletType topodatapb.TabletType, serving func (tsv *TabletServer) fullStart() (err error) { c, err := dbconnpool.NewDBConnection(&tsv.dbconfigs.App, tabletenv.MySQLStats) if err != nil { - log.Errorf("error creating db app connection: %v", err) return err } c.Close() @@ -1976,19 +1975,6 @@ func (tsv *TabletServer) MaxDMLRows() int { return int(tsv.qe.maxDMLRows.Get()) } -// 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 -} - // queryAsString prints a readable version of query+bind variables, // and also truncates data if it's too long func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable) string { From 48058d7250871052b4347756ebaaf7b11ce76e3d Mon Sep 17 00:00:00 2001 From: Maggie Zhou Date: Tue, 20 Mar 2018 11:36:19 -0700 Subject: [PATCH 24/72] Update vendor.json with the `promhttp` package and a newer `prometheus` package. --- vendor/vendor.json | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/vendor/vendor.json b/vendor/vendor.json index a0f3bf92d25..a1cbb0998a2 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -623,10 +623,16 @@ "revisionTime": "2016-01-10T10:55:54Z" }, { - "checksumSHA1": "f+c772l+hly4ZdqJqe1dryr0398=", + "checksumSHA1": "I87tkF1e/hrl4d/XIKFfkPRq1ww=", "path": "github.com/prometheus/client_golang/prometheus", - "revision": "488edd04dc224ba64c401747cd0a4b5f05dfb234", - "revisionTime": "2016-05-31T09:15:28Z" + "revision": "d49167c4b9f3c4451707560c5c71471ff5291aaa", + "revisionTime": "2018-03-19T13:17:21Z" + }, + { + "checksumSHA1": "mIWVz1E1QJ6yZnf7ELNwLboyK4w=", + "path": "github.com/prometheus/client_golang/prometheus/promhttp", + "revision": "d49167c4b9f3c4451707560c5c71471ff5291aaa", + "revisionTime": "2018-03-19T13:17:21Z" }, { "checksumSHA1": "DvwvOlPNAgRntBzt3b3OSRMS2N4=", From 60047681c0023c381866840dea5519ece1e10020 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 21 Mar 2018 15:28:49 -0700 Subject: [PATCH 25/72] update bootstrap script to include better diagnostics of versions Signed-off-by: Michael Demmer --- bootstrap.sh | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/bootstrap.sh b/bootstrap.sh index d969ff29b96..f12413b1806 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -1,13 +1,13 @@ #!/bin/bash # Copyright 2017 Google Inc. -# +# # 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. @@ -55,6 +55,7 @@ zk_dist=$VTROOT/dist/vt-zookeeper-$zk_ver if [ -f $zk_dist/.build_finished ]; then echo "skipping zookeeper build. remove $zk_dist to force rebuild." else + echo "installing zookeeper $zk_ver" rm -rf $zk_dist (cd $VTROOT/dist && \ wget http://apache.org/dist/zookeeper/zookeeper-$zk_ver/zookeeper-$zk_ver.tar.gz && \ @@ -74,6 +75,7 @@ etcd_version_file=$etcd_dist/version if [[ -f $etcd_version_file && "$(cat $etcd_version_file)" == "$etcd_version" ]]; then echo "skipping etcd install. remove $etcd_version_file to force re-install." else + echo "installing etcd $etcd_version" rm -rf $etcd_dist mkdir -p $etcd_dist download_url=https://github.com/coreos/etcd/releases/download @@ -92,6 +94,7 @@ consul_version_file=$consul_dist/version if [[ -f $consul_version_file && "$(cat $consul_version_file)" == "$consul_version" ]]; then echo "skipping consul install. remove $consul_version_file to force re-install." else + echo "installing consul $consul_version" rm -rf $consul_dist mkdir -p $consul_dist download_url=https://releases.hashicorp.com/consul @@ -112,6 +115,7 @@ if [ $SKIP_ROOT_INSTALLS == "True" ]; then elif [[ -f $grpc_dist/.build_finished && "$(cat $grpc_dist/.build_finished)" == "$grpc_ver" ]]; then echo "skipping gRPC build. remove $grpc_dist to force rebuild." else + echo "installing grpc $grpc_ver" # unlink homebrew's protobuf, to be able to compile the downloaded protobuf package if [[ `uname -s` == "Darwin" && "$(brew list -1 | grep google-protobuf)" ]]; then brew unlink grpc/grpc/google-protobuf From 1657d6888f5cc6ab620e036b5888b153f31c0ff6 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Wed, 21 Mar 2018 15:29:50 -0700 Subject: [PATCH 26/72] use consul version 1.0.6 in bootstrap Signed-off-by: Michael Demmer --- bootstrap.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bootstrap.sh b/bootstrap.sh index f12413b1806..65d504b4b34 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -88,7 +88,7 @@ fi ln -snf $etcd_dist/etcd-${etcd_version}-linux-amd64/etcd $VTROOT/bin/etcd # Download and install consul, link consul binary into our root. -consul_version=0.7.2 +consul_version=1.0.6 consul_dist=$VTROOT/dist/consul consul_version_file=$consul_dist/version if [[ -f $consul_version_file && "$(cat $consul_version_file)" == "$consul_version" ]]; then From 7c02d4a28829c5d54de3b4b05243c9e1116fa655 Mon Sep 17 00:00:00 2001 From: Alex Charis Date: Thu, 14 Jun 2018 08:17:22 -0400 Subject: [PATCH 27/72] drop the "from" clause in "show tables from dbname like 'foo%'" we already have a destination, and the database name is set in the mysql connection already, so we use that. this means that, with keyspaces A and B: use A show tables from B will show the tables from A, which may be unexpected. an alternative is to find the tablet from the database name and route the show statement to it. Signed-off-by: Alex Charis --- go/vt/vtgate/executor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index bce2f15ca03..3d85027ecc7 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -753,7 +753,7 @@ func (e *Executor) handleShow(ctx context.Context, safeSession *SafeSession, sql } case sqlparser.KeywordString(sqlparser.TABLES): if show.ShowTablesOpt != nil && show.ShowTablesOpt.DbName != "" { - show.ShowTablesOpt.DbName = "vt_" + destKeyspace + show.ShowTablesOpt.DbName = "" } sql = sqlparser.String(show) case sqlparser.KeywordString(sqlparser.DATABASES), sqlparser.KeywordString(sqlparser.VITESS_KEYSPACES): From 4db514633a63c67354b42f67c12d0611d3b2122f Mon Sep 17 00:00:00 2001 From: Alex Charis Date: Wed, 27 Jun 2018 20:15:11 -0400 Subject: [PATCH 28/72] re-add support for: show full columns from messages from test_keyspace like '%' Signed-off-by: Alex Charis --- go/vt/sqlparser/ast.go | 31 +- go/vt/sqlparser/parse_test.go | 2 + go/vt/sqlparser/sql.go | 4074 ++++++++++++++++----------------- go/vt/sqlparser/sql.y | 35 +- go/vt/sqlparser/token.go | 1 - 5 files changed, 2058 insertions(+), 2085 deletions(-) diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go index 8401d2cf24a..8416dc56187 100644 --- a/go/vt/sqlparser/ast.go +++ b/go/vt/sqlparser/ast.go @@ -1290,19 +1290,14 @@ type Show struct { func (node *Show) Format(buf *TrackedBuffer) { if (node.Type == "tables" || node.Type == "columns") && node.ShowTablesOpt != nil { opt := node.ShowTablesOpt + buf.Myprintf("show %s%s", opt.Full, node.Type) + if node.Type == "columns" && node.HasOnTable() { + buf.Myprintf(" from %v", node.OnTable) + } if opt.DbName != "" { - if opt.Filter != nil { - buf.Myprintf("show %s%s%s from %s %v", opt.Extended, opt.Full, node.Type, opt.DbName, opt.Filter) - } else { - buf.Myprintf("show %s%s%s from %s", opt.Extended, opt.Full, node.Type, opt.DbName) - } - } else { - if opt.Filter != nil { - buf.Myprintf("show %s%s%s %v", opt.Extended, opt.Full, node.Type, opt.Filter) - } else { - buf.Myprintf("show %s%s%s", opt.Extended, opt.Full, node.Type) - } + buf.Myprintf(" from %s", opt.DbName) } + buf.Myprintf("%v", opt.Filter) return } if node.Scope == "" { @@ -1326,10 +1321,9 @@ func (node *Show) walkSubtree(visit Visit) error { // ShowTablesOpt is show tables option type ShowTablesOpt struct { - Extended string - Full string - DbName string - Filter *ShowFilter + Full string + DbName string + Filter *ShowFilter } // ShowFilter is show tables filter @@ -1340,10 +1334,13 @@ type ShowFilter struct { // Format formats the node. func (node *ShowFilter) Format(buf *TrackedBuffer) { + if node == nil { + return + } if node.Like != "" { - buf.Myprintf("like '%s'", node.Like) + buf.Myprintf(" like '%s'", node.Like) } else { - buf.Myprintf("where %v", node.Filter) + buf.Myprintf(" where %v", node.Filter) } } diff --git a/go/vt/sqlparser/parse_test.go b/go/vt/sqlparser/parse_test.go index 4824ed7d65a..23360514d58 100644 --- a/go/vt/sqlparser/parse_test.go +++ b/go/vt/sqlparser/parse_test.go @@ -1130,6 +1130,8 @@ var ( input: "show full tables where 1 = 0", }, { input: "show full columns from a like '%'", + }, { + input: "show full columns from messages from test_keyspace like '%'", }, { input: "show triggers", output: "show triggers", diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index 557f1e4d2c8..8e968012588 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -290,47 +290,46 @@ const VITESS_KEYSPACES = 57533 const VITESS_SHARDS = 57534 const VITESS_TABLETS = 57535 const VSCHEMA_TABLES = 57536 -const EXTENDED = 57537 -const FULL = 57538 -const PROCESSLIST = 57539 -const COLUMNS = 57540 -const NAMES = 57541 -const CHARSET = 57542 -const GLOBAL = 57543 -const SESSION = 57544 -const ISOLATION = 57545 -const LEVEL = 57546 -const READ = 57547 -const WRITE = 57548 -const ONLY = 57549 -const REPEATABLE = 57550 -const COMMITTED = 57551 -const UNCOMMITTED = 57552 -const SERIALIZABLE = 57553 -const CURRENT_TIMESTAMP = 57554 -const DATABASE = 57555 -const CURRENT_DATE = 57556 -const CURRENT_TIME = 57557 -const LOCALTIME = 57558 -const LOCALTIMESTAMP = 57559 -const UTC_DATE = 57560 -const UTC_TIME = 57561 -const UTC_TIMESTAMP = 57562 -const REPLACE = 57563 -const CONVERT = 57564 -const CAST = 57565 -const SUBSTR = 57566 -const SUBSTRING = 57567 -const GROUP_CONCAT = 57568 -const SEPARATOR = 57569 -const MATCH = 57570 -const AGAINST = 57571 -const BOOLEAN = 57572 -const LANGUAGE = 57573 -const WITH = 57574 -const QUERY = 57575 -const EXPANSION = 57576 -const UNUSED = 57577 +const FULL = 57537 +const PROCESSLIST = 57538 +const COLUMNS = 57539 +const NAMES = 57540 +const CHARSET = 57541 +const GLOBAL = 57542 +const SESSION = 57543 +const ISOLATION = 57544 +const LEVEL = 57545 +const READ = 57546 +const WRITE = 57547 +const ONLY = 57548 +const REPEATABLE = 57549 +const COMMITTED = 57550 +const UNCOMMITTED = 57551 +const SERIALIZABLE = 57552 +const CURRENT_TIMESTAMP = 57553 +const DATABASE = 57554 +const CURRENT_DATE = 57555 +const CURRENT_TIME = 57556 +const LOCALTIME = 57557 +const LOCALTIMESTAMP = 57558 +const UTC_DATE = 57559 +const UTC_TIME = 57560 +const UTC_TIMESTAMP = 57561 +const REPLACE = 57562 +const CONVERT = 57563 +const CAST = 57564 +const SUBSTR = 57565 +const SUBSTRING = 57566 +const GROUP_CONCAT = 57567 +const SEPARATOR = 57568 +const MATCH = 57569 +const AGAINST = 57570 +const BOOLEAN = 57571 +const LANGUAGE = 57572 +const WITH = 57573 +const QUERY = 57574 +const EXPANSION = 57575 +const UNUSED = 57576 var yyToknames = [...]string{ "$end", @@ -544,7 +543,6 @@ var yyToknames = [...]string{ "VITESS_SHARDS", "VITESS_TABLETS", "VSCHEMA_TABLES", - "EXTENDED", "FULL", "PROCESSLIST", "COLUMNS", @@ -602,316 +600,318 @@ var yyExca = [...]int{ 5, 27, -2, 4, -1, 36, - 151, 265, - 152, 265, + 151, 263, + 152, 263, -2, 255, -1, 240, - 110, 590, - -2, 586, + 110, 588, + -2, 584, -1, 241, - 110, 591, - -2, 587, + 110, 589, + -2, 585, -1, 311, - 80, 750, + 80, 748, -2, 58, -1, 312, - 80, 711, + 80, 709, -2, 59, -1, 317, - 80, 694, - -2, 552, + 80, 692, + -2, 550, -1, 319, - 80, 732, - -2, 554, - -1, 579, + 80, 730, + -2, 552, + -1, 581, 52, 41, 54, 41, -2, 43, - -1, 713, - 110, 593, - -2, 589, - -1, 917, + -1, 715, + 110, 591, + -2, 587, + -1, 922, 5, 28, - -2, 397, - -1, 942, + -2, 395, + -1, 947, 5, 27, - -2, 527, - -1, 1166, + -2, 525, + -1, 1168, 5, 28, - -2, 528, - -1, 1210, + -2, 526, + -1, 1212, 5, 27, - -2, 530, - -1, 1272, + -2, 528, + -1, 1274, 5, 28, - -2, 531, + -2, 529, } const yyPrivate = 57344 -const yyLast = 11150 +const yyLast = 11133 var yyAct = [...]int{ - 272, 47, 1263, 650, 775, 859, 526, 245, 1074, 1172, - 271, 1102, 793, 1075, 811, 1003, 1221, 1071, 815, 247, - 213, 219, 776, 571, 573, 839, 853, 814, 961, 738, - 945, 748, 53, 1048, 909, 745, 1006, 316, 589, 825, - 994, 950, 764, 458, 715, 525, 3, 849, 47, 464, - 413, 772, 560, 575, 588, 298, 224, 310, 470, 478, - 891, 228, 302, 307, 214, 215, 216, 217, 243, 305, - 540, 52, 1292, 876, 297, 1282, 1290, 1270, 1288, 860, - 1281, 301, 1066, 1160, 417, 296, 1230, 875, 445, 438, - 590, 232, 591, 218, 1245, 491, 490, 500, 501, 493, - 494, 495, 496, 497, 498, 499, 492, 1269, 1096, 502, - 57, 1097, 1098, 806, 880, 182, 178, 179, 180, 807, - 808, 679, 453, 874, 985, 1108, 1109, 1110, 680, 681, - 832, 1184, 840, 1113, 1111, 59, 60, 61, 62, 63, - 969, 1149, 426, 968, 1147, 1199, 970, 212, 449, 450, - 1264, 1289, 313, 440, 1287, 442, 1027, 773, 1222, 794, - 796, 1228, 427, 420, 176, 175, 747, 176, 658, 649, - 960, 1224, 871, 868, 869, 827, 867, 959, 958, 827, - 439, 441, 415, 423, 190, 177, 241, 514, 515, 1250, - 1169, 1035, 925, 903, 687, 444, 444, 444, 444, 684, - 444, 878, 881, 979, 482, 433, 1024, 492, 444, 812, - 502, 1117, 1026, 502, 722, 476, 475, 76, 477, 1255, - 886, 187, 1070, 1127, 187, 47, 181, 196, 720, 721, - 719, 475, 477, 948, 795, 1049, 873, 1031, 592, 1223, - 511, 1068, 765, 513, 932, 765, 653, 477, 187, 187, - 76, 207, 829, 1246, 187, 466, 76, 830, 872, 1229, - 1227, 1118, 512, 437, 983, 1258, 1051, 840, 826, 472, - 467, 524, 826, 528, 529, 530, 531, 532, 533, 534, - 535, 536, 833, 539, 541, 541, 541, 541, 541, 541, - 541, 541, 549, 550, 551, 552, 1112, 877, 1053, 887, - 1057, 191, 1052, 572, 1050, 1025, 1268, 1023, 193, 1055, - 879, 476, 475, 1030, 1274, 199, 195, 419, 1054, 174, - 922, 1190, 301, 429, 430, 431, 1189, 50, 477, 690, - 691, 1056, 1058, 705, 707, 708, 468, 718, 706, 900, - 901, 902, 197, 456, 998, 201, 495, 496, 497, 498, - 499, 492, 997, 986, 502, 542, 543, 544, 545, 546, - 547, 548, 187, 1014, 187, 739, 1275, 740, 476, 475, - 187, 1256, 586, 192, 580, 476, 475, 187, 1206, 686, - 295, 76, 76, 76, 76, 477, 76, 1187, 1135, 421, - 422, 1012, 477, 921, 76, 920, 995, 313, 1278, 457, - 202, 194, 1253, 203, 204, 205, 206, 210, 1214, 1261, - 457, 476, 475, 209, 208, 685, 1105, 444, 1104, 827, - 980, 76, 1214, 457, 1234, 444, 1214, 1215, 477, 1181, - 1180, 476, 475, 1093, 457, 1233, 444, 444, 444, 444, - 444, 444, 444, 444, 1168, 457, 23, 414, 477, 971, - 444, 444, 1124, 1123, 1120, 1121, 1013, 1120, 1119, 915, - 457, 1018, 1015, 1008, 1009, 1016, 1011, 1010, 862, 741, - 940, 667, 664, 941, 557, 457, 750, 457, 1017, 663, - 654, 652, 187, 647, 1020, 270, 599, 598, 692, 187, - 187, 187, 583, 50, 665, 76, 716, 435, 428, 414, - 21, 76, 1114, 1072, 947, 717, 946, 23, 1038, 54, - 947, 800, 826, 582, 47, 946, 74, 824, 822, 23, - 713, 823, 261, 260, 263, 264, 265, 266, 750, 528, - 694, 262, 267, 584, 1209, 582, 1164, 557, 1126, 709, - 1122, 757, 760, 711, 556, 972, 557, 766, 915, 315, - 805, 915, 946, 927, 50, 418, 223, 915, 302, 302, - 302, 302, 302, 924, 777, 585, 50, 1107, 557, 688, - 50, 712, 225, 572, 752, 797, 1194, 301, 301, 301, - 301, 301, 302, 742, 743, 834, 769, 562, 565, 566, - 567, 563, 301, 564, 568, 762, 926, 951, 952, 1072, - 854, 301, 1087, 76, 975, 801, 923, 951, 952, 187, - 187, 76, 850, 187, 779, 780, 187, 782, 752, 50, - 187, 790, 76, 76, 76, 76, 76, 76, 76, 76, - 798, 799, 778, 845, 844, 781, 76, 76, 841, 842, - 843, 187, 804, 803, 65, 562, 565, 566, 567, 563, - 819, 564, 568, 444, 651, 444, 76, 857, 999, 954, - 187, 661, 454, 444, 787, 785, 76, 700, 957, 788, - 786, 956, 784, 855, 313, 789, 783, 566, 567, 1286, - 315, 315, 315, 315, 1280, 315, 1034, 816, 888, 1014, - 471, 851, 852, 315, 229, 230, 753, 754, 1285, 898, - 897, 990, 761, 459, 469, 597, 436, 982, 1260, 1259, - 76, 1207, 1162, 904, 976, 460, 768, 1012, 770, 771, - 480, 1195, 864, 660, 570, 713, 226, 227, 471, 896, - 716, 220, 1239, 221, 54, 1238, 893, 895, 892, 717, - 1197, 187, 947, 473, 187, 187, 187, 187, 187, 1247, - 1185, 683, 56, 58, 581, 51, 187, 1, 861, 187, - 1002, 870, 1262, 187, 905, 1220, 1101, 821, 187, 187, - 813, 412, 76, 943, 944, 64, 712, 234, 1254, 820, - 1226, 1183, 1013, 828, 984, 76, 831, 1018, 1015, 1008, - 1009, 1016, 1011, 1010, 315, 1106, 1257, 981, 604, 602, - 594, 302, 603, 601, 1017, 606, 931, 605, 600, 198, - 1007, 693, 308, 569, 593, 856, 474, 66, 942, 1022, - 301, 1021, 866, 955, 964, 1029, 678, 885, 452, 200, - 963, 510, 965, 973, 894, 967, 187, 314, 1079, 76, - 689, 76, 463, 1237, 966, 187, 1196, 930, 187, 76, - 537, 763, 246, 704, 989, 259, 991, 992, 993, 444, - 256, 987, 988, 977, 978, 258, 899, 257, 695, 749, - 751, 493, 494, 495, 496, 497, 498, 499, 492, 443, - 939, 502, 484, 244, 444, 767, 996, 1005, 236, 300, - 835, 836, 837, 838, 553, 561, 559, 558, 953, 949, - 299, 1037, 315, 816, 1159, 1019, 846, 847, 848, 1244, - 315, 699, 25, 914, 55, 792, 231, 19, 18, 17, - 20, 315, 315, 315, 315, 315, 315, 315, 315, 929, - 16, 15, 14, 29, 13, 315, 315, 12, 11, 10, - 1041, 1042, 1077, 9, 47, 1073, 8, 1047, 777, 1004, - 7, 1059, 1076, 6, 777, 696, 1060, 5, 4, 1089, - 1090, 1091, 1067, 713, 222, 480, 22, 2, 315, 1083, - 0, 0, 0, 1081, 0, 0, 0, 0, 1082, 0, - 238, 0, 0, 0, 76, 0, 0, 187, 0, 1078, - 0, 0, 1040, 1100, 0, 1095, 1094, 0, 461, 465, - 1099, 76, 1115, 1116, 0, 0, 0, 0, 0, 744, - 0, 0, 0, 0, 1063, 483, 0, 0, 0, 758, - 758, 0, 0, 0, 0, 758, 0, 1128, 490, 500, - 501, 493, 494, 495, 496, 497, 498, 499, 492, 302, - 1130, 502, 758, 1133, 76, 76, 0, 76, 0, 0, - 527, 0, 0, 0, 0, 0, 0, 0, 301, 538, - 0, 816, 1138, 816, 0, 0, 0, 1158, 1137, 0, - 76, 315, 0, 187, 187, 446, 447, 448, 0, 451, - 1145, 187, 0, 0, 315, 912, 0, 455, 0, 913, - 76, 1163, 0, 0, 0, 0, 917, 918, 919, 1174, - 1175, 1176, 0, 0, 1171, 928, 0, 0, 0, 0, - 934, 0, 935, 936, 937, 938, 1179, 1177, 973, 0, - 0, 0, 0, 0, 1040, 0, 444, 0, 0, 0, - 76, 76, 0, 0, 0, 0, 0, 0, 315, 0, - 315, 0, 1192, 0, 0, 1186, 1193, 1188, 315, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 76, - 0, 76, 76, 0, 0, 1077, 0, 0, 1211, 1198, - 0, 0, 0, 315, 0, 1076, 0, 0, 1208, 0, - 0, 0, 0, 0, 0, 0, 187, 0, 816, 0, - 1219, 0, 0, 1225, 76, 0, 1236, 0, 0, 0, - 0, 1231, 0, 1232, 1235, 0, 0, 76, 187, 0, - 1077, 0, 47, 1210, 76, 1004, 816, 0, 1248, 0, - 1076, 0, 76, 0, 0, 187, 0, 0, 0, 516, - 517, 518, 519, 520, 521, 522, 523, 1252, 0, 0, - 0, 0, 0, 0, 0, 0, 1266, 0, 0, 0, - 0, 0, 1271, 702, 703, 777, 0, 1249, 1046, 0, - 0, 0, 0, 462, 0, 1276, 0, 0, 0, 0, - 0, 0, 0, 0, 76, 0, 76, 76, 76, 187, - 76, 1284, 1283, 962, 0, 0, 76, 0, 0, 0, - 1291, 0, 0, 0, 0, 0, 648, 0, 185, 0, - 315, 211, 0, 0, 657, 1092, 527, 0, 0, 755, - 756, 0, 76, 76, 76, 668, 669, 670, 671, 672, - 673, 674, 675, 235, 0, 185, 185, 0, 0, 676, - 677, 185, 1142, 1143, 0, 1144, 0, 0, 1146, 0, - 1148, 0, 0, 1000, 315, 0, 315, 0, 0, 0, - 0, 0, 0, 0, 0, 76, 76, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 76, 315, - 810, 500, 501, 493, 494, 495, 496, 497, 498, 499, - 492, 76, 0, 502, 0, 1139, 1182, 0, 0, 315, - 0, 0, 1141, 0, 0, 0, 0, 0, 0, 0, - 0, 76, 0, 1150, 1151, 1152, 0, 0, 1155, 0, - 0, 315, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1165, 1166, 1167, 0, 1170, 758, 0, 0, 1080, - 962, 0, 758, 0, 0, 0, 0, 76, 0, 185, - 0, 185, 0, 0, 0, 0, 0, 185, 0, 0, - 0, 76, 0, 0, 185, 0, 0, 0, 315, 0, - 315, 1103, 0, 889, 890, 714, 465, 0, 723, 724, + 272, 47, 1265, 861, 652, 528, 1077, 1223, 1105, 795, + 245, 271, 527, 3, 1174, 841, 1078, 777, 219, 817, + 1074, 1008, 950, 575, 888, 816, 778, 966, 213, 316, + 813, 855, 740, 750, 914, 747, 573, 681, 1011, 1051, + 827, 999, 591, 955, 460, 53, 766, 466, 47, 717, + 413, 590, 774, 562, 310, 472, 224, 577, 480, 896, + 218, 851, 302, 243, 228, 298, 307, 305, 52, 1294, + 1284, 1292, 214, 215, 216, 217, 878, 1272, 835, 1290, + 862, 1283, 1069, 1162, 1271, 247, 417, 1232, 1099, 438, + 877, 297, 542, 232, 1100, 1101, 809, 810, 296, 1247, + 493, 492, 502, 503, 495, 496, 497, 498, 499, 500, + 501, 494, 808, 990, 504, 57, 834, 882, 182, 178, + 179, 180, 974, 1186, 1201, 973, 876, 842, 975, 749, + 1111, 1112, 1113, 592, 1151, 593, 426, 1149, 1116, 1114, + 59, 60, 61, 62, 63, 454, 212, 301, 449, 450, + 1291, 455, 452, 440, 1224, 442, 1289, 1266, 1032, 775, + 427, 829, 420, 175, 176, 176, 1230, 1226, 241, 796, + 798, 660, 651, 829, 829, 873, 870, 871, 965, 869, + 439, 441, 814, 964, 963, 415, 423, 190, 1252, 414, + 177, 516, 517, 1171, 1038, 444, 444, 444, 444, 76, + 444, 984, 930, 187, 880, 883, 187, 908, 444, 493, + 492, 502, 503, 495, 496, 497, 498, 499, 500, 501, + 494, 1029, 689, 504, 484, 47, 433, 1031, 181, 504, + 187, 187, 76, 686, 682, 1225, 187, 469, 76, 875, + 513, 1052, 494, 515, 797, 504, 842, 497, 498, 499, + 500, 501, 494, 479, 828, 504, 915, 1248, 1257, 826, + 824, 874, 437, 825, 1231, 1229, 828, 828, 468, 890, + 1130, 526, 1054, 530, 531, 532, 533, 534, 535, 536, + 537, 538, 1270, 541, 543, 543, 543, 543, 543, 543, + 543, 543, 551, 552, 553, 554, 953, 1019, 1120, 879, + 1115, 594, 655, 574, 1056, 1071, 1060, 477, 1055, 767, + 1053, 937, 881, 683, 767, 1058, 831, 429, 430, 431, + 1030, 832, 1028, 479, 1057, 1017, 988, 1260, 514, 495, + 496, 497, 498, 499, 500, 501, 494, 1059, 1061, 504, + 724, 470, 474, 927, 187, 889, 187, 1276, 1121, 419, + 174, 926, 187, 925, 722, 723, 721, 688, 1277, 187, + 692, 693, 1192, 76, 76, 76, 76, 1191, 76, 478, + 477, 741, 582, 742, 1258, 588, 76, 544, 545, 546, + 547, 548, 549, 550, 50, 1003, 479, 1002, 301, 991, + 1018, 478, 477, 687, 720, 1023, 1020, 1013, 1014, 1021, + 1016, 1015, 1208, 76, 1019, 1189, 478, 477, 479, 478, + 477, 295, 1022, 478, 477, 1255, 1035, 444, 1025, 1000, + 1073, 421, 422, 479, 1108, 444, 479, 478, 477, 1107, + 479, 985, 1017, 905, 906, 907, 444, 444, 444, 444, + 444, 444, 444, 444, 479, 1280, 459, 707, 709, 710, + 444, 444, 708, 1216, 1263, 1216, 459, 1216, 1217, 459, + 458, 1183, 1182, 669, 187, 261, 260, 263, 264, 265, + 266, 187, 187, 187, 262, 267, 976, 76, 564, 567, + 568, 569, 565, 76, 566, 570, 864, 694, 956, 957, + 1096, 459, 21, 667, 1170, 459, 1236, 1018, 743, 718, + 666, 270, 1023, 1020, 1013, 1014, 1021, 1016, 1015, 1127, + 1126, 1123, 1124, 1235, 715, 665, 47, 1123, 1122, 1022, + 920, 459, 559, 459, 1117, 1012, 656, 23, 752, 459, + 952, 530, 74, 654, 649, 696, 601, 600, 951, 435, + 711, 428, 414, 754, 713, 752, 759, 762, 223, 23, + 1075, 945, 768, 951, 946, 1166, 585, 1041, 558, 952, + 302, 302, 302, 302, 302, 315, 802, 23, 584, 779, + 443, 418, 559, 719, 50, 574, 1211, 799, 54, 932, + 744, 745, 559, 559, 302, 76, 1129, 754, 771, 445, + 929, 187, 187, 76, 764, 187, 50, 586, 187, 584, + 920, 951, 187, 1125, 76, 76, 76, 76, 76, 76, + 76, 76, 977, 807, 50, 1110, 920, 920, 76, 76, + 803, 587, 931, 690, 792, 187, 781, 782, 50, 784, + 843, 844, 845, 928, 801, 1196, 800, 836, 225, 856, + 76, 806, 805, 1090, 187, 301, 301, 301, 301, 301, + 76, 780, 821, 313, 783, 444, 980, 444, 956, 957, + 301, 755, 756, 852, 847, 444, 846, 763, 65, 301, + 502, 503, 495, 496, 497, 498, 499, 500, 501, 494, + 857, 770, 504, 772, 773, 50, 653, 859, 837, 838, + 839, 840, 1075, 1004, 76, 959, 315, 315, 315, 315, + 663, 315, 456, 702, 848, 849, 850, 853, 854, 315, + 564, 567, 568, 569, 565, 909, 566, 570, 789, 715, + 962, 787, 961, 790, 786, 187, 788, 785, 187, 187, + 187, 187, 187, 718, 229, 230, 482, 1288, 897, 898, + 187, 1282, 791, 187, 568, 569, 1037, 187, 893, 473, + 1287, 903, 187, 187, 902, 995, 76, 599, 1262, 436, + 987, 461, 1261, 471, 1209, 981, 446, 447, 448, 76, + 451, 910, 488, 462, 491, 948, 949, 1164, 457, 234, + 505, 506, 507, 508, 509, 510, 511, 947, 489, 490, + 487, 493, 492, 502, 503, 495, 496, 497, 498, 499, + 500, 501, 494, 302, 1197, 504, 866, 719, 662, 572, + 315, 473, 936, 226, 227, 220, 596, 1241, 221, 54, + 187, 1240, 1199, 76, 968, 76, 970, 960, 952, 187, + 238, 904, 187, 76, 901, 680, 978, 475, 1249, 969, + 1187, 685, 900, 56, 58, 583, 51, 1, 863, 187, + 971, 187, 187, 992, 993, 1007, 872, 1264, 1222, 1104, + 823, 444, 815, 412, 64, 1256, 982, 983, 822, 1228, + 1185, 830, 994, 989, 996, 997, 998, 833, 919, 1109, + 1259, 986, 606, 604, 605, 603, 444, 608, 301, 1001, + 607, 602, 198, 308, 934, 571, 595, 1010, 313, 858, + 476, 66, 1027, 1026, 868, 453, 200, 512, 899, 1024, + 972, 314, 1082, 691, 465, 1239, 1198, 935, 315, 539, + 765, 246, 706, 259, 256, 1034, 315, 258, 257, 697, + 695, 944, 486, 244, 236, 300, 555, 315, 315, 315, + 315, 315, 315, 315, 315, 1044, 1045, 1080, 563, 47, + 561, 315, 315, 560, 1050, 958, 779, 954, 1079, 1062, + 715, 1081, 779, 1076, 1092, 1093, 1094, 1063, 76, 299, + 1040, 187, 1161, 698, 1246, 701, 25, 1084, 1086, 55, + 1070, 231, 19, 482, 18, 76, 315, 650, 751, 753, + 17, 20, 1098, 16, 15, 659, 1085, 14, 29, 1103, + 463, 467, 13, 1102, 769, 12, 670, 671, 672, 673, + 674, 675, 676, 677, 1097, 11, 10, 485, 9, 8, + 678, 679, 7, 1118, 1119, 6, 5, 746, 76, 76, + 4, 76, 222, 22, 794, 2, 0, 760, 760, 0, + 1131, 0, 302, 760, 0, 0, 0, 0, 0, 0, + 0, 0, 529, 1133, 76, 0, 1136, 0, 0, 1138, + 760, 540, 0, 1139, 0, 0, 0, 0, 187, 0, + 1160, 0, 0, 0, 714, 0, 0, 76, 1140, 518, + 519, 520, 521, 522, 523, 524, 525, 0, 1147, 315, + 0, 0, 0, 0, 0, 0, 0, 1165, 0, 0, + 0, 0, 315, 0, 1173, 0, 0, 1176, 1177, 1178, + 0, 0, 0, 0, 0, 0, 1181, 76, 76, 1179, + 0, 0, 978, 0, 0, 0, 0, 301, 0, 444, + 0, 1144, 1145, 0, 1146, 0, 0, 1148, 1188, 1150, + 1190, 0, 0, 0, 0, 0, 76, 0, 76, 76, + 1195, 1194, 0, 0, 0, 0, 315, 0, 315, 0, + 0, 1200, 0, 0, 0, 0, 315, 1080, 0, 0, + 1213, 0, 0, 187, 0, 0, 0, 313, 1079, 0, + 0, 76, 1212, 1210, 0, 1184, 0, 0, 0, 1221, + 818, 315, 1227, 0, 76, 187, 0, 0, 1238, 0, + 0, 76, 0, 0, 917, 0, 0, 76, 918, 0, + 187, 1237, 1080, 0, 47, 922, 923, 924, 1250, 1233, + 0, 1234, 0, 1079, 933, 865, 1251, 867, 0, 939, + 1254, 940, 941, 942, 943, 886, 492, 502, 503, 495, + 496, 497, 498, 499, 500, 501, 494, 1268, 0, 504, + 0, 0, 0, 0, 0, 0, 0, 704, 705, 76, + 779, 76, 76, 76, 187, 76, 0, 1273, 1278, 0, + 0, 76, 0, 0, 0, 0, 0, 0, 0, 714, + 0, 0, 1285, 1286, 0, 0, 0, 0, 0, 0, + 0, 0, 1293, 0, 0, 0, 0, 76, 76, 76, + 0, 967, 0, 0, 0, 0, 0, 0, 0, 0, + 529, 0, 0, 757, 758, 0, 0, 716, 315, 0, 725, 726, 727, 728, 729, 730, 731, 732, 733, 734, - 735, 736, 737, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1129, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1205, 0, 0, 1131, 0, 0, 0, - 0, 0, 0, 1134, 0, 0, 0, 0, 1216, 1217, - 1218, 315, 0, 0, 0, 0, 0, 0, 916, 0, - 0, 0, 863, 0, 865, 0, 0, 0, 0, 0, - 0, 0, 884, 933, 1240, 1241, 1242, 1243, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 185, - 0, 0, 0, 303, 0, 0, 185, 577, 185, 0, - 0, 0, 0, 1173, 0, 1173, 1173, 1173, 0, 1178, - 0, 0, 0, 0, 0, 315, 0, 0, 1267, 0, - 0, 0, 0, 1272, 0, 0, 0, 0, 184, 0, - 0, 0, 1156, 457, 0, 0, 1277, 0, 0, 0, - 0, 315, 315, 315, 491, 490, 500, 501, 493, 494, - 495, 496, 497, 498, 499, 492, 306, 0, 502, 0, - 0, 416, 457, 0, 0, 0, 0, 1295, 1296, 491, - 490, 500, 501, 493, 494, 495, 496, 497, 498, 499, - 492, 0, 0, 502, 1212, 1213, 0, 0, 0, 0, - 0, 910, 0, 0, 0, 0, 0, 1103, 491, 490, - 500, 501, 493, 494, 495, 496, 497, 498, 499, 492, - 1173, 0, 502, 0, 0, 0, 185, 185, 0, 0, - 185, 0, 0, 185, 0, 0, 0, 666, 0, 0, - 1251, 906, 907, 908, 0, 0, 0, 0, 0, 0, - 1069, 0, 0, 0, 0, 0, 0, 0, 185, 0, - 0, 0, 0, 0, 0, 1084, 1085, 0, 0, 1086, - 0, 0, 1088, 758, 0, 0, 1273, 185, 1001, 424, - 0, 425, 0, 0, 0, 0, 666, 432, 0, 0, - 1279, 0, 0, 0, 434, 0, 486, 0, 489, 0, - 0, 0, 0, 1028, 503, 504, 505, 506, 507, 508, - 509, 0, 487, 488, 485, 491, 490, 500, 501, 493, - 494, 495, 496, 497, 498, 499, 492, 0, 235, 502, - 0, 0, 0, 235, 235, 0, 0, 759, 759, 235, - 0, 0, 0, 759, 0, 0, 0, 0, 0, 1136, - 0, 0, 0, 235, 235, 235, 235, 0, 185, 0, - 759, 185, 185, 185, 185, 185, 0, 0, 0, 0, - 0, 0, 0, 791, 0, 0, 185, 0, 1153, 457, - 577, 0, 0, 0, 0, 185, 185, 0, 1161, 0, - 23, 24, 48, 26, 27, 527, 1157, 0, 0, 555, - 0, 0, 0, 0, 0, 0, 0, 0, 579, 42, - 0, 0, 0, 0, 28, 491, 490, 500, 501, 493, - 494, 495, 496, 497, 498, 499, 492, 0, 0, 502, - 0, 1044, 1045, 37, 0, 0, 0, 50, 0, 0, - 0, 0, 0, 0, 1061, 1062, 0, 1064, 1065, 0, - 0, 0, 0, 185, 0, 0, 0, 0, 0, 0, - 0, 0, 185, 0, 0, 185, 0, 0, 491, 490, - 500, 501, 493, 494, 495, 496, 497, 498, 499, 492, - 0, 0, 502, 0, 0, 0, 0, 621, 0, 0, - 0, 666, 0, 0, 0, 0, 0, 30, 31, 33, - 32, 35, 0, 235, 0, 0, 1154, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 36, 43, - 44, 0, 0, 45, 46, 34, 655, 656, 0, 0, - 659, 0, 0, 662, 0, 0, 0, 38, 39, 0, - 40, 41, 0, 0, 0, 1191, 0, 0, 0, 0, - 235, 0, 0, 0, 0, 0, 1265, 527, 682, 0, - 0, 0, 0, 609, 1140, 0, 235, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 701, 491, 490, - 500, 501, 493, 494, 495, 496, 497, 498, 499, 492, - 0, 0, 502, 0, 622, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 185, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 635, 636, 637, 638, 639, - 640, 641, 49, 642, 643, 644, 645, 646, 623, 624, - 625, 626, 607, 608, 0, 0, 610, 0, 611, 612, - 613, 614, 615, 616, 617, 618, 619, 620, 627, 628, - 629, 630, 631, 632, 633, 634, 0, 0, 774, 491, - 490, 500, 501, 493, 494, 495, 496, 497, 498, 499, - 492, 0, 0, 502, 1200, 1201, 0, 1202, 1203, 1204, - 0, 0, 0, 0, 0, 0, 802, 0, 0, 0, - 1032, 1033, 0, 0, 0, 0, 0, 0, 185, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 235, 0, - 0, 1043, 0, 0, 0, 0, 0, 0, 0, 235, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 666, - 0, 491, 490, 500, 501, 493, 494, 495, 496, 497, - 498, 499, 492, 0, 759, 502, 911, 0, 0, 0, - 759, 0, 0, 858, 0, 0, 0, 0, 0, 0, - 0, 0, 882, 0, 0, 883, 491, 490, 500, 501, - 493, 494, 495, 496, 497, 498, 499, 492, 0, 0, - 502, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 185, 0, 0, 0, 0, 0, 0, - 0, 0, 1293, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 185, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 185, 0, 0, 0, 0, 0, 0, 0, + 735, 736, 737, 738, 739, 0, 0, 0, 0, 76, + 76, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 76, 0, 0, 464, 0, 0, 0, 0, + 0, 1005, 315, 0, 315, 76, 0, 0, 0, 0, + 0, 0, 0, 0, 812, 0, 0, 0, 0, 0, + 1049, 0, 0, 0, 0, 76, 0, 315, 0, 0, + 185, 0, 0, 211, 0, 303, 0, 0, 0, 0, + 0, 0, 0, 0, 196, 0, 818, 0, 0, 0, + 315, 0, 0, 0, 0, 235, 0, 185, 185, 0, + 0, 76, 0, 185, 0, 0, 0, 1095, 207, 0, + 184, 1006, 315, 0, 0, 76, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 760, 0, 0, + 1083, 967, 1009, 760, 0, 0, 1033, 0, 306, 0, + 0, 0, 0, 416, 0, 0, 0, 894, 895, 0, + 467, 0, 0, 0, 0, 0, 0, 0, 191, 315, + 0, 315, 1106, 0, 0, 193, 0, 0, 0, 0, + 0, 0, 199, 195, 0, 0, 0, 0, 1043, 0, + 0, 0, 0, 0, 0, 1141, 0, 0, 0, 0, + 0, 0, 1143, 0, 1132, 0, 0, 0, 0, 197, + 1066, 0, 201, 1152, 1153, 1154, 0, 1134, 1157, 0, + 0, 185, 921, 185, 1137, 0, 0, 0, 0, 185, + 315, 1167, 1168, 1169, 0, 1172, 185, 938, 0, 0, + 192, 0, 0, 911, 912, 913, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 818, 0, 818, + 459, 424, 0, 425, 0, 0, 0, 202, 194, 432, + 203, 204, 205, 206, 210, 0, 434, 0, 0, 209, + 208, 0, 1175, 0, 1175, 1175, 1175, 0, 1180, 0, + 0, 0, 0, 0, 315, 0, 493, 492, 502, 503, + 495, 496, 497, 498, 499, 500, 501, 494, 0, 0, + 504, 0, 1207, 0, 0, 0, 0, 0, 1043, 0, + 315, 315, 315, 0, 0, 0, 0, 1218, 1219, 1220, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 185, 0, 0, 0, 0, 0, 0, 185, 579, + 185, 0, 0, 1242, 1243, 1244, 1245, 0, 0, 0, + 1036, 0, 1214, 1215, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1106, 0, 0, 0, 0, + 0, 557, 818, 0, 0, 0, 0, 0, 1175, 1193, + 581, 0, 0, 0, 0, 0, 0, 1269, 0, 0, + 0, 0, 1274, 0, 0, 0, 0, 1072, 1253, 1009, + 818, 1159, 0, 0, 0, 1279, 0, 0, 0, 0, + 0, 0, 1087, 1088, 0, 0, 1089, 0, 0, 1091, + 0, 0, 0, 0, 0, 0, 1047, 1048, 0, 0, + 0, 760, 0, 0, 1275, 0, 1297, 1298, 0, 1064, + 1065, 0, 1067, 1068, 0, 0, 0, 0, 1281, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 185, 185, + 0, 0, 185, 0, 0, 185, 0, 0, 0, 668, + 0, 0, 0, 493, 492, 502, 503, 495, 496, 497, + 498, 499, 500, 501, 494, 0, 0, 504, 0, 0, + 0, 0, 185, 0, 0, 0, 0, 0, 657, 658, + 0, 0, 661, 0, 0, 664, 0, 0, 0, 0, + 0, 185, 0, 0, 0, 0, 0, 0, 0, 0, + 668, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 684, 1163, 0, 0, 0, 0, 0, 0, + 529, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 703, 0, 0, 0, 0, 0, 1142, 0, 0, + 0, 0, 235, 0, 0, 0, 0, 235, 235, 0, + 0, 761, 761, 235, 0, 0, 0, 761, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 235, 235, 235, + 235, 0, 185, 0, 761, 185, 185, 185, 185, 185, + 0, 0, 0, 0, 1158, 459, 0, 793, 0, 0, + 185, 0, 0, 0, 579, 0, 0, 0, 0, 185, + 185, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1156, 776, 0, 0, 23, 24, 48, 26, 27, + 0, 493, 492, 502, 503, 495, 496, 497, 498, 499, + 500, 501, 494, 0, 42, 504, 0, 0, 0, 28, + 804, 0, 0, 0, 0, 0, 1202, 1203, 0, 1204, + 1205, 1206, 0, 0, 0, 0, 0, 0, 37, 0, + 0, 0, 50, 0, 0, 0, 0, 185, 0, 0, + 623, 0, 0, 0, 0, 0, 185, 0, 0, 185, + 1267, 529, 0, 493, 492, 502, 503, 495, 496, 497, + 498, 499, 500, 501, 494, 0, 185, 504, 891, 892, + 0, 0, 0, 0, 0, 668, 0, 860, 0, 0, + 0, 0, 0, 0, 0, 0, 884, 235, 0, 885, + 0, 0, 30, 31, 33, 32, 35, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 887, 0, 0, 0, + 0, 0, 0, 36, 43, 44, 611, 0, 45, 46, + 34, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 38, 39, 235, 40, 41, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 624, 0, 0, + 235, 0, 0, 0, 1295, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 637, 638, + 639, 640, 641, 642, 643, 0, 644, 645, 646, 647, + 648, 625, 626, 627, 628, 609, 610, 0, 185, 612, + 0, 613, 614, 615, 616, 617, 618, 619, 620, 621, + 622, 629, 630, 631, 632, 633, 634, 635, 636, 1155, + 459, 1046, 0, 0, 0, 0, 49, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 577, 0, 0, 0, + 0, 493, 492, 502, 503, 495, 496, 497, 498, 499, + 500, 501, 494, 916, 0, 504, 493, 492, 502, 503, + 495, 496, 497, 498, 499, 500, 501, 494, 0, 0, + 504, 0, 0, 493, 492, 502, 503, 495, 496, 497, + 498, 499, 500, 501, 494, 0, 0, 504, 0, 0, + 0, 0, 0, 0, 0, 185, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 235, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 235, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 668, 0, 0, 0, + 0, 0, 0, 0, 0, 1039, 0, 0, 0, 0, + 0, 761, 0, 0, 0, 0, 0, 761, 493, 492, + 502, 503, 495, 496, 497, 498, 499, 500, 501, 494, + 0, 0, 504, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 185, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 185, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 185, 0, 0, + 1128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1135, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 579, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1036, 0, 0, 401, 391, 0, 363, 403, 341, 355, 411, 356, 357, 384, 327, 371, 126, 353, 0, 344, 322, 350, 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, 378, 404, 354, 382, 405, 0, - 0, 759, 75, 0, 817, 818, 0, 0, 0, 0, + 0, 0, 75, 0, 819, 820, 0, 0, 0, 0, 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, - 325, 328, 410, 398, 347, 348, 974, 0, 0, 0, + 325, 328, 410, 398, 347, 348, 979, 0, 0, 0, 0, 0, 0, 366, 370, 386, 360, 0, 0, 0, 0, 0, 0, 0, 0, 345, 0, 377, 0, 0, - 0, 329, 326, 1125, 0, 364, 0, 0, 0, 331, + 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, - 359, 358, 402, 132, 0, 1132, 146, 99, 98, 107, + 359, 358, 402, 132, 0, 761, 146, 99, 98, 107, 394, 343, 351, 90, 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, @@ -920,56 +920,31 @@ var yyAct = [...]int{ 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, - 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, - 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, - 392, 375, 77, 0, 109, 170, 134, 96, 161, 401, - 391, 0, 363, 403, 341, 355, 411, 356, 357, 384, - 327, 371, 126, 353, 0, 344, 322, 350, 323, 342, - 365, 94, 368, 340, 393, 374, 108, 409, 110, 379, - 0, 143, 119, 0, 0, 367, 395, 369, 389, 362, - 385, 332, 378, 404, 354, 382, 405, 0, 0, 0, - 75, 0, 817, 818, 0, 0, 0, 0, 0, 87, - 0, 381, 400, 352, 383, 321, 380, 0, 325, 328, - 410, 398, 347, 348, 0, 0, 0, 0, 0, 0, - 0, 366, 370, 386, 360, 0, 0, 0, 0, 0, - 0, 0, 0, 345, 0, 377, 0, 0, 0, 329, - 326, 0, 0, 364, 0, 0, 0, 331, 0, 346, - 387, 0, 320, 390, 396, 361, 188, 399, 359, 358, - 402, 132, 0, 0, 146, 99, 98, 107, 394, 343, - 351, 90, 349, 138, 128, 158, 376, 129, 137, 111, - 150, 133, 157, 189, 165, 148, 164, 78, 147, 156, - 88, 140, 80, 154, 145, 117, 103, 104, 79, 0, - 136, 93, 97, 92, 125, 151, 152, 91, 172, 83, - 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, - 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, - 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, - 339, 397, 166, 167, 168, 169, 0, 0, 0, 0, - 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, - 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, - 77, 0, 109, 170, 134, 96, 161, 401, 391, 0, - 363, 403, 341, 355, 411, 356, 357, 384, 327, 371, - 126, 353, 0, 344, 322, 350, 323, 342, 365, 94, - 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, - 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, - 378, 404, 354, 382, 405, 50, 0, 0, 75, 0, - 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, - 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, - 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, - 370, 386, 360, 0, 0, 0, 0, 0, 0, 0, - 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, - 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, - 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, - 0, 0, 146, 99, 98, 107, 394, 343, 351, 90, - 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, - 166, 167, 168, 169, 0, 0, 0, 0, 123, 85, + 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, + 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, + 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, + 375, 77, 0, 109, 170, 134, 96, 161, 401, 391, + 0, 363, 403, 341, 355, 411, 356, 357, 384, 327, + 371, 126, 353, 0, 344, 322, 350, 323, 342, 365, + 94, 368, 340, 393, 374, 108, 409, 110, 379, 0, + 143, 119, 0, 0, 367, 395, 369, 389, 362, 385, + 332, 378, 404, 354, 382, 405, 0, 0, 0, 75, + 0, 819, 820, 0, 0, 0, 0, 0, 87, 0, + 381, 400, 352, 383, 321, 380, 0, 325, 328, 410, + 398, 347, 348, 0, 0, 0, 0, 0, 0, 0, + 366, 370, 386, 360, 0, 0, 0, 0, 0, 0, + 0, 0, 345, 0, 377, 0, 0, 0, 329, 326, + 0, 0, 364, 0, 0, 0, 331, 0, 346, 387, + 0, 320, 390, 396, 361, 188, 399, 359, 358, 402, + 132, 0, 0, 146, 99, 98, 107, 394, 343, 351, + 90, 349, 138, 128, 158, 376, 129, 137, 111, 150, + 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, + 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, + 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, + 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, + 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, + 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, + 397, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, 77, 0, @@ -978,11 +953,11 @@ var yyAct = [...]int{ 0, 344, 322, 350, 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, 378, 404, - 354, 382, 405, 0, 0, 0, 75, 0, 0, 0, + 354, 382, 405, 50, 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, 370, 386, - 360, 0, 0, 0, 0, 0, 0, 1039, 0, 345, + 360, 0, 0, 0, 0, 0, 0, 0, 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, 0, 0, @@ -994,56 +969,31 @@ var yyAct = [...]int{ 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, - 168, 169, 0, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, - 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, - 0, 336, 337, 0, 392, 375, 77, 0, 109, 170, - 134, 96, 161, 401, 391, 0, 363, 403, 341, 355, - 411, 356, 357, 384, 327, 371, 126, 353, 0, 344, - 322, 350, 323, 342, 365, 94, 368, 340, 393, 374, - 108, 409, 110, 379, 0, 143, 119, 0, 0, 367, - 395, 369, 389, 362, 385, 332, 378, 404, 354, 382, - 405, 0, 0, 0, 240, 0, 0, 0, 0, 0, - 0, 0, 0, 87, 0, 381, 400, 352, 383, 321, - 380, 0, 325, 328, 410, 398, 347, 348, 0, 0, - 0, 0, 0, 0, 0, 366, 370, 386, 360, 0, - 0, 0, 0, 0, 0, 710, 0, 345, 0, 377, - 0, 0, 0, 329, 326, 0, 0, 364, 0, 0, - 0, 331, 0, 346, 387, 0, 320, 390, 396, 361, - 188, 399, 359, 358, 402, 132, 0, 0, 146, 99, - 98, 107, 394, 343, 351, 90, 349, 138, 128, 158, - 376, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, - 144, 160, 173, 86, 339, 397, 166, 167, 168, 169, - 0, 0, 0, 0, 123, 85, 102, 141, 105, 112, - 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, - 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, - 337, 0, 392, 375, 77, 0, 109, 170, 134, 96, - 161, 401, 391, 0, 363, 403, 341, 355, 411, 356, - 357, 384, 327, 371, 126, 353, 0, 344, 322, 350, - 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, - 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, - 389, 362, 385, 332, 378, 404, 354, 382, 405, 0, - 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, - 325, 328, 410, 398, 347, 348, 0, 0, 0, 0, - 0, 0, 0, 366, 370, 386, 360, 0, 0, 0, - 0, 0, 0, 0, 0, 345, 0, 377, 0, 0, - 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, - 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, - 359, 358, 402, 132, 0, 0, 146, 99, 98, 107, - 394, 343, 351, 90, 349, 138, 128, 158, 376, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, - 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, + 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, + 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, + 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, + 336, 337, 0, 392, 375, 77, 0, 109, 170, 134, + 96, 161, 401, 391, 0, 363, 403, 341, 355, 411, + 356, 357, 384, 327, 371, 126, 353, 0, 344, 322, + 350, 323, 342, 365, 94, 368, 340, 393, 374, 108, + 409, 110, 379, 0, 143, 119, 0, 0, 367, 395, + 369, 389, 362, 385, 332, 378, 404, 354, 382, 405, + 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, + 0, 0, 87, 0, 381, 400, 352, 383, 321, 380, + 0, 325, 328, 410, 398, 347, 348, 0, 0, 0, + 0, 0, 0, 0, 366, 370, 386, 360, 0, 0, + 0, 0, 0, 0, 1042, 0, 345, 0, 377, 0, + 0, 0, 329, 326, 0, 0, 364, 0, 0, 0, + 331, 0, 346, 387, 0, 320, 390, 396, 361, 188, + 399, 359, 358, 402, 132, 0, 0, 146, 99, 98, + 107, 394, 343, 351, 90, 349, 138, 128, 158, 376, + 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, + 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, + 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, + 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, + 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, + 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, + 160, 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, @@ -1057,7 +1007,7 @@ var yyAct = [...]int{ 0, 381, 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, 370, 386, 360, 0, 0, 0, 0, 0, - 0, 0, 0, 345, 0, 377, 0, 0, 0, 329, + 0, 712, 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, 0, 0, 146, 99, 98, 107, 394, 343, @@ -1068,57 +1018,32 @@ var yyAct = [...]int{ 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, - 339, 397, 166, 167, 168, 169, 0, 0, 0, 0, - 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, - 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, - 77, 0, 109, 170, 134, 96, 161, 401, 391, 0, - 363, 403, 341, 355, 411, 356, 357, 384, 327, 371, - 126, 353, 0, 344, 322, 350, 323, 342, 365, 94, - 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, - 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, - 378, 404, 354, 382, 405, 0, 0, 0, 75, 0, - 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, - 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, - 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, - 370, 386, 360, 0, 0, 0, 0, 0, 0, 0, - 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, - 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, - 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, - 0, 0, 146, 99, 98, 107, 394, 343, 351, 90, - 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 318, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, - 166, 167, 168, 169, 0, 0, 0, 0, 319, 317, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, - 388, 330, 0, 336, 337, 0, 392, 375, 77, 0, - 109, 170, 134, 96, 161, 401, 391, 0, 363, 403, - 341, 355, 411, 356, 357, 384, 327, 371, 126, 353, - 0, 344, 322, 350, 323, 342, 365, 94, 368, 340, - 393, 374, 108, 409, 110, 379, 0, 143, 119, 0, - 0, 367, 395, 369, 389, 362, 385, 332, 378, 404, - 354, 382, 405, 0, 0, 0, 186, 0, 0, 0, - 0, 0, 0, 0, 0, 87, 0, 381, 400, 352, - 383, 321, 380, 0, 325, 328, 410, 398, 347, 348, - 0, 0, 0, 0, 0, 0, 0, 366, 370, 386, - 360, 0, 0, 0, 0, 0, 0, 0, 0, 345, - 0, 377, 0, 0, 0, 329, 326, 0, 0, 364, - 0, 0, 0, 331, 0, 346, 387, 0, 320, 390, - 396, 361, 188, 399, 359, 358, 402, 132, 0, 0, - 146, 99, 98, 107, 394, 343, 351, 90, 349, 138, - 128, 158, 376, 129, 137, 111, 150, 133, 157, 189, - 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, - 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, - 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, - 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, - 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, - 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, - 168, 169, 0, 0, 0, 0, 123, 85, 102, 141, + 339, 397, 166, 167, 168, 169, 0, 0, 0, 123, + 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, + 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, + 408, 388, 330, 0, 336, 337, 0, 392, 375, 77, + 0, 109, 170, 134, 96, 161, 401, 391, 0, 363, + 403, 341, 355, 411, 356, 357, 384, 327, 371, 126, + 353, 0, 344, 322, 350, 323, 342, 365, 94, 368, + 340, 393, 374, 108, 409, 110, 379, 0, 143, 119, + 0, 0, 367, 395, 369, 389, 362, 385, 332, 378, + 404, 354, 382, 405, 0, 0, 0, 75, 0, 0, + 0, 0, 0, 0, 0, 0, 87, 0, 381, 400, + 352, 383, 321, 380, 0, 325, 328, 410, 398, 347, + 348, 0, 0, 0, 0, 0, 0, 0, 366, 370, + 386, 360, 0, 0, 0, 0, 0, 0, 0, 0, + 345, 0, 377, 0, 0, 0, 329, 326, 0, 0, + 364, 0, 0, 0, 331, 0, 346, 387, 0, 320, + 390, 396, 361, 188, 399, 359, 358, 402, 132, 0, + 0, 146, 99, 98, 107, 394, 343, 351, 90, 349, + 138, 128, 158, 376, 129, 137, 111, 150, 133, 157, + 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, + 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, + 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, + 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, + 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, + 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, 77, 0, 109, 170, @@ -1127,7 +1052,7 @@ var yyAct = [...]int{ 322, 350, 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, 378, 404, 354, 382, - 405, 0, 0, 0, 75, 0, 0, 0, 0, 0, + 405, 0, 0, 0, 240, 0, 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, 370, 386, 360, 0, @@ -1137,13 +1062,87 @@ var yyAct = [...]int{ 188, 399, 359, 358, 402, 132, 0, 0, 146, 99, 98, 107, 394, 343, 351, 90, 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 587, 88, 140, 80, 154, 145, 117, + 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 318, 162, 124, 149, + 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, 168, 169, - 0, 0, 0, 0, 319, 317, 102, 141, 105, 112, + 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, + 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, + 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, + 0, 392, 375, 77, 0, 109, 170, 134, 96, 161, + 401, 391, 0, 363, 403, 341, 355, 411, 356, 357, + 384, 327, 371, 126, 353, 0, 344, 322, 350, 323, + 342, 365, 94, 368, 340, 393, 374, 108, 409, 110, + 379, 0, 143, 119, 0, 0, 367, 395, 369, 389, + 362, 385, 332, 378, 404, 354, 382, 405, 0, 0, + 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, + 87, 0, 381, 400, 352, 383, 321, 380, 0, 325, + 328, 410, 398, 347, 348, 0, 0, 0, 0, 0, + 0, 0, 366, 370, 386, 360, 0, 0, 0, 0, + 0, 0, 0, 0, 345, 0, 377, 0, 0, 0, + 329, 326, 0, 0, 364, 0, 0, 0, 331, 0, + 346, 387, 0, 320, 390, 396, 361, 188, 399, 359, + 358, 402, 132, 0, 0, 146, 99, 98, 107, 394, + 343, 351, 90, 349, 138, 128, 158, 376, 129, 137, + 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, + 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, + 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, + 83, 163, 82, 318, 162, 124, 149, 155, 118, 115, + 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, + 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, + 86, 339, 397, 166, 167, 168, 169, 0, 0, 0, + 319, 317, 102, 141, 105, 112, 135, 171, 127, 139, + 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, + 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, + 77, 0, 109, 170, 134, 96, 161, 401, 391, 0, + 363, 403, 341, 355, 411, 356, 357, 384, 327, 371, + 126, 353, 0, 344, 322, 350, 323, 342, 365, 94, + 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, + 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, + 378, 404, 354, 382, 405, 0, 0, 0, 186, 0, + 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, + 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, + 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, + 370, 386, 360, 0, 0, 0, 0, 0, 0, 0, + 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, + 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, + 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, + 0, 0, 146, 99, 98, 107, 394, 343, 351, 90, + 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, + 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, + 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, + 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, + 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, + 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, + 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, + 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, + 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, + 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, + 330, 0, 336, 337, 0, 392, 375, 77, 0, 109, + 170, 134, 96, 161, 401, 391, 0, 363, 403, 341, + 355, 411, 356, 357, 384, 327, 371, 126, 353, 0, + 344, 322, 350, 323, 342, 365, 94, 368, 340, 393, + 374, 108, 409, 110, 379, 0, 143, 119, 0, 0, + 367, 395, 369, 389, 362, 385, 332, 378, 404, 354, + 382, 405, 0, 0, 0, 75, 0, 0, 0, 0, + 0, 0, 0, 0, 87, 0, 381, 400, 352, 383, + 321, 380, 0, 325, 328, 410, 398, 347, 348, 0, + 0, 0, 0, 0, 0, 0, 366, 370, 386, 360, + 0, 0, 0, 0, 0, 0, 0, 0, 345, 0, + 377, 0, 0, 0, 329, 326, 0, 0, 364, 0, + 0, 0, 331, 0, 346, 387, 0, 320, 390, 396, + 361, 188, 399, 359, 358, 402, 132, 0, 0, 146, + 99, 98, 107, 394, 343, 351, 90, 349, 138, 128, + 158, 376, 129, 137, 111, 150, 133, 157, 189, 165, + 148, 164, 78, 147, 589, 88, 140, 80, 154, 145, + 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, + 151, 152, 91, 172, 83, 163, 82, 318, 162, 124, + 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, + 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, + 0, 144, 160, 173, 86, 339, 397, 166, 167, 168, + 169, 0, 0, 0, 319, 317, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, 77, 0, 109, 170, 134, 96, @@ -1168,110 +1167,39 @@ var yyAct = [...]int{ 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, - 0, 0, 319, 317, 312, 311, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, - 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, - 392, 375, 77, 0, 109, 170, 134, 96, 161, 126, - 0, 0, 746, 0, 242, 0, 0, 0, 94, 0, - 239, 0, 0, 108, 282, 110, 0, 0, 143, 119, - 0, 0, 0, 0, 273, 274, 0, 0, 0, 0, - 0, 0, 0, 0, 50, 0, 0, 240, 261, 260, - 263, 264, 265, 266, 0, 0, 87, 262, 267, 268, - 269, 0, 0, 237, 254, 0, 281, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 251, 252, 233, 0, - 0, 0, 293, 0, 253, 0, 0, 248, 249, 250, - 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 291, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, - 283, 292, 289, 290, 287, 288, 286, 285, 284, 294, - 275, 276, 277, 278, 280, 0, 279, 77, 0, 109, - 170, 134, 96, 161, 126, 0, 0, 0, 0, 242, - 0, 0, 0, 94, 0, 239, 0, 0, 108, 282, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 273, - 274, 0, 0, 0, 0, 0, 0, 0, 0, 50, - 0, 457, 240, 261, 260, 263, 264, 265, 266, 0, - 0, 87, 262, 267, 268, 269, 0, 0, 237, 254, - 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 319, 317, 312, 311, 105, 112, 135, 171, 127, + 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, + 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, + 375, 77, 0, 109, 170, 134, 96, 161, 126, 0, + 0, 748, 0, 242, 0, 0, 0, 94, 0, 239, + 0, 0, 108, 282, 110, 0, 0, 143, 119, 0, + 0, 0, 0, 273, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 50, 0, 0, 240, 261, 260, 263, + 264, 265, 266, 0, 0, 87, 262, 267, 268, 269, + 0, 0, 237, 254, 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 251, 252, 0, 0, 0, 0, 293, 0, 253, - 0, 0, 248, 249, 250, 255, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 291, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 283, 292, 289, 290, 287, - 288, 286, 285, 284, 294, 275, 276, 277, 278, 280, - 0, 279, 77, 0, 109, 170, 134, 96, 161, 126, - 0, 0, 0, 0, 242, 0, 0, 0, 94, 0, - 239, 0, 0, 108, 282, 110, 0, 0, 143, 119, - 0, 0, 0, 0, 273, 274, 0, 0, 0, 0, - 0, 0, 0, 0, 50, 0, 0, 240, 261, 260, - 263, 264, 265, 266, 0, 0, 87, 262, 267, 268, - 269, 0, 0, 237, 254, 0, 281, 0, 0, 0, + 0, 0, 0, 0, 0, 251, 252, 233, 0, 0, + 0, 293, 0, 253, 0, 0, 248, 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 251, 252, 233, 0, - 0, 0, 293, 0, 253, 0, 0, 248, 249, 250, - 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 291, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, - 283, 292, 289, 290, 287, 288, 286, 285, 284, 294, - 275, 276, 277, 278, 280, 0, 279, 77, 0, 109, - 170, 134, 96, 161, 126, 0, 0, 0, 0, 242, - 0, 0, 0, 94, 0, 239, 0, 0, 108, 282, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 273, - 274, 0, 0, 0, 0, 0, 0, 809, 0, 50, - 0, 0, 240, 261, 260, 263, 264, 265, 266, 0, - 0, 87, 262, 267, 268, 269, 0, 0, 237, 254, - 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 251, 252, 0, 0, 0, 0, 293, 0, 253, - 0, 0, 248, 249, 250, 255, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 291, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 283, 292, 289, 290, 287, - 288, 286, 285, 284, 294, 275, 276, 277, 278, 280, - 23, 279, 77, 0, 109, 170, 134, 96, 161, 0, - 0, 0, 126, 0, 0, 0, 0, 242, 0, 0, - 0, 94, 0, 239, 0, 0, 108, 282, 110, 0, - 0, 143, 119, 0, 0, 0, 0, 273, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 50, 0, 0, - 240, 261, 260, 263, 264, 265, 266, 0, 0, 87, - 262, 267, 268, 269, 0, 0, 237, 254, 0, 281, + 0, 0, 188, 0, 0, 291, 0, 132, 0, 0, + 146, 99, 98, 107, 0, 0, 0, 90, 0, 138, + 128, 158, 0, 129, 137, 111, 150, 133, 157, 189, + 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, + 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, + 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, + 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, + 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, + 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, + 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, + 112, 135, 171, 127, 139, 89, 159, 142, 283, 292, + 289, 290, 287, 288, 286, 285, 284, 294, 275, 276, + 277, 278, 280, 0, 279, 77, 0, 109, 170, 134, + 96, 161, 126, 0, 0, 0, 0, 242, 0, 0, + 0, 94, 0, 239, 0, 0, 108, 282, 110, 0, + 0, 143, 119, 0, 0, 0, 0, 273, 274, 0, + 0, 0, 0, 0, 0, 0, 0, 50, 0, 459, + 240, 261, 260, 263, 264, 265, 266, 0, 0, 87, + 262, 267, 268, 269, 0, 0, 237, 254, 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 251, 252, 0, 0, 0, 0, 293, 0, 253, 0, 0, @@ -1285,11 +1213,58 @@ var yyAct = [...]int{ 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, - 0, 0, 166, 167, 168, 169, 0, 0, 0, 0, - 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 283, 292, 289, 290, 287, 288, 286, - 285, 284, 294, 275, 276, 277, 278, 280, 0, 279, - 77, 0, 109, 170, 134, 96, 161, 126, 0, 0, + 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, + 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, + 159, 142, 283, 292, 289, 290, 287, 288, 286, 285, + 284, 294, 275, 276, 277, 278, 280, 0, 279, 77, + 0, 109, 170, 134, 96, 161, 126, 0, 0, 0, + 0, 242, 0, 0, 0, 94, 0, 239, 0, 0, + 108, 282, 110, 0, 0, 143, 119, 0, 0, 0, + 0, 273, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 50, 0, 0, 240, 261, 260, 263, 264, 265, + 266, 0, 0, 87, 262, 267, 268, 269, 0, 0, + 237, 254, 0, 281, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 251, 252, 233, 0, 0, 0, 293, + 0, 253, 0, 0, 248, 249, 250, 255, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 188, 0, 0, 291, 0, 132, 0, 0, 146, 99, + 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, + 0, 129, 137, 111, 150, 133, 157, 189, 165, 148, + 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, + 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, + 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, + 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, + 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, + 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, + 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, + 171, 127, 139, 89, 159, 142, 283, 292, 289, 290, + 287, 288, 286, 285, 284, 294, 275, 276, 277, 278, + 280, 0, 279, 77, 0, 109, 170, 134, 96, 161, + 126, 0, 0, 0, 0, 242, 0, 0, 0, 94, + 0, 239, 0, 0, 108, 282, 110, 0, 0, 143, + 119, 0, 0, 0, 0, 273, 274, 0, 0, 0, + 0, 0, 0, 811, 0, 50, 0, 0, 240, 261, + 260, 263, 264, 265, 266, 0, 0, 87, 262, 267, + 268, 269, 0, 0, 237, 254, 0, 281, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 251, 252, 0, + 0, 0, 0, 293, 0, 253, 0, 0, 248, 249, + 250, 255, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 188, 0, 0, 291, 0, 132, + 0, 0, 146, 99, 98, 107, 0, 0, 0, 90, + 0, 138, 128, 158, 0, 129, 137, 111, 150, 133, + 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, + 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, + 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, + 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, + 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, + 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, + 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, + 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, + 283, 292, 289, 290, 287, 288, 286, 285, 284, 294, + 275, 276, 277, 278, 280, 23, 279, 77, 0, 109, + 170, 134, 96, 161, 0, 0, 0, 126, 0, 0, 0, 0, 242, 0, 0, 0, 94, 0, 239, 0, 0, 108, 282, 110, 0, 0, 143, 119, 0, 0, 0, 0, 273, 274, 0, 0, 0, 0, 0, 0, @@ -1309,85 +1284,108 @@ var yyAct = [...]int{ 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, 168, - 169, 0, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 283, 292, - 289, 290, 287, 288, 286, 285, 284, 294, 275, 276, - 277, 278, 280, 126, 279, 77, 0, 109, 170, 134, - 96, 161, 94, 0, 0, 0, 0, 108, 282, 110, - 0, 0, 143, 119, 0, 0, 0, 0, 273, 274, - 0, 0, 0, 0, 0, 0, 0, 0, 50, 0, - 0, 240, 261, 260, 263, 264, 265, 266, 0, 0, - 87, 262, 267, 268, 269, 0, 0, 0, 254, 0, - 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, + 135, 171, 127, 139, 89, 159, 142, 283, 292, 289, + 290, 287, 288, 286, 285, 284, 294, 275, 276, 277, + 278, 280, 0, 279, 77, 0, 109, 170, 134, 96, + 161, 126, 0, 0, 0, 0, 242, 0, 0, 0, + 94, 0, 239, 0, 0, 108, 282, 110, 0, 0, + 143, 119, 0, 0, 0, 0, 273, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 50, 0, 0, 240, + 261, 260, 263, 264, 265, 266, 0, 0, 87, 262, + 267, 268, 269, 0, 0, 237, 254, 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 251, 252, 0, 0, 0, 0, 293, 0, 253, 0, - 0, 248, 249, 250, 255, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, - 291, 0, 132, 0, 0, 146, 99, 98, 107, 0, - 0, 0, 90, 0, 138, 128, 158, 1294, 129, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, - 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 283, 292, 289, 290, 287, 288, - 286, 285, 284, 294, 275, 276, 277, 278, 280, 126, - 279, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 0, 0, 0, 108, 282, 110, 0, 0, 143, 119, - 0, 0, 0, 0, 273, 274, 0, 0, 0, 0, - 0, 0, 0, 0, 50, 0, 0, 240, 261, 260, - 263, 264, 265, 266, 0, 0, 87, 262, 267, 268, - 269, 0, 0, 0, 254, 0, 281, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 251, 252, + 0, 0, 0, 0, 293, 0, 253, 0, 0, 248, + 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 188, 0, 0, 291, 0, + 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, + 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, + 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, + 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, + 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, + 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, + 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, + 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, + 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, + 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, + 142, 283, 292, 289, 290, 287, 288, 286, 285, 284, + 294, 275, 276, 277, 278, 280, 126, 279, 77, 0, + 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, + 108, 282, 110, 0, 0, 143, 119, 0, 0, 0, + 0, 273, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 50, 0, 0, 240, 261, 260, 263, 264, 265, + 266, 0, 0, 87, 262, 267, 268, 269, 0, 0, + 0, 254, 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 251, 252, 0, 0, - 0, 0, 293, 0, 253, 0, 0, 248, 249, 250, - 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 291, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, - 283, 292, 289, 290, 287, 288, 286, 285, 284, 294, - 275, 276, 277, 278, 280, 126, 279, 77, 0, 109, - 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, - 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, + 0, 0, 0, 251, 252, 0, 0, 0, 0, 293, + 0, 253, 0, 0, 248, 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, - 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, + 188, 0, 0, 291, 0, 132, 0, 0, 146, 99, + 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, + 1296, 129, 137, 111, 150, 133, 157, 189, 165, 148, + 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, + 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, + 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, + 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, + 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, + 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, + 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, + 171, 127, 139, 89, 159, 142, 283, 292, 289, 290, + 287, 288, 286, 285, 284, 294, 275, 276, 277, 278, + 280, 126, 279, 77, 0, 109, 170, 134, 96, 161, + 94, 0, 0, 0, 0, 108, 282, 110, 0, 0, + 143, 119, 0, 0, 0, 0, 273, 274, 0, 0, + 0, 0, 0, 0, 0, 0, 50, 0, 0, 240, + 261, 260, 263, 264, 265, 266, 0, 0, 87, 262, + 267, 268, 269, 0, 0, 0, 254, 0, 281, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 491, 490, - 500, 501, 493, 494, 495, 496, 497, 498, 499, 492, - 0, 0, 502, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 188, - 0, 0, 0, 0, 132, 0, 0, 146, 99, 98, - 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, - 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, - 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 251, 252, + 0, 0, 0, 0, 293, 0, 253, 0, 0, 248, + 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 188, 0, 0, 291, 0, + 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, + 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, + 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, + 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, + 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, + 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, + 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, + 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, + 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, + 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, + 142, 283, 292, 289, 290, 287, 288, 286, 285, 284, + 294, 275, 276, 277, 278, 280, 126, 279, 77, 0, + 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, + 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 75, 0, 0, 0, 0, 0, + 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 493, + 492, 502, 503, 495, 496, 497, 498, 499, 500, 501, + 494, 0, 0, 504, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 188, 0, 0, 0, 0, 132, 0, 0, 146, 99, + 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, + 0, 129, 137, 111, 150, 133, 157, 189, 165, 148, + 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, + 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, + 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, + 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, + 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, + 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 77, 0, 109, 170, 134, 96, 161, - 126, 0, 0, 0, 479, 0, 0, 0, 0, 94, + 126, 0, 0, 0, 481, 0, 0, 0, 0, 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 75, 0, - 481, 0, 0, 0, 0, 0, 0, 87, 0, 0, - 0, 0, 476, 475, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 477, + 483, 0, 0, 0, 0, 0, 0, 87, 0, 0, + 0, 0, 478, 477, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 479, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1400,106 +1398,83 @@ var yyAct = [...]int{ 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, - 166, 167, 168, 169, 0, 0, 0, 0, 123, 85, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 126, 0, 77, 0, - 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, - 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 75, 0, 0, 0, 0, 0, - 0, 0, 0, 87, 0, 0, 0, 0, 68, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, + 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 71, 72, 0, - 67, 0, 0, 0, 73, 132, 0, 0, 146, 99, - 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, - 0, 129, 137, 111, 150, 133, 157, 69, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, - 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, - 0, 0, 0, 0, 123, 85, 102, 141, 105, 112, - 135, 171, 127, 139, 89, 159, 142, 0, 70, 0, + 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, + 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, + 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 77, 0, 109, 170, 134, 96, - 161, 126, 0, 0, 0, 576, 0, 0, 0, 0, - 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, - 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 186, - 0, 578, 0, 0, 0, 0, 0, 0, 87, 0, + 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, + 0, 0, 87, 0, 0, 0, 0, 68, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 71, 72, 0, 67, + 0, 0, 0, 73, 132, 0, 0, 146, 99, 98, + 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, + 129, 137, 111, 150, 133, 157, 69, 165, 148, 164, + 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, + 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, + 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, + 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, + 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, + 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, + 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, + 127, 139, 89, 159, 142, 0, 70, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 188, 0, 0, 0, 0, - 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, - 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, - 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, - 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, - 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, - 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, - 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, - 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, - 0, 166, 167, 168, 169, 0, 0, 0, 0, 123, - 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, - 159, 142, 0, 0, 0, 23, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 126, 0, 77, - 0, 109, 170, 134, 96, 161, 94, 0, 0, 0, - 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, + 0, 0, 77, 0, 109, 170, 134, 96, 161, 126, + 0, 0, 0, 578, 0, 0, 0, 0, 94, 0, + 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 50, 0, 0, 75, 0, 0, 0, 0, - 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 186, 0, 580, + 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 188, 0, 0, 0, 0, 132, 0, 0, 146, - 99, 98, 107, 0, 0, 0, 90, 0, 138, 128, - 158, 0, 129, 137, 111, 150, 133, 157, 189, 165, - 148, 164, 78, 147, 156, 88, 140, 80, 154, 145, - 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, - 151, 152, 91, 172, 83, 163, 82, 84, 162, 124, - 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, - 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, - 0, 144, 160, 173, 86, 0, 0, 166, 167, 168, - 169, 0, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, - 0, 23, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, - 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, - 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 50, 0, - 0, 186, 0, 0, 0, 0, 0, 0, 0, 0, - 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, + 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, + 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, + 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, + 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, + 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, + 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, + 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, + 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 23, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, + 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, + 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 50, + 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, + 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, - 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, - 0, 0, 90, 0, 138, 128, 158, 0, 129, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, - 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, + 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, + 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, + 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, + 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, + 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, + 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, + 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, + 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, + 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, + 139, 89, 159, 142, 0, 0, 0, 23, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 75, 0, 0, - 697, 0, 0, 698, 0, 0, 87, 0, 0, 0, + 0, 0, 0, 0, 50, 0, 0, 186, 0, 0, + 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1514,105 +1489,127 @@ var yyAct = [...]int{ 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, - 170, 134, 96, 161, 94, 0, 596, 0, 0, 108, - 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, + 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, + 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, + 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 75, 0, 595, 0, 0, 0, 0, - 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 75, 0, 0, 699, 0, 0, 700, 0, + 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 188, - 0, 0, 0, 0, 132, 0, 0, 146, 99, 98, - 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, - 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, - 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, + 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, + 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, + 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, + 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, + 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, + 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, + 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, + 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, + 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, + 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, + 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, + 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, + 598, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 77, 0, 109, 170, 134, 96, 161, - 126, 0, 0, 0, 576, 0, 0, 0, 0, 94, - 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, - 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 186, 0, - 578, 0, 0, 0, 0, 0, 0, 87, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 75, 0, 597, + 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 188, 0, 0, 0, 0, 132, - 0, 0, 146, 99, 98, 107, 0, 0, 0, 90, - 0, 138, 128, 158, 0, 574, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, - 166, 167, 168, 169, 0, 0, 0, 0, 123, 85, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 126, 0, 77, 0, - 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, - 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, + 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, + 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, + 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, + 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, + 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, + 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, + 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, + 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, + 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 50, 0, 0, 186, 0, 0, 0, 0, 0, - 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 77, 0, 109, 170, + 134, 96, 161, 126, 0, 0, 0, 578, 0, 0, + 0, 0, 94, 0, 0, 0, 0, 108, 0, 110, + 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 186, 0, 580, 0, 0, 0, 0, 0, 0, + 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 188, 0, 0, 0, 0, 132, 0, 0, 146, 99, - 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, - 0, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, - 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, - 0, 0, 0, 0, 123, 85, 102, 141, 105, 112, - 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, + 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, + 0, 0, 90, 0, 138, 128, 158, 0, 576, 137, + 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, + 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, + 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, + 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, + 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, + 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, + 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, + 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, + 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, + 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, + 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 126, 0, 77, 0, 109, 170, 134, 96, - 161, 94, 0, 0, 0, 0, 108, 0, 110, 0, - 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 50, 0, 0, 186, 0, 0, 0, + 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 186, 0, 578, 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 188, 0, 0, 0, 0, 132, 0, 0, + 146, 99, 98, 107, 0, 0, 0, 90, 0, 138, + 128, 158, 0, 129, 137, 111, 150, 133, 157, 189, + 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, + 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, + 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, + 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, + 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, + 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, + 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, + 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 188, 0, 0, 0, - 0, 132, 0, 0, 146, 99, 98, 107, 0, 0, - 0, 90, 0, 138, 128, 158, 0, 129, 137, 111, - 150, 133, 157, 189, 165, 148, 164, 78, 147, 156, - 88, 140, 80, 154, 145, 117, 103, 104, 79, 0, - 136, 93, 97, 92, 125, 151, 152, 91, 172, 83, - 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, - 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, - 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, - 0, 0, 166, 167, 168, 169, 0, 0, 0, 0, + 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, + 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, + 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 186, 0, 580, 0, 0, 0, 0, 0, 0, + 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, + 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, + 0, 0, 90, 0, 138, 128, 158, 0, 129, 137, + 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, + 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, + 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, + 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, + 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, + 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, + 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 75, 0, 481, 0, + 0, 0, 0, 0, 0, 0, 75, 0, 483, 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1628,104 +1625,81 @@ var yyAct = [...]int{ 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, - 168, 169, 0, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 126, 77, 0, 109, 170, - 134, 96, 161, 554, 94, 0, 0, 0, 0, 108, - 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 186, 0, 0, 0, 0, 0, 0, - 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, + 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 126, 77, 0, 109, 170, 134, + 96, 161, 556, 94, 0, 0, 0, 0, 108, 0, + 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 188, - 0, 0, 0, 0, 132, 0, 0, 146, 99, 98, - 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, - 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, - 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, - 0, 0, 0, 0, 304, 0, 0, 0, 0, 0, - 0, 126, 0, 77, 0, 109, 170, 134, 96, 161, - 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, - 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 186, - 0, 0, 0, 0, 0, 0, 0, 0, 87, 0, + 0, 0, 186, 0, 0, 0, 0, 0, 0, 0, + 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, + 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, + 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, + 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, + 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, + 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, + 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, + 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, + 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, + 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, + 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, + 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, + 0, 0, 304, 0, 0, 0, 0, 0, 0, 126, + 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, + 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 188, 0, 0, 0, 0, - 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, - 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, - 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, - 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, - 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, - 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, - 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, - 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, - 0, 166, 167, 168, 169, 0, 0, 0, 0, 123, - 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, - 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 126, 0, 77, - 0, 109, 170, 134, 96, 161, 94, 0, 0, 0, - 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, + 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 186, 0, 0, 0, 0, - 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 183, - 0, 188, 0, 0, 0, 0, 132, 0, 0, 146, - 99, 98, 107, 0, 0, 0, 90, 0, 138, 128, - 158, 0, 129, 137, 111, 150, 133, 157, 189, 165, - 148, 164, 78, 147, 156, 88, 140, 80, 154, 145, - 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, - 151, 152, 91, 172, 83, 163, 82, 84, 162, 124, - 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, - 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, - 0, 144, 160, 173, 86, 0, 0, 166, 167, 168, - 169, 0, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, + 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, + 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, + 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, + 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, + 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, + 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, + 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, + 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, + 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, - 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, - 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, + 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, + 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, - 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 186, 0, 0, 0, 0, 0, 0, 0, + 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, - 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, - 0, 0, 90, 0, 138, 128, 158, 0, 129, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, - 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 183, 0, 188, 0, + 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, + 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, + 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, + 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, + 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, + 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, + 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, + 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, + 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 240, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -1741,45 +1715,67 @@ var yyAct = [...]int{ 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, - 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, - 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, + 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, + 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, + 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 186, 0, 0, 0, 0, 0, 0, - 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 240, 0, 0, 0, 0, 0, 0, 0, + 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 188, - 0, 0, 0, 0, 132, 0, 0, 146, 99, 98, - 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, - 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, - 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, + 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, + 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, + 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, + 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, + 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, + 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, + 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, + 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, + 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, + 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, + 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, + 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, + 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, + 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 77, 0, 109, 170, 134, 96, 161, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, + 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, + 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, + 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, + 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, + 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, + 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, + 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, + 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, + 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, + 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 77, 0, 109, 170, + 134, 96, 161, } var yyPact = [...]int{ - 1844, -1000, -182, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 719, 747, -1000, -1000, -1000, -1000, -1000, -1000, 591, - 7488, 43, 65, -4, 10219, 64, 195, 10897, -1000, -8, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 513, -1000, -1000, - -1000, -1000, -1000, 714, 717, 566, 706, 655, -1000, 5641, - 40, 9088, 9993, 4936, -1000, 443, 61, 10897, -154, 10445, - 38, 38, 38, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1949, -1000, -184, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 804, 838, -1000, -1000, -1000, -1000, -1000, -1000, 615, + 7487, 41, 70, -1, 10206, 67, 1372, 10881, -1000, -9, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 561, -1000, -1000, + -1000, -1000, -1000, 798, 802, 632, 793, 695, -1000, 5648, + 40, 9080, 9981, 4946, -1000, 486, 64, 10881, -151, 10431, + 37, 37, 37, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1788,22 +1784,22 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 63, 10897, -1000, 10897, 37, 442, - 37, 37, 37, 10897, -1000, 95, -1000, -1000, -1000, -1000, - 10897, 441, 676, 33, 2952, 2952, 2952, 2952, -3, 2952, - -91, 611, -1000, -1000, -1000, -1000, -1000, 2952, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 355, 684, - 6349, 6349, 719, -1000, 513, -1000, -1000, -1000, 669, -1000, - -1000, 205, 732, -1000, 7262, 94, -1000, 6349, 1684, 517, - -1000, -1000, 517, -1000, -1000, 76, -1000, -1000, 6801, 6801, - 6801, 6801, 6801, 6801, 6801, 6801, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 517, -1000, 6114, 517, 517, 517, 517, 517, 517, 517, - 517, 6349, 517, 517, 517, 517, 517, 517, 517, 517, - 517, 517, 517, 517, 517, 9767, 514, 604, -1000, -1000, - -1000, 702, 8175, 8862, 10897, 481, -1000, 511, 4688, -130, - -1000, -1000, -1000, 158, 8627, -1000, -1000, -1000, 675, -1000, + -1000, -1000, -1000, -1000, 66, 10881, -1000, 10881, 35, 485, + 35, 35, 35, 10881, -1000, 116, -1000, -1000, -1000, -1000, + 10881, 483, 729, 33, 2970, 2970, 2970, 2970, -3, 2970, + -62, 651, -1000, -1000, -1000, -1000, -1000, 2970, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 404, 742, + 6353, 6353, 804, -1000, 561, -1000, -1000, -1000, 728, -1000, + -1000, 278, 826, -1000, 7262, 114, -1000, 6353, 700, 575, + -1000, -1000, 575, -1000, -1000, 80, -1000, -1000, 6803, 6803, + 6803, 6803, 6803, 6803, 6803, 6803, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 575, -1000, 6119, 575, 575, 575, 575, 575, 575, 575, + 575, 6353, 575, 575, 575, 575, 575, 575, 575, 575, + 575, 575, 575, 575, 575, 9756, 528, 669, -1000, -1000, + -1000, 787, 8171, 8855, 10881, 545, -1000, 567, 4699, -86, + -1000, -1000, -1000, 221, 8621, -1000, -1000, -1000, 727, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1812,180 +1808,190 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 432, -1000, 1917, 427, 2952, 47, 602, 425, - 174, 424, 10897, 10897, 2952, 45, 10897, 700, 610, 10897, - 423, 416, -1000, 4440, -1000, 2952, 2952, 2952, 2952, 2952, - 2952, 2952, 2952, -1000, -1000, -1000, -1000, -1000, -1000, 2952, - 2952, -1000, -86, -1000, 10897, -1000, -1000, -1000, -1000, 742, - 109, 361, 84, 515, -1000, 305, 714, 355, 655, 8401, - 625, -1000, -1000, 10897, -1000, 6349, 6349, 266, -1000, 9540, - -1000, -1000, 3448, 131, 6801, 274, 140, 6801, 6801, 6801, - 6801, 6801, 6801, 6801, 6801, 6801, 6801, 6801, 6801, 6801, - 6801, 6801, 309, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 413, -1000, 513, 465, 465, 108, 108, 108, 108, - 108, 108, 108, 7027, 5171, 355, 422, 241, 6114, 5641, - 5641, 6349, 6349, 10671, 10671, 5641, 707, 169, 241, 10671, - -1000, 355, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 5641, - 5641, 5641, 5641, 14, 10897, -1000, 10671, 9088, 9088, 9088, - 9088, 9088, -1000, 635, 631, -1000, 624, 623, 634, 10897, - -1000, 420, 8175, 110, 517, -1000, 9314, -1000, -1000, 14, - 459, 9088, 10897, -1000, -1000, 4192, 511, -130, 496, -1000, - -108, -104, 5876, 104, -1000, -1000, -1000, -1000, 2704, 391, - 185, -73, -1000, -1000, -1000, 532, -1000, 532, 532, 532, - 532, -46, -46, -46, -46, -1000, -1000, -1000, -1000, -1000, - 581, 580, -1000, 532, 532, 532, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 559, 559, 559, 547, 547, 605, -1000, 10897, - -170, 412, 2952, 699, 2952, -1000, 58, -1000, 10897, -1000, - -1000, 10897, 2952, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 209, -1000, - -1000, -1000, -1000, -1000, 651, 6349, 6349, 3944, 6349, -1000, - -1000, -1000, 684, -1000, 707, 718, -1000, 667, 666, 5641, - -1000, -1000, 131, 160, -1000, -1000, 272, -1000, -1000, -1000, - -1000, 83, 517, -1000, 2028, -1000, -1000, -1000, -1000, 274, - 6801, 6801, 6801, 1523, 2028, 2135, 1278, 936, 108, 249, - 249, 105, 105, 105, 105, 105, 776, 776, -1000, -1000, - -1000, 355, -1000, -1000, -1000, 355, 5641, 503, -1000, -1000, - 6349, -1000, 355, 405, 405, 341, 298, 552, -1000, 82, - 542, 405, 5641, 166, -1000, 6349, 355, -1000, 405, 355, - 405, 405, 440, 517, -1000, 498, -1000, 153, 604, 556, - 608, 546, -1000, -1000, -1000, -1000, 630, -1000, 627, -1000, - -1000, -1000, -1000, -1000, 57, 56, 49, 10445, -1000, 730, - 9088, 492, -1000, -1000, 496, -130, -82, -1000, -1000, -1000, - 241, -1000, 393, 491, 2456, -1000, -1000, -1000, -1000, -1000, - -1000, 551, 686, 151, 147, 364, -1000, -1000, 678, -1000, - 197, -80, -1000, -1000, 294, -46, -46, -1000, -1000, 104, - 671, 104, 104, 104, 338, 338, -1000, -1000, -1000, -1000, - 293, -1000, -1000, -1000, 285, -1000, 607, 10445, 2952, -1000, - 3696, -1000, -1000, -1000, -1000, -1000, -1000, 661, 335, 184, + -1000, -1000, 482, -1000, 1980, 478, 2970, 50, 634, 477, + 230, 470, 10881, 10881, 2970, 48, 10881, 785, 649, 10881, + 459, 444, -1000, 4452, -1000, 2970, 2970, 2970, 2970, 2970, + 2970, 2970, 2970, -1000, -1000, -1000, -1000, -1000, -1000, 2970, + 2970, -1000, 824, 223, -1000, -1000, 10881, -1000, -1000, -1000, + -1000, 832, 143, 339, 112, 569, -1000, 336, 798, 404, + 695, 8396, 661, -1000, -1000, 10881, -1000, 6353, 6353, 380, + -1000, 9530, -1000, -1000, 3464, 166, 6803, 331, 266, 6803, + 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, + 6803, 6803, 6803, 6803, 315, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 442, -1000, 561, 408, 408, 124, 124, + 124, 124, 124, 124, 124, 7028, 5180, 404, 474, 357, + 6119, 5648, 5648, 6353, 6353, 10656, 10656, 5648, 790, 238, + 357, 10656, -1000, 404, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 5648, 5648, 5648, 5648, 16, 10881, -1000, 10656, 9080, + 9080, 9080, 9080, 9080, -1000, 686, 683, -1000, 680, 677, + 701, 10881, -1000, 468, 8171, 120, 575, -1000, 9305, -1000, + -1000, 16, 514, 9080, 10881, -1000, -1000, 4205, 567, -86, + 559, -1000, -108, -126, 5882, 77, -1000, -1000, -1000, -1000, + 2723, 133, 249, -87, -1000, -1000, -1000, 584, -1000, 584, + 584, 584, 584, -51, -51, -51, -51, -1000, -1000, -1000, + -1000, -1000, 613, 611, -1000, 584, 584, 584, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 13, -1000, 2952, -1000, 225, 10897, 10897, 648, 241, - 241, 81, -1000, -1000, 10897, -1000, -1000, -1000, -1000, 497, - -1000, -1000, -1000, 3200, 5641, -1000, 1523, 2028, 2100, -1000, - 6801, 6801, -1000, -1000, 405, 5641, 241, -1000, -1000, -1000, - 129, 309, 129, 6801, 6801, 3944, 6801, 6801, -164, 494, - 162, -1000, 6349, 145, -1000, -1000, -1000, -1000, -1000, 548, - 10671, 517, -1000, 7949, 10445, 719, 10671, 6349, 6349, -1000, - -1000, 6349, 549, -1000, 6349, -1000, -1000, -1000, 517, 517, - 517, 379, -1000, 719, 492, -1000, -1000, -1000, -114, -115, - -1000, -1000, 2704, -1000, 2704, 10445, -1000, 362, 360, -1000, - -1000, 516, 67, -1000, -1000, -1000, 447, 104, 104, -1000, - 155, -1000, -1000, -1000, 403, -1000, 400, 486, 398, 10897, - -1000, -1000, 484, -1000, 143, -1000, -1000, 10445, -1000, -1000, + -1000, -1000, -1000, -1000, 610, 610, 610, 586, 586, 635, + -1000, 10881, -168, 430, 2970, 783, 2970, -1000, 61, -1000, + 10881, -1000, -1000, 10881, 2970, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 10445, 10897, -1000, -1000, -1000, -1000, -1000, 10445, -1000, -1000, - 330, 6349, -1000, -1000, -1000, 3696, -1000, 730, 9088, -1000, - -1000, 355, -1000, 6801, 2028, 2028, -1000, -1000, 355, 532, - 532, -1000, 532, 547, -1000, 532, -19, 532, -22, 355, - 355, 1784, 1947, -1000, 1548, 1837, 517, -161, -1000, 241, - 6349, -1000, 685, 452, 482, -1000, -1000, 5406, 355, 390, - 80, 379, 714, -1000, 241, 241, 241, 10445, 241, 10445, - 10445, 10445, 7723, 10445, 714, -1000, -1000, -1000, -1000, 2456, - -1000, 375, -1000, 532, -1000, -1000, -69, 741, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -46, - 329, -46, 267, -1000, 262, 2952, 3696, 2704, -1000, 523, - -1000, -1000, -1000, -1000, 695, -1000, 241, 727, 483, -1000, - 2028, -1000, -1000, 89, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 6801, 6801, -1000, 6801, 6801, 6801, 355, - 320, 241, 683, -1000, 517, -1000, -1000, 501, 10445, 10445, - -1000, -1000, 372, -1000, 368, 368, 368, 110, -1000, -1000, - 106, 10445, -1000, 133, -1000, -143, 104, -1000, 104, 380, - 369, -1000, -1000, -1000, 10445, 517, 721, 716, -1000, -1000, - 1577, 1577, 1577, 1577, 4, -1000, -1000, 740, -1000, 517, - -1000, 513, 79, -1000, 10445, -1000, -1000, -1000, -1000, -1000, - 106, -1000, 346, 139, 313, -1000, 200, 681, -1000, 680, - -1000, -1000, -1000, -1000, -1000, 354, 7, -1000, 6349, 6349, - -1000, -1000, -1000, -1000, 355, 59, -173, 10671, 482, 355, - 10445, -1000, -1000, -1000, 255, -1000, -1000, -1000, 308, -1000, - -1000, 602, 344, -1000, 10445, 241, 474, -1000, 646, -168, - -176, 461, -1000, -1000, -1000, -1000, -170, -1000, 7, 665, - -1000, 641, -1000, -1000, -1000, 9, -171, 5, -174, 517, - -179, 6575, -1000, 1577, 355, -1000, -1000, + 10881, 257, 10881, 10881, -1000, -1000, 711, 6353, 6353, 3958, + 6353, -1000, -1000, -1000, 742, -1000, 790, 823, -1000, 721, + 718, 5648, -1000, -1000, 166, 236, -1000, -1000, 366, -1000, + -1000, -1000, -1000, 97, 575, -1000, 2217, -1000, -1000, -1000, + -1000, 331, 6803, 6803, 6803, 118, 2217, 2142, 577, 1144, + 124, 150, 150, 140, 140, 140, 140, 140, 234, 234, + -1000, -1000, -1000, 404, -1000, -1000, -1000, 404, 5648, 562, + -1000, -1000, 6353, -1000, 404, 466, 466, 299, 321, 579, + -1000, 92, 568, 466, 5648, 233, -1000, 6353, 404, -1000, + 466, 404, 466, 466, 521, 575, -1000, 547, -1000, 216, + 669, 607, 644, 437, -1000, -1000, -1000, -1000, 681, -1000, + 679, -1000, -1000, -1000, -1000, -1000, 63, 62, 57, 10431, + -1000, 816, 9080, 518, -1000, -1000, 559, -86, -99, -1000, + -1000, -1000, 357, -1000, 420, 558, 2476, -1000, -1000, -1000, + -1000, -1000, -1000, 603, 737, 146, 145, 375, -1000, -1000, + 731, -1000, 259, -91, -1000, -1000, 330, -51, -51, -1000, + -1000, 77, 725, 77, 77, 77, 361, 361, -1000, -1000, + -1000, -1000, 328, -1000, -1000, -1000, 326, -1000, 642, 10431, + 2970, -1000, 3711, -1000, -1000, -1000, -1000, -1000, -1000, 376, + 269, 199, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 15, -1000, 2970, -1000, 223, -1000, 358, + 6353, -1000, -1000, 708, 357, 357, 84, -1000, -1000, 10881, + -1000, -1000, -1000, -1000, 546, -1000, -1000, -1000, 3217, 5648, + -1000, 118, 2217, 2110, -1000, 6803, 6803, -1000, -1000, 466, + 5648, 357, -1000, -1000, -1000, 135, 315, 135, 6803, 6803, + 3958, 6803, 6803, -163, 563, 226, -1000, 6353, 343, -1000, + -1000, -1000, -1000, -1000, 641, 10656, 575, -1000, 7946, 10431, + 804, 10656, 6353, 6353, -1000, -1000, 6353, 590, -1000, 6353, + -1000, -1000, -1000, 575, 575, 575, 436, -1000, 804, 518, + -1000, -1000, -1000, -133, -131, -1000, -1000, 2723, -1000, 2723, + 10431, -1000, 373, 368, -1000, -1000, 564, 72, -1000, -1000, + -1000, 469, 77, 77, -1000, 242, -1000, -1000, -1000, 463, + -1000, 457, 549, 455, 10881, -1000, -1000, 532, -1000, 190, + -1000, -1000, 10431, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 10431, 10881, -1000, -1000, -1000, + -1000, -1000, 10431, -1000, 257, -1000, 357, -1000, 3711, -1000, + 816, 9080, -1000, -1000, 404, -1000, 6803, 2217, 2217, -1000, + -1000, 404, 584, 584, -1000, 584, 586, -1000, 584, -26, + 584, -29, 404, 404, 2125, 1932, -1000, 1870, 1702, 575, + -160, -1000, 357, 6353, -1000, 750, 499, 501, -1000, -1000, + 5414, 404, 440, 83, 436, 798, -1000, 357, 357, 357, + 10431, 357, 10431, 10431, 10431, 7721, 10431, 798, -1000, -1000, + -1000, -1000, 2476, -1000, 407, -1000, 584, -1000, -1000, -77, + 831, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -51, 347, -51, 308, -1000, 303, 2970, 3711, + 2723, -1000, 582, -1000, -1000, -1000, -1000, 778, -1000, 809, + 529, -1000, 2217, -1000, -1000, 68, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 6803, 6803, -1000, 6803, 6803, + 6803, 404, 344, 357, 736, -1000, 575, -1000, -1000, 543, + 10431, 10431, -1000, -1000, 403, -1000, 401, 401, 401, 120, + -1000, -1000, 102, 10431, -1000, 138, -1000, -141, 77, -1000, + 77, 458, 441, -1000, -1000, -1000, 10431, 575, 807, 801, + -1000, -1000, 1515, 1515, 1515, 1515, 9, -1000, -1000, 829, + -1000, 575, -1000, 561, 78, -1000, 10431, -1000, -1000, -1000, + -1000, -1000, 102, -1000, 359, 178, 316, -1000, 262, 734, + -1000, 730, -1000, -1000, -1000, -1000, -1000, 399, 14, -1000, + 6353, 6353, -1000, -1000, -1000, -1000, 404, 36, -172, 10656, + 501, 404, 10431, -1000, -1000, -1000, 288, -1000, -1000, -1000, + 300, -1000, -1000, 634, 391, -1000, 10431, 357, 491, -1000, + 703, -166, -180, 484, -1000, -1000, -1000, -1000, -168, -1000, + 14, 717, -1000, 699, -1000, -1000, -1000, 11, -169, 4, + -178, 575, -181, 6578, -1000, 1515, 404, -1000, -1000, } var yyPgo = [...]int{ - 0, 967, 45, 500, 966, 964, 958, 957, 953, 950, - 946, 943, 939, 938, 937, 934, 933, 932, 931, 930, - 920, 919, 918, 917, 110, 916, 914, 912, 58, 911, - 61, 909, 904, 34, 166, 35, 31, 777, 901, 23, - 74, 55, 900, 41, 899, 898, 69, 897, 52, 896, - 895, 1563, 894, 889, 12, 30, 888, 883, 882, 880, - 68, 980, 868, 867, 865, 860, 855, 853, 44, 6, - 8, 10, 13, 852, 19, 7, 851, 42, 850, 847, - 846, 843, 32, 842, 49, 840, 21, 43, 838, 9, - 51, 28, 17, 4, 63, 54, 837, 22, 57, 38, - 835, 834, 319, 831, 829, 828, 827, 826, 825, 142, - 317, 822, 821, 819, 817, 37, 186, 485, 88, 59, - 816, 815, 814, 1263, 60, 53, 24, 813, 20, 879, - 29, 812, 809, 33, 808, 807, 805, 803, 802, 799, - 798, 282, 797, 796, 795, 25, 14, 786, 784, 47, - 26, 783, 781, 780, 40, 50, 779, 39, 778, 775, - 771, 770, 27, 18, 767, 11, 766, 16, 765, 762, - 2, 761, 15, 760, 5, 758, 3, 36, 757, 755, - 0, 343, 754, 753, 70, + 0, 1035, 12, 492, 1033, 1032, 1030, 1026, 1025, 1022, + 1019, 1018, 1016, 1015, 1005, 1002, 998, 997, 994, 993, + 991, 990, 984, 982, 115, 981, 979, 976, 55, 975, + 64, 974, 972, 34, 129, 35, 33, 779, 970, 36, + 91, 65, 969, 43, 957, 955, 67, 953, 53, 950, + 948, 1395, 936, 935, 9, 22, 934, 933, 932, 931, + 63, 830, 929, 928, 927, 924, 923, 922, 49, 5, + 6, 11, 16, 921, 85, 10, 920, 46, 919, 917, + 916, 915, 45, 914, 47, 913, 18, 44, 912, 14, + 52, 27, 20, 17, 66, 51, 911, 26, 54, 42, + 910, 908, 350, 907, 906, 37, 905, 24, 136, 349, + 904, 903, 902, 901, 29, 168, 501, 589, 58, 900, + 899, 896, 1355, 59, 57, 23, 895, 28, 570, 32, + 893, 892, 39, 891, 890, 887, 885, 884, 883, 882, + 78, 881, 880, 879, 15, 30, 877, 873, 61, 31, + 871, 870, 869, 41, 50, 868, 40, 865, 864, 863, + 862, 25, 19, 860, 8, 859, 7, 858, 857, 2, + 856, 21, 855, 3, 848, 4, 38, 847, 846, 0, + 460, 845, 844, 92, } var yyR1 = [...]int{ - 0, 178, 179, 179, 1, 1, 1, 1, 1, 1, + 0, 177, 178, 178, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 6, 3, 4, 4, 5, - 5, 7, 7, 27, 27, 8, 9, 9, 9, 182, - 182, 46, 46, 90, 90, 10, 10, 10, 10, 95, - 95, 99, 99, 99, 100, 100, 100, 100, 131, 131, - 11, 11, 11, 11, 11, 11, 11, 176, 176, 175, - 174, 174, 173, 173, 172, 16, 159, 160, 160, 160, - 155, 134, 134, 134, 134, 137, 137, 135, 135, 135, - 135, 135, 135, 135, 136, 136, 136, 136, 136, 138, - 138, 138, 138, 138, 139, 139, 139, 139, 139, 139, - 139, 139, 139, 139, 139, 139, 139, 139, 139, 140, - 140, 140, 140, 140, 140, 140, 140, 154, 154, 141, - 141, 149, 149, 150, 150, 150, 147, 147, 148, 148, - 151, 151, 151, 142, 142, 142, 142, 142, 142, 142, - 144, 144, 152, 152, 145, 145, 145, 146, 146, 153, - 153, 153, 153, 153, 143, 143, 156, 156, 168, 168, - 167, 167, 167, 158, 158, 164, 164, 164, 164, 164, - 157, 157, 166, 166, 165, 161, 161, 161, 162, 162, - 162, 163, 163, 163, 12, 12, 12, 12, 12, 12, - 12, 12, 12, 177, 177, 177, 177, 177, 177, 177, - 177, 177, 177, 177, 171, 169, 169, 170, 170, 13, + 5, 7, 7, 27, 27, 8, 9, 9, 9, 181, + 181, 46, 46, 90, 90, 10, 10, 10, 10, 95, + 95, 99, 99, 99, 100, 100, 100, 100, 130, 130, + 11, 11, 11, 11, 11, 11, 11, 175, 175, 174, + 173, 173, 172, 172, 171, 16, 158, 159, 159, 159, + 154, 133, 133, 133, 133, 136, 136, 134, 134, 134, + 134, 134, 134, 134, 135, 135, 135, 135, 135, 137, + 137, 137, 137, 137, 138, 138, 138, 138, 138, 138, + 138, 138, 138, 138, 138, 138, 138, 138, 138, 139, + 139, 139, 139, 139, 139, 139, 139, 153, 153, 140, + 140, 148, 148, 149, 149, 149, 146, 146, 147, 147, + 150, 150, 150, 141, 141, 141, 141, 141, 141, 141, + 143, 143, 151, 151, 144, 144, 144, 145, 145, 152, + 152, 152, 152, 152, 142, 142, 155, 155, 167, 167, + 166, 166, 166, 157, 157, 163, 163, 163, 163, 163, + 156, 156, 165, 165, 164, 160, 160, 160, 161, 161, + 161, 162, 162, 162, 12, 12, 12, 12, 12, 12, + 12, 12, 12, 176, 176, 176, 176, 176, 176, 176, + 176, 176, 176, 176, 170, 168, 168, 169, 169, 13, 14, 14, 14, 14, 14, 15, 15, 17, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, - 18, 18, 107, 107, 107, 104, 104, 105, 105, 106, - 106, 106, 108, 108, 108, 132, 132, 132, 19, 19, - 21, 21, 22, 23, 20, 20, 20, 20, 20, 183, - 24, 25, 25, 26, 26, 26, 30, 30, 30, 28, - 28, 29, 29, 35, 35, 34, 34, 36, 36, 36, - 36, 120, 120, 120, 119, 119, 38, 38, 39, 39, - 40, 40, 41, 41, 41, 53, 53, 89, 89, 91, - 91, 42, 42, 42, 42, 43, 43, 44, 44, 45, - 45, 127, 127, 126, 126, 126, 125, 125, 47, 47, - 47, 49, 48, 48, 48, 48, 50, 50, 52, 52, - 51, 51, 54, 54, 54, 54, 55, 55, 37, 37, - 37, 37, 37, 37, 37, 103, 103, 57, 57, 56, - 56, 56, 56, 56, 56, 56, 56, 56, 56, 67, - 67, 67, 67, 67, 67, 58, 58, 58, 58, 58, - 58, 58, 33, 33, 68, 68, 68, 74, 69, 69, + 18, 18, 18, 106, 106, 104, 104, 105, 105, 105, + 107, 107, 107, 131, 131, 131, 19, 19, 21, 21, + 22, 23, 20, 20, 20, 20, 20, 182, 24, 25, + 25, 26, 26, 26, 30, 30, 30, 28, 28, 29, + 29, 35, 35, 34, 34, 36, 36, 36, 36, 119, + 119, 119, 118, 118, 38, 38, 39, 39, 40, 40, + 41, 41, 41, 53, 53, 89, 89, 91, 91, 42, + 42, 42, 42, 43, 43, 44, 44, 45, 45, 126, + 126, 125, 125, 125, 124, 124, 47, 47, 47, 49, + 48, 48, 48, 48, 50, 50, 52, 52, 51, 51, + 54, 54, 54, 54, 55, 55, 37, 37, 37, 37, + 37, 37, 37, 103, 103, 57, 57, 56, 56, 56, + 56, 56, 56, 56, 56, 56, 56, 67, 67, 67, + 67, 67, 67, 58, 58, 58, 58, 58, 58, 58, + 33, 33, 68, 68, 68, 74, 69, 69, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, - 61, 61, 65, 65, 65, 63, 63, 63, 63, 63, - 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, - 64, 64, 64, 64, 64, 64, 64, 64, 184, 184, - 66, 66, 66, 66, 31, 31, 31, 31, 31, 130, - 130, 133, 133, 133, 133, 133, 133, 133, 133, 133, - 133, 133, 133, 133, 78, 78, 32, 32, 76, 76, - 77, 79, 79, 75, 75, 75, 60, 60, 60, 60, - 60, 60, 60, 60, 62, 62, 62, 80, 80, 81, - 81, 82, 82, 83, 83, 84, 85, 85, 85, 86, - 86, 86, 86, 87, 87, 87, 59, 59, 59, 59, - 59, 59, 88, 88, 88, 88, 92, 92, 70, 70, - 72, 72, 71, 73, 93, 93, 97, 94, 94, 98, - 98, 98, 96, 96, 96, 122, 122, 122, 101, 101, - 109, 109, 110, 110, 102, 102, 111, 111, 111, 111, - 111, 111, 111, 111, 111, 111, 112, 112, 112, 113, - 113, 114, 114, 114, 121, 121, 117, 117, 118, 118, - 123, 123, 124, 124, 115, 115, 115, 115, 115, 115, + 65, 65, 65, 63, 63, 63, 63, 63, 63, 63, + 63, 63, 63, 63, 63, 63, 63, 63, 64, 64, + 64, 64, 64, 64, 64, 64, 183, 183, 66, 66, + 66, 66, 31, 31, 31, 31, 31, 129, 129, 132, + 132, 132, 132, 132, 132, 132, 132, 132, 132, 132, + 132, 132, 78, 78, 32, 32, 76, 76, 77, 79, + 79, 75, 75, 75, 60, 60, 60, 60, 60, 60, + 60, 60, 62, 62, 62, 80, 80, 81, 81, 82, + 82, 83, 83, 84, 85, 85, 85, 86, 86, 86, + 86, 87, 87, 87, 59, 59, 59, 59, 59, 59, + 88, 88, 88, 88, 92, 92, 70, 70, 72, 72, + 71, 73, 93, 93, 97, 94, 94, 98, 98, 98, + 96, 96, 96, 121, 121, 121, 101, 101, 108, 108, + 109, 109, 102, 102, 110, 110, 110, 110, 110, 110, + 110, 110, 110, 110, 111, 111, 111, 112, 112, 113, + 113, 113, 120, 120, 116, 116, 117, 117, 122, 122, + 123, 123, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 114, 114, 114, + 114, 114, 114, 114, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, @@ -1994,18 +2000,8 @@ var yyR1 = [...]int{ 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, - 115, 115, 115, 115, 115, 115, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 116, 116, 116, 116, 116, - 116, 116, 116, 116, 116, 180, 181, 128, 129, 129, - 129, + 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, + 115, 115, 115, 179, 180, 127, 128, 128, 128, } var yyR2 = [...]int{ @@ -2033,41 +2029,41 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 7, 1, 3, 8, 8, 5, 4, 6, 5, 4, 4, 3, 2, 3, 4, 4, 4, 4, 4, 4, 4, 4, 3, 3, 3, 3, - 4, 3, 6, 4, 2, 2, 4, 2, 2, 2, - 2, 3, 1, 1, 1, 0, 1, 0, 1, 0, - 2, 2, 0, 2, 2, 0, 1, 1, 2, 1, - 1, 2, 1, 1, 2, 2, 2, 2, 2, 0, - 2, 0, 2, 1, 2, 2, 0, 1, 1, 0, - 1, 0, 1, 0, 1, 1, 3, 1, 2, 3, - 5, 0, 1, 2, 1, 1, 0, 2, 1, 3, - 1, 1, 1, 3, 3, 3, 7, 1, 3, 1, - 3, 4, 4, 4, 3, 2, 4, 0, 1, 0, - 2, 0, 1, 0, 1, 2, 1, 1, 1, 2, - 2, 1, 2, 3, 2, 3, 2, 2, 2, 1, - 1, 3, 0, 5, 5, 5, 0, 2, 1, 3, - 3, 2, 3, 1, 2, 0, 3, 1, 1, 3, - 3, 4, 4, 5, 3, 4, 5, 6, 2, 1, - 2, 1, 2, 1, 2, 1, 1, 1, 1, 1, - 1, 1, 0, 2, 1, 1, 1, 3, 1, 3, - 1, 1, 1, 1, 1, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 2, 2, 2, 2, 2, 2, 2, 3, 1, 1, - 1, 1, 4, 5, 6, 4, 4, 6, 6, 6, - 6, 8, 8, 6, 8, 8, 9, 7, 5, 4, - 2, 2, 2, 2, 2, 2, 2, 2, 0, 2, - 4, 4, 4, 4, 0, 3, 4, 7, 3, 1, - 1, 2, 3, 3, 1, 2, 2, 1, 2, 1, - 2, 2, 1, 2, 0, 1, 0, 2, 1, 2, - 4, 0, 2, 1, 3, 5, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 2, 2, 0, 3, 0, - 2, 0, 3, 1, 3, 2, 0, 1, 1, 0, - 2, 4, 4, 0, 2, 4, 2, 1, 3, 5, - 4, 6, 1, 3, 3, 5, 0, 5, 1, 3, - 1, 2, 3, 1, 1, 3, 3, 1, 3, 3, - 3, 3, 1, 2, 1, 1, 1, 1, 1, 1, - 0, 2, 0, 3, 0, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, - 1, 0, 1, 1, 0, 2, 1, 1, 1, 1, + 4, 3, 7, 5, 4, 2, 2, 4, 2, 2, + 2, 2, 3, 1, 1, 0, 1, 0, 2, 2, + 0, 2, 2, 0, 1, 1, 2, 1, 1, 2, + 1, 1, 2, 2, 2, 2, 2, 0, 2, 0, + 2, 1, 2, 2, 0, 1, 1, 0, 1, 0, + 1, 0, 1, 1, 3, 1, 2, 3, 5, 0, + 1, 2, 1, 1, 0, 2, 1, 3, 1, 1, + 1, 3, 3, 3, 7, 1, 3, 1, 3, 4, + 4, 4, 3, 2, 4, 0, 1, 0, 2, 0, + 1, 0, 1, 2, 1, 1, 1, 2, 2, 1, + 2, 3, 2, 3, 2, 2, 2, 1, 1, 3, + 0, 5, 5, 5, 0, 2, 1, 3, 3, 2, + 3, 1, 2, 0, 3, 1, 1, 3, 3, 4, + 4, 5, 3, 4, 5, 6, 2, 1, 2, 1, + 2, 1, 2, 1, 1, 1, 1, 1, 1, 1, + 0, 2, 1, 1, 1, 3, 1, 3, 1, 1, + 1, 1, 1, 3, 3, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 2, 2, + 2, 2, 2, 2, 2, 3, 1, 1, 1, 1, + 4, 5, 6, 4, 4, 6, 6, 6, 6, 8, + 8, 6, 8, 8, 9, 7, 5, 4, 2, 2, + 2, 2, 2, 2, 2, 2, 0, 2, 4, 4, + 4, 4, 0, 3, 4, 7, 3, 1, 1, 2, + 3, 3, 1, 2, 2, 1, 2, 1, 2, 2, + 1, 2, 0, 1, 0, 2, 1, 2, 4, 0, + 2, 1, 3, 5, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 2, 2, 0, 3, 0, 2, 0, + 3, 1, 3, 2, 0, 1, 1, 0, 2, 4, + 4, 0, 2, 4, 2, 1, 3, 5, 4, 6, + 1, 3, 3, 5, 0, 5, 1, 3, 1, 2, + 3, 1, 1, 3, 3, 1, 3, 3, 3, 3, + 1, 2, 1, 1, 1, 1, 1, 1, 0, 2, + 0, 3, 0, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 0, 1, 1, 1, 1, 0, + 1, 1, 0, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, @@ -2087,274 +2083,273 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 0, 0, 1, - 1, + 1, 1, 1, 1, 1, 0, 0, 1, 1, } var yyChk = [...]int{ - -1000, -178, -1, -2, -6, -7, -8, -9, -10, -11, + -1000, -177, -1, -2, -6, -7, -8, -9, -10, -11, -12, -13, -14, -15, -17, -18, -19, -21, -22, -23, -20, -3, -4, 6, 7, -27, 9, 10, 30, -16, 113, 114, 116, 115, 141, 117, 134, 49, 153, 154, - 156, 157, 25, 135, 136, 139, 140, -180, 8, 238, - 53, -179, 253, -82, 15, -26, 5, -24, -183, -24, - -24, -24, -24, -24, -159, 53, -114, 122, 70, 149, - 230, 119, 120, 126, -117, 56, -116, 246, 153, 164, - 158, 185, 177, 175, 178, 217, 205, 65, 156, 226, - 137, 173, 169, 167, 27, 190, 251, 168, 132, 131, - 191, 195, 218, 162, 163, 220, 189, 133, 32, 248, - 34, 145, 221, 193, 188, 184, 187, 161, 183, 38, - 197, 196, 198, 216, 180, 170, 18, 224, 140, 143, - 192, 194, 127, 147, 250, 222, 166, 144, 139, 225, - 157, 219, 228, 37, 202, 160, 130, 154, 151, 181, - 146, 171, 172, 186, 159, 182, 155, 148, 141, 227, - 203, 252, 179, 176, 152, 150, 208, 209, 210, 211, - 249, 223, 174, 204, -102, 122, 124, 120, 120, 121, - 122, 230, 119, 120, -51, -123, 56, -116, 122, 149, - 120, 106, 178, 113, 206, 121, 32, 147, -132, 120, - -104, 150, 205, 208, 209, 210, 211, 56, 219, 218, - 212, -123, 155, -128, -128, -128, -128, -128, -2, -86, - 17, 16, -5, -3, -180, 6, 20, 21, -30, 39, - 40, -25, -36, 97, -37, -123, -56, 72, -61, 29, - 56, -116, 23, -60, -57, -75, -73, -74, 106, 107, + 156, 157, 25, 135, 136, 139, 140, -179, 8, 237, + 53, -178, 252, -82, 15, -26, 5, -24, -182, -24, + -24, -24, -24, -24, -158, 53, -113, 122, 70, 149, + 229, 119, 120, 126, -116, 56, -115, 245, 153, 164, + 158, 185, 177, 175, 178, 216, 205, 65, 156, 225, + 137, 173, 169, 167, 27, 190, 250, 168, 132, 131, + 191, 195, 217, 162, 163, 219, 189, 133, 32, 247, + 34, 145, 220, 193, 188, 184, 187, 161, 183, 38, + 197, 196, 198, 215, 180, 170, 18, 223, 140, 143, + 192, 194, 127, 147, 249, 221, 166, 144, 139, 224, + 157, 218, 227, 37, 202, 160, 130, 154, 151, 181, + 146, 171, 172, 186, 159, 182, 155, 148, 141, 226, + 203, 251, 179, 176, 152, 150, 208, 209, 210, 211, + 248, 222, 174, 204, -102, 122, 124, 120, 120, 121, + 122, 229, 119, 120, -51, -122, 56, -115, 122, 149, + 120, 106, 178, 113, 206, 121, 32, 147, -131, 120, + -104, 150, 205, 208, 209, 210, 211, 56, 218, 217, + 212, -122, 155, -127, -127, -127, -127, -127, -2, -86, + 17, 16, -5, -3, -179, 6, 20, 21, -30, 39, + 40, -25, -36, 97, -37, -122, -56, 72, -61, 29, + 56, -115, 23, -60, -57, -75, -73, -74, 106, 107, 108, 95, 96, 103, 73, 109, -65, -63, -64, -66, 58, 57, 66, 59, 60, 61, 62, 67, 68, 69, - -117, -71, -180, 43, 44, 239, 240, 241, 242, 245, - 243, 75, 33, 229, 237, 236, 235, 233, 234, 231, - 232, 125, 230, 101, 238, -102, -39, -40, -41, -42, - -53, -74, -180, -51, 11, -46, -51, -94, -131, 155, - -98, 219, 218, -118, -96, -117, -115, 217, 178, 216, + -116, -71, -179, 43, 44, 238, 239, 240, 241, 244, + 242, 75, 33, 228, 236, 235, 234, 232, 233, 230, + 231, 125, 229, 101, 237, -102, -39, -40, -41, -42, + -53, -74, -179, -51, 11, -46, -51, -94, -130, 155, + -98, 218, 217, -117, -96, -116, -114, 216, 178, 215, 118, 71, 22, 24, 200, 74, 106, 16, 75, 105, - 239, 113, 47, 231, 232, 229, 241, 242, 230, 206, + 238, 113, 47, 230, 231, 228, 240, 241, 229, 206, 29, 10, 25, 135, 21, 99, 115, 78, 79, 138, 23, 136, 69, 19, 50, 11, 13, 14, 125, 124, 90, 121, 45, 8, 109, 26, 87, 41, 28, 43, - 88, 17, 233, 234, 31, 245, 142, 101, 48, 35, - 72, 67, 51, 70, 15, 46, 89, 116, 238, 44, - 119, 6, 244, 30, 134, 42, 120, 207, 77, 123, - 68, 5, 126, 9, 49, 52, 235, 236, 237, 33, - 76, 12, -160, -155, 56, 121, -51, 238, -117, -110, - 125, -110, -110, 120, -51, -51, -109, 125, 56, -109, - -109, -109, -51, 110, -51, 56, 30, 230, 56, 147, - 120, 148, 122, -129, -180, -118, -129, -129, -129, 151, - 152, -129, -105, 213, 51, -129, -181, 55, -87, 19, - 31, -37, -123, -83, -84, -37, -82, -2, -24, 35, - -28, 21, 64, 11, -120, 71, 70, 87, -119, 22, - -117, 58, 110, -37, -58, 90, 72, 88, 89, 74, - 92, 91, 102, 95, 96, 97, 98, 99, 100, 101, - 93, 94, 105, 80, 81, 82, 83, 84, 85, 86, - -103, -180, -74, -180, 111, 112, -61, -61, -61, -61, - -61, -61, -61, -61, -180, -2, -69, -37, -180, -180, - -180, -180, -180, -180, -180, -180, -180, -78, -37, -180, - -184, -180, -184, -184, -184, -184, -184, -184, -184, -180, - -180, -180, -180, -52, 26, -51, 30, 54, -47, -49, - -48, -50, 41, 45, 47, 42, 43, 44, 48, -127, - 22, -39, -180, -126, 143, -125, 22, -123, 58, -51, - -46, -182, 54, 11, 52, 54, -94, 155, -95, -99, - 220, 222, 80, -122, -117, 58, 29, 30, 55, 54, - -134, -137, -139, -138, -140, -135, -136, 175, 176, 106, - 179, 181, 182, 183, 184, 185, 186, 187, 188, 189, - 190, 30, 137, 171, 172, 173, 174, 191, 192, 193, - 194, 195, 196, 197, 198, 158, 159, 160, 161, 162, - 163, 164, 166, 167, 168, 169, 170, 56, -129, 122, - -176, 52, 56, 72, 56, -51, -51, -129, 123, -51, - 23, 51, -51, 56, 56, -124, -123, -115, -129, -129, - -129, -129, -129, -129, -129, -129, -129, -129, -107, 207, - 214, 215, -51, 9, 90, 54, 18, 110, 54, -85, - 24, 25, -86, -181, -30, -62, -117, 59, 62, -29, - 42, -51, -37, -37, -67, 67, 72, 68, 69, -119, - 97, -124, -118, -115, -61, -68, -71, -74, 63, 90, - 88, 89, 74, -61, -61, -61, -61, -61, -61, -61, - -61, -61, -61, -61, -61, -61, -61, -61, -130, 56, - 58, 56, -60, -60, -117, -35, 21, -34, -36, -181, - 54, -181, -2, -34, -34, -37, -37, -75, -117, -123, - -75, -34, -28, -76, -77, 76, -75, -181, -34, -35, - -34, -34, -90, 143, -51, -93, -97, -75, -40, -41, - -41, -40, -41, 41, 41, 41, 46, 41, 46, 41, - -48, -123, -181, -54, 49, 124, 50, -180, -125, -90, - 52, -39, -51, -98, -95, 54, 221, 223, 224, 51, - -37, -146, 105, -161, -162, -163, -118, 58, 59, -155, - -156, -164, 127, 130, 126, -157, 121, 28, -151, 67, - 72, -147, 203, -141, 53, -141, -141, -141, -141, -145, - 178, -145, -145, -145, 53, 53, -141, -141, -141, -149, - 53, -149, -149, -150, 53, -150, -121, 52, -51, -174, - 249, -175, 56, -129, 23, -129, -111, 118, 115, 116, - -171, 114, 200, 178, 65, 29, 15, 239, 143, 252, - 56, 144, -51, -51, -129, -106, 11, 90, 37, -37, - -37, -124, -84, -87, -101, 19, 11, 33, 33, -34, - 67, 68, 69, 110, -180, -68, -61, -61, -61, -33, - 138, 71, -181, -181, -34, 54, -37, -181, -181, -181, - 54, 52, 22, 54, 11, 110, 54, 11, -181, -34, - -79, -77, 78, -37, -181, -181, -181, -181, -181, -59, - 30, 33, -2, -180, -180, -55, 54, 12, 80, -44, - -43, 51, 52, -45, 51, -43, 41, 41, 121, 121, - 121, -91, -117, -55, -39, -55, -99, -100, 225, 222, - 228, 56, 54, -163, 80, 53, 28, -157, -157, 56, - 56, -142, 29, 67, -148, 204, 59, -145, -145, -146, - 30, -146, -146, -146, -154, 58, -154, 59, 59, 51, - -117, -129, -173, -172, -118, -128, -177, 149, 128, 129, - 132, 131, 56, 121, 28, 127, 130, 143, 126, -177, - 149, -112, -113, 123, 22, 121, 28, 143, -129, -108, - 88, 12, -123, -123, 38, 110, -51, -38, 11, 97, - -118, -35, -33, 71, -61, -61, -181, -36, -133, 106, - 175, 137, 173, 169, 189, 180, 202, 171, 203, -130, - -133, -61, -61, -118, -61, -61, 246, -82, 79, -37, - 77, -92, 51, -93, -70, -72, -71, -180, -2, -88, - -117, -91, -82, -97, -37, -37, -37, 53, -37, -180, - -180, -180, -181, 54, -82, -55, 222, 226, 227, -162, - -163, -166, -165, -117, 56, 56, -144, 51, 58, 59, - 60, 67, 229, 66, 55, -146, -146, 56, 106, 55, - 54, 55, 54, 55, 54, -51, 54, 80, -128, -117, - -128, -117, -51, -128, -117, 58, -37, -55, -39, -181, - -61, -181, -141, -141, -141, -150, -141, 163, -141, 163, - -181, -181, -181, 54, 19, -181, 54, 19, -180, -32, - 244, -37, 27, -92, 54, -181, -181, -181, 54, 110, - -181, -86, -89, -117, -89, -89, -89, -126, -117, -86, - 55, 54, -141, -152, 200, 9, -145, 58, -145, 59, - 59, -129, -172, -163, 53, 26, -80, 13, -145, 56, - -61, -61, -61, -61, -61, -181, 58, 28, -72, 33, - -2, -180, -117, -117, 54, 55, -181, -181, -181, -54, - -168, -167, 52, 133, 65, -165, -153, 127, 28, 126, - 229, -146, -146, 55, 55, -89, -180, -81, 14, 16, - -181, -181, -181, -181, -31, 90, 249, 9, -70, -2, - 110, -117, -167, 56, -158, 80, 58, -143, 65, 28, - 28, 55, -169, -170, 143, -37, -69, -181, 247, 48, - 250, -93, -181, -117, 59, 58, -176, -181, 54, -117, - 38, 248, 251, -174, -170, 33, 38, 145, 249, 146, - 250, -180, 251, -61, 142, -181, -181, + 88, 17, 232, 233, 31, 244, 142, 101, 48, 35, + 72, 67, 51, 70, 15, 46, 89, 116, 237, 44, + 119, 6, 243, 30, 134, 42, 120, 207, 77, 123, + 68, 5, 126, 9, 49, 52, 234, 235, 236, 33, + 76, 12, -159, -154, 56, 121, -51, 237, -116, -109, + 125, -109, -109, 120, -51, -51, -108, 125, 56, -108, + -108, -108, -51, 110, -51, 56, 30, 229, 56, 147, + 120, 148, 122, -128, -179, -117, -128, -128, -128, 151, + 152, -128, 214, -106, 207, 213, 51, -128, -180, 55, + -87, 19, 31, -37, -122, -83, -84, -37, -82, -2, + -24, 35, -28, 21, 64, 11, -119, 71, 70, 87, + -118, 22, -116, 58, 110, -37, -58, 90, 72, 88, + 89, 74, 92, 91, 102, 95, 96, 97, 98, 99, + 100, 101, 93, 94, 105, 80, 81, 82, 83, 84, + 85, 86, -103, -179, -74, -179, 111, 112, -61, -61, + -61, -61, -61, -61, -61, -61, -179, -2, -69, -37, + -179, -179, -179, -179, -179, -179, -179, -179, -179, -78, + -37, -179, -183, -179, -183, -183, -183, -183, -183, -183, + -183, -179, -179, -179, -179, -52, 26, -51, 30, 54, + -47, -49, -48, -50, 41, 45, 47, 42, 43, 44, + 48, -126, 22, -39, -179, -125, 143, -124, 22, -122, + 58, -51, -46, -181, 54, 11, 52, 54, -94, 155, + -95, -99, 219, 221, 80, -121, -116, 58, 29, 30, + 55, 54, -133, -136, -138, -137, -139, -134, -135, 175, + 176, 106, 179, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 30, 137, 171, 172, 173, 174, 191, + 192, 193, 194, 195, 196, 197, 198, 158, 159, 160, + 161, 162, 163, 164, 166, 167, 168, 169, 170, 56, + -128, 122, -175, 52, 56, 72, 56, -51, -51, -128, + 123, -51, 23, 51, -51, 56, 56, -123, -122, -114, + -128, -128, -128, -128, -128, -128, -128, -128, -128, -128, + 11, -105, 11, 90, -51, 9, 90, 54, 18, 110, + 54, -85, 24, 25, -86, -180, -30, -62, -116, 59, + 62, -29, 42, -51, -37, -37, -67, 67, 72, 68, + 69, -118, 97, -123, -117, -114, -61, -68, -71, -74, + 63, 90, 88, 89, 74, -61, -61, -61, -61, -61, + -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, + -129, 56, 58, 56, -60, -60, -116, -35, 21, -34, + -36, -180, 54, -180, -2, -34, -34, -37, -37, -75, + -116, -122, -75, -34, -28, -76, -77, 76, -75, -180, + -34, -35, -34, -34, -90, 143, -51, -93, -97, -75, + -40, -41, -41, -40, -41, 41, 41, 41, 46, 41, + 46, 41, -48, -122, -180, -54, 49, 124, 50, -179, + -124, -90, 52, -39, -51, -98, -95, 54, 220, 222, + 223, 51, -37, -145, 105, -160, -161, -162, -117, 58, + 59, -154, -155, -163, 127, 130, 126, -156, 121, 28, + -150, 67, 72, -146, 203, -140, 53, -140, -140, -140, + -140, -144, 178, -144, -144, -144, 53, 53, -140, -140, + -140, -148, 53, -148, -148, -149, 53, -149, -120, 52, + -51, -173, 248, -174, 56, -128, 23, -128, -110, 118, + 115, 116, -170, 114, 200, 178, 65, 29, 15, 238, + 143, 251, 56, 144, -51, -51, -128, -51, -107, 88, + 12, -122, -122, 37, -37, -37, -123, -84, -87, -101, + 19, 11, 33, 33, -34, 67, 68, 69, 110, -179, + -68, -61, -61, -61, -33, 138, 71, -180, -180, -34, + 54, -37, -180, -180, -180, 54, 52, 22, 54, 11, + 110, 54, 11, -180, -34, -79, -77, 78, -37, -180, + -180, -180, -180, -180, -59, 30, 33, -2, -179, -179, + -55, 54, 12, 80, -44, -43, 51, 52, -45, 51, + -43, 41, 41, 121, 121, 121, -91, -116, -55, -39, + -55, -99, -100, 224, 221, 227, 56, 54, -162, 80, + 53, 28, -156, -156, 56, 56, -141, 29, 67, -147, + 204, 59, -144, -144, -145, 30, -145, -145, -145, -153, + 58, -153, 59, 59, 51, -116, -128, -172, -171, -117, + -127, -176, 149, 128, 129, 132, 131, 56, 121, 28, + 127, 130, 143, 126, -176, 149, -111, -112, 123, 22, + 121, 28, 143, -128, -105, 58, -37, 38, 110, -51, + -38, 11, 97, -117, -35, -33, 71, -61, -61, -180, + -36, -132, 106, 175, 137, 173, 169, 189, 180, 202, + 171, 203, -129, -132, -61, -61, -117, -61, -61, 245, + -82, 79, -37, 77, -92, 51, -93, -70, -72, -71, + -179, -2, -88, -116, -91, -82, -97, -37, -37, -37, + 53, -37, -179, -179, -179, -180, 54, -82, -55, 221, + 225, 226, -161, -162, -165, -164, -116, 56, 56, -143, + 51, 58, 59, 60, 67, 228, 66, 55, -145, -145, + 56, 106, 55, 54, 55, 54, 55, 54, -51, 54, + 80, -127, -116, -127, -116, -51, -127, -116, -107, -55, + -39, -180, -61, -180, -140, -140, -140, -149, -140, 163, + -140, 163, -180, -180, -180, 54, 19, -180, 54, 19, + -179, -32, 243, -37, 27, -92, 54, -180, -180, -180, + 54, 110, -180, -86, -89, -116, -89, -89, -89, -125, + -116, -86, 55, 54, -140, -151, 200, 9, -144, 58, + -144, 59, 59, -128, -171, -162, 53, 26, -80, 13, + -144, 56, -61, -61, -61, -61, -61, -180, 58, 28, + -72, 33, -2, -179, -116, -116, 54, 55, -180, -180, + -180, -54, -167, -166, 52, 133, 65, -164, -152, 127, + 28, 126, 228, -145, -145, 55, 55, -89, -179, -81, + 14, 16, -180, -180, -180, -180, -31, 90, 248, 9, + -70, -2, 110, -116, -166, 56, -157, 80, 58, -142, + 65, 28, 28, 55, -168, -169, 143, -37, -69, -180, + 246, 48, 249, -93, -180, -116, 59, 58, -175, -180, + 54, -116, 38, 247, 250, -173, -169, 33, 38, 145, + 248, 146, 249, -179, 250, -61, 142, -180, -180, } var yyDef = [...]int{ 0, -2, 2, -2, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, - 21, 511, 0, 279, 279, 279, 279, 279, 279, 0, - 581, 564, 0, 0, 0, 0, -2, 269, 270, 0, - 272, 273, 787, 787, 787, 787, 787, 0, 33, 34, - 785, 1, 3, 519, 0, 0, 283, 286, 281, 0, - 564, 0, 0, 0, 60, 0, 0, 774, 0, 775, - 562, 562, 562, 582, 583, 586, 587, 686, 687, 688, - 689, 690, 691, 692, 693, 694, 695, 696, 697, 698, - 699, 700, 701, 702, 703, 704, 705, 706, 707, 708, - 709, 710, 711, 712, 713, 714, 715, 716, 717, 718, - 719, 720, 721, 722, 723, 724, 725, 726, 727, 728, - 729, 730, 731, 732, 733, 734, 735, 736, 737, 738, - 739, 740, 741, 742, 743, 744, 745, 746, 747, 748, - 749, 750, 751, 752, 753, 754, 755, 756, 757, 758, - 759, 760, 761, 762, 763, 764, 765, 766, 767, 768, - 769, 770, 771, 772, 773, 776, 777, 778, 779, 780, - 781, 782, 783, 784, 0, 0, 565, 0, 560, 0, - 560, 560, 560, 0, 226, 350, 590, 591, 774, 775, - 0, 0, 0, 0, 788, 788, 788, 788, 0, 788, - 257, 244, 245, 247, 248, 249, 250, 788, 266, 267, - 256, 268, 271, 274, 275, 276, 277, 278, 27, 523, - 0, 0, 511, 29, 0, 279, 284, 285, 289, 287, - 288, 280, 0, 297, 301, 0, 358, 0, 363, 365, - -2, -2, 0, 400, 401, 402, 403, 404, 0, 0, - 0, 0, 0, 0, 0, 0, 428, 429, 430, 431, - 496, 497, 498, 499, 500, 501, 502, 503, 367, 368, - 493, 543, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 484, 0, 458, 458, 458, 458, 458, 458, 458, - 458, 0, 0, 0, 0, 0, 0, 308, 310, 311, - 312, 331, 0, 333, 0, 0, 41, 45, 0, 765, - 547, -2, -2, 0, 0, 588, 589, -2, 693, -2, - 594, 595, 596, 597, 598, 599, 600, 601, 602, 603, - 604, 605, 606, 607, 608, 609, 610, 611, 612, 613, - 614, 615, 616, 617, 618, 619, 620, 621, 622, 623, - 624, 625, 626, 627, 628, 629, 630, 631, 632, 633, - 634, 635, 636, 637, 638, 639, 640, 641, 642, 643, - 644, 645, 646, 647, 648, 649, 650, 651, 652, 653, - 654, 655, 656, 657, 658, 659, 660, 661, 662, 663, - 664, 665, 666, 667, 668, 669, 670, 671, 672, 673, - 674, 675, 676, 677, 678, 679, 680, 681, 682, 683, - 684, 685, 0, 77, 0, 0, 788, 0, 67, 0, - 0, 0, 0, 0, 788, 0, 0, 0, 0, 0, - 0, 0, 225, 0, 227, 788, 788, 788, 788, 788, - 788, 788, 788, 236, 789, 790, 237, 238, 239, 788, - 788, 241, 0, 258, 0, 251, 28, 786, 22, 0, - 0, 520, 0, 512, 513, 516, 519, 27, 286, 0, - 291, 290, 282, 0, 298, 0, 0, 0, 302, 0, - 304, 305, 0, 361, 0, 0, 0, 0, 0, 0, + 21, 509, 0, 277, 277, 277, 277, 277, 277, 0, + 579, 562, 0, 0, 0, 0, -2, 267, 268, 0, + 270, 271, 785, 785, 785, 785, 785, 0, 33, 34, + 783, 1, 3, 517, 0, 0, 281, 284, 279, 0, + 562, 0, 0, 0, 60, 0, 0, 772, 0, 773, + 560, 560, 560, 580, 581, 584, 585, 684, 685, 686, + 687, 688, 689, 690, 691, 692, 693, 694, 695, 696, + 697, 698, 699, 700, 701, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 713, 714, 715, 716, + 717, 718, 719, 720, 721, 722, 723, 724, 725, 726, + 727, 728, 729, 730, 731, 732, 733, 734, 735, 736, + 737, 738, 739, 740, 741, 742, 743, 744, 745, 746, + 747, 748, 749, 750, 751, 752, 753, 754, 755, 756, + 757, 758, 759, 760, 761, 762, 763, 764, 765, 766, + 767, 768, 769, 770, 771, 774, 775, 776, 777, 778, + 779, 780, 781, 782, 0, 0, 563, 0, 558, 0, + 558, 558, 558, 0, 226, 348, 588, 589, 772, 773, + 0, 0, 0, 0, 786, 786, 786, 786, 0, 786, + 0, 245, 246, 248, 249, 250, 251, 786, 264, 265, + 256, 266, 269, 272, 273, 274, 275, 276, 27, 521, + 0, 0, 509, 29, 0, 277, 282, 283, 287, 285, + 286, 278, 0, 295, 299, 0, 356, 0, 361, 363, + -2, -2, 0, 398, 399, 400, 401, 402, 0, 0, + 0, 0, 0, 0, 0, 0, 426, 427, 428, 429, + 494, 495, 496, 497, 498, 499, 500, 501, 365, 366, + 491, 541, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 482, 0, 456, 456, 456, 456, 456, 456, 456, + 456, 0, 0, 0, 0, 0, 0, 306, 308, 309, + 310, 329, 0, 331, 0, 0, 41, 45, 0, 763, + 545, -2, -2, 0, 0, 586, 587, -2, 691, -2, + 592, 593, 594, 595, 596, 597, 598, 599, 600, 601, + 602, 603, 604, 605, 606, 607, 608, 609, 610, 611, + 612, 613, 614, 615, 616, 617, 618, 619, 620, 621, + 622, 623, 624, 625, 626, 627, 628, 629, 630, 631, + 632, 633, 634, 635, 636, 637, 638, 639, 640, 641, + 642, 643, 644, 645, 646, 647, 648, 649, 650, 651, + 652, 653, 654, 655, 656, 657, 658, 659, 660, 661, + 662, 663, 664, 665, 666, 667, 668, 669, 670, 671, + 672, 673, 674, 675, 676, 677, 678, 679, 680, 681, + 682, 683, 0, 77, 0, 0, 786, 0, 67, 0, + 0, 0, 0, 0, 786, 0, 0, 0, 0, 0, + 0, 0, 225, 0, 227, 786, 786, 786, 786, 786, + 786, 786, 786, 236, 787, 788, 237, 238, 239, 786, + 786, 241, 0, 257, 253, 254, 0, 252, 28, 784, + 22, 0, 0, 518, 0, 510, 511, 514, 517, 27, + 284, 0, 289, 288, 280, 0, 296, 0, 0, 0, + 300, 0, 302, 303, 0, 359, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 385, 386, 387, 388, 389, 390, 391, - 364, 0, 378, 0, 0, 0, 420, 421, 422, 423, - 424, 425, 426, 0, 293, 27, 0, 398, 0, 0, - 0, 0, 0, 0, 0, 0, 289, 0, 485, 0, - 450, 0, 451, 452, 453, 454, 455, 456, 457, 0, - 293, 0, 0, 43, 0, 349, 0, 0, 0, 0, - 0, 0, 338, 0, 0, 341, 0, 0, 0, 0, - 332, 0, 0, 352, 738, 334, 0, 336, 337, -2, - 0, 0, 0, 39, 40, 0, 46, 765, 48, 49, - 0, 0, 0, 157, 555, 556, 557, 553, 185, 0, - 140, 136, 82, 83, 84, 129, 86, 129, 129, 129, - 129, 154, 154, 154, 154, 112, 113, 114, 115, 116, - 0, 0, 99, 129, 129, 129, 103, 119, 120, 121, - 122, 123, 124, 125, 126, 87, 88, 89, 90, 91, - 92, 93, 131, 131, 131, 133, 133, 584, 62, 0, - 70, 0, 788, 0, 788, 75, 0, 201, 0, 220, - 561, 0, 788, 223, 224, 351, 592, 593, 228, 229, - 230, 231, 232, 233, 234, 235, 240, 243, 259, 252, - 253, 254, 246, 524, 0, 0, 0, 0, 0, 515, - 517, 518, 523, 30, 289, 0, 504, 0, 0, 0, - 292, 25, 359, 360, 362, 379, 0, 381, 383, 303, - 299, 0, 494, -2, 369, 370, 394, 395, 396, 0, - 0, 0, 0, 392, 374, 0, 405, 406, 407, 408, - 409, 410, 411, 412, 413, 414, 415, 416, 419, 469, - 470, 0, 417, 418, 427, 0, 0, 294, 295, 397, - 0, 542, 27, 0, 0, 0, 0, 0, 493, 0, - 0, 0, 0, 491, 488, 0, 0, 459, 0, 0, - 0, 0, 0, 0, 348, 356, 544, 0, 309, 327, - 329, 0, 324, 339, 340, 342, 0, 344, 0, 346, - 347, 313, 314, 315, 0, 0, 0, 0, 335, 356, - 0, 356, 42, 548, 47, 0, 0, 52, 53, 549, - 550, 551, 0, 76, 186, 188, 191, 192, 193, 78, - 79, 0, 0, 0, 0, 0, 180, 181, 143, 141, - 0, 138, 137, 85, 0, 154, 154, 106, 107, 157, - 0, 157, 157, 157, 0, 0, 100, 101, 102, 94, - 0, 95, 96, 97, 0, 98, 0, 0, 788, 64, - 0, 68, 69, 65, 563, 66, 787, 0, 0, 576, - 202, 566, 567, 568, 569, 570, 571, 572, 573, 574, - 575, 0, 219, 788, 222, 262, 0, 0, 0, 521, - 522, 0, 514, 23, 0, 558, 559, 505, 506, 306, - 380, 382, 384, 0, 293, 371, 392, 375, 0, 372, - 0, 0, 366, 432, 0, 0, 399, -2, 435, 436, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 511, - 0, 489, 0, 0, 449, 460, 461, 462, 463, 536, - 0, 0, -2, 0, 0, 511, 0, 0, 0, 321, - 328, 0, 0, 322, 0, 323, 343, 345, 0, 0, - 0, 0, 319, 511, 356, 38, 50, 51, 0, 0, - 57, 158, 0, 189, 0, 0, 175, 0, 0, 178, - 179, 150, 0, 142, 81, 139, 0, 157, 157, 108, - 0, 109, 110, 111, 0, 127, 0, 0, 0, 0, - 585, 63, 71, 72, 0, 194, 787, 0, 203, 204, - 205, 206, 207, 208, 209, 210, 211, 212, 213, 787, - 0, 0, 787, 577, 578, 579, 580, 0, 221, 242, - 0, 0, 260, 261, 525, 0, 24, 356, 0, 300, - 495, 0, 373, 0, 393, 376, 433, 296, 0, 129, - 129, 474, 129, 133, 477, 129, 479, 129, 482, 0, - 0, 0, 0, 494, 0, 0, 0, 486, 448, 492, - 0, 31, 0, 536, 526, 538, 540, 0, 27, 0, - 532, 0, 519, 545, 357, 546, 325, 0, 330, 0, - 0, 0, 333, 0, 519, 37, 54, 55, 56, 187, - 190, 0, 182, 129, 176, 177, 152, 0, 144, 145, - 146, 147, 148, 149, 130, 104, 105, 155, 156, 154, - 0, 154, 0, 134, 0, 788, 0, 0, 195, 0, - 196, 198, 199, 200, 0, 263, 264, 507, 307, 434, - 377, 437, 471, 154, 475, 476, 478, 480, 481, 483, - 439, 438, 440, 0, 0, 443, 0, 0, 0, 0, - 0, 490, 0, 32, 0, 541, -2, 0, 0, 0, - 44, 35, 0, 317, 0, 0, 0, 352, 320, 36, - 167, 0, 184, 159, 153, 0, 157, 128, 157, 0, - 0, 61, 73, 74, 0, 0, 509, 0, 472, 473, - 0, 0, 0, 0, 464, 447, 487, 0, 539, 0, - -2, 0, 534, 533, 0, 326, 353, 354, 355, 316, - 166, 168, 0, 173, 0, 183, 164, 0, 161, 163, - 151, 117, 118, 132, 135, 0, 0, 26, 0, 0, - 441, 442, 444, 445, 0, 0, 0, 0, 529, 27, - 0, 318, 169, 170, 0, 174, 172, 80, 0, 160, - 162, 67, 0, 215, 0, 510, 508, 446, 0, 0, - 0, 537, -2, 535, 171, 165, 70, 214, 0, 0, - 465, 0, 468, 197, 216, 0, 466, 0, 0, 0, - 0, 0, 467, 0, 0, 217, 218, + 0, 0, 0, 0, 0, 383, 384, 385, 386, 387, + 388, 389, 362, 0, 376, 0, 0, 0, 418, 419, + 420, 421, 422, 423, 424, 0, 291, 27, 0, 396, + 0, 0, 0, 0, 0, 0, 0, 0, 287, 0, + 483, 0, 448, 0, 449, 450, 451, 452, 453, 454, + 455, 0, 291, 0, 0, 43, 0, 347, 0, 0, + 0, 0, 0, 0, 336, 0, 0, 339, 0, 0, + 0, 0, 330, 0, 0, 350, 736, 332, 0, 334, + 335, -2, 0, 0, 0, 39, 40, 0, 46, 763, + 48, 49, 0, 0, 0, 157, 553, 554, 555, 551, + 185, 0, 140, 136, 82, 83, 84, 129, 86, 129, + 129, 129, 129, 154, 154, 154, 154, 112, 113, 114, + 115, 116, 0, 0, 99, 129, 129, 129, 103, 119, + 120, 121, 122, 123, 124, 125, 126, 87, 88, 89, + 90, 91, 92, 93, 131, 131, 131, 133, 133, 582, + 62, 0, 70, 0, 786, 0, 786, 75, 0, 201, + 0, 220, 559, 0, 786, 223, 224, 349, 590, 591, + 228, 229, 230, 231, 232, 233, 234, 235, 240, 244, + 0, 260, 0, 0, 247, 522, 0, 0, 0, 0, + 0, 513, 515, 516, 521, 30, 287, 0, 502, 0, + 0, 0, 290, 25, 357, 358, 360, 377, 0, 379, + 381, 301, 297, 0, 492, -2, 367, 368, 392, 393, + 394, 0, 0, 0, 0, 390, 372, 0, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 417, 467, 468, 0, 415, 416, 425, 0, 0, 292, + 293, 395, 0, 540, 27, 0, 0, 0, 0, 0, + 491, 0, 0, 0, 0, 489, 486, 0, 0, 457, + 0, 0, 0, 0, 0, 0, 346, 354, 542, 0, + 307, 325, 327, 0, 322, 337, 338, 340, 0, 342, + 0, 344, 345, 311, 312, 313, 0, 0, 0, 0, + 333, 354, 0, 354, 42, 546, 47, 0, 0, 52, + 53, 547, 548, 549, 0, 76, 186, 188, 191, 192, + 193, 78, 79, 0, 0, 0, 0, 0, 180, 181, + 143, 141, 0, 138, 137, 85, 0, 154, 154, 106, + 107, 157, 0, 157, 157, 157, 0, 0, 100, 101, + 102, 94, 0, 95, 96, 97, 0, 98, 0, 0, + 786, 64, 0, 68, 69, 65, 561, 66, 785, 0, + 0, 574, 202, 564, 565, 566, 567, 568, 569, 570, + 571, 572, 573, 0, 219, 786, 222, 257, 243, 0, + 0, 258, 259, 0, 519, 520, 0, 512, 23, 0, + 556, 557, 503, 504, 304, 378, 380, 382, 0, 291, + 369, 390, 373, 0, 370, 0, 0, 364, 430, 0, + 0, 397, -2, 433, 434, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 509, 0, 487, 0, 0, 447, + 458, 459, 460, 461, 534, 0, 0, -2, 0, 0, + 509, 0, 0, 0, 319, 326, 0, 0, 320, 0, + 321, 341, 343, 0, 0, 0, 0, 317, 509, 354, + 38, 50, 51, 0, 0, 57, 158, 0, 189, 0, + 0, 175, 0, 0, 178, 179, 150, 0, 142, 81, + 139, 0, 157, 157, 108, 0, 109, 110, 111, 0, + 127, 0, 0, 0, 0, 583, 63, 71, 72, 0, + 194, 785, 0, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, 213, 785, 0, 0, 785, 575, 576, + 577, 578, 0, 221, 260, 261, 262, 523, 0, 24, + 354, 0, 298, 493, 0, 371, 0, 391, 374, 431, + 294, 0, 129, 129, 472, 129, 133, 475, 129, 477, + 129, 480, 0, 0, 0, 0, 492, 0, 0, 0, + 484, 446, 490, 0, 31, 0, 534, 524, 536, 538, + 0, 27, 0, 530, 0, 517, 543, 355, 544, 323, + 0, 328, 0, 0, 0, 331, 0, 517, 37, 54, + 55, 56, 187, 190, 0, 182, 129, 176, 177, 152, + 0, 144, 145, 146, 147, 148, 149, 130, 104, 105, + 155, 156, 154, 0, 154, 0, 134, 0, 786, 0, + 0, 195, 0, 196, 198, 199, 200, 0, 242, 505, + 305, 432, 375, 435, 469, 154, 473, 474, 476, 478, + 479, 481, 437, 436, 438, 0, 0, 441, 0, 0, + 0, 0, 0, 488, 0, 32, 0, 539, -2, 0, + 0, 0, 44, 35, 0, 315, 0, 0, 0, 350, + 318, 36, 167, 0, 184, 159, 153, 0, 157, 128, + 157, 0, 0, 61, 73, 74, 0, 0, 507, 0, + 470, 471, 0, 0, 0, 0, 462, 445, 485, 0, + 537, 0, -2, 0, 532, 531, 0, 324, 351, 352, + 353, 314, 166, 168, 0, 173, 0, 183, 164, 0, + 161, 163, 151, 117, 118, 132, 135, 0, 0, 26, + 0, 0, 439, 440, 442, 443, 0, 0, 0, 0, + 527, 27, 0, 316, 169, 170, 0, 174, 172, 80, + 0, 160, 162, 67, 0, 215, 0, 508, 506, 444, + 0, 0, 0, 535, -2, 533, 171, 165, 70, 214, + 0, 0, 463, 0, 466, 197, 216, 0, 464, 0, + 0, 0, 0, 0, 465, 0, 0, 217, 218, } var yyTok1 = [...]int{ @@ -2363,7 +2358,7 @@ var yyTok1 = [...]int{ 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 73, 3, 3, 3, 100, 92, 3, 53, 55, 97, 95, 54, 96, 110, 98, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 253, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 252, 81, 80, 82, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, @@ -2397,7 +2392,7 @@ var yyTok2 = [...]int{ 219, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, + 249, 250, 251, } var yyTok3 = [...]int{ 0, @@ -4081,621 +4076,610 @@ yydefault: yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 242: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-7 : yypt+1] //line sql.y:1375 + { + showTablesOpt := &ShowTablesOpt{Full: yyDollar[2].str, DbName: yyDollar[6].str, Filter: yyDollar[7].showFilter} + yyVAL.statement = &Show{Type: string(yyDollar[3].bytes), ShowTablesOpt: showTablesOpt, OnTable: yyDollar[5].tableName} + } + case 243: + yyDollar = yyS[yypt-5 : yypt+1] + //line sql.y:1380 { // this is ugly, but I couldn't find a better way for now - if yyDollar[4].str == "processlist" { - yyVAL.statement = &Show{Type: yyDollar[4].str} + if yyDollar[3].str == "processlist" { + yyVAL.statement = &Show{Type: yyDollar[3].str} } else { - showTablesOpt := &ShowTablesOpt{Extended: yyDollar[2].str, Full: yyDollar[3].str, DbName: yyDollar[5].str, Filter: yyDollar[6].showFilter} - yyVAL.statement = &Show{Type: yyDollar[4].str, ShowTablesOpt: showTablesOpt} + showTablesOpt := &ShowTablesOpt{Full: yyDollar[2].str, DbName: yyDollar[4].str, Filter: yyDollar[5].showFilter} + yyVAL.statement = &Show{Type: yyDollar[3].str, ShowTablesOpt: showTablesOpt} } } - case 243: + case 244: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1385 + //line sql.y:1390 { yyVAL.statement = &Show{Scope: yyDollar[2].str, Type: string(yyDollar[3].bytes)} } - case 244: + case 245: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1389 + //line sql.y:1394 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } - case 245: + case 246: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1393 + //line sql.y:1398 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } - case 246: + case 247: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1397 + //line sql.y:1402 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes), OnTable: yyDollar[4].tableName} } - case 247: - yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1401 - { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} - } case 248: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1405 + //line sql.y:1406 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 249: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1409 + //line sql.y:1410 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 250: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1413 + //line sql.y:1414 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 251: - yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1423 + yyDollar = yyS[yypt-2 : yypt+1] + //line sql.y:1418 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 252: - yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1429 + yyDollar = yyS[yypt-3 : yypt+1] + //line sql.y:1428 { - yyVAL.str = string(yyDollar[1].bytes) + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 253: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1433 + //line sql.y:1434 { yyVAL.str = string(yyDollar[1].bytes) } case 254: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1437 + //line sql.y:1438 { yyVAL.str = string(yyDollar[1].bytes) } case 255: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1443 + //line sql.y:1444 { yyVAL.str = "" } case 256: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1447 + //line sql.y:1448 { - yyVAL.str = "extended " + yyVAL.str = "full " } case 257: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1453 + //line sql.y:1454 { yyVAL.str = "" } case 258: - yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1457 - { - yyVAL.str = "full " - } - case 259: - yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1463 - { - yyVAL.str = "" - } - case 260: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1467 + //line sql.y:1458 { yyVAL.str = yyDollar[2].tableIdent.v } - case 261: + case 259: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1471 + //line sql.y:1462 { yyVAL.str = yyDollar[2].tableIdent.v } - case 262: + case 260: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1477 + //line sql.y:1468 { yyVAL.showFilter = nil } - case 263: + case 261: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1481 + //line sql.y:1472 { yyVAL.showFilter = &ShowFilter{Like: string(yyDollar[2].bytes)} } - case 264: + case 262: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1485 + //line sql.y:1476 { yyVAL.showFilter = &ShowFilter{Filter: yyDollar[2].expr} } - case 265: + case 263: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1491 + //line sql.y:1482 { yyVAL.str = "" } - case 266: + case 264: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1495 + //line sql.y:1486 { yyVAL.str = SessionStr } - case 267: + case 265: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1499 + //line sql.y:1490 { yyVAL.str = GlobalStr } - case 268: + case 266: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1505 + //line sql.y:1496 { yyVAL.statement = &Use{DBName: yyDollar[2].tableIdent} } - case 269: + case 267: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1509 + //line sql.y:1500 { yyVAL.statement = &Use{DBName: TableIdent{v: ""}} } - case 270: + case 268: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1515 + //line sql.y:1506 { yyVAL.statement = &Begin{} } - case 271: + case 269: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1519 + //line sql.y:1510 { yyVAL.statement = &Begin{} } - case 272: + case 270: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1525 + //line sql.y:1516 { yyVAL.statement = &Commit{} } - case 273: + case 271: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1531 + //line sql.y:1522 { yyVAL.statement = &Rollback{} } - case 274: + case 272: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1537 + //line sql.y:1528 { yyVAL.statement = &OtherRead{} } - case 275: + case 273: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1541 + //line sql.y:1532 { yyVAL.statement = &OtherRead{} } - case 276: + case 274: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1545 + //line sql.y:1536 { yyVAL.statement = &OtherRead{} } - case 277: + case 275: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1549 + //line sql.y:1540 { yyVAL.statement = &OtherAdmin{} } - case 278: + case 276: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1553 + //line sql.y:1544 { yyVAL.statement = &OtherAdmin{} } - case 279: + case 277: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1558 + //line sql.y:1549 { setAllowComments(yylex, true) } - case 280: + case 278: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1562 + //line sql.y:1553 { yyVAL.bytes2 = yyDollar[2].bytes2 setAllowComments(yylex, false) } - case 281: + case 279: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1568 + //line sql.y:1559 { yyVAL.bytes2 = nil } - case 282: + case 280: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1572 + //line sql.y:1563 { yyVAL.bytes2 = append(yyDollar[1].bytes2, yyDollar[2].bytes) } - case 283: + case 281: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1578 + //line sql.y:1569 { yyVAL.str = UnionStr } - case 284: + case 282: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1582 + //line sql.y:1573 { yyVAL.str = UnionAllStr } - case 285: + case 283: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1586 + //line sql.y:1577 { yyVAL.str = UnionDistinctStr } - case 286: + case 284: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1591 + //line sql.y:1582 { yyVAL.str = "" } - case 287: + case 285: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1595 + //line sql.y:1586 { yyVAL.str = SQLNoCacheStr } - case 288: + case 286: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1599 + //line sql.y:1590 { yyVAL.str = SQLCacheStr } - case 289: + case 287: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1604 + //line sql.y:1595 { yyVAL.str = "" } - case 290: + case 288: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1608 + //line sql.y:1599 { yyVAL.str = DistinctStr } - case 291: + case 289: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1613 + //line sql.y:1604 { yyVAL.str = "" } - case 292: + case 290: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1617 + //line sql.y:1608 { yyVAL.str = StraightJoinHint } - case 293: + case 291: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1622 + //line sql.y:1613 { yyVAL.selectExprs = nil } - case 294: + case 292: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1626 + //line sql.y:1617 { yyVAL.selectExprs = yyDollar[1].selectExprs } - case 295: + case 293: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1632 + //line sql.y:1623 { yyVAL.selectExprs = SelectExprs{yyDollar[1].selectExpr} } - case 296: + case 294: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1636 + //line sql.y:1627 { yyVAL.selectExprs = append(yyVAL.selectExprs, yyDollar[3].selectExpr) } - case 297: + case 295: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1642 + //line sql.y:1633 { yyVAL.selectExpr = &StarExpr{} } - case 298: + case 296: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1646 + //line sql.y:1637 { yyVAL.selectExpr = &AliasedExpr{Expr: yyDollar[1].expr, As: yyDollar[2].colIdent} } - case 299: + case 297: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1650 + //line sql.y:1641 { yyVAL.selectExpr = &StarExpr{TableName: TableName{Name: yyDollar[1].tableIdent}} } - case 300: + case 298: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1654 + //line sql.y:1645 { yyVAL.selectExpr = &StarExpr{TableName: TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent}} } - case 301: + case 299: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1659 + //line sql.y:1650 { yyVAL.colIdent = ColIdent{} } - case 302: + case 300: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1663 + //line sql.y:1654 { yyVAL.colIdent = yyDollar[1].colIdent } - case 303: + case 301: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1667 + //line sql.y:1658 { yyVAL.colIdent = yyDollar[2].colIdent } - case 305: + case 303: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1674 + //line sql.y:1665 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 306: + case 304: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1679 + //line sql.y:1670 { yyVAL.tableExprs = TableExprs{&AliasedTableExpr{Expr: TableName{Name: NewTableIdent("dual")}}} } - case 307: + case 305: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1683 + //line sql.y:1674 { yyVAL.tableExprs = yyDollar[2].tableExprs } - case 308: + case 306: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1689 + //line sql.y:1680 { yyVAL.tableExprs = TableExprs{yyDollar[1].tableExpr} } - case 309: + case 307: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1693 + //line sql.y:1684 { yyVAL.tableExprs = append(yyVAL.tableExprs, yyDollar[3].tableExpr) } - case 312: + case 310: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1703 + //line sql.y:1694 { yyVAL.tableExpr = yyDollar[1].aliasedTableName } - case 313: + case 311: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1707 + //line sql.y:1698 { yyVAL.tableExpr = &AliasedTableExpr{Expr: yyDollar[1].subquery, As: yyDollar[3].tableIdent} } - case 314: + case 312: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1711 + //line sql.y:1702 { yyVAL.tableExpr = &ParenTableExpr{Exprs: yyDollar[2].tableExprs} } - case 315: + case 313: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1717 + //line sql.y:1708 { yyVAL.aliasedTableName = &AliasedTableExpr{Expr: yyDollar[1].tableName, As: yyDollar[2].tableIdent, Hints: yyDollar[3].indexHints} } - case 316: + case 314: yyDollar = yyS[yypt-7 : yypt+1] - //line sql.y:1721 + //line sql.y:1712 { yyVAL.aliasedTableName = &AliasedTableExpr{Expr: yyDollar[1].tableName, Partitions: yyDollar[4].partitions, As: yyDollar[6].tableIdent, Hints: yyDollar[7].indexHints} } - case 317: + case 315: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1727 + //line sql.y:1718 { yyVAL.columns = Columns{yyDollar[1].colIdent} } - case 318: + case 316: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1731 + //line sql.y:1722 { yyVAL.columns = append(yyVAL.columns, yyDollar[3].colIdent) } - case 319: + case 317: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1737 + //line sql.y:1728 { yyVAL.partitions = Partitions{yyDollar[1].colIdent} } - case 320: + case 318: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1741 + //line sql.y:1732 { yyVAL.partitions = append(yyVAL.partitions, yyDollar[3].colIdent) } - case 321: + case 319: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1754 + //line sql.y:1745 { yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} } - case 322: + case 320: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1758 + //line sql.y:1749 { yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} } - case 323: + case 321: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1762 + //line sql.y:1753 { yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} } - case 324: + case 322: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1766 + //line sql.y:1757 { yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr} } - case 325: + case 323: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1772 + //line sql.y:1763 { yyVAL.joinCondition = JoinCondition{On: yyDollar[2].expr} } - case 326: + case 324: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1774 + //line sql.y:1765 { yyVAL.joinCondition = JoinCondition{Using: yyDollar[3].columns} } - case 327: + case 325: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1778 + //line sql.y:1769 { yyVAL.joinCondition = JoinCondition{} } - case 328: + case 326: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1780 + //line sql.y:1771 { yyVAL.joinCondition = yyDollar[1].joinCondition } - case 329: + case 327: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1784 + //line sql.y:1775 { yyVAL.joinCondition = JoinCondition{} } - case 330: + case 328: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1786 + //line sql.y:1777 { yyVAL.joinCondition = JoinCondition{On: yyDollar[2].expr} } - case 331: + case 329: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1789 + //line sql.y:1780 { yyVAL.empty = struct{}{} } - case 332: + case 330: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1791 + //line sql.y:1782 { yyVAL.empty = struct{}{} } - case 333: + case 331: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1794 + //line sql.y:1785 { yyVAL.tableIdent = NewTableIdent("") } - case 334: + case 332: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1798 + //line sql.y:1789 { yyVAL.tableIdent = yyDollar[1].tableIdent } - case 335: + case 333: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1802 + //line sql.y:1793 { yyVAL.tableIdent = yyDollar[2].tableIdent } - case 337: + case 335: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1809 + //line sql.y:1800 { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 338: + case 336: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1815 + //line sql.y:1806 { yyVAL.str = JoinStr } - case 339: + case 337: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1819 + //line sql.y:1810 { yyVAL.str = JoinStr } - case 340: + case 338: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1823 + //line sql.y:1814 { yyVAL.str = JoinStr } - case 341: + case 339: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1829 + //line sql.y:1820 { yyVAL.str = StraightJoinStr } - case 342: + case 340: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1835 + //line sql.y:1826 { yyVAL.str = LeftJoinStr } - case 343: + case 341: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1839 + //line sql.y:1830 { yyVAL.str = LeftJoinStr } - case 344: + case 342: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1843 + //line sql.y:1834 { yyVAL.str = RightJoinStr } - case 345: + case 343: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1847 + //line sql.y:1838 { yyVAL.str = RightJoinStr } - case 346: + case 344: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1853 + //line sql.y:1844 { yyVAL.str = NaturalJoinStr } - case 347: + case 345: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1857 + //line sql.y:1848 { if yyDollar[2].str == LeftJoinStr { yyVAL.str = NaturalLeftJoinStr @@ -4703,459 +4687,459 @@ yydefault: yyVAL.str = NaturalRightJoinStr } } - case 348: + case 346: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1867 + //line sql.y:1858 { yyVAL.tableName = yyDollar[2].tableName } - case 349: + case 347: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1871 + //line sql.y:1862 { yyVAL.tableName = yyDollar[1].tableName } - case 350: + case 348: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1877 + //line sql.y:1868 { yyVAL.tableName = TableName{Name: yyDollar[1].tableIdent} } - case 351: + case 349: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1881 + //line sql.y:1872 { yyVAL.tableName = TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent} } - case 352: + case 350: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1886 + //line sql.y:1877 { yyVAL.indexHints = nil } - case 353: + case 351: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1890 + //line sql.y:1881 { yyVAL.indexHints = &IndexHints{Type: UseStr, Indexes: yyDollar[4].columns} } - case 354: + case 352: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1894 + //line sql.y:1885 { yyVAL.indexHints = &IndexHints{Type: IgnoreStr, Indexes: yyDollar[4].columns} } - case 355: + case 353: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1898 + //line sql.y:1889 { yyVAL.indexHints = &IndexHints{Type: ForceStr, Indexes: yyDollar[4].columns} } - case 356: + case 354: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1903 + //line sql.y:1894 { yyVAL.expr = nil } - case 357: + case 355: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1907 + //line sql.y:1898 { yyVAL.expr = yyDollar[2].expr } - case 358: + case 356: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1913 + //line sql.y:1904 { yyVAL.expr = yyDollar[1].expr } - case 359: + case 357: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1917 + //line sql.y:1908 { yyVAL.expr = &AndExpr{Left: yyDollar[1].expr, Right: yyDollar[3].expr} } - case 360: + case 358: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1921 + //line sql.y:1912 { yyVAL.expr = &OrExpr{Left: yyDollar[1].expr, Right: yyDollar[3].expr} } - case 361: + case 359: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1925 + //line sql.y:1916 { yyVAL.expr = &NotExpr{Expr: yyDollar[2].expr} } - case 362: + case 360: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1929 + //line sql.y:1920 { yyVAL.expr = &IsExpr{Operator: yyDollar[3].str, Expr: yyDollar[1].expr} } - case 363: + case 361: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1933 + //line sql.y:1924 { yyVAL.expr = yyDollar[1].expr } - case 364: + case 362: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1937 + //line sql.y:1928 { yyVAL.expr = &Default{ColName: yyDollar[2].str} } - case 365: + case 363: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:1943 + //line sql.y:1934 { yyVAL.str = "" } - case 366: + case 364: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1947 + //line sql.y:1938 { yyVAL.str = string(yyDollar[2].bytes) } - case 367: + case 365: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1953 + //line sql.y:1944 { yyVAL.boolVal = BoolVal(true) } - case 368: + case 366: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:1957 + //line sql.y:1948 { yyVAL.boolVal = BoolVal(false) } - case 369: + case 367: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1963 + //line sql.y:1954 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: yyDollar[2].str, Right: yyDollar[3].expr} } - case 370: + case 368: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1967 + //line sql.y:1958 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: InStr, Right: yyDollar[3].colTuple} } - case 371: + case 369: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1971 + //line sql.y:1962 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotInStr, Right: yyDollar[4].colTuple} } - case 372: + case 370: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1975 + //line sql.y:1966 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: LikeStr, Right: yyDollar[3].expr, Escape: yyDollar[4].expr} } - case 373: + case 371: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1979 + //line sql.y:1970 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotLikeStr, Right: yyDollar[4].expr, Escape: yyDollar[5].expr} } - case 374: + case 372: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:1983 + //line sql.y:1974 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: RegexpStr, Right: yyDollar[3].expr} } - case 375: + case 373: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:1987 + //line sql.y:1978 { yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotRegexpStr, Right: yyDollar[4].expr} } - case 376: + case 374: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:1991 + //line sql.y:1982 { yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: BetweenStr, From: yyDollar[3].expr, To: yyDollar[5].expr} } - case 377: + case 375: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:1995 + //line sql.y:1986 { yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: NotBetweenStr, From: yyDollar[4].expr, To: yyDollar[6].expr} } - case 378: + case 376: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:1999 + //line sql.y:1990 { yyVAL.expr = &ExistsExpr{Subquery: yyDollar[2].subquery} } - case 379: + case 377: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2005 + //line sql.y:1996 { yyVAL.str = IsNullStr } - case 380: + case 378: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2009 + //line sql.y:2000 { yyVAL.str = IsNotNullStr } - case 381: + case 379: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2013 + //line sql.y:2004 { yyVAL.str = IsTrueStr } - case 382: + case 380: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2017 + //line sql.y:2008 { yyVAL.str = IsNotTrueStr } - case 383: + case 381: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2021 + //line sql.y:2012 { yyVAL.str = IsFalseStr } - case 384: + case 382: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2025 + //line sql.y:2016 { yyVAL.str = IsNotFalseStr } - case 385: + case 383: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2031 + //line sql.y:2022 { yyVAL.str = EqualStr } - case 386: + case 384: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2035 + //line sql.y:2026 { yyVAL.str = LessThanStr } - case 387: + case 385: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2039 + //line sql.y:2030 { yyVAL.str = GreaterThanStr } - case 388: + case 386: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2043 + //line sql.y:2034 { yyVAL.str = LessEqualStr } - case 389: + case 387: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2047 + //line sql.y:2038 { yyVAL.str = GreaterEqualStr } - case 390: + case 388: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2051 + //line sql.y:2042 { yyVAL.str = NotEqualStr } - case 391: + case 389: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2055 + //line sql.y:2046 { yyVAL.str = NullSafeEqualStr } - case 392: + case 390: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2060 + //line sql.y:2051 { yyVAL.expr = nil } - case 393: + case 391: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2064 + //line sql.y:2055 { yyVAL.expr = yyDollar[2].expr } - case 394: + case 392: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2070 + //line sql.y:2061 { yyVAL.colTuple = yyDollar[1].valTuple } - case 395: + case 393: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2074 + //line sql.y:2065 { yyVAL.colTuple = yyDollar[1].subquery } - case 396: + case 394: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2078 + //line sql.y:2069 { yyVAL.colTuple = ListArg(yyDollar[1].bytes) } - case 397: + case 395: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2084 + //line sql.y:2075 { yyVAL.subquery = &Subquery{yyDollar[2].selStmt} } - case 398: + case 396: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2090 + //line sql.y:2081 { yyVAL.exprs = Exprs{yyDollar[1].expr} } - case 399: + case 397: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2094 + //line sql.y:2085 { yyVAL.exprs = append(yyDollar[1].exprs, yyDollar[3].expr) } - case 400: + case 398: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2100 + //line sql.y:2091 { yyVAL.expr = yyDollar[1].expr } - case 401: + case 399: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2104 + //line sql.y:2095 { yyVAL.expr = yyDollar[1].boolVal } - case 402: + case 400: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2108 + //line sql.y:2099 { yyVAL.expr = yyDollar[1].colName } - case 403: + case 401: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2112 + //line sql.y:2103 { yyVAL.expr = yyDollar[1].expr } - case 404: + case 402: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2116 + //line sql.y:2107 { yyVAL.expr = yyDollar[1].subquery } - case 405: + case 403: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2120 + //line sql.y:2111 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitAndStr, Right: yyDollar[3].expr} } - case 406: + case 404: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2124 + //line sql.y:2115 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitOrStr, Right: yyDollar[3].expr} } - case 407: + case 405: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2128 + //line sql.y:2119 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitXorStr, Right: yyDollar[3].expr} } - case 408: + case 406: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2132 + //line sql.y:2123 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: PlusStr, Right: yyDollar[3].expr} } - case 409: + case 407: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2136 + //line sql.y:2127 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MinusStr, Right: yyDollar[3].expr} } - case 410: + case 408: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2140 + //line sql.y:2131 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MultStr, Right: yyDollar[3].expr} } - case 411: + case 409: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2144 + //line sql.y:2135 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: DivStr, Right: yyDollar[3].expr} } - case 412: + case 410: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2148 + //line sql.y:2139 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: IntDivStr, Right: yyDollar[3].expr} } - case 413: + case 411: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2152 + //line sql.y:2143 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ModStr, Right: yyDollar[3].expr} } - case 414: + case 412: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2156 + //line sql.y:2147 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ModStr, Right: yyDollar[3].expr} } - case 415: + case 413: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2160 + //line sql.y:2151 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftLeftStr, Right: yyDollar[3].expr} } - case 416: + case 414: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2164 + //line sql.y:2155 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftRightStr, Right: yyDollar[3].expr} } - case 417: + case 415: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2168 + //line sql.y:2159 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONExtractOp, Right: yyDollar[3].expr} } - case 418: + case 416: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2172 + //line sql.y:2163 { yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONUnquoteExtractOp, Right: yyDollar[3].expr} } - case 419: + case 417: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2176 + //line sql.y:2167 { yyVAL.expr = &CollateExpr{Expr: yyDollar[1].expr, Charset: yyDollar[3].str} } - case 420: + case 418: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2180 + //line sql.y:2171 { yyVAL.expr = &UnaryExpr{Operator: BinaryStr, Expr: yyDollar[2].expr} } - case 421: + case 419: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2184 + //line sql.y:2175 { yyVAL.expr = &UnaryExpr{Operator: UBinaryStr, Expr: yyDollar[2].expr} } - case 422: + case 420: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2188 + //line sql.y:2179 { yyVAL.expr = &UnaryExpr{Operator: Utf8mb4Str, Expr: yyDollar[2].expr} } - case 423: + case 421: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2192 + //line sql.y:2183 { if num, ok := yyDollar[2].expr.(*SQLVal); ok && num.Type == IntVal { yyVAL.expr = num @@ -5163,9 +5147,9 @@ yydefault: yyVAL.expr = &UnaryExpr{Operator: UPlusStr, Expr: yyDollar[2].expr} } } - case 424: + case 422: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2200 + //line sql.y:2191 { if num, ok := yyDollar[2].expr.(*SQLVal); ok && num.Type == IntVal { // Handle double negative @@ -5179,21 +5163,21 @@ yydefault: yyVAL.expr = &UnaryExpr{Operator: UMinusStr, Expr: yyDollar[2].expr} } } - case 425: + case 423: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2214 + //line sql.y:2205 { yyVAL.expr = &UnaryExpr{Operator: TildaStr, Expr: yyDollar[2].expr} } - case 426: + case 424: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2218 + //line sql.y:2209 { yyVAL.expr = &UnaryExpr{Operator: BangStr, Expr: yyDollar[2].expr} } - case 427: + case 425: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2222 + //line sql.y:2213 { // This rule prevents the usage of INTERVAL // as a function. If support is needed for that, @@ -5201,431 +5185,431 @@ yydefault: // will be non-trivial because of grammar conflicts. yyVAL.expr = &IntervalExpr{Expr: yyDollar[2].expr, Unit: yyDollar[3].colIdent.String()} } - case 432: + case 430: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2240 + //line sql.y:2231 { yyVAL.expr = &FuncExpr{Name: yyDollar[1].colIdent, Exprs: yyDollar[3].selectExprs} } - case 433: + case 431: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2244 + //line sql.y:2235 { yyVAL.expr = &FuncExpr{Name: yyDollar[1].colIdent, Distinct: true, Exprs: yyDollar[4].selectExprs} } - case 434: + case 432: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2248 + //line sql.y:2239 { yyVAL.expr = &FuncExpr{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].colIdent, Exprs: yyDollar[5].selectExprs} } - case 435: + case 433: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2258 + //line sql.y:2249 { yyVAL.expr = &FuncExpr{Name: NewColIdent("left"), Exprs: yyDollar[3].selectExprs} } - case 436: + case 434: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2262 + //line sql.y:2253 { yyVAL.expr = &FuncExpr{Name: NewColIdent("right"), Exprs: yyDollar[3].selectExprs} } - case 437: + case 435: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2266 + //line sql.y:2257 { yyVAL.expr = &ConvertExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].convertType} } - case 438: + case 436: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2270 + //line sql.y:2261 { yyVAL.expr = &ConvertExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].convertType} } - case 439: + case 437: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2274 + //line sql.y:2265 { yyVAL.expr = &ConvertUsingExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].str} } - case 440: + case 438: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2278 + //line sql.y:2269 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} } - case 441: + case 439: yyDollar = yyS[yypt-8 : yypt+1] - //line sql.y:2282 + //line sql.y:2273 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } - case 442: + case 440: yyDollar = yyS[yypt-8 : yypt+1] - //line sql.y:2286 + //line sql.y:2277 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } - case 443: + case 441: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2290 + //line sql.y:2281 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} } - case 444: + case 442: yyDollar = yyS[yypt-8 : yypt+1] - //line sql.y:2294 + //line sql.y:2285 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } - case 445: + case 443: yyDollar = yyS[yypt-8 : yypt+1] - //line sql.y:2298 + //line sql.y:2289 { yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } - case 446: + case 444: yyDollar = yyS[yypt-9 : yypt+1] - //line sql.y:2302 + //line sql.y:2293 { yyVAL.expr = &MatchExpr{Columns: yyDollar[3].selectExprs, Expr: yyDollar[7].expr, Option: yyDollar[8].str} } - case 447: + case 445: yyDollar = yyS[yypt-7 : yypt+1] - //line sql.y:2306 + //line sql.y:2297 { yyVAL.expr = &GroupConcatExpr{Distinct: yyDollar[3].str, Exprs: yyDollar[4].selectExprs, OrderBy: yyDollar[5].orderBy, Separator: yyDollar[6].str} } - case 448: + case 446: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2310 + //line sql.y:2301 { yyVAL.expr = &CaseExpr{Expr: yyDollar[2].expr, Whens: yyDollar[3].whens, Else: yyDollar[4].expr} } - case 449: + case 447: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2314 + //line sql.y:2305 { yyVAL.expr = &ValuesFuncExpr{Name: yyDollar[3].colName} } - case 450: + case 448: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2324 + //line sql.y:2315 { yyVAL.expr = &FuncExpr{Name: NewColIdent("current_timestamp")} } - case 451: + case 449: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2328 + //line sql.y:2319 { yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_timestamp")} } - case 452: + case 450: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2332 + //line sql.y:2323 { yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_time")} } - case 453: + case 451: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2336 + //line sql.y:2327 { yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_date")} } - case 454: + case 452: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2341 + //line sql.y:2332 { yyVAL.expr = &FuncExpr{Name: NewColIdent("localtime")} } - case 455: + case 453: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2346 + //line sql.y:2337 { yyVAL.expr = &FuncExpr{Name: NewColIdent("localtimestamp")} } - case 456: + case 454: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2351 + //line sql.y:2342 { yyVAL.expr = &FuncExpr{Name: NewColIdent("current_date")} } - case 457: + case 455: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2356 + //line sql.y:2347 { yyVAL.expr = &FuncExpr{Name: NewColIdent("current_time")} } - case 460: + case 458: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2370 + //line sql.y:2361 { yyVAL.expr = &FuncExpr{Name: NewColIdent("if"), Exprs: yyDollar[3].selectExprs} } - case 461: + case 459: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2374 + //line sql.y:2365 { yyVAL.expr = &FuncExpr{Name: NewColIdent("database"), Exprs: yyDollar[3].selectExprs} } - case 462: + case 460: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2378 + //line sql.y:2369 { yyVAL.expr = &FuncExpr{Name: NewColIdent("mod"), Exprs: yyDollar[3].selectExprs} } - case 463: + case 461: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2382 + //line sql.y:2373 { yyVAL.expr = &FuncExpr{Name: NewColIdent("replace"), Exprs: yyDollar[3].selectExprs} } - case 464: + case 462: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2388 + //line sql.y:2379 { yyVAL.str = "" } - case 465: + case 463: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2392 + //line sql.y:2383 { yyVAL.str = BooleanModeStr } - case 466: + case 464: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2396 + //line sql.y:2387 { yyVAL.str = NaturalLanguageModeStr } - case 467: + case 465: yyDollar = yyS[yypt-7 : yypt+1] - //line sql.y:2400 + //line sql.y:2391 { yyVAL.str = NaturalLanguageModeWithQueryExpansionStr } - case 468: + case 466: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2404 + //line sql.y:2395 { yyVAL.str = QueryExpansionStr } - case 469: + case 467: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2410 + //line sql.y:2401 { yyVAL.str = string(yyDollar[1].bytes) } - case 470: + case 468: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2414 + //line sql.y:2405 { yyVAL.str = string(yyDollar[1].bytes) } - case 471: + case 469: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2420 + //line sql.y:2411 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal} } - case 472: + case 470: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2424 + //line sql.y:2415 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal, Charset: yyDollar[3].str, Operator: CharacterSetStr} } - case 473: + case 471: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2428 + //line sql.y:2419 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal, Charset: string(yyDollar[3].bytes)} } - case 474: + case 472: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2432 + //line sql.y:2423 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 475: + case 473: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2436 + //line sql.y:2427 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal} } - case 476: + case 474: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2440 + //line sql.y:2431 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} yyVAL.convertType.Length = yyDollar[2].LengthScaleOption.Length yyVAL.convertType.Scale = yyDollar[2].LengthScaleOption.Scale } - case 477: + case 475: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2446 + //line sql.y:2437 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 478: + case 476: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2450 + //line sql.y:2441 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal} } - case 479: + case 477: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2454 + //line sql.y:2445 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 480: + case 478: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2458 + //line sql.y:2449 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 481: + case 479: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2462 + //line sql.y:2453 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].optVal} } - case 482: + case 480: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2466 + //line sql.y:2457 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 483: + case 481: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2470 + //line sql.y:2461 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } - case 484: + case 482: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2475 + //line sql.y:2466 { yyVAL.expr = nil } - case 485: + case 483: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2479 + //line sql.y:2470 { yyVAL.expr = yyDollar[1].expr } - case 486: + case 484: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2484 + //line sql.y:2475 { yyVAL.str = string("") } - case 487: + case 485: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2488 + //line sql.y:2479 { yyVAL.str = " separator '" + string(yyDollar[2].bytes) + "'" } - case 488: + case 486: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2494 + //line sql.y:2485 { yyVAL.whens = []*When{yyDollar[1].when} } - case 489: + case 487: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2498 + //line sql.y:2489 { yyVAL.whens = append(yyDollar[1].whens, yyDollar[2].when) } - case 490: + case 488: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2504 + //line sql.y:2495 { yyVAL.when = &When{Cond: yyDollar[2].expr, Val: yyDollar[4].expr} } - case 491: + case 489: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2509 + //line sql.y:2500 { yyVAL.expr = nil } - case 492: + case 490: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2513 + //line sql.y:2504 { yyVAL.expr = yyDollar[2].expr } - case 493: + case 491: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2519 + //line sql.y:2510 { yyVAL.colName = &ColName{Name: yyDollar[1].colIdent} } - case 494: + case 492: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2523 + //line sql.y:2514 { yyVAL.colName = &ColName{Qualifier: TableName{Name: yyDollar[1].tableIdent}, Name: yyDollar[3].colIdent} } - case 495: + case 493: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2527 + //line sql.y:2518 { yyVAL.colName = &ColName{Qualifier: TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent}, Name: yyDollar[5].colIdent} } - case 496: + case 494: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2533 + //line sql.y:2524 { yyVAL.expr = NewStrVal(yyDollar[1].bytes) } - case 497: + case 495: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2537 + //line sql.y:2528 { yyVAL.expr = NewHexVal(yyDollar[1].bytes) } - case 498: + case 496: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2541 + //line sql.y:2532 { yyVAL.expr = NewBitVal(yyDollar[1].bytes) } - case 499: + case 497: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2545 + //line sql.y:2536 { yyVAL.expr = NewIntVal(yyDollar[1].bytes) } - case 500: + case 498: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2549 + //line sql.y:2540 { yyVAL.expr = NewFloatVal(yyDollar[1].bytes) } - case 501: + case 499: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2553 + //line sql.y:2544 { yyVAL.expr = NewHexNum(yyDollar[1].bytes) } - case 502: + case 500: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2557 + //line sql.y:2548 { yyVAL.expr = NewValArg(yyDollar[1].bytes) } - case 503: + case 501: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2561 + //line sql.y:2552 { yyVAL.expr = &NullVal{} } - case 504: + case 502: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2567 + //line sql.y:2558 { // TODO(sougou): Deprecate this construct. if yyDollar[1].colIdent.Lowered() != "value" { @@ -5634,239 +5618,239 @@ yydefault: } yyVAL.expr = NewIntVal([]byte("1")) } - case 505: + case 503: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2576 + //line sql.y:2567 { yyVAL.expr = NewIntVal(yyDollar[1].bytes) } - case 506: + case 504: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2580 + //line sql.y:2571 { yyVAL.expr = NewValArg(yyDollar[1].bytes) } - case 507: + case 505: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2585 + //line sql.y:2576 { yyVAL.exprs = nil } - case 508: + case 506: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2589 + //line sql.y:2580 { yyVAL.exprs = yyDollar[3].exprs } - case 509: + case 507: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2594 + //line sql.y:2585 { yyVAL.expr = nil } - case 510: + case 508: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2598 + //line sql.y:2589 { yyVAL.expr = yyDollar[2].expr } - case 511: + case 509: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2603 + //line sql.y:2594 { yyVAL.orderBy = nil } - case 512: + case 510: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2607 + //line sql.y:2598 { yyVAL.orderBy = yyDollar[3].orderBy } - case 513: + case 511: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2613 + //line sql.y:2604 { yyVAL.orderBy = OrderBy{yyDollar[1].order} } - case 514: + case 512: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2617 + //line sql.y:2608 { yyVAL.orderBy = append(yyDollar[1].orderBy, yyDollar[3].order) } - case 515: + case 513: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2623 + //line sql.y:2614 { yyVAL.order = &Order{Expr: yyDollar[1].expr, Direction: yyDollar[2].str} } - case 516: + case 514: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2628 + //line sql.y:2619 { yyVAL.str = AscScr } - case 517: + case 515: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2632 + //line sql.y:2623 { yyVAL.str = AscScr } - case 518: + case 516: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2636 + //line sql.y:2627 { yyVAL.str = DescScr } - case 519: + case 517: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2641 + //line sql.y:2632 { yyVAL.limit = nil } - case 520: + case 518: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2645 + //line sql.y:2636 { yyVAL.limit = &Limit{Rowcount: yyDollar[2].expr} } - case 521: + case 519: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2649 + //line sql.y:2640 { yyVAL.limit = &Limit{Offset: yyDollar[2].expr, Rowcount: yyDollar[4].expr} } - case 522: + case 520: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2653 + //line sql.y:2644 { yyVAL.limit = &Limit{Offset: yyDollar[4].expr, Rowcount: yyDollar[2].expr} } - case 523: + case 521: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2658 + //line sql.y:2649 { yyVAL.str = "" } - case 524: + case 522: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2662 + //line sql.y:2653 { yyVAL.str = ForUpdateStr } - case 525: + case 523: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2666 + //line sql.y:2657 { yyVAL.str = ShareModeStr } - case 526: + case 524: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2679 + //line sql.y:2670 { yyVAL.ins = &Insert{Rows: yyDollar[2].values} } - case 527: + case 525: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2683 + //line sql.y:2674 { yyVAL.ins = &Insert{Rows: yyDollar[1].selStmt} } - case 528: + case 526: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2687 + //line sql.y:2678 { // Drop the redundant parenthesis. yyVAL.ins = &Insert{Rows: yyDollar[2].selStmt} } - case 529: + case 527: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2692 + //line sql.y:2683 { yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[5].values} } - case 530: + case 528: yyDollar = yyS[yypt-4 : yypt+1] - //line sql.y:2696 + //line sql.y:2687 { yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[4].selStmt} } - case 531: + case 529: yyDollar = yyS[yypt-6 : yypt+1] - //line sql.y:2700 + //line sql.y:2691 { // Drop the redundant parenthesis. yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[5].selStmt} } - case 532: + case 530: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2707 + //line sql.y:2698 { yyVAL.columns = Columns{yyDollar[1].colIdent} } - case 533: + case 531: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2711 + //line sql.y:2702 { yyVAL.columns = Columns{yyDollar[3].colIdent} } - case 534: + case 532: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2715 + //line sql.y:2706 { yyVAL.columns = append(yyVAL.columns, yyDollar[3].colIdent) } - case 535: + case 533: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2719 + //line sql.y:2710 { yyVAL.columns = append(yyVAL.columns, yyDollar[5].colIdent) } - case 536: + case 534: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2724 + //line sql.y:2715 { yyVAL.updateExprs = nil } - case 537: + case 535: yyDollar = yyS[yypt-5 : yypt+1] - //line sql.y:2728 + //line sql.y:2719 { yyVAL.updateExprs = yyDollar[5].updateExprs } - case 538: + case 536: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2734 + //line sql.y:2725 { yyVAL.values = Values{yyDollar[1].valTuple} } - case 539: + case 537: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2738 + //line sql.y:2729 { yyVAL.values = append(yyDollar[1].values, yyDollar[3].valTuple) } - case 540: + case 538: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2744 + //line sql.y:2735 { yyVAL.valTuple = yyDollar[1].valTuple } - case 541: + case 539: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2748 + //line sql.y:2739 { yyVAL.valTuple = ValTuple{} } - case 542: + case 540: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2754 + //line sql.y:2745 { yyVAL.valTuple = ValTuple(yyDollar[2].exprs) } - case 543: + case 541: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2760 + //line sql.y:2751 { if len(yyDollar[1].valTuple) == 1 { yyVAL.expr = &ParenExpr{yyDollar[1].valTuple[0]} @@ -5874,306 +5858,306 @@ yydefault: yyVAL.expr = yyDollar[1].valTuple } } - case 544: + case 542: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2770 + //line sql.y:2761 { yyVAL.updateExprs = UpdateExprs{yyDollar[1].updateExpr} } - case 545: + case 543: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2774 + //line sql.y:2765 { yyVAL.updateExprs = append(yyDollar[1].updateExprs, yyDollar[3].updateExpr) } - case 546: + case 544: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2780 + //line sql.y:2771 { yyVAL.updateExpr = &UpdateExpr{Name: yyDollar[1].colName, Expr: yyDollar[3].expr} } - case 547: + case 545: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2786 + //line sql.y:2777 { yyVAL.setExprs = SetExprs{yyDollar[1].setExpr} } - case 548: + case 546: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2790 + //line sql.y:2781 { yyVAL.setExprs = append(yyDollar[1].setExprs, yyDollar[3].setExpr) } - case 549: + case 547: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2796 + //line sql.y:2787 { yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: NewStrVal([]byte("on"))} } - case 550: + case 548: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2800 + //line sql.y:2791 { yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: yyDollar[3].expr} } - case 551: + case 549: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2804 + //line sql.y:2795 { yyVAL.setExpr = &SetExpr{Name: NewColIdent(string(yyDollar[1].bytes)), Expr: yyDollar[2].expr} } - case 553: + case 551: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2811 + //line sql.y:2802 { yyVAL.bytes = []byte("charset") } - case 555: + case 553: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2818 + //line sql.y:2809 { yyVAL.expr = NewStrVal([]byte(yyDollar[1].colIdent.String())) } - case 556: + case 554: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2822 + //line sql.y:2813 { yyVAL.expr = NewStrVal(yyDollar[1].bytes) } - case 557: + case 555: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2826 + //line sql.y:2817 { yyVAL.expr = &Default{} } - case 560: + case 558: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2835 + //line sql.y:2826 { yyVAL.byt = 0 } - case 561: + case 559: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2837 + //line sql.y:2828 { yyVAL.byt = 1 } - case 562: + case 560: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2840 + //line sql.y:2831 { yyVAL.empty = struct{}{} } - case 563: + case 561: yyDollar = yyS[yypt-3 : yypt+1] - //line sql.y:2842 + //line sql.y:2833 { yyVAL.empty = struct{}{} } - case 564: + case 562: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2845 + //line sql.y:2836 { yyVAL.str = "" } - case 565: + case 563: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2847 + //line sql.y:2838 { yyVAL.str = IgnoreStr } + case 564: + yyDollar = yyS[yypt-1 : yypt+1] + //line sql.y:2842 + { + yyVAL.empty = struct{}{} + } + case 565: + yyDollar = yyS[yypt-1 : yypt+1] + //line sql.y:2844 + { + yyVAL.empty = struct{}{} + } case 566: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2851 + //line sql.y:2846 { yyVAL.empty = struct{}{} } case 567: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2853 + //line sql.y:2848 { yyVAL.empty = struct{}{} } case 568: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2855 + //line sql.y:2850 { yyVAL.empty = struct{}{} } case 569: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2857 + //line sql.y:2852 { yyVAL.empty = struct{}{} } case 570: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2859 + //line sql.y:2854 { yyVAL.empty = struct{}{} } case 571: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2861 + //line sql.y:2856 { yyVAL.empty = struct{}{} } case 572: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2863 + //line sql.y:2858 { yyVAL.empty = struct{}{} } case 573: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2865 + //line sql.y:2860 { yyVAL.empty = struct{}{} } case 574: - yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2867 + yyDollar = yyS[yypt-0 : yypt+1] + //line sql.y:2863 { yyVAL.empty = struct{}{} } case 575: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2869 + //line sql.y:2865 { yyVAL.empty = struct{}{} } case 576: - yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2872 + yyDollar = yyS[yypt-1 : yypt+1] + //line sql.y:2867 { yyVAL.empty = struct{}{} } case 577: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2874 + //line sql.y:2871 { yyVAL.empty = struct{}{} } case 578: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2876 + //line sql.y:2873 { yyVAL.empty = struct{}{} } case 579: - yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2880 + yyDollar = yyS[yypt-0 : yypt+1] + //line sql.y:2876 { yyVAL.empty = struct{}{} } case 580: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2882 + //line sql.y:2878 { yyVAL.empty = struct{}{} } case 581: - yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2885 - { - yyVAL.empty = struct{}{} - } - case 582: - yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2887 - { - yyVAL.empty = struct{}{} - } - case 583: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2889 + //line sql.y:2880 { yyVAL.empty = struct{}{} } - case 584: + case 582: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:2892 + //line sql.y:2883 { yyVAL.colIdent = ColIdent{} } - case 585: + case 583: yyDollar = yyS[yypt-2 : yypt+1] - //line sql.y:2894 + //line sql.y:2885 { yyVAL.colIdent = yyDollar[2].colIdent } - case 586: + case 584: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2898 + //line sql.y:2889 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 587: + case 585: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2902 + //line sql.y:2893 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 589: + case 587: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2909 + //line sql.y:2900 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 590: + case 588: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2915 + //line sql.y:2906 { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 591: + case 589: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2919 + //line sql.y:2910 { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 593: + case 591: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:2926 + //line sql.y:2917 { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 785: + case 783: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3143 + //line sql.y:3134 { if incNesting(yylex) { yylex.Error("max nesting level reached") return 1 } } - case 786: + case 784: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3152 + //line sql.y:3143 { decNesting(yylex) } - case 787: + case 785: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:3157 + //line sql.y:3148 { forceEOF(yylex) } - case 788: + case 786: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:3162 + //line sql.y:3153 { forceEOF(yylex) } - case 789: + case 787: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3166 + //line sql.y:3157 { forceEOF(yylex) } - case 790: + case 788: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3170 + //line sql.y:3161 { forceEOF(yylex) } diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index 6b0ab4327f3..6dd3a361548 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -175,7 +175,7 @@ func forceEOF(yylex interface{}) { %token NULLX AUTO_INCREMENT APPROXNUM SIGNED UNSIGNED ZEROFILL // Supported SHOW tokens -%token COLLATION DATABASES TABLES VITESS_KEYSPACES VITESS_SHARDS VITESS_TABLETS VSCHEMA_TABLES EXTENDED FULL PROCESSLIST COLUMNS +%token COLLATION DATABASES TABLES VITESS_KEYSPACES VITESS_SHARDS VITESS_TABLETS VSCHEMA_TABLES FULL PROCESSLIST COLUMNS // SET tokens %token NAMES CHARSET GLOBAL SESSION ISOLATION LEVEL READ WRITE ONLY REPEATABLE COMMITTED UNCOMMITTED SERIALIZABLE @@ -252,7 +252,7 @@ func forceEOF(yylex interface{}) { %type set_expression transaction_char isolation_level %type for_from %type ignore_opt default_opt -%type extended_opt full_opt from_database_opt tables_or_processlist_or_columns +%type full_opt from_database_opt tables_or_processlist %type like_or_where_opt %type exists_opt %type not_exists_opt non_add_drop_or_rename_operation to_opt index_opt constraint_opt @@ -1371,14 +1371,19 @@ show_statement: { $$ = &Show{Type: string($2)} } -| SHOW extended_opt full_opt tables_or_processlist_or_columns from_database_opt like_or_where_opt +| SHOW full_opt COLUMNS FROM table_name from_database_opt like_or_where_opt + { + showTablesOpt := &ShowTablesOpt{Full:$2, DbName:$6, Filter:$7} + $$ = &Show{Type: string($3), ShowTablesOpt: showTablesOpt, OnTable: $5} + } +| SHOW full_opt tables_or_processlist from_database_opt like_or_where_opt { // this is ugly, but I couldn't find a better way for now - if $4 == "processlist" { - $$ = &Show{Type: $4} + if $3 == "processlist" { + $$ = &Show{Type: $3} } else { - showTablesOpt := &ShowTablesOpt{Extended: $2, Full:$3, DbName:$5, Filter:$6} - $$ = &Show{Type: $4, ShowTablesOpt: showTablesOpt} + showTablesOpt := &ShowTablesOpt{Full:$2, DbName:$4, Filter:$5} + $$ = &Show{Type: $3, ShowTablesOpt: showTablesOpt} } } | SHOW show_session_or_global VARIABLES ddl_force_eof @@ -1424,7 +1429,7 @@ show_statement: $$ = &Show{Type: string($2)} } -tables_or_processlist_or_columns: +tables_or_processlist: TABLES { $$ = string($1) @@ -1433,20 +1438,6 @@ tables_or_processlist_or_columns: { $$ = string($1) } -| COLUMNS - { - $$ = string($1) - } - -extended_opt: - /* empty */ - { - $$ = "" - } -| EXTENDED - { - $$ = "extended " - } full_opt: /* empty */ diff --git a/go/vt/sqlparser/token.go b/go/vt/sqlparser/token.go index 2e08f3c8adb..4e391b57af5 100644 --- a/go/vt/sqlparser/token.go +++ b/go/vt/sqlparser/token.go @@ -171,7 +171,6 @@ var keywords = map[string]int{ "exit": UNUSED, "explain": EXPLAIN, "expansion": EXPANSION, - "extended": EXTENDED, "false": FALSE, "fetch": UNUSED, "float": FLOAT_TYPE, From d954c87c2dc4f0cb166b4ddb2990d41389339237 Mon Sep 17 00:00:00 2001 From: Alex Charis Date: Thu, 28 Jun 2018 23:46:42 -0400 Subject: [PATCH 29/72] make 'columns' not reserved keyword Signed-off-by: Alex Charis --- go/vt/sqlparser/sql.go | 2747 ++++++++++++++++++++-------------------- go/vt/sqlparser/sql.y | 1 + 2 files changed, 1361 insertions(+), 1387 deletions(-) diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index 8e968012588..791ca8cbcb6 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -603,1179 +603,1152 @@ var yyExca = [...]int{ 151, 263, 152, 263, -2, 255, - -1, 240, + -1, 241, 110, 588, -2, 584, - -1, 241, + -1, 242, 110, 589, -2, 585, - -1, 311, - 80, 748, - -2, 58, -1, 312, - 80, 709, + 80, 749, + -2, 58, + -1, 313, + 80, 710, -2, 59, - -1, 317, + -1, 318, 80, 692, -2, 550, - -1, 319, - 80, 730, + -1, 320, + 80, 731, -2, 552, - -1, 581, + -1, 582, 52, 41, 54, 41, -2, 43, - -1, 715, + -1, 716, 110, 591, -2, 587, - -1, 922, + -1, 923, 5, 28, -2, 395, - -1, 947, + -1, 948, 5, 27, -2, 525, - -1, 1168, + -1, 1169, 5, 28, -2, 526, - -1, 1212, + -1, 1213, 5, 27, -2, 528, - -1, 1274, + -1, 1275, 5, 28, -2, 529, } const yyPrivate = 57344 -const yyLast = 11133 +const yyLast = 10866 var yyAct = [...]int{ - 272, 47, 1265, 861, 652, 528, 1077, 1223, 1105, 795, - 245, 271, 527, 3, 1174, 841, 1078, 777, 219, 817, - 1074, 1008, 950, 575, 888, 816, 778, 966, 213, 316, - 813, 855, 740, 750, 914, 747, 573, 681, 1011, 1051, - 827, 999, 591, 955, 460, 53, 766, 466, 47, 717, - 413, 590, 774, 562, 310, 472, 224, 577, 480, 896, - 218, 851, 302, 243, 228, 298, 307, 305, 52, 1294, - 1284, 1292, 214, 215, 216, 217, 878, 1272, 835, 1290, - 862, 1283, 1069, 1162, 1271, 247, 417, 1232, 1099, 438, - 877, 297, 542, 232, 1100, 1101, 809, 810, 296, 1247, - 493, 492, 502, 503, 495, 496, 497, 498, 499, 500, - 501, 494, 808, 990, 504, 57, 834, 882, 182, 178, - 179, 180, 974, 1186, 1201, 973, 876, 842, 975, 749, - 1111, 1112, 1113, 592, 1151, 593, 426, 1149, 1116, 1114, - 59, 60, 61, 62, 63, 454, 212, 301, 449, 450, - 1291, 455, 452, 440, 1224, 442, 1289, 1266, 1032, 775, - 427, 829, 420, 175, 176, 176, 1230, 1226, 241, 796, - 798, 660, 651, 829, 829, 873, 870, 871, 965, 869, - 439, 441, 814, 964, 963, 415, 423, 190, 1252, 414, - 177, 516, 517, 1171, 1038, 444, 444, 444, 444, 76, - 444, 984, 930, 187, 880, 883, 187, 908, 444, 493, - 492, 502, 503, 495, 496, 497, 498, 499, 500, 501, - 494, 1029, 689, 504, 484, 47, 433, 1031, 181, 504, - 187, 187, 76, 686, 682, 1225, 187, 469, 76, 875, - 513, 1052, 494, 515, 797, 504, 842, 497, 498, 499, - 500, 501, 494, 479, 828, 504, 915, 1248, 1257, 826, - 824, 874, 437, 825, 1231, 1229, 828, 828, 468, 890, - 1130, 526, 1054, 530, 531, 532, 533, 534, 535, 536, - 537, 538, 1270, 541, 543, 543, 543, 543, 543, 543, - 543, 543, 551, 552, 553, 554, 953, 1019, 1120, 879, - 1115, 594, 655, 574, 1056, 1071, 1060, 477, 1055, 767, - 1053, 937, 881, 683, 767, 1058, 831, 429, 430, 431, - 1030, 832, 1028, 479, 1057, 1017, 988, 1260, 514, 495, - 496, 497, 498, 499, 500, 501, 494, 1059, 1061, 504, - 724, 470, 474, 927, 187, 889, 187, 1276, 1121, 419, - 174, 926, 187, 925, 722, 723, 721, 688, 1277, 187, - 692, 693, 1192, 76, 76, 76, 76, 1191, 76, 478, - 477, 741, 582, 742, 1258, 588, 76, 544, 545, 546, - 547, 548, 549, 550, 50, 1003, 479, 1002, 301, 991, - 1018, 478, 477, 687, 720, 1023, 1020, 1013, 1014, 1021, - 1016, 1015, 1208, 76, 1019, 1189, 478, 477, 479, 478, - 477, 295, 1022, 478, 477, 1255, 1035, 444, 1025, 1000, - 1073, 421, 422, 479, 1108, 444, 479, 478, 477, 1107, - 479, 985, 1017, 905, 906, 907, 444, 444, 444, 444, - 444, 444, 444, 444, 479, 1280, 459, 707, 709, 710, - 444, 444, 708, 1216, 1263, 1216, 459, 1216, 1217, 459, - 458, 1183, 1182, 669, 187, 261, 260, 263, 264, 265, - 266, 187, 187, 187, 262, 267, 976, 76, 564, 567, - 568, 569, 565, 76, 566, 570, 864, 694, 956, 957, - 1096, 459, 21, 667, 1170, 459, 1236, 1018, 743, 718, - 666, 270, 1023, 1020, 1013, 1014, 1021, 1016, 1015, 1127, - 1126, 1123, 1124, 1235, 715, 665, 47, 1123, 1122, 1022, - 920, 459, 559, 459, 1117, 1012, 656, 23, 752, 459, - 952, 530, 74, 654, 649, 696, 601, 600, 951, 435, - 711, 428, 414, 754, 713, 752, 759, 762, 223, 23, - 1075, 945, 768, 951, 946, 1166, 585, 1041, 558, 952, - 302, 302, 302, 302, 302, 315, 802, 23, 584, 779, - 443, 418, 559, 719, 50, 574, 1211, 799, 54, 932, - 744, 745, 559, 559, 302, 76, 1129, 754, 771, 445, - 929, 187, 187, 76, 764, 187, 50, 586, 187, 584, - 920, 951, 187, 1125, 76, 76, 76, 76, 76, 76, - 76, 76, 977, 807, 50, 1110, 920, 920, 76, 76, - 803, 587, 931, 690, 792, 187, 781, 782, 50, 784, - 843, 844, 845, 928, 801, 1196, 800, 836, 225, 856, - 76, 806, 805, 1090, 187, 301, 301, 301, 301, 301, - 76, 780, 821, 313, 783, 444, 980, 444, 956, 957, - 301, 755, 756, 852, 847, 444, 846, 763, 65, 301, - 502, 503, 495, 496, 497, 498, 499, 500, 501, 494, - 857, 770, 504, 772, 773, 50, 653, 859, 837, 838, - 839, 840, 1075, 1004, 76, 959, 315, 315, 315, 315, - 663, 315, 456, 702, 848, 849, 850, 853, 854, 315, - 564, 567, 568, 569, 565, 909, 566, 570, 789, 715, - 962, 787, 961, 790, 786, 187, 788, 785, 187, 187, - 187, 187, 187, 718, 229, 230, 482, 1288, 897, 898, - 187, 1282, 791, 187, 568, 569, 1037, 187, 893, 473, - 1287, 903, 187, 187, 902, 995, 76, 599, 1262, 436, - 987, 461, 1261, 471, 1209, 981, 446, 447, 448, 76, - 451, 910, 488, 462, 491, 948, 949, 1164, 457, 234, - 505, 506, 507, 508, 509, 510, 511, 947, 489, 490, - 487, 493, 492, 502, 503, 495, 496, 497, 498, 499, - 500, 501, 494, 302, 1197, 504, 866, 719, 662, 572, - 315, 473, 936, 226, 227, 220, 596, 1241, 221, 54, - 187, 1240, 1199, 76, 968, 76, 970, 960, 952, 187, - 238, 904, 187, 76, 901, 680, 978, 475, 1249, 969, - 1187, 685, 900, 56, 58, 583, 51, 1, 863, 187, - 971, 187, 187, 992, 993, 1007, 872, 1264, 1222, 1104, - 823, 444, 815, 412, 64, 1256, 982, 983, 822, 1228, - 1185, 830, 994, 989, 996, 997, 998, 833, 919, 1109, - 1259, 986, 606, 604, 605, 603, 444, 608, 301, 1001, - 607, 602, 198, 308, 934, 571, 595, 1010, 313, 858, - 476, 66, 1027, 1026, 868, 453, 200, 512, 899, 1024, - 972, 314, 1082, 691, 465, 1239, 1198, 935, 315, 539, - 765, 246, 706, 259, 256, 1034, 315, 258, 257, 697, - 695, 944, 486, 244, 236, 300, 555, 315, 315, 315, - 315, 315, 315, 315, 315, 1044, 1045, 1080, 563, 47, - 561, 315, 315, 560, 1050, 958, 779, 954, 1079, 1062, - 715, 1081, 779, 1076, 1092, 1093, 1094, 1063, 76, 299, - 1040, 187, 1161, 698, 1246, 701, 25, 1084, 1086, 55, - 1070, 231, 19, 482, 18, 76, 315, 650, 751, 753, - 17, 20, 1098, 16, 15, 659, 1085, 14, 29, 1103, - 463, 467, 13, 1102, 769, 12, 670, 671, 672, 673, - 674, 675, 676, 677, 1097, 11, 10, 485, 9, 8, - 678, 679, 7, 1118, 1119, 6, 5, 746, 76, 76, - 4, 76, 222, 22, 794, 2, 0, 760, 760, 0, - 1131, 0, 302, 760, 0, 0, 0, 0, 0, 0, - 0, 0, 529, 1133, 76, 0, 1136, 0, 0, 1138, - 760, 540, 0, 1139, 0, 0, 0, 0, 187, 0, - 1160, 0, 0, 0, 714, 0, 0, 76, 1140, 518, - 519, 520, 521, 522, 523, 524, 525, 0, 1147, 315, - 0, 0, 0, 0, 0, 0, 0, 1165, 0, 0, - 0, 0, 315, 0, 1173, 0, 0, 1176, 1177, 1178, - 0, 0, 0, 0, 0, 0, 1181, 76, 76, 1179, - 0, 0, 978, 0, 0, 0, 0, 301, 0, 444, - 0, 1144, 1145, 0, 1146, 0, 0, 1148, 1188, 1150, - 1190, 0, 0, 0, 0, 0, 76, 0, 76, 76, - 1195, 1194, 0, 0, 0, 0, 315, 0, 315, 0, - 0, 1200, 0, 0, 0, 0, 315, 1080, 0, 0, - 1213, 0, 0, 187, 0, 0, 0, 313, 1079, 0, - 0, 76, 1212, 1210, 0, 1184, 0, 0, 0, 1221, - 818, 315, 1227, 0, 76, 187, 0, 0, 1238, 0, - 0, 76, 0, 0, 917, 0, 0, 76, 918, 0, - 187, 1237, 1080, 0, 47, 922, 923, 924, 1250, 1233, - 0, 1234, 0, 1079, 933, 865, 1251, 867, 0, 939, - 1254, 940, 941, 942, 943, 886, 492, 502, 503, 495, - 496, 497, 498, 499, 500, 501, 494, 1268, 0, 504, - 0, 0, 0, 0, 0, 0, 0, 704, 705, 76, - 779, 76, 76, 76, 187, 76, 0, 1273, 1278, 0, - 0, 76, 0, 0, 0, 0, 0, 0, 0, 714, - 0, 0, 1285, 1286, 0, 0, 0, 0, 0, 0, - 0, 0, 1293, 0, 0, 0, 0, 76, 76, 76, - 0, 967, 0, 0, 0, 0, 0, 0, 0, 0, - 529, 0, 0, 757, 758, 0, 0, 716, 315, 0, - 725, 726, 727, 728, 729, 730, 731, 732, 733, 734, - 735, 736, 737, 738, 739, 0, 0, 0, 0, 76, - 76, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 76, 0, 0, 464, 0, 0, 0, 0, - 0, 1005, 315, 0, 315, 76, 0, 0, 0, 0, - 0, 0, 0, 0, 812, 0, 0, 0, 0, 0, - 1049, 0, 0, 0, 0, 76, 0, 315, 0, 0, - 185, 0, 0, 211, 0, 303, 0, 0, 0, 0, - 0, 0, 0, 0, 196, 0, 818, 0, 0, 0, - 315, 0, 0, 0, 0, 235, 0, 185, 185, 0, - 0, 76, 0, 185, 0, 0, 0, 1095, 207, 0, - 184, 1006, 315, 0, 0, 76, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 760, 0, 0, - 1083, 967, 1009, 760, 0, 0, 1033, 0, 306, 0, - 0, 0, 0, 416, 0, 0, 0, 894, 895, 0, - 467, 0, 0, 0, 0, 0, 0, 0, 191, 315, - 0, 315, 1106, 0, 0, 193, 0, 0, 0, 0, - 0, 0, 199, 195, 0, 0, 0, 0, 1043, 0, - 0, 0, 0, 0, 0, 1141, 0, 0, 0, 0, - 0, 0, 1143, 0, 1132, 0, 0, 0, 0, 197, - 1066, 0, 201, 1152, 1153, 1154, 0, 1134, 1157, 0, - 0, 185, 921, 185, 1137, 0, 0, 0, 0, 185, - 315, 1167, 1168, 1169, 0, 1172, 185, 938, 0, 0, - 192, 0, 0, 911, 912, 913, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 818, 0, 818, - 459, 424, 0, 425, 0, 0, 0, 202, 194, 432, - 203, 204, 205, 206, 210, 0, 434, 0, 0, 209, - 208, 0, 1175, 0, 1175, 1175, 1175, 0, 1180, 0, - 0, 0, 0, 0, 315, 0, 493, 492, 502, 503, - 495, 496, 497, 498, 499, 500, 501, 494, 0, 0, - 504, 0, 1207, 0, 0, 0, 0, 0, 1043, 0, - 315, 315, 315, 0, 0, 0, 0, 1218, 1219, 1220, + 273, 47, 778, 653, 529, 1224, 246, 1078, 1175, 1266, + 528, 3, 862, 1106, 796, 1009, 1079, 1075, 814, 576, + 272, 818, 856, 779, 889, 951, 967, 53, 220, 574, + 817, 915, 317, 214, 751, 748, 682, 1012, 592, 1000, + 1052, 956, 767, 842, 718, 852, 741, 414, 47, 591, + 828, 461, 467, 775, 311, 298, 225, 578, 219, 563, + 299, 473, 303, 481, 897, 229, 308, 306, 52, 1295, + 1285, 1293, 244, 1273, 1291, 863, 1284, 215, 216, 217, + 218, 1070, 1163, 418, 1233, 879, 439, 1112, 1113, 1114, + 593, 297, 594, 543, 233, 1117, 1115, 1100, 975, 878, + 1272, 974, 809, 235, 976, 1248, 494, 493, 503, 504, + 496, 497, 498, 499, 500, 501, 502, 495, 57, 446, + 505, 1101, 1102, 810, 811, 991, 883, 835, 1187, 427, + 183, 179, 180, 181, 455, 877, 1202, 843, 1152, 1150, + 456, 453, 213, 59, 60, 61, 62, 63, 450, 451, + 441, 1292, 443, 1290, 830, 1267, 494, 493, 503, 504, + 496, 497, 498, 499, 500, 501, 502, 495, 1033, 242, + 505, 776, 428, 421, 1225, 177, 248, 440, 442, 797, + 799, 661, 415, 314, 874, 871, 872, 1227, 870, 176, + 830, 177, 424, 652, 966, 965, 445, 445, 445, 445, + 76, 445, 964, 916, 188, 416, 830, 188, 1231, 445, + 191, 178, 1253, 881, 884, 1030, 517, 518, 985, 1172, + 1039, 1032, 931, 909, 690, 485, 47, 434, 815, 505, + 683, 188, 188, 76, 687, 750, 470, 188, 302, 76, + 182, 514, 495, 480, 516, 505, 1258, 829, 876, 459, + 1131, 469, 827, 825, 798, 1226, 826, 1116, 843, 438, + 1121, 954, 836, 1249, 595, 1072, 1053, 768, 891, 938, + 875, 768, 527, 656, 531, 532, 533, 534, 535, 536, + 537, 538, 539, 829, 542, 544, 544, 544, 544, 544, + 544, 544, 544, 552, 553, 554, 555, 1055, 1271, 829, + 1020, 989, 1261, 725, 575, 689, 1232, 1230, 880, 684, + 1122, 430, 431, 432, 1031, 475, 1029, 723, 724, 722, + 927, 882, 926, 928, 420, 464, 468, 1278, 1018, 1057, + 1277, 1061, 1020, 1056, 478, 1054, 479, 478, 479, 478, + 1059, 688, 486, 1074, 890, 471, 188, 832, 188, 1058, + 480, 1193, 833, 480, 188, 480, 175, 479, 478, 50, + 1018, 188, 1060, 1062, 1192, 76, 76, 76, 76, 721, + 76, 479, 478, 583, 480, 1004, 589, 530, 76, 545, + 546, 547, 548, 549, 550, 551, 541, 1003, 480, 708, + 710, 711, 992, 1019, 709, 197, 422, 423, 1024, 1021, + 1014, 1015, 1022, 1017, 1016, 76, 496, 497, 498, 499, + 500, 501, 502, 495, 1259, 1023, 505, 296, 445, 208, + 515, 1026, 906, 907, 908, 1019, 445, 1209, 1190, 314, + 1024, 1021, 1014, 1015, 1022, 1017, 1016, 445, 445, 445, + 445, 445, 445, 445, 445, 693, 694, 1023, 479, 478, + 1036, 445, 445, 1013, 498, 499, 500, 501, 502, 495, + 1001, 742, 505, 743, 1256, 480, 188, 670, 1109, 192, + 1281, 460, 460, 188, 188, 188, 194, 1217, 1264, 76, + 302, 1217, 460, 200, 196, 76, 1217, 1218, 1237, 271, + 1108, 479, 478, 1184, 1183, 1097, 460, 1236, 695, 668, + 1171, 460, 262, 261, 264, 265, 266, 267, 480, 719, + 198, 263, 268, 202, 1128, 1127, 1118, 47, 716, 986, + 74, 493, 503, 504, 496, 497, 498, 499, 500, 501, + 502, 495, 531, 977, 505, 1124, 1125, 697, 1124, 1123, + 21, 193, 755, 760, 763, 865, 712, 921, 460, 769, + 714, 560, 460, 316, 753, 460, 50, 744, 667, 419, + 666, 303, 303, 303, 303, 303, 780, 657, 203, 195, + 655, 204, 205, 206, 207, 211, 575, 650, 800, 436, + 210, 209, 705, 706, 429, 303, 755, 76, 415, 772, + 745, 746, 953, 188, 188, 76, 224, 188, 602, 601, + 188, 765, 23, 953, 188, 715, 76, 76, 76, 76, + 76, 76, 76, 76, 804, 1076, 781, 23, 952, 784, + 76, 76, 782, 783, 54, 785, 946, 188, 586, 947, + 803, 793, 585, 559, 560, 530, 802, 801, 758, 759, + 807, 952, 76, 806, 1212, 952, 188, 753, 1042, 50, + 822, 933, 76, 1167, 930, 23, 445, 560, 445, 844, + 845, 846, 560, 921, 50, 720, 445, 1130, 1126, 587, + 444, 585, 858, 565, 568, 569, 570, 566, 978, 567, + 571, 808, 921, 957, 958, 316, 316, 316, 316, 588, + 316, 921, 854, 855, 932, 691, 76, 929, 316, 813, + 1197, 226, 50, 565, 568, 569, 570, 566, 314, 567, + 571, 837, 857, 1091, 981, 654, 910, 957, 958, 860, + 696, 819, 853, 716, 848, 483, 847, 188, 65, 1111, + 188, 188, 188, 188, 188, 1076, 1005, 302, 302, 302, + 302, 302, 188, 719, 898, 188, 703, 899, 50, 188, + 960, 664, 302, 457, 188, 188, 963, 790, 76, 1289, + 788, 302, 791, 962, 787, 789, 786, 911, 756, 757, + 792, 76, 569, 570, 764, 1283, 949, 950, 752, 754, + 230, 231, 1038, 894, 239, 1288, 948, 474, 771, 904, + 773, 774, 895, 896, 770, 468, 903, 996, 600, 316, + 437, 472, 988, 1263, 303, 597, 1262, 1210, 982, 937, + 715, 503, 504, 496, 497, 498, 499, 500, 501, 502, + 495, 1165, 188, 505, 795, 76, 961, 76, 969, 462, + 971, 188, 1198, 970, 188, 76, 867, 663, 573, 979, + 474, 463, 227, 228, 902, 221, 1242, 972, 222, 54, + 1241, 188, 901, 188, 188, 1200, 953, 922, 681, 476, + 1250, 995, 445, 997, 998, 999, 1188, 447, 448, 449, + 686, 452, 939, 838, 839, 840, 841, 983, 984, 458, + 56, 58, 993, 994, 584, 51, 1, 445, 1002, 849, + 850, 851, 864, 1008, 873, 1265, 1223, 1105, 824, 720, + 816, 413, 64, 1011, 1257, 823, 1229, 316, 1186, 1025, + 831, 990, 834, 1110, 1260, 316, 987, 607, 605, 606, + 604, 609, 608, 603, 199, 1035, 316, 316, 316, 316, + 316, 316, 316, 316, 309, 572, 596, 819, 905, 859, + 316, 316, 477, 66, 1046, 1028, 1045, 1027, 1081, 1077, + 47, 869, 454, 780, 201, 513, 1051, 900, 973, 780, + 1082, 315, 699, 1071, 716, 1093, 1094, 1095, 1080, 1064, + 76, 1083, 483, 188, 1063, 316, 1087, 1085, 692, 1086, + 302, 466, 1240, 1010, 1199, 920, 936, 76, 540, 766, + 247, 707, 260, 257, 918, 1037, 1099, 1098, 919, 259, + 258, 935, 1104, 698, 945, 923, 924, 925, 487, 1103, + 245, 237, 1119, 1120, 934, 301, 747, 556, 564, 940, + 562, 941, 942, 943, 944, 561, 761, 761, 959, 1044, + 76, 76, 761, 76, 519, 520, 521, 522, 523, 524, + 525, 526, 1073, 303, 955, 300, 1132, 1041, 1162, 761, + 1247, 1067, 702, 25, 55, 232, 76, 1088, 1089, 1134, + 1139, 1090, 1137, 19, 1092, 18, 17, 1140, 20, 16, + 188, 1161, 1141, 15, 14, 29, 13, 12, 316, 76, + 1148, 11, 10, 9, 8, 7, 6, 5, 651, 4, + 223, 316, 22, 2, 0, 1166, 660, 0, 819, 0, + 819, 0, 1177, 1178, 1179, 0, 0, 671, 672, 673, + 674, 675, 676, 677, 678, 1174, 1180, 0, 0, 76, + 76, 679, 680, 0, 0, 979, 0, 1182, 0, 0, + 445, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 316, 1195, 316, 76, 0, + 76, 76, 0, 1196, 0, 316, 0, 0, 0, 1044, + 0, 0, 1160, 0, 0, 0, 0, 1189, 1081, 1191, + 1050, 1214, 0, 0, 0, 188, 0, 0, 1164, 0, + 316, 1213, 0, 76, 1211, 530, 0, 0, 1080, 0, + 1201, 0, 0, 0, 0, 1222, 76, 188, 1228, 1239, + 0, 0, 0, 76, 0, 0, 1238, 0, 1234, 76, + 1235, 0, 188, 1081, 0, 47, 0, 1096, 0, 302, + 1251, 0, 0, 819, 0, 1252, 0, 0, 0, 1255, + 0, 0, 0, 1080, 494, 493, 503, 504, 496, 497, + 498, 499, 500, 501, 502, 495, 0, 1269, 505, 0, + 1010, 819, 0, 1274, 0, 0, 0, 780, 0, 0, + 0, 76, 0, 76, 76, 76, 188, 76, 1279, 0, + 0, 0, 717, 76, 0, 726, 727, 728, 729, 730, + 731, 732, 733, 734, 735, 736, 737, 738, 739, 740, + 968, 1287, 1286, 1294, 0, 1142, 0, 0, 0, 76, + 76, 76, 1144, 304, 0, 0, 0, 316, 0, 0, + 0, 0, 0, 1153, 1154, 1155, 1145, 1146, 1158, 1147, + 0, 465, 1149, 0, 1151, 0, 866, 0, 868, 0, + 0, 1168, 1169, 1170, 0, 1173, 887, 0, 185, 0, + 0, 76, 76, 0, 0, 1268, 530, 0, 0, 0, + 1006, 316, 0, 316, 76, 0, 186, 0, 0, 212, + 0, 0, 0, 1159, 460, 0, 307, 76, 0, 0, + 1185, 417, 0, 0, 0, 0, 316, 0, 0, 0, + 0, 236, 0, 186, 186, 0, 0, 76, 0, 186, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 316, + 494, 493, 503, 504, 496, 497, 498, 499, 500, 501, + 502, 495, 1208, 0, 505, 0, 0, 0, 0, 0, + 0, 316, 0, 76, 0, 0, 0, 1219, 1220, 1221, + 0, 0, 0, 0, 0, 0, 761, 76, 0, 1084, + 968, 0, 761, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1243, 1244, 1245, 1246, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 316, 0, + 316, 1107, 0, 0, 0, 0, 0, 0, 0, 0, + 425, 0, 426, 0, 0, 0, 0, 0, 433, 0, + 0, 0, 0, 0, 0, 435, 0, 1270, 186, 0, + 186, 0, 1275, 1133, 0, 0, 186, 0, 912, 913, + 914, 0, 0, 186, 0, 1280, 1135, 0, 0, 0, + 0, 0, 0, 1138, 1156, 460, 0, 0, 0, 316, + 0, 0, 1007, 0, 0, 23, 24, 48, 26, 27, + 0, 0, 0, 0, 0, 0, 1298, 1299, 0, 0, + 0, 0, 0, 0, 42, 0, 0, 1034, 0, 28, + 0, 494, 493, 503, 504, 496, 497, 498, 499, 500, + 501, 502, 495, 0, 0, 505, 0, 0, 37, 0, + 0, 1176, 50, 1176, 1176, 1176, 0, 1181, 0, 0, + 0, 0, 0, 316, 0, 0, 0, 0, 0, 0, + 558, 0, 0, 0, 0, 0, 0, 0, 0, 582, + 0, 0, 0, 0, 0, 0, 0, 0, 186, 316, + 316, 316, 0, 0, 0, 186, 580, 186, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 185, 0, 0, 0, 0, 0, 0, 185, 579, - 185, 0, 0, 1242, 1243, 1244, 1245, 0, 0, 0, - 1036, 0, 1214, 1215, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1106, 0, 0, 0, 0, - 0, 557, 818, 0, 0, 0, 0, 0, 1175, 1193, - 581, 0, 0, 0, 0, 0, 0, 1269, 0, 0, - 0, 0, 1274, 0, 0, 0, 0, 1072, 1253, 1009, - 818, 1159, 0, 0, 0, 1279, 0, 0, 0, 0, - 0, 0, 1087, 1088, 0, 0, 1089, 0, 0, 1091, - 0, 0, 0, 0, 0, 0, 1047, 1048, 0, 0, - 0, 760, 0, 0, 1275, 0, 1297, 1298, 0, 1064, - 1065, 0, 1067, 1068, 0, 0, 0, 0, 1281, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 185, 185, - 0, 0, 185, 0, 0, 185, 0, 0, 0, 668, - 0, 0, 0, 493, 492, 502, 503, 495, 496, 497, - 498, 499, 500, 501, 494, 0, 0, 504, 0, 0, - 0, 0, 185, 0, 0, 0, 0, 0, 657, 658, - 0, 0, 661, 0, 0, 664, 0, 0, 0, 0, - 0, 185, 0, 0, 0, 0, 0, 0, 0, 0, - 668, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 684, 1163, 0, 0, 0, 0, 0, 0, - 529, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 703, 0, 0, 0, 0, 0, 1142, 0, 0, - 0, 0, 235, 0, 0, 0, 0, 235, 235, 0, - 0, 761, 761, 235, 0, 0, 0, 761, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 235, 235, 235, - 235, 0, 185, 0, 761, 185, 185, 185, 185, 185, - 0, 0, 0, 0, 1158, 459, 0, 793, 0, 0, - 185, 0, 0, 0, 579, 0, 0, 0, 0, 185, - 185, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1156, 776, 0, 0, 23, 24, 48, 26, 27, - 0, 493, 492, 502, 503, 495, 496, 497, 498, 499, - 500, 501, 494, 0, 42, 504, 0, 0, 0, 28, - 804, 0, 0, 0, 0, 0, 1202, 1203, 0, 1204, - 1205, 1206, 0, 0, 0, 0, 0, 0, 37, 0, - 0, 0, 50, 0, 0, 0, 0, 185, 0, 0, - 623, 0, 0, 0, 0, 0, 185, 0, 0, 185, - 1267, 529, 0, 493, 492, 502, 503, 495, 496, 497, - 498, 499, 500, 501, 494, 0, 185, 504, 891, 892, - 0, 0, 0, 0, 0, 668, 0, 860, 0, 0, - 0, 0, 0, 0, 0, 0, 884, 235, 0, 885, 0, 0, 30, 31, 33, 32, 35, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 887, 0, 0, 0, - 0, 0, 0, 36, 43, 44, 611, 0, 45, 46, - 34, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 38, 39, 235, 40, 41, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 624, 0, 0, - 235, 0, 0, 0, 1295, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 637, 638, - 639, 640, 641, 642, 643, 0, 644, 645, 646, 647, - 648, 625, 626, 627, 628, 609, 610, 0, 185, 612, - 0, 613, 614, 615, 616, 617, 618, 619, 620, 621, - 622, 629, 630, 631, 632, 633, 634, 635, 636, 1155, - 459, 1046, 0, 0, 0, 0, 49, 0, 0, 0, + 0, 0, 0, 460, 0, 0, 0, 0, 0, 0, + 0, 1215, 1216, 36, 43, 44, 0, 0, 45, 46, + 34, 0, 0, 0, 1107, 0, 0, 0, 0, 0, + 0, 0, 38, 39, 0, 40, 41, 1176, 0, 494, + 493, 503, 504, 496, 497, 498, 499, 500, 501, 502, + 495, 1048, 1049, 505, 0, 0, 0, 1254, 0, 1157, + 0, 0, 0, 0, 1065, 1066, 0, 1068, 1069, 0, + 0, 0, 0, 0, 0, 0, 0, 658, 659, 0, + 0, 662, 0, 0, 665, 0, 0, 0, 0, 0, + 761, 0, 0, 1276, 0, 186, 186, 0, 0, 186, + 0, 0, 186, 0, 0, 0, 669, 1282, 0, 0, + 0, 685, 0, 0, 0, 0, 49, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 186, + 704, 494, 493, 503, 504, 496, 497, 498, 499, 500, + 501, 502, 495, 0, 0, 505, 0, 0, 186, 0, + 1194, 489, 0, 492, 0, 0, 0, 669, 0, 506, + 507, 508, 509, 510, 511, 512, 1047, 490, 491, 488, + 494, 493, 503, 504, 496, 497, 498, 499, 500, 501, + 502, 495, 1143, 0, 505, 0, 494, 493, 503, 504, + 496, 497, 498, 499, 500, 501, 502, 495, 0, 236, + 505, 0, 0, 0, 236, 236, 0, 0, 762, 762, + 236, 777, 0, 0, 762, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 236, 236, 236, 236, 0, 186, + 0, 762, 186, 186, 186, 186, 186, 0, 917, 805, + 0, 0, 0, 0, 794, 0, 0, 186, 0, 0, + 0, 580, 0, 0, 0, 0, 186, 186, 494, 493, + 503, 504, 496, 497, 498, 499, 500, 501, 502, 495, + 0, 0, 505, 494, 493, 503, 504, 496, 497, 498, + 499, 500, 501, 502, 495, 0, 0, 505, 0, 0, + 0, 1203, 1204, 0, 1205, 1206, 1207, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 861, 0, 0, 0, + 0, 0, 0, 0, 0, 885, 0, 0, 886, 0, + 0, 0, 0, 0, 186, 0, 0, 0, 0, 0, + 0, 0, 0, 186, 0, 888, 186, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 493, 492, 502, 503, 495, 496, 497, 498, 499, - 500, 501, 494, 916, 0, 504, 493, 492, 502, 503, - 495, 496, 497, 498, 499, 500, 501, 494, 0, 0, - 504, 0, 0, 493, 492, 502, 503, 495, 496, 497, - 498, 499, 500, 501, 494, 0, 0, 504, 0, 0, - 0, 0, 0, 0, 0, 185, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 235, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 235, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 668, 0, 0, 0, - 0, 0, 0, 0, 0, 1039, 0, 0, 0, 0, - 0, 761, 0, 0, 0, 0, 0, 761, 493, 492, - 502, 503, 495, 496, 497, 498, 499, 500, 501, 494, - 0, 0, 504, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 186, 0, 892, 893, 0, 0, 127, + 0, 0, 669, 482, 0, 0, 0, 0, 95, 0, + 0, 0, 0, 109, 236, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 75, 0, 484, + 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, + 0, 479, 478, 0, 0, 0, 0, 0, 0, 0, + 0, 236, 0, 0, 0, 0, 0, 0, 480, 1296, + 0, 0, 0, 0, 0, 0, 0, 236, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 185, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 189, 0, 0, 0, 0, 133, 0, + 0, 147, 100, 99, 108, 186, 0, 0, 91, 0, + 139, 129, 159, 0, 130, 138, 112, 151, 134, 158, + 190, 166, 149, 165, 78, 148, 157, 89, 141, 80, + 155, 146, 118, 104, 105, 79, 0, 137, 94, 98, + 93, 126, 152, 153, 92, 173, 83, 164, 82, 84, + 163, 125, 150, 156, 119, 116, 81, 154, 117, 115, + 107, 96, 101, 131, 114, 132, 102, 122, 121, 123, + 0, 0, 0, 145, 161, 174, 86, 0, 0, 167, + 168, 169, 170, 0, 1040, 87, 124, 85, 103, 142, + 106, 113, 136, 172, 128, 140, 90, 160, 143, 0, + 0, 0, 186, 0, 0, 0, 0, 0, 0, 624, + 0, 0, 236, 0, 0, 0, 77, 0, 110, 171, + 135, 97, 162, 236, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 669, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 762, 0, + 0, 0, 0, 0, 762, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 185, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 185, 0, 0, - 1128, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1135, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 612, 0, 0, 0, 1129, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, + 0, 1136, 0, 0, 0, 0, 625, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 186, + 0, 0, 0, 0, 0, 0, 0, 638, 639, 640, + 641, 642, 643, 644, 186, 645, 646, 647, 648, 649, + 626, 627, 628, 629, 610, 611, 0, 0, 613, 0, + 614, 615, 616, 617, 618, 619, 620, 621, 622, 623, + 630, 631, 632, 633, 634, 635, 636, 637, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 579, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 580, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 401, 391, 0, 363, 403, 341, 355, 411, 356, - 357, 384, 327, 371, 126, 353, 0, 344, 322, 350, - 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, - 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, - 389, 362, 385, 332, 378, 404, 354, 382, 405, 0, - 0, 0, 75, 0, 819, 820, 0, 0, 0, 0, - 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, - 325, 328, 410, 398, 347, 348, 979, 0, 0, 0, - 0, 0, 0, 366, 370, 386, 360, 0, 0, 0, - 0, 0, 0, 0, 0, 345, 0, 377, 0, 0, - 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, - 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, - 359, 358, 402, 132, 0, 761, 146, 99, 98, 107, - 394, 343, 351, 90, 349, 138, 128, 158, 376, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, - 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, - 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, - 375, 77, 0, 109, 170, 134, 96, 161, 401, 391, - 0, 363, 403, 341, 355, 411, 356, 357, 384, 327, - 371, 126, 353, 0, 344, 322, 350, 323, 342, 365, - 94, 368, 340, 393, 374, 108, 409, 110, 379, 0, - 143, 119, 0, 0, 367, 395, 369, 389, 362, 385, - 332, 378, 404, 354, 382, 405, 0, 0, 0, 75, - 0, 819, 820, 0, 0, 0, 0, 0, 87, 0, - 381, 400, 352, 383, 321, 380, 0, 325, 328, 410, - 398, 347, 348, 0, 0, 0, 0, 0, 0, 0, - 366, 370, 386, 360, 0, 0, 0, 0, 0, 0, - 0, 0, 345, 0, 377, 0, 0, 0, 329, 326, - 0, 0, 364, 0, 0, 0, 331, 0, 346, 387, - 0, 320, 390, 396, 361, 188, 399, 359, 358, 402, - 132, 0, 0, 146, 99, 98, 107, 394, 343, 351, - 90, 349, 138, 128, 158, 376, 129, 137, 111, 150, - 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, - 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, - 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, - 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, - 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, - 120, 122, 0, 324, 0, 144, 160, 173, 86, 339, - 397, 166, 167, 168, 169, 0, 0, 0, 123, 85, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 335, 338, 333, 334, 372, 373, 406, 407, 408, - 388, 330, 0, 336, 337, 0, 392, 375, 77, 0, - 109, 170, 134, 96, 161, 401, 391, 0, 363, 403, - 341, 355, 411, 356, 357, 384, 327, 371, 126, 353, - 0, 344, 322, 350, 323, 342, 365, 94, 368, 340, - 393, 374, 108, 409, 110, 379, 0, 143, 119, 0, - 0, 367, 395, 369, 389, 362, 385, 332, 378, 404, - 354, 382, 405, 50, 0, 0, 75, 0, 0, 0, - 0, 0, 0, 0, 0, 87, 0, 381, 400, 352, - 383, 321, 380, 0, 325, 328, 410, 398, 347, 348, - 0, 0, 0, 0, 0, 0, 0, 366, 370, 386, - 360, 0, 0, 0, 0, 0, 0, 0, 0, 345, - 0, 377, 0, 0, 0, 329, 326, 0, 0, 364, - 0, 0, 0, 331, 0, 346, 387, 0, 320, 390, - 396, 361, 188, 399, 359, 358, 402, 132, 0, 0, - 146, 99, 98, 107, 394, 343, 351, 90, 349, 138, - 128, 158, 376, 129, 137, 111, 150, 133, 157, 189, - 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, - 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, - 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, - 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, - 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, - 324, 0, 144, 160, 173, 86, 339, 397, 166, 167, - 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 335, 338, - 333, 334, 372, 373, 406, 407, 408, 388, 330, 0, - 336, 337, 0, 392, 375, 77, 0, 109, 170, 134, - 96, 161, 401, 391, 0, 363, 403, 341, 355, 411, - 356, 357, 384, 327, 371, 126, 353, 0, 344, 322, - 350, 323, 342, 365, 94, 368, 340, 393, 374, 108, - 409, 110, 379, 0, 143, 119, 0, 0, 367, 395, - 369, 389, 362, 385, 332, 378, 404, 354, 382, 405, - 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, - 0, 0, 87, 0, 381, 400, 352, 383, 321, 380, - 0, 325, 328, 410, 398, 347, 348, 0, 0, 0, - 0, 0, 0, 0, 366, 370, 386, 360, 0, 0, - 0, 0, 0, 0, 1042, 0, 345, 0, 377, 0, - 0, 0, 329, 326, 0, 0, 364, 0, 0, 0, - 331, 0, 346, 387, 0, 320, 390, 396, 361, 188, - 399, 359, 358, 402, 132, 0, 0, 146, 99, 98, - 107, 394, 343, 351, 90, 349, 138, 128, 158, 376, - 129, 137, 111, 150, 133, 157, 189, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 324, 0, 144, - 160, 173, 86, 339, 397, 166, 167, 168, 169, 0, - 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 335, 338, 333, 334, 372, - 373, 406, 407, 408, 388, 330, 0, 336, 337, 0, - 392, 375, 77, 0, 109, 170, 134, 96, 161, 401, - 391, 0, 363, 403, 341, 355, 411, 356, 357, 384, - 327, 371, 126, 353, 0, 344, 322, 350, 323, 342, - 365, 94, 368, 340, 393, 374, 108, 409, 110, 379, - 0, 143, 119, 0, 0, 367, 395, 369, 389, 362, - 385, 332, 378, 404, 354, 382, 405, 0, 0, 0, - 240, 0, 0, 0, 0, 0, 0, 0, 0, 87, - 0, 381, 400, 352, 383, 321, 380, 0, 325, 328, - 410, 398, 347, 348, 0, 0, 0, 0, 0, 0, - 0, 366, 370, 386, 360, 0, 0, 0, 0, 0, - 0, 712, 0, 345, 0, 377, 0, 0, 0, 329, - 326, 0, 0, 364, 0, 0, 0, 331, 0, 346, - 387, 0, 320, 390, 396, 361, 188, 399, 359, 358, - 402, 132, 0, 0, 146, 99, 98, 107, 394, 343, - 351, 90, 349, 138, 128, 158, 376, 129, 137, 111, - 150, 133, 157, 189, 165, 148, 164, 78, 147, 156, - 88, 140, 80, 154, 145, 117, 103, 104, 79, 0, - 136, 93, 97, 92, 125, 151, 152, 91, 172, 83, - 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, - 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, - 121, 120, 122, 0, 324, 0, 144, 160, 173, 86, - 339, 397, 166, 167, 168, 169, 0, 0, 0, 123, - 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, - 159, 142, 335, 338, 333, 334, 372, 373, 406, 407, - 408, 388, 330, 0, 336, 337, 0, 392, 375, 77, - 0, 109, 170, 134, 96, 161, 401, 391, 0, 363, - 403, 341, 355, 411, 356, 357, 384, 327, 371, 126, - 353, 0, 344, 322, 350, 323, 342, 365, 94, 368, - 340, 393, 374, 108, 409, 110, 379, 0, 143, 119, - 0, 0, 367, 395, 369, 389, 362, 385, 332, 378, - 404, 354, 382, 405, 0, 0, 0, 75, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 381, 400, - 352, 383, 321, 380, 0, 325, 328, 410, 398, 347, - 348, 0, 0, 0, 0, 0, 0, 0, 366, 370, - 386, 360, 0, 0, 0, 0, 0, 0, 0, 0, - 345, 0, 377, 0, 0, 0, 329, 326, 0, 0, - 364, 0, 0, 0, 331, 0, 346, 387, 0, 320, - 390, 396, 361, 188, 399, 359, 358, 402, 132, 0, - 0, 146, 99, 98, 107, 394, 343, 351, 90, 349, - 138, 128, 158, 376, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 324, 0, 144, 160, 173, 86, 339, 397, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 335, - 338, 333, 334, 372, 373, 406, 407, 408, 388, 330, - 0, 336, 337, 0, 392, 375, 77, 0, 109, 170, - 134, 96, 161, 401, 391, 0, 363, 403, 341, 355, - 411, 356, 357, 384, 327, 371, 126, 353, 0, 344, - 322, 350, 323, 342, 365, 94, 368, 340, 393, 374, - 108, 409, 110, 379, 0, 143, 119, 0, 0, 367, - 395, 369, 389, 362, 385, 332, 378, 404, 354, 382, - 405, 0, 0, 0, 240, 0, 0, 0, 0, 0, - 0, 0, 0, 87, 0, 381, 400, 352, 383, 321, - 380, 0, 325, 328, 410, 398, 347, 348, 0, 0, - 0, 0, 0, 0, 0, 366, 370, 386, 360, 0, - 0, 0, 0, 0, 0, 0, 0, 345, 0, 377, - 0, 0, 0, 329, 326, 0, 0, 364, 0, 0, - 0, 331, 0, 346, 387, 0, 320, 390, 396, 361, - 188, 399, 359, 358, 402, 132, 0, 0, 146, 99, - 98, 107, 394, 343, 351, 90, 349, 138, 128, 158, - 376, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 324, 0, - 144, 160, 173, 86, 339, 397, 166, 167, 168, 169, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 335, 338, 333, 334, - 372, 373, 406, 407, 408, 388, 330, 0, 336, 337, - 0, 392, 375, 77, 0, 109, 170, 134, 96, 161, - 401, 391, 0, 363, 403, 341, 355, 411, 356, 357, - 384, 327, 371, 126, 353, 0, 344, 322, 350, 323, - 342, 365, 94, 368, 340, 393, 374, 108, 409, 110, - 379, 0, 143, 119, 0, 0, 367, 395, 369, 389, - 362, 385, 332, 378, 404, 354, 382, 405, 0, 0, - 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, - 87, 0, 381, 400, 352, 383, 321, 380, 0, 325, - 328, 410, 398, 347, 348, 0, 0, 0, 0, 0, - 0, 0, 366, 370, 386, 360, 0, 0, 0, 0, - 0, 0, 0, 0, 345, 0, 377, 0, 0, 0, - 329, 326, 0, 0, 364, 0, 0, 0, 331, 0, - 346, 387, 0, 320, 390, 396, 361, 188, 399, 359, - 358, 402, 132, 0, 0, 146, 99, 98, 107, 394, - 343, 351, 90, 349, 138, 128, 158, 376, 129, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 318, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 324, 0, 144, 160, 173, - 86, 339, 397, 166, 167, 168, 169, 0, 0, 0, - 319, 317, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 335, 338, 333, 334, 372, 373, 406, - 407, 408, 388, 330, 0, 336, 337, 0, 392, 375, - 77, 0, 109, 170, 134, 96, 161, 401, 391, 0, - 363, 403, 341, 355, 411, 356, 357, 384, 327, 371, - 126, 353, 0, 344, 322, 350, 323, 342, 365, 94, - 368, 340, 393, 374, 108, 409, 110, 379, 0, 143, - 119, 0, 0, 367, 395, 369, 389, 362, 385, 332, - 378, 404, 354, 382, 405, 0, 0, 0, 186, 0, - 0, 0, 0, 0, 0, 0, 0, 87, 0, 381, - 400, 352, 383, 321, 380, 0, 325, 328, 410, 398, - 347, 348, 0, 0, 0, 0, 0, 0, 0, 366, - 370, 386, 360, 0, 0, 0, 0, 0, 0, 0, - 0, 345, 0, 377, 0, 0, 0, 329, 326, 0, - 0, 364, 0, 0, 0, 331, 0, 346, 387, 0, - 320, 390, 396, 361, 188, 399, 359, 358, 402, 132, - 0, 0, 146, 99, 98, 107, 394, 343, 351, 90, - 349, 138, 128, 158, 376, 129, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 324, 0, 144, 160, 173, 86, 339, 397, - 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, - 335, 338, 333, 334, 372, 373, 406, 407, 408, 388, - 330, 0, 336, 337, 0, 392, 375, 77, 0, 109, - 170, 134, 96, 161, 401, 391, 0, 363, 403, 341, - 355, 411, 356, 357, 384, 327, 371, 126, 353, 0, - 344, 322, 350, 323, 342, 365, 94, 368, 340, 393, - 374, 108, 409, 110, 379, 0, 143, 119, 0, 0, - 367, 395, 369, 389, 362, 385, 332, 378, 404, 354, - 382, 405, 0, 0, 0, 75, 0, 0, 0, 0, - 0, 0, 0, 0, 87, 0, 381, 400, 352, 383, - 321, 380, 0, 325, 328, 410, 398, 347, 348, 0, - 0, 0, 0, 0, 0, 0, 366, 370, 386, 360, - 0, 0, 0, 0, 0, 0, 0, 0, 345, 0, - 377, 0, 0, 0, 329, 326, 0, 0, 364, 0, - 0, 0, 331, 0, 346, 387, 0, 320, 390, 396, - 361, 188, 399, 359, 358, 402, 132, 0, 0, 146, - 99, 98, 107, 394, 343, 351, 90, 349, 138, 128, - 158, 376, 129, 137, 111, 150, 133, 157, 189, 165, - 148, 164, 78, 147, 589, 88, 140, 80, 154, 145, - 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, - 151, 152, 91, 172, 83, 163, 82, 318, 162, 124, - 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, - 100, 130, 113, 131, 101, 121, 120, 122, 0, 324, - 0, 144, 160, 173, 86, 339, 397, 166, 167, 168, - 169, 0, 0, 0, 319, 317, 102, 141, 105, 112, - 135, 171, 127, 139, 89, 159, 142, 335, 338, 333, - 334, 372, 373, 406, 407, 408, 388, 330, 0, 336, - 337, 0, 392, 375, 77, 0, 109, 170, 134, 96, - 161, 401, 391, 0, 363, 403, 341, 355, 411, 356, - 357, 384, 327, 371, 126, 353, 0, 344, 322, 350, - 323, 342, 365, 94, 368, 340, 393, 374, 108, 409, - 110, 379, 0, 143, 119, 0, 0, 367, 395, 369, - 389, 362, 385, 332, 378, 404, 354, 382, 405, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 402, 392, + 0, 364, 404, 342, 356, 412, 357, 358, 385, 328, + 372, 127, 354, 0, 345, 323, 351, 324, 343, 366, + 95, 369, 341, 394, 375, 109, 410, 111, 380, 0, + 144, 120, 0, 0, 368, 396, 370, 390, 363, 386, + 333, 379, 405, 355, 383, 406, 0, 0, 0, 75, + 0, 820, 821, 0, 0, 0, 0, 0, 88, 0, + 382, 401, 353, 384, 322, 381, 0, 326, 329, 411, + 399, 348, 349, 980, 0, 0, 0, 0, 0, 0, + 367, 371, 387, 361, 0, 0, 0, 0, 0, 0, + 0, 0, 346, 0, 378, 0, 0, 0, 330, 327, + 0, 0, 365, 0, 0, 0, 332, 0, 347, 388, + 0, 321, 391, 397, 362, 189, 400, 360, 359, 403, + 133, 0, 762, 147, 100, 99, 108, 395, 344, 352, + 91, 350, 139, 129, 159, 377, 130, 138, 112, 151, + 134, 158, 190, 166, 149, 165, 78, 148, 157, 89, + 141, 80, 155, 146, 118, 104, 105, 79, 0, 137, + 94, 98, 93, 126, 152, 153, 92, 173, 83, 164, + 82, 84, 163, 125, 150, 156, 119, 116, 81, 154, + 117, 115, 107, 96, 101, 131, 114, 132, 102, 122, + 121, 123, 0, 325, 0, 145, 161, 174, 86, 340, + 398, 167, 168, 169, 170, 0, 0, 87, 124, 85, + 103, 142, 106, 113, 136, 172, 128, 140, 90, 160, + 143, 336, 339, 334, 335, 373, 374, 407, 408, 409, + 389, 331, 0, 337, 338, 0, 393, 376, 77, 0, + 110, 171, 135, 97, 162, 402, 392, 0, 364, 404, + 342, 356, 412, 357, 358, 385, 328, 372, 127, 354, + 0, 345, 323, 351, 324, 343, 366, 95, 369, 341, + 394, 375, 109, 410, 111, 380, 0, 144, 120, 0, + 0, 368, 396, 370, 390, 363, 386, 333, 379, 405, + 355, 383, 406, 0, 0, 0, 75, 0, 820, 821, + 0, 0, 0, 0, 0, 88, 0, 382, 401, 353, + 384, 322, 381, 0, 326, 329, 411, 399, 348, 349, + 0, 0, 0, 0, 0, 0, 0, 367, 371, 387, + 361, 0, 0, 0, 0, 0, 0, 0, 0, 346, + 0, 378, 0, 0, 0, 330, 327, 0, 0, 365, + 0, 0, 0, 332, 0, 347, 388, 0, 321, 391, + 397, 362, 189, 400, 360, 359, 403, 133, 0, 0, + 147, 100, 99, 108, 395, 344, 352, 91, 350, 139, + 129, 159, 377, 130, 138, 112, 151, 134, 158, 190, + 166, 149, 165, 78, 148, 157, 89, 141, 80, 155, + 146, 118, 104, 105, 79, 0, 137, 94, 98, 93, + 126, 152, 153, 92, 173, 83, 164, 82, 84, 163, + 125, 150, 156, 119, 116, 81, 154, 117, 115, 107, + 96, 101, 131, 114, 132, 102, 122, 121, 123, 0, + 325, 0, 145, 161, 174, 86, 340, 398, 167, 168, + 169, 170, 0, 0, 87, 124, 85, 103, 142, 106, + 113, 136, 172, 128, 140, 90, 160, 143, 336, 339, + 334, 335, 373, 374, 407, 408, 409, 389, 331, 0, + 337, 338, 0, 393, 376, 77, 0, 110, 171, 135, + 97, 162, 402, 392, 0, 364, 404, 342, 356, 412, + 357, 358, 385, 328, 372, 127, 354, 0, 345, 323, + 351, 324, 343, 366, 95, 369, 341, 394, 375, 109, + 410, 111, 380, 0, 144, 120, 0, 0, 368, 396, + 370, 390, 363, 386, 333, 379, 405, 355, 383, 406, + 50, 0, 0, 75, 0, 0, 0, 0, 0, 0, + 0, 0, 88, 0, 382, 401, 353, 384, 322, 381, + 0, 326, 329, 411, 399, 348, 349, 0, 0, 0, + 0, 0, 0, 0, 367, 371, 387, 361, 0, 0, + 0, 0, 0, 0, 0, 0, 346, 0, 378, 0, + 0, 0, 330, 327, 0, 0, 365, 0, 0, 0, + 332, 0, 347, 388, 0, 321, 391, 397, 362, 189, + 400, 360, 359, 403, 133, 0, 0, 147, 100, 99, + 108, 395, 344, 352, 91, 350, 139, 129, 159, 377, + 130, 138, 112, 151, 134, 158, 190, 166, 149, 165, + 78, 148, 157, 89, 141, 80, 155, 146, 118, 104, + 105, 79, 0, 137, 94, 98, 93, 126, 152, 153, + 92, 173, 83, 164, 82, 84, 163, 125, 150, 156, + 119, 116, 81, 154, 117, 115, 107, 96, 101, 131, + 114, 132, 102, 122, 121, 123, 0, 325, 0, 145, + 161, 174, 86, 340, 398, 167, 168, 169, 170, 0, + 0, 87, 124, 85, 103, 142, 106, 113, 136, 172, + 128, 140, 90, 160, 143, 336, 339, 334, 335, 373, + 374, 407, 408, 409, 389, 331, 0, 337, 338, 0, + 393, 376, 77, 0, 110, 171, 135, 97, 162, 402, + 392, 0, 364, 404, 342, 356, 412, 357, 358, 385, + 328, 372, 127, 354, 0, 345, 323, 351, 324, 343, + 366, 95, 369, 341, 394, 375, 109, 410, 111, 380, + 0, 144, 120, 0, 0, 368, 396, 370, 390, 363, + 386, 333, 379, 405, 355, 383, 406, 0, 0, 0, + 75, 0, 0, 0, 0, 0, 0, 0, 0, 88, + 0, 382, 401, 353, 384, 322, 381, 0, 326, 329, + 411, 399, 348, 349, 0, 0, 0, 0, 0, 0, + 0, 367, 371, 387, 361, 0, 0, 0, 0, 0, + 0, 1043, 0, 346, 0, 378, 0, 0, 0, 330, + 327, 0, 0, 365, 0, 0, 0, 332, 0, 347, + 388, 0, 321, 391, 397, 362, 189, 400, 360, 359, + 403, 133, 0, 0, 147, 100, 99, 108, 395, 344, + 352, 91, 350, 139, 129, 159, 377, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 325, 0, 145, 161, 174, 86, + 340, 398, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 336, 339, 334, 335, 373, 374, 407, 408, + 409, 389, 331, 0, 337, 338, 0, 393, 376, 77, + 0, 110, 171, 135, 97, 162, 402, 392, 0, 364, + 404, 342, 356, 412, 357, 358, 385, 328, 372, 127, + 354, 0, 345, 323, 351, 324, 343, 366, 95, 369, + 341, 394, 375, 109, 410, 111, 380, 0, 144, 120, + 0, 0, 368, 396, 370, 390, 363, 386, 333, 379, + 405, 355, 383, 406, 0, 0, 0, 241, 0, 0, + 0, 0, 0, 0, 0, 0, 88, 0, 382, 401, + 353, 384, 322, 381, 0, 326, 329, 411, 399, 348, + 349, 0, 0, 0, 0, 0, 0, 0, 367, 371, + 387, 361, 0, 0, 0, 0, 0, 0, 713, 0, + 346, 0, 378, 0, 0, 0, 330, 327, 0, 0, + 365, 0, 0, 0, 332, 0, 347, 388, 0, 321, + 391, 397, 362, 189, 400, 360, 359, 403, 133, 0, + 0, 147, 100, 99, 108, 395, 344, 352, 91, 350, + 139, 129, 159, 377, 130, 138, 112, 151, 134, 158, + 190, 166, 149, 165, 78, 148, 157, 89, 141, 80, + 155, 146, 118, 104, 105, 79, 0, 137, 94, 98, + 93, 126, 152, 153, 92, 173, 83, 164, 82, 84, + 163, 125, 150, 156, 119, 116, 81, 154, 117, 115, + 107, 96, 101, 131, 114, 132, 102, 122, 121, 123, + 0, 325, 0, 145, 161, 174, 86, 340, 398, 167, + 168, 169, 170, 0, 0, 87, 124, 85, 103, 142, + 106, 113, 136, 172, 128, 140, 90, 160, 143, 336, + 339, 334, 335, 373, 374, 407, 408, 409, 389, 331, + 0, 337, 338, 0, 393, 376, 77, 0, 110, 171, + 135, 97, 162, 402, 392, 0, 364, 404, 342, 356, + 412, 357, 358, 385, 328, 372, 127, 354, 0, 345, + 323, 351, 324, 343, 366, 95, 369, 341, 394, 375, + 109, 410, 111, 380, 0, 144, 120, 0, 0, 368, + 396, 370, 390, 363, 386, 333, 379, 405, 355, 383, + 406, 0, 0, 0, 75, 0, 0, 0, 0, 0, + 0, 0, 0, 88, 0, 382, 401, 353, 384, 322, + 381, 0, 326, 329, 411, 399, 348, 349, 0, 0, + 0, 0, 0, 0, 0, 367, 371, 387, 361, 0, + 0, 0, 0, 0, 0, 0, 0, 346, 0, 378, + 0, 0, 0, 330, 327, 0, 0, 365, 0, 0, + 0, 332, 0, 347, 388, 0, 321, 391, 397, 362, + 189, 400, 360, 359, 403, 133, 0, 0, 147, 100, + 99, 108, 395, 344, 352, 91, 350, 139, 129, 159, + 377, 130, 138, 112, 151, 134, 158, 190, 166, 149, + 165, 78, 148, 157, 89, 141, 80, 155, 146, 118, + 104, 105, 79, 0, 137, 94, 98, 93, 126, 152, + 153, 92, 173, 83, 164, 82, 84, 163, 125, 150, + 156, 119, 116, 81, 154, 117, 115, 107, 96, 101, + 131, 114, 132, 102, 122, 121, 123, 0, 325, 0, + 145, 161, 174, 86, 340, 398, 167, 168, 169, 170, + 0, 0, 87, 124, 85, 103, 142, 106, 113, 136, + 172, 128, 140, 90, 160, 143, 336, 339, 334, 335, + 373, 374, 407, 408, 409, 389, 331, 0, 337, 338, + 0, 393, 376, 77, 0, 110, 171, 135, 97, 162, + 402, 392, 0, 364, 404, 342, 356, 412, 357, 358, + 385, 328, 372, 127, 354, 0, 345, 323, 351, 324, + 343, 366, 95, 369, 341, 394, 375, 109, 410, 111, + 380, 0, 144, 120, 0, 0, 368, 396, 370, 390, + 363, 386, 333, 379, 405, 355, 383, 406, 0, 0, + 0, 241, 0, 0, 0, 0, 0, 0, 0, 0, + 88, 0, 382, 401, 353, 384, 322, 381, 0, 326, + 329, 411, 399, 348, 349, 0, 0, 0, 0, 0, + 0, 0, 367, 371, 387, 361, 0, 0, 0, 0, + 0, 0, 0, 0, 346, 0, 378, 0, 0, 0, + 330, 327, 0, 0, 365, 0, 0, 0, 332, 0, + 347, 388, 0, 321, 391, 397, 362, 189, 400, 360, + 359, 403, 133, 0, 0, 147, 100, 99, 108, 395, + 344, 352, 91, 350, 139, 129, 159, 377, 130, 138, + 112, 151, 134, 158, 190, 166, 149, 165, 78, 148, + 157, 89, 141, 80, 155, 146, 118, 104, 105, 79, + 0, 137, 94, 98, 93, 126, 152, 153, 92, 173, + 83, 164, 82, 84, 163, 125, 150, 156, 119, 116, + 81, 154, 117, 115, 107, 96, 101, 131, 114, 132, + 102, 122, 121, 123, 0, 325, 0, 145, 161, 174, + 86, 340, 398, 167, 168, 169, 170, 0, 0, 87, + 124, 85, 103, 142, 106, 113, 136, 172, 128, 140, + 90, 160, 143, 336, 339, 334, 335, 373, 374, 407, + 408, 409, 389, 331, 0, 337, 338, 0, 393, 376, + 77, 0, 110, 171, 135, 97, 162, 402, 392, 0, + 364, 404, 342, 356, 412, 357, 358, 385, 328, 372, + 127, 354, 0, 345, 323, 351, 324, 343, 366, 95, + 369, 341, 394, 375, 109, 410, 111, 380, 0, 144, + 120, 0, 0, 368, 396, 370, 390, 363, 386, 333, + 379, 405, 355, 383, 406, 0, 0, 0, 75, 0, + 0, 0, 0, 0, 0, 0, 0, 88, 0, 382, + 401, 353, 384, 322, 381, 0, 326, 329, 411, 399, + 348, 349, 0, 0, 0, 0, 0, 0, 0, 367, + 371, 387, 361, 0, 0, 0, 0, 0, 0, 0, + 0, 346, 0, 378, 0, 0, 0, 330, 327, 0, + 0, 365, 0, 0, 0, 332, 0, 347, 388, 0, + 321, 391, 397, 362, 189, 400, 360, 359, 403, 133, + 0, 0, 147, 100, 99, 108, 395, 344, 352, 91, + 350, 139, 129, 159, 377, 130, 138, 112, 151, 134, + 158, 190, 166, 149, 165, 78, 148, 157, 89, 141, + 80, 155, 146, 118, 104, 105, 79, 0, 137, 94, + 98, 93, 126, 152, 153, 92, 173, 83, 164, 82, + 319, 163, 125, 150, 156, 119, 116, 81, 154, 117, + 115, 107, 96, 101, 131, 114, 132, 102, 122, 121, + 123, 0, 325, 0, 145, 161, 174, 86, 340, 398, + 167, 168, 169, 170, 0, 0, 87, 320, 318, 103, + 142, 106, 113, 136, 172, 128, 140, 90, 160, 143, + 336, 339, 334, 335, 373, 374, 407, 408, 409, 389, + 331, 0, 337, 338, 0, 393, 376, 77, 0, 110, + 171, 135, 97, 162, 402, 392, 0, 364, 404, 342, + 356, 412, 357, 358, 385, 328, 372, 127, 354, 0, + 345, 323, 351, 324, 343, 366, 95, 369, 341, 394, + 375, 109, 410, 111, 380, 0, 144, 120, 0, 0, + 368, 396, 370, 390, 363, 386, 333, 379, 405, 355, + 383, 406, 0, 0, 0, 187, 0, 0, 0, 0, + 0, 0, 0, 0, 88, 0, 382, 401, 353, 384, + 322, 381, 0, 326, 329, 411, 399, 348, 349, 0, + 0, 0, 0, 0, 0, 0, 367, 371, 387, 361, + 0, 0, 0, 0, 0, 0, 0, 0, 346, 0, + 378, 0, 0, 0, 330, 327, 0, 0, 365, 0, + 0, 0, 332, 0, 347, 388, 0, 321, 391, 397, + 362, 189, 400, 360, 359, 403, 133, 0, 0, 147, + 100, 99, 108, 395, 344, 352, 91, 350, 139, 129, + 159, 377, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 325, + 0, 145, 161, 174, 86, 340, 398, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 336, 339, 334, + 335, 373, 374, 407, 408, 409, 389, 331, 0, 337, + 338, 0, 393, 376, 77, 0, 110, 171, 135, 97, + 162, 402, 392, 0, 364, 404, 342, 356, 412, 357, + 358, 385, 328, 372, 127, 354, 0, 345, 323, 351, + 324, 343, 366, 95, 369, 341, 394, 375, 109, 410, + 111, 380, 0, 144, 120, 0, 0, 368, 396, 370, + 390, 363, 386, 333, 379, 405, 355, 383, 406, 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 381, 400, 352, 383, 321, 380, 0, - 325, 328, 410, 398, 347, 348, 0, 0, 0, 0, - 0, 0, 0, 366, 370, 386, 360, 0, 0, 0, - 0, 0, 0, 0, 0, 345, 0, 377, 0, 0, - 0, 329, 326, 0, 0, 364, 0, 0, 0, 331, - 0, 346, 387, 0, 320, 390, 396, 361, 188, 399, - 359, 358, 402, 132, 0, 0, 146, 99, 98, 107, - 394, 343, 351, 90, 349, 138, 128, 158, 376, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 309, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 318, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 324, 0, 144, 160, - 173, 86, 339, 397, 166, 167, 168, 169, 0, 0, - 0, 319, 317, 312, 311, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 335, 338, 333, 334, 372, 373, - 406, 407, 408, 388, 330, 0, 336, 337, 0, 392, - 375, 77, 0, 109, 170, 134, 96, 161, 126, 0, - 0, 748, 0, 242, 0, 0, 0, 94, 0, 239, - 0, 0, 108, 282, 110, 0, 0, 143, 119, 0, - 0, 0, 0, 273, 274, 0, 0, 0, 0, 0, - 0, 0, 0, 50, 0, 0, 240, 261, 260, 263, - 264, 265, 266, 0, 0, 87, 262, 267, 268, 269, - 0, 0, 237, 254, 0, 281, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 251, 252, 233, 0, 0, - 0, 293, 0, 253, 0, 0, 248, 249, 250, 255, + 0, 88, 0, 382, 401, 353, 384, 322, 381, 0, + 326, 329, 411, 399, 348, 349, 0, 0, 0, 0, + 0, 0, 0, 367, 371, 387, 361, 0, 0, 0, + 0, 0, 0, 0, 0, 346, 0, 378, 0, 0, + 0, 330, 327, 0, 0, 365, 0, 0, 0, 332, + 0, 347, 388, 0, 321, 391, 397, 362, 189, 400, + 360, 359, 403, 133, 0, 0, 147, 100, 99, 108, + 395, 344, 352, 91, 350, 139, 129, 159, 377, 130, + 138, 112, 151, 134, 158, 190, 166, 149, 165, 78, + 148, 590, 89, 141, 80, 155, 146, 118, 104, 105, + 79, 0, 137, 94, 98, 93, 126, 152, 153, 92, + 173, 83, 164, 82, 319, 163, 125, 150, 156, 119, + 116, 81, 154, 117, 115, 107, 96, 101, 131, 114, + 132, 102, 122, 121, 123, 0, 325, 0, 145, 161, + 174, 86, 340, 398, 167, 168, 169, 170, 0, 0, + 87, 320, 318, 103, 142, 106, 113, 136, 172, 128, + 140, 90, 160, 143, 336, 339, 334, 335, 373, 374, + 407, 408, 409, 389, 331, 0, 337, 338, 0, 393, + 376, 77, 0, 110, 171, 135, 97, 162, 402, 392, + 0, 364, 404, 342, 356, 412, 357, 358, 385, 328, + 372, 127, 354, 0, 345, 323, 351, 324, 343, 366, + 95, 369, 341, 394, 375, 109, 410, 111, 380, 0, + 144, 120, 0, 0, 368, 396, 370, 390, 363, 386, + 333, 379, 405, 355, 383, 406, 0, 0, 0, 75, + 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, + 382, 401, 353, 384, 322, 381, 0, 326, 329, 411, + 399, 348, 349, 0, 0, 0, 0, 0, 0, 0, + 367, 371, 387, 361, 0, 0, 0, 0, 0, 0, + 0, 0, 346, 0, 378, 0, 0, 0, 330, 327, + 0, 0, 365, 0, 0, 0, 332, 0, 347, 388, + 0, 321, 391, 397, 362, 189, 400, 360, 359, 403, + 133, 0, 0, 147, 100, 99, 108, 395, 344, 352, + 91, 350, 139, 129, 159, 377, 130, 138, 112, 151, + 134, 158, 190, 166, 149, 165, 78, 148, 310, 89, + 141, 80, 155, 146, 118, 104, 105, 79, 0, 137, + 94, 98, 93, 126, 152, 153, 92, 173, 83, 164, + 82, 319, 163, 125, 150, 156, 119, 116, 81, 154, + 117, 115, 107, 96, 101, 131, 114, 132, 102, 122, + 121, 123, 0, 325, 0, 145, 161, 174, 86, 340, + 398, 167, 168, 169, 170, 0, 0, 87, 320, 318, + 313, 312, 106, 113, 136, 172, 128, 140, 90, 160, + 143, 336, 339, 334, 335, 373, 374, 407, 408, 409, + 389, 331, 0, 337, 338, 0, 393, 376, 77, 0, + 110, 171, 135, 97, 162, 127, 0, 0, 749, 0, + 243, 0, 0, 0, 95, 0, 240, 0, 0, 109, + 283, 111, 0, 0, 144, 120, 0, 0, 0, 0, + 274, 275, 0, 0, 0, 0, 0, 0, 0, 0, + 50, 0, 0, 241, 262, 261, 264, 265, 266, 267, + 0, 0, 88, 263, 268, 269, 270, 0, 0, 238, + 255, 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 188, 0, 0, 291, 0, 132, 0, 0, - 146, 99, 98, 107, 0, 0, 0, 90, 0, 138, - 128, 158, 0, 129, 137, 111, 150, 133, 157, 189, - 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, - 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, - 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, - 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, - 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, - 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, - 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 283, 292, - 289, 290, 287, 288, 286, 285, 284, 294, 275, 276, - 277, 278, 280, 0, 279, 77, 0, 109, 170, 134, - 96, 161, 126, 0, 0, 0, 0, 242, 0, 0, - 0, 94, 0, 239, 0, 0, 108, 282, 110, 0, - 0, 143, 119, 0, 0, 0, 0, 273, 274, 0, - 0, 0, 0, 0, 0, 0, 0, 50, 0, 459, - 240, 261, 260, 263, 264, 265, 266, 0, 0, 87, - 262, 267, 268, 269, 0, 0, 237, 254, 0, 281, + 0, 0, 252, 253, 234, 0, 0, 0, 294, 0, + 254, 0, 0, 249, 250, 251, 256, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 189, + 0, 0, 292, 0, 133, 0, 0, 147, 100, 99, + 108, 0, 0, 0, 91, 0, 139, 129, 159, 0, + 130, 138, 112, 151, 134, 158, 190, 166, 149, 165, + 78, 148, 157, 89, 141, 80, 155, 146, 118, 104, + 105, 79, 0, 137, 94, 98, 93, 126, 152, 153, + 92, 173, 83, 164, 82, 84, 163, 125, 150, 156, + 119, 116, 81, 154, 117, 115, 107, 96, 101, 131, + 114, 132, 102, 122, 121, 123, 0, 0, 0, 145, + 161, 174, 86, 0, 0, 167, 168, 169, 170, 0, + 0, 87, 124, 85, 103, 142, 106, 113, 136, 172, + 128, 140, 90, 160, 143, 284, 293, 290, 291, 288, + 289, 287, 286, 285, 295, 276, 277, 278, 279, 281, + 0, 280, 77, 0, 110, 171, 135, 97, 162, 127, + 0, 0, 0, 0, 243, 0, 0, 0, 95, 0, + 240, 0, 0, 109, 283, 111, 0, 0, 144, 120, + 0, 0, 0, 0, 274, 275, 0, 0, 0, 0, + 0, 0, 0, 0, 50, 0, 460, 241, 262, 261, + 264, 265, 266, 267, 0, 0, 88, 263, 268, 269, + 270, 0, 0, 238, 255, 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 251, - 252, 0, 0, 0, 0, 293, 0, 253, 0, 0, - 248, 249, 250, 255, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 188, 0, 0, 291, - 0, 132, 0, 0, 146, 99, 98, 107, 0, 0, - 0, 90, 0, 138, 128, 158, 0, 129, 137, 111, - 150, 133, 157, 189, 165, 148, 164, 78, 147, 156, - 88, 140, 80, 154, 145, 117, 103, 104, 79, 0, - 136, 93, 97, 92, 125, 151, 152, 91, 172, 83, - 163, 82, 84, 162, 124, 149, 155, 118, 115, 81, - 153, 116, 114, 106, 95, 100, 130, 113, 131, 101, - 121, 120, 122, 0, 0, 0, 144, 160, 173, 86, - 0, 0, 166, 167, 168, 169, 0, 0, 0, 123, - 85, 102, 141, 105, 112, 135, 171, 127, 139, 89, - 159, 142, 283, 292, 289, 290, 287, 288, 286, 285, - 284, 294, 275, 276, 277, 278, 280, 0, 279, 77, - 0, 109, 170, 134, 96, 161, 126, 0, 0, 0, - 0, 242, 0, 0, 0, 94, 0, 239, 0, 0, - 108, 282, 110, 0, 0, 143, 119, 0, 0, 0, - 0, 273, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 50, 0, 0, 240, 261, 260, 263, 264, 265, - 266, 0, 0, 87, 262, 267, 268, 269, 0, 0, - 237, 254, 0, 281, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 252, 253, 0, 0, + 0, 0, 294, 0, 254, 0, 0, 249, 250, 251, + 256, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 189, 0, 0, 292, 0, 133, 0, + 0, 147, 100, 99, 108, 0, 0, 0, 91, 0, + 139, 129, 159, 0, 130, 138, 112, 151, 134, 158, + 190, 166, 149, 165, 78, 148, 157, 89, 141, 80, + 155, 146, 118, 104, 105, 79, 0, 137, 94, 98, + 93, 126, 152, 153, 92, 173, 83, 164, 82, 84, + 163, 125, 150, 156, 119, 116, 81, 154, 117, 115, + 107, 96, 101, 131, 114, 132, 102, 122, 121, 123, + 0, 0, 0, 145, 161, 174, 86, 0, 0, 167, + 168, 169, 170, 0, 0, 87, 124, 85, 103, 142, + 106, 113, 136, 172, 128, 140, 90, 160, 143, 284, + 293, 290, 291, 288, 289, 287, 286, 285, 295, 276, + 277, 278, 279, 281, 0, 280, 77, 0, 110, 171, + 135, 97, 162, 127, 0, 0, 0, 0, 243, 0, + 0, 0, 95, 0, 240, 0, 0, 109, 283, 111, + 0, 0, 144, 120, 0, 0, 0, 0, 274, 275, + 0, 0, 0, 0, 0, 0, 0, 0, 50, 0, + 0, 241, 262, 261, 264, 265, 266, 267, 0, 0, + 88, 263, 268, 269, 270, 0, 0, 238, 255, 0, + 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 251, 252, 233, 0, 0, 0, 293, - 0, 253, 0, 0, 248, 249, 250, 255, 0, 0, + 252, 253, 234, 0, 0, 0, 294, 0, 254, 0, + 0, 249, 250, 251, 256, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 189, 0, 0, + 292, 0, 133, 0, 0, 147, 100, 99, 108, 0, + 0, 0, 91, 0, 139, 129, 159, 0, 130, 138, + 112, 151, 134, 158, 190, 166, 149, 165, 78, 148, + 157, 89, 141, 80, 155, 146, 118, 104, 105, 79, + 0, 137, 94, 98, 93, 126, 152, 153, 92, 173, + 83, 164, 82, 84, 163, 125, 150, 156, 119, 116, + 81, 154, 117, 115, 107, 96, 101, 131, 114, 132, + 102, 122, 121, 123, 0, 0, 0, 145, 161, 174, + 86, 0, 0, 167, 168, 169, 170, 0, 0, 87, + 124, 85, 103, 142, 106, 113, 136, 172, 128, 140, + 90, 160, 143, 284, 293, 290, 291, 288, 289, 287, + 286, 285, 295, 276, 277, 278, 279, 281, 0, 280, + 77, 0, 110, 171, 135, 97, 162, 127, 0, 0, + 0, 0, 243, 0, 0, 0, 95, 0, 240, 0, + 0, 109, 283, 111, 0, 0, 144, 120, 0, 0, + 0, 0, 274, 275, 0, 0, 0, 0, 0, 0, + 812, 0, 50, 0, 0, 241, 262, 261, 264, 265, + 266, 267, 0, 0, 88, 263, 268, 269, 270, 0, + 0, 238, 255, 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 188, 0, 0, 291, 0, 132, 0, 0, 146, 99, - 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, - 0, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, - 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 283, 292, 289, 290, - 287, 288, 286, 285, 284, 294, 275, 276, 277, 278, - 280, 0, 279, 77, 0, 109, 170, 134, 96, 161, - 126, 0, 0, 0, 0, 242, 0, 0, 0, 94, - 0, 239, 0, 0, 108, 282, 110, 0, 0, 143, - 119, 0, 0, 0, 0, 273, 274, 0, 0, 0, - 0, 0, 0, 811, 0, 50, 0, 0, 240, 261, - 260, 263, 264, 265, 266, 0, 0, 87, 262, 267, - 268, 269, 0, 0, 237, 254, 0, 281, 0, 0, + 0, 0, 0, 0, 252, 253, 0, 0, 0, 0, + 294, 0, 254, 0, 0, 249, 250, 251, 256, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 251, 252, 0, - 0, 0, 0, 293, 0, 253, 0, 0, 248, 249, - 250, 255, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 188, 0, 0, 291, 0, 132, - 0, 0, 146, 99, 98, 107, 0, 0, 0, 90, - 0, 138, 128, 158, 0, 129, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, - 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, - 283, 292, 289, 290, 287, 288, 286, 285, 284, 294, - 275, 276, 277, 278, 280, 23, 279, 77, 0, 109, - 170, 134, 96, 161, 0, 0, 0, 126, 0, 0, - 0, 0, 242, 0, 0, 0, 94, 0, 239, 0, - 0, 108, 282, 110, 0, 0, 143, 119, 0, 0, - 0, 0, 273, 274, 0, 0, 0, 0, 0, 0, - 0, 0, 50, 0, 0, 240, 261, 260, 263, 264, - 265, 266, 0, 0, 87, 262, 267, 268, 269, 0, - 0, 237, 254, 0, 281, 0, 0, 0, 0, 0, + 0, 189, 0, 0, 292, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 284, 293, 290, + 291, 288, 289, 287, 286, 285, 295, 276, 277, 278, + 279, 281, 23, 280, 77, 0, 110, 171, 135, 97, + 162, 0, 0, 0, 127, 0, 0, 0, 0, 243, + 0, 0, 0, 95, 0, 240, 0, 0, 109, 283, + 111, 0, 0, 144, 120, 0, 0, 0, 0, 274, + 275, 0, 0, 0, 0, 0, 0, 0, 0, 50, + 0, 0, 241, 262, 261, 264, 265, 266, 267, 0, + 0, 88, 263, 268, 269, 270, 0, 0, 238, 255, + 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 251, 252, 0, 0, 0, 0, - 293, 0, 253, 0, 0, 248, 249, 250, 255, 0, + 0, 252, 253, 0, 0, 0, 0, 294, 0, 254, + 0, 0, 249, 250, 251, 256, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 189, 0, + 0, 292, 0, 133, 0, 0, 147, 100, 99, 108, + 0, 0, 0, 91, 0, 139, 129, 159, 0, 130, + 138, 112, 151, 134, 158, 190, 166, 149, 165, 78, + 148, 157, 89, 141, 80, 155, 146, 118, 104, 105, + 79, 0, 137, 94, 98, 93, 126, 152, 153, 92, + 173, 83, 164, 82, 84, 163, 125, 150, 156, 119, + 116, 81, 154, 117, 115, 107, 96, 101, 131, 114, + 132, 102, 122, 121, 123, 0, 0, 0, 145, 161, + 174, 86, 0, 0, 167, 168, 169, 170, 0, 0, + 87, 124, 85, 103, 142, 106, 113, 136, 172, 128, + 140, 90, 160, 143, 284, 293, 290, 291, 288, 289, + 287, 286, 285, 295, 276, 277, 278, 279, 281, 0, + 280, 77, 0, 110, 171, 135, 97, 162, 127, 0, + 0, 0, 0, 243, 0, 0, 0, 95, 0, 240, + 0, 0, 109, 283, 111, 0, 0, 144, 120, 0, + 0, 0, 0, 274, 275, 0, 0, 0, 0, 0, + 0, 0, 0, 50, 0, 0, 241, 262, 261, 264, + 265, 266, 267, 0, 0, 88, 263, 268, 269, 270, + 0, 0, 238, 255, 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 188, 0, 0, 291, 0, 132, 0, 0, 146, - 99, 98, 107, 0, 0, 0, 90, 0, 138, 128, - 158, 0, 129, 137, 111, 150, 133, 157, 189, 165, - 148, 164, 78, 147, 156, 88, 140, 80, 154, 145, - 117, 103, 104, 79, 0, 136, 93, 97, 92, 125, - 151, 152, 91, 172, 83, 163, 82, 84, 162, 124, - 149, 155, 118, 115, 81, 153, 116, 114, 106, 95, - 100, 130, 113, 131, 101, 121, 120, 122, 0, 0, - 0, 144, 160, 173, 86, 0, 0, 166, 167, 168, - 169, 0, 0, 0, 123, 85, 102, 141, 105, 112, - 135, 171, 127, 139, 89, 159, 142, 283, 292, 289, - 290, 287, 288, 286, 285, 284, 294, 275, 276, 277, - 278, 280, 0, 279, 77, 0, 109, 170, 134, 96, - 161, 126, 0, 0, 0, 0, 242, 0, 0, 0, - 94, 0, 239, 0, 0, 108, 282, 110, 0, 0, - 143, 119, 0, 0, 0, 0, 273, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 50, 0, 0, 240, - 261, 260, 263, 264, 265, 266, 0, 0, 87, 262, - 267, 268, 269, 0, 0, 237, 254, 0, 281, 0, + 0, 0, 0, 0, 0, 252, 253, 0, 0, 0, + 0, 294, 0, 254, 0, 0, 249, 250, 251, 256, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 251, 252, - 0, 0, 0, 0, 293, 0, 253, 0, 0, 248, - 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 188, 0, 0, 291, 0, - 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, - 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, - 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, - 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, - 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, - 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, - 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, - 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, - 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 283, 292, 289, 290, 287, 288, 286, 285, 284, - 294, 275, 276, 277, 278, 280, 126, 279, 77, 0, - 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, - 108, 282, 110, 0, 0, 143, 119, 0, 0, 0, - 0, 273, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 50, 0, 0, 240, 261, 260, 263, 264, 265, - 266, 0, 0, 87, 262, 267, 268, 269, 0, 0, - 0, 254, 0, 281, 0, 0, 0, 0, 0, 0, + 0, 0, 189, 0, 0, 292, 0, 133, 0, 0, + 147, 100, 99, 108, 0, 0, 0, 91, 0, 139, + 129, 159, 0, 130, 138, 112, 151, 134, 158, 190, + 166, 149, 165, 78, 148, 157, 89, 141, 80, 155, + 146, 118, 104, 105, 79, 0, 137, 94, 98, 93, + 126, 152, 153, 92, 173, 83, 164, 82, 84, 163, + 125, 150, 156, 119, 116, 81, 154, 117, 115, 107, + 96, 101, 131, 114, 132, 102, 122, 121, 123, 0, + 0, 0, 145, 161, 174, 86, 0, 0, 167, 168, + 169, 170, 0, 0, 87, 124, 85, 103, 142, 106, + 113, 136, 172, 128, 140, 90, 160, 143, 284, 293, + 290, 291, 288, 289, 287, 286, 285, 295, 276, 277, + 278, 279, 281, 127, 280, 77, 0, 110, 171, 135, + 97, 162, 95, 0, 0, 0, 0, 109, 283, 111, + 0, 0, 144, 120, 0, 0, 0, 0, 274, 275, + 0, 0, 0, 0, 0, 0, 0, 0, 50, 0, + 0, 241, 262, 261, 264, 265, 266, 267, 0, 0, + 88, 263, 268, 269, 270, 0, 0, 0, 255, 0, + 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 251, 252, 0, 0, 0, 0, 293, - 0, 253, 0, 0, 248, 249, 250, 255, 0, 0, + 252, 253, 0, 0, 0, 0, 294, 0, 254, 0, + 0, 249, 250, 251, 256, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 189, 0, 0, + 292, 0, 133, 0, 0, 147, 100, 99, 108, 0, + 0, 0, 91, 0, 139, 129, 159, 1297, 130, 138, + 112, 151, 134, 158, 190, 166, 149, 165, 78, 148, + 157, 89, 141, 80, 155, 146, 118, 104, 105, 79, + 0, 137, 94, 98, 93, 126, 152, 153, 92, 173, + 83, 164, 82, 84, 163, 125, 150, 156, 119, 116, + 81, 154, 117, 115, 107, 96, 101, 131, 114, 132, + 102, 122, 121, 123, 0, 0, 0, 145, 161, 174, + 86, 0, 0, 167, 168, 169, 170, 0, 0, 87, + 124, 85, 103, 142, 106, 113, 136, 172, 128, 140, + 90, 160, 143, 284, 293, 290, 291, 288, 289, 287, + 286, 285, 295, 276, 277, 278, 279, 281, 127, 280, + 77, 0, 110, 171, 135, 97, 162, 95, 0, 0, + 0, 0, 109, 283, 111, 0, 0, 144, 120, 0, + 0, 0, 0, 274, 275, 0, 0, 0, 0, 0, + 0, 0, 0, 50, 0, 0, 241, 262, 261, 264, + 265, 266, 267, 0, 0, 88, 263, 268, 269, 270, + 0, 0, 0, 255, 0, 282, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 188, 0, 0, 291, 0, 132, 0, 0, 146, 99, - 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, - 1296, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, - 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 283, 292, 289, 290, - 287, 288, 286, 285, 284, 294, 275, 276, 277, 278, - 280, 126, 279, 77, 0, 109, 170, 134, 96, 161, - 94, 0, 0, 0, 0, 108, 282, 110, 0, 0, - 143, 119, 0, 0, 0, 0, 273, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 50, 0, 0, 240, - 261, 260, 263, 264, 265, 266, 0, 0, 87, 262, - 267, 268, 269, 0, 0, 0, 254, 0, 281, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 251, 252, - 0, 0, 0, 0, 293, 0, 253, 0, 0, 248, - 249, 250, 255, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 188, 0, 0, 291, 0, - 132, 0, 0, 146, 99, 98, 107, 0, 0, 0, - 90, 0, 138, 128, 158, 0, 129, 137, 111, 150, - 133, 157, 189, 165, 148, 164, 78, 147, 156, 88, - 140, 80, 154, 145, 117, 103, 104, 79, 0, 136, - 93, 97, 92, 125, 151, 152, 91, 172, 83, 163, - 82, 84, 162, 124, 149, 155, 118, 115, 81, 153, - 116, 114, 106, 95, 100, 130, 113, 131, 101, 121, - 120, 122, 0, 0, 0, 144, 160, 173, 86, 0, - 0, 166, 167, 168, 169, 0, 0, 0, 123, 85, - 102, 141, 105, 112, 135, 171, 127, 139, 89, 159, - 142, 283, 292, 289, 290, 287, 288, 286, 285, 284, - 294, 275, 276, 277, 278, 280, 126, 279, 77, 0, - 109, 170, 134, 96, 161, 94, 0, 0, 0, 0, - 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 75, 0, 0, 0, 0, 0, - 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 493, - 492, 502, 503, 495, 496, 497, 498, 499, 500, 501, - 494, 0, 0, 504, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 188, 0, 0, 0, 0, 132, 0, 0, 146, 99, - 98, 107, 0, 0, 0, 90, 0, 138, 128, 158, - 0, 129, 137, 111, 150, 133, 157, 189, 165, 148, - 164, 78, 147, 156, 88, 140, 80, 154, 145, 117, - 103, 104, 79, 0, 136, 93, 97, 92, 125, 151, - 152, 91, 172, 83, 163, 82, 84, 162, 124, 149, - 155, 118, 115, 81, 153, 116, 114, 106, 95, 100, - 130, 113, 131, 101, 121, 120, 122, 0, 0, 0, - 144, 160, 173, 86, 0, 0, 166, 167, 168, 169, - 0, 0, 0, 123, 85, 102, 141, 105, 112, 135, - 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 77, 0, 109, 170, 134, 96, 161, - 126, 0, 0, 0, 481, 0, 0, 0, 0, 94, - 0, 0, 0, 0, 108, 0, 110, 0, 0, 143, - 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 75, 0, - 483, 0, 0, 0, 0, 0, 0, 87, 0, 0, - 0, 0, 478, 477, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 479, + 0, 0, 0, 0, 0, 252, 253, 0, 0, 0, + 0, 294, 0, 254, 0, 0, 249, 250, 251, 256, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 189, 0, 0, 292, 0, 133, 0, 0, + 147, 100, 99, 108, 0, 0, 0, 91, 0, 139, + 129, 159, 0, 130, 138, 112, 151, 134, 158, 190, + 166, 149, 165, 78, 148, 157, 89, 141, 80, 155, + 146, 118, 104, 105, 79, 0, 137, 94, 98, 93, + 126, 152, 153, 92, 173, 83, 164, 82, 84, 163, + 125, 150, 156, 119, 116, 81, 154, 117, 115, 107, + 96, 101, 131, 114, 132, 102, 122, 121, 123, 0, + 0, 0, 145, 161, 174, 86, 0, 0, 167, 168, + 169, 170, 0, 0, 87, 124, 85, 103, 142, 106, + 113, 136, 172, 128, 140, 90, 160, 143, 284, 293, + 290, 291, 288, 289, 287, 286, 285, 295, 276, 277, + 278, 279, 281, 127, 280, 77, 0, 110, 171, 135, + 97, 162, 95, 0, 0, 0, 0, 109, 0, 111, + 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 75, 0, 0, 0, 0, 0, 0, 0, 0, + 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 188, 0, 0, 0, 0, 132, - 0, 0, 146, 99, 98, 107, 0, 0, 0, 90, - 0, 138, 128, 158, 0, 129, 137, 111, 150, 133, - 157, 189, 165, 148, 164, 78, 147, 156, 88, 140, - 80, 154, 145, 117, 103, 104, 79, 0, 136, 93, - 97, 92, 125, 151, 152, 91, 172, 83, 163, 82, - 84, 162, 124, 149, 155, 118, 115, 81, 153, 116, - 114, 106, 95, 100, 130, 113, 131, 101, 121, 120, - 122, 0, 0, 0, 144, 160, 173, 86, 0, 0, - 166, 167, 168, 169, 0, 0, 0, 123, 85, 102, - 141, 105, 112, 135, 171, 127, 139, 89, 159, 142, + 0, 0, 0, 0, 0, 0, 494, 493, 503, 504, + 496, 497, 498, 499, 500, 501, 502, 495, 0, 0, + 505, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 189, 0, 0, + 0, 0, 133, 0, 0, 147, 100, 99, 108, 0, + 0, 0, 91, 0, 139, 129, 159, 0, 130, 138, + 112, 151, 134, 158, 190, 166, 149, 165, 78, 148, + 157, 89, 141, 80, 155, 146, 118, 104, 105, 79, + 0, 137, 94, 98, 93, 126, 152, 153, 92, 173, + 83, 164, 82, 84, 163, 125, 150, 156, 119, 116, + 81, 154, 117, 115, 107, 96, 101, 131, 114, 132, + 102, 122, 121, 123, 0, 0, 0, 145, 161, 174, + 86, 0, 0, 167, 168, 169, 170, 0, 0, 87, + 124, 85, 103, 142, 106, 113, 136, 172, 128, 140, + 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 127, 0, + 77, 0, 110, 171, 135, 97, 162, 95, 0, 0, + 0, 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 126, 0, 77, 0, 109, - 170, 134, 96, 161, 94, 0, 0, 0, 0, 108, - 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 75, 0, 0, 0, + 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, + 68, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 75, 0, 0, 0, 0, 0, 0, - 0, 0, 87, 0, 0, 0, 0, 68, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 71, + 72, 0, 67, 0, 0, 0, 73, 133, 0, 0, + 147, 100, 99, 108, 0, 0, 0, 91, 0, 139, + 129, 159, 0, 130, 138, 112, 151, 134, 158, 69, + 166, 149, 165, 78, 148, 157, 89, 141, 80, 155, + 146, 118, 104, 105, 79, 0, 137, 94, 98, 93, + 126, 152, 153, 92, 173, 83, 164, 82, 84, 163, + 125, 150, 156, 119, 116, 81, 154, 117, 115, 107, + 96, 101, 131, 114, 132, 102, 122, 121, 123, 0, + 0, 0, 145, 161, 174, 86, 0, 0, 167, 168, + 169, 170, 0, 0, 87, 124, 85, 103, 142, 106, + 113, 136, 172, 128, 140, 90, 160, 143, 0, 70, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 77, 0, 110, 171, 135, + 97, 162, 127, 0, 0, 0, 579, 0, 0, 0, + 0, 95, 0, 0, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 71, 72, 0, 67, - 0, 0, 0, 73, 132, 0, 0, 146, 99, 98, - 107, 0, 0, 0, 90, 0, 138, 128, 158, 0, - 129, 137, 111, 150, 133, 157, 69, 165, 148, 164, - 78, 147, 156, 88, 140, 80, 154, 145, 117, 103, - 104, 79, 0, 136, 93, 97, 92, 125, 151, 152, - 91, 172, 83, 163, 82, 84, 162, 124, 149, 155, - 118, 115, 81, 153, 116, 114, 106, 95, 100, 130, - 113, 131, 101, 121, 120, 122, 0, 0, 0, 144, - 160, 173, 86, 0, 0, 166, 167, 168, 169, 0, - 0, 0, 123, 85, 102, 141, 105, 112, 135, 171, - 127, 139, 89, 159, 142, 0, 70, 0, 0, 0, + 187, 0, 581, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 77, 0, 109, 170, 134, 96, 161, 126, - 0, 0, 0, 578, 0, 0, 0, 0, 94, 0, - 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 186, 0, 580, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 23, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 127, 0, 77, + 0, 110, 171, 135, 97, 162, 95, 0, 0, 0, + 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 50, 0, 0, 75, 0, 0, 0, 0, + 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, - 0, 0, 23, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, - 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 50, - 0, 0, 75, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 23, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, - 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 189, 0, 0, 0, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 0, 0, 0, + 23, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 127, 0, 77, 0, 110, 171, 135, 97, + 162, 95, 0, 0, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 50, 0, 0, + 187, 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 50, 0, 0, 186, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 127, 0, 77, + 0, 110, 171, 135, 97, 162, 95, 0, 0, 0, + 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 0, 0, 0, 75, 0, 0, 700, 0, + 0, 701, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, - 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 75, 0, 0, 699, 0, 0, 700, 0, - 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 189, 0, 0, 0, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, - 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 598, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 0, 127, 0, 77, 0, 110, 171, 135, 97, + 162, 95, 0, 599, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 75, 0, 597, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, + 75, 0, 598, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 77, + 0, 110, 171, 135, 97, 162, 127, 0, 0, 0, + 579, 0, 0, 0, 0, 95, 0, 0, 0, 0, + 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 77, 0, 109, 170, - 134, 96, 161, 126, 0, 0, 0, 578, 0, 0, - 0, 0, 94, 0, 0, 0, 0, 108, 0, 110, - 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 187, 0, 581, 0, 0, 0, + 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 186, 0, 580, 0, 0, 0, 0, 0, 0, - 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, - 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, - 0, 0, 90, 0, 138, 128, 158, 0, 576, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, - 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, - 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, - 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, - 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, + 189, 0, 0, 0, 0, 133, 0, 0, 147, 100, + 99, 108, 0, 0, 0, 91, 0, 139, 129, 159, + 0, 577, 138, 112, 151, 134, 158, 190, 166, 149, + 165, 78, 148, 157, 89, 141, 80, 155, 146, 118, + 104, 105, 79, 0, 137, 94, 98, 93, 126, 152, + 153, 92, 173, 83, 164, 82, 84, 163, 125, 150, + 156, 119, 116, 81, 154, 117, 115, 107, 96, 101, + 131, 114, 132, 102, 122, 121, 123, 0, 0, 0, + 145, 161, 174, 86, 0, 0, 167, 168, 169, 170, + 0, 0, 87, 124, 85, 103, 142, 106, 113, 136, + 172, 128, 140, 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 50, 0, 0, 186, 0, 0, 0, - 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, + 0, 127, 0, 77, 0, 110, 171, 135, 97, 162, + 95, 0, 0, 0, 0, 109, 0, 111, 0, 0, + 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 50, 0, 0, 187, + 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 188, 0, 0, 0, 0, 132, 0, 0, - 146, 99, 98, 107, 0, 0, 0, 90, 0, 138, - 128, 158, 0, 129, 137, 111, 150, 133, 157, 189, - 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, - 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, - 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, - 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, - 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, - 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, - 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, + 0, 0, 0, 0, 0, 189, 0, 0, 0, 0, + 133, 0, 0, 147, 100, 99, 108, 0, 0, 0, + 91, 0, 139, 129, 159, 0, 130, 138, 112, 151, + 134, 158, 190, 166, 149, 165, 78, 148, 157, 89, + 141, 80, 155, 146, 118, 104, 105, 79, 0, 137, + 94, 98, 93, 126, 152, 153, 92, 173, 83, 164, + 82, 84, 163, 125, 150, 156, 119, 116, 81, 154, + 117, 115, 107, 96, 101, 131, 114, 132, 102, 122, + 121, 123, 0, 0, 0, 145, 161, 174, 86, 0, + 0, 167, 168, 169, 170, 0, 0, 87, 124, 85, + 103, 142, 106, 113, 136, 172, 128, 140, 90, 160, + 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 127, 0, 77, 0, + 110, 171, 135, 97, 162, 95, 0, 0, 0, 0, + 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 126, 0, 77, 0, 109, 170, 134, - 96, 161, 94, 0, 0, 0, 0, 108, 0, 110, - 0, 0, 143, 119, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 187, 0, 581, 0, 0, 0, + 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 186, 0, 580, 0, 0, 0, 0, 0, 0, - 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, - 0, 0, 132, 0, 0, 146, 99, 98, 107, 0, - 0, 0, 90, 0, 138, 128, 158, 0, 129, 137, - 111, 150, 133, 157, 189, 165, 148, 164, 78, 147, - 156, 88, 140, 80, 154, 145, 117, 103, 104, 79, - 0, 136, 93, 97, 92, 125, 151, 152, 91, 172, - 83, 163, 82, 84, 162, 124, 149, 155, 118, 115, - 81, 153, 116, 114, 106, 95, 100, 130, 113, 131, - 101, 121, 120, 122, 0, 0, 0, 144, 160, 173, - 86, 0, 0, 166, 167, 168, 169, 0, 0, 0, - 123, 85, 102, 141, 105, 112, 135, 171, 127, 139, - 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 126, 0, - 77, 0, 109, 170, 134, 96, 161, 94, 0, 0, - 0, 0, 108, 0, 110, 0, 0, 143, 119, 0, + 189, 0, 0, 0, 0, 133, 0, 0, 147, 100, + 99, 108, 0, 0, 0, 91, 0, 139, 129, 159, + 0, 130, 138, 112, 151, 134, 158, 190, 166, 149, + 165, 78, 148, 157, 89, 141, 80, 155, 146, 118, + 104, 105, 79, 0, 137, 94, 98, 93, 126, 152, + 153, 92, 173, 83, 164, 82, 84, 163, 125, 150, + 156, 119, 116, 81, 154, 117, 115, 107, 96, 101, + 131, 114, 132, 102, 122, 121, 123, 0, 0, 0, + 145, 161, 174, 86, 0, 0, 167, 168, 169, 170, + 0, 0, 87, 124, 85, 103, 142, 106, 113, 136, + 172, 128, 140, 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 75, 0, 483, 0, - 0, 0, 0, 0, 0, 87, 0, 0, 0, 0, + 0, 127, 0, 77, 0, 110, 171, 135, 97, 162, + 95, 0, 0, 0, 0, 109, 0, 111, 0, 0, + 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 75, + 0, 484, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 188, 0, 0, 0, 0, 132, 0, 0, - 146, 99, 98, 107, 0, 0, 0, 90, 0, 138, - 128, 158, 0, 129, 137, 111, 150, 133, 157, 189, - 165, 148, 164, 78, 147, 156, 88, 140, 80, 154, - 145, 117, 103, 104, 79, 0, 136, 93, 97, 92, - 125, 151, 152, 91, 172, 83, 163, 82, 84, 162, - 124, 149, 155, 118, 115, 81, 153, 116, 114, 106, - 95, 100, 130, 113, 131, 101, 121, 120, 122, 0, - 0, 0, 144, 160, 173, 86, 0, 0, 166, 167, - 168, 169, 0, 0, 0, 123, 85, 102, 141, 105, - 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, + 0, 0, 0, 0, 0, 189, 0, 0, 0, 0, + 133, 0, 0, 147, 100, 99, 108, 0, 0, 0, + 91, 0, 139, 129, 159, 0, 130, 138, 112, 151, + 134, 158, 190, 166, 149, 165, 78, 148, 157, 89, + 141, 80, 155, 146, 118, 104, 105, 79, 0, 137, + 94, 98, 93, 126, 152, 153, 92, 173, 83, 164, + 82, 84, 163, 125, 150, 156, 119, 116, 81, 154, + 117, 115, 107, 96, 101, 131, 114, 132, 102, 122, + 121, 123, 0, 0, 0, 145, 161, 174, 86, 0, + 0, 167, 168, 169, 170, 0, 0, 87, 124, 85, + 103, 142, 106, 113, 136, 172, 128, 140, 90, 160, + 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 127, 77, 0, + 110, 171, 135, 97, 162, 557, 95, 0, 0, 0, + 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 126, 77, 0, 109, 170, 134, - 96, 161, 556, 94, 0, 0, 0, 0, 108, 0, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 187, 0, 0, 0, 0, + 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 186, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, - 0, 0, 304, 0, 0, 0, 0, 0, 0, 126, - 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 189, 0, 0, 0, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 0, 0, 0, + 0, 0, 0, 0, 0, 305, 0, 0, 0, 0, + 0, 0, 127, 0, 77, 0, 110, 171, 135, 97, + 162, 95, 0, 0, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, + 187, 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 127, 0, 77, + 0, 110, 171, 135, 97, 162, 95, 0, 0, 0, + 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, - 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 187, 0, 0, 0, 0, + 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 186, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 184, + 0, 189, 0, 0, 0, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 183, 0, 188, 0, - 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, - 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 0, 127, 0, 77, 0, 110, 171, 135, 97, + 162, 95, 0, 0, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 75, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, + 75, 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 127, 0, 77, + 0, 110, 171, 135, 97, 162, 95, 0, 0, 0, + 0, 109, 0, 111, 0, 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 126, 0, 77, 0, 109, 170, - 134, 96, 161, 94, 0, 0, 0, 0, 108, 0, - 110, 0, 0, 143, 119, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 241, 0, 0, 0, 0, + 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 240, 0, 0, 0, 0, 0, 0, 0, - 0, 87, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 188, 0, - 0, 0, 0, 132, 0, 0, 146, 99, 98, 107, - 0, 0, 0, 90, 0, 138, 128, 158, 0, 129, - 137, 111, 150, 133, 157, 189, 165, 148, 164, 78, - 147, 156, 88, 140, 80, 154, 145, 117, 103, 104, - 79, 0, 136, 93, 97, 92, 125, 151, 152, 91, - 172, 83, 163, 82, 84, 162, 124, 149, 155, 118, - 115, 81, 153, 116, 114, 106, 95, 100, 130, 113, - 131, 101, 121, 120, 122, 0, 0, 0, 144, 160, - 173, 86, 0, 0, 166, 167, 168, 169, 0, 0, - 0, 123, 85, 102, 141, 105, 112, 135, 171, 127, - 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 126, - 0, 77, 0, 109, 170, 134, 96, 161, 94, 0, - 0, 0, 0, 108, 0, 110, 0, 0, 143, 119, + 0, 189, 0, 0, 0, 0, 133, 0, 0, 147, + 100, 99, 108, 0, 0, 0, 91, 0, 139, 129, + 159, 0, 130, 138, 112, 151, 134, 158, 190, 166, + 149, 165, 78, 148, 157, 89, 141, 80, 155, 146, + 118, 104, 105, 79, 0, 137, 94, 98, 93, 126, + 152, 153, 92, 173, 83, 164, 82, 84, 163, 125, + 150, 156, 119, 116, 81, 154, 117, 115, 107, 96, + 101, 131, 114, 132, 102, 122, 121, 123, 0, 0, + 0, 145, 161, 174, 86, 0, 0, 167, 168, 169, + 170, 0, 0, 87, 124, 85, 103, 142, 106, 113, + 136, 172, 128, 140, 90, 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, + 0, 0, 127, 0, 77, 0, 110, 171, 135, 97, + 162, 95, 0, 0, 0, 0, 109, 0, 111, 0, + 0, 144, 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 187, 0, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 188, 0, 0, 0, 0, 132, 0, - 0, 146, 99, 98, 107, 0, 0, 0, 90, 0, - 138, 128, 158, 0, 129, 137, 111, 150, 133, 157, - 189, 165, 148, 164, 78, 147, 156, 88, 140, 80, - 154, 145, 117, 103, 104, 79, 0, 136, 93, 97, - 92, 125, 151, 152, 91, 172, 83, 163, 82, 84, - 162, 124, 149, 155, 118, 115, 81, 153, 116, 114, - 106, 95, 100, 130, 113, 131, 101, 121, 120, 122, - 0, 0, 0, 144, 160, 173, 86, 0, 0, 166, - 167, 168, 169, 0, 0, 0, 123, 85, 102, 141, - 105, 112, 135, 171, 127, 139, 89, 159, 142, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 77, 0, 109, 170, - 134, 96, 161, + 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, + 0, 133, 0, 0, 147, 100, 99, 108, 0, 0, + 0, 91, 0, 139, 129, 159, 0, 130, 138, 112, + 151, 134, 158, 190, 166, 149, 165, 78, 148, 157, + 89, 141, 80, 155, 146, 118, 104, 105, 79, 0, + 137, 94, 98, 93, 126, 152, 153, 92, 173, 83, + 164, 82, 84, 163, 125, 150, 156, 119, 116, 81, + 154, 117, 115, 107, 96, 101, 131, 114, 132, 102, + 122, 121, 123, 0, 0, 0, 145, 161, 174, 86, + 0, 0, 167, 168, 169, 170, 0, 0, 87, 124, + 85, 103, 142, 106, 113, 136, 172, 128, 140, 90, + 160, 143, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 77, + 0, 110, 171, 135, 97, 162, } var yyPact = [...]int{ - 1949, -1000, -184, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1529, -1000, -184, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 804, 838, -1000, -1000, -1000, -1000, -1000, -1000, 615, - 7487, 41, 70, -1, 10206, 67, 1372, 10881, -1000, -9, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 561, -1000, -1000, - -1000, -1000, -1000, 798, 802, 632, 793, 695, -1000, 5648, - 40, 9080, 9981, 4946, -1000, 486, 64, 10881, -151, 10431, - 37, 37, 37, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 834, 875, -1000, -1000, -1000, -1000, -1000, -1000, 675, + 7220, 67, 91, 11, 9939, 90, 363, 10614, -1000, -13, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 649, -1000, -1000, + -1000, -1000, -1000, 828, 832, 695, 822, 741, -1000, 5615, + 51, 8813, 9714, 4913, -1000, 532, 84, 10614, -154, 10164, + 48, 48, 48, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1785,21 +1758,21 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 66, 10881, -1000, 10881, 35, 485, - 35, 35, 35, 10881, -1000, 116, -1000, -1000, -1000, -1000, - 10881, 483, 729, 33, 2970, 2970, 2970, 2970, -3, 2970, - -62, 651, -1000, -1000, -1000, -1000, -1000, 2970, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 404, 742, - 6353, 6353, 804, -1000, 561, -1000, -1000, -1000, 728, -1000, - -1000, 278, 826, -1000, 7262, 114, -1000, 6353, 700, 575, - -1000, -1000, 575, -1000, -1000, 80, -1000, -1000, 6803, 6803, - 6803, 6803, 6803, 6803, 6803, 6803, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 72, 10614, -1000, 10614, 47, + 528, 47, 47, 47, 10614, -1000, 117, -1000, -1000, -1000, + -1000, 10614, 523, 770, 30, 2937, 2937, 2937, 2937, -3, + 2937, -73, 702, -1000, -1000, -1000, -1000, -1000, 2937, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 417, + 810, 6320, 6320, 834, -1000, 649, -1000, -1000, -1000, 766, + -1000, -1000, 251, 848, -1000, 1991, 115, -1000, 6320, 1729, + 503, -1000, -1000, 503, -1000, -1000, 105, -1000, -1000, 6770, + 6770, 6770, 6770, 6770, 6770, 6770, 6770, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 575, -1000, 6119, 575, 575, 575, 575, 575, 575, 575, - 575, 6353, 575, 575, 575, 575, 575, 575, 575, 575, - 575, 575, 575, 575, 575, 9756, 528, 669, -1000, -1000, - -1000, 787, 8171, 8855, 10881, 545, -1000, 567, 4699, -86, - -1000, -1000, -1000, 221, 8621, -1000, -1000, -1000, 727, -1000, + -1000, 503, -1000, 6086, 503, 503, 503, 503, 503, 503, + 503, 503, 6320, 503, 503, 503, 503, 503, 503, 503, + 503, 503, 503, 503, 503, 503, 9489, 603, 662, -1000, + -1000, -1000, 816, 7904, 8588, 10614, 617, -1000, 635, 4666, + -129, -1000, -1000, -1000, 184, 8354, -1000, -1000, -1000, 768, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -1809,117 +1782,117 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 482, -1000, 1980, 478, 2970, 50, 634, 477, - 230, 470, 10881, 10881, 2970, 48, 10881, 785, 649, 10881, - 459, 444, -1000, 4452, -1000, 2970, 2970, 2970, 2970, 2970, - 2970, 2970, 2970, -1000, -1000, -1000, -1000, -1000, -1000, 2970, - 2970, -1000, 824, 223, -1000, -1000, 10881, -1000, -1000, -1000, - -1000, 832, 143, 339, 112, 569, -1000, 336, 798, 404, - 695, 8396, 661, -1000, -1000, 10881, -1000, 6353, 6353, 380, - -1000, 9530, -1000, -1000, 3464, 166, 6803, 331, 266, 6803, - 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, 6803, - 6803, 6803, 6803, 6803, 315, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 442, -1000, 561, 408, 408, 124, 124, - 124, 124, 124, 124, 124, 7028, 5180, 404, 474, 357, - 6119, 5648, 5648, 6353, 6353, 10656, 10656, 5648, 790, 238, - 357, 10656, -1000, 404, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 5648, 5648, 5648, 5648, 16, 10881, -1000, 10656, 9080, - 9080, 9080, 9080, 9080, -1000, 686, 683, -1000, 680, 677, - 701, 10881, -1000, 468, 8171, 120, 575, -1000, 9305, -1000, - -1000, 16, 514, 9080, 10881, -1000, -1000, 4205, 567, -86, - 559, -1000, -108, -126, 5882, 77, -1000, -1000, -1000, -1000, - 2723, 133, 249, -87, -1000, -1000, -1000, 584, -1000, 584, - 584, 584, 584, -51, -51, -51, -51, -1000, -1000, -1000, - -1000, -1000, 613, 611, -1000, 584, 584, 584, -1000, -1000, + -1000, -1000, -1000, 544, -1000, 2199, 521, 2937, 71, 663, + 514, 201, 511, 10614, 10614, 2937, 58, 10614, 814, 700, + 10614, 504, 502, -1000, 4419, -1000, 2937, 2937, 2937, 2937, + 2937, 2937, 2937, 2937, -1000, -1000, -1000, -1000, -1000, -1000, + 2937, 2937, -1000, 847, 219, -1000, -1000, 10614, -1000, -1000, + -1000, -1000, 861, 144, 287, 114, 641, -1000, 421, 828, + 417, 741, 8129, 704, -1000, -1000, 10614, -1000, 6320, 6320, + 322, -1000, 9263, -1000, -1000, 3431, 156, 6770, 306, 229, + 6770, 6770, 6770, 6770, 6770, 6770, 6770, 6770, 6770, 6770, + 6770, 6770, 6770, 6770, 6770, 405, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 501, -1000, 649, 445, 445, 124, + 124, 124, 124, 124, 124, 124, 6995, 5147, 417, 500, + 378, 6086, 5615, 5615, 6320, 6320, 10389, 10389, 5615, 819, + 195, 378, 10389, -1000, 417, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 5615, 5615, 5615, 5615, 28, 10614, -1000, 10389, + 8813, 8813, 8813, 8813, 8813, -1000, 725, 723, -1000, 719, + 716, 729, 10614, -1000, 497, 7904, 130, 503, -1000, 9038, + -1000, -1000, 28, 578, 8813, 10614, -1000, -1000, 4172, 635, + -129, 627, -1000, -118, -99, 5849, 123, -1000, -1000, -1000, + -1000, 2690, 126, 280, -76, -1000, -1000, -1000, 658, -1000, + 658, 658, 658, 658, -41, -41, -41, -41, -1000, -1000, + -1000, -1000, -1000, 673, 671, -1000, 658, 658, 658, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 610, 610, 610, 586, 586, 635, - -1000, 10881, -168, 430, 2970, 783, 2970, -1000, 61, -1000, - 10881, -1000, -1000, 10881, 2970, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 669, 669, 669, 659, 659, + 667, -1000, 10614, -173, 489, 2937, 813, 2937, -1000, 70, + -1000, 10614, -1000, -1000, 10614, 2937, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 10881, 257, 10881, 10881, -1000, -1000, 711, 6353, 6353, 3958, - 6353, -1000, -1000, -1000, 742, -1000, 790, 823, -1000, 721, - 718, 5648, -1000, -1000, 166, 236, -1000, -1000, 366, -1000, - -1000, -1000, -1000, 97, 575, -1000, 2217, -1000, -1000, -1000, - -1000, 331, 6803, 6803, 6803, 118, 2217, 2142, 577, 1144, - 124, 150, 150, 140, 140, 140, 140, 140, 234, 234, - -1000, -1000, -1000, 404, -1000, -1000, -1000, 404, 5648, 562, - -1000, -1000, 6353, -1000, 404, 466, 466, 299, 321, 579, - -1000, 92, 568, 466, 5648, 233, -1000, 6353, 404, -1000, - 466, 404, 466, 466, 521, 575, -1000, 547, -1000, 216, - 669, 607, 644, 437, -1000, -1000, -1000, -1000, 681, -1000, - 679, -1000, -1000, -1000, -1000, -1000, 63, 62, 57, 10431, - -1000, 816, 9080, 518, -1000, -1000, 559, -86, -99, -1000, - -1000, -1000, 357, -1000, 420, 558, 2476, -1000, -1000, -1000, - -1000, -1000, -1000, 603, 737, 146, 145, 375, -1000, -1000, - 731, -1000, 259, -91, -1000, -1000, 330, -51, -51, -1000, - -1000, 77, 725, 77, 77, 77, 361, 361, -1000, -1000, - -1000, -1000, 328, -1000, -1000, -1000, 326, -1000, 642, 10431, - 2970, -1000, 3711, -1000, -1000, -1000, -1000, -1000, -1000, 376, - 269, 199, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 15, -1000, 2970, -1000, 223, -1000, 358, - 6353, -1000, -1000, 708, 357, 357, 84, -1000, -1000, 10881, - -1000, -1000, -1000, -1000, 546, -1000, -1000, -1000, 3217, 5648, - -1000, 118, 2217, 2110, -1000, 6803, 6803, -1000, -1000, 466, - 5648, 357, -1000, -1000, -1000, 135, 315, 135, 6803, 6803, - 3958, 6803, 6803, -163, 563, 226, -1000, 6353, 343, -1000, - -1000, -1000, -1000, -1000, 641, 10656, 575, -1000, 7946, 10431, - 804, 10656, 6353, 6353, -1000, -1000, 6353, 590, -1000, 6353, - -1000, -1000, -1000, 575, 575, 575, 436, -1000, 804, 518, - -1000, -1000, -1000, -133, -131, -1000, -1000, 2723, -1000, 2723, - 10431, -1000, 373, 368, -1000, -1000, 564, 72, -1000, -1000, - -1000, 469, 77, 77, -1000, 242, -1000, -1000, -1000, 463, - -1000, 457, 549, 455, 10881, -1000, -1000, 532, -1000, 190, - -1000, -1000, 10431, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 10431, 10881, -1000, -1000, -1000, - -1000, -1000, 10431, -1000, 257, -1000, 357, -1000, 3711, -1000, - 816, 9080, -1000, -1000, 404, -1000, 6803, 2217, 2217, -1000, - -1000, 404, 584, 584, -1000, 584, 586, -1000, 584, -26, - 584, -29, 404, 404, 2125, 1932, -1000, 1870, 1702, 575, - -160, -1000, 357, 6353, -1000, 750, 499, 501, -1000, -1000, - 5414, 404, 440, 83, 436, 798, -1000, 357, 357, 357, - 10431, 357, 10431, 10431, 10431, 7721, 10431, 798, -1000, -1000, - -1000, -1000, 2476, -1000, 407, -1000, 584, -1000, -1000, -77, - 831, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -51, 347, -51, 308, -1000, 303, 2970, 3711, - 2723, -1000, 582, -1000, -1000, -1000, -1000, 778, -1000, 809, - 529, -1000, 2217, -1000, -1000, 68, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 6803, 6803, -1000, 6803, 6803, - 6803, 404, 344, 357, 736, -1000, 575, -1000, -1000, 543, - 10431, 10431, -1000, -1000, 403, -1000, 401, 401, 401, 120, - -1000, -1000, 102, 10431, -1000, 138, -1000, -141, 77, -1000, - 77, 458, 441, -1000, -1000, -1000, 10431, 575, 807, 801, - -1000, -1000, 1515, 1515, 1515, 1515, 9, -1000, -1000, 829, - -1000, 575, -1000, 561, 78, -1000, 10431, -1000, -1000, -1000, - -1000, -1000, 102, -1000, 359, 178, 316, -1000, 262, 734, - -1000, 730, -1000, -1000, -1000, -1000, -1000, 399, 14, -1000, - 6353, 6353, -1000, -1000, -1000, -1000, 404, 36, -172, 10656, - 501, 404, 10431, -1000, -1000, -1000, 288, -1000, -1000, -1000, - 300, -1000, -1000, 634, 391, -1000, 10431, 357, 491, -1000, - 703, -166, -180, 484, -1000, -1000, -1000, -1000, -168, -1000, - 14, 717, -1000, 699, -1000, -1000, -1000, 11, -169, 4, - -178, 575, -181, 6578, -1000, 1515, 404, -1000, -1000, + -1000, 10614, 256, 10614, 10614, -1000, -1000, 746, 6320, 6320, + 3925, 6320, -1000, -1000, -1000, 810, -1000, 819, 833, -1000, + 763, 756, 5615, -1000, -1000, 156, 263, -1000, -1000, 355, + -1000, -1000, -1000, -1000, 113, 503, -1000, 1832, -1000, -1000, + -1000, -1000, 306, 6770, 6770, 6770, 65, 1832, 1817, 718, + 429, 124, 357, 357, 140, 140, 140, 140, 140, 311, + 311, -1000, -1000, -1000, 417, -1000, -1000, -1000, 417, 5615, + 628, -1000, -1000, 6320, -1000, 417, 493, 493, 268, 301, + 643, -1000, 112, 640, 493, 5615, 191, -1000, 6320, 417, + -1000, 493, 417, 493, 493, 596, 503, -1000, 591, -1000, + 181, 662, 666, 699, 632, -1000, -1000, -1000, -1000, 722, + -1000, 715, -1000, -1000, -1000, -1000, -1000, 81, 74, 73, + 10164, -1000, 844, 8813, 580, -1000, -1000, 627, -129, -123, + -1000, -1000, -1000, 378, -1000, 477, 624, 2443, -1000, -1000, + -1000, -1000, -1000, -1000, 661, 780, 178, 162, 463, -1000, + -1000, 773, -1000, 234, -79, -1000, -1000, 333, -41, -41, + -1000, -1000, 123, 767, 123, 123, 123, 402, 402, -1000, + -1000, -1000, -1000, 328, -1000, -1000, -1000, 316, -1000, 685, + 10164, 2937, -1000, 3678, -1000, -1000, -1000, -1000, -1000, -1000, + 304, 272, 193, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 25, -1000, 2937, -1000, 219, -1000, + 392, 6320, -1000, -1000, 744, 378, 378, 110, -1000, -1000, + 10614, -1000, -1000, -1000, -1000, 637, -1000, -1000, -1000, 3184, + 5615, -1000, 65, 1832, 1745, -1000, 6770, 6770, -1000, -1000, + 493, 5615, 378, -1000, -1000, -1000, 160, 405, 160, 6770, + 6770, 3925, 6770, 6770, -164, 609, 186, -1000, 6320, 266, + -1000, -1000, -1000, -1000, -1000, 684, 10389, 503, -1000, 7679, + 10164, 834, 10389, 6320, 6320, -1000, -1000, 6320, 660, -1000, + 6320, -1000, -1000, -1000, 503, 503, 503, 441, -1000, 834, + 580, -1000, -1000, -1000, -124, -104, -1000, -1000, 2690, -1000, + 2690, 10164, -1000, 434, 412, -1000, -1000, 678, 29, -1000, + -1000, -1000, 461, 123, 123, -1000, 204, -1000, -1000, -1000, + 484, -1000, 481, 614, 460, 10614, -1000, -1000, 613, -1000, + 170, -1000, -1000, 10164, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 10164, 10614, -1000, -1000, + -1000, -1000, -1000, 10164, -1000, 256, -1000, 378, -1000, 3678, + -1000, 844, 8813, -1000, -1000, 417, -1000, 6770, 1832, 1832, + -1000, -1000, 417, 658, 658, -1000, 658, 659, -1000, 658, + -24, 658, -25, 417, 417, 1470, 1690, -1000, 1309, 1143, + 503, -161, -1000, 378, 6320, -1000, 794, 564, 599, -1000, + -1000, 5381, 417, 446, 109, 441, 828, -1000, 378, 378, + 378, 10164, 378, 10164, 10164, 10164, 7454, 10164, 828, -1000, + -1000, -1000, -1000, 2443, -1000, 439, -1000, 658, -1000, -1000, + -72, 857, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -41, 370, -41, 305, -1000, 292, 2937, + 3678, 2690, -1000, 647, -1000, -1000, -1000, -1000, 806, -1000, + 842, 608, -1000, 1832, -1000, -1000, 80, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 6770, 6770, -1000, 6770, + 6770, 6770, 417, 369, 378, 779, -1000, 503, -1000, -1000, + 611, 10164, 10164, -1000, -1000, 432, -1000, 427, 427, 427, + 130, -1000, -1000, 122, 10164, -1000, 180, -1000, -144, 123, + -1000, 123, 442, 433, -1000, -1000, -1000, 10164, 503, 836, + 830, -1000, -1000, 1598, 1598, 1598, 1598, 15, -1000, -1000, + 851, -1000, 503, -1000, 649, 102, -1000, 10164, -1000, -1000, + -1000, -1000, -1000, 122, -1000, 408, 166, 356, -1000, 237, + 778, -1000, 775, -1000, -1000, -1000, -1000, -1000, 423, 12, + -1000, 6320, 6320, -1000, -1000, -1000, -1000, 417, 52, -176, + 10389, 599, 417, 10164, -1000, -1000, -1000, 271, -1000, -1000, + -1000, 269, -1000, -1000, 663, 416, -1000, 10164, 378, 593, + -1000, 737, -171, -180, 587, -1000, -1000, -1000, -1000, -173, + -1000, 12, 752, -1000, 721, -1000, -1000, -1000, 8, -174, + 5, -178, 503, -181, 6545, -1000, 1598, 417, -1000, -1000, } var yyPgo = [...]int{ - 0, 1035, 12, 492, 1033, 1032, 1030, 1026, 1025, 1022, - 1019, 1018, 1016, 1015, 1005, 1002, 998, 997, 994, 993, - 991, 990, 984, 982, 115, 981, 979, 976, 55, 975, - 64, 974, 972, 34, 129, 35, 33, 779, 970, 36, - 91, 65, 969, 43, 957, 955, 67, 953, 53, 950, - 948, 1395, 936, 935, 9, 22, 934, 933, 932, 931, - 63, 830, 929, 928, 927, 924, 923, 922, 49, 5, - 6, 11, 16, 921, 85, 10, 920, 46, 919, 917, - 916, 915, 45, 914, 47, 913, 18, 44, 912, 14, - 52, 27, 20, 17, 66, 51, 911, 26, 54, 42, - 910, 908, 350, 907, 906, 37, 905, 24, 136, 349, - 904, 903, 902, 901, 29, 168, 501, 589, 58, 900, - 899, 896, 1355, 59, 57, 23, 895, 28, 570, 32, - 893, 892, 39, 891, 890, 887, 885, 884, 883, 882, - 78, 881, 880, 879, 15, 30, 877, 873, 61, 31, - 871, 870, 869, 41, 50, 868, 40, 865, 864, 863, - 862, 25, 19, 860, 8, 859, 7, 858, 857, 2, - 856, 21, 855, 3, 848, 4, 38, 847, 846, 0, - 460, 845, 844, 92, + 0, 1093, 10, 540, 1092, 1090, 1089, 1087, 1086, 1085, + 1084, 1083, 1082, 1081, 1077, 1076, 1075, 1074, 1073, 1069, + 1068, 1066, 1065, 1063, 118, 1055, 1054, 1053, 61, 1052, + 65, 1050, 1048, 31, 235, 35, 34, 103, 1047, 29, + 55, 60, 1045, 41, 1044, 1028, 67, 1025, 59, 1020, + 1018, 1303, 1017, 1015, 14, 25, 1011, 1010, 1008, 1004, + 72, 784, 1003, 1000, 999, 993, 992, 991, 44, 4, + 7, 20, 16, 990, 176, 6, 989, 42, 988, 986, + 984, 982, 27, 981, 52, 978, 28, 51, 971, 8, + 53, 26, 17, 2, 66, 49, 961, 23, 54, 38, + 958, 957, 356, 955, 954, 36, 952, 24, 129, 324, + 951, 947, 945, 943, 32, 169, 489, 119, 63, 942, + 939, 936, 1321, 64, 57, 19, 935, 33, 670, 46, + 934, 924, 40, 923, 922, 921, 920, 919, 918, 917, + 262, 916, 914, 913, 43, 18, 912, 911, 45, 22, + 910, 908, 906, 39, 47, 905, 50, 904, 902, 901, + 900, 30, 21, 898, 13, 897, 5, 896, 895, 9, + 894, 15, 893, 12, 892, 3, 37, 886, 885, 0, + 249, 884, 881, 93, } var yyR1 = [...]int{ @@ -2001,7 +1974,7 @@ var yyR1 = [...]int{ 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, 115, - 115, 115, 115, 179, 180, 127, 128, 128, 128, + 115, 115, 115, 115, 179, 180, 127, 128, 128, 128, } var yyR2 = [...]int{ @@ -2083,7 +2056,7 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 0, 0, 1, 1, + 1, 1, 1, 1, 1, 1, 0, 0, 1, 1, } var yyChk = [...]int{ @@ -2095,128 +2068,128 @@ var yyChk = [...]int{ 53, -178, 252, -82, 15, -26, 5, -24, -182, -24, -24, -24, -24, -24, -158, 53, -113, 122, 70, 149, 229, 119, 120, 126, -116, 56, -115, 245, 153, 164, - 158, 185, 177, 175, 178, 216, 205, 65, 156, 225, - 137, 173, 169, 167, 27, 190, 250, 168, 132, 131, - 191, 195, 217, 162, 163, 219, 189, 133, 32, 247, - 34, 145, 220, 193, 188, 184, 187, 161, 183, 38, - 197, 196, 198, 215, 180, 170, 18, 223, 140, 143, - 192, 194, 127, 147, 249, 221, 166, 144, 139, 224, - 157, 218, 227, 37, 202, 160, 130, 154, 151, 181, - 146, 171, 172, 186, 159, 182, 155, 148, 141, 226, - 203, 251, 179, 176, 152, 150, 208, 209, 210, 211, - 248, 222, 174, 204, -102, 122, 124, 120, 120, 121, - 122, 229, 119, 120, -51, -122, 56, -115, 122, 149, - 120, 106, 178, 113, 206, 121, 32, 147, -131, 120, - -104, 150, 205, 208, 209, 210, 211, 56, 218, 217, - 212, -122, 155, -127, -127, -127, -127, -127, -2, -86, - 17, 16, -5, -3, -179, 6, 20, 21, -30, 39, - 40, -25, -36, 97, -37, -122, -56, 72, -61, 29, - 56, -115, 23, -60, -57, -75, -73, -74, 106, 107, - 108, 95, 96, 103, 73, 109, -65, -63, -64, -66, - 58, 57, 66, 59, 60, 61, 62, 67, 68, 69, - -116, -71, -179, 43, 44, 238, 239, 240, 241, 244, - 242, 75, 33, 228, 236, 235, 234, 232, 233, 230, - 231, 125, 229, 101, 237, -102, -39, -40, -41, -42, - -53, -74, -179, -51, 11, -46, -51, -94, -130, 155, - -98, 218, 217, -117, -96, -116, -114, 216, 178, 215, - 118, 71, 22, 24, 200, 74, 106, 16, 75, 105, - 238, 113, 47, 230, 231, 228, 240, 241, 229, 206, - 29, 10, 25, 135, 21, 99, 115, 78, 79, 138, - 23, 136, 69, 19, 50, 11, 13, 14, 125, 124, - 90, 121, 45, 8, 109, 26, 87, 41, 28, 43, - 88, 17, 232, 233, 31, 244, 142, 101, 48, 35, - 72, 67, 51, 70, 15, 46, 89, 116, 237, 44, - 119, 6, 243, 30, 134, 42, 120, 207, 77, 123, - 68, 5, 126, 9, 49, 52, 234, 235, 236, 33, - 76, 12, -159, -154, 56, 121, -51, 237, -116, -109, - 125, -109, -109, 120, -51, -51, -108, 125, 56, -108, - -108, -108, -51, 110, -51, 56, 30, 229, 56, 147, - 120, 148, 122, -128, -179, -117, -128, -128, -128, 151, - 152, -128, 214, -106, 207, 213, 51, -128, -180, 55, - -87, 19, 31, -37, -122, -83, -84, -37, -82, -2, - -24, 35, -28, 21, 64, 11, -119, 71, 70, 87, - -118, 22, -116, 58, 110, -37, -58, 90, 72, 88, - 89, 74, 92, 91, 102, 95, 96, 97, 98, 99, - 100, 101, 93, 94, 105, 80, 81, 82, 83, 84, - 85, 86, -103, -179, -74, -179, 111, 112, -61, -61, - -61, -61, -61, -61, -61, -61, -179, -2, -69, -37, - -179, -179, -179, -179, -179, -179, -179, -179, -179, -78, - -37, -179, -183, -179, -183, -183, -183, -183, -183, -183, - -183, -179, -179, -179, -179, -52, 26, -51, 30, 54, - -47, -49, -48, -50, 41, 45, 47, 42, 43, 44, - 48, -126, 22, -39, -179, -125, 143, -124, 22, -122, - 58, -51, -46, -181, 54, 11, 52, 54, -94, 155, - -95, -99, 219, 221, 80, -121, -116, 58, 29, 30, - 55, 54, -133, -136, -138, -137, -139, -134, -135, 175, - 176, 106, 179, 181, 182, 183, 184, 185, 186, 187, - 188, 189, 190, 30, 137, 171, 172, 173, 174, 191, - 192, 193, 194, 195, 196, 197, 198, 158, 159, 160, - 161, 162, 163, 164, 166, 167, 168, 169, 170, 56, - -128, 122, -175, 52, 56, 72, 56, -51, -51, -128, - 123, -51, 23, 51, -51, 56, 56, -123, -122, -114, - -128, -128, -128, -128, -128, -128, -128, -128, -128, -128, - 11, -105, 11, 90, -51, 9, 90, 54, 18, 110, - 54, -85, 24, 25, -86, -180, -30, -62, -116, 59, - 62, -29, 42, -51, -37, -37, -67, 67, 72, 68, - 69, -118, 97, -123, -117, -114, -61, -68, -71, -74, - 63, 90, 88, 89, 74, -61, -61, -61, -61, -61, + 158, 185, 177, 175, 178, 216, 205, 214, 65, 156, + 225, 137, 173, 169, 167, 27, 190, 250, 168, 132, + 131, 191, 195, 217, 162, 163, 219, 189, 133, 32, + 247, 34, 145, 220, 193, 188, 184, 187, 161, 183, + 38, 197, 196, 198, 215, 180, 170, 18, 223, 140, + 143, 192, 194, 127, 147, 249, 221, 166, 144, 139, + 224, 157, 218, 227, 37, 202, 160, 130, 154, 151, + 181, 146, 171, 172, 186, 159, 182, 155, 148, 141, + 226, 203, 251, 179, 176, 152, 150, 208, 209, 210, + 211, 248, 222, 174, 204, -102, 122, 124, 120, 120, + 121, 122, 229, 119, 120, -51, -122, 56, -115, 122, + 149, 120, 106, 178, 113, 206, 121, 32, 147, -131, + 120, -104, 150, 205, 208, 209, 210, 211, 56, 218, + 217, 212, -122, 155, -127, -127, -127, -127, -127, -2, + -86, 17, 16, -5, -3, -179, 6, 20, 21, -30, + 39, 40, -25, -36, 97, -37, -122, -56, 72, -61, + 29, 56, -115, 23, -60, -57, -75, -73, -74, 106, + 107, 108, 95, 96, 103, 73, 109, -65, -63, -64, + -66, 58, 57, 66, 59, 60, 61, 62, 67, 68, + 69, -116, -71, -179, 43, 44, 238, 239, 240, 241, + 244, 242, 75, 33, 228, 236, 235, 234, 232, 233, + 230, 231, 125, 229, 101, 237, -102, -39, -40, -41, + -42, -53, -74, -179, -51, 11, -46, -51, -94, -130, + 155, -98, 218, 217, -117, -96, -116, -114, 216, 178, + 215, 118, 71, 22, 24, 200, 74, 106, 16, 75, + 105, 238, 113, 47, 230, 231, 228, 240, 241, 229, + 206, 29, 10, 25, 135, 21, 99, 115, 78, 79, + 138, 23, 136, 69, 19, 50, 11, 13, 14, 125, + 124, 90, 121, 45, 8, 109, 26, 87, 41, 28, + 43, 88, 17, 232, 233, 31, 244, 142, 101, 48, + 35, 72, 67, 51, 70, 15, 46, 89, 116, 237, + 44, 119, 6, 243, 30, 134, 42, 120, 207, 77, + 123, 68, 5, 126, 9, 49, 52, 234, 235, 236, + 33, 76, 12, -159, -154, 56, 121, -51, 237, -116, + -109, 125, -109, -109, 120, -51, -51, -108, 125, 56, + -108, -108, -108, -51, 110, -51, 56, 30, 229, 56, + 147, 120, 148, 122, -128, -179, -117, -128, -128, -128, + 151, 152, -128, 214, -106, 207, 213, 51, -128, -180, + 55, -87, 19, 31, -37, -122, -83, -84, -37, -82, + -2, -24, 35, -28, 21, 64, 11, -119, 71, 70, + 87, -118, 22, -116, 58, 110, -37, -58, 90, 72, + 88, 89, 74, 92, 91, 102, 95, 96, 97, 98, + 99, 100, 101, 93, 94, 105, 80, 81, 82, 83, + 84, 85, 86, -103, -179, -74, -179, 111, 112, -61, + -61, -61, -61, -61, -61, -61, -61, -179, -2, -69, + -37, -179, -179, -179, -179, -179, -179, -179, -179, -179, + -78, -37, -179, -183, -179, -183, -183, -183, -183, -183, + -183, -183, -179, -179, -179, -179, -52, 26, -51, 30, + 54, -47, -49, -48, -50, 41, 45, 47, 42, 43, + 44, 48, -126, 22, -39, -179, -125, 143, -124, 22, + -122, 58, -51, -46, -181, 54, 11, 52, 54, -94, + 155, -95, -99, 219, 221, 80, -121, -116, 58, 29, + 30, 55, 54, -133, -136, -138, -137, -139, -134, -135, + 175, 176, 106, 179, 181, 182, 183, 184, 185, 186, + 187, 188, 189, 190, 30, 137, 171, 172, 173, 174, + 191, 192, 193, 194, 195, 196, 197, 198, 158, 159, + 160, 161, 162, 163, 164, 166, 167, 168, 169, 170, + 56, -128, 122, -175, 52, 56, 72, 56, -51, -51, + -128, 123, -51, 23, 51, -51, 56, 56, -123, -122, + -114, -128, -128, -128, -128, -128, -128, -128, -128, -128, + -128, 11, -105, 11, 90, -51, 9, 90, 54, 18, + 110, 54, -85, 24, 25, -86, -180, -30, -62, -116, + 59, 62, -29, 42, -51, -37, -37, -67, 67, 72, + 68, 69, -118, 97, -123, -117, -114, -61, -68, -71, + -74, 63, 90, 88, 89, 74, -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, -61, - -129, 56, 58, 56, -60, -60, -116, -35, 21, -34, - -36, -180, 54, -180, -2, -34, -34, -37, -37, -75, - -116, -122, -75, -34, -28, -76, -77, 76, -75, -180, - -34, -35, -34, -34, -90, 143, -51, -93, -97, -75, - -40, -41, -41, -40, -41, 41, 41, 41, 46, 41, - 46, 41, -48, -122, -180, -54, 49, 124, 50, -179, - -124, -90, 52, -39, -51, -98, -95, 54, 220, 222, - 223, 51, -37, -145, 105, -160, -161, -162, -117, 58, - 59, -154, -155, -163, 127, 130, 126, -156, 121, 28, - -150, 67, 72, -146, 203, -140, 53, -140, -140, -140, - -140, -144, 178, -144, -144, -144, 53, 53, -140, -140, - -140, -148, 53, -148, -148, -149, 53, -149, -120, 52, - -51, -173, 248, -174, 56, -128, 23, -128, -110, 118, - 115, 116, -170, 114, 200, 178, 65, 29, 15, 238, - 143, 251, 56, 144, -51, -51, -128, -51, -107, 88, - 12, -122, -122, 37, -37, -37, -123, -84, -87, -101, - 19, 11, 33, 33, -34, 67, 68, 69, 110, -179, - -68, -61, -61, -61, -33, 138, 71, -180, -180, -34, - 54, -37, -180, -180, -180, 54, 52, 22, 54, 11, - 110, 54, 11, -180, -34, -79, -77, 78, -37, -180, - -180, -180, -180, -180, -59, 30, 33, -2, -179, -179, - -55, 54, 12, 80, -44, -43, 51, 52, -45, 51, - -43, 41, 41, 121, 121, 121, -91, -116, -55, -39, - -55, -99, -100, 224, 221, 227, 56, 54, -162, 80, - 53, 28, -156, -156, 56, 56, -141, 29, 67, -147, - 204, 59, -144, -144, -145, 30, -145, -145, -145, -153, - 58, -153, 59, 59, 51, -116, -128, -172, -171, -117, - -127, -176, 149, 128, 129, 132, 131, 56, 121, 28, - 127, 130, 143, 126, -176, 149, -111, -112, 123, 22, - 121, 28, 143, -128, -105, 58, -37, 38, 110, -51, - -38, 11, 97, -117, -35, -33, 71, -61, -61, -180, - -36, -132, 106, 175, 137, 173, 169, 189, 180, 202, - 171, 203, -129, -132, -61, -61, -117, -61, -61, 245, - -82, 79, -37, 77, -92, 51, -93, -70, -72, -71, - -179, -2, -88, -116, -91, -82, -97, -37, -37, -37, - 53, -37, -179, -179, -179, -180, 54, -82, -55, 221, - 225, 226, -161, -162, -165, -164, -116, 56, 56, -143, - 51, 58, 59, 60, 67, 228, 66, 55, -145, -145, - 56, 106, 55, 54, 55, 54, 55, 54, -51, 54, - 80, -127, -116, -127, -116, -51, -127, -116, -107, -55, - -39, -180, -61, -180, -140, -140, -140, -149, -140, 163, - -140, 163, -180, -180, -180, 54, 19, -180, 54, 19, - -179, -32, 243, -37, 27, -92, 54, -180, -180, -180, - 54, 110, -180, -86, -89, -116, -89, -89, -89, -125, - -116, -86, 55, 54, -140, -151, 200, 9, -144, 58, - -144, 59, 59, -128, -171, -162, 53, 26, -80, 13, - -144, 56, -61, -61, -61, -61, -61, -180, 58, 28, - -72, 33, -2, -179, -116, -116, 54, 55, -180, -180, - -180, -54, -167, -166, 52, 133, 65, -164, -152, 127, - 28, 126, 228, -145, -145, 55, 55, -89, -179, -81, - 14, 16, -180, -180, -180, -180, -31, 90, 248, 9, - -70, -2, 110, -116, -166, 56, -157, 80, 58, -142, - 65, 28, 28, 55, -168, -169, 143, -37, -69, -180, - 246, 48, 249, -93, -180, -116, 59, 58, -175, -180, - 54, -116, 38, 247, 250, -173, -169, 33, 38, 145, - 248, 146, 249, -179, 250, -61, 142, -180, -180, + -61, -129, 56, 58, 56, -60, -60, -116, -35, 21, + -34, -36, -180, 54, -180, -2, -34, -34, -37, -37, + -75, -116, -122, -75, -34, -28, -76, -77, 76, -75, + -180, -34, -35, -34, -34, -90, 143, -51, -93, -97, + -75, -40, -41, -41, -40, -41, 41, 41, 41, 46, + 41, 46, 41, -48, -122, -180, -54, 49, 124, 50, + -179, -124, -90, 52, -39, -51, -98, -95, 54, 220, + 222, 223, 51, -37, -145, 105, -160, -161, -162, -117, + 58, 59, -154, -155, -163, 127, 130, 126, -156, 121, + 28, -150, 67, 72, -146, 203, -140, 53, -140, -140, + -140, -140, -144, 178, -144, -144, -144, 53, 53, -140, + -140, -140, -148, 53, -148, -148, -149, 53, -149, -120, + 52, -51, -173, 248, -174, 56, -128, 23, -128, -110, + 118, 115, 116, -170, 114, 200, 178, 65, 29, 15, + 238, 143, 251, 56, 144, -51, -51, -128, -51, -107, + 88, 12, -122, -122, 37, -37, -37, -123, -84, -87, + -101, 19, 11, 33, 33, -34, 67, 68, 69, 110, + -179, -68, -61, -61, -61, -33, 138, 71, -180, -180, + -34, 54, -37, -180, -180, -180, 54, 52, 22, 54, + 11, 110, 54, 11, -180, -34, -79, -77, 78, -37, + -180, -180, -180, -180, -180, -59, 30, 33, -2, -179, + -179, -55, 54, 12, 80, -44, -43, 51, 52, -45, + 51, -43, 41, 41, 121, 121, 121, -91, -116, -55, + -39, -55, -99, -100, 224, 221, 227, 56, 54, -162, + 80, 53, 28, -156, -156, 56, 56, -141, 29, 67, + -147, 204, 59, -144, -144, -145, 30, -145, -145, -145, + -153, 58, -153, 59, 59, 51, -116, -128, -172, -171, + -117, -127, -176, 149, 128, 129, 132, 131, 56, 121, + 28, 127, 130, 143, 126, -176, 149, -111, -112, 123, + 22, 121, 28, 143, -128, -105, 58, -37, 38, 110, + -51, -38, 11, 97, -117, -35, -33, 71, -61, -61, + -180, -36, -132, 106, 175, 137, 173, 169, 189, 180, + 202, 171, 203, -129, -132, -61, -61, -117, -61, -61, + 245, -82, 79, -37, 77, -92, 51, -93, -70, -72, + -71, -179, -2, -88, -116, -91, -82, -97, -37, -37, + -37, 53, -37, -179, -179, -179, -180, 54, -82, -55, + 221, 225, 226, -161, -162, -165, -164, -116, 56, 56, + -143, 51, 58, 59, 60, 67, 228, 66, 55, -145, + -145, 56, 106, 55, 54, 55, 54, 55, 54, -51, + 54, 80, -127, -116, -127, -116, -51, -127, -116, -107, + -55, -39, -180, -61, -180, -140, -140, -140, -149, -140, + 163, -140, 163, -180, -180, -180, 54, 19, -180, 54, + 19, -179, -32, 243, -37, 27, -92, 54, -180, -180, + -180, 54, 110, -180, -86, -89, -116, -89, -89, -89, + -125, -116, -86, 55, 54, -140, -151, 200, 9, -144, + 58, -144, 59, 59, -128, -171, -162, 53, 26, -80, + 13, -144, 56, -61, -61, -61, -61, -61, -180, 58, + 28, -72, 33, -2, -179, -116, -116, 54, 55, -180, + -180, -180, -54, -167, -166, 52, 133, 65, -164, -152, + 127, 28, 126, 228, -145, -145, 55, 55, -89, -179, + -81, 14, 16, -180, -180, -180, -180, -31, 90, 248, + 9, -70, -2, 110, -116, -166, 56, -157, 80, 58, + -142, 65, 28, 28, 55, -168, -169, 143, -37, -69, + -180, 246, 48, 249, -93, -180, -116, 59, 58, -175, + -180, 54, -116, 38, 247, 250, -173, -169, 33, 38, + 145, 248, 146, 249, -179, 250, -61, 142, -180, -180, } var yyDef = [...]int{ @@ -2224,9 +2197,9 @@ var yyDef = [...]int{ 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 509, 0, 277, 277, 277, 277, 277, 277, 0, 579, 562, 0, 0, 0, 0, -2, 267, 268, 0, - 270, 271, 785, 785, 785, 785, 785, 0, 33, 34, - 783, 1, 3, 517, 0, 0, 281, 284, 279, 0, - 562, 0, 0, 0, 60, 0, 0, 772, 0, 773, + 270, 271, 786, 786, 786, 786, 786, 0, 33, 34, + 784, 1, 3, 517, 0, 0, 281, 284, 279, 0, + 562, 0, 0, 0, 60, 0, 0, 773, 0, 774, 560, 560, 560, 580, 581, 584, 585, 684, 685, 686, 687, 688, 689, 690, 691, 692, 693, 694, 695, 696, 697, 698, 699, 700, 701, 702, 703, 704, 705, 706, @@ -2236,120 +2209,120 @@ var yyDef = [...]int{ 737, 738, 739, 740, 741, 742, 743, 744, 745, 746, 747, 748, 749, 750, 751, 752, 753, 754, 755, 756, 757, 758, 759, 760, 761, 762, 763, 764, 765, 766, - 767, 768, 769, 770, 771, 774, 775, 776, 777, 778, - 779, 780, 781, 782, 0, 0, 563, 0, 558, 0, - 558, 558, 558, 0, 226, 348, 588, 589, 772, 773, - 0, 0, 0, 0, 786, 786, 786, 786, 0, 786, - 0, 245, 246, 248, 249, 250, 251, 786, 264, 265, - 256, 266, 269, 272, 273, 274, 275, 276, 27, 521, - 0, 0, 509, 29, 0, 277, 282, 283, 287, 285, - 286, 278, 0, 295, 299, 0, 356, 0, 361, 363, - -2, -2, 0, 398, 399, 400, 401, 402, 0, 0, - 0, 0, 0, 0, 0, 0, 426, 427, 428, 429, - 494, 495, 496, 497, 498, 499, 500, 501, 365, 366, - 491, 541, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 482, 0, 456, 456, 456, 456, 456, 456, 456, - 456, 0, 0, 0, 0, 0, 0, 306, 308, 309, - 310, 329, 0, 331, 0, 0, 41, 45, 0, 763, - 545, -2, -2, 0, 0, 586, 587, -2, 691, -2, - 592, 593, 594, 595, 596, 597, 598, 599, 600, 601, - 602, 603, 604, 605, 606, 607, 608, 609, 610, 611, - 612, 613, 614, 615, 616, 617, 618, 619, 620, 621, - 622, 623, 624, 625, 626, 627, 628, 629, 630, 631, - 632, 633, 634, 635, 636, 637, 638, 639, 640, 641, - 642, 643, 644, 645, 646, 647, 648, 649, 650, 651, - 652, 653, 654, 655, 656, 657, 658, 659, 660, 661, - 662, 663, 664, 665, 666, 667, 668, 669, 670, 671, - 672, 673, 674, 675, 676, 677, 678, 679, 680, 681, - 682, 683, 0, 77, 0, 0, 786, 0, 67, 0, - 0, 0, 0, 0, 786, 0, 0, 0, 0, 0, - 0, 0, 225, 0, 227, 786, 786, 786, 786, 786, - 786, 786, 786, 236, 787, 788, 237, 238, 239, 786, - 786, 241, 0, 257, 253, 254, 0, 252, 28, 784, - 22, 0, 0, 518, 0, 510, 511, 514, 517, 27, - 284, 0, 289, 288, 280, 0, 296, 0, 0, 0, - 300, 0, 302, 303, 0, 359, 0, 0, 0, 0, + 767, 768, 769, 770, 771, 772, 775, 776, 777, 778, + 779, 780, 781, 782, 783, 0, 0, 563, 0, 558, + 0, 558, 558, 558, 0, 226, 348, 588, 589, 773, + 774, 0, 0, 0, 0, 787, 787, 787, 787, 0, + 787, 0, 245, 246, 248, 249, 250, 251, 787, 264, + 265, 256, 266, 269, 272, 273, 274, 275, 276, 27, + 521, 0, 0, 509, 29, 0, 277, 282, 283, 287, + 285, 286, 278, 0, 295, 299, 0, 356, 0, 361, + 363, -2, -2, 0, 398, 399, 400, 401, 402, 0, + 0, 0, 0, 0, 0, 0, 0, 426, 427, 428, + 429, 494, 495, 496, 497, 498, 499, 500, 501, 365, + 366, 491, 541, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 482, 0, 456, 456, 456, 456, 456, 456, + 456, 456, 0, 0, 0, 0, 0, 0, 306, 308, + 309, 310, 329, 0, 331, 0, 0, 41, 45, 0, + 764, 545, -2, -2, 0, 0, 586, 587, -2, 691, + -2, 592, 593, 594, 595, 596, 597, 598, 599, 600, + 601, 602, 603, 604, 605, 606, 607, 608, 609, 610, + 611, 612, 613, 614, 615, 616, 617, 618, 619, 620, + 621, 622, 623, 624, 625, 626, 627, 628, 629, 630, + 631, 632, 633, 634, 635, 636, 637, 638, 639, 640, + 641, 642, 643, 644, 645, 646, 647, 648, 649, 650, + 651, 652, 653, 654, 655, 656, 657, 658, 659, 660, + 661, 662, 663, 664, 665, 666, 667, 668, 669, 670, + 671, 672, 673, 674, 675, 676, 677, 678, 679, 680, + 681, 682, 683, 0, 77, 0, 0, 787, 0, 67, + 0, 0, 0, 0, 0, 787, 0, 0, 0, 0, + 0, 0, 0, 225, 0, 227, 787, 787, 787, 787, + 787, 787, 787, 787, 236, 788, 789, 237, 238, 239, + 787, 787, 241, 0, 257, 253, 254, 0, 252, 28, + 785, 22, 0, 0, 518, 0, 510, 511, 514, 517, + 27, 284, 0, 289, 288, 280, 0, 296, 0, 0, + 0, 300, 0, 302, 303, 0, 359, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 383, 384, 385, 386, 387, - 388, 389, 362, 0, 376, 0, 0, 0, 418, 419, - 420, 421, 422, 423, 424, 0, 291, 27, 0, 396, - 0, 0, 0, 0, 0, 0, 0, 0, 287, 0, - 483, 0, 448, 0, 449, 450, 451, 452, 453, 454, - 455, 0, 291, 0, 0, 43, 0, 347, 0, 0, - 0, 0, 0, 0, 336, 0, 0, 339, 0, 0, - 0, 0, 330, 0, 0, 350, 736, 332, 0, 334, - 335, -2, 0, 0, 0, 39, 40, 0, 46, 763, - 48, 49, 0, 0, 0, 157, 553, 554, 555, 551, - 185, 0, 140, 136, 82, 83, 84, 129, 86, 129, - 129, 129, 129, 154, 154, 154, 154, 112, 113, 114, - 115, 116, 0, 0, 99, 129, 129, 129, 103, 119, - 120, 121, 122, 123, 124, 125, 126, 87, 88, 89, - 90, 91, 92, 93, 131, 131, 131, 133, 133, 582, - 62, 0, 70, 0, 786, 0, 786, 75, 0, 201, - 0, 220, 559, 0, 786, 223, 224, 349, 590, 591, - 228, 229, 230, 231, 232, 233, 234, 235, 240, 244, - 0, 260, 0, 0, 247, 522, 0, 0, 0, 0, - 0, 513, 515, 516, 521, 30, 287, 0, 502, 0, - 0, 0, 290, 25, 357, 358, 360, 377, 0, 379, - 381, 301, 297, 0, 492, -2, 367, 368, 392, 393, - 394, 0, 0, 0, 0, 390, 372, 0, 403, 404, - 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, - 417, 467, 468, 0, 415, 416, 425, 0, 0, 292, - 293, 395, 0, 540, 27, 0, 0, 0, 0, 0, - 491, 0, 0, 0, 0, 489, 486, 0, 0, 457, - 0, 0, 0, 0, 0, 0, 346, 354, 542, 0, - 307, 325, 327, 0, 322, 337, 338, 340, 0, 342, - 0, 344, 345, 311, 312, 313, 0, 0, 0, 0, - 333, 354, 0, 354, 42, 546, 47, 0, 0, 52, - 53, 547, 548, 549, 0, 76, 186, 188, 191, 192, - 193, 78, 79, 0, 0, 0, 0, 0, 180, 181, - 143, 141, 0, 138, 137, 85, 0, 154, 154, 106, - 107, 157, 0, 157, 157, 157, 0, 0, 100, 101, - 102, 94, 0, 95, 96, 97, 0, 98, 0, 0, - 786, 64, 0, 68, 69, 65, 561, 66, 785, 0, - 0, 574, 202, 564, 565, 566, 567, 568, 569, 570, - 571, 572, 573, 0, 219, 786, 222, 257, 243, 0, - 0, 258, 259, 0, 519, 520, 0, 512, 23, 0, - 556, 557, 503, 504, 304, 378, 380, 382, 0, 291, - 369, 390, 373, 0, 370, 0, 0, 364, 430, 0, - 0, 397, -2, 433, 434, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 509, 0, 487, 0, 0, 447, - 458, 459, 460, 461, 534, 0, 0, -2, 0, 0, - 509, 0, 0, 0, 319, 326, 0, 0, 320, 0, - 321, 341, 343, 0, 0, 0, 0, 317, 509, 354, - 38, 50, 51, 0, 0, 57, 158, 0, 189, 0, - 0, 175, 0, 0, 178, 179, 150, 0, 142, 81, - 139, 0, 157, 157, 108, 0, 109, 110, 111, 0, - 127, 0, 0, 0, 0, 583, 63, 71, 72, 0, - 194, 785, 0, 203, 204, 205, 206, 207, 208, 209, - 210, 211, 212, 213, 785, 0, 0, 785, 575, 576, - 577, 578, 0, 221, 260, 261, 262, 523, 0, 24, - 354, 0, 298, 493, 0, 371, 0, 391, 374, 431, - 294, 0, 129, 129, 472, 129, 133, 475, 129, 477, - 129, 480, 0, 0, 0, 0, 492, 0, 0, 0, - 484, 446, 490, 0, 31, 0, 534, 524, 536, 538, - 0, 27, 0, 530, 0, 517, 543, 355, 544, 323, - 0, 328, 0, 0, 0, 331, 0, 517, 37, 54, - 55, 56, 187, 190, 0, 182, 129, 176, 177, 152, - 0, 144, 145, 146, 147, 148, 149, 130, 104, 105, - 155, 156, 154, 0, 154, 0, 134, 0, 786, 0, - 0, 195, 0, 196, 198, 199, 200, 0, 242, 505, - 305, 432, 375, 435, 469, 154, 473, 474, 476, 478, - 479, 481, 437, 436, 438, 0, 0, 441, 0, 0, - 0, 0, 0, 488, 0, 32, 0, 539, -2, 0, - 0, 0, 44, 35, 0, 315, 0, 0, 0, 350, - 318, 36, 167, 0, 184, 159, 153, 0, 157, 128, - 157, 0, 0, 61, 73, 74, 0, 0, 507, 0, - 470, 471, 0, 0, 0, 0, 462, 445, 485, 0, - 537, 0, -2, 0, 532, 531, 0, 324, 351, 352, - 353, 314, 166, 168, 0, 173, 0, 183, 164, 0, - 161, 163, 151, 117, 118, 132, 135, 0, 0, 26, - 0, 0, 439, 440, 442, 443, 0, 0, 0, 0, - 527, 27, 0, 316, 169, 170, 0, 174, 172, 80, - 0, 160, 162, 67, 0, 215, 0, 508, 506, 444, - 0, 0, 0, 535, -2, 533, 171, 165, 70, 214, - 0, 0, 463, 0, 466, 197, 216, 0, 464, 0, - 0, 0, 0, 0, 465, 0, 0, 217, 218, + 0, 0, 0, 0, 0, 0, 383, 384, 385, 386, + 387, 388, 389, 362, 0, 376, 0, 0, 0, 418, + 419, 420, 421, 422, 423, 424, 0, 291, 27, 0, + 396, 0, 0, 0, 0, 0, 0, 0, 0, 287, + 0, 483, 0, 448, 0, 449, 450, 451, 452, 453, + 454, 455, 0, 291, 0, 0, 43, 0, 347, 0, + 0, 0, 0, 0, 0, 336, 0, 0, 339, 0, + 0, 0, 0, 330, 0, 0, 350, 737, 332, 0, + 334, 335, -2, 0, 0, 0, 39, 40, 0, 46, + 764, 48, 49, 0, 0, 0, 157, 553, 554, 555, + 551, 185, 0, 140, 136, 82, 83, 84, 129, 86, + 129, 129, 129, 129, 154, 154, 154, 154, 112, 113, + 114, 115, 116, 0, 0, 99, 129, 129, 129, 103, + 119, 120, 121, 122, 123, 124, 125, 126, 87, 88, + 89, 90, 91, 92, 93, 131, 131, 131, 133, 133, + 582, 62, 0, 70, 0, 787, 0, 787, 75, 0, + 201, 0, 220, 559, 0, 787, 223, 224, 349, 590, + 591, 228, 229, 230, 231, 232, 233, 234, 235, 240, + 244, 0, 260, 0, 0, 247, 522, 0, 0, 0, + 0, 0, 513, 515, 516, 521, 30, 287, 0, 502, + 0, 0, 0, 290, 25, 357, 358, 360, 377, 0, + 379, 381, 301, 297, 0, 492, -2, 367, 368, 392, + 393, 394, 0, 0, 0, 0, 390, 372, 0, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 417, 467, 468, 0, 415, 416, 425, 0, 0, + 292, 293, 395, 0, 540, 27, 0, 0, 0, 0, + 0, 491, 0, 0, 0, 0, 489, 486, 0, 0, + 457, 0, 0, 0, 0, 0, 0, 346, 354, 542, + 0, 307, 325, 327, 0, 322, 337, 338, 340, 0, + 342, 0, 344, 345, 311, 312, 313, 0, 0, 0, + 0, 333, 354, 0, 354, 42, 546, 47, 0, 0, + 52, 53, 547, 548, 549, 0, 76, 186, 188, 191, + 192, 193, 78, 79, 0, 0, 0, 0, 0, 180, + 181, 143, 141, 0, 138, 137, 85, 0, 154, 154, + 106, 107, 157, 0, 157, 157, 157, 0, 0, 100, + 101, 102, 94, 0, 95, 96, 97, 0, 98, 0, + 0, 787, 64, 0, 68, 69, 65, 561, 66, 786, + 0, 0, 574, 202, 564, 565, 566, 567, 568, 569, + 570, 571, 572, 573, 0, 219, 787, 222, 257, 243, + 0, 0, 258, 259, 0, 519, 520, 0, 512, 23, + 0, 556, 557, 503, 504, 304, 378, 380, 382, 0, + 291, 369, 390, 373, 0, 370, 0, 0, 364, 430, + 0, 0, 397, -2, 433, 434, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 509, 0, 487, 0, 0, + 447, 458, 459, 460, 461, 534, 0, 0, -2, 0, + 0, 509, 0, 0, 0, 319, 326, 0, 0, 320, + 0, 321, 341, 343, 0, 0, 0, 0, 317, 509, + 354, 38, 50, 51, 0, 0, 57, 158, 0, 189, + 0, 0, 175, 0, 0, 178, 179, 150, 0, 142, + 81, 139, 0, 157, 157, 108, 0, 109, 110, 111, + 0, 127, 0, 0, 0, 0, 583, 63, 71, 72, + 0, 194, 786, 0, 203, 204, 205, 206, 207, 208, + 209, 210, 211, 212, 213, 786, 0, 0, 786, 575, + 576, 577, 578, 0, 221, 260, 261, 262, 523, 0, + 24, 354, 0, 298, 493, 0, 371, 0, 391, 374, + 431, 294, 0, 129, 129, 472, 129, 133, 475, 129, + 477, 129, 480, 0, 0, 0, 0, 492, 0, 0, + 0, 484, 446, 490, 0, 31, 0, 534, 524, 536, + 538, 0, 27, 0, 530, 0, 517, 543, 355, 544, + 323, 0, 328, 0, 0, 0, 331, 0, 517, 37, + 54, 55, 56, 187, 190, 0, 182, 129, 176, 177, + 152, 0, 144, 145, 146, 147, 148, 149, 130, 104, + 105, 155, 156, 154, 0, 154, 0, 134, 0, 787, + 0, 0, 195, 0, 196, 198, 199, 200, 0, 242, + 505, 305, 432, 375, 435, 469, 154, 473, 474, 476, + 478, 479, 481, 437, 436, 438, 0, 0, 441, 0, + 0, 0, 0, 0, 488, 0, 32, 0, 539, -2, + 0, 0, 0, 44, 35, 0, 315, 0, 0, 0, + 350, 318, 36, 167, 0, 184, 159, 153, 0, 157, + 128, 157, 0, 0, 61, 73, 74, 0, 0, 507, + 0, 470, 471, 0, 0, 0, 0, 462, 445, 485, + 0, 537, 0, -2, 0, 532, 531, 0, 324, 351, + 352, 353, 314, 166, 168, 0, 173, 0, 183, 164, + 0, 161, 163, 151, 117, 118, 132, 135, 0, 0, + 26, 0, 0, 439, 440, 442, 443, 0, 0, 0, + 0, 527, 27, 0, 316, 169, 170, 0, 174, 172, + 80, 0, 160, 162, 67, 0, 215, 0, 508, 506, + 444, 0, 0, 0, 535, -2, 533, 171, 165, 70, + 214, 0, 0, 463, 0, 466, 197, 216, 0, 464, + 0, 0, 0, 0, 0, 465, 0, 0, 217, 218, } var yyTok1 = [...]int{ @@ -6122,42 +6095,42 @@ yydefault: { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 783: + case 784: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3134 + //line sql.y:3135 { if incNesting(yylex) { yylex.Error("max nesting level reached") return 1 } } - case 784: + case 785: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3143 + //line sql.y:3144 { decNesting(yylex) } - case 785: + case 786: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:3148 + //line sql.y:3149 { forceEOF(yylex) } - case 786: + case 787: yyDollar = yyS[yypt-0 : yypt+1] - //line sql.y:3153 + //line sql.y:3154 { forceEOF(yylex) } - case 787: + case 788: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3157 + //line sql.y:3158 { forceEOF(yylex) } - case 788: + case 789: yyDollar = yyS[yypt-1 : yypt+1] - //line sql.y:3161 + //line sql.y:3162 { forceEOF(yylex) } diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index 6dd3a361548..3fc1708233b 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -3039,6 +3039,7 @@ non_reserved_keyword: | CHARACTER | CHARSET | COLLATION +| COLUMNS | COMMENT_KEYWORD | COMMIT | COMMITTED From 3c6edab5f079eed7bca9be259fdaa8345bef4602 Mon Sep 17 00:00:00 2001 From: Maggie Zhou Date: Wed, 11 Jul 2018 09:24:34 -0700 Subject: [PATCH 30/72] Fix bug found in dev: Orchestrator returns an empty length active-recovery response when it has no history of recovery on that cluster. --- go/vt/vttablet/tabletmanager/orchestrator.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletmanager/orchestrator.go b/go/vt/vttablet/tabletmanager/orchestrator.go index c1266d18ddd..f80e5d314d4 100644 --- a/go/vt/vttablet/tabletmanager/orchestrator.go +++ b/go/vt/vttablet/tabletmanager/orchestrator.go @@ -146,8 +146,9 @@ func (orc *orcClient) InActiveShardRecovery(tablet *topodatapb.Tablet) (bool, er return false, err } + // Orchestrator returns a 0-length response when it has no history of recovery on this cluster. if len(r) == 0 { - return false, fmt.Errorf("Orchestrator returned an empty audit-recovery response") + return false, nil } active, ok := r[0]["IsActive"].(bool) From de3d57dc2a8b6efa65f2f0c052216a80df1bd42d Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Mon, 6 Aug 2018 09:42:12 -0700 Subject: [PATCH 31/72] restore the deprecated user-specific "dbname" flags Fix a regression where the dbname flags were removed as part of the earlier cleanup and refactor. This is against policy since we want to keep the flags for now. Unlike the other user-specific db connection flags, this one does not actually do anything at runtime except set a "DeprecatedDBName" field in the DBConfig. Signed-off-by: Michael Demmer --- go/mysql/conn_params.go | 4 ++++ go/vt/dbconfigs/dbconfigs.go | 3 +++ 2 files changed, 7 insertions(+) diff --git a/go/mysql/conn_params.go b/go/mysql/conn_params.go index 053e662b165..ae739c9b59c 100644 --- a/go/mysql/conn_params.go +++ b/go/mysql/conn_params.go @@ -34,6 +34,10 @@ type ConnParams struct { SslCert string `json:"ssl_cert"` SslKey string `json:"ssl_key"` ServerName string `json:"server_name"` + + // The following is only set when the deprecated "dbname" flags are + // supplied and will be removed. + DeprecatedDBName string } // EnableSSL will set the right flag on the parameters. diff --git a/go/vt/dbconfigs/dbconfigs.go b/go/vt/dbconfigs/dbconfigs.go index 802a5376c7f..38d73d08cf0 100644 --- a/go/vt/dbconfigs/dbconfigs.go +++ b/go/vt/dbconfigs/dbconfigs.go @@ -124,6 +124,9 @@ func registerPerUserFlags(dbc *userConfig, userKey string) { flag.StringVar(&dbc.param.SslCaPath, "db-config-"+userKey+"-ssl-ca-path", "", "deprecated: use db_ssl_ca_path") flag.StringVar(&dbc.param.SslCert, "db-config-"+userKey+"-ssl-cert", "", "deprecated: use db_ssl_cert") flag.StringVar(&dbc.param.SslKey, "db-config-"+userKey+"-ssl-key", "", "deprecated: use db_ssl_key") + + flag.StringVar(&dbc.param.DeprecatedDbName, "db-config-"+userKey+"-dbname", "", "deprecated: dbname does not need to be explicitly configured") + } // AppWithDB returns connection parameters for app with dbname set. From 1852654b940342cc707467ff5e98fd2341f81902 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Mon, 6 Aug 2018 10:15:23 -0700 Subject: [PATCH 32/72] fix typo and use DeprecatedDBName properly Signed-off-by: Michael Demmer --- go/vt/dbconfigs/dbconfigs.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/dbconfigs/dbconfigs.go b/go/vt/dbconfigs/dbconfigs.go index 38d73d08cf0..5df9aa2d7da 100644 --- a/go/vt/dbconfigs/dbconfigs.go +++ b/go/vt/dbconfigs/dbconfigs.go @@ -125,7 +125,7 @@ func registerPerUserFlags(dbc *userConfig, userKey string) { flag.StringVar(&dbc.param.SslCert, "db-config-"+userKey+"-ssl-cert", "", "deprecated: use db_ssl_cert") flag.StringVar(&dbc.param.SslKey, "db-config-"+userKey+"-ssl-key", "", "deprecated: use db_ssl_key") - flag.StringVar(&dbc.param.DeprecatedDbName, "db-config-"+userKey+"-dbname", "", "deprecated: dbname does not need to be explicitly configured") + flag.StringVar(&dbc.param.DeprecatedDBName, "db-config-"+userKey+"-dbname", "", "deprecated: dbname does not need to be explicitly configured") } From bb9395b3b589aa09774d322c5cf208234351d351 Mon Sep 17 00:00:00 2001 From: Maggie Zhou Date: Fri, 10 Aug 2018 13:47:48 -0700 Subject: [PATCH 33/72] Set the keepalive server-side enforcement policy min time, so that the server doesn't preemptively close connections due to our client keepalive being more aggressive than the allowed server policy. Signed-off-by: Maggie Zhou --- go/vt/servenv/grpc_server.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/go/vt/servenv/grpc_server.go b/go/vt/servenv/grpc_server.go index 8bfbcae0fff..25dce3617b6 100644 --- a/go/vt/servenv/grpc_server.go +++ b/go/vt/servenv/grpc_server.go @@ -83,6 +83,10 @@ var ( // The lower bound for window size is 64K and any value smaller than that will be ignored. GRPCInitialWindowSize = flag.Int("grpc_server_initial_window_size", 0, "grpc server initial window size") + // EnforcementPolicy MinTime that sets the keepalive enforcement policy on the server. + // This is the minimum amount of time a client should wait before sending a keepalive ping. + GRPCEnforcementPolicyMinTime = flag.Duration("grpc_enforcement_min_time", 5*time.Minute, "grpc server minimum keepalive time") + authPlugin Authenticator ) @@ -143,6 +147,11 @@ func createGRPCServer() { opts = append(opts, grpc.InitialWindowSize(int32(*GRPCInitialWindowSize))) } + ep := keepalive.EnforcementPolicy{ + MinTime: *GRPCEnforcementPolicyMinTime, + } + opts = append(opts, grpc.KeepaliveEnforcementPolicy(ep)) + if GRPCMaxConnectionAge != nil { ka := keepalive.ServerParameters{ MaxConnectionAge: *GRPCMaxConnectionAge, From eb8922775ac21badc4c1ca848775d0748eca135e Mon Sep 17 00:00:00 2001 From: Maggie Zhou Date: Fri, 10 Aug 2018 14:29:35 -0700 Subject: [PATCH 34/72] Change the flag name to respond to code review. Signed-off-by: Maggie Zhou --- go/vt/servenv/grpc_server.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/servenv/grpc_server.go b/go/vt/servenv/grpc_server.go index 25dce3617b6..156e54bc583 100644 --- a/go/vt/servenv/grpc_server.go +++ b/go/vt/servenv/grpc_server.go @@ -85,7 +85,7 @@ var ( // EnforcementPolicy MinTime that sets the keepalive enforcement policy on the server. // This is the minimum amount of time a client should wait before sending a keepalive ping. - GRPCEnforcementPolicyMinTime = flag.Duration("grpc_enforcement_min_time", 5*time.Minute, "grpc server minimum keepalive time") + GRPCKeepAliveEnforcementPolicyMinTime = flag.Duration("grpc_server_keepalive_enforcement_policy_min_time", 5*time.Minute, "grpc server minimum keepalive time") authPlugin Authenticator ) From 9b04fe02787f1c8ddcdb4f41c45872ca7f946aea Mon Sep 17 00:00:00 2001 From: Maggie Zhou Date: Fri, 10 Aug 2018 14:51:27 -0700 Subject: [PATCH 35/72] Use the new variable name. Signed-off-by: Maggie Zhou --- go/vt/servenv/grpc_server.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/servenv/grpc_server.go b/go/vt/servenv/grpc_server.go index 156e54bc583..e596d1702f8 100644 --- a/go/vt/servenv/grpc_server.go +++ b/go/vt/servenv/grpc_server.go @@ -148,7 +148,7 @@ func createGRPCServer() { } ep := keepalive.EnforcementPolicy{ - MinTime: *GRPCEnforcementPolicyMinTime, + MinTime: *GRPCKeepAliveEnforcementPolicyMinTime, } opts = append(opts, grpc.KeepaliveEnforcementPolicy(ep)) From 75f839c7e6965640a15cd6ed7ccdc1ac5b5d97a5 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Thu, 23 Aug 2018 07:56:40 -0700 Subject: [PATCH 36/72] add a callinfo implementation for the mysql server protocol Building off the analogous structure for GRPC, include a callinfo implementation so that query logs and debug output in vtgate includes context about the caller. Signed-off-by: Michael Demmer --- go/vt/callinfo/plugin_mysql.go | 57 +++++++++++++++++++++++++++++ go/vt/vtgate/plugin_mysql_server.go | 3 ++ 2 files changed, 60 insertions(+) create mode 100644 go/vt/callinfo/plugin_mysql.go diff --git a/go/vt/callinfo/plugin_mysql.go b/go/vt/callinfo/plugin_mysql.go new file mode 100644 index 00000000000..d82f1ce1704 --- /dev/null +++ b/go/vt/callinfo/plugin_mysql.go @@ -0,0 +1,57 @@ +/* +Copyright 2018 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 agreedto 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 callinfo + +// This file implements the CallInfo interface for Mysql contexts. + +import ( + "fmt" + "html/template" + + "golang.org/x/net/context" + "vitess.io/vitess/go/mysql" +) + +// MysqlCallInfo returns an augmented context with a CallInfo structure, +// only for Mysql contexts. +func MysqlCallInfo(ctx context.Context, c *mysql.Conn) context.Context { + return NewContext(ctx, &mysqlCallInfoImpl{ + remoteAddr: c.RemoteAddr().String(), + user: c.User, + }) +} + +type mysqlCallInfoImpl struct { + remoteAddr string + user string +} + +func (mci *mysqlCallInfoImpl) RemoteAddr() string { + return mci.remoteAddr +} + +func (mci *mysqlCallInfoImpl) Username() string { + return mci.user +} + +func (mci *mysqlCallInfoImpl) Text() string { + return fmt.Sprintf("%s@%s(Mysql)", mci.user, mci.remoteAddr) +} + +func (mci *mysqlCallInfoImpl) HTML() template.HTML { + return template.HTML("MySQL User: " + mci.user + " Remote Addr: " + mci.remoteAddr) +} diff --git a/go/vt/vtgate/plugin_mysql_server.go b/go/vt/vtgate/plugin_mysql_server.go index 194f04a428b..37628f9cb3c 100644 --- a/go/vt/vtgate/plugin_mysql_server.go +++ b/go/vt/vtgate/plugin_mysql_server.go @@ -30,6 +30,7 @@ import ( "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/callerid" + "vitess.io/vitess/go/vt/callinfo" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/vttls" @@ -105,6 +106,8 @@ func (vh *vtgateHandler) ComQuery(c *mysql.Conn, query string, callback func(*sq ctx = context.Background() } + ctx = callinfo.MysqlCallInfo(ctx, c) + // Fill in the ImmediateCallerID with the UserData returned by // the AuthServer plugin for that user. If nothing was // returned, use the User. This lets the plugin map a MySQL From 01f13d733cff23d86be84f55aae033f2410473f0 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Thu, 23 Aug 2018 06:40:08 -0700 Subject: [PATCH 37/72] rework the grpc callinfo to include the peer remote address Since grpc has a convenient method for getting the peer information from the context, add the necessary hooks to extract this for the vitess Callinfo interface so it shows up in query logs and debug messages. Signed-off-by: Michael Demmer --- go/vt/callinfo/plugin_grpc.go | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/go/vt/callinfo/plugin_grpc.go b/go/vt/callinfo/plugin_grpc.go index 3a79a08e815..558609069d1 100644 --- a/go/vt/callinfo/plugin_grpc.go +++ b/go/vt/callinfo/plugin_grpc.go @@ -24,6 +24,7 @@ import ( "golang.org/x/net/context" "google.golang.org/grpc" + "google.golang.org/grpc/peer" ) // GRPCCallInfo returns an augmented context with a CallInfo structure, @@ -33,17 +34,25 @@ func GRPCCallInfo(ctx context.Context) context.Context { if !ok { return ctx } - return NewContext(ctx, &gRPCCallInfoImpl{ + + callinfo := &gRPCCallInfoImpl{ method: method, - }) + } + peer, ok := peer.FromContext(ctx) + if ok { + callinfo.remoteAddr = peer.Addr.String() + } + + return NewContext(ctx, callinfo) } type gRPCCallInfoImpl struct { - method string + method string + remoteAddr string } func (gci *gRPCCallInfoImpl) RemoteAddr() string { - return "remote" + return gci.remoteAddr } func (gci *gRPCCallInfoImpl) Username() string { @@ -51,9 +60,9 @@ func (gci *gRPCCallInfoImpl) Username() string { } func (gci *gRPCCallInfoImpl) Text() string { - return fmt.Sprintf("%s(gRPC)", gci.method) + return fmt.Sprintf("%s:%s(gRPC)", gci.remoteAddr, gci.method) } func (gci *gRPCCallInfoImpl) HTML() template.HTML { - return template.HTML("Method: " + gci.method) + return template.HTML("Method: " + gci.method + " Remote Addr: " + gci.remoteAddr) } From acde621c5ee9f451c1b87eb5f50b0b9c1eb174f7 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Tue, 28 Aug 2018 11:47:17 -0700 Subject: [PATCH 38/72] Flush binlog while doing a PlannedReparent Motivation for this change in: https://github.com/vitessio/vitess/issues/4161 Signed-off-by: Rafael Chacon --- go/vt/vttablet/faketmclient/fake_client.go | 5 +++++ go/vt/vttablet/grpctmclient/client.go | 14 ++++++++++++++ go/vt/vttablet/tmclient/rpc_client_api.go | 3 +++ go/vt/wrangler/reparent.go | 12 ++++++++++++ 4 files changed, 34 insertions(+) diff --git a/go/vt/vttablet/faketmclient/fake_client.go b/go/vt/vttablet/faketmclient/fake_client.go index 8d87c6fe2fc..db88fd2d17c 100644 --- a/go/vt/vttablet/faketmclient/fake_client.go +++ b/go/vt/vttablet/faketmclient/fake_client.go @@ -262,6 +262,11 @@ func (client *FakeTabletManagerClient) SlaveWasPromoted(ctx context.Context, tab return nil } +// FlushBinaryLogs is part of the tmclient.TabletManagerClient interface. +func (client *FakeTabletManagerClient) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { + return nil +} + // SetMaster is part of the tmclient.TabletManagerClient interface. func (client *FakeTabletManagerClient) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { return nil diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index 1a78ed58b79..03589eba5fb 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -681,6 +681,20 @@ func (client *Client) SlaveWasPromoted(ctx context.Context, tablet *topodatapb.T return err } +// FlushBinaryLogs is part of the tmclient.TabletManagerClient interface. +func (client *Client) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { + flushBinaryLogsStmnt := fmt.Sprintf("FLUSH BINARY LOGS") + _, err := client.ExecuteFetchAsDba( + ctx, + tablet, + true, /* usePool */ + []byte(flushBinaryLogsStmnt), + 0, /* maxRows */ + true, /* disableBinlogs */ + false /* reloadSchema */) + return err +} + // SetMaster is part of the tmclient.TabletManagerClient interface. func (client *Client) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { cc, c, err := client.dial(tablet) diff --git a/go/vt/vttablet/tmclient/rpc_client_api.go b/go/vt/vttablet/tmclient/rpc_client_api.go index 628d3e0987d..db992e2e9d2 100644 --- a/go/vt/vttablet/tmclient/rpc_client_api.go +++ b/go/vt/vttablet/tmclient/rpc_client_api.go @@ -177,6 +177,9 @@ type TabletManagerClient interface { // PromoteSlaveWhenCaughtUp transforms the tablet from a slave to a master. PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topodatapb.Tablet, pos string) (string, error) + // FlushBinaryLogs flushes binary log, so it rotates current binlog file + FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error + // SlaveWasPromoted tells the remote tablet it is now the master SlaveWasPromoted(ctx context.Context, tablet *topodatapb.Tablet) error diff --git a/go/vt/wrangler/reparent.go b/go/vt/wrangler/reparent.go index 7fad88e4f5e..bc1ba24ac02 100644 --- a/go/vt/wrangler/reparent.go +++ b/go/vt/wrangler/reparent.go @@ -405,6 +405,18 @@ func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.R return fmt.Errorf("old master tablet %v DemoteMaster failed: %v", topoproto.TabletAliasString(shardInfo.MasterAlias), err) } + // When using RBR and semi-sync, new master can take a long time to find GTID set to send to replicas. + // This causes slow writes when a master gets promoted. Full details in: + // https://github.com/vitessio/vitess/issues/4161 + // Flushing binlogs before promoting a master fixes this problem. + wr.logger.Infof("flushing binary logs in target master %v", masterElectTabletAliasStr) + event.DispatchUpdate(ev, "flush binary logs") + err = wr.tmc.FlushBinaryLogs(ctx, masterElectTabletInfo.Tablet) + if err != nil { + wr.logger.Warningf("Could not flush binary logs in tablet: %v. New master could be slower to take writes", masterElectTabletAlias) + + } + // Wait on the master-elect tablet until it reaches that position, // then promote it wr.logger.Infof("promote slave %v", masterElectTabletAliasStr) From 977bad0848c75b97f95f33d3e7834e61b0a1c224 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Tue, 28 Aug 2018 11:52:55 -0700 Subject: [PATCH 39/72] We want this to make it to the binlogs Signed-off-by: Rafael Chacon --- go/vt/vttablet/grpctmclient/client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index 03589eba5fb..f3a82f56a15 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -689,8 +689,8 @@ func (client *Client) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Ta tablet, true, /* usePool */ []byte(flushBinaryLogsStmnt), - 0, /* maxRows */ - true, /* disableBinlogs */ + 0, /* maxRows */ + false, /* disableBinlogs */ false /* reloadSchema */) return err } From 213d7bbf79420e449ad1a533eb3ae495e1bf5544 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Tue, 28 Aug 2018 11:55:16 -0700 Subject: [PATCH 40/72] Need to implement interface in vtcombo Signed-off-by: Rafael Chacon --- go/vt/vtcombo/tablet_map.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 93f0a3fa9e6..4c26d020c33 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -668,6 +668,10 @@ func (itmc *internalTabletManagerClient) SlaveWasPromoted(ctx context.Context, t return fmt.Errorf("not implemented in vtcombo") } +func (itmc *internalTabletManagerClient) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { + return fmt.Errorf("not implemented in vtcombo") +} + func (itmc *internalTabletManagerClient) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { return fmt.Errorf("not implemented in vtcombo") } From 105c0a40ec7ae9f6915c2f835666b658ef0da74c Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 29 Aug 2018 11:07:55 -0700 Subject: [PATCH 41/72] Try backups with buffer io Signed-off-by: Rafael Chacon --- go/vt/mysqlctl/s3backupstorage/s3.go | 6 +++++- vendor/vendor.json | 24 ++++++++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index 03bdd41b72b..65cad6f04c0 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -38,6 +38,9 @@ import ( "github.com/aws/aws-sdk-go/service/s3/s3manager" "golang.org/x/net/context" + "gopkg.in/djherbis/buffer.v1" + "gopkg.in/djherbis/nio.v2" + "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" ) @@ -97,7 +100,8 @@ func (bh *S3BackupHandle) AddFile(ctx context.Context, filename string, filesize } } - reader, writer := io.Pipe() + buf := buffer.New(500 * 1024 * 1024) // 500 MB in memory buffer + reader, writer := nio.Pipe(buf) bh.waitGroup.Add(1) go func() { diff --git a/vendor/vendor.json b/vendor/vendor.json index 72e7aaf9735..ab46dd93988 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -364,6 +364,18 @@ "revision": "5215b55f46b2b919f50a1df0eaa5886afe4e3b3d", "revisionTime": "2015-11-05T21:09:06Z" }, + { + "checksumSHA1": "IBbdYHB32oPftUQwQCypCFzWbG4=", + "path": "github.com/djherbis/buffer/limio", + "revision": "9cb938231dfe1a4c407f48ccf24c0d5fd7723b18", + "revisionTime": "2018-06-23T16:40:14Z" + }, + { + "checksumSHA1": "AB9CaOyWTRWtTsFy/BefZtFIzQI=", + "path": "github.com/djherbis/buffer/wrapio", + "revision": "9cb938231dfe1a4c407f48ccf24c0d5fd7723b18", + "revisionTime": "2018-06-23T16:40:14Z" + }, { "checksumSHA1": "muGVyM8mY3/gcap6kr4Ib3F5Xn4=", "path": "github.com/ghodss/yaml", @@ -1296,6 +1308,18 @@ "revision": "4e86f4367175e39f69d9358a5f17b4dda270378d", "revisionTime": "2015-09-24T05:17:56Z" }, + { + "checksumSHA1": "NwR8Jfz6xyRq+6z/ZDOmqssnsq8=", + "path": "gopkg.in/djherbis/buffer.v1", + "revision": "c8b7051e392cf78efc6f82f20ff71e89f20e0ea3", + "revisionTime": "2016-02-03T04:16:34Z" + }, + { + "checksumSHA1": "jGLm7Dl0BJENZ97wCn+nXwA9U+Y=", + "path": "gopkg.in/djherbis/nio.v2", + "revision": "824ca9017eeb2a422fdda7ae9dba05d3ab019dfa", + "revisionTime": "2017-01-25T04:38:21Z" + }, { "checksumSHA1": "itYnRitfdzJjy2mZlvJ+hCJZvtY=", "path": "gopkg.in/ldap.v2", From 3532aa5801506125a4dbe538ade2e7cc301e33d5 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Wed, 29 Aug 2018 11:13:48 -0700 Subject: [PATCH 42/72] Revert "Try backups with buffer io" This reverts commit 105c0a40ec7ae9f6915c2f835666b658ef0da74c. --- go/vt/mysqlctl/s3backupstorage/s3.go | 6 +----- vendor/vendor.json | 24 ------------------------ 2 files changed, 1 insertion(+), 29 deletions(-) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index 65cad6f04c0..03bdd41b72b 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -38,9 +38,6 @@ import ( "github.com/aws/aws-sdk-go/service/s3/s3manager" "golang.org/x/net/context" - "gopkg.in/djherbis/buffer.v1" - "gopkg.in/djherbis/nio.v2" - "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" ) @@ -100,8 +97,7 @@ func (bh *S3BackupHandle) AddFile(ctx context.Context, filename string, filesize } } - buf := buffer.New(500 * 1024 * 1024) // 500 MB in memory buffer - reader, writer := nio.Pipe(buf) + reader, writer := io.Pipe() bh.waitGroup.Add(1) go func() { diff --git a/vendor/vendor.json b/vendor/vendor.json index ab46dd93988..72e7aaf9735 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -364,18 +364,6 @@ "revision": "5215b55f46b2b919f50a1df0eaa5886afe4e3b3d", "revisionTime": "2015-11-05T21:09:06Z" }, - { - "checksumSHA1": "IBbdYHB32oPftUQwQCypCFzWbG4=", - "path": "github.com/djherbis/buffer/limio", - "revision": "9cb938231dfe1a4c407f48ccf24c0d5fd7723b18", - "revisionTime": "2018-06-23T16:40:14Z" - }, - { - "checksumSHA1": "AB9CaOyWTRWtTsFy/BefZtFIzQI=", - "path": "github.com/djherbis/buffer/wrapio", - "revision": "9cb938231dfe1a4c407f48ccf24c0d5fd7723b18", - "revisionTime": "2018-06-23T16:40:14Z" - }, { "checksumSHA1": "muGVyM8mY3/gcap6kr4Ib3F5Xn4=", "path": "github.com/ghodss/yaml", @@ -1308,18 +1296,6 @@ "revision": "4e86f4367175e39f69d9358a5f17b4dda270378d", "revisionTime": "2015-09-24T05:17:56Z" }, - { - "checksumSHA1": "NwR8Jfz6xyRq+6z/ZDOmqssnsq8=", - "path": "gopkg.in/djherbis/buffer.v1", - "revision": "c8b7051e392cf78efc6f82f20ff71e89f20e0ea3", - "revisionTime": "2016-02-03T04:16:34Z" - }, - { - "checksumSHA1": "jGLm7Dl0BJENZ97wCn+nXwA9U+Y=", - "path": "gopkg.in/djherbis/nio.v2", - "revision": "824ca9017eeb2a422fdda7ae9dba05d3ab019dfa", - "revisionTime": "2017-01-25T04:38:21Z" - }, { "checksumSHA1": "itYnRitfdzJjy2mZlvJ+hCJZvtY=", "path": "gopkg.in/ldap.v2", From 919d30076ffadeb5bdf13760125794a53806d1d0 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Thu, 6 Sep 2018 10:56:41 -0700 Subject: [PATCH 43/72] Revert "Need to implement interface in vtcombo" This reverts commit 213d7bbf79420e449ad1a533eb3ae495e1bf5544. --- go/vt/vtcombo/tablet_map.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 4c26d020c33..93f0a3fa9e6 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -668,10 +668,6 @@ func (itmc *internalTabletManagerClient) SlaveWasPromoted(ctx context.Context, t return fmt.Errorf("not implemented in vtcombo") } -func (itmc *internalTabletManagerClient) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { - return fmt.Errorf("not implemented in vtcombo") -} - func (itmc *internalTabletManagerClient) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { return fmt.Errorf("not implemented in vtcombo") } From cc60d793ca286010b319809ec43f426fb888ea61 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Thu, 6 Sep 2018 10:56:52 -0700 Subject: [PATCH 44/72] Revert "We want this to make it to the binlogs" This reverts commit 977bad0848c75b97f95f33d3e7834e61b0a1c224. --- go/vt/vttablet/grpctmclient/client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index f3a82f56a15..03589eba5fb 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -689,8 +689,8 @@ func (client *Client) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Ta tablet, true, /* usePool */ []byte(flushBinaryLogsStmnt), - 0, /* maxRows */ - false, /* disableBinlogs */ + 0, /* maxRows */ + true, /* disableBinlogs */ false /* reloadSchema */) return err } From 6f4aa06c35d9a68372037e5c491567835b0289e5 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Thu, 6 Sep 2018 10:56:59 -0700 Subject: [PATCH 45/72] Revert "Flush binlog while doing a PlannedReparent" This reverts commit acde621c5ee9f451c1b87eb5f50b0b9c1eb174f7. --- go/vt/vttablet/faketmclient/fake_client.go | 5 ----- go/vt/vttablet/grpctmclient/client.go | 14 -------------- go/vt/vttablet/tmclient/rpc_client_api.go | 3 --- go/vt/wrangler/reparent.go | 12 ------------ 4 files changed, 34 deletions(-) diff --git a/go/vt/vttablet/faketmclient/fake_client.go b/go/vt/vttablet/faketmclient/fake_client.go index db88fd2d17c..8d87c6fe2fc 100644 --- a/go/vt/vttablet/faketmclient/fake_client.go +++ b/go/vt/vttablet/faketmclient/fake_client.go @@ -262,11 +262,6 @@ func (client *FakeTabletManagerClient) SlaveWasPromoted(ctx context.Context, tab return nil } -// FlushBinaryLogs is part of the tmclient.TabletManagerClient interface. -func (client *FakeTabletManagerClient) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { - return nil -} - // SetMaster is part of the tmclient.TabletManagerClient interface. func (client *FakeTabletManagerClient) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { return nil diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index 03589eba5fb..1a78ed58b79 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -681,20 +681,6 @@ func (client *Client) SlaveWasPromoted(ctx context.Context, tablet *topodatapb.T return err } -// FlushBinaryLogs is part of the tmclient.TabletManagerClient interface. -func (client *Client) FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error { - flushBinaryLogsStmnt := fmt.Sprintf("FLUSH BINARY LOGS") - _, err := client.ExecuteFetchAsDba( - ctx, - tablet, - true, /* usePool */ - []byte(flushBinaryLogsStmnt), - 0, /* maxRows */ - true, /* disableBinlogs */ - false /* reloadSchema */) - return err -} - // SetMaster is part of the tmclient.TabletManagerClient interface. func (client *Client) SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error { cc, c, err := client.dial(tablet) diff --git a/go/vt/vttablet/tmclient/rpc_client_api.go b/go/vt/vttablet/tmclient/rpc_client_api.go index db992e2e9d2..628d3e0987d 100644 --- a/go/vt/vttablet/tmclient/rpc_client_api.go +++ b/go/vt/vttablet/tmclient/rpc_client_api.go @@ -177,9 +177,6 @@ type TabletManagerClient interface { // PromoteSlaveWhenCaughtUp transforms the tablet from a slave to a master. PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topodatapb.Tablet, pos string) (string, error) - // FlushBinaryLogs flushes binary log, so it rotates current binlog file - FlushBinaryLogs(ctx context.Context, tablet *topodatapb.Tablet) error - // SlaveWasPromoted tells the remote tablet it is now the master SlaveWasPromoted(ctx context.Context, tablet *topodatapb.Tablet) error diff --git a/go/vt/wrangler/reparent.go b/go/vt/wrangler/reparent.go index bc1ba24ac02..7fad88e4f5e 100644 --- a/go/vt/wrangler/reparent.go +++ b/go/vt/wrangler/reparent.go @@ -405,18 +405,6 @@ func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.R return fmt.Errorf("old master tablet %v DemoteMaster failed: %v", topoproto.TabletAliasString(shardInfo.MasterAlias), err) } - // When using RBR and semi-sync, new master can take a long time to find GTID set to send to replicas. - // This causes slow writes when a master gets promoted. Full details in: - // https://github.com/vitessio/vitess/issues/4161 - // Flushing binlogs before promoting a master fixes this problem. - wr.logger.Infof("flushing binary logs in target master %v", masterElectTabletAliasStr) - event.DispatchUpdate(ev, "flush binary logs") - err = wr.tmc.FlushBinaryLogs(ctx, masterElectTabletInfo.Tablet) - if err != nil { - wr.logger.Warningf("Could not flush binary logs in tablet: %v. New master could be slower to take writes", masterElectTabletAlias) - - } - // Wait on the master-elect tablet until it reaches that position, // then promote it wr.logger.Infof("promote slave %v", masterElectTabletAliasStr) From 5a548fdff584fd74ab1a4e175e69806e75e97e7c Mon Sep 17 00:00:00 2001 From: Ameet Kotian Date: Mon, 17 Sep 2018 16:57:00 -0700 Subject: [PATCH 46/72] Initial import of change to parallelize backup compresion Signed-off-by: Ameet Kotian --- go/vt/mysqlctl/backup.go | 17 +++++++++++++---- go/vt/mysqlctl/s3backupstorage/s3.go | 12 ++++++------ vendor/vendor.json | 24 ++++++++++++++++++++++++ 3 files changed, 43 insertions(+), 10 deletions(-) diff --git a/go/vt/mysqlctl/backup.go b/go/vt/mysqlctl/backup.go index a6f69ae290a..61accab83c0 100644 --- a/go/vt/mysqlctl/backup.go +++ b/go/vt/mysqlctl/backup.go @@ -30,9 +30,9 @@ import ( "strings" "sync" + "github.com/klauspost/pgzip" "golang.org/x/net/context" - "vitess.io/vitess/go/cgzip" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sync2" @@ -78,6 +78,14 @@ var ( // on the backups. Usually would be set if a hook is used, and // the hook compresses the data. backupStorageCompress = flag.Bool("backup_storage_compress", true, "if set, the backup files will be compressed (default is true). Set to false for instance if a backup_storage_hook is specified and it compresses the data.") + + // backupCompressBlockSize is the splitting size for each + // compressed block + backupCompressBlockSize = flag.Int("backup_storage_block_size", 250000, "if backup_storage_compress is true, backup_storage_block_size sets the byte size for each block while compressing (default is 250000).") + + // backupCompressBlocks is the number of blocks that are processed + // once before the writer blocks + backupCompressBlocks = flag.Int("backup_storage_number_blocks", 2, "if backup_storage_compress is true, backup_storage_number_blocks sets the number of blocks that can be processed, at once, before the writer blocks, during compression (default is 2). It should be equal to the number of CPUs available for compression") ) // FileEntry is one file to backup @@ -475,12 +483,13 @@ func backupFile(ctx context.Context, cnf *Mycnf, mysqld MysqlDaemon, logger logu } // Create the gzip compression pipe, if necessary. - var gzip *cgzip.Writer + var gzip *pgzip.Writer if *backupStorageCompress { - gzip, err = cgzip.NewWriterLevel(writer, cgzip.Z_BEST_SPEED) + gzip, err = pgzip.NewWriterLevel(writer, pgzip.BestSpeed) if err != nil { return fmt.Errorf("cannot create gziper: %v", err) } + gzip.SetConcurrency(*backupCompressBlockSize, *backupCompressBlocks) writer = gzip } @@ -632,7 +641,7 @@ func restoreFile(ctx context.Context, cnf *Mycnf, bh backupstorage.BackupHandle, // Create the uncompresser if needed. if compress { - gz, err := cgzip.NewReader(reader) + gz, err := pgzip.NewReader(reader) if err != nil { return err } diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index 03bdd41b72b..2ccae58bd0d 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -87,13 +87,13 @@ func (bh *S3BackupHandle) AddFile(ctx context.Context, filename string, filesize } // Calculate s3 upload part size using the source filesize - partSizeMB := s3manager.DefaultUploadPartSize + partSizeBytes := s3manager.DefaultUploadPartSize if filesize > 0 { minimumPartSize := float64(filesize) / float64(s3manager.MaxUploadParts) - // Convert partsize to mb and round up to ensure large enough partsize - calculatedPartSizeMB := int64(math.Ceil(minimumPartSize / 1024 * 1024)) - if calculatedPartSizeMB > partSizeMB { - partSizeMB = calculatedPartSizeMB + // Round up to ensure large enough partsize + calculatedPartSizeBytes := int64(math.Ceil(minimumPartSize)) + if calculatedPartSizeBytes > partSizeBytes { + partSizeBytes = calculatedPartSizeBytes } } @@ -103,7 +103,7 @@ func (bh *S3BackupHandle) AddFile(ctx context.Context, filename string, filesize go func() { defer bh.waitGroup.Done() uploader := s3manager.NewUploaderWithClient(bh.client, func(u *s3manager.Uploader) { - u.PartSize = partSizeMB + u.PartSize = partSizeBytes }) object := objName(bh.dir, bh.name, filename) diff --git a/vendor/vendor.json b/vendor/vendor.json index 72e7aaf9735..f7611722c64 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -638,6 +638,30 @@ "revision": "8ddce2a84170772b95dd5d576c48d517b22cac63", "revisionTime": "2016-01-05T22:08:40Z" }, + { + "checksumSHA1": "rmCbOBewXcbEdHRerzoanS+kI2U=", + "path": "github.com/klauspost/compress/flate", + "revision": "b50017755d442260d792c34c7c43216d9ba7ffc7", + "revisionTime": "2018-08-01T09:52:37Z" + }, + { + "checksumSHA1": "vGHBCcWkLCbAc3PJcRs7vFbvaYM=", + "path": "github.com/klauspost/cpuid", + "revision": "e7e905edc00ea8827e58662220139109efea09db", + "revisionTime": "2018-04-05T13:32:22Z" + }, + { + "checksumSHA1": "6/zXof97s7P9tlNp3mUioXgeEVI=", + "path": "github.com/klauspost/crc32", + "revision": "bab58d77464aa9cf4e84200c3276da0831fe0c03", + "revisionTime": "2017-06-28T07:24:49Z" + }, + { + "checksumSHA1": "N4EMcnfxHl1f4TQsKQWX/yLF/BE=", + "path": "github.com/klauspost/pgzip", + "revision": "c4ad2ed77aece7b3270909769a30a3fcea262a66", + "revisionTime": "2018-07-17T08:42:24Z" + }, { "checksumSHA1": "DdH3xAkzAWJ4B/LGYJyCeRsly2I=", "path": "github.com/mattn/go-runewidth", From 90d728018cd10c020a6b5d524a218b5dc0c5e2f5 Mon Sep 17 00:00:00 2001 From: Arthur Neves Date: Fri, 12 Oct 2018 17:37:08 -0400 Subject: [PATCH 47/72] Follow the MySQL handshake protocol and send 10 NULL bytes See the docs: https://dev.mysql.com/doc/internals/en/connection-phase-packets.html#packet-Protocol::Handshake MySQL server will send a 10 bytes (0 value) in the handshake. Before this change, Vitess was sending same random value, that is because the buffer we use here comes from the buffer pool, so that could be same random memory values. We caught this because our mysql client validates those 10 bytes being value 0. Signed-off-by: Arthur Neves --- go/mysql/server.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/mysql/server.go b/go/mysql/server.go index 6b634b9c8cc..40b38622145 100644 --- a/go/mysql/server.go +++ b/go/mysql/server.go @@ -521,8 +521,8 @@ func (c *Conn) writeHandshakeV10(serverVersion string, authServer AuthServer, en // Always 21 (8 + 13). pos = writeByte(data, pos, 21) - // Reserved - pos += 10 + // Reserved 10 bytes: all 0 + pos = writeZeroes(data, pos, 10) // Second part of auth plugin data. pos += copy(data[pos:], salt[8:]) From 327fea19ea32600494d3a5bf3eec3eeaaee345ba Mon Sep 17 00:00:00 2001 From: Leo Xuzhang Lin Date: Fri, 12 Oct 2018 15:47:40 +0800 Subject: [PATCH 48/72] Fix golint install in bootstrap.sh Signed-off-by: Leo Xuzhang Lin --- bootstrap.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bootstrap.sh b/bootstrap.sh index ec228ceeb40..489a4675a76 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -266,9 +266,9 @@ install_dep "chromedriver" "2.40" "$VTROOT/dist/chromedriver" install_chromedriv # Note: We explicitly do not vendor the tools below because a) we want to stay # on their latest version and b) it's easier to "go install" them this way. gotools=" \ - github.com/golang/lint/golint \ github.com/golang/mock/mockgen \ github.com/kardianos/govendor \ + golang.org/x/lint/golint \ golang.org/x/tools/cmd/cover \ golang.org/x/tools/cmd/goimports \ golang.org/x/tools/cmd/goyacc \ From c8c34f43677e2c6df3b5cf7f55cfff0c91aa3e59 Mon Sep 17 00:00:00 2001 From: Scott Lanning Date: Tue, 30 Oct 2018 12:46:03 +0100 Subject: [PATCH 49/72] change s3_backup_aws_region flag to `""` based on feedback by @demmer in PR 4200 For reference, the docs for Config.Endpoint: https://github.com/aws/aws-sdk-go/blob/8d83316e1e48/aws/config.go#L44 say "Set this to `""` to use the default generated endpoint." Signed-off-by: Scott Lanning --- go/vt/mysqlctl/s3backupstorage/s3.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index fe63c6c5191..40260884122 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -51,7 +51,7 @@ var ( region = flag.String("s3_backup_aws_region", "us-east-1", "AWS region to use") // AWS endpoint, defaults to amazonaws.com but appliances may use a different location - endpoint = flag.String("s3_backup_aws_endpoint", "amazonaws.com", "endpoint of the S3 backend (region must be provided)") + endpoint = flag.String("s3_backup_aws_endpoint", "", "endpoint of the S3 backend (region must be provided)") // bucket is where the backups will go. bucket = flag.String("s3_backup_storage_bucket", "", "S3 bucket to use for backups") From b8316de8ab9a81db95f2a76ad26de610b2ae1cc9 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Mon, 29 Oct 2018 13:36:31 -0700 Subject: [PATCH 50/72] Improve warning so it's not that verbose. We should only warn when the creds file is provided and a cred is not found. Before this change it was going to warn even when the file was not provided. Signed-off-by: Rafael Chacon --- go/vt/topo/consultopo/server.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/go/vt/topo/consultopo/server.go b/go/vt/topo/consultopo/server.go index 56e1026f34a..55bd575f8fc 100644 --- a/go/vt/topo/consultopo/server.go +++ b/go/vt/topo/consultopo/server.go @@ -110,11 +110,14 @@ func NewServer(cell, serverAddr, root string) (*Server, error) { } cfg := api.DefaultConfig() cfg.Address = serverAddr - if creds != nil && creds[cell] != nil { - cfg.Token = creds[cell].ACLToken - } else { - log.Warningf("Client auth not configured for cell: %v", cell) + if creds != nil { + if creds[cell] != nil { + cfg.Token = creds[cell].ACLToken + } else { + log.Warningf("Client auth not configured for cell: %v", cell) + } } + client, err := api.NewClient(cfg) if err != nil { return nil, err From 6e4f991e7775d32df765f4da8b813e062cfa2277 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sun, 16 Dec 2018 18:40:45 -0800 Subject: [PATCH 51/72] List all tablets in all cells Signed-off-by: Rafael Chacon --- go/vt/vtctl/vtctl.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index f01f416efe6..cc83193a936 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -335,6 +335,9 @@ var commands = []commandGroup{ {"ListAllTablets", commandListAllTablets, "", "Lists all tablets in an awk-friendly way."}, + {"ListAllTabletsAllCells", commandListAllTabletsAllCells, + "", + "Lists all tablets in all cells an awk-friendly way."}, {"ListTablets", commandListTablets, " ...", "Lists specified tablets in an awk-friendly way."}, @@ -1795,6 +1798,20 @@ func commandValidate(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag. return wr.Validate(ctx, *pingTablets) } +func commandListAllTabletsAllCells(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + cells, err := wr.TopoServer().GetKnownCells(ctx) + if err != nil { + return err + } + for _, cell := range cells { + err := dumpAllTablets(ctx, wr, cell) + if err != nil { + return err + } + } + return nil +} + func commandListAllTablets(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { if err := subFlags.Parse(args); err != nil { return err From c8023fef5adb8a4162f2a5a3e4f873665877241f Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sun, 16 Dec 2018 18:55:26 -0800 Subject: [PATCH 52/72] Refactor to not require a new command Signed-off-by: Rafael Chacon --- go/vt/vtctl/vtctl.go | 33 ++++++++++++++------------------- 1 file changed, 14 insertions(+), 19 deletions(-) diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index cc83193a936..d79902547ed 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -333,11 +333,8 @@ var commands = []commandGroup{ "[-ping-tablets]", "Validates that all nodes reachable from the global replication graph and that all tablets in all discoverable cells are consistent."}, {"ListAllTablets", commandListAllTablets, - "", + ", , ...", "Lists all tablets in an awk-friendly way."}, - {"ListAllTabletsAllCells", commandListAllTabletsAllCells, - "", - "Lists all tablets in all cells an awk-friendly way."}, {"ListTablets", commandListTablets, " ...", "Lists specified tablets in an awk-friendly way."}, @@ -1798,11 +1795,21 @@ func commandValidate(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag. return wr.Validate(ctx, *pingTablets) } -func commandListAllTabletsAllCells(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { - cells, err := wr.TopoServer().GetKnownCells(ctx) - if err != nil { +func commandListAllTablets(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + if err := subFlags.Parse(args); err != nil { return err } + var cells []string + var err error + if subFlags.NArg() == 1 { + cells = strings.Split(subFlags.Arg(0), ",") + } else { + cells, err = wr.TopoServer().GetKnownCells(ctx) + if err != nil { + return err + } + } + for _, cell := range cells { err := dumpAllTablets(ctx, wr, cell) if err != nil { @@ -1812,18 +1819,6 @@ func commandListAllTabletsAllCells(ctx context.Context, wr *wrangler.Wrangler, s return nil } -func commandListAllTablets(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { - if err := subFlags.Parse(args); err != nil { - return err - } - if subFlags.NArg() != 1 { - return fmt.Errorf("the argument is required for the ListAllTablets command") - } - - cell := subFlags.Arg(0) - return dumpAllTablets(ctx, wr, cell) -} - func commandListTablets(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { if err := subFlags.Parse(args); err != nil { return err From ac8d661566d7a4de22f86c092d7f554f44c1d2f3 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sat, 15 Dec 2018 12:18:04 -0800 Subject: [PATCH 53/72] Add region context to aggr stats Signed-off-by: Rafael Chacon --- go/vt/discovery/tablet_stats_cache.go | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index af0add2bf4b..34c62864137 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -498,11 +498,28 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } - agg, ok := e.aggregates[target.Cell] - if !ok { + targetRegion := tc.getRegionByCell(target.Cell) + regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} + regionAggrFound := false + for cell, agg := range e.aggregates { + cellRegion := tc.getRegionByCell(cell) + if cellRegion == targetRegion { + regionAggrFound = true + regionAggregateStats.HealthyTabletCount += agg.HealthyTabletCount + regionAggregateStats.UnhealthyTabletCount += agg.UnhealthyTabletCount + if agg.SecondsBehindMasterMin < regionAggregateStats.SecondsBehindMasterMin { + regionAggregateStats.SecondsBehindMasterMin = agg.SecondsBehindMasterMin + } + if regionAggregateStats.SecondsBehindMasterMax > agg.SecondsBehindMasterMax { + regionAggregateStats.SecondsBehindMasterMax = agg.SecondsBehindMasterMax + } + } + } + if !regionAggrFound { return nil, topo.NewError(topo.NoNode, topotools.TargetIdent(target)) } - return agg, nil + return regionAggregateStats, nil + } // GetMasterCell is part of the TargetStatsListener interface. From 6e95f149ca751a7d177970c60045c7d8bcd61d42 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sat, 15 Dec 2018 15:47:10 -0800 Subject: [PATCH 54/72] Adds tests for discovery gateway aggregate stats Signed-off-by: Rafael Chacon --- go/vt/discovery/tablet_stats_cache.go | 1 + go/vt/vtgate/gateway/discoverygateway_test.go | 145 ++++++++++++++++++ 2 files changed, 146 insertions(+) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index 34c62864137..b148d68ee7a 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -498,6 +498,7 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } + targetRegion := tc.getRegionByCell(target.Cell) regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} regionAggrFound := false diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 4f0680f5763..510a86d459c 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -206,6 +206,107 @@ func TestShuffleTablets(t *testing.T) { } } +func TestDiscoveryGatewayGetAggregateStats(t *testing.T) { + keyspace := "ks" + shard := "0" + hc := discovery.NewFakeHealthCheck() + dg := createDiscoveryGateway(hc, nil, "cell1", 2).(*discoveryGateway) + + // replica should only use local ones + hc.Reset() + dg.tsc.ResetForTesting() + hc.AddTestTablet("cell1", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + hc.AddTestTablet("cell1", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + target := &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: topodatapb.TabletType_REPLICA, + Cell: "cell1", + } + tsl, err := dg.tsc.GetAggregateStats(target) + if err != nil { + t.Error(err) + } + if tsl.HealthyTabletCount != 2 { + t.Errorf("Expected 2 healthy replica tablets, got: %v", tsl.HealthyTabletCount) + } +} + +func TestDiscoveryGatewayGetAggregateStatsRegion(t *testing.T) { + keyspace := "ks" + shard := "0" + hc := discovery.NewFakeHealthCheck() + dg := createDiscoveryGateway(hc, nil, "local-east", 2).(*discoveryGateway) + + topo.UpdateCellsToRegionsForTests(map[string]string{ + "local-west": "local", + "local-east": "local", + "remote": "remote", + }) + + hc.Reset() + dg.tsc.ResetForTesting() + hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + hc.AddTestTablet("local-west", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + + // Non master targets in the same region as the gateway should be discoverable + target := &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: topodatapb.TabletType_REPLICA, + Cell: "local-west", + } + tsl, err := dg.tsc.GetAggregateStats(target) + if err != nil { + t.Fatalf("Expected no error, got %v", err) + } + if tsl.HealthyTabletCount != 2 { + t.Errorf("Expected 2 healthy replica tablets, got: %v", tsl.HealthyTabletCount) + } +} + +func TestDiscoveryGatewayGetAggregateStatsMaster(t *testing.T) { + keyspace := "ks" + shard := "0" + hc := discovery.NewFakeHealthCheck() + dg := createDiscoveryGateway(hc, nil, "cell1", 2).(*discoveryGateway) + + // replica should only use local ones + hc.Reset() + dg.tsc.ResetForTesting() + hc.AddTestTablet("cell1", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_MASTER, true, 10, nil) + target := &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: topodatapb.TabletType_MASTER, + Cell: "cell1", + } + tsl, err := dg.tsc.GetAggregateStats(target) + if err != nil { + t.Fatalf("Expected no error, got %v", err) + } + if tsl.HealthyTabletCount != 1 { + t.Errorf("Expected one healthy master, got: %v", tsl.HealthyTabletCount) + } + + // You can get aggregate regardless of the cell when requesting a master + target = &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: topodatapb.TabletType_MASTER, + Cell: "cell2", + } + + tsl, err = dg.tsc.GetAggregateStats(target) + if err != nil { + t.Fatalf("Expected no error, got %v", err) + } + if tsl.HealthyTabletCount != 1 { + t.Errorf("Expected one healthy master, got: %v", tsl.HealthyTabletCount) + } +} + func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { keyspace := "ks" shard := "0" @@ -230,6 +331,50 @@ func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { } } +func BenchmarkOneCellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(1, b) } + +func BenchmarkTenCellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(10, b) } + +func Benchmark100CellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(100, b) } + +func Benchmark1000CellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(1000, b) } + +func benchmarkCellsGetAggregateStats(i int, b *testing.B) { + keyspace := "ks" + shard := "0" + hc := discovery.NewFakeHealthCheck() + dg := createDiscoveryGateway(hc, nil, "cell0", 2).(*discoveryGateway) + cellsToregions := make(map[string]string) + for j := 0; j < i; j++ { + cell := fmt.Sprintf("cell%v", j) + cellsToregions[cell] = "local" + } + + topo.UpdateCellsToRegionsForTests(cellsToregions) + hc.Reset() + dg.tsc.ResetForTesting() + + for j := 0; j < i; j++ { + cell := fmt.Sprintf("cell%v", j) + ip := fmt.Sprintf("%v.%v.%v,%v", j, j, j, j) + hc.AddTestTablet(cell, ip, 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + } + + target := &querypb.Target{ + Keyspace: keyspace, + Shard: shard, + TabletType: topodatapb.TabletType_REPLICA, + Cell: "cell0", + } + + for n := 0; n < b.N; n++ { + _, err := dg.tsc.GetAggregateStats(target) + if err != nil { + b.Fatalf("Expected no error, got %v", err) + } + } +} + func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway, target *querypb.Target) error) { keyspace := "ks" shard := "0" From 0befc7e4c1df600d166d1435c62268a9862006f8 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sat, 15 Dec 2018 16:45:46 -0800 Subject: [PATCH 55/72] Change implementation to have aggregates per region * This change allows to have constant time resolution of aggregates per region Signed-off-by: Rafael Chacon --- go/vt/discovery/tablet_stats_cache.go | 48 ++++++++----------- go/vt/vtgate/gateway/discoverygateway_test.go | 7 ++- 2 files changed, 25 insertions(+), 30 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index b148d68ee7a..0998139facb 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -70,6 +70,9 @@ type tabletStatsCacheEntry struct { healthy []*TabletStats // aggregates has the per-cell aggregates. aggregates map[string]*querypb.AggregateStats + + // aggregatesPerRegion has the per-region aggregates. + aggregatesPerRegion map[string]*querypb.AggregateStats } func (e *tabletStatsCacheEntry) updateHealthyMapForMaster(ts *TabletStats) { @@ -266,18 +269,21 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { tc.updateAggregateMap(ts.Target.Keyspace, ts.Target.Shard, ts.Target.TabletType, e, allArray) } -// MakeAggregateMap takes a list of TabletStats and builds a per-cell +// makeAggregateMap takes a list of TabletStats and builds a per-cell // AggregateStats map. -func MakeAggregateMap(stats []*TabletStats) map[string]*querypb.AggregateStats { +func (tc *TabletStatsCache) makeAggregateMap(stats []*TabletStats, buildForRegion bool) map[string]*querypb.AggregateStats { result := make(map[string]*querypb.AggregateStats) for _, ts := range stats { - cell := ts.Tablet.Alias.Cell - agg, ok := result[cell] + cellOrRegion := ts.Tablet.Alias.Cell + if buildForRegion { + cellOrRegion = tc.getRegionByCell(cellOrRegion) + } + agg, ok := result[cellOrRegion] if !ok { agg = &querypb.AggregateStats{ SecondsBehindMasterMin: math.MaxUint32, } - result[cell] = agg + result[cellOrRegion] = agg } if ts.Serving && ts.LastError == nil { @@ -295,9 +301,9 @@ func MakeAggregateMap(stats []*TabletStats) map[string]*querypb.AggregateStats { return result } -// MakeAggregateMapDiff computes the entries that need to be broadcast +// makeAggregateMapDiff computes the entries that need to be broadcast // when the map goes from oldMap to newMap. -func MakeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletType, ter int64, oldMap map[string]*querypb.AggregateStats, newMap map[string]*querypb.AggregateStats) []*srvtopo.TargetStatsEntry { +func makeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletType, ter int64, oldMap map[string]*querypb.AggregateStats, newMap map[string]*querypb.AggregateStats) []*srvtopo.TargetStatsEntry { var result []*srvtopo.TargetStatsEntry for cell, oldValue := range oldMap { newValue, ok := newMap[cell] @@ -360,8 +366,9 @@ func MakeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletTy func (tc *TabletStatsCache) updateAggregateMap(keyspace, shard string, tabletType topodatapb.TabletType, e *tabletStatsCacheEntry, stats []*TabletStats) { // Save the new value oldAgg := e.aggregates - newAgg := MakeAggregateMap(stats) + newAgg := tc.makeAggregateMap(stats /* buildForRegion */, false) e.aggregates = newAgg + e.aggregatesPerRegion = tc.makeAggregateMap(stats /* buildForRegion */, true) // And broadcast the change in the background, if we need to. tc.mu.RLock() @@ -376,7 +383,7 @@ func (tc *TabletStatsCache) updateAggregateMap(keyspace, shard string, tabletTyp if len(stats) > 0 { ter = stats[0].TabletExternallyReparentedTimestamp } - diffs := MakeAggregateMapDiff(keyspace, shard, tabletType, ter, oldAgg, newAgg) + diffs := makeAggregateMapDiff(keyspace, shard, tabletType, ter, oldAgg, newAgg) tc.aggregatesChan <- diffs } @@ -498,29 +505,12 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } - targetRegion := tc.getRegionByCell(target.Cell) - regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} - regionAggrFound := false - for cell, agg := range e.aggregates { - cellRegion := tc.getRegionByCell(cell) - if cellRegion == targetRegion { - regionAggrFound = true - regionAggregateStats.HealthyTabletCount += agg.HealthyTabletCount - regionAggregateStats.UnhealthyTabletCount += agg.UnhealthyTabletCount - if agg.SecondsBehindMasterMin < regionAggregateStats.SecondsBehindMasterMin { - regionAggregateStats.SecondsBehindMasterMin = agg.SecondsBehindMasterMin - } - if regionAggregateStats.SecondsBehindMasterMax > agg.SecondsBehindMasterMax { - regionAggregateStats.SecondsBehindMasterMax = agg.SecondsBehindMasterMax - } - } - } - if !regionAggrFound { + agg, ok := e.aggregatesPerRegion[targetRegion] + if !ok { return nil, topo.NewError(topo.NoNode, topotools.TargetIdent(target)) } - return regionAggregateStats, nil - + return agg, nil } // GetMasterCell is part of the TargetStatsListener interface. diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 510a86d459c..353189805d1 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -347,7 +347,12 @@ func benchmarkCellsGetAggregateStats(i int, b *testing.B) { cellsToregions := make(map[string]string) for j := 0; j < i; j++ { cell := fmt.Sprintf("cell%v", j) - cellsToregions[cell] = "local" + // let's assume that there are few local cells + if j <= 4 { + cellsToregions[cell] = "local" + } else { + cellsToregions[cell] = "remote" + } } topo.UpdateCellsToRegionsForTests(cellsToregions) From a51c547c0f7b1fcd576be099c9b7f340b6ebcef1 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Sat, 15 Dec 2018 17:26:56 -0800 Subject: [PATCH 56/72] Refactor test to make it even more generic. Remove extra line in struct def Signed-off-by: Rafael Chacon --- go/vt/discovery/tablet_stats_cache.go | 1 - go/vt/vtgate/gateway/discoverygateway_test.go | 7 +------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index 0998139facb..720d8089dc5 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -70,7 +70,6 @@ type tabletStatsCacheEntry struct { healthy []*TabletStats // aggregates has the per-cell aggregates. aggregates map[string]*querypb.AggregateStats - // aggregatesPerRegion has the per-region aggregates. aggregatesPerRegion map[string]*querypb.AggregateStats } diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 353189805d1..510a86d459c 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -347,12 +347,7 @@ func benchmarkCellsGetAggregateStats(i int, b *testing.B) { cellsToregions := make(map[string]string) for j := 0; j < i; j++ { cell := fmt.Sprintf("cell%v", j) - // let's assume that there are few local cells - if j <= 4 { - cellsToregions[cell] = "local" - } else { - cellsToregions[cell] = "remote" - } + cellsToregions[cell] = "local" } topo.UpdateCellsToRegionsForTests(cellsToregions) From 4ec10ebae8d6eef4c6f53e8aa5db15851000b1a8 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Fri, 21 Dec 2018 09:39:05 -0800 Subject: [PATCH 57/72] Revert "Refactor test to make it even more generic. Remove extra line in struct def" This reverts commit a51c547c0f7b1fcd576be099c9b7f340b6ebcef1. --- go/vt/discovery/tablet_stats_cache.go | 1 + go/vt/vtgate/gateway/discoverygateway_test.go | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index 720d8089dc5..0998139facb 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -70,6 +70,7 @@ type tabletStatsCacheEntry struct { healthy []*TabletStats // aggregates has the per-cell aggregates. aggregates map[string]*querypb.AggregateStats + // aggregatesPerRegion has the per-region aggregates. aggregatesPerRegion map[string]*querypb.AggregateStats } diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 510a86d459c..353189805d1 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -347,7 +347,12 @@ func benchmarkCellsGetAggregateStats(i int, b *testing.B) { cellsToregions := make(map[string]string) for j := 0; j < i; j++ { cell := fmt.Sprintf("cell%v", j) - cellsToregions[cell] = "local" + // let's assume that there are few local cells + if j <= 4 { + cellsToregions[cell] = "local" + } else { + cellsToregions[cell] = "remote" + } } topo.UpdateCellsToRegionsForTests(cellsToregions) From 46d4d28a6a1977d3630b42aa11e61734c892d190 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Fri, 21 Dec 2018 09:39:20 -0800 Subject: [PATCH 58/72] Revert "Change implementation to have aggregates per region" This reverts commit 0befc7e4c1df600d166d1435c62268a9862006f8. --- go/vt/discovery/tablet_stats_cache.go | 48 +++++++++++-------- go/vt/vtgate/gateway/discoverygateway_test.go | 7 +-- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index 0998139facb..b148d68ee7a 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -70,9 +70,6 @@ type tabletStatsCacheEntry struct { healthy []*TabletStats // aggregates has the per-cell aggregates. aggregates map[string]*querypb.AggregateStats - - // aggregatesPerRegion has the per-region aggregates. - aggregatesPerRegion map[string]*querypb.AggregateStats } func (e *tabletStatsCacheEntry) updateHealthyMapForMaster(ts *TabletStats) { @@ -269,21 +266,18 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { tc.updateAggregateMap(ts.Target.Keyspace, ts.Target.Shard, ts.Target.TabletType, e, allArray) } -// makeAggregateMap takes a list of TabletStats and builds a per-cell +// MakeAggregateMap takes a list of TabletStats and builds a per-cell // AggregateStats map. -func (tc *TabletStatsCache) makeAggregateMap(stats []*TabletStats, buildForRegion bool) map[string]*querypb.AggregateStats { +func MakeAggregateMap(stats []*TabletStats) map[string]*querypb.AggregateStats { result := make(map[string]*querypb.AggregateStats) for _, ts := range stats { - cellOrRegion := ts.Tablet.Alias.Cell - if buildForRegion { - cellOrRegion = tc.getRegionByCell(cellOrRegion) - } - agg, ok := result[cellOrRegion] + cell := ts.Tablet.Alias.Cell + agg, ok := result[cell] if !ok { agg = &querypb.AggregateStats{ SecondsBehindMasterMin: math.MaxUint32, } - result[cellOrRegion] = agg + result[cell] = agg } if ts.Serving && ts.LastError == nil { @@ -301,9 +295,9 @@ func (tc *TabletStatsCache) makeAggregateMap(stats []*TabletStats, buildForRegio return result } -// makeAggregateMapDiff computes the entries that need to be broadcast +// MakeAggregateMapDiff computes the entries that need to be broadcast // when the map goes from oldMap to newMap. -func makeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletType, ter int64, oldMap map[string]*querypb.AggregateStats, newMap map[string]*querypb.AggregateStats) []*srvtopo.TargetStatsEntry { +func MakeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletType, ter int64, oldMap map[string]*querypb.AggregateStats, newMap map[string]*querypb.AggregateStats) []*srvtopo.TargetStatsEntry { var result []*srvtopo.TargetStatsEntry for cell, oldValue := range oldMap { newValue, ok := newMap[cell] @@ -366,9 +360,8 @@ func makeAggregateMapDiff(keyspace, shard string, tabletType topodatapb.TabletTy func (tc *TabletStatsCache) updateAggregateMap(keyspace, shard string, tabletType topodatapb.TabletType, e *tabletStatsCacheEntry, stats []*TabletStats) { // Save the new value oldAgg := e.aggregates - newAgg := tc.makeAggregateMap(stats /* buildForRegion */, false) + newAgg := MakeAggregateMap(stats) e.aggregates = newAgg - e.aggregatesPerRegion = tc.makeAggregateMap(stats /* buildForRegion */, true) // And broadcast the change in the background, if we need to. tc.mu.RLock() @@ -383,7 +376,7 @@ func (tc *TabletStatsCache) updateAggregateMap(keyspace, shard string, tabletTyp if len(stats) > 0 { ter = stats[0].TabletExternallyReparentedTimestamp } - diffs := makeAggregateMapDiff(keyspace, shard, tabletType, ter, oldAgg, newAgg) + diffs := MakeAggregateMapDiff(keyspace, shard, tabletType, ter, oldAgg, newAgg) tc.aggregatesChan <- diffs } @@ -505,12 +498,29 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } + targetRegion := tc.getRegionByCell(target.Cell) - agg, ok := e.aggregatesPerRegion[targetRegion] - if !ok { + regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} + regionAggrFound := false + for cell, agg := range e.aggregates { + cellRegion := tc.getRegionByCell(cell) + if cellRegion == targetRegion { + regionAggrFound = true + regionAggregateStats.HealthyTabletCount += agg.HealthyTabletCount + regionAggregateStats.UnhealthyTabletCount += agg.UnhealthyTabletCount + if agg.SecondsBehindMasterMin < regionAggregateStats.SecondsBehindMasterMin { + regionAggregateStats.SecondsBehindMasterMin = agg.SecondsBehindMasterMin + } + if regionAggregateStats.SecondsBehindMasterMax > agg.SecondsBehindMasterMax { + regionAggregateStats.SecondsBehindMasterMax = agg.SecondsBehindMasterMax + } + } + } + if !regionAggrFound { return nil, topo.NewError(topo.NoNode, topotools.TargetIdent(target)) } - return agg, nil + return regionAggregateStats, nil + } // GetMasterCell is part of the TargetStatsListener interface. diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 353189805d1..510a86d459c 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -347,12 +347,7 @@ func benchmarkCellsGetAggregateStats(i int, b *testing.B) { cellsToregions := make(map[string]string) for j := 0; j < i; j++ { cell := fmt.Sprintf("cell%v", j) - // let's assume that there are few local cells - if j <= 4 { - cellsToregions[cell] = "local" - } else { - cellsToregions[cell] = "remote" - } + cellsToregions[cell] = "local" } topo.UpdateCellsToRegionsForTests(cellsToregions) From de85143a18cdf4e07e7e71a1d9c8f4656c878209 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Fri, 21 Dec 2018 09:39:29 -0800 Subject: [PATCH 59/72] Revert "Adds tests for discovery gateway aggregate stats" This reverts commit 6e95f149ca751a7d177970c60045c7d8bcd61d42. --- go/vt/discovery/tablet_stats_cache.go | 1 - go/vt/vtgate/gateway/discoverygateway_test.go | 145 ------------------ 2 files changed, 146 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index b148d68ee7a..34c62864137 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -498,7 +498,6 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } - targetRegion := tc.getRegionByCell(target.Cell) regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} regionAggrFound := false diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index 510a86d459c..4f0680f5763 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -206,107 +206,6 @@ func TestShuffleTablets(t *testing.T) { } } -func TestDiscoveryGatewayGetAggregateStats(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "cell1", 2).(*discoveryGateway) - - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("cell1", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - hc.AddTestTablet("cell1", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: topodatapb.TabletType_REPLICA, - Cell: "cell1", - } - tsl, err := dg.tsc.GetAggregateStats(target) - if err != nil { - t.Error(err) - } - if tsl.HealthyTabletCount != 2 { - t.Errorf("Expected 2 healthy replica tablets, got: %v", tsl.HealthyTabletCount) - } -} - -func TestDiscoveryGatewayGetAggregateStatsRegion(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "local-east", 2).(*discoveryGateway) - - topo.UpdateCellsToRegionsForTests(map[string]string{ - "local-west": "local", - "local-east": "local", - "remote": "remote", - }) - - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - hc.AddTestTablet("local-west", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - - // Non master targets in the same region as the gateway should be discoverable - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: topodatapb.TabletType_REPLICA, - Cell: "local-west", - } - tsl, err := dg.tsc.GetAggregateStats(target) - if err != nil { - t.Fatalf("Expected no error, got %v", err) - } - if tsl.HealthyTabletCount != 2 { - t.Errorf("Expected 2 healthy replica tablets, got: %v", tsl.HealthyTabletCount) - } -} - -func TestDiscoveryGatewayGetAggregateStatsMaster(t *testing.T) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "cell1", 2).(*discoveryGateway) - - // replica should only use local ones - hc.Reset() - dg.tsc.ResetForTesting() - hc.AddTestTablet("cell1", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_MASTER, true, 10, nil) - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: topodatapb.TabletType_MASTER, - Cell: "cell1", - } - tsl, err := dg.tsc.GetAggregateStats(target) - if err != nil { - t.Fatalf("Expected no error, got %v", err) - } - if tsl.HealthyTabletCount != 1 { - t.Errorf("Expected one healthy master, got: %v", tsl.HealthyTabletCount) - } - - // You can get aggregate regardless of the cell when requesting a master - target = &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: topodatapb.TabletType_MASTER, - Cell: "cell2", - } - - tsl, err = dg.tsc.GetAggregateStats(target) - if err != nil { - t.Fatalf("Expected no error, got %v", err) - } - if tsl.HealthyTabletCount != 1 { - t.Errorf("Expected one healthy master, got: %v", tsl.HealthyTabletCount) - } -} - func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { keyspace := "ks" shard := "0" @@ -331,50 +230,6 @@ func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { } } -func BenchmarkOneCellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(1, b) } - -func BenchmarkTenCellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(10, b) } - -func Benchmark100CellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(100, b) } - -func Benchmark1000CellGetAggregateStats(b *testing.B) { benchmarkCellsGetAggregateStats(1000, b) } - -func benchmarkCellsGetAggregateStats(i int, b *testing.B) { - keyspace := "ks" - shard := "0" - hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "cell0", 2).(*discoveryGateway) - cellsToregions := make(map[string]string) - for j := 0; j < i; j++ { - cell := fmt.Sprintf("cell%v", j) - cellsToregions[cell] = "local" - } - - topo.UpdateCellsToRegionsForTests(cellsToregions) - hc.Reset() - dg.tsc.ResetForTesting() - - for j := 0; j < i; j++ { - cell := fmt.Sprintf("cell%v", j) - ip := fmt.Sprintf("%v.%v.%v,%v", j, j, j, j) - hc.AddTestTablet(cell, ip, 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) - } - - target := &querypb.Target{ - Keyspace: keyspace, - Shard: shard, - TabletType: topodatapb.TabletType_REPLICA, - Cell: "cell0", - } - - for n := 0; n < b.N; n++ { - _, err := dg.tsc.GetAggregateStats(target) - if err != nil { - b.Fatalf("Expected no error, got %v", err) - } - } -} - func testDiscoveryGatewayGeneric(t *testing.T, streaming bool, f func(dg Gateway, target *querypb.Target) error) { keyspace := "ks" shard := "0" From 49b98bedf3b9dfb8e3b23dbc10d7a6aafed7b96f Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Fri, 21 Dec 2018 09:39:47 -0800 Subject: [PATCH 60/72] Revert "Add region context to aggr stats" This reverts commit ac8d661566d7a4de22f86c092d7f554f44c1d2f3. --- go/vt/discovery/tablet_stats_cache.go | 23 +++-------------------- 1 file changed, 3 insertions(+), 20 deletions(-) diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index 34c62864137..af0add2bf4b 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -498,28 +498,11 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } - targetRegion := tc.getRegionByCell(target.Cell) - regionAggregateStats := &querypb.AggregateStats{SecondsBehindMasterMin: math.MaxUint32} - regionAggrFound := false - for cell, agg := range e.aggregates { - cellRegion := tc.getRegionByCell(cell) - if cellRegion == targetRegion { - regionAggrFound = true - regionAggregateStats.HealthyTabletCount += agg.HealthyTabletCount - regionAggregateStats.UnhealthyTabletCount += agg.UnhealthyTabletCount - if agg.SecondsBehindMasterMin < regionAggregateStats.SecondsBehindMasterMin { - regionAggregateStats.SecondsBehindMasterMin = agg.SecondsBehindMasterMin - } - if regionAggregateStats.SecondsBehindMasterMax > agg.SecondsBehindMasterMax { - regionAggregateStats.SecondsBehindMasterMax = agg.SecondsBehindMasterMax - } - } - } - if !regionAggrFound { + agg, ok := e.aggregates[target.Cell] + if !ok { return nil, topo.NewError(topo.NoNode, topotools.TargetIdent(target)) } - return regionAggregateStats, nil - + return agg, nil } // GetMasterCell is part of the TargetStatsListener interface. From f0d747a438d0251ce782ce73cb924e2e522d6c44 Mon Sep 17 00:00:00 2001 From: Richard Date: Mon, 25 Feb 2019 13:36:54 -0800 Subject: [PATCH 61/72] Patch sqltypes.MakeRowTrusted to lean on field count (#126) Patch to handle this case while we move towards a permanent fix upstream. cf. https://github.com/vitessio/vitess/issues/4661 https://github.com/vitessio/vitess/issues/4669 --- go/sqltypes/result.go | 7 ++--- go/sqltypes/result_test.go | 55 ++++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+), 3 deletions(-) diff --git a/go/sqltypes/result.go b/go/sqltypes/result.go index 8c3c8317976..8c95716895c 100644 --- a/go/sqltypes/result.go +++ b/go/sqltypes/result.go @@ -185,13 +185,14 @@ func ResultsEqual(r1, r2 []Result) bool { // Every place this function is called, a comment is needed that explains // why it's justified. func MakeRowTrusted(fields []*querypb.Field, row *querypb.Row) []Value { - sqlRow := make([]Value, len(row.Lengths)) + sqlRow := make([]Value, len(fields)) var offset int64 - for i, length := range row.Lengths { + for i, fld := range fields { + length := row.Lengths[i] if length < 0 { continue } - sqlRow[i] = MakeTrusted(fields[i].Type, row.Values[offset:offset+length]) + sqlRow[i] = MakeTrusted(fld.Type, row.Values[offset:offset+length]) offset += length } return sqlRow diff --git a/go/sqltypes/result_test.go b/go/sqltypes/result_test.go index 27ba4494184..fa33a8c6fae 100644 --- a/go/sqltypes/result_test.go +++ b/go/sqltypes/result_test.go @@ -23,6 +23,61 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" ) +func TestMakeRowTrusted(t *testing.T) { + fields := MakeTestFields( + "some_int|some_text|another_int", + "int8|varchar|int8", + ) + + values := []byte{} + hw := []byte("hello, world") + values = append(values, hw...) + values = append(values, byte(42)) + + row := &querypb.Row{ + Lengths: []int64{-1, int64(len(hw)), 1}, + Values: values, + } + + want := []Value{ + MakeTrusted(querypb.Type_NULL_TYPE, nil), + MakeTrusted(querypb.Type_VARCHAR, []byte("hello, world")), + MakeTrusted(querypb.Type_INT8, []byte{byte(42)}), + } + + result := MakeRowTrusted(fields, row) + if !reflect.DeepEqual(result, want) { + t.Errorf("MakeRowTrusted:\ngot: %#v\nwant: %#v", result, want) + } +} + +func TestMakeRowTrustedDoesNotPanicOnNewColumns(t *testing.T) { + fields := MakeTestFields( + "some_int|some_text", + "int8|varchar", + ) + + values := []byte{byte(123)} + hw := []byte("hello, world") + values = append(values, hw...) + values = append(values, byte(42)) + + row := &querypb.Row{ + Lengths: []int64{1, int64(len(hw)), 1}, + Values: values, + } + + want := []Value{ + MakeTrusted(querypb.Type_INT8, []byte{byte(123)}), + MakeTrusted(querypb.Type_VARCHAR, []byte("hello, world")), + } + + result := MakeRowTrusted(fields, row) + if !reflect.DeepEqual(result, want) { + t.Errorf("MakeRowTrusted:\ngot: %#v\nwant: %#v", result, want) + } +} + func TestRepair(t *testing.T) { fields := []*querypb.Field{{ Type: Int64, From e59cf9d00340ec6d50c800f64be42b90be51d19d Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Thu, 28 Mar 2019 11:28:41 -0700 Subject: [PATCH 62/72] Fix potential nil pointer error Signed-off-by: Rafael Chacon --- go/vt/vttablet/tabletserver/tabletserver.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index d9d2529604b..003b27708a0 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1076,12 +1076,16 @@ func (tsv *TabletServer) ExecuteBatch(ctx context.Context, target *querypb.Targe defer tsv.endRequest(false) defer tsv.handlePanicAndSendLogStats("batch", nil, nil) + if options == nil { + options = &querypb.ExecuteOptions{} + } + // When all these conditions are met, we send the queries directly // to the MySQL without creating a transaction. This optimization // yields better throughput. // Setting ExecuteOptions_AUTOCOMMIT will get a connection out of the // pool without actually begin/commit the transaction. - if (options == nil || options.TransactionIsolation == querypb.ExecuteOptions_DEFAULT) && + if (options.TransactionIsolation == querypb.ExecuteOptions_DEFAULT) && tsv.qe.autoCommit.Get() && asTransaction && tsv.qe.passthroughDMLs.Get() { From 6351eba71aaa6ce291d13a68ecf7b06bf59ef0e3 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Thu, 28 Mar 2019 08:58:37 -0700 Subject: [PATCH 63/72] Refactor region to aliases * Deprecate the concept of regions. * Favor aliases instead Signed-off-by: Rafael Chacon --- go/vt/discovery/tablet_stats_cache.go | 36 +-- go/vt/discovery/tablet_stats_cache_test.go | 28 ++- go/vt/proto/topodata/topodata.pb.go | 231 ++++++++++-------- go/vt/topo/cells_aliases.go | 174 +++++++++++++ go/vt/topo/server.go | 64 +++-- go/vt/topo/topotests/cells_aliases_test.go | 151 ++++++++++++ go/vt/vtctl/cell_info.go | 13 +- go/vt/vtctl/cells_aliases.go | 130 ++++++++++ go/vt/vtgate/gateway/discoverygateway_test.go | 80 ++++-- proto/topodata.proto | 11 +- py/vtproto/topodata_pb2.py | 52 +++- 11 files changed, 774 insertions(+), 196 deletions(-) create mode 100644 go/vt/topo/cells_aliases.go create mode 100644 go/vt/topo/topotests/cells_aliases_test.go create mode 100644 go/vt/vtctl/cells_aliases.go diff --git a/go/vt/discovery/tablet_stats_cache.go b/go/vt/discovery/tablet_stats_cache.go index db8b6780bce..326e15e2d30 100644 --- a/go/vt/discovery/tablet_stats_cache.go +++ b/go/vt/discovery/tablet_stats_cache.go @@ -56,8 +56,8 @@ type TabletStatsCache struct { entries map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry // tsm is a helper to broadcast aggregate stats. tsm srvtopo.TargetStatsMultiplexer - // cellRegions is a cache of cell regions - cellRegions map[string]string + // cellAliases is a cache of cell aliases + cellAliases map[string]string } // tabletStatsCacheEntry is the per keyspace/shard/tabletType @@ -70,7 +70,7 @@ type tabletStatsCacheEntry struct { all map[string]*TabletStats // healthy only has the healthy ones. healthy []*TabletStats - // aggregates has the per-region aggregates. + // aggregates has the per-alias aggregates. aggregates map[string]*querypb.AggregateStats } @@ -136,7 +136,7 @@ func newTabletStatsCache(hc HealthCheck, ts *topo.Server, cell string, setListen aggregatesChan: make(chan []*srvtopo.TargetStatsEntry, 100), entries: make(map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry), tsm: srvtopo.NewTargetStatsMultiplexer(), - cellRegions: make(map[string]string), + cellAliases: make(map[string]string), } if setListener { @@ -196,26 +196,26 @@ func (tc *TabletStatsCache) getOrCreateEntry(target *querypb.Target) *tabletStat return e } -func (tc *TabletStatsCache) getRegionByCell(cell string) string { +func (tc *TabletStatsCache) getAliasByCell(cell string) string { tc.mu.Lock() defer tc.mu.Unlock() - if region, ok := tc.cellRegions[cell]; ok { - return region + if alias, ok := tc.cellAliases[cell]; ok { + return alias } - region := topo.GetRegionByCell(context.Background(), tc.ts, cell) - tc.cellRegions[cell] = region + alias := topo.GetAliasByCell(context.Background(), tc.ts, cell) + tc.cellAliases[cell] = alias - return region + return alias } // StatsUpdate is part of the HealthCheckStatsListener interface. func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { if ts.Target.TabletType != topodatapb.TabletType_MASTER && ts.Tablet.Alias.Cell != tc.cell && - tc.getRegionByCell(ts.Tablet.Alias.Cell) != tc.getRegionByCell(tc.cell) { - // this is for a non-master tablet in a different cell and a different region, drop it + tc.getAliasByCell(ts.Tablet.Alias.Cell) != tc.getAliasByCell(tc.cell) { + // this is for a non-master tablet in a different cell and a different alias, drop it return } @@ -280,18 +280,18 @@ func (tc *TabletStatsCache) StatsUpdate(ts *TabletStats) { tc.updateAggregateMap(ts.Target.Keyspace, ts.Target.Shard, ts.Target.TabletType, e, allArray) } -// makeAggregateMap takes a list of TabletStats and builds a per-region +// makeAggregateMap takes a list of TabletStats and builds a per-alias // AggregateStats map. func (tc *TabletStatsCache) makeAggregateMap(stats []*TabletStats) map[string]*querypb.AggregateStats { result := make(map[string]*querypb.AggregateStats) for _, ts := range stats { - region := tc.getRegionByCell(ts.Tablet.Alias.Cell) - agg, ok := result[region] + alias := tc.getAliasByCell(ts.Tablet.Alias.Cell) + agg, ok := result[alias] if !ok { agg = &querypb.AggregateStats{ SecondsBehindMasterMin: math.MaxUint32, } - result[region] = agg + result[alias] = agg } if ts.Serving && ts.LastError == nil { @@ -378,8 +378,8 @@ func (tc *TabletStatsCache) GetAggregateStats(target *querypb.Target) (*querypb. return agg, nil } } - targetRegion := tc.getRegionByCell(target.Cell) - agg, ok := e.aggregates[targetRegion] + targetAlias := tc.getAliasByCell(target.Cell) + agg, ok := e.aggregates[targetAlias] if !ok { return nil, topo.NewError(topo.NoNode, topotools.TargetIdent(target)) } diff --git a/go/vt/discovery/tablet_stats_cache_test.go b/go/vt/discovery/tablet_stats_cache_test.go index 8bc5e8caafd..635d3daca2d 100644 --- a/go/vt/discovery/tablet_stats_cache_test.go +++ b/go/vt/discovery/tablet_stats_cache_test.go @@ -17,9 +17,11 @@ limitations under the License. package discovery import ( + "context" "testing" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -27,20 +29,32 @@ import ( // TestTabletStatsCache tests the functionality of the TabletStatsCache class. func TestTabletStatsCache(t *testing.T) { - defer topo.UpdateCellsToRegionsForTests(map[string]string{}) - topo.UpdateCellsToRegionsForTests(map[string]string{ - "cell": "region1", - "cell1": "region1", - "cell2": "region2", - }) + ts := memorytopo.NewServer("cell", "cell1", "cell2") + + cellsAlias := &topodatapb.CellsAlias{ + Cells: []string{"cell", "cell1"}, + } + + ts.CreateCellsAlias(context.Background(), "region1", cellsAlias) + + defer ts.DeleteCellsAlias(context.Background(), "region1") + + cellsAlias = &topodatapb.CellsAlias{ + Cells: []string{"cell2"}, + } + + ts.CreateCellsAlias(context.Background(), "region2", cellsAlias) + + defer ts.DeleteCellsAlias(context.Background(), "region2") // We want to unit test TabletStatsCache without a full-blown // HealthCheck object, so we can't call NewTabletStatsCache. // So we just construct this object here. tsc := &TabletStatsCache{ cell: "cell", + ts: ts, entries: make(map[string]map[string]map[topodatapb.TabletType]*tabletStatsCacheEntry), - cellRegions: make(map[string]string), + cellAliases: make(map[string]string), } // empty diff --git a/go/vt/proto/topodata/topodata.pb.go b/go/vt/proto/topodata/topodata.pb.go index 2c9a5415926..9e22bf51309 100644 --- a/go/vt/proto/topodata/topodata.pb.go +++ b/go/vt/proto/topodata/topodata.pb.go @@ -48,7 +48,7 @@ func (x KeyspaceIdType) String() string { return proto.EnumName(KeyspaceIdType_name, int32(x)) } func (KeyspaceIdType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{0} } // TabletType represents the type of a given tablet. @@ -117,7 +117,7 @@ func (x TabletType) String() string { return proto.EnumName(TabletType_name, int32(x)) } func (TabletType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{1} + return fileDescriptor_topodata_5ff7209b363fe950, []int{1} } // KeyRange describes a range of sharding keys, when range-based @@ -134,7 +134,7 @@ func (m *KeyRange) Reset() { *m = KeyRange{} } func (m *KeyRange) String() string { return proto.CompactTextString(m) } func (*KeyRange) ProtoMessage() {} func (*KeyRange) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{0} } func (m *KeyRange) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_KeyRange.Unmarshal(m, b) @@ -184,7 +184,7 @@ func (m *TabletAlias) Reset() { *m = TabletAlias{} } func (m *TabletAlias) String() string { return proto.CompactTextString(m) } func (*TabletAlias) ProtoMessage() {} func (*TabletAlias) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{1} + return fileDescriptor_topodata_5ff7209b363fe950, []int{1} } func (m *TabletAlias) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_TabletAlias.Unmarshal(m, b) @@ -260,7 +260,7 @@ func (m *Tablet) Reset() { *m = Tablet{} } func (m *Tablet) String() string { return proto.CompactTextString(m) } func (*Tablet) ProtoMessage() {} func (*Tablet) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{2} + return fileDescriptor_topodata_5ff7209b363fe950, []int{2} } func (m *Tablet) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Tablet.Unmarshal(m, b) @@ -395,7 +395,7 @@ func (m *Shard) Reset() { *m = Shard{} } func (m *Shard) String() string { return proto.CompactTextString(m) } func (*Shard) ProtoMessage() {} func (*Shard) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{3} + return fileDescriptor_topodata_5ff7209b363fe950, []int{3} } func (m *Shard) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Shard.Unmarshal(m, b) @@ -470,7 +470,7 @@ func (m *Shard_ServedType) Reset() { *m = Shard_ServedType{} } func (m *Shard_ServedType) String() string { return proto.CompactTextString(m) } func (*Shard_ServedType) ProtoMessage() {} func (*Shard_ServedType) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{3, 0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{3, 0} } func (m *Shard_ServedType) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Shard_ServedType.Unmarshal(m, b) @@ -527,7 +527,7 @@ func (m *Shard_SourceShard) Reset() { *m = Shard_SourceShard{} } func (m *Shard_SourceShard) String() string { return proto.CompactTextString(m) } func (*Shard_SourceShard) ProtoMessage() {} func (*Shard_SourceShard) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{3, 1} + return fileDescriptor_topodata_5ff7209b363fe950, []int{3, 1} } func (m *Shard_SourceShard) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Shard_SourceShard.Unmarshal(m, b) @@ -600,7 +600,7 @@ func (m *Shard_TabletControl) Reset() { *m = Shard_TabletControl{} } func (m *Shard_TabletControl) String() string { return proto.CompactTextString(m) } func (*Shard_TabletControl) ProtoMessage() {} func (*Shard_TabletControl) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{3, 2} + return fileDescriptor_topodata_5ff7209b363fe950, []int{3, 2} } func (m *Shard_TabletControl) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Shard_TabletControl.Unmarshal(m, b) @@ -668,7 +668,7 @@ func (m *Keyspace) Reset() { *m = Keyspace{} } func (m *Keyspace) String() string { return proto.CompactTextString(m) } func (*Keyspace) ProtoMessage() {} func (*Keyspace) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{4} + return fileDescriptor_topodata_5ff7209b363fe950, []int{4} } func (m *Keyspace) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Keyspace.Unmarshal(m, b) @@ -727,7 +727,7 @@ func (m *Keyspace_ServedFrom) Reset() { *m = Keyspace_ServedFrom{} } func (m *Keyspace_ServedFrom) String() string { return proto.CompactTextString(m) } func (*Keyspace_ServedFrom) ProtoMessage() {} func (*Keyspace_ServedFrom) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{4, 0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{4, 0} } func (m *Keyspace_ServedFrom) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Keyspace_ServedFrom.Unmarshal(m, b) @@ -783,7 +783,7 @@ func (m *ShardReplication) Reset() { *m = ShardReplication{} } func (m *ShardReplication) String() string { return proto.CompactTextString(m) } func (*ShardReplication) ProtoMessage() {} func (*ShardReplication) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{5} + return fileDescriptor_topodata_5ff7209b363fe950, []int{5} } func (m *ShardReplication) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ShardReplication.Unmarshal(m, b) @@ -822,7 +822,7 @@ func (m *ShardReplication_Node) Reset() { *m = ShardReplication_Node{} } func (m *ShardReplication_Node) String() string { return proto.CompactTextString(m) } func (*ShardReplication_Node) ProtoMessage() {} func (*ShardReplication_Node) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{5, 0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{5, 0} } func (m *ShardReplication_Node) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ShardReplication_Node.Unmarshal(m, b) @@ -863,7 +863,7 @@ func (m *ShardReference) Reset() { *m = ShardReference{} } func (m *ShardReference) String() string { return proto.CompactTextString(m) } func (*ShardReference) ProtoMessage() {} func (*ShardReference) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{6} + return fileDescriptor_topodata_5ff7209b363fe950, []int{6} } func (m *ShardReference) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ShardReference.Unmarshal(m, b) @@ -913,7 +913,7 @@ func (m *ShardTabletControl) Reset() { *m = ShardTabletControl{} } func (m *ShardTabletControl) String() string { return proto.CompactTextString(m) } func (*ShardTabletControl) ProtoMessage() {} func (*ShardTabletControl) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{7} + return fileDescriptor_topodata_5ff7209b363fe950, []int{7} } func (m *ShardTabletControl) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ShardTabletControl.Unmarshal(m, b) @@ -971,7 +971,7 @@ func (m *SrvKeyspace) Reset() { *m = SrvKeyspace{} } func (m *SrvKeyspace) String() string { return proto.CompactTextString(m) } func (*SrvKeyspace) ProtoMessage() {} func (*SrvKeyspace) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{8} + return fileDescriptor_topodata_5ff7209b363fe950, []int{8} } func (m *SrvKeyspace) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SrvKeyspace.Unmarshal(m, b) @@ -1035,7 +1035,7 @@ func (m *SrvKeyspace_KeyspacePartition) Reset() { *m = SrvKeyspace_Keysp func (m *SrvKeyspace_KeyspacePartition) String() string { return proto.CompactTextString(m) } func (*SrvKeyspace_KeyspacePartition) ProtoMessage() {} func (*SrvKeyspace_KeyspacePartition) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{8, 0} + return fileDescriptor_topodata_5ff7209b363fe950, []int{8, 0} } func (m *SrvKeyspace_KeyspacePartition) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SrvKeyspace_KeyspacePartition.Unmarshal(m, b) @@ -1092,7 +1092,7 @@ func (m *SrvKeyspace_ServedFrom) Reset() { *m = SrvKeyspace_ServedFrom{} func (m *SrvKeyspace_ServedFrom) String() string { return proto.CompactTextString(m) } func (*SrvKeyspace_ServedFrom) ProtoMessage() {} func (*SrvKeyspace_ServedFrom) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{8, 1} + return fileDescriptor_topodata_5ff7209b363fe950, []int{8, 1} } func (m *SrvKeyspace_ServedFrom) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SrvKeyspace_ServedFrom.Unmarshal(m, b) @@ -1137,10 +1137,7 @@ type CellInfo struct { ServerAddress string `protobuf:"bytes,1,opt,name=server_address,json=serverAddress,proto3" json:"server_address,omitempty"` // Root is the path to store data in. It is only used when talking // to server_address. - Root string `protobuf:"bytes,2,opt,name=root,proto3" json:"root,omitempty"` - // Region is a group this cell belongs to. Used by vtgate to route traffic to - // other cells (in same region) when there is no available tablet in the current cell. - Region string `protobuf:"bytes,3,opt,name=region,proto3" json:"region,omitempty"` + Root string `protobuf:"bytes,2,opt,name=root,proto3" json:"root,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1150,7 +1147,7 @@ func (m *CellInfo) Reset() { *m = CellInfo{} } func (m *CellInfo) String() string { return proto.CompactTextString(m) } func (*CellInfo) ProtoMessage() {} func (*CellInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_topodata_e30d0a39cbb15c1e, []int{9} + return fileDescriptor_topodata_5ff7209b363fe950, []int{9} } func (m *CellInfo) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_CellInfo.Unmarshal(m, b) @@ -1184,11 +1181,44 @@ func (m *CellInfo) GetRoot() string { return "" } -func (m *CellInfo) GetRegion() string { +// CellsAlias +type CellsAlias struct { + // Cells that map to this alias + Cells []string `protobuf:"bytes,2,rep,name=cells,proto3" json:"cells,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CellsAlias) Reset() { *m = CellsAlias{} } +func (m *CellsAlias) String() string { return proto.CompactTextString(m) } +func (*CellsAlias) ProtoMessage() {} +func (*CellsAlias) Descriptor() ([]byte, []int) { + return fileDescriptor_topodata_5ff7209b363fe950, []int{10} +} +func (m *CellsAlias) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CellsAlias.Unmarshal(m, b) +} +func (m *CellsAlias) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CellsAlias.Marshal(b, m, deterministic) +} +func (dst *CellsAlias) XXX_Merge(src proto.Message) { + xxx_messageInfo_CellsAlias.Merge(dst, src) +} +func (m *CellsAlias) XXX_Size() int { + return xxx_messageInfo_CellsAlias.Size(m) +} +func (m *CellsAlias) XXX_DiscardUnknown() { + xxx_messageInfo_CellsAlias.DiscardUnknown(m) +} + +var xxx_messageInfo_CellsAlias proto.InternalMessageInfo + +func (m *CellsAlias) GetCells() []string { if m != nil { - return m.Region + return m.Cells } - return "" + return nil } func init() { @@ -1211,89 +1241,90 @@ func init() { proto.RegisterType((*SrvKeyspace_KeyspacePartition)(nil), "topodata.SrvKeyspace.KeyspacePartition") proto.RegisterType((*SrvKeyspace_ServedFrom)(nil), "topodata.SrvKeyspace.ServedFrom") proto.RegisterType((*CellInfo)(nil), "topodata.CellInfo") + proto.RegisterType((*CellsAlias)(nil), "topodata.CellsAlias") proto.RegisterEnum("topodata.KeyspaceIdType", KeyspaceIdType_name, KeyspaceIdType_value) proto.RegisterEnum("topodata.TabletType", TabletType_name, TabletType_value) } -func init() { proto.RegisterFile("topodata.proto", fileDescriptor_topodata_e30d0a39cbb15c1e) } +func init() { proto.RegisterFile("topodata.proto", fileDescriptor_topodata_5ff7209b363fe950) } -var fileDescriptor_topodata_e30d0a39cbb15c1e = []byte{ - // 1217 bytes of a gzipped FileDescriptorProto +var fileDescriptor_topodata_5ff7209b363fe950 = []byte{ + // 1218 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xe1, 0x6e, 0x1b, 0x45, 0x10, 0xe6, 0xec, 0xb3, 0x63, 0x8f, 0x1d, 0xe7, 0xba, 0xa4, 0xd5, 0xe9, 0xa0, 0x22, 0xb2, 0x54, - 0x11, 0x15, 0xe1, 0xa0, 0xb4, 0x85, 0xa8, 0x12, 0x52, 0x5d, 0xc7, 0xa5, 0x69, 0x1b, 0xc7, 0x5a, - 0x3b, 0x82, 0x22, 0xa1, 0xd3, 0xc5, 0xb7, 0x71, 0x4f, 0x39, 0xdf, 0xba, 0xbb, 0x9b, 0x48, 0xe6, - 0x15, 0xf8, 0x01, 0xfc, 0xe5, 0x0d, 0x78, 0x04, 0x9e, 0x80, 0xe7, 0x80, 0x27, 0x41, 0x3b, 0x7b, - 0x67, 0x9f, 0x1d, 0x1a, 0x52, 0x94, 0x7f, 0x33, 0xbb, 0x33, 0x73, 0x33, 0xdf, 0xcc, 0x37, 0x6b, - 0x43, 0x43, 0xf1, 0x29, 0x0f, 0x03, 0x15, 0xb4, 0xa6, 0x82, 0x2b, 0x4e, 0x2a, 0x99, 0xde, 0xdc, - 0x85, 0xca, 0x4b, 0x36, 0xa3, 0x41, 0x32, 0x66, 0x64, 0x13, 0x4a, 0x52, 0x05, 0x42, 0xb9, 0xd6, - 0x96, 0xb5, 0x5d, 0xa7, 0x46, 0x21, 0x0e, 0x14, 0x59, 0x12, 0xba, 0x05, 0x3c, 0xd3, 0x62, 0xf3, - 0x01, 0xd4, 0x86, 0xc1, 0x49, 0xcc, 0x54, 0x3b, 0x8e, 0x02, 0x49, 0x08, 0xd8, 0x23, 0x16, 0xc7, - 0xe8, 0x55, 0xa5, 0x28, 0x6b, 0xa7, 0xf3, 0xc8, 0x38, 0xad, 0x53, 0x2d, 0x36, 0xff, 0xb0, 0xa1, - 0x6c, 0xbc, 0xc8, 0x67, 0x50, 0x0a, 0xb4, 0x27, 0x7a, 0xd4, 0x76, 0x6f, 0xb7, 0xe6, 0xd9, 0xe5, - 0xc2, 0x52, 0x63, 0x43, 0x3c, 0xa8, 0xbc, 0xe1, 0x52, 0x25, 0xc1, 0x84, 0x61, 0xb8, 0x2a, 0x9d, - 0xeb, 0x64, 0x0f, 0x2a, 0x53, 0x2e, 0x94, 0x3f, 0x09, 0xa6, 0xae, 0xbd, 0x55, 0xdc, 0xae, 0xed, - 0xde, 0x5d, 0x8d, 0xd5, 0xea, 0x73, 0xa1, 0x0e, 0x83, 0x69, 0x37, 0x51, 0x62, 0x46, 0xd7, 0xa6, - 0x46, 0xd3, 0x51, 0xcf, 0xd8, 0x4c, 0x4e, 0x83, 0x11, 0x73, 0x4b, 0x26, 0x6a, 0xa6, 0x23, 0x0c, - 0x6f, 0x02, 0x11, 0xba, 0x65, 0xbc, 0x30, 0x0a, 0xd9, 0x81, 0xea, 0x19, 0x9b, 0xf9, 0x42, 0x23, - 0xe5, 0xae, 0x61, 0xe2, 0x64, 0xf1, 0xb1, 0x0c, 0x43, 0x0c, 0x63, 0xd0, 0xdc, 0x06, 0x5b, 0xcd, - 0xa6, 0xcc, 0xad, 0x6c, 0x59, 0xdb, 0x8d, 0xdd, 0xcd, 0xd5, 0xc4, 0x86, 0xb3, 0x29, 0xa3, 0x68, - 0x41, 0xb6, 0xc1, 0x09, 0x4f, 0x7c, 0x5d, 0x91, 0xcf, 0x2f, 0x98, 0x10, 0x51, 0xc8, 0xdc, 0x2a, - 0x7e, 0xbb, 0x11, 0x9e, 0xf4, 0x82, 0x09, 0x3b, 0x4a, 0x4f, 0x49, 0x0b, 0x6c, 0x15, 0x8c, 0xa5, - 0x0b, 0x58, 0xac, 0x77, 0xa9, 0xd8, 0x61, 0x30, 0x96, 0xa6, 0x52, 0xb4, 0x23, 0xf7, 0xa0, 0x31, - 0x99, 0xc9, 0xb7, 0xb1, 0x3f, 0x87, 0xb0, 0x8e, 0x71, 0xd7, 0xf1, 0xf4, 0x79, 0x86, 0xe3, 0x5d, + 0x11, 0x15, 0xe1, 0xa0, 0xb4, 0x85, 0xa8, 0x12, 0x52, 0x5d, 0xc7, 0xa5, 0x69, 0x1a, 0xc7, 0x5a, + 0x3b, 0x82, 0xf2, 0xe7, 0x74, 0xf1, 0x6d, 0xd2, 0x53, 0xce, 0xb7, 0xee, 0xee, 0x26, 0x92, 0x79, + 0x05, 0x7e, 0x00, 0x7f, 0x79, 0x03, 0x1e, 0x81, 0x27, 0xe0, 0x39, 0xe0, 0x49, 0xd0, 0xce, 0xde, + 0xd9, 0x67, 0xbb, 0x2d, 0x29, 0xca, 0xbf, 0x99, 0xdd, 0x99, 0xb9, 0x99, 0x6f, 0xe6, 0x9b, 0xb5, + 0xa1, 0xa1, 0xf8, 0x84, 0x87, 0x81, 0x0a, 0x5a, 0x13, 0xc1, 0x15, 0x27, 0x95, 0x4c, 0x6f, 0xee, + 0x42, 0xe5, 0x90, 0x4d, 0x69, 0x90, 0x9c, 0x33, 0xb2, 0x09, 0x25, 0xa9, 0x02, 0xa1, 0x5c, 0x6b, + 0xcb, 0xda, 0xae, 0x53, 0xa3, 0x10, 0x07, 0x8a, 0x2c, 0x09, 0xdd, 0x02, 0x9e, 0x69, 0xb1, 0xf9, + 0x00, 0x6a, 0xc3, 0xe0, 0x34, 0x66, 0xaa, 0x1d, 0x47, 0x81, 0x24, 0x04, 0xec, 0x11, 0x8b, 0x63, + 0xf4, 0xaa, 0x52, 0x94, 0xb5, 0xd3, 0x65, 0x64, 0x9c, 0xd6, 0xa9, 0x16, 0x9b, 0x7f, 0xda, 0x50, + 0x36, 0x5e, 0xe4, 0x0b, 0x28, 0x05, 0xda, 0x13, 0x3d, 0x6a, 0xbb, 0xb7, 0x5b, 0xb3, 0xec, 0x72, + 0x61, 0xa9, 0xb1, 0x21, 0x1e, 0x54, 0x5e, 0x73, 0xa9, 0x92, 0x60, 0xcc, 0x30, 0x5c, 0x95, 0xce, + 0x74, 0xb2, 0x07, 0x95, 0x09, 0x17, 0xca, 0x1f, 0x07, 0x13, 0xd7, 0xde, 0x2a, 0x6e, 0xd7, 0x76, + 0xef, 0x2e, 0xc7, 0x6a, 0xf5, 0xb9, 0x50, 0x47, 0xc1, 0xa4, 0x9b, 0x28, 0x31, 0xa5, 0x6b, 0x13, + 0xa3, 0xe9, 0xa8, 0x17, 0x6c, 0x2a, 0x27, 0xc1, 0x88, 0xb9, 0x25, 0x13, 0x35, 0xd3, 0x11, 0x86, + 0xd7, 0x81, 0x08, 0xdd, 0x32, 0x5e, 0x18, 0x85, 0xec, 0x40, 0xf5, 0x82, 0x4d, 0x7d, 0xa1, 0x91, + 0x72, 0xd7, 0x30, 0x71, 0x32, 0xff, 0x58, 0x86, 0x21, 0x86, 0x31, 0x68, 0x6e, 0x83, 0xad, 0xa6, + 0x13, 0xe6, 0x56, 0xb6, 0xac, 0xed, 0xc6, 0xee, 0xe6, 0x72, 0x62, 0xc3, 0xe9, 0x84, 0x51, 0xb4, + 0x20, 0xdb, 0xe0, 0x84, 0xa7, 0xbe, 0xae, 0xc8, 0xe7, 0x57, 0x4c, 0x88, 0x28, 0x64, 0x6e, 0x15, + 0xbf, 0xdd, 0x08, 0x4f, 0x7b, 0xc1, 0x98, 0x1d, 0xa7, 0xa7, 0xa4, 0x05, 0xb6, 0x0a, 0xce, 0xa5, + 0x0b, 0x58, 0xac, 0xb7, 0x52, 0xec, 0x30, 0x38, 0x97, 0xa6, 0x52, 0xb4, 0x23, 0xf7, 0xa0, 0x31, + 0x9e, 0xca, 0x37, 0xb1, 0x3f, 0x83, 0xb0, 0x8e, 0x71, 0xd7, 0xf1, 0xf4, 0x79, 0x86, 0xe3, 0x5d, 0x00, 0x63, 0xa6, 0xe1, 0x71, 0xd7, 0xb7, 0xac, 0xed, 0x12, 0xad, 0xe2, 0x89, 0x46, 0xcf, 0x7b, - 0x0c, 0xf5, 0x3c, 0x8a, 0xba, 0xb9, 0x67, 0x6c, 0x96, 0xf6, 0x5b, 0x8b, 0x1a, 0xb2, 0x8b, 0x20, - 0x3e, 0x37, 0x1d, 0x2a, 0x51, 0xa3, 0x3c, 0x2e, 0xec, 0x59, 0xde, 0x57, 0x50, 0x9d, 0x27, 0xf5, + 0x0c, 0xf5, 0x3c, 0x8a, 0xba, 0xb9, 0x17, 0x6c, 0x9a, 0xf6, 0x5b, 0x8b, 0x1a, 0xb2, 0xab, 0x20, + 0xbe, 0x34, 0x1d, 0x2a, 0x51, 0xa3, 0x3c, 0x2e, 0xec, 0x59, 0xde, 0x37, 0x50, 0x9d, 0x25, 0xf5, 0x5f, 0x8e, 0xd5, 0x9c, 0xe3, 0x0b, 0xbb, 0x52, 0x74, 0xec, 0x17, 0x76, 0xa5, 0xe6, 0xd4, 0x9b, - 0xbf, 0x96, 0xa1, 0x34, 0xc0, 0x2e, 0xec, 0x41, 0x7d, 0x12, 0x48, 0xc5, 0x84, 0x7f, 0x8d, 0x09, - 0xaa, 0x19, 0x53, 0x33, 0xa5, 0x4b, 0xfd, 0x2b, 0x5c, 0xa3, 0x7f, 0x5f, 0x43, 0x5d, 0x32, 0x71, - 0xc1, 0x42, 0x5f, 0x37, 0x49, 0xba, 0xc5, 0x55, 0xcc, 0x31, 0xa3, 0xd6, 0x00, 0x6d, 0xb0, 0x9b, - 0x35, 0x39, 0x97, 0x25, 0x79, 0x02, 0xeb, 0x92, 0x9f, 0x8b, 0x11, 0xf3, 0x71, 0x7e, 0x64, 0x3a, - 0xa0, 0x1f, 0x5d, 0xf2, 0x47, 0x23, 0x94, 0x69, 0x5d, 0x2e, 0x14, 0x49, 0x9e, 0xc1, 0x86, 0xc2, - 0x6a, 0xfc, 0x11, 0x4f, 0x94, 0xe0, 0xb1, 0x74, 0xcb, 0xab, 0x43, 0x6e, 0x62, 0x98, 0xa2, 0x3b, - 0xc6, 0x8a, 0x36, 0x54, 0x5e, 0x95, 0xe4, 0x3e, 0xdc, 0x8a, 0xa4, 0x9f, 0xc2, 0xa6, 0x53, 0x8c, - 0x92, 0x31, 0x4e, 0x70, 0x85, 0x6e, 0x44, 0xf2, 0x10, 0xcf, 0x07, 0xe6, 0xd8, 0x7b, 0x0d, 0xb0, - 0x28, 0x88, 0x3c, 0x82, 0x5a, 0x9a, 0x01, 0x4e, 0xb2, 0x75, 0xc5, 0x24, 0x83, 0x9a, 0xcb, 0xba, - 0xa9, 0x7a, 0x09, 0x48, 0xb7, 0xb0, 0x55, 0xd4, 0x4d, 0x45, 0xc5, 0xfb, 0xcd, 0x82, 0x5a, 0xae, - 0xd8, 0x6c, 0x45, 0x58, 0xf3, 0x15, 0xb1, 0x44, 0xca, 0xc2, 0xbb, 0x48, 0x59, 0x7c, 0x27, 0x29, - 0xed, 0x6b, 0x34, 0xf5, 0x0e, 0x94, 0x31, 0x51, 0xe9, 0x96, 0x30, 0xb7, 0x54, 0xf3, 0x7e, 0xb7, - 0x60, 0x7d, 0x09, 0xc5, 0x1b, 0xad, 0x9d, 0x7c, 0x0e, 0xe4, 0x24, 0x0e, 0x46, 0x67, 0x71, 0x24, - 0x95, 0x1e, 0x28, 0x93, 0x82, 0x8d, 0x26, 0xb7, 0x72, 0x37, 0x18, 0x54, 0xea, 0x2c, 0x4f, 0x05, - 0xff, 0x91, 0x25, 0xb8, 0x9b, 0x2a, 0x34, 0xd5, 0xe6, 0x9c, 0x28, 0x39, 0xe5, 0xe6, 0x9f, 0x05, - 0xdc, 0xdc, 0x06, 0x9d, 0x2f, 0x60, 0x13, 0x01, 0x89, 0x92, 0xb1, 0x3f, 0xe2, 0xf1, 0xf9, 0x24, - 0xc1, 0x75, 0x92, 0x32, 0x8d, 0x64, 0x77, 0x1d, 0xbc, 0xd2, 0x1b, 0x85, 0xbc, 0xb8, 0xec, 0x81, - 0x75, 0x16, 0xb0, 0x4e, 0x77, 0x09, 0x44, 0xfc, 0xc6, 0x81, 0x99, 0xf1, 0x95, 0x58, 0x58, 0xf3, - 0x93, 0x39, 0x53, 0x4e, 0x05, 0x9f, 0xc8, 0xcb, 0xab, 0x38, 0x8b, 0x91, 0x92, 0xe5, 0x99, 0xe0, - 0x93, 0x8c, 0x2c, 0x5a, 0x96, 0xde, 0x79, 0x36, 0x76, 0x5a, 0xbd, 0x59, 0xe8, 0xf3, 0x43, 0x55, - 0x5c, 0x1e, 0x2a, 0x83, 0x67, 0xf3, 0x27, 0x0b, 0x1c, 0xc3, 0x3f, 0x36, 0x8d, 0xa3, 0x51, 0xa0, - 0x22, 0x9e, 0x90, 0x47, 0x50, 0x4a, 0x78, 0xc8, 0xf4, 0x86, 0xd1, 0xc5, 0x7c, 0xb2, 0x42, 0xb9, + 0xbf, 0x95, 0xa1, 0x34, 0xc0, 0x2e, 0xec, 0x41, 0x7d, 0x1c, 0x48, 0xc5, 0x84, 0x7f, 0x8d, 0x09, + 0xaa, 0x19, 0x53, 0x33, 0xa5, 0x0b, 0xfd, 0x2b, 0x5c, 0xa3, 0x7f, 0xdf, 0x42, 0x5d, 0x32, 0x71, + 0xc5, 0x42, 0x5f, 0x37, 0x49, 0xba, 0xc5, 0x65, 0xcc, 0x31, 0xa3, 0xd6, 0x00, 0x6d, 0xb0, 0x9b, + 0x35, 0x39, 0x93, 0x25, 0x79, 0x02, 0xeb, 0x92, 0x5f, 0x8a, 0x11, 0xf3, 0x71, 0x7e, 0x64, 0x3a, + 0xa0, 0x9f, 0xac, 0xf8, 0xa3, 0x11, 0xca, 0xb4, 0x2e, 0xe7, 0x8a, 0x24, 0xcf, 0x60, 0x43, 0x61, + 0x35, 0xfe, 0x88, 0x27, 0x4a, 0xf0, 0x58, 0xba, 0xe5, 0xe5, 0x21, 0x37, 0x31, 0x4c, 0xd1, 0x1d, + 0x63, 0x45, 0x1b, 0x2a, 0xaf, 0x4a, 0x72, 0x1f, 0x6e, 0x45, 0xd2, 0x4f, 0x61, 0xd3, 0x29, 0x46, + 0xc9, 0x39, 0x4e, 0x70, 0x85, 0x6e, 0x44, 0xf2, 0x08, 0xcf, 0x07, 0xe6, 0xd8, 0x7b, 0x05, 0x30, + 0x2f, 0x88, 0x3c, 0x82, 0x5a, 0x9a, 0x01, 0x4e, 0xb2, 0xf5, 0x9e, 0x49, 0x06, 0x35, 0x93, 0x75, + 0x53, 0xf5, 0x12, 0x90, 0x6e, 0x61, 0xab, 0xa8, 0x9b, 0x8a, 0x8a, 0xf7, 0xbb, 0x05, 0xb5, 0x5c, + 0xb1, 0xd9, 0x8a, 0xb0, 0x66, 0x2b, 0x62, 0x81, 0x94, 0x85, 0x77, 0x91, 0xb2, 0xf8, 0x4e, 0x52, + 0xda, 0xd7, 0x68, 0xea, 0x1d, 0x28, 0x63, 0xa2, 0xd2, 0x2d, 0x61, 0x6e, 0xa9, 0xe6, 0xfd, 0x61, + 0xc1, 0xfa, 0x02, 0x8a, 0x37, 0x5a, 0x3b, 0xf9, 0x12, 0xc8, 0x69, 0x1c, 0x8c, 0x2e, 0xe2, 0x48, + 0x2a, 0x3d, 0x50, 0x26, 0x05, 0x1b, 0x4d, 0x6e, 0xe5, 0x6e, 0x30, 0xa8, 0xd4, 0x59, 0x9e, 0x09, + 0xfe, 0x13, 0x4b, 0x70, 0x37, 0x55, 0x68, 0xaa, 0xcd, 0x38, 0x51, 0x72, 0xca, 0xcd, 0xbf, 0x0a, + 0xb8, 0xb9, 0x0d, 0x3a, 0x5f, 0xc1, 0x26, 0x02, 0x12, 0x25, 0xe7, 0xfe, 0x88, 0xc7, 0x97, 0xe3, + 0x04, 0xd7, 0x49, 0xca, 0x34, 0x92, 0xdd, 0x75, 0xf0, 0x4a, 0x6f, 0x14, 0xf2, 0x62, 0xd5, 0x03, + 0xeb, 0x2c, 0x60, 0x9d, 0xee, 0x02, 0x88, 0xf8, 0x8d, 0x03, 0x33, 0xe3, 0x4b, 0xb1, 0xb0, 0xe6, + 0x27, 0x33, 0xa6, 0x9c, 0x09, 0x3e, 0x96, 0xab, 0xab, 0x38, 0x8b, 0x91, 0x92, 0xe5, 0x99, 0xe0, + 0xe3, 0x8c, 0x2c, 0x5a, 0x96, 0xde, 0x65, 0x36, 0x76, 0x5a, 0xbd, 0x59, 0xe8, 0xf3, 0x43, 0x55, + 0x5c, 0x1c, 0x2a, 0x83, 0x67, 0xf3, 0x67, 0x0b, 0x1c, 0xc3, 0x3f, 0x36, 0x89, 0xa3, 0x51, 0xa0, + 0x22, 0x9e, 0x90, 0x47, 0x50, 0x4a, 0x78, 0xc8, 0xf4, 0x86, 0xd1, 0xc5, 0x7c, 0xb6, 0x44, 0xb9, 0x9c, 0x69, 0xab, 0xc7, 0x43, 0x46, 0x8d, 0xb5, 0xf7, 0x04, 0x6c, 0xad, 0xea, 0x3d, 0x95, 0x96, - 0x70, 0x9d, 0x3d, 0xa5, 0x16, 0x4a, 0xf3, 0x18, 0x1a, 0xe9, 0x17, 0x4e, 0x99, 0x60, 0xc9, 0x88, - 0xe9, 0xf7, 0x35, 0xd7, 0x4c, 0x94, 0xdf, 0x7b, 0x9b, 0x35, 0x7f, 0xb6, 0x80, 0x60, 0xdc, 0xe5, - 0x29, 0xbf, 0x89, 0xd8, 0xe4, 0x21, 0xdc, 0x79, 0x7b, 0xce, 0xc4, 0xcc, 0x2c, 0x97, 0x11, 0xf3, + 0x70, 0x9d, 0x3d, 0xa5, 0xe6, 0x4a, 0xf3, 0x04, 0x1a, 0xe9, 0x17, 0xce, 0x98, 0x60, 0xc9, 0x88, + 0xe9, 0xf7, 0x35, 0xd7, 0x4c, 0x94, 0x3f, 0x78, 0x9b, 0x35, 0x7f, 0xb1, 0x80, 0x60, 0xdc, 0xc5, + 0x29, 0xbf, 0x89, 0xd8, 0xe4, 0x21, 0xdc, 0x79, 0x73, 0xc9, 0xc4, 0xd4, 0x2c, 0x97, 0x11, 0xf3, 0xc3, 0x48, 0xea, 0xaf, 0x18, 0xb2, 0x56, 0xe8, 0x26, 0xde, 0x0e, 0xcc, 0xe5, 0x7e, 0x7a, 0xd7, - 0xfc, 0xdb, 0x86, 0xda, 0x40, 0x5c, 0xcc, 0x67, 0xf8, 0x1b, 0x80, 0x69, 0x20, 0x54, 0xa4, 0x31, - 0xcd, 0x60, 0xff, 0x34, 0x07, 0xfb, 0xc2, 0x74, 0x3e, 0x4f, 0xfd, 0xcc, 0x9e, 0xe6, 0x5c, 0xdf, - 0x49, 0x86, 0xc2, 0x7b, 0x93, 0xa1, 0xf8, 0x3f, 0xc8, 0xd0, 0x86, 0x5a, 0x8e, 0x0c, 0x29, 0x17, - 0xb6, 0xfe, 0xbd, 0x8e, 0x1c, 0x1d, 0x60, 0x41, 0x07, 0xef, 0x2f, 0x0b, 0x6e, 0x5d, 0x2a, 0x51, - 0xb3, 0x22, 0xf7, 0x1e, 0x5d, 0xcd, 0x8a, 0xc5, 0x43, 0x44, 0x3a, 0xe0, 0x60, 0x96, 0xbe, 0xc8, - 0x06, 0xca, 0x10, 0xa4, 0x96, 0xaf, 0x6b, 0x79, 0xe2, 0xe8, 0x86, 0x5c, 0xd2, 0x25, 0xe9, 0xc3, - 0x6d, 0x13, 0x64, 0xf5, 0x41, 0x32, 0x8f, 0xe2, 0xc7, 0x2b, 0x91, 0x96, 0xdf, 0xa3, 0x0f, 0xe5, - 0xa5, 0x33, 0xe9, 0xf9, 0x37, 0xc1, 0xf8, 0x2b, 0x1e, 0x8c, 0x74, 0x4b, 0xfe, 0x00, 0x95, 0x0e, - 0x8b, 0xe3, 0x83, 0xe4, 0x94, 0xeb, 0x1f, 0x43, 0x88, 0x8b, 0xf0, 0x83, 0x30, 0x14, 0x4c, 0xca, - 0x74, 0xea, 0xd7, 0xcd, 0x69, 0xdb, 0x1c, 0x6a, 0x4a, 0x08, 0xce, 0x55, 0x1a, 0x10, 0x65, 0xbd, - 0x90, 0x05, 0x1b, 0x47, 0x3c, 0x49, 0x57, 0x48, 0xaa, 0xdd, 0xdf, 0x85, 0xc6, 0xf2, 0x48, 0x90, - 0x2a, 0x94, 0x8e, 0x7b, 0x83, 0xee, 0xd0, 0xf9, 0x80, 0x00, 0x94, 0x8f, 0x0f, 0x7a, 0xc3, 0x2f, - 0x1f, 0x3a, 0x96, 0x3e, 0x7e, 0xfa, 0x7a, 0xd8, 0x1d, 0x38, 0x85, 0xfb, 0xbf, 0x58, 0x00, 0x8b, - 0x7a, 0x48, 0x0d, 0xd6, 0x8e, 0x7b, 0x2f, 0x7b, 0x47, 0xdf, 0xf6, 0x8c, 0xcb, 0x61, 0x7b, 0x30, - 0xec, 0x52, 0xc7, 0xd2, 0x17, 0xb4, 0xdb, 0x7f, 0x75, 0xd0, 0x69, 0x3b, 0x05, 0x7d, 0x41, 0xf7, - 0x8f, 0x7a, 0xaf, 0x5e, 0x3b, 0x45, 0x8c, 0xd5, 0x1e, 0x76, 0x9e, 0x1b, 0x71, 0xd0, 0x6f, 0xd3, - 0xae, 0x63, 0x13, 0x07, 0xea, 0xdd, 0xef, 0xfa, 0x5d, 0x7a, 0x70, 0xd8, 0xed, 0x0d, 0xdb, 0xaf, - 0x9c, 0x92, 0xf6, 0x79, 0xda, 0xee, 0xbc, 0x3c, 0xee, 0x3b, 0x65, 0x13, 0x6c, 0x30, 0x3c, 0xa2, - 0x5d, 0x67, 0x4d, 0x2b, 0xfb, 0xb4, 0x7d, 0xd0, 0xeb, 0xee, 0x3b, 0x15, 0xaf, 0xe0, 0x58, 0x4f, - 0xf7, 0x60, 0x23, 0xe2, 0xad, 0x8b, 0x48, 0x31, 0x29, 0xcd, 0x3f, 0x84, 0xef, 0xef, 0xa5, 0x5a, - 0xc4, 0x77, 0x8c, 0xb4, 0x33, 0xe6, 0x3b, 0x17, 0x6a, 0x07, 0x6f, 0x77, 0xb2, 0xc6, 0x9c, 0x94, - 0x51, 0x7f, 0xf0, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd2, 0x59, 0xb6, 0xea, 0x61, 0x0c, 0x00, - 0x00, + 0xfc, 0xc7, 0x86, 0xda, 0x40, 0x5c, 0xcd, 0x66, 0xf8, 0x3b, 0x80, 0x49, 0x20, 0x54, 0xa4, 0x31, + 0xcd, 0x60, 0xff, 0x3c, 0x07, 0xfb, 0xdc, 0x74, 0x36, 0x4f, 0xfd, 0xcc, 0x9e, 0xe6, 0x5c, 0xdf, + 0x49, 0x86, 0xc2, 0x07, 0x93, 0xa1, 0xf8, 0x3f, 0xc8, 0xd0, 0x86, 0x5a, 0x8e, 0x0c, 0x29, 0x17, + 0xb6, 0xde, 0x5e, 0x47, 0x8e, 0x0e, 0x30, 0xa7, 0x83, 0xf7, 0xb7, 0x05, 0xb7, 0x56, 0x4a, 0xd4, + 0xac, 0xc8, 0xbd, 0x47, 0xef, 0x67, 0xc5, 0xfc, 0x21, 0x22, 0x1d, 0x70, 0x30, 0x4b, 0x5f, 0x64, + 0x03, 0x65, 0x08, 0x52, 0xcb, 0xd7, 0xb5, 0x38, 0x71, 0x74, 0x43, 0x2e, 0xe8, 0x92, 0xf4, 0xe1, + 0xb6, 0x09, 0xb2, 0xfc, 0x20, 0x99, 0x47, 0xf1, 0xd3, 0xa5, 0x48, 0x8b, 0xef, 0xd1, 0xc7, 0x72, + 0xe5, 0x4c, 0x7a, 0xfe, 0x4d, 0x30, 0xfe, 0x3d, 0x0f, 0x46, 0xba, 0x25, 0x0f, 0xa1, 0xd2, 0x61, + 0x71, 0x7c, 0x90, 0x9c, 0x71, 0xfd, 0x63, 0x08, 0x71, 0x11, 0x7e, 0x10, 0x86, 0x82, 0x49, 0x99, + 0x4e, 0xfd, 0xba, 0x39, 0x6d, 0x9b, 0x43, 0x4d, 0x09, 0xc1, 0xb9, 0x4a, 0x03, 0xa2, 0x9c, 0x2e, + 0x8a, 0x26, 0x80, 0x0e, 0x26, 0xcd, 0x0f, 0x8a, 0xb7, 0xae, 0x9b, 0xfb, 0xbb, 0xd0, 0x58, 0x1c, + 0x12, 0x52, 0x85, 0xd2, 0x49, 0x6f, 0xd0, 0x1d, 0x3a, 0x1f, 0x11, 0x80, 0xf2, 0xc9, 0x41, 0x6f, + 0xf8, 0xf5, 0x43, 0xc7, 0xd2, 0xc7, 0x4f, 0x5f, 0x0d, 0xbb, 0x03, 0xa7, 0x70, 0xff, 0x57, 0x0b, + 0x60, 0x5e, 0x21, 0xa9, 0xc1, 0xda, 0x49, 0xef, 0xb0, 0x77, 0xfc, 0x7d, 0xcf, 0xb8, 0x1c, 0xb5, + 0x07, 0xc3, 0x2e, 0x75, 0x2c, 0x7d, 0x41, 0xbb, 0xfd, 0x97, 0x07, 0x9d, 0xb6, 0x53, 0xd0, 0x17, + 0x74, 0xff, 0xb8, 0xf7, 0xf2, 0x95, 0x53, 0xc4, 0x58, 0xed, 0x61, 0xe7, 0xb9, 0x11, 0x07, 0xfd, + 0x36, 0xed, 0x3a, 0x36, 0x71, 0xa0, 0xde, 0xfd, 0xa1, 0xdf, 0xa5, 0x07, 0x47, 0xdd, 0xde, 0xb0, + 0xfd, 0xd2, 0x29, 0x69, 0x9f, 0xa7, 0xed, 0xce, 0xe1, 0x49, 0xdf, 0x29, 0x9b, 0x60, 0x83, 0xe1, + 0x31, 0xed, 0x3a, 0x6b, 0x5a, 0xd9, 0xa7, 0xed, 0x83, 0x5e, 0x77, 0xdf, 0xa9, 0x78, 0x05, 0xc7, + 0x7a, 0xba, 0x07, 0x1b, 0x11, 0x6f, 0x5d, 0x45, 0x8a, 0x49, 0x69, 0xfe, 0x33, 0xfc, 0x78, 0x2f, + 0xd5, 0x22, 0xbe, 0x63, 0xa4, 0x9d, 0x73, 0xbe, 0x73, 0xa5, 0x76, 0xf0, 0x76, 0x27, 0x6b, 0xd5, + 0x69, 0x19, 0xf5, 0x07, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x89, 0xf4, 0xf5, 0x28, 0x73, 0x0c, + 0x00, 0x00, } diff --git a/go/vt/topo/cells_aliases.go b/go/vt/topo/cells_aliases.go new file mode 100644 index 00000000000..214521acec2 --- /dev/null +++ b/go/vt/topo/cells_aliases.go @@ -0,0 +1,174 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 topo + +import ( + "fmt" + "path" + + "github.com/golang/protobuf/proto" + "golang.org/x/net/context" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// This file provides the utility methods to save / retrieve CellsAliases +// in the topology server. +// +// CellsAliases records are not meant to be changed while the system is +// running. In a running system, a CellsAlias can be added, and +// topology server implementations should be able to read them to +// access the cells upon demand. + +func pathForCellsAlias(alias string) string { + return path.Join(CellsAliasesPath, alias, CellsAliasFile) +} + +// GetCellsAliases returns the names of the existing cells. They are +// sorted by name. +func (ts *Server) GetCellsAliases(ctx context.Context, strongRead bool) (ret map[string]*topodatapb.CellsAlias, err error) { + conn := ts.globalCell + if !strongRead { + conn = ts.globalReadOnlyCell + } + + entries, err := ts.globalCell.ListDir(ctx, CellsAliasesPath, false /*full*/) + switch { + case IsErrType(err, NoNode): + return nil, nil + case err == nil: + aliases := DirEntriesToStringArray(entries) + ret = make(map[string]*topodatapb.CellsAlias, len(aliases)) + for _, alias := range aliases { + aliasPath := pathForCellsAlias(alias) + contents, _, err := conn.Get(ctx, aliasPath) + if err != nil { + return nil, err + } + + // Unpack the contents. + cellsAlias := &topodatapb.CellsAlias{} + if err := proto.Unmarshal(contents, cellsAlias); err != nil { + return nil, err + } + + ret[alias] = cellsAlias + } + return ret, nil + default: + return nil, err + } +} + +// DeleteCellsAlias deletes the specified CellsAlias +func (ts *Server) DeleteCellsAlias(ctx context.Context, alias string) error { + ts.clearCellAliasesCache() + + filePath := pathForCellsAlias(alias) + return ts.globalCell.Delete(ctx, filePath, nil) +} + +// CreateCellsAlias creates a new CellInfo with the provided content. +func (ts *Server) CreateCellsAlias(ctx context.Context, alias string, cellsAlias *topodatapb.CellsAlias) error { + currentAliases, err := ts.GetCellsAliases(ctx, true) + if err != nil { + return err + } + + if overlappingAliases(currentAliases, cellsAlias) { + return fmt.Errorf("unsupported: you can't over overlapping aliases. Cells alias: %v, has an overlap with existent aliases", cellsAlias) + + } + + ts.clearCellAliasesCache() + + // Pack the content. + contents, err := proto.Marshal(cellsAlias) + if err != nil { + return err + } + + // Save it. + filePath := pathForCellsAlias(alias) + _, err = ts.globalCell.Create(ctx, filePath, contents) + return err +} + +// UpdateCellsAlias updates cells for a given alias +func (ts *Server) UpdateCellsAlias(ctx context.Context, alias string, update func(*topodatapb.CellsAlias) error) error { + ts.clearCellAliasesCache() + + filePath := pathForCellsAlias(alias) + for { + ca := &topodatapb.CellsAlias{} + + // Read the file, unpack the contents. + contents, version, err := ts.globalCell.Get(ctx, filePath) + switch { + case err == nil: + if err := proto.Unmarshal(contents, ca); err != nil { + return err + } + case IsErrType(err, NoNode): + // Nothing to do. + default: + return err + } + + // Call update method. + if err = update(ca); err != nil { + if IsErrType(err, NoUpdateNeeded) { + return nil + } + return err + } + + currentAliases, err := ts.GetCellsAliases(ctx, true) + if err != nil { + return err + } + + if overlappingAliases(currentAliases, ca) { + return fmt.Errorf("unsupported: you can't over overlapping aliases. Cells alias: %v, has an overlap with existent aliases", ca) + + } + + // Pack and save. + contents, err = proto.Marshal(ca) + if err != nil { + return err + } + if _, err = ts.globalCell.Update(ctx, filePath, contents, version); !IsErrType(err, BadVersion) { + // This includes the 'err=nil' case. + return err + } + } +} + +func overlappingAliases(currentAliases map[string]*topodatapb.CellsAlias, newAlias *topodatapb.CellsAlias) bool { + for _, cellsAlias := range currentAliases { + for _, cell := range cellsAlias.Cells { + for _, newCell := range newAlias.Cells { + if cell == newCell { + return true + } + } + } + + } + return false +} diff --git a/go/vt/topo/server.go b/go/vt/topo/server.go index cf6b86acc36..5b390e8673b 100644 --- a/go/vt/topo/server.go +++ b/go/vt/topo/server.go @@ -66,6 +66,7 @@ const ( // Filenames for all object types. const ( CellInfoFile = "CellInfo" + CellsAliasFile = "CellsAlias" KeyspaceFile = "Keyspace" ShardFile = "Shard" VSchemaFile = "VSchema" @@ -77,10 +78,11 @@ const ( // Path for all object types. const ( - CellsPath = "cells" - KeyspacesPath = "keyspaces" - ShardsPath = "shards" - TabletsPath = "tablets" + CellsPath = "cells" + CellsAliasesPath = "cells_aliases" + KeyspacesPath = "keyspaces" + ShardsPath = "shards" + TabletsPath = "tablets" ) // Factory is a factory interface to create Conn objects. @@ -134,10 +136,10 @@ type Server struct { cells map[string]Conn } -type cellsToRegionsMap struct { +type cellsToAliasesMap struct { mu sync.Mutex - // cellsToRegions contains all cell->region mappings - cellsToRegions map[string]string + // cellsToAliases contains all cell->alias mappings + cellsToAliases map[string]string } var ( @@ -155,8 +157,8 @@ var ( // factories has the factories for the Conn objects. factories = make(map[string]Factory) - regions = cellsToRegionsMap{ - cellsToRegions: make(map[string]string), + cellsAliases = cellsToAliasesMap{ + cellsToAliases: make(map[string]string), } ) @@ -271,32 +273,34 @@ func (ts *Server) ConnForCell(ctx context.Context, cell string) (Conn, error) { } } -// GetRegionByCell returns the region group this `cell` belongs to, if there's none, it returns the `cell` as region. -func GetRegionByCell(ctx context.Context, ts *Server, cell string) string { - regions.mu.Lock() - defer regions.mu.Unlock() - if region, ok := regions.cellsToRegions[cell]; ok { +// GetAliasByCell returns the alias group this `cell` belongs to, if there's none, it returns the `cell` as alias. +func GetAliasByCell(ctx context.Context, ts *Server, cell string) string { + cellsAliases.mu.Lock() + defer cellsAliases.mu.Unlock() + if region, ok := cellsAliases.cellsToAliases[cell]; ok { return region } if ts != nil { - // lazily get the region from cell info if `regions.ts` is available - info, err := ts.GetCellInfo(ctx, cell, false) - if err == nil && info.Region != "" { - regions.cellsToRegions[cell] = info.Region - return info.Region + // lazily get the region from cell info if `aliases` are available + cellAliases, err := ts.GetCellsAliases(ctx, false) + if err != nil { + // for backward compatibility + return cell + } + + for alias, cellsAlias := range cellAliases { + for _, cellAlias := range cellsAlias.Cells { + if cellAlias == cell { + cellsAliases.cellsToAliases[cell] = alias + return alias + } + } } } - // for backward compatability + // for backward compatibility return cell } -// UpdateCellsToRegionsForTests overwrites the global map built by topo server init, and is meant for testing purpose only. -func UpdateCellsToRegionsForTests(cellsToRegions map[string]string) { - regions.mu.Lock() - defer regions.mu.Unlock() - regions.cellsToRegions = cellsToRegions -} - // Close will close all connections to underlying topo Server. // It will nil all member variables, so any further access will panic. func (ts *Server) Close() { @@ -313,3 +317,9 @@ func (ts *Server) Close() { } ts.cells = make(map[string]Conn) } + +func (ts *Server) clearCellAliasesCache() { + cellsAliases.mu.Lock() + defer cellsAliases.mu.Unlock() + cellsAliases.cellsToAliases = make(map[string]string) +} diff --git a/go/vt/topo/topotests/cells_aliases_test.go b/go/vt/topo/topotests/cells_aliases_test.go new file mode 100644 index 00000000000..110890d2955 --- /dev/null +++ b/go/vt/topo/topotests/cells_aliases_test.go @@ -0,0 +1,151 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 topotests + +import ( + "reflect" + "sort" + "testing" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/vt/topo/memorytopo" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// This file tests the CellsAliases part of the topo.Server API. + +func TestCellsAliases(t *testing.T) { + // Create an alias + + cell := "cell1" + ctx := context.Background() + ts := memorytopo.NewServer(cell) + + if err := ts.CreateCellsAlias(ctx, "alias", &topodatapb.CellsAlias{Cells: []string{"cell1", "cell2"}}); err != nil { + t.Fatalf("CreateCellsAlias failed: %v", err) + } + + if err := ts.CreateCellsAlias(ctx, "aliasb", &topodatapb.CellsAlias{Cells: []string{"cell3", "cell4"}}); err != nil { + t.Fatalf("CreateCellsAlias failed: %v", err) + } + + aliases, err := ts.GetCellsAliases(ctx, true /*strongRead*/) + if err != nil { + t.Fatalf("GetCellsAliases failed: %v", err) + } + + var aliasesName []string + for aliasName := range aliases { + aliasesName = append(aliasesName, aliasName) + } + sort.Strings(aliasesName) + + if len(aliasesName) != 2 { + t.Fatalf("Expected to have 2 aliases. Got %v", len(aliasesName)) + + } + + if aliasesName[0] != "alias" { + t.Fatalf("Expected alias name to be alias, got: %v", aliasesName[0]) + } + + if aliasesName[1] != "aliasb" { + t.Fatalf("Expected alias name to be aliasb, got: %v", aliasesName[0]) + } + + want := []string{"cell1", "cell2"} + + if !reflect.DeepEqual(aliases[aliasesName[0]].Cells, want) { + t.Fatalf("Expected alias to be: %v, got %v", want, aliases[aliasesName[0]]) + } + + want = []string{"cell3", "cell4"} + + if !reflect.DeepEqual(aliases[aliasesName[1]].Cells, want) { + t.Fatalf("Expected aliasb to be: %v, got %v", want, aliases[aliasesName[1]]) + } + + // Test update on non-existing object. + + want = []string{"newcell"} + + if err := ts.UpdateCellsAlias(ctx, "newalias", func(ca *topodatapb.CellsAlias) error { + ca.Cells = want + return nil + }); err != nil { + t.Fatalf("UpdateCellsAlias failed: %v", err) + } + + aliases, err = ts.GetCellsAliases(ctx, true /*strongRead*/) + if err != nil { + t.Fatalf("GetCellsAliases failed: %v", err) + } + + if !reflect.DeepEqual(aliases["newalias"].Cells, want) { + t.Fatalf("Expected newalias to be: %v, got %v", want, aliases["newalias"]) + } + + // Test update on existing object. + + want = []string{"newcell2"} + + if err := ts.UpdateCellsAlias(ctx, "newalias", func(ca *topodatapb.CellsAlias) error { + ca.Cells = want + return nil + }); err != nil { + t.Fatalf("UpdateCellsAlias failed: %v", err) + } + + aliases, err = ts.GetCellsAliases(ctx, true /*strongRead*/) + if err != nil { + t.Fatalf("GetCellsAliases failed: %v", err) + } + + if !reflect.DeepEqual(aliases["newalias"].Cells, want) { + t.Fatalf("Expected newalias to be: %v, got %v", want, aliases["newalias"]) + } + + // Test delete alias + + if err := ts.DeleteCellsAlias(ctx, "newalias"); err != nil { + t.Fatalf("UpdateCellsAlias failed: %v", err) + } + + aliases, err = ts.GetCellsAliases(ctx, true /*strongRead*/) + if err != nil { + t.Fatalf("GetCellsAliases failed: %v", err) + } + + if aliases["newalias"] != nil { + t.Fatalf("Expected newalias to be: nil, got %v", aliases["newalias"]) + } + + // Create an alias that adds an overlapping cell is not supported + if err := ts.CreateCellsAlias(ctx, "invalid", &topodatapb.CellsAlias{Cells: []string{"cell1", "cell2"}}); err == nil { + t.Fatal("CreateCellsAlias should fail, got nil") + } + + // Update an alias that adds an overlapping cell is not supported + if err := ts.UpdateCellsAlias(ctx, "aliasb", func(ca *topodatapb.CellsAlias) error { + ca.Cells = []string{"cell1"} + return nil + }); err == nil { + t.Fatalf("UpdateCellsAlias should fail, got nil") + } +} diff --git a/go/vt/vtctl/cell_info.go b/go/vt/vtctl/cell_info.go index d47bb6cb134..c1917c8934c 100644 --- a/go/vt/vtctl/cell_info.go +++ b/go/vt/vtctl/cell_info.go @@ -39,13 +39,13 @@ func init() { addCommand(cellsGroupName, command{ "AddCellInfo", commandAddCellInfo, - "[-server_address ] [-root ] [-region ] ", + "[-server_address ] [-root ] ", "Registers a local topology service in a new cell by creating the CellInfo with the provided parameters. The address will be used to connect to the topology service, and we'll put Vitess data starting at the provided root."}) addCommand(cellsGroupName, command{ "UpdateCellInfo", commandUpdateCellInfo, - "[-server_address ] [-root ] [-region ] ", + "[-server_address ] [-root ] ", "Updates the content of a CellInfo with the provided parameters. If a value is empty, it is not updated. The CellInfo will be created if it doesn't exist."}) addCommand(cellsGroupName, command{ @@ -70,7 +70,6 @@ func init() { func commandAddCellInfo(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { serverAddress := subFlags.String("server_address", "", "The address the topology server is using for that cell.") root := subFlags.String("root", "", "The root path the topology server is using for that cell.") - region := subFlags.String("region", "", "The region this cell belongs to.") if err := subFlags.Parse(args); err != nil { return err } @@ -82,14 +81,12 @@ func commandAddCellInfo(ctx context.Context, wr *wrangler.Wrangler, subFlags *fl return wr.TopoServer().CreateCellInfo(ctx, cell, &topodatapb.CellInfo{ ServerAddress: *serverAddress, Root: *root, - Region: *region, }) } func commandUpdateCellInfo(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { serverAddress := subFlags.String("server_address", "", "The address the topology server is using for that cell.") root := subFlags.String("root", "", "The root path the topology server is using for that cell.") - region := subFlags.String("region", "", "The region this cell belongs to.") if err := subFlags.Parse(args); err != nil { return err } @@ -100,8 +97,7 @@ func commandUpdateCellInfo(ctx context.Context, wr *wrangler.Wrangler, subFlags return wr.TopoServer().UpdateCellInfoFields(ctx, cell, func(ci *topodatapb.CellInfo) error { if (*serverAddress == "" || ci.ServerAddress == *serverAddress) && - (*root == "" || ci.Root == *root) && - (*region == "" || ci.Region == *region) { + (*root == "" || ci.Root == *root) { return topo.NewError(topo.NoUpdateNeeded, cell) } if *serverAddress != "" { @@ -110,9 +106,6 @@ func commandUpdateCellInfo(ctx context.Context, wr *wrangler.Wrangler, subFlags if *root != "" { ci.Root = *root } - if *region != "" { - ci.Region = *region - } return nil }) } diff --git a/go/vt/vtctl/cells_aliases.go b/go/vt/vtctl/cells_aliases.go new file mode 100644 index 00000000000..1f78835244c --- /dev/null +++ b/go/vt/vtctl/cells_aliases.go @@ -0,0 +1,130 @@ +/* +Copyright 2017 Google Inc. + +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 agreedto 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 vtctl + +import ( + "flag" + "fmt" + "strings" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/vt/wrangler" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// This file contains the CellsAliases command group for vtctl. + +const cellsAliasesGroupName = "CellsAliases" + +func init() { + addCommandGroup(cellsAliasesGroupName) + + addCommand(cellsAliasesGroupName, command{ + "AddCellsAlias", + commandAddCellsAlias, + "[-cells ] ", + "Registers a local topology service in a new cell by creating the CellInfo with the provided parameters. The address will be used to connect to the topology service, and we'll put Vitess data starting at the provided root."}) + + addCommand(cellsAliasesGroupName, command{ + "UpdateCellInfo", + commandUpdateCellsAlias, + "[-cells ] ", + "Updates the content of a CellInfo with the provided parameters. If a value is empty, it is not updated. The CellInfo will be created if it doesn't exist."}) + + addCommand(cellsAliasesGroupName, command{ + "DeleteCellInfo", + commandDeleteCellsAlias, + "", + "Deletes the CellsAlias for the provided alias."}) + + addCommand(cellsAliasesGroupName, command{ + "GetCellsAliases", + commandGetCellsAliases, + "", + "Lists all the cells for which we have a CellInfo object, meaning we have a local topology service registered."}) +} + +func commandAddCellsAlias(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + cellsString := subFlags.String("cells", "", "The address the topology server is using for that cell.") + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 1 { + return fmt.Errorf("the argument is required for the AddCellsAlias command") + } + + cells := strings.Split(*cellsString, ",") + for i, cell := range cells { + cells[i] = strings.TrimSpace(cell) + } + + alias := subFlags.Arg(0) + + return wr.TopoServer().CreateCellsAlias(ctx, alias, &topodatapb.CellsAlias{ + Cells: cells, + }) +} + +func commandUpdateCellsAlias(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + cellsString := subFlags.String("cells", "", "The address the topology server is using for that cell.") + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 1 { + return fmt.Errorf("the argument is required for the UpdateCellsAlias command") + } + + cells := strings.Split(*cellsString, ",") + for i, cell := range cells { + cells[i] = strings.TrimSpace(cell) + } + + alias := subFlags.Arg(0) + + return wr.TopoServer().UpdateCellsAlias(ctx, alias, func(ca *topodatapb.CellsAlias) error { + ca.Cells = cells + return nil + }) +} + +func commandDeleteCellsAlias(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 1 { + return fmt.Errorf("the argument is required for the DeleteCellsAlias command") + } + alias := subFlags.Arg(0) + + return wr.TopoServer().DeleteCellsAlias(ctx, alias) +} + +func commandGetCellsAliases(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 0 { + return fmt.Errorf("GetCellsAliases command takes no parameter") + } + aliases, err := wr.TopoServer().GetCellsAliases(ctx, true /*strongRead*/) + if err != nil { + return err + } + return printJSON(wr.Logger(), aliases) +} diff --git a/go/vt/vtgate/gateway/discoverygateway_test.go b/go/vt/vtgate/gateway/discoverygateway_test.go index e8dd72cdeb7..1acf96f169b 100644 --- a/go/vt/vtgate/gateway/discoverygateway_test.go +++ b/go/vt/vtgate/gateway/discoverygateway_test.go @@ -25,7 +25,9 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/discovery" + "vitess.io/vitess/go/vt/srvtopo/srvtopotest" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vterrors" @@ -129,12 +131,6 @@ func TestDiscoveryGatewayGetTablets(t *testing.T) { } func TestShuffleTablets(t *testing.T) { - defer topo.UpdateCellsToRegionsForTests(map[string]string{}) - topo.UpdateCellsToRegionsForTests(map[string]string{ - "cell1": "region1", - "cell2": "region1", - }) - ts1 := discovery.TabletStats{ Key: "t1", Tablet: topo.NewTablet(10, "cell1", "host1"), @@ -237,13 +233,18 @@ func TestDiscoveryGatewayGetAggregateStatsRegion(t *testing.T) { keyspace := "ks" shard := "0" hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "local-east", 2).(*discoveryGateway) + ts := memorytopo.NewServer("local-west", "local-east", "remote") + srvTopo := srvtopotest.NewPassthroughSrvTopoServer() + srvTopo.TopoServer = ts + dg := createDiscoveryGateway(hc, srvTopo, "local-east", 2).(*discoveryGateway) - topo.UpdateCellsToRegionsForTests(map[string]string{ - "local-west": "local", - "local-east": "local", - "remote": "remote", - }) + cellsAlias := &topodatapb.CellsAlias{ + Cells: []string{"local-west", "local-east"}, + } + + ts.CreateCellsAlias(context.Background(), "local", cellsAlias) + + defer ts.DeleteCellsAlias(context.Background(), "local") hc.Reset() dg.tsc.ResetForTesting() @@ -308,18 +309,55 @@ func TestDiscoveryGatewayGetAggregateStatsMaster(t *testing.T) { } } +func TestDiscoveryGatewayGetTabletsInRegion(t *testing.T) { + keyspace := "ks" + shard := "0" + hc := discovery.NewFakeHealthCheck() + ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") + srvTopo := srvtopotest.NewPassthroughSrvTopoServer() + srvTopo.TopoServer = ts + + cellsAlias := &topodatapb.CellsAlias{ + Cells: []string{"local-west", "local-east"}, + } + + dg := createDiscoveryGateway(hc, srvTopo, "local-west", 2).(*discoveryGateway) + + ts.CreateCellsAlias(context.Background(), "local", cellsAlias) + + defer ts.DeleteCellsAlias(context.Background(), "local") + + // this is a test + // replica should only use local ones + hc.Reset() + dg.tsc.ResetForTesting() + hc.AddTestTablet("remote", "1.1.1.1", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil) + ep1 := hc.AddTestTablet("local-west", "2.2.2.2", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() + ep2 := hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() + tsl := dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_REPLICA) + if len(tsl) != 2 || (!topo.TabletEquality(tsl[0].Tablet, ep1) && !topo.TabletEquality(tsl[0].Tablet, ep2)) { + t.Fatalf("want %+v or %+v, got %+v", ep1, ep2, tsl) + } +} func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { keyspace := "ks" shard := "0" hc := discovery.NewFakeHealthCheck() - dg := createDiscoveryGateway(hc, nil, "local", 2).(*discoveryGateway) - topo.UpdateCellsToRegionsForTests(map[string]string{ - "local-west": "local", - "local-east": "local", - "local": "local", - "remote": "remote", - }) + ts := memorytopo.NewServer("local-west", "local-east", "local", "remote") + srvTopo := srvtopotest.NewPassthroughSrvTopoServer() + srvTopo.TopoServer = ts + + cellsAlias := &topodatapb.CellsAlias{ + Cells: []string{"local-west", "local-east"}, + } + + dg := createDiscoveryGateway(hc, srvTopo, "local", 2).(*discoveryGateway) + + ts.CreateCellsAlias(context.Background(), "local", cellsAlias) + + defer ts.DeleteCellsAlias(context.Background(), "local") + // this is a test // replica should only use local ones hc.Reset() dg.tsc.ResetForTesting() @@ -328,7 +366,7 @@ func TestDiscoveryGatewayGetTabletsWithRegion(t *testing.T) { ep2 := hc.AddTestTablet("local-east", "3.3.3.3", 1001, keyspace, shard, topodatapb.TabletType_REPLICA, true, 10, nil).Tablet() tsl := dg.tsc.GetHealthyTabletStats(keyspace, shard, topodatapb.TabletType_REPLICA) if len(tsl) != 2 || (!topo.TabletEquality(tsl[0].Tablet, ep1) && !topo.TabletEquality(tsl[0].Tablet, ep2)) { - t.Errorf("want %+v or %+v, got %+v", ep1, ep2, tsl) + t.Fatalf("want %+v or %+v, got %+v", ep1, ep2, tsl) } } @@ -351,7 +389,7 @@ func benchmarkCellsGetAggregateStats(i int, b *testing.B) { cellsToregions[cell] = "local" } - topo.UpdateCellsToRegionsForTests(cellsToregions) + //topo.UpdateCellsToRegionsForTests(cellsToregions) hc.Reset() dg.tsc.ResetForTesting() diff --git a/proto/topodata.proto b/proto/topodata.proto index 870c794895a..2677f721911 100644 --- a/proto/topodata.proto +++ b/proto/topodata.proto @@ -347,7 +347,12 @@ message CellInfo { // to server_address. string root = 2; - // Region is a group this cell belongs to. Used by vtgate to route traffic to - // other cells (in same region) when there is no available tablet in the current cell. - string region = 3; + // OBSOLETE: region 3 + reserved 3; +} + +// CellsAlias +message CellsAlias { + // Cells that map to this alias + repeated string cells = 2; } diff --git a/py/vtproto/topodata_pb2.py b/py/vtproto/topodata_pb2.py index 7970c86e338..66ac9fea9b0 100644 --- a/py/vtproto/topodata_pb2.py +++ b/py/vtproto/topodata_pb2.py @@ -20,7 +20,7 @@ package='topodata', syntax='proto3', serialized_options=_b('\n\017io.vitess.protoZ%vitess.io/vitess/go/vt/proto/topodata'), - serialized_pb=_b('\n\x0etopodata.proto\x12\x08topodata\"&\n\x08KeyRange\x12\r\n\x05start\x18\x01 \x01(\x0c\x12\x0b\n\x03\x65nd\x18\x02 \x01(\x0c\"(\n\x0bTabletAlias\x12\x0c\n\x04\x63\x65ll\x18\x01 \x01(\t\x12\x0b\n\x03uid\x18\x02 \x01(\r\"\xb6\x03\n\x06Tablet\x12$\n\x05\x61lias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\x12\x10\n\x08hostname\x18\x02 \x01(\t\x12/\n\x08port_map\x18\x04 \x03(\x0b\x32\x1d.topodata.Tablet.PortMapEntry\x12\x10\n\x08keyspace\x18\x05 \x01(\t\x12\r\n\x05shard\x18\x06 \x01(\t\x12%\n\tkey_range\x18\x07 \x01(\x0b\x32\x12.topodata.KeyRange\x12\"\n\x04type\x18\x08 \x01(\x0e\x32\x14.topodata.TabletType\x12\x18\n\x10\x64\x62_name_override\x18\t \x01(\t\x12(\n\x04tags\x18\n \x03(\x0b\x32\x1a.topodata.Tablet.TagsEntry\x12\x16\n\x0emysql_hostname\x18\x0c \x01(\t\x12\x12\n\nmysql_port\x18\r \x01(\x05\x1a.\n\x0cPortMapEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x05:\x02\x38\x01\x1a+\n\tTagsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01J\x04\x08\x03\x10\x04J\x04\x08\x0b\x10\x0c\"\xd3\x04\n\x05Shard\x12+\n\x0cmaster_alias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x30\n\x0cserved_types\x18\x03 \x03(\x0b\x32\x1a.topodata.Shard.ServedType\x12\x32\n\rsource_shards\x18\x04 \x03(\x0b\x32\x1b.topodata.Shard.SourceShard\x12\x36\n\x0ftablet_controls\x18\x06 \x03(\x0b\x32\x1d.topodata.Shard.TabletControl\x12\x19\n\x11is_master_serving\x18\x07 \x01(\x08\x1a\x46\n\nServedType\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x1ar\n\x0bSourceShard\x12\x0b\n\x03uid\x18\x01 \x01(\r\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\r\n\x05shard\x18\x03 \x01(\t\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x1a{\n\rTabletControl\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x12\x1a\n\x12\x62lacklisted_tables\x18\x04 \x03(\t\x12\x0e\n\x06\x66rozen\x18\x05 \x01(\x08J\x04\x08\x03\x10\x04J\x04\x08\x05\x10\x06\"\xf5\x01\n\x08Keyspace\x12\x1c\n\x14sharding_column_name\x18\x01 \x01(\t\x12\x36\n\x14sharding_column_type\x18\x02 \x01(\x0e\x32\x18.topodata.KeyspaceIdType\x12\x33\n\x0cserved_froms\x18\x04 \x03(\x0b\x32\x1d.topodata.Keyspace.ServedFrom\x1aX\n\nServedFrom\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x12\x10\n\x08keyspace\x18\x03 \x01(\tJ\x04\x08\x03\x10\x04\"w\n\x10ShardReplication\x12.\n\x05nodes\x18\x01 \x03(\x0b\x32\x1f.topodata.ShardReplication.Node\x1a\x33\n\x04Node\x12+\n\x0ctablet_alias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\"E\n\x0eShardReference\x12\x0c\n\x04name\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\"i\n\x12ShardTabletControl\x12\x0c\n\x04name\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x1e\n\x16query_service_disabled\x18\x03 \x01(\x08\"\xda\x03\n\x0bSrvKeyspace\x12;\n\npartitions\x18\x01 \x03(\x0b\x32\'.topodata.SrvKeyspace.KeyspacePartition\x12\x1c\n\x14sharding_column_name\x18\x02 \x01(\t\x12\x36\n\x14sharding_column_type\x18\x03 \x01(\x0e\x32\x18.topodata.KeyspaceIdType\x12\x35\n\x0bserved_from\x18\x04 \x03(\x0b\x32 .topodata.SrvKeyspace.ServedFrom\x1a\xaf\x01\n\x11KeyspacePartition\x12)\n\x0bserved_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\x32\n\x10shard_references\x18\x02 \x03(\x0b\x32\x18.topodata.ShardReference\x12;\n\x15shard_tablet_controls\x18\x03 \x03(\x0b\x32\x1c.topodata.ShardTabletControl\x1aI\n\nServedFrom\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\x10\n\x08keyspace\x18\x02 \x01(\tJ\x04\x08\x05\x10\x06\"@\n\x08\x43\x65llInfo\x12\x16\n\x0eserver_address\x18\x01 \x01(\t\x12\x0c\n\x04root\x18\x02 \x01(\t\x12\x0e\n\x06region\x18\x03 \x01(\t*2\n\x0eKeyspaceIdType\x12\t\n\x05UNSET\x10\x00\x12\n\n\x06UINT64\x10\x01\x12\t\n\x05\x42YTES\x10\x02*\x90\x01\n\nTabletType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\n\n\x06MASTER\x10\x01\x12\x0b\n\x07REPLICA\x10\x02\x12\n\n\x06RDONLY\x10\x03\x12\t\n\x05\x42\x41TCH\x10\x03\x12\t\n\x05SPARE\x10\x04\x12\x10\n\x0c\x45XPERIMENTAL\x10\x05\x12\n\n\x06\x42\x41\x43KUP\x10\x06\x12\x0b\n\x07RESTORE\x10\x07\x12\x0b\n\x07\x44RAINED\x10\x08\x1a\x02\x10\x01\x42\x38\n\x0fio.vitess.protoZ%vitess.io/vitess/go/vt/proto/topodatab\x06proto3') + serialized_pb=_b('\n\x0etopodata.proto\x12\x08topodata\"&\n\x08KeyRange\x12\r\n\x05start\x18\x01 \x01(\x0c\x12\x0b\n\x03\x65nd\x18\x02 \x01(\x0c\"(\n\x0bTabletAlias\x12\x0c\n\x04\x63\x65ll\x18\x01 \x01(\t\x12\x0b\n\x03uid\x18\x02 \x01(\r\"\xb6\x03\n\x06Tablet\x12$\n\x05\x61lias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\x12\x10\n\x08hostname\x18\x02 \x01(\t\x12/\n\x08port_map\x18\x04 \x03(\x0b\x32\x1d.topodata.Tablet.PortMapEntry\x12\x10\n\x08keyspace\x18\x05 \x01(\t\x12\r\n\x05shard\x18\x06 \x01(\t\x12%\n\tkey_range\x18\x07 \x01(\x0b\x32\x12.topodata.KeyRange\x12\"\n\x04type\x18\x08 \x01(\x0e\x32\x14.topodata.TabletType\x12\x18\n\x10\x64\x62_name_override\x18\t \x01(\t\x12(\n\x04tags\x18\n \x03(\x0b\x32\x1a.topodata.Tablet.TagsEntry\x12\x16\n\x0emysql_hostname\x18\x0c \x01(\t\x12\x12\n\nmysql_port\x18\r \x01(\x05\x1a.\n\x0cPortMapEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x05:\x02\x38\x01\x1a+\n\tTagsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01J\x04\x08\x03\x10\x04J\x04\x08\x0b\x10\x0c\"\xd3\x04\n\x05Shard\x12+\n\x0cmaster_alias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x30\n\x0cserved_types\x18\x03 \x03(\x0b\x32\x1a.topodata.Shard.ServedType\x12\x32\n\rsource_shards\x18\x04 \x03(\x0b\x32\x1b.topodata.Shard.SourceShard\x12\x36\n\x0ftablet_controls\x18\x06 \x03(\x0b\x32\x1d.topodata.Shard.TabletControl\x12\x19\n\x11is_master_serving\x18\x07 \x01(\x08\x1a\x46\n\nServedType\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x1ar\n\x0bSourceShard\x12\x0b\n\x03uid\x18\x01 \x01(\r\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\r\n\x05shard\x18\x03 \x01(\t\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x1a{\n\rTabletControl\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x12\x1a\n\x12\x62lacklisted_tables\x18\x04 \x03(\t\x12\x0e\n\x06\x66rozen\x18\x05 \x01(\x08J\x04\x08\x03\x10\x04J\x04\x08\x05\x10\x06\"\xf5\x01\n\x08Keyspace\x12\x1c\n\x14sharding_column_name\x18\x01 \x01(\t\x12\x36\n\x14sharding_column_type\x18\x02 \x01(\x0e\x32\x18.topodata.KeyspaceIdType\x12\x33\n\x0cserved_froms\x18\x04 \x03(\x0b\x32\x1d.topodata.Keyspace.ServedFrom\x1aX\n\nServedFrom\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t\x12\x10\n\x08keyspace\x18\x03 \x01(\tJ\x04\x08\x03\x10\x04\"w\n\x10ShardReplication\x12.\n\x05nodes\x18\x01 \x03(\x0b\x32\x1f.topodata.ShardReplication.Node\x1a\x33\n\x04Node\x12+\n\x0ctablet_alias\x18\x01 \x01(\x0b\x32\x15.topodata.TabletAlias\"E\n\x0eShardReference\x12\x0c\n\x04name\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\"i\n\x12ShardTabletControl\x12\x0c\n\x04name\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x1e\n\x16query_service_disabled\x18\x03 \x01(\x08\"\xda\x03\n\x0bSrvKeyspace\x12;\n\npartitions\x18\x01 \x03(\x0b\x32\'.topodata.SrvKeyspace.KeyspacePartition\x12\x1c\n\x14sharding_column_name\x18\x02 \x01(\t\x12\x36\n\x14sharding_column_type\x18\x03 \x01(\x0e\x32\x18.topodata.KeyspaceIdType\x12\x35\n\x0bserved_from\x18\x04 \x03(\x0b\x32 .topodata.SrvKeyspace.ServedFrom\x1a\xaf\x01\n\x11KeyspacePartition\x12)\n\x0bserved_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\x32\n\x10shard_references\x18\x02 \x03(\x0b\x32\x18.topodata.ShardReference\x12;\n\x15shard_tablet_controls\x18\x03 \x03(\x0b\x32\x1c.topodata.ShardTabletControl\x1aI\n\nServedFrom\x12)\n\x0btablet_type\x18\x01 \x01(\x0e\x32\x14.topodata.TabletType\x12\x10\n\x08keyspace\x18\x02 \x01(\tJ\x04\x08\x05\x10\x06\"6\n\x08\x43\x65llInfo\x12\x16\n\x0eserver_address\x18\x01 \x01(\t\x12\x0c\n\x04root\x18\x02 \x01(\tJ\x04\x08\x03\x10\x04\"\x1b\n\nCellsAlias\x12\r\n\x05\x63\x65lls\x18\x02 \x03(\t*2\n\x0eKeyspaceIdType\x12\t\n\x05UNSET\x10\x00\x12\n\n\x06UINT64\x10\x01\x12\t\n\x05\x42YTES\x10\x02*\x90\x01\n\nTabletType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\n\n\x06MASTER\x10\x01\x12\x0b\n\x07REPLICA\x10\x02\x12\n\n\x06RDONLY\x10\x03\x12\t\n\x05\x42\x41TCH\x10\x03\x12\t\n\x05SPARE\x10\x04\x12\x10\n\x0c\x45XPERIMENTAL\x10\x05\x12\n\n\x06\x42\x41\x43KUP\x10\x06\x12\x0b\n\x07RESTORE\x10\x07\x12\x0b\n\x07\x44RAINED\x10\x08\x1a\x02\x10\x01\x42\x38\n\x0fio.vitess.protoZ%vitess.io/vitess/go/vt/proto/topodatab\x06proto3') ) _KEYSPACEIDTYPE = _descriptor.EnumDescriptor( @@ -44,8 +44,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=2239, - serialized_end=2289, + serialized_start=2258, + serialized_end=2308, ) _sym_db.RegisterEnumDescriptor(_KEYSPACEIDTYPE) @@ -99,8 +99,8 @@ ], containing_type=None, serialized_options=_b('\020\001'), - serialized_start=2292, - serialized_end=2436, + serialized_start=2311, + serialized_end=2455, ) _sym_db.RegisterEnumDescriptor(_TABLETTYPE) @@ -971,10 +971,34 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2173, + serialized_end=2227, +) + + +_CELLSALIAS = _descriptor.Descriptor( + name='CellsAlias', + full_name='topodata.CellsAlias', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ _descriptor.FieldDescriptor( - name='region', full_name='topodata.CellInfo.region', index=2, - number=3, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='cells', full_name='topodata.CellsAlias.cells', index=0, + number=2, type=9, cpp_type=9, label=3, + has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), @@ -990,8 +1014,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=2173, - serialized_end=2237, + serialized_start=2229, + serialized_end=2256, ) _TABLET_PORTMAPENTRY.containing_type = _TABLET @@ -1040,6 +1064,7 @@ DESCRIPTOR.message_types_by_name['ShardTabletControl'] = _SHARDTABLETCONTROL DESCRIPTOR.message_types_by_name['SrvKeyspace'] = _SRVKEYSPACE DESCRIPTOR.message_types_by_name['CellInfo'] = _CELLINFO +DESCRIPTOR.message_types_by_name['CellsAlias'] = _CELLSALIAS DESCRIPTOR.enum_types_by_name['KeyspaceIdType'] = _KEYSPACEIDTYPE DESCRIPTOR.enum_types_by_name['TabletType'] = _TABLETTYPE _sym_db.RegisterFileDescriptor(DESCRIPTOR) @@ -1186,6 +1211,13 @@ )) _sym_db.RegisterMessage(CellInfo) +CellsAlias = _reflection.GeneratedProtocolMessageType('CellsAlias', (_message.Message,), dict( + DESCRIPTOR = _CELLSALIAS, + __module__ = 'topodata_pb2' + # @@protoc_insertion_point(class_scope:topodata.CellsAlias) + )) +_sym_db.RegisterMessage(CellsAlias) + DESCRIPTOR._options = None _TABLETTYPE._options = None From 5021e65b130e64298c5fdeebc106d9629fb19618 Mon Sep 17 00:00:00 2001 From: Rafael Chacon Date: Mon, 1 Apr 2019 17:04:08 -0700 Subject: [PATCH 64/72] Adds tests and fixes typo in vtctl command Signed-off-by: Rafael Chacon --- go/vt/topo/cells_aliases.go | 4 +- go/vt/vtctl/cells_aliases.go | 8 +- test/cell_aliases.py | 308 +++++++++++++++++++++++++++++++++++ test/cell_no_aliases.py | 24 +++ test/config.json | 22 +++ test/utils.py | 10 +- 6 files changed, 368 insertions(+), 8 deletions(-) create mode 100755 test/cell_aliases.py create mode 100755 test/cell_no_aliases.py diff --git a/go/vt/topo/cells_aliases.go b/go/vt/topo/cells_aliases.go index 214521acec2..6488a095aba 100644 --- a/go/vt/topo/cells_aliases.go +++ b/go/vt/topo/cells_aliases.go @@ -1,5 +1,5 @@ /* -Copyright 2017 Google Inc. +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. @@ -7,7 +7,7 @@ You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreedto in writing, software +Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and diff --git a/go/vt/vtctl/cells_aliases.go b/go/vt/vtctl/cells_aliases.go index 1f78835244c..5ef2a78252b 100644 --- a/go/vt/vtctl/cells_aliases.go +++ b/go/vt/vtctl/cells_aliases.go @@ -1,5 +1,5 @@ /* -Copyright 2017 Google Inc. +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. @@ -7,7 +7,7 @@ You may obtain a copy of the License at http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreedto in writing, software +Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and @@ -42,13 +42,13 @@ func init() { "Registers a local topology service in a new cell by creating the CellInfo with the provided parameters. The address will be used to connect to the topology service, and we'll put Vitess data starting at the provided root."}) addCommand(cellsAliasesGroupName, command{ - "UpdateCellInfo", + "UpdateCellsAlias", commandUpdateCellsAlias, "[-cells ] ", "Updates the content of a CellInfo with the provided parameters. If a value is empty, it is not updated. The CellInfo will be created if it doesn't exist."}) addCommand(cellsAliasesGroupName, command{ - "DeleteCellInfo", + "DeleteCellsAlias", commandDeleteCellsAlias, "", "Deletes the CellsAlias for the provided alias."}) diff --git a/test/cell_aliases.py b/test/cell_aliases.py new file mode 100755 index 00000000000..bb910daadb4 --- /dev/null +++ b/test/cell_aliases.py @@ -0,0 +1,308 @@ +#!/usr/bin/env python + +# 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. + +"""This test cell aliases feature + +We start with no aliases and assert that vtgates can't route to replicas/rondly tablets. +Then we add an alias, and these tablets should be routable + +""" + +import threading +import time + +import logging +import unittest + +import base_sharding +import environment +import tablet +import utils + +from vtproto import topodata_pb2 +from vtdb import keyrange_constants +from vtdb import vtgate_client + +use_alias = False + +# initial shards +# range '' - 80 +shard_0_master = tablet.Tablet() +shard_0_replica = tablet.Tablet(cell='ny') +shard_0_rdonly = tablet.Tablet(cell='ny') + +#shard_0_replica = tablet.Tablet() +#shard_0_rdonly = tablet.Tablet() +# range 80 - '' +shard_1_master = tablet.Tablet() +#shard_1_replica = tablet.Tablet() +#shard_1_rdonly = tablet.Tablet() + +shard_1_replica = tablet.Tablet(cell='ny') +shard_1_rdonly = tablet.Tablet(cell='ny') + +all_tablets = ([shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica,shard_1_rdonly]) + +vschema = { + 'test_keyspace': '''{ + "sharded": true, + "vindexes": { + "hash_index": { + "type": "hash" + } + }, + "tables": { + "test_table": { + "column_vindexes": [ + { + "column": "custom_ksid_col", + "name": "hash_index" + } + ] + } + } + }''', +} + +def setUpModule(): + try: + environment.topo_server().setup() + setup_procs = [t.init_mysql() + for t in all_tablets] + utils.Vtctld().start() + utils.wait_procs(setup_procs) + except: + tearDownModule() + raise + +def tearDownModule(): + utils.required_teardown() + if utils.options.skip_teardown: + return + + teardown_procs = [t.teardown_mysql() for t in all_tablets] + utils.wait_procs(teardown_procs, raise_on_error=False) + environment.topo_server().teardown() + utils.kill_sub_processes() + utils.remove_tmp_files() + for t in all_tablets: + t.remove_tree() + +class TestCellsAliases(unittest.TestCase, base_sharding.BaseShardingTest): + + int_type = 265 + + # Gets a vtgate connection + def _get_connection(self, timeout=10.0): + protocol, endpoint = utils.vtgate.rpc_endpoint(python=True) + try: + return vtgate_client.connect(protocol, endpoint, timeout) + except Exception: + logging.exception('Connection to vtgate (timeout=%s) failed.', timeout) + raise + + # executes requetest in tablet type + def _execute_on_tablet_type(self, vtgate_conn, tablet_type, sql, bind_vars): + return vtgate_conn._execute( + sql, bind_vars, tablet_type=tablet_type, keyspace_name=None) + + + # create_schema will create the same schema on the keyspace + # then insert some values + def _create_schema(self): + if base_sharding.keyspace_id_type == keyrange_constants.KIT_BYTES: + t = 'varbinary(64)' + else: + t = 'bigint(20) unsigned' + # Note that the primary key columns are not defined first on purpose to test + # that a reordered column list is correctly used everywhere in vtworker. + create_table_template = '''create table %s( +custom_ksid_col ''' + t + ''' not null, +msg varchar(64), +id bigint not null, +parent_id bigint not null, +primary key (parent_id, id), +index by_msg (msg) +) Engine=InnoDB''' + + utils.run_vtctl(['ApplySchema', + '-sql=' + create_table_template % ('test_table'), + 'test_keyspace'], + auto_log=True) + + def _insert_startup_values(self): + self._insert_value(shard_0_master, 'test_table', 1, 'msg1', + 0x1000000000000000) + self._insert_value(shard_1_master, 'test_table', 2, 'msg2', + 0x9000000000000000) + self._insert_value(shard_1_master, 'test_table', 3, 'msg3', + 0xD000000000000000) + + def test_cells_aliases(self): + utils.run_vtctl(['CreateKeyspace', + '--sharding_column_name', 'custom_ksid_col', + '--sharding_column_type', base_sharding.keyspace_id_type, + 'test_keyspace']) + + shard_0_master.init_tablet('replica', 'test_keyspace', '-80') + shard_0_replica.init_tablet('replica', 'test_keyspace', '-80') + shard_0_rdonly.init_tablet('rdonly', 'test_keyspace', '-80') + shard_1_master.init_tablet('replica', 'test_keyspace', '80-') + shard_1_replica.init_tablet('replica', 'test_keyspace', '80-') + shard_1_rdonly.init_tablet('rdonly', 'test_keyspace', '80-') + + utils.run_vtctl(['RebuildKeyspaceGraph', 'test_keyspace'], auto_log=True) + ks = utils.run_vtctl_json(['GetSrvKeyspace', 'test_nj', 'test_keyspace']) + self.assertEqual(ks['sharding_column_name'], 'custom_ksid_col') + + # we set full_mycnf_args to True as a test in the KIT_BYTES case + full_mycnf_args = (base_sharding.keyspace_id_type == + keyrange_constants.KIT_BYTES) + + # create databases so vttablet can start behaving somewhat normally + for t in [shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly]: + t.create_db('vt_test_keyspace') + t.start_vttablet(wait_for_state=None, full_mycnf_args=full_mycnf_args, + binlog_use_v3_resharding_mode=False) + + # wait for the tablets (replication is not setup, they won't be healthy) + for t in [shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly]: + t.wait_for_vttablet_state('NOT_SERVING') + + # reparent to make the tablets work + utils.run_vtctl(['InitShardMaster', '-force', 'test_keyspace/-80', + shard_0_master.tablet_alias], auto_log=True) + utils.run_vtctl(['InitShardMaster', '-force', 'test_keyspace/80-', + shard_1_master.tablet_alias], auto_log=True) + + # check the shards + shards = utils.run_vtctl_json(['FindAllShardsInKeyspace', 'test_keyspace']) + self.assertIn('-80', shards, 'unexpected shards: %s' % str(shards)) + self.assertIn('80-', shards, 'unexpected shards: %s' % str(shards)) + self.assertEqual(len(shards), 2, 'unexpected shards: %s' % str(shards)) + + # create the tables + self._create_schema() + self._insert_startup_values() + + + utils.run_vtctl(['RebuildKeyspaceGraph', 'test_keyspace'], auto_log=True) + + # Make sure srv keyspace graph looks as expected + utils.check_srv_keyspace( + 'test_nj', 'test_keyspace', + 'Partitions(master): -80 80-\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -80 80-\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + utils.check_srv_keyspace( + 'test_ny', 'test_keyspace', + 'Partitions(master): -80 80-\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -80 80-\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # Bootstrap vtgate + + utils.apply_vschema(vschema) + + # Adds alias so vtgate can route to replica/rdonly tablets that are not in the same cell, but same alias + + if use_alias: + utils.run_vtctl(['AddCellsAlias', '-cells', 'test_nj,test_ny','region_east_coast'], auto_log=True) + tablet_types_to_wait='MASTER,REPLICA' + else: + tablet_types_to_wait='MASTER' + + utils.VtGate().start( + tablets=[shard_0_master, shard_1_master], + tablet_types_to_wait=tablet_types_to_wait, + cells_to_watch='test_nj,test_ny', + ) + utils.vtgate.wait_for_endpoints('test_keyspace.-80.master', 1) + utils.vtgate.wait_for_endpoints('test_keyspace.80-.master', 1) + + vtgate_conn = self._get_connection() + result = self._execute_on_tablet_type( + vtgate_conn, + 'master', + 'select count(*) from test_table', {}) + self.assertEqual( + result, + ([(3,)], 1, 0, + [('count(*)', self.int_type)])) + + if use_alias: + vtgate_conn = self._get_connection() + result = self._execute_on_tablet_type( + vtgate_conn, + 'master', + 'select count(*) from test_table', {}) + self.assertEqual( + result, + ([(3,)], 1, 0, + [('count(*)', self.int_type)])) + + vtgate_conn = self._get_connection() + result = self._execute_on_tablet_type( + vtgate_conn, + 'replica', + 'select count(*) from test_table', {}) + self.assertEqual( + result, + ([(3,)], 1, 0, + [('count(*)', self.int_type)])) + + vtgate_conn = self._get_connection() + result = self._execute_on_tablet_type( + vtgate_conn, + 'rdonly', + 'select count(*) from test_table', {}) + self.assertEqual( + result, + ([(3,)], 1, 0, + [('count(*)', self.int_type)])) + else: + vtgate_conn = self._get_connection() + try: + self._execute_on_tablet_type( + vtgate_conn, + 'replica', + 'select count(*) from test_table', {}) + self.fail('Expected execute to fail, did not get error') + except Exception as e: + s = str(e) + self.assertIn('80.replica, no valid tablet: node', s) + + vtgate_conn = self._get_connection() + try: + self._execute_on_tablet_type( + vtgate_conn, + 'rdonly', + 'select count(*) from test_table', {}) + self.fail('Expected execute to fail, did not get error') + except Exception as e: + s = str(e) + self.assertIn('80.rdonly, no valid tablet: node', s) + +if __name__ == '__main__': + utils.main() diff --git a/test/cell_no_aliases.py b/test/cell_no_aliases.py new file mode 100755 index 00000000000..5344cfa0a7d --- /dev/null +++ b/test/cell_no_aliases.py @@ -0,0 +1,24 @@ +#!/usr/bin/env python +# +# Copyright 2017 Google Inc. +# +# 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. + +"""Re-runs cell_aliases.py with no aliases.""" + +import cell_aliases +import utils + +if __name__ == '__main__': + cell_aliases.use_aliases = False + utils.main(cell_aliases) diff --git a/test/config.json b/test/config.json index d629184ec8e..87616c1491f 100644 --- a/test/config.json +++ b/test/config.json @@ -170,6 +170,28 @@ "RetryMax": 0, "Tags": [] }, + "cell_no_aliases": { + "File": "cell_no_aliases.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 1, + "RetryMax": 0, + "Tags": [ + "site_test" + ] + }, + "cell_aliases": { + "File": "cell_aliases.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 1, + "RetryMax": 0, + "Tags": [ + "site_test" + ] + }, "mysql_server": { "File": "mysql_server_test.py", "Args": [], diff --git a/test/utils.py b/test/utils.py index e134ee4a414..e73dccfdfd2 100644 --- a/test/utils.py +++ b/test/utils.py @@ -552,7 +552,8 @@ def start(self, cell='test_nj', retry_count=2, topo_impl=None, cache_ttl='1s', extra_args=None, tablets=None, tablet_types_to_wait='MASTER,REPLICA', - l2vtgates=None): + l2vtgates=None, + cells_to_watch=None): """Start vtgate. Saves it into the global vtgate variable if not set yet.""" args = environment.binary_args('vtgate') + [ @@ -567,7 +568,12 @@ def start(self, cell='test_nj', retry_count=2, '-normalize_queries', '-gateway_implementation', vtgate_gateway_flavor().flavor(), ] - args.extend(vtgate_gateway_flavor().flags(cell=cell, tablets=tablets)) + + if cells_to_watch: + args.extend(vtgate_gateway_flavor().flags(cell=cells_to_watch, tablets=tablets)) + else: + args.extend(vtgate_gateway_flavor().flags(cell=cell, tablets=tablets)) + if l2vtgates: args.extend(['-l2vtgate_addrs', ','.join(l2vtgates)]) if tablet_types_to_wait: From 3489b15f56de716d852c517d285bb61b6244d5a5 Mon Sep 17 00:00:00 2001 From: Richard Date: Fri, 26 Apr 2019 10:36:35 -0700 Subject: [PATCH 65/72] Merge upstream 2019-04-25 (#130) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It's been busy and exciting in vitess upstream. Some changes that felt worth calling out as they could impact assumptions about behavior: - #4832 — this adds a timeout that is 3x the `healthCheckInterval` which at which point the tablet will stop reporting as healthy - #4814 — this changes when command line args are used by vttablet - #4811 — a planned reparent now sets `super_read_only` - #4805 — behavior coming back from a failed backup is different **Slack changes** @demmer #4827 Correct suppress logging for begin...commit in autocommit @rafael #4824 adds timeouts for all statements @demmer #4826 adds logging of stack traces as opt-in @demmer #4819 remove begin/commit logs from autocommit txns @demmer #4796 improves support for vtexplain for begin/dml/dml/commit txns **Non slack changes** #4839 Improve behavior with reference table routing & vreplication #4833 Support query routing given there could be multiple targets for a table #4832 tablets get new health check behavior (health checks time out) #4837 refresh bug around row streamer handling creds #4830 apply a default value to the db_name attribute in local_metadata #4785 SHOW SCHEMAS aliased to SHOW DATABASES #4829 add experimental support for split clone & vertical split clone via vreplication #4822 do not normalize in order by #4791 Adds vreplication row streaming support #4814 command line flags now used regardless of management mode #4811 vttablet sets super_read_only during planned reparent #4803 xtrabackup testing #4727 support multiple vttablets running against one mysql instance #4746 TopoCat can now produce JSON output #4805 mysqld is now restarted after a failed backup #4685 macos bootstrap bug #4874 ZK opts and java land #4695 Adds support for xtrabackup #4794 fixes build failures #4725 changes mysql8.0 start args #4736 introduce new states to support vreplication #4788 expands orc error **Docs etc** #4831 lint #4827 docs #4816 const declaration cleanup #4820 const declaration cleanup #4825 docs #4818 docs #4809 docs #4812 moves consts around #4813 docs #4808 docs #4800 docs #4795 docs --- Makefile | 10 +- bootstrap.sh | 32 +- config/init_db.sql | 21 +- config/mycnf/default-fast.cnf | 2 +- config/mycnf/master_mysql80.cnf | 38 + doc/ServerConfiguration.md | 2 +- doc/TopologyService.md | 4 +- doc/VTTabletModes.md | 2 +- doc/VitessApi.md | 10 +- doc/VitessSequences.md | 2 +- doc/VitessTransportSecurityModel.md | 2 +- doc/meetups_notes/06-14-2018.md | 4 +- doc/vtctlReference.md | 2 +- docker/bootstrap/Dockerfile.mariadb | 10 +- docker/bootstrap/Dockerfile.mysql56 | 9 +- docker/bootstrap/Dockerfile.mysql57 | 9 +- docker/bootstrap/Dockerfile.mysql80 | 13 +- docker/bootstrap/Dockerfile.percona80 | 10 +- docker/k8s/Dockerfile | 1 + docker/lite/Dockerfile | 3 +- docker/lite/Dockerfile.mariadb | 2 +- docker/lite/Dockerfile.mariadb103 | 2 +- docker/lite/Dockerfile.mysql56 | 3 +- docker/lite/Dockerfile.mysql57 | 3 +- docker/lite/Dockerfile.mysql80 | 3 +- docker/lite/Dockerfile.percona | 2 +- docker/lite/Dockerfile.percona57 | 2 +- docker/lite/Dockerfile.percona80 | 2 +- examples/helm/kmysql.sh | 2 +- examples/local/vttablet-up.sh | 2 +- go/cacheservice/cacheservice.go | 6 +- go/cmd/automation_client/automation_client.go | 2 +- go/cmd/mysqlctl/mysqlctl.go | 6 +- go/cmd/topo2topo/topo2topo.go | 5 +- go/cmd/vtctld/schema.go | 2 +- go/cmd/vtexplain/vtexplain.go | 3 +- go/cmd/vtgateclienttest/services/services.go | 1 + go/cmd/zk/zkcmd.go | 2 +- go/memcache/memcache.go | 6 +- go/mysql/flavor.go | 15 +- go/vt/binlog/binlogplayer/binlog_player.go | 103 +- .../binlog/binlogplayer/binlog_player_test.go | 37 +- go/vt/binlog/binlogplayer/fake_dbclient.go | 1 - go/vt/binlog/binlogplayer/mock_dbclient.go | 1 - go/vt/binlog/binlogplayertest/player.go | 2 +- go/vt/dbconfigs/dbconfigs.go | 30 +- go/vt/discovery/healthcheck.go | 6 +- go/vt/key/key.go | 2 +- go/vt/mysqlctl/backup.go | 37 +- go/vt/mysqlctl/backupengine.go | 12 +- go/vt/mysqlctl/builtinbackupengine.go | 88 +- go/vt/mysqlctl/cephbackupstorage/ceph.go | 2 +- .../fakemysqldaemon/fakemysqldaemon.go | 10 + go/vt/mysqlctl/grpcmysqlctlclient/client.go | 12 +- go/vt/mysqlctl/metadata_tables.go | 54 +- go/vt/mysqlctl/mysql_daemon.go | 1 + go/vt/mysqlctl/mysqld.go | 5 + go/vt/mysqlctl/query.go | 6 +- go/vt/mysqlctl/replication.go | 16 +- go/vt/mysqlctl/s3backupstorage/s3.go | 6 +- go/vt/mysqlctl/xtrabackupengine.go | 494 ++ go/vt/proto/binlogdata/binlogdata.pb.go | 349 +- go/vt/proto/queryservice/queryservice.pb.go | 140 +- go/vt/proto/vschema/vschema.pb.go | 198 +- go/vt/proto/vtgateservice/vtgateservice.pb.go | 12 +- go/vt/schemamanager/schemaswap/schema_swap.go | 41 +- go/vt/sqlparser/ast.go | 6 +- go/vt/sqlparser/normalizer.go | 3 + go/vt/sqlparser/normalizer_test.go | 10 + go/vt/sqlparser/parse_test.go | 3 + go/vt/sqlparser/parsed_query.go | 16 +- go/vt/sqlparser/sql.go | 4851 +++++++++-------- go/vt/sqlparser/sql.y | 7 +- go/vt/sqlparser/token.go | 2 +- go/vt/topo/etcd2topo/election.go | 14 +- go/vt/topo/etcd2topo/server_test.go | 2 +- go/vt/topo/helpers/compare.go | 17 + go/vt/topo/helpers/compare_test.go | 12 + go/vt/topo/helpers/copy.go | 11 + go/vt/topo/helpers/copy_test.go | 15 +- go/vt/topo/keyspace.go | 14 + go/vt/topo/locks.go | 2 +- go/vt/topo/memorytopo/election.go | 14 +- go/vt/topo/memorytopo/file.go | 1 + go/vt/topo/server.go | 1 + go/vt/topo/srv_keyspace.go | 2 +- go/vt/topo/test/file.go | 4 +- go/vt/topo/test/lock.go | 4 +- go/vt/topo/test/testing.go | 5 + go/vt/topo/test/vschema.go | 34 +- go/vt/topo/test/watch.go | 6 +- go/vt/topo/topoproto/shard.go | 2 +- go/vt/topo/topotests/srv_keyspace_test.go | 154 +- go/vt/topo/vschema.go | 33 + go/vt/topo/zk2topo/file.go | 2 +- go/vt/topotools/rebuild_vschema.go | 6 + go/vt/topotools/rebuild_vschema_test.go | 39 +- go/vt/vtaclcheck/vtaclcheck.go | 2 +- go/vt/vtcombo/tablet_map.go | 6 + go/vt/vtctl/cells_aliases.go | 6 +- .../fake_loggerevent_streamingclient.go | 2 +- go/vt/vtctl/throttler.go | 8 +- go/vt/vtctl/topo.go | 172 +- go/vt/vtctl/vtctl.go | 100 + go/vt/vtctl/vtctlclient/wrapper.go | 8 +- go/vt/vtctl/vtctlclienttest/client.go | 4 +- go/vt/vtctld/explorer.go | 2 +- go/vt/vtctld/tablet_data_test.go | 12 +- go/vt/vtctld/tablet_stats_cache.go | 24 +- go/vt/vterrors/errors_test.go | 44 +- go/vt/vterrors/vterrors.go | 17 +- .../multi-output/updatesharded-output.txt | 73 + .../testdata/updatesharded-queries.sql | 22 +- go/vt/vtexplain/vtexplain.go | 13 +- go/vt/vtexplain/vtexplain_vttablet.go | 12 +- go/vt/vtgate/buffer/flags_test.go | 2 +- go/vt/vtgate/buffer/timeout_thread.go | 2 +- go/vt/vtgate/buffer/variables.go | 11 +- go/vt/vtgate/engine/delete.go | 11 + go/vt/vtgate/engine/insert.go | 15 +- go/vt/vtgate/engine/limit_test.go | 10 +- go/vt/vtgate/engine/primitive.go | 15 +- go/vt/vtgate/engine/route.go | 47 +- go/vt/vtgate/engine/route_test.go | 70 +- go/vt/vtgate/engine/update.go | 11 + go/vt/vtgate/executor.go | 2 +- go/vt/vtgate/executor_framework_test.go | 8 +- go/vt/vtgate/executor_test.go | 4 +- go/vt/vtgate/fakerpcvtgateconn/conn.go | 6 +- go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go | 2 +- go/vt/vtgate/grpcvtgateservice/server.go | 3 +- go/vt/vtgate/planbuilder/builder.go | 11 +- go/vt/vtgate/planbuilder/delete.go | 41 +- go/vt/vtgate/planbuilder/expr.go | 21 +- go/vt/vtgate/planbuilder/from.go | 262 +- go/vt/vtgate/planbuilder/insert.go | 28 +- go/vt/vtgate/planbuilder/join.go | 13 +- go/vt/vtgate/planbuilder/ordered_aggregate.go | 82 +- go/vt/vtgate/planbuilder/plan_test.go | 11 +- go/vt/vtgate/planbuilder/postprocess.go | 2 +- go/vt/vtgate/planbuilder/route.go | 359 +- go/vt/vtgate/planbuilder/route_option.go | 351 ++ go/vt/vtgate/planbuilder/route_option_test.go | 186 + go/vt/vtgate/planbuilder/select.go | 52 +- go/vt/vtgate/planbuilder/symtab.go | 141 +- go/vt/vtgate/planbuilder/symtab_test.go | 134 +- .../planbuilder/testdata/aggr_cases.txt | 45 + .../vtgate/planbuilder/testdata/dml_cases.txt | 174 +- .../planbuilder/testdata/filter_cases.txt | 86 + .../planbuilder/testdata/from_cases.txt | 310 ++ .../testdata/postprocess_cases.txt | 53 + .../planbuilder/testdata/schema_test.json | 15 + .../planbuilder/testdata/select_cases.txt | 52 +- .../testdata/unsupported_cases.txt | 47 +- go/vt/vtgate/planbuilder/union.go | 27 +- go/vt/vtgate/planbuilder/update.go | 33 +- go/vt/vtgate/vcursor_impl.go | 8 +- go/vt/vtgate/vindexes/binary.go | 2 +- go/vt/vtgate/vindexes/binary_test.go | 2 +- go/vt/vtgate/vindexes/binarymd5.go | 2 +- go/vt/vtgate/vindexes/hash.go | 2 +- go/vt/vtgate/vindexes/lookup_hash_test.go | 2 +- .../vindexes/lookup_hash_unique_test.go | 4 +- go/vt/vtgate/vindexes/lookup_test.go | 6 +- .../lookup_unicodeloosemd5_hash_test.go | 14 +- go/vt/vtgate/vindexes/lookup_unique_test.go | 4 +- go/vt/vtgate/vindexes/null.go | 2 +- go/vt/vtgate/vindexes/numeric.go | 2 +- go/vt/vtgate/vindexes/numeric_static_map.go | 2 +- go/vt/vtgate/vindexes/reverse_bits.go | 2 +- go/vt/vtgate/vindexes/unicodeloosemd5.go | 6 +- go/vt/vtgate/vindexes/unicodeloosemd5_test.go | 10 +- go/vt/vtgate/vindexes/vschema.go | 309 +- go/vt/vtgate/vindexes/vschema_test.go | 282 +- go/vt/vtgate/vtgate.go | 1 - go/vt/vtgate/vtgate_test.go | 14 +- go/vt/vtgate/vtgateconn/vtgateconn_test.go | 4 +- go/vt/vtgate/vtgateconntest/client.go | 16 +- go/vt/vttablet/endtoend/misc_test.go | 25 +- go/vt/vttablet/grpcqueryservice/server.go | 49 +- go/vt/vttablet/grpctabletconn/conn.go | 41 + go/vt/vttablet/queryservice/queryservice.go | 3 + go/vt/vttablet/queryservice/wrapped.go | 11 +- go/vt/vttablet/sandboxconn/sandboxconn.go | 11 +- .../tabletconntest/fakequeryservice.go | 15 +- go/vt/vttablet/tabletmanager/action_agent.go | 21 +- go/vt/vttablet/tabletmanager/init_tablet.go | 5 +- .../tabletmanager/init_tablet_test.go | 4 +- go/vt/vttablet/tabletmanager/orchestrator.go | 4 +- go/vt/vttablet/tabletmanager/rpc_backup.go | 54 +- .../vttablet/tabletmanager/rpc_replication.go | 26 +- go/vt/vttablet/tabletmanager/state_change.go | 17 +- .../tabletmanager/vreplication/controller.go | 7 +- .../vreplication/controller_test.go | 16 +- .../tabletmanager/vreplication/engine.go | 40 +- .../tabletmanager/vreplication/engine_test.go | 41 +- .../vreplication/framework_test.go | 93 +- .../tabletmanager/vreplication/player_plan.go | 125 - .../vreplication/replicator_plan.go | 267 + ...r_plan_test.go => replicator_plan_test.go} | 246 +- .../vreplication/table_plan_builder.go | 362 +- .../tabletmanager/vreplication/vcopier.go | 303 + .../vreplication/vcopier_test.go | 405 ++ .../tabletmanager/vreplication/vplayer.go | 372 +- .../vreplication/vplayer_test.go | 177 +- .../tabletmanager/vreplication/vreplicator.go | 187 + .../vttablet/tabletserver/connpool/dbconn.go | 2 +- .../tabletserver/connpool/dbconn_test.go | 30 +- go/vt/vttablet/tabletserver/controller.go | 2 +- .../messager/message_manager_test.go | 4 +- .../vttablet/tabletserver/planbuilder/plan.go | 3 +- go/vt/vttablet/tabletserver/query_executor.go | 15 +- go/vt/vttablet/tabletserver/rules/map_test.go | 11 +- go/vt/vttablet/tabletserver/rules/rules.go | 4 +- .../vttablet/tabletserver/rules/rules_test.go | 8 +- go/vt/vttablet/tabletserver/schema/engine.go | 2 +- .../tabletserver/schema/load_table.go | 6 +- .../tabletserver/splitquery/split_params.go | 2 +- .../tabletserver/splitquery/splitter_test.go | 2 +- .../tabletserver/tabletenv/logstats.go | 2 +- go/vt/vttablet/tabletserver/tabletserver.go | 145 +- go/vt/vttablet/tabletserver/tx_engine.go | 14 +- go/vt/vttablet/tabletserver/tx_engine_test.go | 15 +- go/vt/vttablet/tabletserver/tx_executor.go | 31 +- go/vt/vttablet/tabletserver/tx_pool.go | 51 +- go/vt/vttablet/tabletserver/tx_pool_test.go | 108 +- .../tabletserver/txthrottler/tx_throttler.go | 4 +- .../vttablet/tabletserver/vstreamer/engine.go | 61 +- .../tabletserver/vstreamer/planbuilder.go | 219 +- .../vstreamer/planbuilder_test.go | 39 +- .../tabletserver/vstreamer/rowstreamer.go | 297 + .../vstreamer/rowstreamer_test.go | 237 + .../tabletserver/vstreamer/vstreamer.go | 50 +- .../tabletserver/vstreamer/vstreamer_test.go | 14 +- go/vt/vttablet/tabletservermock/controller.go | 2 +- go/vt/vttest/environment.go | 3 + go/vt/worker/legacy_split_clone.go | 7 +- go/vt/worker/split_clone.go | 6 +- go/vt/worker/split_diff.go | 2 +- .../vtworkerclienttest/client_testsuite.go | 2 +- go/vt/workflow/resharding/workflow.go | 24 +- .../reshardingworkflowgen/workflow.go | 2 +- ...orkflow_test.go => workflow_flaky_test.go} | 0 go/vt/wrangler/cleaner.go | 53 +- go/vt/wrangler/keyspace.go | 121 +- go/vt/wrangler/schema.go | 12 +- go/vt/wrangler/testlib/backup_test.go | 4 + .../testlib/copy_schema_shard_test.go | 2 +- go/vt/wrangler/testlib/fake_tablet.go | 2 +- .../testlib/migrate_served_from_test.go | 4 +- .../testlib/migrate_served_types_test.go | 24 +- .../testlib/reparent_external_test.go | 6 +- go/vt/zkctl/zksrv.sh | 4 +- proto/binlogdata.proto | 19 + proto/queryservice.proto | 3 + proto/vschema.proto | 17 +- proto/vtgateservice.proto | 6 +- py/vtproto/binlogdata_pb2.py | 152 +- py/vtproto/queryservice_pb2.py | 13 +- py/vtproto/queryservice_pb2_grpc.py | 17 + py/vtproto/vschema_pb2.py | 235 +- py/vtproto/vtgateservice_pb2_grpc.py | 6 +- py/vttest/mysql_flavor.py | 11 + test.go | 6 +- test/backup.py | 122 +- test/base_sharding.py | 6 +- test/config.json | 36 + test/initial_sharding_multi.py | 801 +++ test/mysql_flavor.py | 8 +- test/resharding.py | 2 + test/tablet.py | 17 +- test/xtrabackup.py | 25 + test/xtrabackup_xbstream.py | 26 + test/xtrabackup_xtra.py | 299 + vagrant-scripts/bootstrap_vm.sh | 3 +- vagrant-scripts/vitess/build.sh | 4 + 276 files changed, 12396 insertions(+), 5215 deletions(-) create mode 100644 config/mycnf/master_mysql80.cnf create mode 100644 go/vt/mysqlctl/xtrabackupengine.go create mode 100644 go/vt/vtgate/planbuilder/route_option.go create mode 100644 go/vt/vtgate/planbuilder/route_option_test.go delete mode 100644 go/vt/vttablet/tabletmanager/vreplication/player_plan.go create mode 100644 go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go rename go/vt/vttablet/tabletmanager/vreplication/{player_plan_test.go => replicator_plan_test.go} (52%) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vcopier.go create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vreplicator.go create mode 100644 go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go create mode 100644 go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go rename go/vt/workflow/reshardingworkflowgen/{workflow_test.go => workflow_flaky_test.go} (100%) create mode 100755 test/initial_sharding_multi.py create mode 100755 test/xtrabackup.py create mode 100755 test/xtrabackup_xbstream.py create mode 100755 test/xtrabackup_xtra.py diff --git a/Makefile b/Makefile index 4c3a7b72bed..f1676f9d6ba 100644 --- a/Makefile +++ b/Makefile @@ -1,11 +1,11 @@ # Copyright 2017 Google Inc. -# +# # 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. @@ -186,7 +186,7 @@ docker_base_mysql56: docker_base_mysql80: chmod -R o=g * - docker build -f docker/base/Dockerfile.mysql56 -t vitess/base:mysql80 . + docker build -f docker/base/Dockerfile.mysql80 -t vitess/base:mysql80 . docker_base_mariadb: chmod -R o=g * @@ -206,7 +206,7 @@ docker_base_percona57: docker_base_percona80: chmod -R o=g * - docker build -f docker/base/Dockerfile.percona57 -t vitess/base:percona80 . + docker build -f docker/base/Dockerfile.percona80 -t vitess/base:percona80 . # Run "make docker_lite PROMPT_NOTICE=false" to avoid that the script # prompts you to press ENTER and confirm that the vitess/base image is not diff --git a/bootstrap.sh b/bootstrap.sh index 3c58a603efc..4ac499cc18d 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -195,13 +195,22 @@ function install_etcd() { local version="$1" local dist="$2" + case $(uname) in + Linux) local platform=linux; local ext=tar.gz;; + Darwin) local platform=darwin; local ext=zip;; + esac + download_url=https://github.com/coreos/etcd/releases/download - tar_file="etcd-${version}-linux-amd64.tar.gz" + file="etcd-${version}-${platform}-amd64.${ext}" - wget "$download_url/$version/$tar_file" - tar xzf "$tar_file" - rm "$tar_file" - ln -snf "$dist/etcd-${version}-linux-amd64/etcd" "$VTROOT/bin/etcd" + wget "$download_url/$version/$file" + if [ "$ext" = "tar.gz" ]; then + tar xzf "$file" + else + unzip "$file" + fi + rm "$file" + ln -snf "$dist/etcd-${version}-${platform}-amd64/etcd" "$VTROOT/bin/etcd" } install_dep "etcd" "v3.3.10" "$VTROOT/dist/etcd" install_etcd @@ -211,9 +220,14 @@ function install_consul() { local version="$1" local dist="$2" + case $(uname) in + Linux) local platform=linux;; + Darwin) local platform=darwin;; + esac + download_url=https://releases.hashicorp.com/consul - wget "${download_url}/${version}/consul_${version}_linux_amd64.zip" - unzip "consul_${version}_linux_amd64.zip" + wget "${download_url}/${version}/consul_${version}_${platform}_amd64.zip" + unzip "consul_${version}_${platform}_amd64.zip" ln -snf "$dist/consul" "$VTROOT/bin/consul" } install_dep "Consul" "1.4.0" "$VTROOT/dist/consul" install_consul @@ -325,13 +339,13 @@ if [ "$BUILD_TESTS" == 1 ] ; then echo "MYSQL_FLAVOR environment variable not set. Using default: $MYSQL_FLAVOR" fi case "$MYSQL_FLAVOR" in - "MySQL56") + "MySQL56" | "MySQL80") myversion="$("$VT_MYSQL_ROOT/bin/mysql" --version)" [[ "$myversion" =~ Distrib\ 5\.[67] || "$myversion" =~ Ver\ 8\. ]] || fail "Couldn't find MySQL 5.6+ in $VT_MYSQL_ROOT. Set VT_MYSQL_ROOT to override search location." echo "Found MySQL 5.6+ installation in $VT_MYSQL_ROOT." ;; - "MariaDB" | "MariaDB103" ) + "MariaDB" | "MariaDB103") myversion="$("$VT_MYSQL_ROOT/bin/mysql" --version)" [[ "$myversion" =~ MariaDB ]] || fail "Couldn't find MariaDB in $VT_MYSQL_ROOT. Set VT_MYSQL_ROOT to override search location." echo "Found MariaDB installation in $VT_MYSQL_ROOT." diff --git a/config/init_db.sql b/config/init_db.sql index 6bf2ac52023..5b56939c3c3 100644 --- a/config/init_db.sql +++ b/config/init_db.sql @@ -1,9 +1,16 @@ # This file is executed immediately after mysql_install_db, # to initialize a fresh data directory. -########################################## +############################################################################### +# WARNING: This sql is *NOT* safe for production use, +# as it contains default well-known users and passwords. +# Care should be taken to change these users and passwords +# for production. +############################################################################### + +############################################################################### # Equivalent of mysql_secure_installation -########################################## +############################################################################### # Changes during the init db should not make it to the binlog. # They could potentially create errant transactions on replicas. @@ -17,9 +24,9 @@ DELETE FROM mysql.user WHERE User = 'root' AND Host != 'localhost'; # Remove test database. DROP DATABASE IF EXISTS test; -########################################## +############################################################################### # Vitess defaults -########################################## +############################################################################### # Vitess-internal database. CREATE DATABASE IF NOT EXISTS _vt; @@ -28,12 +35,14 @@ CREATE DATABASE IF NOT EXISTS _vt; CREATE TABLE IF NOT EXISTS _vt.local_metadata ( name VARCHAR(255) NOT NULL, value VARCHAR(255) NOT NULL, - PRIMARY KEY (name) + db_name VARBINARY(255) NOT NULL, + PRIMARY KEY (db_name, name) ) ENGINE=InnoDB; CREATE TABLE IF NOT EXISTS _vt.shard_metadata ( name VARCHAR(255) NOT NULL, value MEDIUMBLOB NOT NULL, - PRIMARY KEY (name) + db_name VARBINARY(255) NOT NULL, + PRIMARY KEY (db_name, name) ) ENGINE=InnoDB; # Admin user with all privileges. diff --git a/config/mycnf/default-fast.cnf b/config/mycnf/default-fast.cnf index 3b732fdc6df..0046bad32a2 100644 --- a/config/mycnf/default-fast.cnf +++ b/config/mycnf/default-fast.cnf @@ -18,7 +18,7 @@ innodb_flush_log_at_trx_commit = 2 innodb_flush_method = O_DIRECT innodb_lock_wait_timeout = 20 innodb_log_buffer_size = 1M -innodb_log_file_size = 1M +innodb_log_file_size = 4M innodb_log_files_in_group = 2 innodb_log_group_home_dir = {{.InnodbLogGroupHomeDir}} innodb_max_dirty_pages_pct = 75 diff --git a/config/mycnf/master_mysql80.cnf b/config/mycnf/master_mysql80.cnf new file mode 100644 index 00000000000..f81761ad906 --- /dev/null +++ b/config/mycnf/master_mysql80.cnf @@ -0,0 +1,38 @@ +# Options for enabling GTID +# https://dev.mysql.com/doc/refman/5.6/en/replication-gtids-howto.html +gtid_mode = ON +log_bin +log_slave_updates +enforce_gtid_consistency + +# Crash-safe replication settings. +master_info_repository = TABLE +relay_log_info_repository = TABLE +relay_log_purge = 1 +relay_log_recovery = 1 + +# Native AIO tends to run into aio-max-nr limit during test startup. +innodb_use_native_aio = 0 + +# Semi-sync replication is required for automated unplanned failover +# (when the master goes away). Here we just load the plugin so it's +# available if desired, but it's disabled at startup. +# +# If the -enable_semi_sync flag is used, VTTablet will enable semi-sync +# at the proper time when replication is set up, or when masters are +# promoted or demoted. +plugin-load = rpl_semi_sync_master=semisync_master.so;rpl_semi_sync_slave=semisync_slave.so + +# When semi-sync is enabled, don't allow fallback to async +# if you get no ack, or have no slaves. This is necessary to +# prevent alternate futures when doing a failover in response to +# a master that becomes unresponsive. +rpl_semi_sync_master_timeout = 1000000000000000000 +rpl_semi_sync_master_wait_no_slave = 1 + +# disable mysqlx +mysqlx = 0 + +# 8.0 changes the default auth-plugin to caching_sha2_password +default_authentication_plugin = mysql_native_password +secure_file_priv = NULL diff --git a/doc/ServerConfiguration.md b/doc/ServerConfiguration.md index f95b0d76c57..737757ae726 100644 --- a/doc/ServerConfiguration.md +++ b/doc/ServerConfiguration.md @@ -258,7 +258,7 @@ VTTablet requires multiple user credentials to perform its tasks. Since it's req **connection** parameters -* **db\_socket**: The unix socket to connect on. If this is specifed, host and port will not be used. +* **db\_socket**: The unix socket to connect on. If this is specified, host and port will not be used. * **db\_host**: The host name for the tcp connection. * **db\_port**: The tcp port to be used with the db\_host. * **db\_charset**: Character set. Only utf8 or latin1 based character sets are supported. diff --git a/doc/TopologyService.md b/doc/TopologyService.md index c5d690da58e..d4c63348d6a 100644 --- a/doc/TopologyService.md +++ b/doc/TopologyService.md @@ -451,7 +451,7 @@ all cells to route traffic. Note this is necessary to access the master in another cell. After the extension to two cells, the original topo service contains both the -global topology data, and the first cell topology data. The more symetrical +global topology data, and the first cell topology data. The more symmetrical configuration we're after would be to split that original service into two: a global one that only contains the global data (spread across both cells), and a local one to the original cells. To achieve that split: @@ -471,7 +471,7 @@ local one to the original cells. To achieve that split: * Remove all remaining data in the global topology service that are in the old local server root. -After this split, the configuration is completely symetrical: +After this split, the configuration is completely symmetrical: * a global topology service, with servers in all cells. Only contains global topology data about Keyspaces, Shards and VSchema. Typically it has 5 servers diff --git a/doc/VTTabletModes.md b/doc/VTTabletModes.md index 5e89fe65cff..1d37efdf4af 100644 --- a/doc/VTTabletModes.md +++ b/doc/VTTabletModes.md @@ -36,7 +36,7 @@ Specifically, the absence of a my.cnf file indicates to vttablet that it's conne Even if a MySQL is remote, you can still make vttablet perform some management functions. They are as follows: -* `-disable_active_reparents`: If this flag is set, then any reparent or slave commands will not be allowed. These are InitShardMaster, PlannedReparent, PlannedReparent, EmergencyReparent, and ReparentTablet. In this mode, you should use the TabletExternallyReparented command to inform vitess of the current master. +* `-disable_active_reparents`: If this flag is set, then any reparent or slave commands will not be allowed. These are InitShardMaster, PlannedReparent, EmergencyReparent, and ReparentTablet. In this mode, you should use the TabletExternallyReparented command to inform vitess of the current master. * `-master_connect_retry`: This value is give to mysql when it connects a slave to the master as the retry duration parameter. * `-enable_replication_reporter`: If this flag is set, then vttablet will transmit replica lag related information to the vtgates, which will allow it to balance load better. Additionally, enabling this will also cause vttablet to restart replication if it was stopped. However, it will do this only if -disable_active_reparents was not turned on. * `-enable_semi_sync`: This option will automatically enable semi-sync on new replicas as well as on any tablet that transitions into a replica type. This includes the demotion of a master to a replica. diff --git a/doc/VitessApi.md b/doc/VitessApi.md index ab7ac7a01dd..61b5ffc10a9 100644 --- a/doc/VitessApi.md +++ b/doc/VitessApi.md @@ -79,11 +79,11 @@ The following table lists the methods in each group and links to more detail abo v3 API (alpha) Execute -Execute tries to route the query to the right shard. It depends on the query and bind variables to provide enough information in conjonction with the vindexes to route the query. +Execute tries to route the query to the right shard. It depends on the query and bind variables to provide enough information in conjunction with the vindexes to route the query. StreamExecute -StreamExecute executes a streaming query based on shards. It depends on the query and bind variables to provide enough information in conjonction with the vindexes to route the query. Use this method if the query returns a large number of rows. +StreamExecute executes a streaming query based on shards. It depends on the query and bind variables to provide enough information in conjunction with the vindexes to route the query. Use this method if the query returns a large number of rows. ##Range-based Sharding @@ -572,7 +572,7 @@ GetSrvKeyspace returns a SrvKeyspace object (as seen by this vtgate). This metho ##v3 API (alpha) ### Execute -Execute tries to route the query to the right shard. It depends on the query and bind variables to provide enough information in conjonction with the vindexes to route the query. +Execute tries to route the query to the right shard. It depends on the query and bind variables to provide enough information in conjunction with the vindexes to route the query. #### Request @@ -604,7 +604,7 @@ Execute tries to route the query to the right shard. It depends on the query and ### StreamExecute -StreamExecute executes a streaming query based on shards. It depends on the query and bind variables to provide enough information in conjonction with the vindexes to route the query. Use this method if the query returns a large number of rows. +StreamExecute executes a streaming query based on shards. It depends on the query and bind variables to provide enough information in conjunction with the vindexes to route the query. Use this method if the query returns a large number of rows. #### Request @@ -839,7 +839,7 @@ Field describes a single column returned by a query | org_name
string| | | column_length
uint32| column_length is really a uint32. All 32 bits can be used. | | charset
uint32| charset is actually a uint16. Only the lower 16 bits are used. | -| decimals
uint32| decimals is actualy a uint8. Only the lower 8 bits are used. | +| decimals
uint32| decimals is actually a uint8. Only the lower 8 bits are used. | | flags
uint32| flags is actually a uint16. Only the lower 16 bits are used. | ### query.QueryResult diff --git a/doc/VitessSequences.md b/doc/VitessSequences.md index d2a68ceb4ad..155b315acfb 100644 --- a/doc/VitessSequences.md +++ b/doc/VitessSequences.md @@ -87,7 +87,7 @@ Let's start by looking at the MySQL auto-increment feature: An early design was to use a single unsharded database and a table with an auto-increment value to generate new values. However, this has serious -limitations, in particular throughtput, and storing one entry for each value in +limitations, in particular throughput, and storing one entry for each value in that table, for no reason. So we decided instead to base sequences on a MySQL table, and use a single value diff --git a/doc/VitessTransportSecurityModel.md b/doc/VitessTransportSecurityModel.md index a0df28cc11b..bc9d0ebe832 100644 --- a/doc/VitessTransportSecurityModel.md +++ b/doc/VitessTransportSecurityModel.md @@ -55,7 +55,7 @@ When using gRPC transport, Vitess can use the usual TLS security features * grpc\_cert, grpc\_key: server cert and key to use. * grpc\_ca (optional): client cert chains to trust. If specified, the client must use a certificate signed by one ca in the provided file. -* A Vitess go client can be configured with symetrical parameters to enable TLS: +* A Vitess go client can be configured with symmetrical parameters to enable TLS: * ...\_grpc\_ca: list of server cert signers to trust. * ...\_grpc\_server\_name: name of the server cert to trust, instead of the hostname used to connect. diff --git a/doc/meetups_notes/06-14-2018.md b/doc/meetups_notes/06-14-2018.md index 904c9a7ee0c..cb174fd8a88 100644 --- a/doc/meetups_notes/06-14-2018.md +++ b/doc/meetups_notes/06-14-2018.md @@ -56,7 +56,7 @@ Meeting started at 8am PDT on Zoom. ### New Vitess Logo / Logo Changes -We kicked off the meeting talking about the first agenda item, *Vitess's new logo*. The new logo was mooted in the first place because of its current similarites to [Vivid IT Corp](https://www.vividitcorp.com/). +We kicked off the meeting talking about the first agenda item, *Vitess's new logo*. The new logo was mooted in the first place because of its current similarities to [Vivid IT Corp](https://www.vividitcorp.com/). Based on [logos](https://docs.google.com/forms/d/e/1FAIpQLScp5hGY98vpRMxs3oRT8c-XJ_b04ei6uCFiYiQe3nDunFbyuw/viewform) created with CNCF's help, logos 1 and 4 got the highest no of votes. @@ -124,7 +124,7 @@ We are looking for 5 engineers. Job scope at [our site](https://planetscale.com/ We discussed that we need to work on / update official roadmap. -Question from Ameet about whether there's anything on ther roadmap for online backups, because increasingly backups take more time. +Question from Ameet about whether there's anything on the roadmap for online backups, because increasingly backups take more time. Sugu says there are 500 lines of code re backup. But we will discuss this in our roadmap discussions. diff --git a/doc/vtctlReference.md b/doc/vtctlReference.md index 75e14860f5c..04c0eb1cfd4 100644 --- a/doc/vtctlReference.md +++ b/doc/vtctlReference.md @@ -530,7 +530,7 @@ Validates that all nodes reachable from the specified keyspace are consistent. ### WaitForDrain -Blocks until no new queries were observed on all tablets with the given tablet type in the specifed keyspace. This can be used as sanity check to ensure that the tablets were drained after running vtctl MigrateServedTypes and vtgate is no longer using them. If -timeout is set, it fails when the timeout is reached. +Blocks until no new queries were observed on all tablets with the given tablet type in the specified keyspace. This can be used as sanity check to ensure that the tablets were drained after running vtctl MigrateServedTypes and vtgate is no longer using them. If -timeout is set, it fails when the timeout is reached. #### Example diff --git a/docker/bootstrap/Dockerfile.mariadb b/docker/bootstrap/Dockerfile.mariadb index 83dfdff8b74..81ee6bb2678 100644 --- a/docker/bootstrap/Dockerfile.mariadb +++ b/docker/bootstrap/Dockerfile.mariadb @@ -1,12 +1,18 @@ FROM vitess/bootstrap:common # Install MariaDB 10 -RUN apt-get update \ +RUN apt-get update -y \ && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ bzip2 \ mariadb-server \ libmariadbclient-dev \ - && rm -rf /var/lib/apt/lists/* + libdbd-mysql-perl \ + rsync \ + libev4 \ + && rm -rf /var/lib/apt/lists/* \ + && wget https://www.percona.com/downloads/XtraBackup/Percona-XtraBackup-2.4.13/binary/debian/stretch/x86_64/percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb \ + && dpkg -i percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb \ + && rm -f percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb # Bootstrap Vitess WORKDIR /vt/src/vitess.io/vitess diff --git a/docker/bootstrap/Dockerfile.mysql56 b/docker/bootstrap/Dockerfile.mysql56 index 56fd5dba41b..5da4ec29946 100644 --- a/docker/bootstrap/Dockerfile.mysql56 +++ b/docker/bootstrap/Dockerfile.mysql56 @@ -3,9 +3,12 @@ FROM vitess/bootstrap:common # Install MySQL 5.6 RUN for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver pool.sks-keyservers.net 5072E1F5 && break; done && \ add-apt-repository 'deb http://repo.mysql.com/apt/debian/ stretch mysql-5.6' && \ - apt-get update && \ - DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev && \ - rm -rf /var/lib/apt/lists/* + apt-get update -y && \ + DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev libdbd-mysql-perl rsync libev4 && \ + rm -rf /var/lib/apt/lists/* && \ + wget https://www.percona.com/downloads/XtraBackup/Percona-XtraBackup-2.4.13/binary/debian/stretch/x86_64/percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb && \ + dpkg -i percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb && \ + rm -f percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb # Bootstrap Vitess WORKDIR /vt/src/vitess.io/vitess diff --git a/docker/bootstrap/Dockerfile.mysql57 b/docker/bootstrap/Dockerfile.mysql57 index 75d7d03aa17..a251e03031d 100644 --- a/docker/bootstrap/Dockerfile.mysql57 +++ b/docker/bootstrap/Dockerfile.mysql57 @@ -4,9 +4,12 @@ FROM vitess/bootstrap:common RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates && \ for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver ha.pool.sks-keyservers.net 5072E1F5 && break; done && \ add-apt-repository 'deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7' && \ - apt-get update && \ - DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev && \ - rm -rf /var/lib/apt/lists/* + apt-get update -y && \ + DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev libdbd-mysql-perl rsync libev4 && \ + rm -rf /var/lib/apt/lists/* && \ + wget https://www.percona.com/downloads/XtraBackup/Percona-XtraBackup-2.4.13/binary/debian/stretch/x86_64/percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb && \ + dpkg -i percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb && \ + rm -f percona-xtrabackup-24_2.4.13-1.stretch_amd64.deb # Bootstrap Vitess WORKDIR /vt/src/vitess.io/vitess diff --git a/docker/bootstrap/Dockerfile.mysql80 b/docker/bootstrap/Dockerfile.mysql80 index c82ee741dec..1d21887d850 100644 --- a/docker/bootstrap/Dockerfile.mysql80 +++ b/docker/bootstrap/Dockerfile.mysql80 @@ -1,16 +1,19 @@ FROM vitess/bootstrap:common -# Install MySQL 5.7 +# Install MySQL 8.0 RUN for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver ha.pool.sks-keyservers.net 8C718D3B5072E1F5 && break; done && \ add-apt-repository 'deb http://repo.mysql.com/apt/debian/ stretch mysql-8.0' && \ - apt-get update && \ - DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev && \ - rm -rf /var/lib/apt/lists/* + apt-get update -y && \ + DEBIAN_FRONTEND=noninteractive apt-get install -y mysql-server libmysqlclient-dev libdbd-mysql-perl rsync libev4 && \ + rm -rf /var/lib/apt/lists/* && \ + wget https://www.percona.com/downloads/XtraBackup/Percona-XtraBackup-8.0.4/binary/debian/stretch/x86_64/percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb && \ + dpkg -i percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb && \ + rm -f percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb # Bootstrap Vitess WORKDIR /vt/src/vitess.io/vitess -ENV MYSQL_FLAVOR MySQL56 +ENV MYSQL_FLAVOR MySQL80 USER vitess RUN ./bootstrap.sh \ No newline at end of file diff --git a/docker/bootstrap/Dockerfile.percona80 b/docker/bootstrap/Dockerfile.percona80 index 9968e1dde8f..8b91905b4b1 100644 --- a/docker/bootstrap/Dockerfile.percona80 +++ b/docker/bootstrap/Dockerfile.percona80 @@ -15,11 +15,17 @@ RUN for i in $(seq 1 10); do apt-key adv --no-tty --keyserver keys.gnupg.net --r percona-server-tokudb \ percona-server-rocksdb \ bzip2 \ - && rm -rf /var/lib/apt/lists/* + libdbd-mysql-perl \ + rsync \ + libev4 \ + && rm -rf /var/lib/apt/lists/* \ + && wget https://www.percona.com/downloads/XtraBackup/Percona-XtraBackup-8.0.4/binary/debian/stretch/x86_64/percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb \ + && dpkg -i percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb \ + && rm -f percona-xtrabackup-80_8.0.4-1.stretch_amd64.deb # Bootstrap Vitess WORKDIR /vt/src/vitess.io/vitess -ENV MYSQL_FLAVOR MySQL56 +ENV MYSQL_FLAVOR MySQL80 USER vitess RUN ./bootstrap.sh diff --git a/docker/k8s/Dockerfile b/docker/k8s/Dockerfile index 0eee530a1bc..744bd7700b0 100644 --- a/docker/k8s/Dockerfile +++ b/docker/k8s/Dockerfile @@ -42,6 +42,7 @@ COPY --from=base $VTTOP/config/init_db.sql /vt/config/ # mysql flavor files for db specific .cnf settings COPY --from=base $VTTOP/config/mycnf/master_mysql56.cnf /vt/config/mycnf/ +COPY --from=base $VTTOP/config/mycnf/master_mysql80.cnf /vt/config/mycnf/ COPY --from=base $VTTOP/config/mycnf/master_mariadb.cnf /vt/config/mycnf/ COPY --from=base $VTTOP/config/mycnf/master_mariadb103.cnf /vt/config/mycnf/ diff --git a/docker/lite/Dockerfile b/docker/lite/Dockerfile index ebfdb21d5c4..53e0006653d 100644 --- a/docker/lite/Dockerfile +++ b/docker/lite/Dockerfile @@ -21,7 +21,8 @@ RUN chown -R vitess:vitess /vt FROM debian:stretch-slim # Install dependencies -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 \ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 && break; done \ && echo 'deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7' > /etc/apt/sources.list.d/mysql.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive \ diff --git a/docker/lite/Dockerfile.mariadb b/docker/lite/Dockerfile.mariadb index d25e274c46c..8684e2b59f8 100644 --- a/docker/lite/Dockerfile.mariadb +++ b/docker/lite/Dockerfile.mariadb @@ -22,7 +22,7 @@ FROM debian:stretch-slim # Install dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ - && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 0xF1656F24C74CD1D8 \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 0xF1656F24C74CD1D8 && break; done \ && echo 'deb http://sfo1.mirrors.digitalocean.com/mariadb/repo/10.2/debian stretch main' > /etc/apt/sources.list.d/mariadb.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ diff --git a/docker/lite/Dockerfile.mariadb103 b/docker/lite/Dockerfile.mariadb103 index e01e394813f..fd6e9f48d88 100644 --- a/docker/lite/Dockerfile.mariadb103 +++ b/docker/lite/Dockerfile.mariadb103 @@ -21,7 +21,7 @@ FROM debian:stretch-slim # Install dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ - && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 0xF1656F24C74CD1D8 \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 0xF1656F24C74CD1D8 && break; done \ && echo 'deb http://sfo1.mirrors.digitalocean.com/mariadb/repo/10.3/debian stretch main' > /etc/apt/sources.list.d/mariadb.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ diff --git a/docker/lite/Dockerfile.mysql56 b/docker/lite/Dockerfile.mysql56 index a005a21cb46..f0bb24ecc3b 100644 --- a/docker/lite/Dockerfile.mysql56 +++ b/docker/lite/Dockerfile.mysql56 @@ -21,7 +21,8 @@ RUN chown -R vitess:vitess /vt FROM debian:stretch-slim # Install dependencies -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 \ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 && break; done \ && echo 'deb http://repo.mysql.com/apt/debian/ stretch mysql-5.6' > /etc/apt/sources.list.d/mysql.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive \ diff --git a/docker/lite/Dockerfile.mysql57 b/docker/lite/Dockerfile.mysql57 index ebfdb21d5c4..53e0006653d 100644 --- a/docker/lite/Dockerfile.mysql57 +++ b/docker/lite/Dockerfile.mysql57 @@ -21,7 +21,8 @@ RUN chown -R vitess:vitess /vt FROM debian:stretch-slim # Install dependencies -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 \ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 && break; done \ && echo 'deb http://repo.mysql.com/apt/debian/ stretch mysql-5.7' > /etc/apt/sources.list.d/mysql.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive \ diff --git a/docker/lite/Dockerfile.mysql80 b/docker/lite/Dockerfile.mysql80 index 1b98aae565f..3d20b60dee5 100644 --- a/docker/lite/Dockerfile.mysql80 +++ b/docker/lite/Dockerfile.mysql80 @@ -21,7 +21,8 @@ RUN chown -R vitess:vitess /vt FROM debian:stretch-slim # Install dependencies -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates && apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 \ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keyserver.ubuntu.com 8C718D3B5072E1F5 && break; done \ && echo 'deb http://repo.mysql.com/apt/debian/ stretch mysql-8.0' > /etc/apt/sources.list.d/mysql.list \ && apt-get update \ && DEBIAN_FRONTEND=noninteractive \ diff --git a/docker/lite/Dockerfile.percona b/docker/lite/Dockerfile.percona index 5dd69b7b2cc..bf3b06a69d8 100644 --- a/docker/lite/Dockerfile.percona +++ b/docker/lite/Dockerfile.percona @@ -22,7 +22,7 @@ FROM debian:stretch-slim # Install dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ - && apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 && break; done \ && echo 'deb http://repo.percona.com/apt stretch main' > /etc/apt/sources.list.d/percona.list && \ { \ echo debconf debconf/frontend select Noninteractive; \ diff --git a/docker/lite/Dockerfile.percona57 b/docker/lite/Dockerfile.percona57 index 3dc21bcf573..4b1a55b478d 100644 --- a/docker/lite/Dockerfile.percona57 +++ b/docker/lite/Dockerfile.percona57 @@ -22,7 +22,7 @@ FROM debian:stretch-slim # Install dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ - && apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 && break; done \ && echo 'deb http://repo.percona.com/apt stretch main' > /etc/apt/sources.list.d/percona.list && \ { \ echo debconf debconf/frontend select Noninteractive; \ diff --git a/docker/lite/Dockerfile.percona80 b/docker/lite/Dockerfile.percona80 index b48b8db5c99..ceec322e36c 100644 --- a/docker/lite/Dockerfile.percona80 +++ b/docker/lite/Dockerfile.percona80 @@ -22,7 +22,7 @@ FROM debian:stretch-slim # Install dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends gnupg dirmngr ca-certificates \ - && apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 \ + && for i in $(seq 1 10); do apt-key adv --no-tty --recv-keys --keyserver keys.gnupg.net 9334A25F8507EFA5 && break; done \ && echo 'deb http://repo.percona.com/ps-80/apt stretch main' > /etc/apt/sources.list.d/percona.list && \ { \ echo debconf debconf/frontend select Noninteractive; \ diff --git a/examples/helm/kmysql.sh b/examples/helm/kmysql.sh index eb9d5c5c68a..6525448a62e 100755 --- a/examples/helm/kmysql.sh +++ b/examples/helm/kmysql.sh @@ -20,7 +20,7 @@ host=$(minikube service vtgate-zone1 --format "{{.IP}}" | tail -n 1) port=$(minikube service vtgate-zone1 --format "{{.Port}}" | tail -n 1) if [ -z $port ]; then - #This checks K8s runing on an single node by kubeadm + #This checks K8s running on an single node by kubeadm if [ $(kubectl get nodes | grep -v NAM | wc -l) -eq 1 -o $(kubectl get nodes | grep -v NAM | grep master | wc -l ) -eq 1 ]; then host="127.0.0.1" port=`kubectl describe service vtgate-zone1 | grep NodePort | grep mysql | awk '{print $3}' | awk -F'/' '{print $1}'` diff --git a/examples/local/vttablet-up.sh b/examples/local/vttablet-up.sh index 00cea50a5b5..bdd16ba6df6 100755 --- a/examples/local/vttablet-up.sh +++ b/examples/local/vttablet-up.sh @@ -38,7 +38,7 @@ source $script_root/env.sh init_db_sql_file="$VTROOT/config/init_db.sql" -export EXTRA_MY_CNF=$VTROOT/config/mycnf/default-fast.cnf +export EXTRA_MY_CNF=$VTROOT/config/mycnf/default-fast.cnf:$VTROOT/config/mycnf/rbr.cnf case "$MYSQL_FLAVOR" in "MySQL56") diff --git a/go/cacheservice/cacheservice.go b/go/cacheservice/cacheservice.go index 39a2c34ef64..7a2c85b3644 100644 --- a/go/cacheservice/cacheservice.go +++ b/go/cacheservice/cacheservice.go @@ -56,9 +56,9 @@ type CacheService interface { // for using with CAS. Gets returns a CAS identifier with the item. If // the item's CAS value has changed since you Gets'ed it, it will not be stored. Gets(keys ...string) (results []Result, err error) - // Set set the value with specified cache key. + // Set sets the value with specified cache key. Set(key string, flags uint16, timeout uint64, value []byte) (stored bool, err error) - // Add store the value only if it does not already exist. + // Add stores the value only if it does not already exist. Add(key string, flags uint16, timeout uint64, value []byte) (stored bool, err error) // Replace replaces the value, only if the value already exists, // for the specified cache key. @@ -69,7 +69,7 @@ type CacheService interface { Prepend(key string, flags uint16, timeout uint64, value []byte) (stored bool, err error) // Cas stores the value only if no one else has updated the data since you read it last. Cas(key string, flags uint16, timeout uint64, value []byte, cas uint64) (stored bool, err error) - // Delete delete the value for the specified cache key. + // Delete deletes the value for the specified cache key. Delete(key string) (deleted bool, err error) // FlushAll purges the entire cache. FlushAll() (err error) diff --git a/go/cmd/automation_client/automation_client.go b/go/cmd/automation_client/automation_client.go index 73e16245b7d..1f4b6d965eb 100644 --- a/go/cmd/automation_client/automation_client.go +++ b/go/cmd/automation_client/automation_client.go @@ -106,7 +106,7 @@ func main() { fmt.Printf("SUCCESS: ClusterOperation finished.\n\nDetails:\n%v", proto.MarshalTextString(resp)) } -// waitForClusterOp polls and blocks until the ClusterOperation invocation specified by "id" has finished. If an error occured, it will be returned. +// waitForClusterOp polls and blocks until the ClusterOperation invocation specified by "id" has finished. If an error occurred, it will be returned. func waitForClusterOp(client automationservicepb.AutomationClient, id string) (*automationpb.GetClusterOperationDetailsResponse, error) { for { req := &automationpb.GetClusterOperationDetailsRequest{ diff --git a/go/cmd/mysqlctl/mysqlctl.go b/go/cmd/mysqlctl/mysqlctl.go index 23db8721b78..2f39206a003 100644 --- a/go/cmd/mysqlctl/mysqlctl.go +++ b/go/cmd/mysqlctl/mysqlctl.go @@ -197,11 +197,11 @@ type command struct { var commands = []command{ {"init", initCmd, "[-wait_time=5m] [-init_db_sql_file=]", - "Initalizes the directory structure and starts mysqld"}, + "Initializes the directory structure and starts mysqld"}, {"init_config", initConfigCmd, "", - "Initalizes the directory structure, creates my.cnf file, but does not start mysqld"}, + "Initializes the directory structure, creates my.cnf file, but does not start mysqld"}, {"reinit_config", reinitConfigCmd, "", - "Reinitalizes my.cnf file with new server_id"}, + "Reinitializes my.cnf file with new server_id"}, {"teardown", teardownCmd, "[-wait_time=5m] [-force]", "Shuts mysqld down, and removes the directory"}, {"start", startCmd, "[-wait_time=5m]", diff --git a/go/cmd/topo2topo/topo2topo.go b/go/cmd/topo2topo/topo2topo.go index 78ab89d984b..7b25dd6849d 100644 --- a/go/cmd/topo2topo/topo2topo.go +++ b/go/cmd/topo2topo/topo2topo.go @@ -43,6 +43,7 @@ var ( doShards = flag.Bool("do-shards", false, "copies the shard information") doShardReplications = flag.Bool("do-shard-replications", false, "copies the shard replication information") doTablets = flag.Bool("do-tablets", false, "copies the tablet information") + doRoutingRules = flag.Bool("do-routing-rules", false, "copies the routing rules") ) func main() { @@ -87,7 +88,9 @@ func copyTopos(ctx context.Context, fromTS, toTS *topo.Server) { if *doTablets { helpers.CopyTablets(ctx, fromTS, toTS) } - + if *doRoutingRules { + helpers.CopyRoutingRules(ctx, fromTS, toTS) + } } func compareTopos(ctx context.Context, fromTS, toTS *topo.Server) { diff --git a/go/cmd/vtctld/schema.go b/go/cmd/vtctld/schema.go index 1248938d6a7..3268e2de1cc 100644 --- a/go/cmd/vtctld/schema.go +++ b/go/cmd/vtctld/schema.go @@ -33,7 +33,7 @@ import ( var ( schemaChangeDir = flag.String("schema_change_dir", "", "directory contains schema changes for all keyspaces. Each keyspace has its own directory and schema changes are expected to live in '$KEYSPACE/input' dir. e.g. test_keyspace/input/*sql, each sql file represents a schema change") - schemaChangeController = flag.String("schema_change_controller", "", "schema change controller is responsible for finding schema changes and responsing schema change events") + schemaChangeController = flag.String("schema_change_controller", "", "schema change controller is responsible for finding schema changes and responding to schema change events") schemaChangeCheckInterval = flag.Int("schema_change_check_interval", 60, "this value decides how often we check schema change dir, in seconds") schemaChangeUser = flag.String("schema_change_user", "", "The user who submits this schema change.") schemaChangeSlaveTimeout = flag.Duration("schema_change_slave_timeout", 10*time.Second, "how long to wait for slaves to receive the schema change") diff --git a/go/cmd/vtexplain/vtexplain.go b/go/cmd/vtexplain/vtexplain.go index 90e8280d067..50dde581071 100644 --- a/go/cmd/vtexplain/vtexplain.go +++ b/go/cmd/vtexplain/vtexplain.go @@ -55,6 +55,7 @@ var ( "vschema", "vschema-file", "dbname", + "queryserver-config-passthrough-dmls", } ) @@ -63,7 +64,7 @@ func usage() { for _, name := range vtexplainFlags { f := flag.Lookup(name) if f == nil { - panic("unkown flag " + name) + panic("unknown flag " + name) } flagUsage(f) } diff --git a/go/cmd/vtgateclienttest/services/services.go b/go/cmd/vtgateclienttest/services/services.go index 9cf9d5a34c6..75dd3308199 100644 --- a/go/cmd/vtgateclienttest/services/services.go +++ b/go/cmd/vtgateclienttest/services/services.go @@ -23,6 +23,7 @@ import ( // CreateServices creates the implementation chain of all the test cases func CreateServices() vtgateservice.VTGateService { + //lint:ignore S1021 declare s of type interface that all the clients implement var s vtgateservice.VTGateService s = newTerminalClient() s = newSuccessClient(s) diff --git a/go/cmd/zk/zkcmd.go b/go/cmd/zk/zkcmd.go index c04579d52bb..6b57a666423 100644 --- a/go/cmd/zk/zkcmd.go +++ b/go/cmd/zk/zkcmd.go @@ -548,7 +548,7 @@ func cmdCat(ctx context.Context, subFlags *flag.FlagSet, args []string) error { } decoded := "" if *decodeProto { - decoded, err = vtctl.DecodeContent(zkPath, data) + decoded, err = vtctl.DecodeContent(zkPath, data, false) if err != nil { log.Warningf("cat: cannot proto decode %v: %v", zkPath, err) decoded = string(data) diff --git a/go/memcache/memcache.go b/go/memcache/memcache.go index 5a5804928de..3ee2666295c 100644 --- a/go/memcache/memcache.go +++ b/go/memcache/memcache.go @@ -80,13 +80,13 @@ func (mc *Connection) Gets(keys ...string) (results []cacheservice.Result, err e return } -// Set set the value with specified cache key. +// Set sets the value with specified cache key. func (mc *Connection) Set(key string, flags uint16, timeout uint64, value []byte) (stored bool, err error) { defer handleError(&err) return mc.store("set", key, flags, timeout, value, 0), nil } -// Add store the value only if it does not already exist. +// Add stores the value only if it does not already exist. func (mc *Connection) Add(key string, flags uint16, timeout uint64, value []byte) (stored bool, err error) { defer handleError(&err) return mc.store("add", key, flags, timeout, value, 0), nil @@ -117,7 +117,7 @@ func (mc *Connection) Cas(key string, flags uint16, timeout uint64, value []byte return mc.store("cas", key, flags, timeout, value, cas), nil } -// Delete delete the value for the specified cache key. +// Delete deletes the value for the specified cache key. func (mc *Connection) Delete(key string) (deleted bool, err error) { defer handleError(&err) mc.setDeadline() diff --git a/go/mysql/flavor.go b/go/mysql/flavor.go index ffa806509ee..2fdbe315722 100644 --- a/go/mysql/flavor.go +++ b/go/mysql/flavor.go @@ -34,6 +34,14 @@ var ( ErrNotSlave = errors.New("no slave status") ) +const ( + // mariaDBReplicationHackPrefix is the prefix of a version for MariaDB 10.0 + // versions, to work around replication bugs. + mariaDBReplicationHackPrefix = "5.5.5-" + // mariaDBVersionString is present in + mariaDBVersionString = "MariaDB" +) + // flavor is the abstract interface for a flavor. // Flavors are auto-detected upon connection using the server version. // We have two major implementations (the main difference is the GTID @@ -91,13 +99,6 @@ type flavor interface { disableBinlogPlaybackCommand() string } -// mariaDBReplicationHackPrefix is the prefix of a version for MariaDB 10.0 -// versions, to work around replication bugs. -const mariaDBReplicationHackPrefix = "5.5.5-" - -// mariaDBVersionString is present in -const mariaDBVersionString = "MariaDB" - // fillFlavor fills in c.Flavor based on c.ServerVersion. // This is the same logic as the ConnectorJ java client. We try to recognize // MariaDB as much as we can, but default to MySQL. diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index ceff67eebf5..b628486fb93 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -23,6 +23,7 @@ import ( "bytes" "encoding/hex" "fmt" + "math" "sync" "time" @@ -52,6 +53,10 @@ var ( // BlplTransaction is the key for the stats map. BlplTransaction = "Transaction" + // VReplicationInit is for the Init state. + VReplicationInit = "Init" + // VReplicationCopying is for the Copying state. + VReplicationCopying = "Copying" // BlpRunning is for the Running state. BlpRunning = "Running" // BlpStopped is for the Stopped state. @@ -108,6 +113,7 @@ func NewStats() *Stats { bps.Timings = stats.NewTimings("", "", "") bps.Rates = stats.NewRates("", bps.Timings, 15, 60e9) bps.History = history.New(3) + bps.SecondsBehindMaster.Set(math.MaxInt64) return bps } @@ -191,29 +197,19 @@ func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { // applyEvents returns a recordable status message on termination or an error otherwise. func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { // Read starting values for vreplication. - pos, stopPos, maxTPS, maxReplicationLag, err := ReadVRSettings(blp.dbClient, blp.uid) + settings, err := ReadVRSettings(blp.dbClient, blp.uid) if err != nil { log.Error(err) return err } - blp.position, err = mysql.DecodePosition(pos) - if err != nil { - log.Error(err) - return err - } - if stopPos != "" { - blp.stopPosition, err = mysql.DecodePosition(stopPos) - if err != nil { - log.Error(err) - return err - } - } + blp.position = settings.StartPos + blp.stopPosition = settings.StopPos t, err := throttler.NewThrottler( fmt.Sprintf("BinlogPlayer/%d", blp.uid), "transactions", 1, /* threadCount */ - maxTPS, - maxReplicationLag, + settings.MaxTPS, + settings.MaxReplicationLag, ) if err != nil { err := fmt.Errorf("failed to instantiate throttler: %v", err) @@ -425,7 +421,7 @@ func (blp *BinlogPlayer) exec(sql string) (*sqltypes.Result, error) { queryStartTime := time.Now() qr, err := blp.dbClient.ExecuteFetch(sql, 0) blp.blplStats.Timings.Record(BlplQuery, queryStartTime) - if d := time.Now().Sub(queryStartTime); d > SlowQueryThreshold { + if d := time.Since(queryStartTime); d > SlowQueryThreshold { log.Infof("SLOW QUERY (took %.2fs) '%s'", d.Seconds(), sql) } return qr, err @@ -499,8 +495,15 @@ func CreateVReplicationTable() []string { transaction_timestamp BIGINT(20) NOT NULL, state VARBINARY(100) NOT NULL, message VARBINARY(1000) DEFAULT NULL, + db_name VARBINARY(255) NOT NULL, PRIMARY KEY (id) -) ENGINE=InnoDB`} +) ENGINE=InnoDB`, + } +} + +// AlterVReplicationTable adds new columns to vreplication table +func AlterVReplicationTable() []string { + return []string{"ALTER TABLE _vt.vreplication ADD COLUMN db_name VARBINARY(255) NOT NULL"} } // SetVReplicationState updates the state in the _vt.vreplication table. @@ -512,46 +515,70 @@ func SetVReplicationState(dbClient DBClient, uid uint32, state, message string) return nil } +// VRSettings contains the settings of a vreplication table. +type VRSettings struct { + StartPos mysql.Position + StopPos mysql.Position + MaxTPS int64 + MaxReplicationLag int64 + State string +} + // ReadVRSettings retrieves the throttler settings for // vreplication from the checkpoint table. -func ReadVRSettings(dbClient DBClient, uid uint32) (pos, stopPos string, maxTPS, maxReplicationLag int64, err error) { - query := fmt.Sprintf("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=%v", uid) +func ReadVRSettings(dbClient DBClient, uid uint32) (VRSettings, error) { + query := fmt.Sprintf("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=%v", uid) qr, err := dbClient.ExecuteFetch(query, 1) if err != nil { - return "", "", throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("error %v in selecting vreplication settings %v", err, query) + return VRSettings{}, fmt.Errorf("error %v in selecting vreplication settings %v", err, query) } - if qr.RowsAffected != 1 { - return "", "", throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("checkpoint information not available in db for %v", uid) + if len(qr.Rows) != 1 { + return VRSettings{}, fmt.Errorf("checkpoint information not available in db for %v", uid) } + vrRow := qr.Rows[0] - maxTPS, err = sqltypes.ToInt64(qr.Rows[0][2]) + maxTPS, err := sqltypes.ToInt64(vrRow[2]) + if err != nil { + return VRSettings{}, fmt.Errorf("failed to parse max_tps column: %v", err) + } + maxReplicationLag, err := sqltypes.ToInt64(vrRow[3]) + if err != nil { + return VRSettings{}, fmt.Errorf("failed to parse max_replication_lag column: %v", err) + } + startPos, err := mysql.DecodePosition(vrRow[0].ToString()) if err != nil { - return "", "", throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("failed to parse max_tps column: %v", err) + return VRSettings{}, fmt.Errorf("failed to parse pos column: %v", err) } - maxReplicationLag, err = sqltypes.ToInt64(qr.Rows[0][3]) + stopPos, err := mysql.DecodePosition(vrRow[1].ToString()) if err != nil { - return "", "", throttler.InvalidMaxRate, throttler.InvalidMaxReplicationLag, fmt.Errorf("failed to parse max_replication_lag column: %v", err) + return VRSettings{}, fmt.Errorf("failed to parse stop_pos column: %v", err) } - return qr.Rows[0][0].ToString(), qr.Rows[0][1].ToString(), maxTPS, maxReplicationLag, nil + return VRSettings{ + StartPos: startPos, + StopPos: stopPos, + MaxTPS: maxTPS, + MaxReplicationLag: maxReplicationLag, + State: vrRow[4].ToString(), + }, nil } // CreateVReplication returns a statement to populate the first value into // the _vt.vreplication table. -func CreateVReplication(workflow string, source *binlogdatapb.BinlogSource, position string, maxTPS, maxReplicationLag, timeUpdated int64) string { +func CreateVReplication(workflow string, source *binlogdatapb.BinlogSource, position string, maxTPS, maxReplicationLag, timeUpdated int64, dbName string) string { return fmt.Sprintf("insert into _vt.vreplication "+ - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state) "+ - "values (%v, %v, %v, %v, %v, %v, 0, '%v')", - encodeString(workflow), encodeString(source.String()), encodeString(position), maxTPS, maxReplicationLag, timeUpdated, BlpRunning) + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) "+ + "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v)", + encodeString(workflow), encodeString(source.String()), encodeString(position), maxTPS, maxReplicationLag, timeUpdated, BlpRunning, encodeString(dbName)) } -// CreateVReplicationStopped returns a statement to create a stopped vreplication. -func CreateVReplicationStopped(workflow string, source *binlogdatapb.BinlogSource, position string) string { +// CreateVReplicationState returns a statement to create a stopped vreplication. +func CreateVReplicationState(workflow string, source *binlogdatapb.BinlogSource, position, state string, dbName string) string { return fmt.Sprintf("insert into _vt.vreplication "+ - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state) "+ - "values (%v, %v, %v, %v, %v, %v, 0, '%v')", - encodeString(workflow), encodeString(source.String()), encodeString(position), throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), BlpStopped) + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) "+ + "values (%v, %v, %v, %v, %v, %v, 0, '%v', %v)", + encodeString(workflow), encodeString(source.String()), encodeString(position), throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), state, encodeString(dbName)) } // GenerateUpdatePos returns a statement to update a value in the @@ -559,12 +586,12 @@ func CreateVReplicationStopped(workflow string, source *binlogdatapb.BinlogSourc func GenerateUpdatePos(uid uint32, pos mysql.Position, timeUpdated int64, txTimestamp int64) string { if txTimestamp != 0 { return fmt.Sprintf( - "update _vt.vreplication set pos=%v, time_updated=%v, transaction_timestamp=%v where id=%v", + "update _vt.vreplication set pos=%v, time_updated=%v, transaction_timestamp=%v, message='' where id=%v", encodeString(mysql.EncodePosition(pos)), timeUpdated, txTimestamp, uid) } return fmt.Sprintf( - "update _vt.vreplication set pos=%v, time_updated=%v where id=%v", + "update _vt.vreplication set pos=%v, time_updated=%v, message='' where id=%v", encodeString(mysql.EncodePosition(pos)), timeUpdated, uid) } diff --git a/go/vt/binlog/binlogplayer/binlog_player_test.go b/go/vt/binlog/binlogplayer/binlog_player_test.go index 4493dc33a37..dedcfcc0858 100644 --- a/go/vt/binlog/binlogplayer/binlog_player_test.go +++ b/go/vt/binlog/binlogplayer/binlog_player_test.go @@ -41,6 +41,7 @@ var ( sqltypes.NULL, // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } @@ -51,7 +52,7 @@ var ( func TestNewBinlogPlayerKeyRange(t *testing.T) { dbClient := NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -82,7 +83,7 @@ func TestNewBinlogPlayerKeyRange(t *testing.T) { func TestNewBinlogPlayerTables(t *testing.T) { dbClient := NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -114,7 +115,7 @@ func TestNewBinlogPlayerTables(t *testing.T) { func TestApplyEventsFail(t *testing.T) { dbClient := NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, errors.New("err")) dbClient.ExpectRequest("update _vt.vreplication set state='Error', message='error in processing binlog event failed query BEGIN, err: err' where id=1", testDMLResponse, nil) @@ -145,10 +146,11 @@ func TestStopPosEqual(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1083"), // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", posEqual, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", posEqual, nil) dbClient.ExpectRequest(`update _vt.vreplication set state='Stopped', message='not starting BinlogPlayer, we\'re already at the desired position 0-1-1083' where id=1`, testDMLResponse, nil) _ = newFakeBinlogClient() @@ -177,10 +179,11 @@ func TestStopPosLess(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1082"), // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", posEqual, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", posEqual, nil) dbClient.ExpectRequest(`update _vt.vreplication set state='Stopped', message='starting point 0-1-1083 greater than stopping point 0-1-1082' where id=1`, testDMLResponse, nil) _ = newFakeBinlogClient() @@ -209,10 +212,11 @@ func TestStopPosGreater(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", posEqual, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", posEqual, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -245,10 +249,11 @@ func TestContextCancel(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1085"), // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", posEqual, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", posEqual, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -275,7 +280,7 @@ func TestContextCancel(t *testing.T) { func TestRetryOnDeadlock(t *testing.T) { dbClient := NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) deadlocked := &mysql.SQLError{Num: 1213, Message: "deadlocked"} dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", nil, deadlocked) @@ -315,8 +320,8 @@ func applyEvents(blp *BinlogPlayer) func() error { func TestCreateVReplicationKeyRange(t *testing.T) { want := "insert into _vt.vreplication " + - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state) " + - `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" key_range: ', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running')` + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) " + + `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" key_range: ', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db')` bls := binlogdatapb.BinlogSource{ Keyspace: "ks", @@ -326,7 +331,7 @@ func TestCreateVReplicationKeyRange(t *testing.T) { }, } - got := CreateVReplication("Resharding", &bls, "MariaDB/0-1-1083", throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, 481823) + got := CreateVReplication("Resharding", &bls, "MariaDB/0-1-1083", throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, 481823, "db") if got != want { t.Errorf("CreateVReplication() =\n%v, want\n%v", got, want) } @@ -334,8 +339,8 @@ func TestCreateVReplicationKeyRange(t *testing.T) { func TestCreateVReplicationTables(t *testing.T) { want := "insert into _vt.vreplication " + - "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state) " + - `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" tables:\"a\" tables:\"b\" ', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running')` + "(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state, db_name) " + + `values ('Resharding', 'keyspace:\"ks\" shard:\"0\" tables:\"a\" tables:\"b\" ', 'MariaDB/0-1-1083', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running', 'db')` bls := binlogdatapb.BinlogSource{ Keyspace: "ks", @@ -343,7 +348,7 @@ func TestCreateVReplicationTables(t *testing.T) { Tables: []string{"a", "b"}, } - got := CreateVReplication("Resharding", &bls, "MariaDB/0-1-1083", throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, 481823) + got := CreateVReplication("Resharding", &bls, "MariaDB/0-1-1083", throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, 481823, "db") if got != want { t.Errorf("CreateVReplication() =\n%v, want\n%v", got, want) } @@ -352,7 +357,7 @@ func TestCreateVReplicationTables(t *testing.T) { func TestUpdateVReplicationPos(t *testing.T) { gtid := mysql.MustParseGTID("MariaDB", "0-1-8283") want := "update _vt.vreplication " + - "set pos='MariaDB/0-1-8283', time_updated=88822 " + + "set pos='MariaDB/0-1-8283', time_updated=88822, message='' " + "where id=78522" got := GenerateUpdatePos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 0) @@ -364,7 +369,7 @@ func TestUpdateVReplicationPos(t *testing.T) { func TestUpdateVReplicationTimestamp(t *testing.T) { gtid := mysql.MustParseGTID("MariaDB", "0-2-582") want := "update _vt.vreplication " + - "set pos='MariaDB/0-2-582', time_updated=88822, transaction_timestamp=481828 " + + "set pos='MariaDB/0-2-582', time_updated=88822, transaction_timestamp=481828, message='' " + "where id=78522" got := GenerateUpdatePos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 481828) diff --git a/go/vt/binlog/binlogplayer/fake_dbclient.go b/go/vt/binlog/binlogplayer/fake_dbclient.go index 24bb2a4b043..ec6237073f1 100644 --- a/go/vt/binlog/binlogplayer/fake_dbclient.go +++ b/go/vt/binlog/binlogplayer/fake_dbclient.go @@ -53,7 +53,6 @@ func (dc *fakeDBClient) Rollback() error { } func (dc *fakeDBClient) Close() { - return } func (dc *fakeDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { diff --git a/go/vt/binlog/binlogplayer/mock_dbclient.go b/go/vt/binlog/binlogplayer/mock_dbclient.go index 749ff49cd44..fd606c7d482 100644 --- a/go/vt/binlog/binlogplayer/mock_dbclient.go +++ b/go/vt/binlog/binlogplayer/mock_dbclient.go @@ -123,7 +123,6 @@ func (dc *MockDBClient) Rollback() error { // Close is part of the DBClient interface func (dc *MockDBClient) Close() { - return } // ExecuteFetch is part of the DBClient interface diff --git a/go/vt/binlog/binlogplayertest/player.go b/go/vt/binlog/binlogplayertest/player.go index c8c0155a8d0..161f16397d7 100644 --- a/go/vt/binlog/binlogplayertest/player.go +++ b/go/vt/binlog/binlogplayertest/player.go @@ -217,7 +217,7 @@ func testStreamTablesPanics(t *testing.T, bpc binlogplayer.Client) { // HandlePanic is part of the UpdateStream interface func (fake *FakeBinlogStreamer) HandlePanic(err *error) { if x := recover(); x != nil { - *err = fmt.Errorf("Caught panic: %v", x) + *err = fmt.Errorf("caught panic: %v", x) } } diff --git a/go/vt/dbconfigs/dbconfigs.go b/go/vt/dbconfigs/dbconfigs.go index c8a86106b16..24764d37bbb 100644 --- a/go/vt/dbconfigs/dbconfigs.go +++ b/go/vt/dbconfigs/dbconfigs.go @@ -237,27 +237,23 @@ func HasConnectionParams() bool { // is used to initialize the per-user conn params. func Init(defaultSocketFile string) (*DBConfigs, error) { // The new base configs, if set, supersede legacy settings. - if HasConnectionParams() { - for _, uc := range dbConfigs.userConfigs { + for _, uc := range dbConfigs.userConfigs { + if HasConnectionParams() { uc.param.Host = baseConfig.Host uc.param.Port = baseConfig.Port uc.param.UnixSocket = baseConfig.UnixSocket - uc.param.Charset = baseConfig.Charset - uc.param.Flags = baseConfig.Flags - if uc.useSSL { - uc.param.SslCa = baseConfig.SslCa - uc.param.SslCaPath = baseConfig.SslCaPath - uc.param.SslCert = baseConfig.SslCert - uc.param.SslKey = baseConfig.SslKey - uc.param.ServerName = baseConfig.ServerName - } + } else if uc.param.UnixSocket == "" && uc.param.Host == "" { + uc.param.UnixSocket = defaultSocketFile } - } else { - // Use supplied socket value if conn parameters are not specified. - for _, uc := range dbConfigs.userConfigs { - if uc.param.UnixSocket == "" && uc.param.Host == "" { - uc.param.UnixSocket = defaultSocketFile - } + + uc.param.Charset = baseConfig.Charset + uc.param.Flags = baseConfig.Flags + if uc.useSSL { + uc.param.SslCa = baseConfig.SslCa + uc.param.SslCaPath = baseConfig.SslCaPath + uc.param.SslCert = baseConfig.SslCert + uc.param.SslKey = baseConfig.SslKey + uc.param.ServerName = baseConfig.ServerName } } diff --git a/go/vt/discovery/healthcheck.go b/go/vt/discovery/healthcheck.go index 8a840dd3471..2e5e527b9f8 100644 --- a/go/vt/discovery/healthcheck.go +++ b/go/vt/discovery/healthcheck.go @@ -77,9 +77,7 @@ var ( const ( DefaultHealthCheckRetryDelay = 5 * time.Second DefaultHealthCheckTimeout = 1 * time.Minute -) -const ( // DefaultTopoReadConcurrency can be used as default value for the topoReadConcurrency parameter of a TopologyWatcher. DefaultTopoReadConcurrency int = 5 // DefaultTopologyWatcherRefreshInterval can be used as the default value for @@ -576,6 +574,10 @@ func (hc *HealthCheckImpl) checkConn(hcc *healthCheckConn, name string) { case <-time.After(retryDelay): // Exponentially back-off to prevent tight-loop. retryDelay *= 2 + // Limit the retry delay backoff to the health check timeout + if retryDelay > hc.healthCheckTimeout { + retryDelay = hc.healthCheckTimeout + } } } } diff --git a/go/vt/key/key.go b/go/vt/key/key.go index 88c51e26f22..367311ba5c8 100644 --- a/go/vt/key/key.go +++ b/go/vt/key/key.go @@ -144,7 +144,7 @@ func ParseKeyRangeParts(start, end string) (*topodatapb.KeyRange, error) { // KeyRangeString prints a topodatapb.KeyRange func KeyRangeString(k *topodatapb.KeyRange) string { if k == nil { - return "" + return "-" } return hex.EncodeToString(k.Start) + "-" + hex.EncodeToString(k.End) } diff --git a/go/vt/mysqlctl/backup.go b/go/vt/mysqlctl/backup.go index 2248cfb4bf2..1b8f498e6f5 100644 --- a/go/vt/mysqlctl/backup.go +++ b/go/vt/mysqlctl/backup.go @@ -19,7 +19,6 @@ package mysqlctl import ( "errors" "flag" - "fmt" "os" "path/filepath" "strings" @@ -31,6 +30,8 @@ import ( "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" ) // This file handles the backup and restore related code @@ -86,17 +87,17 @@ func Backup(ctx context.Context, cnf *Mycnf, mysqld MysqlDaemon, logger logutil. // Start the backup with the BackupStorage. bs, err := backupstorage.GetBackupStorage() if err != nil { - return err + return vterrors.Wrap(err, "unable to get backup storage") } defer bs.Close() bh, err := bs.StartBackup(ctx, dir, name) if err != nil { - return fmt.Errorf("StartBackup failed: %v", err) + return vterrors.Wrap(err, "StartBackup failed") } be, err := GetBackupEngine() if err != nil { - return fmt.Errorf("Failed to find backup engine: %v", err) + return vterrors.Wrap(err, "failed to find backup engine") } // Take the backup, and either AbortBackup or EndBackup. @@ -132,7 +133,7 @@ func Backup(ctx context.Context, cnf *Mycnf, mysqld MysqlDaemon, logger logutil. func checkNoDB(ctx context.Context, mysqld MysqlDaemon, dbName string) (bool, error) { qr, err := mysqld.FetchSuperQuery(ctx, "SHOW DATABASES") if err != nil { - return false, fmt.Errorf("checkNoDB failed: %v", err) + return false, vterrors.Wrap(err, "checkNoDB failed") } backtickDBName := sqlescape.EscapeID(dbName) @@ -140,7 +141,7 @@ func checkNoDB(ctx context.Context, mysqld MysqlDaemon, dbName string) (bool, er if row[0].ToString() == dbName { tableQr, err := mysqld.FetchSuperQuery(ctx, "SHOW TABLES FROM "+backtickDBName) if err != nil { - return false, fmt.Errorf("checkNoDB failed: %v", err) + return false, vterrors.Wrap(err, "checkNoDB failed") } if len(tableQr.Rows) == 0 { // no tables == empty db, all is well @@ -171,7 +172,7 @@ func removeExistingFiles(cnf *Mycnf) error { } for name, path := range paths { if path == "" { - return fmt.Errorf("can't remove existing files: %v is unknown", name) + return vterrors.Errorf(vtrpc.Code_UNKNOWN, "can't remove existing files: %v is unknown", name) } if strings.HasSuffix(name, ".*") { @@ -181,11 +182,11 @@ func removeExistingFiles(cnf *Mycnf) error { log.Infof("Restore: removing files in %v (%v)", name, path) matches, err := filepath.Glob(path) if err != nil { - return fmt.Errorf("can't expand path glob %q: %v", path, err) + return vterrors.Wrapf(err, "can't expand path glob %q", path) } for _, match := range matches { if err := os.Remove(match); err != nil { - return fmt.Errorf("can't remove existing file from %v (%v): %v", name, match, err) + return vterrors.Wrapf(err, "can't remove existing file from %v (%v)", name, match) } } continue @@ -198,7 +199,7 @@ func removeExistingFiles(cnf *Mycnf) error { } log.Infof("Restore: removing files in %v (%v)", name, path) if err := os.RemoveAll(path); err != nil { - return fmt.Errorf("can't remove existing files in %v (%v): %v", name, path, err) + return vterrors.Wrapf(err, "can't remove existing files in %v (%v)", name, path) } } return nil @@ -234,7 +235,7 @@ func Restore( } if !ok { logger.Infof("Auto-restore is enabled, but mysqld already contains data. Assuming vttablet was just restarted.") - if err = PopulateMetadataTables(mysqld, localMetadata); err == nil { + if err = PopulateMetadataTables(mysqld, localMetadata, dbName); err == nil { err = ErrExistingDB } return mysql.Position{}, err @@ -251,19 +252,19 @@ func Restore( bhs, err := bs.ListBackups(ctx, dir) if err != nil { - return mysql.Position{}, fmt.Errorf("ListBackups failed: %v", err) + return mysql.Position{}, vterrors.Wrap(err, "ListBackups failed") } if len(bhs) == 0 { // There are no backups (not even broken/incomplete ones). - logger.Errorf("No backup to restore on BackupStorage for directory %v. Starting up empty.", dir) + logger.Errorf("no backup to restore on BackupStorage for directory %v. Starting up empty.", dir) // Since this is an empty database make sure we start replication at the beginning if err = mysqld.ResetReplication(ctx); err == nil { - logger.Errorf("Error reseting slave replication: %v. Continuing", err) + logger.Errorf("error reseting slave replication: %v. Continuing", err) err = ErrNoBackup } - if err2 := PopulateMetadataTables(mysqld, localMetadata); err2 == nil { + if err2 := PopulateMetadataTables(mysqld, localMetadata, dbName); err2 == nil { err = ErrNoBackup } return mysql.Position{}, err @@ -271,7 +272,7 @@ func Restore( be, err := GetBackupEngine() if err != nil { - return mysql.Position{}, fmt.Errorf("Failed to find backup engine: %v", err) + return mysql.Position{}, vterrors.Wrap(err, "Failed to find backup engine") } if rval, err = be.ExecuteRestore(ctx, cnf, mysqld, logger, dir, bhs, restoreConcurrency, hookExtraEnv); err != nil { return rval, err @@ -293,13 +294,13 @@ func Restore( logger.Infof("Restore: running mysql_upgrade") if err := mysqld.RunMysqlUpgrade(); err != nil { - return mysql.Position{}, fmt.Errorf("mysql_upgrade failed: %v", err) + return mysql.Position{}, vterrors.Wrap(err, "mysql_upgrade failed") } // Populate local_metadata before starting without --skip-networking, // so it's there before we start announcing ourselves. logger.Infof("Restore: populating local_metadata") - err = PopulateMetadataTables(mysqld, localMetadata) + err = PopulateMetadataTables(mysqld, localMetadata, dbName) if err != nil { return mysql.Position{}, err } diff --git a/go/vt/mysqlctl/backupengine.go b/go/vt/mysqlctl/backupengine.go index c793a0493bb..725ea894ae3 100644 --- a/go/vt/mysqlctl/backupengine.go +++ b/go/vt/mysqlctl/backupengine.go @@ -19,17 +19,17 @@ package mysqlctl import ( "context" "flag" - "fmt" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" ) var ( - // BackupEngineImplementation is the implementation to use - // for BackupEngine. Exported for test purposes. - BackupEngineImplementation = flag.String("backup_engine_implementation", "builtin", "which implementation to use for the backup storage engine") + // BackupEngineImplementation is the implementation to use for BackupEngine + backupEngineImplementation = flag.String("backup_engine_implementation", builtin, "which implementation to use for the backup method, builtin or xtrabackup") ) // BackupEngine is the interface to the backup engine @@ -44,9 +44,9 @@ var BackupEngineMap = make(map[string]BackupEngine) // GetBackupEngine returns the current BackupEngine implementation. // Should be called after flags have been initialized. func GetBackupEngine() (BackupEngine, error) { - be, ok := BackupEngineMap[*BackupEngineImplementation] + be, ok := BackupEngineMap[*backupEngineImplementation] if !ok { - return nil, fmt.Errorf("no registered implementation of BackupEngine") + return nil, vterrors.New(vtrpc.Code_NOT_FOUND, "no registered implementation of BackupEngine") } return be, nil } diff --git a/go/vt/mysqlctl/builtinbackupengine.go b/go/vt/mysqlctl/builtinbackupengine.go index 38a23b85d4f..1ed80a8f12b 100644 --- a/go/vt/mysqlctl/builtinbackupengine.go +++ b/go/vt/mysqlctl/builtinbackupengine.go @@ -37,6 +37,14 @@ import ( "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/mysqlctl/backupstorage" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" +) + +const ( + builtin = "builtin" + writerBufferSize = 2 * 1024 * 1024 + dataDictionaryFile = "mysql.ibd" ) // BuiltinBackupEngine encapsulates the logic of the builtin engine @@ -46,10 +54,10 @@ import ( type BuiltinBackupEngine struct { } -// BackupManifest represents the backup. It lists all the files, the +// builtinBackupManifest represents the backup. It lists all the files, the // Position that the backup was taken at, and the transform hook used, // if any. -type BackupManifest struct { +type builtinBackupManifest struct { // FileEntries contains all the files in the backup FileEntries []FileEntry @@ -93,7 +101,7 @@ func (fe *FileEntry) open(cnf *Mycnf, readOnly bool) (*os.File, error) { case backupData: root = cnf.DataDir default: - return nil, fmt.Errorf("unknown base: %v", fe.Base) + return nil, vterrors.Errorf(vtrpc.Code_UNKNOWN, "unknown base: %v", fe.Base) } // and open the file @@ -102,15 +110,15 @@ func (fe *FileEntry) open(cnf *Mycnf, readOnly bool) (*os.File, error) { var err error if readOnly { if fd, err = os.Open(name); err != nil { - return nil, fmt.Errorf("cannot open source file %v: %v", name, err) + return nil, vterrors.Wrapf(err, "cannot open source file %v", name) } } else { dir := path.Dir(name) if err := os.MkdirAll(dir, os.ModePerm); err != nil { - return nil, fmt.Errorf("cannot create destination directory %v: %v", dir, err) + return nil, vterrors.Wrapf(err, "cannot create destination directory %v", dir) } if fd, err = os.Create(name); err != nil { - return nil, fmt.Errorf("cannot create destination file %v: %v", name, err) + return nil, vterrors.Wrapf(err, "cannot create destination file %v", name) } } return fd, nil @@ -173,7 +181,6 @@ func addDirectory(fes []FileEntry, base string, baseDir string, subDir string) ( // and adds it to the backup manifest if it does // https://dev.mysql.com/doc/refman/8.0/en/data-dictionary-transactional-storage.html func addMySQL8DataDictionary(fes []FileEntry, base string, baseDir string) ([]FileEntry, error) { - const dataDictionaryFile = "mysql.ibd" filePath := path.Join(baseDir, dataDictionaryFile) // no-op if this file doesn't exist @@ -250,13 +257,13 @@ func (be *BuiltinBackupEngine) ExecuteBackup(ctx context.Context, cnf *Mycnf, my // keep going if we're the master, might be a degenerate case sourceIsMaster = true default: - return false, fmt.Errorf("can't get slave status: %v", err) + return false, vterrors.Wrap(err, "can't get slave status") } // get the read-only flag readOnly, err = mysqld.IsReadOnly() if err != nil { - return false, fmt.Errorf("can't get read-only status: %v", err) + return false, vterrors.Wrap(err, "can't get read-only status") } // get the replication position @@ -264,21 +271,21 @@ func (be *BuiltinBackupEngine) ExecuteBackup(ctx context.Context, cnf *Mycnf, my if !readOnly { logger.Infof("turning master read-only before backup") if err = mysqld.SetReadOnly(true); err != nil { - return false, fmt.Errorf("can't set read-only status: %v", err) + return false, vterrors.Wrap(err, "can't set read-only status") } } replicationPosition, err = mysqld.MasterPosition() if err != nil { - return false, fmt.Errorf("can't get master position: %v", err) + return false, vterrors.Wrap(err, "can't get master position") } } else { if err = mysqld.StopSlave(hookExtraEnv); err != nil { - return false, fmt.Errorf("can't stop slave: %v", err) + return false, vterrors.Wrapf(err, "can't stop slave") } var slaveStatus mysql.SlaveStatus slaveStatus, err = mysqld.SlaveStatus() if err != nil { - return false, fmt.Errorf("can't get slave status: %v", err) + return false, vterrors.Wrap(err, "can't get slave status") } replicationPosition = slaveStatus.Position } @@ -287,17 +294,17 @@ func (be *BuiltinBackupEngine) ExecuteBackup(ctx context.Context, cnf *Mycnf, my // shutdown mysqld err = mysqld.Shutdown(ctx, cnf, true) if err != nil { - return false, fmt.Errorf("can't shutdown mysqld: %v", err) + return false, vterrors.Wrap(err, "can't shutdown mysqld") } // Backup everything, capture the error. backupErr := be.backupFiles(ctx, cnf, mysqld, logger, bh, replicationPosition, backupConcurrency, hookExtraEnv) usable := backupErr == nil - // Try to restart mysqld - err = mysqld.Start(ctx, cnf) + // Try to restart mysqld, use background context in case we timed out the original context + err = mysqld.Start(context.Background(), cnf) if err != nil { - return usable, fmt.Errorf("can't restart mysqld: %v", err) + return usable, vterrors.Wrap(err, "can't restart mysqld") } // Restore original mysqld state that we saved above. @@ -314,12 +321,12 @@ func (be *BuiltinBackupEngine) ExecuteBackup(ctx context.Context, cnf *Mycnf, my if slaveStartRequired { logger.Infof("restarting mysql replication") if err := mysqld.StartSlave(hookExtraEnv); err != nil { - return usable, fmt.Errorf("cannot restart slave: %v", err) + return usable, vterrors.Wrap(err, "cannot restart slave") } // this should be quick, but we might as well just wait if err := WaitForSlaveStart(mysqld, slaveStartDeadline); err != nil { - return usable, fmt.Errorf("slave is not restarting: %v", err) + return usable, vterrors.Wrap(err, "slave is not restarting") } } @@ -337,7 +344,7 @@ func (be *BuiltinBackupEngine) backupFiles(ctx context.Context, cnf *Mycnf, mysq // Get the files to backup. fes, err := findFilesToBackup(cnf) if err != nil { - return fmt.Errorf("can't find files to backup: %v", err) + return vterrors.Wrap(err, "can't find files to backup") } logger.Infof("found %v files to backup", len(fes)) @@ -372,7 +379,7 @@ func (be *BuiltinBackupEngine) backupFiles(ctx context.Context, cnf *Mycnf, mysq // open the MANIFEST wc, err := bh.AddFile(ctx, backupManifest, 0) if err != nil { - return fmt.Errorf("cannot add %v to backup: %v", backupManifest, err) + return vterrors.Wrapf(err, "cannot add %v to backup", backupManifest) } defer func() { if closeErr := wc.Close(); err == nil { @@ -381,7 +388,7 @@ func (be *BuiltinBackupEngine) backupFiles(ctx context.Context, cnf *Mycnf, mysq }() // JSON-encode and write the MANIFEST - bm := &BackupManifest{ + bm := &builtinBackupManifest{ FileEntries: fes, Position: replicationPosition, TransformHook: *backupStorageHook, @@ -389,10 +396,10 @@ func (be *BuiltinBackupEngine) backupFiles(ctx context.Context, cnf *Mycnf, mysq } data, err := json.MarshalIndent(bm, "", " ") if err != nil { - return fmt.Errorf("cannot JSON encode %v: %v", backupManifest, err) + return vterrors.Wrapf(err, "cannot JSON encode %v", backupManifest) } if _, err := wc.Write([]byte(data)); err != nil { - return fmt.Errorf("cannot write %v: %v", backupManifest, err) + return vterrors.Wrapf(err, "cannot write %v", backupManifest) } return nil @@ -413,22 +420,23 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, cnf *Mycnf, mysql return err } + logger.Infof("Backing up file: %v", fe.Name) // Open the destination file for writing, and a buffer. wc, err := bh.AddFile(ctx, name, fi.Size()) if err != nil { - return fmt.Errorf("cannot add file: %v", err) + return vterrors.Wrapf(err, "cannot add file: %v,%v", name, fe.Name) } - defer func() { + defer func(name, fileName string) { if rerr := wc.Close(); rerr != nil { if err != nil { // We already have an error, just log this one. - logger.Errorf2(rerr, "failed to close file %v", name) + logger.Errorf2(rerr, "failed to close file %v,%v", name, fe.Name) } else { err = rerr } } - }() - dst := bufio.NewWriterSize(wc, 2*1024*1024) + }(name, fe.Name) + dst := bufio.NewWriterSize(wc, writerBufferSize) // Create the hasher and the tee on top. hasher := newHasher() @@ -442,7 +450,7 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, cnf *Mycnf, mysql h.ExtraEnv = hookExtraEnv pipe, wait, _, err = h.ExecuteAsWritePipe(writer) if err != nil { - return fmt.Errorf("'%v' hook returned error: %v", *backupStorageHook, err) + return vterrors.Wrapf(err, "'%v' hook returned error", *backupStorageHook) } writer = pipe } @@ -452,7 +460,7 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, cnf *Mycnf, mysql if *backupStorageCompress { gzip, err = pgzip.NewWriterLevel(writer, pgzip.BestSpeed) if err != nil { - return fmt.Errorf("cannot create gziper: %v", err) + return vterrors.Wrap(err, "cannot create gziper") } gzip.SetConcurrency(*backupCompressBlockSize, *backupCompressBlocks) writer = gzip @@ -462,33 +470,33 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, cnf *Mycnf, mysql // optional pipe, tee, output file and hasher). _, err = io.Copy(writer, source) if err != nil { - return fmt.Errorf("cannot copy data: %v", err) + return vterrors.Wrap(err, "cannot copy data") } // Close gzip to flush it, after that all data is sent to writer. if gzip != nil { if err = gzip.Close(); err != nil { - return fmt.Errorf("cannot close gzip: %v", err) + return vterrors.Wrap(err, "cannot close gzip") } } // Close the hook pipe if necessary. if pipe != nil { if err := pipe.Close(); err != nil { - return fmt.Errorf("cannot close hook pipe: %v", err) + return vterrors.Wrap(err, "cannot close hook pipe") } stderr, err := wait() if stderr != "" { logger.Infof("'%v' hook returned stderr: %v", *backupStorageHook, stderr) } if err != nil { - return fmt.Errorf("'%v' returned error: %v", *backupStorageHook, err) + return vterrors.Wrapf(err, "'%v' returned error", *backupStorageHook) } } // Flush the buffer to finish writing on destination. if err = dst.Flush(); err != nil { - return fmt.Errorf("cannot flush dst: %v", err) + return vterrors.Wrapf(err, "cannot flush destination: %v", name) } // Save the hash. @@ -510,7 +518,7 @@ func (be *BuiltinBackupEngine) ExecuteRestore( hookExtraEnv map[string]string) (mysql.Position, error) { var bh backupstorage.BackupHandle - var bm BackupManifest + var bm builtinBackupManifest var toRestore int for toRestore = len(bhs) - 1; toRestore >= 0; toRestore-- { @@ -637,7 +645,7 @@ func (be *BuiltinBackupEngine) restoreFile(ctx context.Context, cnf *Mycnf, bh b h.ExtraEnv = hookExtraEnv reader, wait, _, err = h.ExecuteAsReadPipe(reader) if err != nil { - return fmt.Errorf("'%v' hook returned error: %v", transformHook, err) + return vterrors.Wrapf(err, "'%v' hook returned error", transformHook) } } @@ -672,14 +680,14 @@ func (be *BuiltinBackupEngine) restoreFile(ctx context.Context, cnf *Mycnf, bh b log.Infof("'%v' hook returned stderr: %v", transformHook, stderr) } if err != nil { - return fmt.Errorf("'%v' returned error: %v", transformHook, err) + return vterrors.Wrapf(err, "'%v' returned error", transformHook) } } // Check the hash. hash := hasher.HashString() if hash != fe.Hash { - return fmt.Errorf("hash mismatch for %v, got %v expected %v", fe.Name, hash, fe.Hash) + return vterrors.Errorf(vtrpc.Code_INTERNAL, "hash mismatch for %v, got %v expected %v", fe.Name, hash, fe.Hash) } // Flush the buffer. diff --git a/go/vt/mysqlctl/cephbackupstorage/ceph.go b/go/vt/mysqlctl/cephbackupstorage/ceph.go index d047f4af0e3..a98e5229d7e 100644 --- a/go/vt/mysqlctl/cephbackupstorage/ceph.go +++ b/go/vt/mysqlctl/cephbackupstorage/ceph.go @@ -268,7 +268,7 @@ func (bs *CephBackupStorage) client() (*minio.Client, error) { defer configFile.Close() jsonParser := json.NewDecoder(configFile) if err = jsonParser.Decode(&storageConfig); err != nil { - return nil, fmt.Errorf("Error parsing the json file : %v", err) + return nil, fmt.Errorf("error parsing the json file : %v", err) } accessKey := storageConfig.AccessKey diff --git a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go index 61b66338ec5..0487a7aa29e 100644 --- a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go @@ -76,6 +76,9 @@ type FakeMysqlDaemon struct { // ReadOnly is the current value of the flag ReadOnly bool + // SuperReadOnly is the current value of the flag + SuperReadOnly bool + // SetSlavePositionPos is matched against the input of SetSlavePosition. // If it doesn't match, SetSlavePosition will return an error. SetSlavePositionPos mysql.Position @@ -240,6 +243,13 @@ func (fmd *FakeMysqlDaemon) SetReadOnly(on bool) error { return nil } +// SetSuperReadOnly is part of the MysqlDaemon interface +func (fmd *FakeMysqlDaemon) SetSuperReadOnly(on bool) error { + fmd.SuperReadOnly = on + fmd.ReadOnly = on + return nil +} + // StartSlave is part of the MysqlDaemon interface. func (fmd *FakeMysqlDaemon) StartSlave(hookExtraEnv map[string]string) error { return fmd.ExecuteSuperQueryList(context.Background(), []string{ diff --git a/go/vt/mysqlctl/grpcmysqlctlclient/client.go b/go/vt/mysqlctl/grpcmysqlctlclient/client.go index aa6229be56b..5fdd18ba04c 100644 --- a/go/vt/mysqlctl/grpcmysqlctlclient/client.go +++ b/go/vt/mysqlctl/grpcmysqlctlclient/client.go @@ -25,6 +25,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "golang.org/x/net/context" @@ -115,10 +116,13 @@ func (c *client) withRetry(ctx context.Context, f func() error) error { default: } if err := f(); err != nil { - if grpc.Code(err) == codes.Unavailable { - lastError = err - time.Sleep(100 * time.Millisecond) - continue + if st, ok := status.FromError(err); ok { + code := st.Code() + if code == codes.Unavailable { + lastError = err + time.Sleep(100 * time.Millisecond) + continue + } } return err } diff --git a/go/vt/mysqlctl/metadata_tables.go b/go/vt/mysqlctl/metadata_tables.go index 5fd4aee1134..d1607723467 100644 --- a/go/vt/mysqlctl/metadata_tables.go +++ b/go/vt/mysqlctl/metadata_tables.go @@ -18,23 +18,40 @@ package mysqlctl import ( "bytes" + "fmt" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/log" ) // Note that definitions of local_metadata and shard_metadata should be the same // as in testing which is defined in config/init_db.sql. -const sqlCreateLocalMetadataTable = `CREATE TABLE IF NOT EXISTS _vt.local_metadata ( +const ( + sqlCreateLocalMetadataTable = `CREATE TABLE IF NOT EXISTS _vt.local_metadata ( name VARCHAR(255) NOT NULL, value VARCHAR(255) NOT NULL, PRIMARY KEY (name) ) ENGINE=InnoDB` -const sqlCreateShardMetadataTable = `CREATE TABLE IF NOT EXISTS _vt.shard_metadata ( + sqlCreateShardMetadataTable = `CREATE TABLE IF NOT EXISTS _vt.shard_metadata ( name VARCHAR(255) NOT NULL, value MEDIUMBLOB NOT NULL, PRIMARY KEY (name) ) ENGINE=InnoDB` + sqlUpdateLocalMetadataTable = "UPDATE _vt.local_metadata SET db_name='%s' WHERE db_name=''" + sqlUpdateShardMetadataTable = "UPDATE _vt.shard_metadata SET db_name='%s' WHERE db_name=''" +) + +var ( + sqlAlterLocalMetadataTable = []string{ + `ALTER TABLE _vt.local_metadata ADD COLUMN db_name VARBINARY(255) NOT NULL DEFAULT ''`, + `ALTER TABLE _vt.local_metadata DROP PRIMARY KEY, ADD PRIMARY KEY(name, db_name)`, + } + sqlAlterShardMetadataTable = []string{ + `ALTER TABLE _vt.shard_metadata ADD COLUMN db_name VARBINARY(255) NOT NULL DEFAULT ''`, + `ALTER TABLE _vt.shard_metadata DROP PRIMARY KEY, ADD PRIMARY KEY(name, db_name)`, + } +) // PopulateMetadataTables creates and fills the _vt.local_metadata table and // creates _vt.shard_metadata table. _vt.local_metadata table is @@ -46,7 +63,7 @@ const sqlCreateShardMetadataTable = `CREATE TABLE IF NOT EXISTS _vt.shard_metada // created here to make it easier to create it on databases that were running // old version of Vitess, or databases that are getting converted to run under // Vitess. -func PopulateMetadataTables(mysqld MysqlDaemon, localMetadata map[string]string) error { +func PopulateMetadataTables(mysqld MysqlDaemon, localMetadata map[string]string, dbName string) error { log.Infof("Populating _vt.local_metadata table...") // Get a non-pooled DBA connection. @@ -69,9 +86,35 @@ func PopulateMetadataTables(mysqld MysqlDaemon, localMetadata map[string]string) if _, err := conn.ExecuteFetch(sqlCreateLocalMetadataTable, 0, false); err != nil { return err } + for _, sql := range sqlAlterLocalMetadataTable { + if _, err := conn.ExecuteFetch(sql, 0, false); err != nil { + if merr, ok := err.(*mysql.SQLError); ok && merr.Num == mysql.ERDupFieldName { + log.Errorf("Expected error executing %v: %v", sql, err) + } else { + log.Errorf("Unexpected error executing %v: %v", sql, err) + return err + } + } + } + if _, err := conn.ExecuteFetch(fmt.Sprintf(sqlUpdateLocalMetadataTable, dbName), 0, false); err != nil { + return err + } if _, err := conn.ExecuteFetch(sqlCreateShardMetadataTable, 0, false); err != nil { return err } + for _, sql := range sqlAlterShardMetadataTable { + if _, err := conn.ExecuteFetch(sql, 0, false); err != nil { + if merr, ok := err.(*mysql.SQLError); ok && merr.Num == mysql.ERDupFieldName { + log.Errorf("Expected error executing %v: %v", sql, err) + } else { + log.Errorf("Unexpected error executing %v: %v", sql, err) + return err + } + } + } + if _, err := conn.ExecuteFetch(fmt.Sprintf(sqlUpdateShardMetadataTable, dbName), 0, false); err != nil { + return err + } // Populate local_metadata from the passed list of values. if _, err := conn.ExecuteFetch("BEGIN", 0, false); err != nil { @@ -80,12 +123,15 @@ func PopulateMetadataTables(mysqld MysqlDaemon, localMetadata map[string]string) for name, val := range localMetadata { nameValue := sqltypes.NewVarChar(name) valValue := sqltypes.NewVarChar(val) + dbNameValue := sqltypes.NewVarBinary(dbName) queryBuf := bytes.Buffer{} - queryBuf.WriteString("INSERT INTO _vt.local_metadata (name,value) VALUES (") + queryBuf.WriteString("INSERT INTO _vt.local_metadata (name,value, db_name) VALUES (") nameValue.EncodeSQL(&queryBuf) queryBuf.WriteByte(',') valValue.EncodeSQL(&queryBuf) + queryBuf.WriteByte(',') + dbNameValue.EncodeSQL(&queryBuf) queryBuf.WriteString(") ON DUPLICATE KEY UPDATE value = ") valValue.EncodeSQL(&queryBuf) diff --git a/go/vt/mysqlctl/mysql_daemon.go b/go/vt/mysqlctl/mysql_daemon.go index 9d6e18295d8..b3e62f9d3c8 100644 --- a/go/vt/mysqlctl/mysql_daemon.go +++ b/go/vt/mysqlctl/mysql_daemon.go @@ -53,6 +53,7 @@ type MysqlDaemon interface { MasterPosition() (mysql.Position, error) IsReadOnly() (bool, error) SetReadOnly(on bool) error + SetSuperReadOnly(on bool) error SetSlavePosition(ctx context.Context, pos mysql.Position) error SetMaster(ctx context.Context, masterHost string, masterPort int, slaveStopBefore bool, slaveStartAfter bool) error WaitForReparentJournal(ctx context.Context, timeCreatedNS int64) error diff --git a/go/vt/mysqlctl/mysqld.go b/go/vt/mysqlctl/mysqld.go index c491e67fef2..bfa973203a5 100644 --- a/go/vt/mysqlctl/mysqld.go +++ b/go/vt/mysqlctl/mysqld.go @@ -653,6 +653,11 @@ func getMycnfTemplates(root string) []string { if !contains(cnfTemplatePaths, path) { cnfTemplatePaths = append(cnfTemplatePaths, path) } + case "MySQL80": + path := path.Join(root, "config/mycnf/master_mysql80.cnf") + if !contains(cnfTemplatePaths, path) { + cnfTemplatePaths = append(cnfTemplatePaths, path) + } default: path := path.Join(root, "config/mycnf/master_mysql56.cnf") // By default we assume Mysql56 compatable diff --git a/go/vt/mysqlctl/query.go b/go/vt/mysqlctl/query.go index a5757d2a999..0051f5bd6d5 100644 --- a/go/vt/mysqlctl/query.go +++ b/go/vt/mysqlctl/query.go @@ -199,8 +199,10 @@ func (mysqld *Mysqld) fetchVariables(ctx context.Context, pattern string) (map[s return varMap, nil } -const masterPasswordStart = " MASTER_PASSWORD = '" -const masterPasswordEnd = "',\n" +const ( + masterPasswordStart = " MASTER_PASSWORD = '" + masterPasswordEnd = "',\n" +) func redactMasterPassword(input string) string { i := strings.Index(input, masterPasswordStart) diff --git a/go/vt/mysqlctl/replication.go b/go/vt/mysqlctl/replication.go index 8d926fb9e66..6774918733d 100644 --- a/go/vt/mysqlctl/replication.go +++ b/go/vt/mysqlctl/replication.go @@ -161,6 +161,17 @@ var ( ErrNotMaster = errors.New("no master status") ) +// SetSuperReadOnly set/unset the super_read_only flag +func (mysqld *Mysqld) SetSuperReadOnly(on bool) error { + query := "SET GLOBAL super_read_only = " + if on { + query += "ON" + } else { + query += "OFF" + } + return mysqld.ExecuteSuperQuery(context.TODO(), query) +} + // WaitMasterPos lets slaves wait to given replication position func (mysqld *Mysqld) WaitMasterPos(ctx context.Context, targetPos mysql.Position) error { // Get a connection. @@ -224,6 +235,7 @@ func (mysqld *Mysqld) SetSlavePosition(ctx context.Context, pos mysql.Position) defer conn.Recycle() cmds := conn.SetSlavePositionCommands(pos) + log.Infof("Executing commands to set slave position: %v", cmds) return mysqld.executeSuperQueryListConn(ctx, conn, cmds) } @@ -273,10 +285,12 @@ func (mysqld *Mysqld) ResetReplication(ctx context.Context) error { // // Array indices for the results of SHOW PROCESSLIST. const ( - //lint:ignore U1000 unused fields are needed for correct indexing of result columns + //lint:ignore U1000 needed for correct indexing of result columns colConnectionID = iota + //lint:ignore U1000 needed for correct indexing of result columns colUsername colClientAddr + //lint:ignore U1000 needed for correct indexing of result columns colDbName colCommand ) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index 40260884122..b6e3261b2ef 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -336,7 +336,11 @@ func (bs *S3BackupStorage) client() (*s3.S3, error) { httpTransport := &http.Transport{TLSClientConfig: tlsClientConf} httpClient := &http.Client{Transport: httpTransport} - bs._client = s3.New(session.New(), + session, err := session.NewSession() + if err != nil { + return nil, err + } + bs._client = s3.New(session, &aws.Config{ HTTPClient: httpClient, LogLevel: logLevel, diff --git a/go/vt/mysqlctl/xtrabackupengine.go b/go/vt/mysqlctl/xtrabackupengine.go new file mode 100644 index 00000000000..2399c1ffd40 --- /dev/null +++ b/go/vt/mysqlctl/xtrabackupengine.go @@ -0,0 +1,494 @@ +/* +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 mysqlctl + +import ( + "bufio" + "context" + "encoding/json" + "errors" + "flag" + "io" + "io/ioutil" + "os/exec" + "path" + "strings" + + "github.com/klauspost/pgzip" + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/mysqlctl/backupstorage" + "vitess.io/vitess/go/vt/proto/vtrpc" + "vitess.io/vitess/go/vt/vterrors" +) + +// XtrabackupEngine encapsulates the logic of the xtrabackup engine +// it implements the BackupEngine interface and contains all the logic +// required to implement a backup/restore by invoking xtrabackup with +// the appropriate parameters +type XtrabackupEngine struct { +} + +var ( + // path where backup engine program is located + xtrabackupEnginePath = flag.String("xtrabackup_root_path", "", "directory location of the xtrabackup executable, e.g., /usr/bin") + // flags to pass through to backup engine + xtrabackupBackupFlags = flag.String("xtrabackup_backup_flags", "", "flags to pass to backup command. These should be space separated and will be added to the end of the command") + // flags to pass through to restore phase + xbstreamRestoreFlags = flag.String("xbstream_restore_flags", "", "flags to pass to xbstream command during restore. These should be space separated and will be added to the end of the command. These need to match the ones used for backup e.g. --compress / --decompress, --encrypt / --decrypt") + // streaming mode + xtrabackupStreamMode = flag.String("xtrabackup_stream_mode", "tar", "which mode to use if streaming, valid values are tar and xbstream") + xtrabackupUser = flag.String("xtrabackup_user", "", "User that xtrabackup will use to connect to the database server. This user must have all necessary privileges. For details, please refer to xtrabackup documentation.") +) + +const ( + streamModeTar = "tar" + xtrabackupBinaryName = "xtrabackup" + xtrabackupBackupMethod = "xtrabackup" + xbstream = "xbstream" +) + +// xtraBackupManifest represents a backup. +// It stores the name of the backup file, the replication position, +// whether the backup is compressed using gzip, and any extra +// command line parameters used while invoking it. +type xtraBackupManifest struct { + // Name of the backup file + FileName string + // BackupMethod, set to xtrabackup + BackupMethod string + // Position at which the backup was taken + Position mysql.Position + // SkipCompress can be set if the backup files were not run + // through gzip. + SkipCompress bool + // Params are the parameters that backup was run with + Params string `json:"ExtraCommandLineParams"` +} + +func (be *XtrabackupEngine) backupFileName() string { + fileName := "backup" + if *xtrabackupStreamMode != "" { + fileName += "." + fileName += *xtrabackupStreamMode + } + if *backupStorageCompress { + fileName += ".gz" + } + return fileName +} + +// ExecuteBackup returns a boolean that indicates if the backup is usable, +// and an overall error. +func (be *XtrabackupEngine) ExecuteBackup(ctx context.Context, cnf *Mycnf, mysqld MysqlDaemon, logger logutil.Logger, bh backupstorage.BackupHandle, backupConcurrency int, hookExtraEnv map[string]string) (bool, error) { + + if *xtrabackupUser == "" { + return false, vterrors.New(vtrpc.Code_INVALID_ARGUMENT, "xtrabackupUser must be specified.") + } + // use a mysql connection to detect flavor at runtime + conn, err := mysqld.GetDbaConnection() + if conn != nil && err == nil { + defer conn.Close() + } + + if err != nil { + return false, vterrors.Wrap(err, "unable to obtain a connection to the database") + } + pos, err := conn.MasterPosition() + if err != nil { + return false, vterrors.Wrap(err, "unable to obtain master position") + } + flavor := pos.GTIDSet.Flavor() + logger.Infof("Detected MySQL flavor: %v", flavor) + + backupProgram := path.Join(*xtrabackupEnginePath, xtrabackupBinaryName) + + flagsToExec := []string{"--defaults-file=" + cnf.path, + "--backup", + "--socket=" + cnf.SocketFile, + "--slave-info", + "--user=" + *xtrabackupUser, + "--target-dir=" + cnf.TmpDir, + } + if *xtrabackupStreamMode != "" { + flagsToExec = append(flagsToExec, "--stream="+*xtrabackupStreamMode) + } + + if *xtrabackupBackupFlags != "" { + flagsToExec = append(flagsToExec, strings.Fields(*xtrabackupBackupFlags)...) + } + + backupFileName := be.backupFileName() + + wc, err := bh.AddFile(ctx, backupFileName, 0) + if err != nil { + return false, vterrors.Wrapf(err, "cannot create backup file %v", backupFileName) + } + closeFile := func(wc io.WriteCloser, fileName string) { + if closeErr := wc.Close(); err == nil { + err = closeErr + } else if closeErr != nil { + // since we already have an error just log this + logger.Errorf("error closing file %v: %v", fileName, err) + } + } + defer closeFile(wc, backupFileName) + + backupCmd := exec.Command(backupProgram, flagsToExec...) + backupOut, _ := backupCmd.StdoutPipe() + backupErr, _ := backupCmd.StderrPipe() + dst := bufio.NewWriterSize(wc, writerBufferSize) + writer := io.MultiWriter(dst) + + // Create the gzip compression pipe, if necessary. + var gzip *pgzip.Writer + if *backupStorageCompress { + gzip, err = pgzip.NewWriterLevel(writer, pgzip.BestSpeed) + if err != nil { + return false, vterrors.Wrap(err, "cannot create gziper") + } + gzip.SetConcurrency(*backupCompressBlockSize, *backupCompressBlocks) + writer = gzip + } + + if err = backupCmd.Start(); err != nil { + return false, vterrors.Wrap(err, "unable to start backup") + } + + // Copy from the stream output to destination file (optional gzip) + _, err = io.Copy(writer, backupOut) + if err != nil { + return false, vterrors.Wrap(err, "cannot copy output from xtrabackup command") + } + + // Close gzip to flush it, after that all data is sent to writer. + if gzip != nil { + if err = gzip.Close(); err != nil { + return false, vterrors.Wrap(err, "cannot close gzip") + } + } + + // Flush the buffer to finish writing on destination. + if err = dst.Flush(); err != nil { + return false, vterrors.Wrapf(err, "cannot flush destination: %v", backupFileName) + } + + stderrOutput, err := ioutil.ReadAll(backupErr) + if err != nil { + return false, vterrors.Wrap(err, "backup failed while reading command output") + } + err = backupCmd.Wait() + output := string(stderrOutput) + logger.Infof("Xtrabackup backup command output: %v", output) + if err != nil { + return false, vterrors.Wrap(err, "xtrabackup failed with error") + } + + replicationPosition, rerr := findReplicationPosition(output, flavor, logger) + if rerr != nil { + return false, vterrors.Wrap(rerr, "backup failed trying to find replication position") + } + // open the MANIFEST + mwc, err := bh.AddFile(ctx, backupManifest, 0) + if err != nil { + return false, vterrors.Wrapf(err, "cannot add %v to backup", backupManifest) + } + defer closeFile(mwc, backupManifest) + + // JSON-encode and write the MANIFEST + bm := &xtraBackupManifest{ + FileName: backupFileName, + BackupMethod: xtrabackupBackupMethod, + Position: replicationPosition, + SkipCompress: !*backupStorageCompress, + Params: *xtrabackupBackupFlags, + } + + data, err := json.MarshalIndent(bm, "", " ") + if err != nil { + return false, vterrors.Wrapf(err, "cannot JSON encode %v", backupManifest) + } + if _, err := mwc.Write([]byte(data)); err != nil { + return false, vterrors.Wrapf(err, "cannot write %v", backupManifest) + } + + return true, nil +} + +// ExecuteRestore restores from a backup. Any error is returned. +func (be *XtrabackupEngine) ExecuteRestore( + ctx context.Context, + cnf *Mycnf, + mysqld MysqlDaemon, + logger logutil.Logger, + dir string, + bhs []backupstorage.BackupHandle, + restoreConcurrency int, + hookExtraEnv map[string]string) (mysql.Position, error) { + + var bh backupstorage.BackupHandle + var bm xtraBackupManifest + var index int + zeroPosition := mysql.Position{} + + for index = len(bhs) - 1; index >= 0; index-- { + bh = bhs[index] + rc, err := bh.ReadFile(ctx, backupManifest) + if err != nil { + log.Warningf("Possibly incomplete backup %v in directory %v on BackupStorage: can't read MANIFEST: %v)", bh.Name(), dir, err) + continue + } + + err = json.NewDecoder(rc).Decode(&bm) + rc.Close() + if err != nil { + log.Warningf("Possibly incomplete backup %v in directory %v on BackupStorage (cannot JSON decode MANIFEST: %v)", bh.Name(), dir, err) + continue + } + + logger.Infof("Restore: found backup %v %v to restore with %v file", bh.Directory(), bh.Name(), bm.FileName) + break + } + if index < 0 { + // There is at least one attempted backup, but none could be read. + // This implies there is data we ought to have, so it's not safe to start + // up empty. + return zeroPosition, errors.New("backup(s) found but none could be read, unsafe to start up empty, restart to retry restore") + } + + // Starting from here we won't be able to recover if we get stopped by a cancelled + // context. Thus we use the background context to get through to the finish. + + logger.Infof("Restore: shutdown mysqld") + err := mysqld.Shutdown(context.Background(), cnf, true) + if err != nil { + return zeroPosition, err + } + + logger.Infof("Restore: deleting existing files") + if err := removeExistingFiles(cnf); err != nil { + return zeroPosition, err + } + + logger.Infof("Restore: reinit config file") + err = mysqld.ReinitConfig(context.Background(), cnf) + if err != nil { + return zeroPosition, err + } + + // copy / extract files + logger.Infof("Restore: Extracting all files") + + // first download the file into a tmp dir + // extract all the files + if err := be.restoreFile(ctx, cnf, logger, bh, !bm.SkipCompress, be.backupFileName()); err != nil { + logger.Errorf("error restoring backup file %v:%v", be.backupFileName(), err) + return zeroPosition, err + } + + // copy / extract files + logger.Infof("Restore: Preparing the files") + // prepare the backup + restoreProgram := path.Join(*xtrabackupEnginePath, xtrabackupBinaryName) + flagsToExec := []string{"--defaults-file=" + cnf.path, + "--prepare", + "--target-dir=" + cnf.TmpDir, + } + prepareCmd := exec.Command(restoreProgram, flagsToExec...) + prepareOut, _ := prepareCmd.StdoutPipe() + prepareErr, _ := prepareCmd.StderrPipe() + if err = prepareCmd.Start(); err != nil { + return zeroPosition, vterrors.Wrap(err, "unable to start prepare") + } + + errOutput, _ := ioutil.ReadAll(prepareErr) + stdOutput, _ := ioutil.ReadAll(prepareOut) + err = prepareCmd.Wait() + if string(stdOutput) != "" { + logger.Infof("Prepare stdout %v", string(stdOutput)) + } + output := string(errOutput) + if output != "" { + logger.Infof("Prepare stderr %v", output) + } + + if err != nil { + return zeroPosition, vterrors.Wrap(err, "prepare step failed") + } + + // then copy-back + logger.Infof("Restore: Copying the files") + + flagsToExec = []string{"--defaults-file=" + cnf.path, + "--copy-back", + "--target-dir=" + cnf.TmpDir, + } + copybackCmd := exec.Command(restoreProgram, flagsToExec...) + copybackErr, _ := copybackCmd.StderrPipe() + copybackOut, _ := copybackCmd.StdoutPipe() + + if err = copybackCmd.Start(); err != nil { + return zeroPosition, vterrors.Wrap(err, "unable to start copy-back") + } + + errOutput, _ = ioutil.ReadAll(copybackErr) + stdOutput, _ = ioutil.ReadAll(copybackOut) + err = copybackCmd.Wait() + output = string(errOutput) + if output != "" { + logger.Infof("Copy-back stderr %v", string(output)) + } + if string(stdOutput) != "" { + logger.Infof("Copy-back stdout %v", string(stdOutput)) + } + + if err != nil { + return zeroPosition, vterrors.Wrap(err, "copy-back step failed") + } + + // now find the slave position and return that + logger.Infof("Returning replication position %v", bm.Position) + return bm.Position, nil +} + +// restoreFile restores an individual file. +func (be *XtrabackupEngine) restoreFile( + ctx context.Context, + cnf *Mycnf, + logger logutil.Logger, + bh backupstorage.BackupHandle, + compress bool, + name string) (err error) { + + streamMode := *xtrabackupStreamMode + // Open the source file for reading. + var source io.ReadCloser + source, err = bh.ReadFile(ctx, name) + if err != nil { + return err + } + defer source.Close() + + reader := io.MultiReader(source) + + // Create the uncompresser if needed. + if compress { + gz, err := pgzip.NewReader(reader) + if err != nil { + return err + } + defer func() { + if cerr := gz.Close(); cerr != nil { + if err != nil { + // We already have an error, just log this one. + logger.Errorf("failed to close gunziper %v: %v", name, cerr) + } else { + err = cerr + } + } + }() + reader = gz + } + + switch streamMode { + case streamModeTar: + // now extract the files by running tar + // error if we can't find tar + flagsToExec := []string{"-C", cnf.TmpDir, "-xi"} + tarCmd := exec.Command("tar", flagsToExec...) + logger.Infof("Executing tar cmd with flags %v", flagsToExec) + tarCmd.Stdin = reader + tarOut, _ := tarCmd.StdoutPipe() + tarErr, _ := tarCmd.StderrPipe() + tarCmd.Start() + output, _ := ioutil.ReadAll(tarOut) + errOutput, _ := ioutil.ReadAll(tarErr) + err := tarCmd.Wait() + + if string(output) != "" { + logger.Infof("output from tar: %v ", string(output)) + } + if string(errOutput) != "" { + logger.Infof("error from tar: %v ", string(errOutput)) + } + if err != nil { + return vterrors.Wrap(err, "error from tar") + } + + case xbstream: + // now extract the files by running xbstream + xbstreamProgram := xbstream + flagsToExec := []string{} + if *xbstreamRestoreFlags != "" { + flagsToExec = append(flagsToExec, strings.Fields(*xbstreamRestoreFlags)...) + } + flagsToExec = append(flagsToExec, "-C", cnf.TmpDir, "-x") + xbstreamCmd := exec.Command(xbstreamProgram, flagsToExec...) + logger.Infof("Executing xbstream cmd: %v %v", xbstreamProgram, flagsToExec) + xbstreamCmd.Stdin = reader + xbstreamOut, _ := xbstreamCmd.StdoutPipe() + xbstreamErr, _ := xbstreamCmd.StderrPipe() + xbstreamCmd.Start() + output, _ := ioutil.ReadAll(xbstreamOut) + errOutput, _ := ioutil.ReadAll(xbstreamErr) + err := xbstreamCmd.Wait() + + if string(output) != "" { + logger.Infof("Output from xbstream: %v ", string(output)) + } + if string(errOutput) != "" { + logger.Infof("error from xbstream: %v", string(errOutput)) + } + if err != nil { + return vterrors.Wrap(err, "error from xbstream") + } + default: + return vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "%v is not a valid value for xtrabackup_stream_mode, supported modes are tar and xbstream", streamMode) + } + return nil +} + +func findReplicationPosition(input, flavor string, logger logutil.Logger) (mysql.Position, error) { + substrs := strings.Split(input, "'") + index := -1 + for i, str := range substrs { + if strings.Contains(str, "GTID of the last change") { + index = i + 1 + break + } + } + position := "" + // asserts that xtrabackup output comes with GTIDs in the format we expect + if index != -1 && index < len(substrs) { + // since we are extracting this from the log, it contains newlines + // replace them with a single space to match the SET GLOBAL gtid_purged command in xtrabackup_slave_info + position = strings.Replace(substrs[index], "\n", " ", -1) + } + logger.Infof("Found position: %v", position) + + // flavor is required to parse a string into a mysql.Position + replicationPosition, err := mysql.ParsePosition(flavor, position) + if err != nil { + return mysql.Position{}, err + } + return replicationPosition, nil +} + +func init() { + BackupEngineMap[xtrabackupBackupMethod] = &XtrabackupEngine{} +} diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 87f39855ed3..5d880bbc098 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -48,7 +48,7 @@ func (x OnDDLAction) String() string { return proto.EnumName(OnDDLAction_name, int32(x)) } func (OnDDLAction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{0} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{0} } // VEventType enumerates the event types. @@ -113,7 +113,7 @@ func (x VEventType) String() string { return proto.EnumName(VEventType_name, int32(x)) } func (VEventType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{1} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{1} } type BinlogTransaction_Statement_Category int32 @@ -161,7 +161,7 @@ func (x BinlogTransaction_Statement_Category) String() string { return proto.EnumName(BinlogTransaction_Statement_Category_name, int32(x)) } func (BinlogTransaction_Statement_Category) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{1, 0, 0} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{1, 0, 0} } // Charset is the per-statement charset info from a QUERY_EVENT binlog entry. @@ -181,7 +181,7 @@ func (m *Charset) Reset() { *m = Charset{} } func (m *Charset) String() string { return proto.CompactTextString(m) } func (*Charset) ProtoMessage() {} func (*Charset) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{0} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{0} } func (m *Charset) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Charset.Unmarshal(m, b) @@ -238,7 +238,7 @@ func (m *BinlogTransaction) Reset() { *m = BinlogTransaction{} } func (m *BinlogTransaction) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction) ProtoMessage() {} func (*BinlogTransaction) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{1} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{1} } func (m *BinlogTransaction) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction.Unmarshal(m, b) @@ -288,7 +288,7 @@ func (m *BinlogTransaction_Statement) Reset() { *m = BinlogTransaction_S func (m *BinlogTransaction_Statement) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction_Statement) ProtoMessage() {} func (*BinlogTransaction_Statement) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{1, 0} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{1, 0} } func (m *BinlogTransaction_Statement) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction_Statement.Unmarshal(m, b) @@ -346,7 +346,7 @@ func (m *StreamKeyRangeRequest) Reset() { *m = StreamKeyRangeRequest{} } func (m *StreamKeyRangeRequest) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeRequest) ProtoMessage() {} func (*StreamKeyRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{2} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{2} } func (m *StreamKeyRangeRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeRequest.Unmarshal(m, b) @@ -399,7 +399,7 @@ func (m *StreamKeyRangeResponse) Reset() { *m = StreamKeyRangeResponse{} func (m *StreamKeyRangeResponse) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeResponse) ProtoMessage() {} func (*StreamKeyRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{3} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{3} } func (m *StreamKeyRangeResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeResponse.Unmarshal(m, b) @@ -443,7 +443,7 @@ func (m *StreamTablesRequest) Reset() { *m = StreamTablesRequest{} } func (m *StreamTablesRequest) String() string { return proto.CompactTextString(m) } func (*StreamTablesRequest) ProtoMessage() {} func (*StreamTablesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{4} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{4} } func (m *StreamTablesRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesRequest.Unmarshal(m, b) @@ -496,7 +496,7 @@ func (m *StreamTablesResponse) Reset() { *m = StreamTablesResponse{} } func (m *StreamTablesResponse) String() string { return proto.CompactTextString(m) } func (*StreamTablesResponse) ProtoMessage() {} func (*StreamTablesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{5} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{5} } func (m *StreamTablesResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesResponse.Unmarshal(m, b) @@ -541,7 +541,7 @@ func (m *Rule) Reset() { *m = Rule{} } func (m *Rule) String() string { return proto.CompactTextString(m) } func (*Rule) ProtoMessage() {} func (*Rule) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{6} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{6} } func (m *Rule) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Rule.Unmarshal(m, b) @@ -588,7 +588,7 @@ func (m *Filter) Reset() { *m = Filter{} } func (m *Filter) String() string { return proto.CompactTextString(m) } func (*Filter) ProtoMessage() {} func (*Filter) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{7} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{7} } func (m *Filter) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filter.Unmarshal(m, b) @@ -643,7 +643,7 @@ func (m *BinlogSource) Reset() { *m = BinlogSource{} } func (m *BinlogSource) String() string { return proto.CompactTextString(m) } func (*BinlogSource) ProtoMessage() {} func (*BinlogSource) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{8} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{8} } func (m *BinlogSource) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogSource.Unmarshal(m, b) @@ -725,7 +725,7 @@ func (m *RowChange) Reset() { *m = RowChange{} } func (m *RowChange) String() string { return proto.CompactTextString(m) } func (*RowChange) ProtoMessage() {} func (*RowChange) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{9} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{9} } func (m *RowChange) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowChange.Unmarshal(m, b) @@ -772,7 +772,7 @@ func (m *RowEvent) Reset() { *m = RowEvent{} } func (m *RowEvent) String() string { return proto.CompactTextString(m) } func (*RowEvent) ProtoMessage() {} func (*RowEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{10} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{10} } func (m *RowEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowEvent.Unmarshal(m, b) @@ -818,7 +818,7 @@ func (m *FieldEvent) Reset() { *m = FieldEvent{} } func (m *FieldEvent) String() string { return proto.CompactTextString(m) } func (*FieldEvent) ProtoMessage() {} func (*FieldEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{11} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{11} } func (m *FieldEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FieldEvent.Unmarshal(m, b) @@ -871,7 +871,7 @@ func (m *VEvent) Reset() { *m = VEvent{} } func (m *VEvent) String() string { return proto.CompactTextString(m) } func (*VEvent) ProtoMessage() {} func (*VEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{12} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{12} } func (m *VEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VEvent.Unmarshal(m, b) @@ -956,7 +956,7 @@ func (m *VStreamRequest) Reset() { *m = VStreamRequest{} } func (m *VStreamRequest) String() string { return proto.CompactTextString(m) } func (*VStreamRequest) ProtoMessage() {} func (*VStreamRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{13} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{13} } func (m *VStreamRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamRequest.Unmarshal(m, b) @@ -1023,7 +1023,7 @@ func (m *VStreamResponse) Reset() { *m = VStreamResponse{} } func (m *VStreamResponse) String() string { return proto.CompactTextString(m) } func (*VStreamResponse) ProtoMessage() {} func (*VStreamResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_60517ed2deb82a7b, []int{14} + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{14} } func (m *VStreamResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamResponse.Unmarshal(m, b) @@ -1050,6 +1050,148 @@ func (m *VStreamResponse) GetEvents() []*VEvent { return nil } +// VStreamRowsRequest is the payload for VStreamRows +type VStreamRowsRequest struct { + EffectiveCallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=effective_caller_id,json=effectiveCallerId,proto3" json:"effective_caller_id,omitempty"` + ImmediateCallerId *query.VTGateCallerID `protobuf:"bytes,2,opt,name=immediate_caller_id,json=immediateCallerId,proto3" json:"immediate_caller_id,omitempty"` + Target *query.Target `protobuf:"bytes,3,opt,name=target,proto3" json:"target,omitempty"` + Query string `protobuf:"bytes,4,opt,name=query,proto3" json:"query,omitempty"` + Lastpk *query.QueryResult `protobuf:"bytes,5,opt,name=lastpk,proto3" json:"lastpk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *VStreamRowsRequest) Reset() { *m = VStreamRowsRequest{} } +func (m *VStreamRowsRequest) String() string { return proto.CompactTextString(m) } +func (*VStreamRowsRequest) ProtoMessage() {} +func (*VStreamRowsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{15} +} +func (m *VStreamRowsRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_VStreamRowsRequest.Unmarshal(m, b) +} +func (m *VStreamRowsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_VStreamRowsRequest.Marshal(b, m, deterministic) +} +func (dst *VStreamRowsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_VStreamRowsRequest.Merge(dst, src) +} +func (m *VStreamRowsRequest) XXX_Size() int { + return xxx_messageInfo_VStreamRowsRequest.Size(m) +} +func (m *VStreamRowsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_VStreamRowsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_VStreamRowsRequest proto.InternalMessageInfo + +func (m *VStreamRowsRequest) GetEffectiveCallerId() *vtrpc.CallerID { + if m != nil { + return m.EffectiveCallerId + } + return nil +} + +func (m *VStreamRowsRequest) GetImmediateCallerId() *query.VTGateCallerID { + if m != nil { + return m.ImmediateCallerId + } + return nil +} + +func (m *VStreamRowsRequest) GetTarget() *query.Target { + if m != nil { + return m.Target + } + return nil +} + +func (m *VStreamRowsRequest) GetQuery() string { + if m != nil { + return m.Query + } + return "" +} + +func (m *VStreamRowsRequest) GetLastpk() *query.QueryResult { + if m != nil { + return m.Lastpk + } + return nil +} + +// VStreamRowsResponse is the response from VStreamRows +type VStreamRowsResponse struct { + Fields []*query.Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` + Pkfields []*query.Field `protobuf:"bytes,2,rep,name=pkfields,proto3" json:"pkfields,omitempty"` + Gtid string `protobuf:"bytes,3,opt,name=gtid,proto3" json:"gtid,omitempty"` + Rows []*query.Row `protobuf:"bytes,4,rep,name=rows,proto3" json:"rows,omitempty"` + Lastpk *query.Row `protobuf:"bytes,5,opt,name=lastpk,proto3" json:"lastpk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *VStreamRowsResponse) Reset() { *m = VStreamRowsResponse{} } +func (m *VStreamRowsResponse) String() string { return proto.CompactTextString(m) } +func (*VStreamRowsResponse) ProtoMessage() {} +func (*VStreamRowsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_binlogdata_4e5c97d173ca1608, []int{16} +} +func (m *VStreamRowsResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_VStreamRowsResponse.Unmarshal(m, b) +} +func (m *VStreamRowsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_VStreamRowsResponse.Marshal(b, m, deterministic) +} +func (dst *VStreamRowsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_VStreamRowsResponse.Merge(dst, src) +} +func (m *VStreamRowsResponse) XXX_Size() int { + return xxx_messageInfo_VStreamRowsResponse.Size(m) +} +func (m *VStreamRowsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_VStreamRowsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_VStreamRowsResponse proto.InternalMessageInfo + +func (m *VStreamRowsResponse) GetFields() []*query.Field { + if m != nil { + return m.Fields + } + return nil +} + +func (m *VStreamRowsResponse) GetPkfields() []*query.Field { + if m != nil { + return m.Pkfields + } + return nil +} + +func (m *VStreamRowsResponse) GetGtid() string { + if m != nil { + return m.Gtid + } + return "" +} + +func (m *VStreamRowsResponse) GetRows() []*query.Row { + if m != nil { + return m.Rows + } + return nil +} + +func (m *VStreamRowsResponse) GetLastpk() *query.Row { + if m != nil { + return m.Lastpk + } + return nil +} + func init() { proto.RegisterType((*Charset)(nil), "binlogdata.Charset") proto.RegisterType((*BinlogTransaction)(nil), "binlogdata.BinlogTransaction") @@ -1067,89 +1209,98 @@ func init() { proto.RegisterType((*VEvent)(nil), "binlogdata.VEvent") proto.RegisterType((*VStreamRequest)(nil), "binlogdata.VStreamRequest") proto.RegisterType((*VStreamResponse)(nil), "binlogdata.VStreamResponse") + proto.RegisterType((*VStreamRowsRequest)(nil), "binlogdata.VStreamRowsRequest") + proto.RegisterType((*VStreamRowsResponse)(nil), "binlogdata.VStreamRowsResponse") proto.RegisterEnum("binlogdata.OnDDLAction", OnDDLAction_name, OnDDLAction_value) proto.RegisterEnum("binlogdata.VEventType", VEventType_name, VEventType_value) proto.RegisterEnum("binlogdata.BinlogTransaction_Statement_Category", BinlogTransaction_Statement_Category_name, BinlogTransaction_Statement_Category_value) } -func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_60517ed2deb82a7b) } - -var fileDescriptor_binlogdata_60517ed2deb82a7b = []byte{ - // 1215 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x5d, 0x6e, 0xdb, 0x46, - 0x10, 0x8e, 0x44, 0x8a, 0x12, 0x87, 0x8e, 0x4d, 0xaf, 0x7f, 0x2a, 0x18, 0x0d, 0xe0, 0x12, 0x6d, - 0xe3, 0x1a, 0xa8, 0x9c, 0xaa, 0x7f, 0x4f, 0x6d, 0x21, 0x89, 0x8c, 0xa3, 0x84, 0x96, 0x9c, 0x35, - 0x93, 0x14, 0x79, 0x21, 0x68, 0x72, 0x65, 0x13, 0xa6, 0x48, 0x85, 0x5c, 0xdb, 0xd5, 0x09, 0x7a, - 0x80, 0xbe, 0xf6, 0x02, 0x3d, 0x42, 0x2f, 0xd0, 0x9b, 0xf4, 0x1e, 0xc5, 0xfe, 0x90, 0x92, 0x1c, - 0xa0, 0x71, 0x1f, 0xfa, 0x36, 0xff, 0x3b, 0xf3, 0xcd, 0x70, 0x86, 0x60, 0x9e, 0xc7, 0x69, 0x92, - 0x5d, 0x44, 0x01, 0x0d, 0x3a, 0xb3, 0x3c, 0xa3, 0x19, 0x82, 0x85, 0x64, 0xcf, 0xb8, 0xa1, 0xf9, - 0x2c, 0x14, 0x8a, 0x3d, 0xe3, 0xdd, 0x35, 0xc9, 0xe7, 0x92, 0x59, 0xa7, 0xd9, 0x2c, 0x5b, 0x78, - 0x59, 0x27, 0xd0, 0x1c, 0x5c, 0x06, 0x79, 0x41, 0x28, 0xda, 0x05, 0x2d, 0x4c, 0x62, 0x92, 0xd2, - 0x76, 0x6d, 0xbf, 0x76, 0xd0, 0xc0, 0x92, 0x43, 0x08, 0xd4, 0x30, 0x4b, 0xd3, 0x76, 0x9d, 0x4b, - 0x39, 0xcd, 0x6c, 0x0b, 0x92, 0xdf, 0x90, 0xbc, 0xad, 0x08, 0x5b, 0xc1, 0x59, 0x7f, 0x2b, 0xb0, - 0xd9, 0xe7, 0x79, 0x78, 0x79, 0x90, 0x16, 0x41, 0x48, 0xe3, 0x2c, 0x45, 0xc7, 0x00, 0x05, 0x0d, - 0x28, 0x99, 0x92, 0x94, 0x16, 0xed, 0xda, 0xbe, 0x72, 0x60, 0x74, 0x1f, 0x77, 0x96, 0x2a, 0x78, - 0xcf, 0xa5, 0x73, 0x56, 0xda, 0xe3, 0x25, 0x57, 0xd4, 0x05, 0x83, 0xdc, 0x90, 0x94, 0xfa, 0x34, - 0xbb, 0x22, 0x69, 0x5b, 0xdd, 0xaf, 0x1d, 0x18, 0xdd, 0xcd, 0x8e, 0x28, 0xd0, 0x61, 0x1a, 0x8f, - 0x29, 0x30, 0x90, 0x8a, 0xde, 0xfb, 0xab, 0x0e, 0x7a, 0x15, 0x0d, 0xb9, 0xd0, 0x0a, 0x03, 0x4a, - 0x2e, 0xb2, 0x7c, 0xce, 0xcb, 0x5c, 0xef, 0x3e, 0xb9, 0x67, 0x22, 0x9d, 0x81, 0xf4, 0xc3, 0x55, - 0x04, 0xf4, 0x25, 0x34, 0x43, 0x81, 0x1e, 0x47, 0xc7, 0xe8, 0x6e, 0x2d, 0x07, 0x93, 0xc0, 0xe2, - 0xd2, 0x06, 0x99, 0xa0, 0x14, 0xef, 0x12, 0x0e, 0xd9, 0x1a, 0x66, 0xa4, 0xf5, 0x47, 0x0d, 0x5a, - 0x65, 0x5c, 0xb4, 0x05, 0x1b, 0x7d, 0xd7, 0x7f, 0x35, 0xc2, 0xce, 0x60, 0x7c, 0x3c, 0x1a, 0xbe, - 0x75, 0x6c, 0xf3, 0x01, 0x5a, 0x83, 0x56, 0xdf, 0xf5, 0xfb, 0xce, 0xf1, 0x70, 0x64, 0xd6, 0xd0, - 0x43, 0xd0, 0xfb, 0xae, 0x3f, 0x18, 0x9f, 0x9c, 0x0c, 0x3d, 0xb3, 0x8e, 0x36, 0xc0, 0xe8, 0xbb, - 0x3e, 0x1e, 0xbb, 0x6e, 0xbf, 0x37, 0x78, 0x61, 0x2a, 0x68, 0x07, 0x36, 0xfb, 0xae, 0x6f, 0x9f, - 0xb8, 0xbe, 0xed, 0x9c, 0x62, 0x67, 0xd0, 0xf3, 0x1c, 0xdb, 0x54, 0x11, 0x80, 0xc6, 0xc4, 0xb6, - 0x6b, 0x36, 0x24, 0x7d, 0xe6, 0x78, 0xa6, 0x26, 0xc3, 0x0d, 0x47, 0x67, 0x0e, 0xf6, 0xcc, 0xa6, - 0x64, 0x5f, 0x9d, 0xda, 0x3d, 0xcf, 0x31, 0x5b, 0x92, 0xb5, 0x1d, 0xd7, 0xf1, 0x1c, 0x53, 0x7f, - 0xae, 0xb6, 0xea, 0xa6, 0xf2, 0x5c, 0x6d, 0x29, 0xa6, 0x6a, 0xfd, 0x56, 0x83, 0x9d, 0x33, 0x9a, - 0x93, 0x60, 0xfa, 0x82, 0xcc, 0x71, 0x90, 0x5e, 0x10, 0x4c, 0xde, 0x5d, 0x93, 0x82, 0xa2, 0x3d, - 0x68, 0xcd, 0xb2, 0x22, 0x66, 0xd8, 0x71, 0x80, 0x75, 0x5c, 0xf1, 0xe8, 0x08, 0xf4, 0x2b, 0x32, - 0xf7, 0x73, 0x66, 0x2f, 0x01, 0x43, 0x9d, 0x6a, 0x20, 0xab, 0x48, 0xad, 0x2b, 0x49, 0x2d, 0xe3, - 0xab, 0x7c, 0x18, 0x5f, 0x6b, 0x02, 0xbb, 0x77, 0x93, 0x2a, 0x66, 0x59, 0x5a, 0x10, 0xe4, 0x02, - 0x12, 0x8e, 0x3e, 0x5d, 0xf4, 0x96, 0xe7, 0x67, 0x74, 0x1f, 0xfd, 0xeb, 0x00, 0xe0, 0xcd, 0xf3, - 0xbb, 0x22, 0xeb, 0x17, 0xd8, 0x12, 0xef, 0x78, 0xc1, 0x79, 0x42, 0x8a, 0xfb, 0x94, 0xbe, 0x0b, - 0x1a, 0xe5, 0xc6, 0xed, 0xfa, 0xbe, 0x72, 0xa0, 0x63, 0xc9, 0xfd, 0xd7, 0x0a, 0x23, 0xd8, 0x5e, - 0x7d, 0xf9, 0x7f, 0xa9, 0xef, 0x1b, 0x50, 0xf1, 0x75, 0x42, 0xd0, 0x36, 0x34, 0xa6, 0x01, 0x0d, - 0x2f, 0x65, 0x35, 0x82, 0x61, 0xa5, 0x4c, 0xe2, 0x84, 0x92, 0x9c, 0xb7, 0x50, 0xc7, 0x92, 0xb3, - 0x9e, 0x80, 0xf6, 0x94, 0x53, 0xe8, 0x73, 0x68, 0xe4, 0xd7, 0xac, 0x56, 0xf1, 0xa9, 0x9b, 0xcb, - 0x09, 0xb0, 0xc0, 0x58, 0xa8, 0xad, 0xdf, 0xeb, 0xb0, 0x26, 0x12, 0x3a, 0xcb, 0xae, 0xf3, 0x90, - 0x30, 0x04, 0xaf, 0xc8, 0xbc, 0x98, 0x05, 0x21, 0x29, 0x11, 0x2c, 0x79, 0x96, 0x4c, 0x71, 0x19, - 0xe4, 0x91, 0x7c, 0x55, 0x30, 0xe8, 0x5b, 0x30, 0x38, 0x92, 0xd4, 0xa7, 0xf3, 0x19, 0xe1, 0x18, - 0xae, 0x77, 0xb7, 0x17, 0x43, 0xc5, 0x71, 0xa2, 0xde, 0x7c, 0x46, 0x30, 0xd0, 0x8a, 0x5e, 0x9d, - 0x44, 0xf5, 0x1e, 0x93, 0xb8, 0xe8, 0x5f, 0x63, 0xa5, 0x7f, 0x87, 0x15, 0x18, 0x9a, 0x8c, 0xb2, - 0x54, 0xab, 0x80, 0xa3, 0x04, 0x08, 0x75, 0x40, 0xcb, 0x52, 0x3f, 0x8a, 0x92, 0x76, 0x93, 0xa7, - 0xf9, 0xd1, 0xb2, 0xed, 0x38, 0xb5, 0x6d, 0xb7, 0x27, 0x5a, 0xd2, 0xc8, 0x52, 0x3b, 0x4a, 0xac, - 0x97, 0xa0, 0xe3, 0xec, 0x76, 0x70, 0xc9, 0x13, 0xb0, 0x40, 0x3b, 0x27, 0x93, 0x2c, 0x27, 0xb2, - 0xab, 0x20, 0xb7, 0x1e, 0xce, 0x6e, 0xb1, 0xd4, 0xa0, 0x7d, 0x68, 0x04, 0x93, 0xb2, 0x31, 0xab, - 0x26, 0x42, 0x61, 0x05, 0xd0, 0xc2, 0xd9, 0x2d, 0xdf, 0x94, 0xe8, 0x11, 0x08, 0x44, 0xfc, 0x34, - 0x98, 0x96, 0x70, 0xeb, 0x5c, 0x32, 0x0a, 0xa6, 0x04, 0x7d, 0x07, 0x46, 0x9e, 0xdd, 0xfa, 0x21, - 0x7f, 0x5e, 0x8c, 0xad, 0xd1, 0xdd, 0x59, 0x69, 0x65, 0x99, 0x1c, 0x86, 0xbc, 0x24, 0x0b, 0xeb, - 0x25, 0xc0, 0xd3, 0x98, 0x24, 0xd1, 0xbd, 0x1e, 0xf9, 0x94, 0xc1, 0x47, 0x92, 0xa8, 0x8c, 0xbf, - 0x26, 0x53, 0xe6, 0x11, 0xb0, 0xd4, 0x59, 0xbf, 0xd6, 0x41, 0x7b, 0x2d, 0xe2, 0x1d, 0x82, 0xca, - 0x1b, 0x2d, 0x76, 0xf7, 0xee, 0x72, 0x3a, 0xc2, 0x82, 0xb7, 0x9a, 0xdb, 0xa0, 0x8f, 0x41, 0xa7, - 0xf1, 0x94, 0x14, 0x34, 0x98, 0xce, 0x38, 0x24, 0x0a, 0x5e, 0x08, 0xd8, 0x59, 0xbb, 0xa0, 0x71, - 0xc4, 0x47, 0x46, 0xc7, 0x9c, 0x66, 0x0b, 0x9a, 0xb5, 0x47, 0xe5, 0x22, 0x46, 0xa2, 0xaf, 0x40, - 0x67, 0x28, 0xf0, 0x7b, 0xd2, 0x6e, 0x70, 0x58, 0xb7, 0xef, 0x60, 0xc0, 0x9f, 0xc5, 0xad, 0xbc, - 0xc4, 0xf5, 0x7b, 0x30, 0x78, 0xde, 0xd2, 0x49, 0xcc, 0xc5, 0xee, 0xea, 0x5c, 0x94, 0xf8, 0x60, - 0x98, 0x2c, 0xb0, 0xfa, 0x04, 0xd6, 0xc2, 0xeb, 0x3c, 0xe7, 0xf7, 0x2d, 0x9e, 0x92, 0xf6, 0x36, - 0x4f, 0xd9, 0x90, 0x32, 0x2f, 0x9e, 0x12, 0x86, 0xc4, 0xfa, 0x6b, 0xb1, 0x01, 0xca, 0xad, 0xf3, - 0x13, 0x6c, 0x91, 0xc9, 0x84, 0x84, 0x34, 0xbe, 0x21, 0x7e, 0x18, 0x24, 0x09, 0xc9, 0xfd, 0x38, - 0x92, 0x53, 0xb2, 0xd1, 0x11, 0x7f, 0x02, 0x03, 0x2e, 0x1f, 0xda, 0x78, 0xb3, 0xb2, 0x95, 0xa2, - 0x08, 0x39, 0xb0, 0x15, 0x4f, 0xa7, 0x24, 0x8a, 0x03, 0xba, 0x1c, 0x40, 0xcc, 0xd0, 0x8e, 0x6c, - 0xc8, 0x6b, 0xef, 0x38, 0xa0, 0x64, 0x11, 0xa6, 0xf2, 0xa8, 0xc2, 0x7c, 0xc6, 0xbe, 0x90, 0xfc, - 0xa2, 0x5a, 0x64, 0x0f, 0xa5, 0xa7, 0xc7, 0x85, 0x58, 0x2a, 0x57, 0x96, 0xa4, 0x7a, 0x67, 0x49, - 0x2e, 0x3e, 0xa6, 0xc6, 0x87, 0x3e, 0x26, 0xeb, 0x07, 0xd8, 0xa8, 0x80, 0x90, 0x4b, 0xf0, 0x10, - 0x34, 0x0e, 0x79, 0xb9, 0x77, 0xd0, 0xfb, 0xd3, 0x81, 0xa5, 0xc5, 0xe1, 0x8f, 0x60, 0x2c, 0x7d, - 0x71, 0xec, 0x28, 0x0e, 0x8f, 0x47, 0x63, 0xec, 0x98, 0x0f, 0x50, 0x0b, 0xd4, 0x33, 0x6f, 0x7c, - 0x6a, 0xd6, 0x18, 0xe5, 0xfc, 0xec, 0x0c, 0xc4, 0xa1, 0x65, 0x94, 0x2f, 0x8d, 0x94, 0xc3, 0x3f, - 0x6b, 0x00, 0x8b, 0x81, 0x43, 0x06, 0x34, 0x5f, 0x8d, 0x5e, 0x8c, 0xc6, 0x6f, 0x46, 0x22, 0xc0, - 0xb1, 0x37, 0xb4, 0xcd, 0x1a, 0xd2, 0xa1, 0x21, 0x2e, 0x77, 0x9d, 0xbd, 0x20, 0xcf, 0xb6, 0xc2, - 0x6e, 0x7a, 0x75, 0xb3, 0x55, 0xd4, 0x04, 0xa5, 0xba, 0xcc, 0xf2, 0x14, 0x6b, 0x2c, 0x20, 0x76, - 0x4e, 0xdd, 0xde, 0xc0, 0x31, 0x9b, 0x4c, 0x51, 0x1d, 0x65, 0x00, 0xad, 0xbc, 0xc8, 0xcc, 0x93, - 0xdd, 0x71, 0x60, 0xef, 0x8c, 0xbd, 0x67, 0x0e, 0x36, 0x0d, 0x26, 0xc3, 0xe3, 0x37, 0xe6, 0x1a, - 0x93, 0x3d, 0x1d, 0x3a, 0xae, 0x6d, 0x3e, 0x64, 0x87, 0xfc, 0x99, 0xd3, 0xc3, 0x5e, 0xdf, 0xe9, - 0x79, 0xe6, 0x7a, 0xff, 0x8b, 0xb7, 0x8f, 0x6f, 0x62, 0x4a, 0x8a, 0xa2, 0x13, 0x67, 0x47, 0x82, - 0x3a, 0xba, 0xc8, 0x8e, 0x6e, 0xe8, 0x11, 0xff, 0x27, 0x3c, 0x5a, 0xa0, 0x76, 0xae, 0x71, 0xc9, - 0xd7, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x0e, 0xe3, 0x8a, 0xa5, 0x6f, 0x0a, 0x00, 0x00, +func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_4e5c97d173ca1608) } + +var fileDescriptor_binlogdata_4e5c97d173ca1608 = []byte{ + // 1315 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x56, 0x4d, 0x72, 0xdb, 0xc6, + 0x12, 0x36, 0x09, 0x10, 0x24, 0x1a, 0xb2, 0x04, 0x8d, 0x7e, 0x1e, 0x4b, 0xf5, 0xfc, 0x4a, 0x0f, + 0xf5, 0x5e, 0xac, 0xa8, 0x2a, 0x94, 0xc3, 0xfc, 0xad, 0x92, 0x14, 0x49, 0xc0, 0x32, 0x6d, 0x88, + 0x94, 0x47, 0xb0, 0x9d, 0xf2, 0x06, 0x05, 0x81, 0x43, 0x09, 0x25, 0x10, 0xa0, 0x81, 0xa1, 0x14, + 0x1e, 0x20, 0x95, 0x03, 0x64, 0x9b, 0x0b, 0xe4, 0x08, 0x59, 0x64, 0x9b, 0x9b, 0xe4, 0x1e, 0xa9, + 0xf9, 0x01, 0x48, 0xca, 0x89, 0xad, 0x2c, 0xb2, 0xc8, 0x06, 0xd5, 0xdd, 0xd3, 0xdd, 0xd3, 0xfd, + 0x75, 0x4f, 0xa3, 0xc1, 0x3c, 0x8f, 0x92, 0x38, 0xbd, 0x18, 0x05, 0x34, 0x68, 0x4d, 0xb3, 0x94, + 0xa6, 0x08, 0x16, 0x92, 0x3d, 0xe3, 0x9a, 0x66, 0xd3, 0x50, 0x1c, 0xec, 0x19, 0x6f, 0x66, 0x24, + 0x9b, 0x4b, 0x66, 0x9d, 0xa6, 0xd3, 0x74, 0x61, 0x65, 0x9d, 0x40, 0xbd, 0x77, 0x19, 0x64, 0x39, + 0xa1, 0x68, 0x17, 0xb4, 0x30, 0x8e, 0x48, 0x42, 0x9b, 0x95, 0xfd, 0xca, 0x41, 0x0d, 0x4b, 0x0e, + 0x21, 0x50, 0xc3, 0x34, 0x49, 0x9a, 0x55, 0x2e, 0xe5, 0x34, 0xd3, 0xcd, 0x49, 0x76, 0x4d, 0xb2, + 0xa6, 0x22, 0x74, 0x05, 0x67, 0xfd, 0xa6, 0xc0, 0x66, 0x97, 0xc7, 0xe1, 0x65, 0x41, 0x92, 0x07, + 0x21, 0x8d, 0xd2, 0x04, 0x1d, 0x03, 0xe4, 0x34, 0xa0, 0x64, 0x42, 0x12, 0x9a, 0x37, 0x2b, 0xfb, + 0xca, 0x81, 0xd1, 0x7e, 0xd8, 0x5a, 0xca, 0xe0, 0x2d, 0x93, 0xd6, 0x59, 0xa1, 0x8f, 0x97, 0x4c, + 0x51, 0x1b, 0x0c, 0x72, 0x4d, 0x12, 0xea, 0xd3, 0xf4, 0x8a, 0x24, 0x4d, 0x75, 0xbf, 0x72, 0x60, + 0xb4, 0x37, 0x5b, 0x22, 0x41, 0x87, 0x9d, 0x78, 0xec, 0x00, 0x03, 0x29, 0xe9, 0xbd, 0x5f, 0xab, + 0xa0, 0x97, 0xde, 0x90, 0x0b, 0x8d, 0x30, 0xa0, 0xe4, 0x22, 0xcd, 0xe6, 0x3c, 0xcd, 0xf5, 0xf6, + 0xa3, 0x3b, 0x06, 0xd2, 0xea, 0x49, 0x3b, 0x5c, 0x7a, 0x40, 0x1f, 0x41, 0x3d, 0x14, 0xe8, 0x71, + 0x74, 0x8c, 0xf6, 0xd6, 0xb2, 0x33, 0x09, 0x2c, 0x2e, 0x74, 0x90, 0x09, 0x4a, 0xfe, 0x26, 0xe6, + 0x90, 0xad, 0x61, 0x46, 0x5a, 0x3f, 0x55, 0xa0, 0x51, 0xf8, 0x45, 0x5b, 0xb0, 0xd1, 0x75, 0xfd, + 0x17, 0x03, 0xec, 0xf4, 0x86, 0xc7, 0x83, 0xfe, 0x6b, 0xc7, 0x36, 0xef, 0xa1, 0x35, 0x68, 0x74, + 0x5d, 0xbf, 0xeb, 0x1c, 0xf7, 0x07, 0x66, 0x05, 0xdd, 0x07, 0xbd, 0xeb, 0xfa, 0xbd, 0xe1, 0xc9, + 0x49, 0xdf, 0x33, 0xab, 0x68, 0x03, 0x8c, 0xae, 0xeb, 0xe3, 0xa1, 0xeb, 0x76, 0x3b, 0xbd, 0x67, + 0xa6, 0x82, 0x76, 0x60, 0xb3, 0xeb, 0xfa, 0xf6, 0x89, 0xeb, 0xdb, 0xce, 0x29, 0x76, 0x7a, 0x1d, + 0xcf, 0xb1, 0x4d, 0x15, 0x01, 0x68, 0x4c, 0x6c, 0xbb, 0x66, 0x4d, 0xd2, 0x67, 0x8e, 0x67, 0x6a, + 0xd2, 0x5d, 0x7f, 0x70, 0xe6, 0x60, 0xcf, 0xac, 0x4b, 0xf6, 0xc5, 0xa9, 0xdd, 0xf1, 0x1c, 0xb3, + 0x21, 0x59, 0xdb, 0x71, 0x1d, 0xcf, 0x31, 0xf5, 0xa7, 0x6a, 0xa3, 0x6a, 0x2a, 0x4f, 0xd5, 0x86, + 0x62, 0xaa, 0xd6, 0x0f, 0x15, 0xd8, 0x39, 0xa3, 0x19, 0x09, 0x26, 0xcf, 0xc8, 0x1c, 0x07, 0xc9, + 0x05, 0xc1, 0xe4, 0xcd, 0x8c, 0xe4, 0x14, 0xed, 0x41, 0x63, 0x9a, 0xe6, 0x11, 0xc3, 0x8e, 0x03, + 0xac, 0xe3, 0x92, 0x47, 0x47, 0xa0, 0x5f, 0x91, 0xb9, 0x9f, 0x31, 0x7d, 0x09, 0x18, 0x6a, 0x95, + 0x0d, 0x59, 0x7a, 0x6a, 0x5c, 0x49, 0x6a, 0x19, 0x5f, 0xe5, 0xfd, 0xf8, 0x5a, 0x63, 0xd8, 0xbd, + 0x1d, 0x54, 0x3e, 0x4d, 0x93, 0x9c, 0x20, 0x17, 0x90, 0x30, 0xf4, 0xe9, 0xa2, 0xb6, 0x3c, 0x3e, + 0xa3, 0xfd, 0xe0, 0x9d, 0x0d, 0x80, 0x37, 0xcf, 0x6f, 0x8b, 0xac, 0x6f, 0x61, 0x4b, 0xdc, 0xe3, + 0x05, 0xe7, 0x31, 0xc9, 0xef, 0x92, 0xfa, 0x2e, 0x68, 0x94, 0x2b, 0x37, 0xab, 0xfb, 0xca, 0x81, + 0x8e, 0x25, 0xf7, 0x57, 0x33, 0x1c, 0xc1, 0xf6, 0xea, 0xcd, 0x7f, 0x4b, 0x7e, 0x9f, 0x82, 0x8a, + 0x67, 0x31, 0x41, 0xdb, 0x50, 0x9b, 0x04, 0x34, 0xbc, 0x94, 0xd9, 0x08, 0x86, 0xa5, 0x32, 0x8e, + 0x62, 0x4a, 0x32, 0x5e, 0x42, 0x1d, 0x4b, 0xce, 0x7a, 0x04, 0xda, 0x63, 0x4e, 0xa1, 0x0f, 0xa0, + 0x96, 0xcd, 0x58, 0xae, 0xe2, 0xa9, 0x9b, 0xcb, 0x01, 0x30, 0xc7, 0x58, 0x1c, 0x5b, 0x3f, 0x56, + 0x61, 0x4d, 0x04, 0x74, 0x96, 0xce, 0xb2, 0x90, 0x30, 0x04, 0xaf, 0xc8, 0x3c, 0x9f, 0x06, 0x21, + 0x29, 0x10, 0x2c, 0x78, 0x16, 0x4c, 0x7e, 0x19, 0x64, 0x23, 0x79, 0xab, 0x60, 0xd0, 0x67, 0x60, + 0x70, 0x24, 0xa9, 0x4f, 0xe7, 0x53, 0xc2, 0x31, 0x5c, 0x6f, 0x6f, 0x2f, 0x9a, 0x8a, 0xe3, 0x44, + 0xbd, 0xf9, 0x94, 0x60, 0xa0, 0x25, 0xbd, 0xda, 0x89, 0xea, 0x1d, 0x3a, 0x71, 0x51, 0xbf, 0xda, + 0x4a, 0xfd, 0x0e, 0x4b, 0x30, 0x34, 0xe9, 0x65, 0x29, 0x57, 0x01, 0x47, 0x01, 0x10, 0x6a, 0x81, + 0x96, 0x26, 0xfe, 0x68, 0x14, 0x37, 0xeb, 0x3c, 0xcc, 0x7f, 0x2d, 0xeb, 0x0e, 0x13, 0xdb, 0x76, + 0x3b, 0xa2, 0x24, 0xb5, 0x34, 0xb1, 0x47, 0xb1, 0xf5, 0x1c, 0x74, 0x9c, 0xde, 0xf4, 0x2e, 0x79, + 0x00, 0x16, 0x68, 0xe7, 0x64, 0x9c, 0x66, 0x44, 0x56, 0x15, 0xe4, 0xd4, 0xc3, 0xe9, 0x0d, 0x96, + 0x27, 0x68, 0x1f, 0x6a, 0xc1, 0xb8, 0x28, 0xcc, 0xaa, 0x8a, 0x38, 0xb0, 0x02, 0x68, 0xe0, 0xf4, + 0x86, 0x4f, 0x4a, 0xf4, 0x00, 0x04, 0x22, 0x7e, 0x12, 0x4c, 0x0a, 0xb8, 0x75, 0x2e, 0x19, 0x04, + 0x13, 0x82, 0x3e, 0x07, 0x23, 0x4b, 0x6f, 0xfc, 0x90, 0x5f, 0x2f, 0xda, 0xd6, 0x68, 0xef, 0xac, + 0x94, 0xb2, 0x08, 0x0e, 0x43, 0x56, 0x90, 0xb9, 0xf5, 0x1c, 0xe0, 0x71, 0x44, 0xe2, 0xd1, 0x9d, + 0x2e, 0xf9, 0x1f, 0x83, 0x8f, 0xc4, 0xa3, 0xc2, 0xff, 0x9a, 0x0c, 0x99, 0x7b, 0xc0, 0xf2, 0xcc, + 0xfa, 0xbe, 0x0a, 0xda, 0x4b, 0xe1, 0xef, 0x10, 0x54, 0x5e, 0x68, 0x31, 0xbb, 0x77, 0x97, 0xc3, + 0x11, 0x1a, 0xbc, 0xd4, 0x5c, 0x07, 0xfd, 0x1b, 0x74, 0x1a, 0x4d, 0x48, 0x4e, 0x83, 0xc9, 0x94, + 0x43, 0xa2, 0xe0, 0x85, 0x80, 0xfd, 0xd6, 0x2e, 0x68, 0x34, 0xe2, 0x2d, 0xa3, 0x63, 0x4e, 0xb3, + 0x01, 0xcd, 0xca, 0xa3, 0x72, 0x11, 0x23, 0xd1, 0xc7, 0xa0, 0x33, 0x14, 0xf8, 0xff, 0xa4, 0x59, + 0xe3, 0xb0, 0x6e, 0xdf, 0xc2, 0x80, 0x5f, 0x8b, 0x1b, 0x59, 0x81, 0xeb, 0x17, 0x60, 0xf0, 0xb8, + 0xa5, 0x91, 0xe8, 0x8b, 0xdd, 0xd5, 0xbe, 0x28, 0xf0, 0xc1, 0x30, 0x5e, 0x60, 0xf5, 0x5f, 0x58, + 0x0b, 0x67, 0x59, 0xc6, 0xff, 0x6f, 0xd1, 0x84, 0x34, 0xb7, 0x79, 0xc8, 0x86, 0x94, 0x79, 0xd1, + 0x84, 0x30, 0x24, 0xd6, 0x5f, 0x8a, 0x09, 0x50, 0x4c, 0x9d, 0xaf, 0x61, 0x8b, 0x8c, 0xc7, 0x24, + 0xa4, 0xd1, 0x35, 0xf1, 0xc3, 0x20, 0x8e, 0x49, 0xe6, 0x47, 0x23, 0xd9, 0x25, 0x1b, 0x2d, 0xb1, + 0x09, 0xf4, 0xb8, 0xbc, 0x6f, 0xe3, 0xcd, 0x52, 0x57, 0x8a, 0x46, 0xc8, 0x81, 0xad, 0x68, 0x32, + 0x21, 0xa3, 0x28, 0xa0, 0xcb, 0x0e, 0x44, 0x0f, 0xed, 0xc8, 0x82, 0xbc, 0xf4, 0x8e, 0x03, 0x4a, + 0x16, 0x6e, 0x4a, 0x8b, 0xd2, 0xcd, 0xff, 0xd9, 0x0b, 0xc9, 0x2e, 0xca, 0x41, 0x76, 0x5f, 0x5a, + 0x7a, 0x5c, 0x88, 0xe5, 0xe1, 0xca, 0x90, 0x54, 0x6f, 0x0d, 0xc9, 0xc5, 0x63, 0xaa, 0xbd, 0xef, + 0x31, 0x59, 0x5f, 0xc2, 0x46, 0x09, 0x84, 0x1c, 0x82, 0x87, 0xa0, 0x71, 0xc8, 0x8b, 0xb9, 0x83, + 0xde, 0xee, 0x0e, 0x2c, 0x35, 0xac, 0xef, 0xaa, 0x80, 0x0a, 0xfb, 0xf4, 0x26, 0xff, 0x87, 0x82, + 0xb9, 0x0d, 0x35, 0x2e, 0x97, 0x48, 0x0a, 0x86, 0xe1, 0x10, 0x07, 0x39, 0x9d, 0x5e, 0x95, 0x30, + 0x0a, 0xe3, 0xe7, 0xec, 0x8b, 0x49, 0x3e, 0x8b, 0x29, 0x96, 0x1a, 0xd6, 0x2f, 0x15, 0xd8, 0x5a, + 0xc1, 0x41, 0x62, 0xb9, 0x78, 0x98, 0x95, 0x3f, 0x7f, 0x98, 0xe8, 0x00, 0x1a, 0xd3, 0xab, 0x77, + 0x3c, 0xe0, 0xf2, 0xf4, 0x0f, 0x5f, 0xdb, 0x7f, 0x40, 0xcd, 0xd2, 0x9b, 0xbc, 0xa9, 0x72, 0xcb, + 0xe5, 0x69, 0xc5, 0xe5, 0x6c, 0xe4, 0xad, 0xe4, 0xb1, 0x32, 0xf2, 0xc4, 0xc9, 0xe1, 0x57, 0x60, + 0x2c, 0x4d, 0x4e, 0xb6, 0xdc, 0xf4, 0x8f, 0x07, 0x43, 0xec, 0x98, 0xf7, 0x50, 0x03, 0xd4, 0x33, + 0x6f, 0x78, 0x6a, 0x56, 0x18, 0xe5, 0x7c, 0xe3, 0xf4, 0xc4, 0xc2, 0xc4, 0x28, 0x5f, 0x2a, 0x29, + 0x87, 0x3f, 0x57, 0x00, 0x16, 0x83, 0x03, 0x19, 0x50, 0x7f, 0x31, 0x78, 0x36, 0x18, 0xbe, 0x1a, + 0x08, 0x07, 0xc7, 0x5e, 0xdf, 0x36, 0x2b, 0x48, 0x87, 0x9a, 0xd8, 0xc0, 0xaa, 0xec, 0x06, 0xb9, + 0x7e, 0x29, 0x6c, 0x37, 0x2b, 0x77, 0x2f, 0x15, 0xd5, 0x41, 0x29, 0x37, 0x2c, 0xb9, 0x52, 0x69, + 0xcc, 0x21, 0x76, 0x4e, 0xdd, 0x4e, 0xcf, 0x31, 0xeb, 0xec, 0xa0, 0x5c, 0xae, 0x00, 0xb4, 0x62, + 0xb3, 0x62, 0x96, 0x6c, 0x1f, 0x03, 0x76, 0xcf, 0xd0, 0x7b, 0xe2, 0x60, 0xd3, 0x60, 0x32, 0x3c, + 0x7c, 0x65, 0xae, 0x31, 0xd9, 0xe3, 0xbe, 0xe3, 0xda, 0xe6, 0x7d, 0xb6, 0x90, 0x3d, 0x71, 0x3a, + 0xd8, 0xeb, 0x3a, 0x1d, 0xcf, 0x5c, 0xef, 0x7e, 0xf8, 0xfa, 0xe1, 0x75, 0x44, 0x49, 0x9e, 0xb7, + 0xa2, 0xf4, 0x48, 0x50, 0x47, 0x17, 0xe9, 0xd1, 0x35, 0x3d, 0xe2, 0xbb, 0xfd, 0xd1, 0xa2, 0xfb, + 0xcf, 0x35, 0x2e, 0xf9, 0xe4, 0xf7, 0x00, 0x00, 0x00, 0xff, 0xff, 0x47, 0x47, 0xf1, 0x1d, 0x37, + 0x0c, 0x00, 0x00, } diff --git a/go/vt/proto/queryservice/queryservice.pb.go b/go/vt/proto/queryservice/queryservice.pb.go index 7a0bce4f401..ac8303d162a 100644 --- a/go/vt/proto/queryservice/queryservice.pb.go +++ b/go/vt/proto/queryservice/queryservice.pb.go @@ -88,6 +88,8 @@ type QueryClient interface { UpdateStream(ctx context.Context, in *query.UpdateStreamRequest, opts ...grpc.CallOption) (Query_UpdateStreamClient, error) // VStream streams vreplication events. VStream(ctx context.Context, in *binlogdata.VStreamRequest, opts ...grpc.CallOption) (Query_VStreamClient, error) + // VStreamRows streams rows from the specified starting point. + VStreamRows(ctx context.Context, in *binlogdata.VStreamRowsRequest, opts ...grpc.CallOption) (Query_VStreamRowsClient, error) } type queryClient struct { @@ -411,6 +413,38 @@ func (x *queryVStreamClient) Recv() (*binlogdata.VStreamResponse, error) { return m, nil } +func (c *queryClient) VStreamRows(ctx context.Context, in *binlogdata.VStreamRowsRequest, opts ...grpc.CallOption) (Query_VStreamRowsClient, error) { + stream, err := c.cc.NewStream(ctx, &_Query_serviceDesc.Streams[5], "/queryservice.Query/VStreamRows", opts...) + if err != nil { + return nil, err + } + x := &queryVStreamRowsClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Query_VStreamRowsClient interface { + Recv() (*binlogdata.VStreamRowsResponse, error) + grpc.ClientStream +} + +type queryVStreamRowsClient struct { + grpc.ClientStream +} + +func (x *queryVStreamRowsClient) Recv() (*binlogdata.VStreamRowsResponse, error) { + m := new(binlogdata.VStreamRowsResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // QueryServer is the server API for Query service. type QueryServer interface { // Execute executes the specified SQL query (might be in a @@ -464,6 +498,8 @@ type QueryServer interface { UpdateStream(*query.UpdateStreamRequest, Query_UpdateStreamServer) error // VStream streams vreplication events. VStream(*binlogdata.VStreamRequest, Query_VStreamServer) error + // VStreamRows streams rows from the specified starting point. + VStreamRows(*binlogdata.VStreamRowsRequest, Query_VStreamRowsServer) error } func RegisterQueryServer(s *grpc.Server, srv QueryServer) { @@ -881,6 +917,27 @@ func (x *queryVStreamServer) Send(m *binlogdata.VStreamResponse) error { return x.ServerStream.SendMsg(m) } +func _Query_VStreamRows_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(binlogdata.VStreamRowsRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QueryServer).VStreamRows(m, &queryVStreamRowsServer{stream}) +} + +type Query_VStreamRowsServer interface { + Send(*binlogdata.VStreamRowsResponse) error + grpc.ServerStream +} + +type queryVStreamRowsServer struct { + grpc.ServerStream +} + +func (x *queryVStreamRowsServer) Send(m *binlogdata.VStreamRowsResponse) error { + return x.ServerStream.SendMsg(m) +} + var _Query_serviceDesc = grpc.ServiceDesc{ ServiceName: "queryservice.Query", HandlerType: (*QueryServer)(nil), @@ -980,46 +1037,53 @@ var _Query_serviceDesc = grpc.ServiceDesc{ Handler: _Query_VStream_Handler, ServerStreams: true, }, + { + StreamName: "VStreamRows", + Handler: _Query_VStreamRows_Handler, + ServerStreams: true, + }, }, Metadata: "queryservice.proto", } -func init() { proto.RegisterFile("queryservice.proto", fileDescriptor_queryservice_17509881eb07629d) } - -var fileDescriptor_queryservice_17509881eb07629d = []byte{ - // 544 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x95, 0xdf, 0x6b, 0xd4, 0x40, - 0x10, 0xc7, 0xf5, 0xa1, 0xad, 0x4c, 0x4f, 0xad, 0x5b, 0xab, 0x36, 0xad, 0x6d, 0xed, 0x9b, 0x08, - 0x17, 0x51, 0x41, 0x28, 0xf8, 0xd0, 0x0b, 0x16, 0xa5, 0xf8, 0xeb, 0xce, 0x16, 0xf1, 0x41, 0xd8, - 0x4b, 0x86, 0x33, 0x34, 0x97, 0x4d, 0x93, 0xbd, 0x43, 0xff, 0x6a, 0xff, 0x05, 0x31, 0x9b, 0x99, - 0xec, 0xee, 0x25, 0xbe, 0xdd, 0x7e, 0xbf, 0x33, 0x1f, 0x26, 0x3b, 0x37, 0xb3, 0x20, 0xae, 0x17, - 0x58, 0xfe, 0xae, 0xb0, 0x5c, 0xa6, 0x31, 0x0e, 0x8b, 0x52, 0x69, 0x25, 0x06, 0xb6, 0x16, 0x6c, - 0xd6, 0x27, 0x63, 0x05, 0x5b, 0xd3, 0x34, 0xcf, 0xd4, 0x2c, 0x91, 0x5a, 0x1a, 0xe5, 0xc5, 0x9f, - 0x01, 0xac, 0x7d, 0xf9, 0x17, 0x21, 0x4e, 0x60, 0xe3, 0xed, 0x2f, 0x8c, 0x17, 0x1a, 0xc5, 0xce, - 0xd0, 0x24, 0x35, 0xe7, 0x31, 0x5e, 0x2f, 0xb0, 0xd2, 0xc1, 0x03, 0x5f, 0xae, 0x0a, 0x95, 0x57, - 0x78, 0x7c, 0x43, 0xbc, 0x87, 0x41, 0x23, 0x8e, 0xa4, 0x8e, 0x7f, 0x8a, 0xc0, 0x8d, 0xac, 0x45, - 0xa2, 0xec, 0x75, 0x7a, 0x8c, 0xfa, 0x08, 0xb7, 0x27, 0xba, 0x44, 0x39, 0xa7, 0x62, 0x28, 0xde, - 0x51, 0x09, 0xb6, 0xdf, 0x6d, 0x12, 0xed, 0xf9, 0x4d, 0xf1, 0x0a, 0xd6, 0x46, 0x38, 0x4b, 0x73, - 0xb1, 0xdd, 0x84, 0xd6, 0x27, 0xca, 0xbf, 0xef, 0x8a, 0x5c, 0xc5, 0x6b, 0x58, 0x8f, 0xd4, 0x7c, - 0x9e, 0x6a, 0x41, 0x11, 0xe6, 0x48, 0x79, 0x3b, 0x9e, 0xca, 0x89, 0x6f, 0xe0, 0xd6, 0x58, 0x65, - 0xd9, 0x54, 0xc6, 0x57, 0x82, 0xee, 0x8b, 0x04, 0x4a, 0x7e, 0xb8, 0xa2, 0x73, 0xfa, 0x09, 0x6c, - 0x7c, 0x2e, 0xb1, 0x90, 0x65, 0xdb, 0x84, 0xe6, 0xec, 0x37, 0x81, 0x65, 0xce, 0xfd, 0x04, 0x77, - 0x4c, 0x39, 0x8d, 0x95, 0x88, 0x7d, 0xa7, 0x4a, 0x92, 0x89, 0xf4, 0xb8, 0xc7, 0x65, 0xe0, 0x05, - 0x6c, 0x51, 0x89, 0x8c, 0x3c, 0xf0, 0x6a, 0xf7, 0xa1, 0x87, 0xbd, 0x3e, 0x63, 0xbf, 0xc1, 0xbd, - 0xa8, 0x44, 0xa9, 0xf1, 0x6b, 0x29, 0xf3, 0x4a, 0xc6, 0x3a, 0x55, 0xb9, 0xa0, 0xbc, 0x15, 0x87, - 0xc0, 0x47, 0xfd, 0x01, 0x4c, 0x3e, 0x83, 0xcd, 0x89, 0x96, 0xa5, 0x6e, 0x5a, 0xb7, 0xcb, 0x7f, - 0x0e, 0xd6, 0x88, 0x16, 0x74, 0x59, 0x0e, 0x07, 0x35, 0xf7, 0x91, 0x39, 0xad, 0xb6, 0xc2, 0xb1, - 0x2d, 0xe6, 0xfc, 0x80, 0xed, 0x48, 0xe5, 0x71, 0xb6, 0x48, 0x9c, 0x6f, 0x7d, 0xc2, 0x17, 0xbf, - 0xe2, 0x11, 0xf7, 0xf8, 0x7f, 0x21, 0xcc, 0x1f, 0xc3, 0xdd, 0x31, 0xca, 0xc4, 0x66, 0x53, 0x53, - 0x3d, 0x9d, 0xb8, 0x07, 0x7d, 0xb6, 0x3d, 0xca, 0xf5, 0x30, 0xd0, 0xf8, 0x05, 0xf6, 0x84, 0x78, - 0xd3, 0xb7, 0xd7, 0xe9, 0xd9, 0x8d, 0xb6, 0x1d, 0xb3, 0x1a, 0x0e, 0x3b, 0x72, 0x9c, 0xfd, 0x70, - 0xd4, 0x1f, 0x60, 0x2f, 0x89, 0x0f, 0x58, 0x55, 0x72, 0x86, 0x66, 0xf0, 0x79, 0x49, 0x38, 0xaa, - 0xbf, 0x24, 0x3c, 0xd3, 0x5a, 0x12, 0x11, 0x40, 0x63, 0x9e, 0xc6, 0x57, 0xe2, 0x91, 0x1b, 0x7f, - 0xda, 0xb6, 0x7b, 0xb7, 0xc3, 0xe1, 0xa2, 0x22, 0x80, 0x49, 0x91, 0xa5, 0xda, 0xac, 0x53, 0x82, - 0xb4, 0x92, 0x0f, 0xb1, 0x1d, 0x86, 0x9c, 0xc3, 0xc0, 0xd4, 0xf7, 0x0e, 0x65, 0xa6, 0xdb, 0x4d, - 0x6a, 0x8b, 0xfe, 0xf5, 0xbb, 0x9e, 0xf5, 0x59, 0xe7, 0x30, 0xb8, 0x28, 0x12, 0xa9, 0xe9, 0x96, - 0x08, 0x66, 0x8b, 0x3e, 0xcc, 0xf5, 0x2c, 0xd8, 0x19, 0x6c, 0x5c, 0x32, 0xc7, 0x7a, 0x47, 0x2e, - 0x7d, 0x4e, 0x97, 0xd7, 0x72, 0x46, 0xcf, 0xbe, 0x3f, 0x5d, 0xa6, 0x1a, 0xab, 0x6a, 0x98, 0xaa, - 0xd0, 0xfc, 0x0a, 0x67, 0x2a, 0x5c, 0xea, 0xb0, 0x7e, 0x91, 0x42, 0xfb, 0xf5, 0x9a, 0xae, 0xd7, - 0xda, 0xcb, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xbd, 0x47, 0x8e, 0x80, 0xe8, 0x06, 0x00, 0x00, +func init() { proto.RegisterFile("queryservice.proto", fileDescriptor_queryservice_98b01c0566d3f32e) } + +var fileDescriptor_queryservice_98b01c0566d3f32e = []byte{ + // 563 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x95, 0x4f, 0x6f, 0xd3, 0x4c, + 0x10, 0xc6, 0xdf, 0xf7, 0xd0, 0x06, 0x4d, 0x52, 0x28, 0x5b, 0x0a, 0xd4, 0x2d, 0x69, 0xe9, 0x0d, + 0x21, 0x25, 0x08, 0x90, 0x90, 0x2a, 0x71, 0x68, 0x2c, 0x2a, 0x50, 0xc5, 0x3f, 0x87, 0x56, 0x88, + 0x03, 0xd2, 0xc6, 0x5e, 0x05, 0xab, 0x8e, 0xd7, 0xf5, 0x6e, 0x52, 0xf8, 0x7c, 0x7c, 0x31, 0x84, + 0xd7, 0x33, 0xde, 0xdd, 0xd8, 0xdc, 0xb2, 0xcf, 0x33, 0xf3, 0xd3, 0x78, 0x27, 0x33, 0x0b, 0xec, + 0x7a, 0x29, 0xca, 0x5f, 0x4a, 0x94, 0xab, 0x34, 0x16, 0xa3, 0xa2, 0x94, 0x5a, 0xb2, 0x81, 0xad, + 0x05, 0xfd, 0xea, 0x64, 0xac, 0x60, 0x7b, 0x96, 0xe6, 0x99, 0x9c, 0x27, 0x5c, 0x73, 0xa3, 0x3c, + 0xff, 0xbd, 0x05, 0x1b, 0x9f, 0xff, 0x46, 0xb0, 0x13, 0xe8, 0xbd, 0xf9, 0x29, 0xe2, 0xa5, 0x16, + 0x6c, 0x77, 0x64, 0x92, 0xea, 0x73, 0x24, 0xae, 0x97, 0x42, 0xe9, 0xe0, 0xbe, 0x2f, 0xab, 0x42, + 0xe6, 0x4a, 0x1c, 0xff, 0xc7, 0xde, 0xc1, 0xa0, 0x16, 0x27, 0x5c, 0xc7, 0x3f, 0x58, 0xe0, 0x46, + 0x56, 0x22, 0x52, 0xf6, 0x5b, 0x3d, 0x42, 0x7d, 0x80, 0xad, 0xa9, 0x2e, 0x05, 0x5f, 0x60, 0x31, + 0x18, 0xef, 0xa8, 0x08, 0x3b, 0x68, 0x37, 0x91, 0xf6, 0xec, 0x7f, 0xf6, 0x12, 0x36, 0x26, 0x62, + 0x9e, 0xe6, 0x6c, 0xa7, 0x0e, 0xad, 0x4e, 0x98, 0x7f, 0xcf, 0x15, 0xa9, 0x8a, 0x57, 0xb0, 0x19, + 0xca, 0xc5, 0x22, 0xd5, 0x0c, 0x23, 0xcc, 0x11, 0xf3, 0x76, 0x3d, 0x95, 0x12, 0x5f, 0xc3, 0xad, + 0x48, 0x66, 0xd9, 0x8c, 0xc7, 0x57, 0x0c, 0xef, 0x0b, 0x05, 0x4c, 0x7e, 0xb0, 0xa6, 0x53, 0xfa, + 0x09, 0xf4, 0x3e, 0x95, 0xa2, 0xe0, 0x65, 0xd3, 0x84, 0xfa, 0xec, 0x37, 0x81, 0x64, 0xca, 0xfd, + 0x08, 0xb7, 0x4d, 0x39, 0xb5, 0x95, 0xb0, 0x03, 0xa7, 0x4a, 0x94, 0x91, 0xf4, 0xa8, 0xc3, 0x25, + 0xe0, 0x05, 0x6c, 0x63, 0x89, 0x84, 0x1c, 0x7a, 0xb5, 0xfb, 0xd0, 0xc3, 0x4e, 0x9f, 0xb0, 0x5f, + 0xe1, 0x6e, 0x58, 0x0a, 0xae, 0xc5, 0x97, 0x92, 0xe7, 0x8a, 0xc7, 0x3a, 0x95, 0x39, 0xc3, 0xbc, + 0x35, 0x07, 0xc1, 0x47, 0xdd, 0x01, 0x44, 0x3e, 0x83, 0xfe, 0x54, 0xf3, 0x52, 0xd7, 0xad, 0xdb, + 0xa3, 0x3f, 0x07, 0x69, 0x48, 0x0b, 0xda, 0x2c, 0x87, 0x23, 0x34, 0xf5, 0x91, 0x38, 0x8d, 0xb6, + 0xc6, 0xb1, 0x2d, 0xe2, 0x7c, 0x87, 0x9d, 0x50, 0xe6, 0x71, 0xb6, 0x4c, 0x9c, 0x6f, 0x7d, 0x4c, + 0x17, 0xbf, 0xe6, 0x21, 0xf7, 0xf8, 0x5f, 0x21, 0xc4, 0x8f, 0xe0, 0x4e, 0x24, 0x78, 0x62, 0xb3, + 0xb1, 0xa9, 0x9e, 0x8e, 0xdc, 0x61, 0x97, 0x6d, 0x8f, 0x72, 0x35, 0x0c, 0x38, 0x7e, 0x81, 0x3d, + 0x21, 0xde, 0xf4, 0xed, 0xb7, 0x7a, 0x76, 0xa3, 0x6d, 0xc7, 0xac, 0x86, 0xc3, 0x96, 0x1c, 0x67, + 0x3f, 0x1c, 0x75, 0x07, 0xd8, 0x4b, 0xe2, 0xbd, 0x50, 0x8a, 0xcf, 0x85, 0x19, 0x7c, 0x5a, 0x12, + 0x8e, 0xea, 0x2f, 0x09, 0xcf, 0xb4, 0x96, 0x44, 0x08, 0x50, 0x9b, 0xa7, 0xf1, 0x15, 0x7b, 0xe8, + 0xc6, 0x9f, 0x36, 0xed, 0xde, 0x6b, 0x71, 0xa8, 0xa8, 0x10, 0x60, 0x5a, 0x64, 0xa9, 0x36, 0xeb, + 0x14, 0x21, 0x8d, 0xe4, 0x43, 0x6c, 0x87, 0x20, 0xe7, 0x30, 0x30, 0xf5, 0xbd, 0x15, 0x3c, 0xd3, + 0xcd, 0x26, 0xb5, 0x45, 0xff, 0xfa, 0x5d, 0xcf, 0xfa, 0xac, 0x73, 0x18, 0x5c, 0x14, 0x09, 0xd7, + 0x78, 0x4b, 0x08, 0xb3, 0x45, 0x1f, 0xe6, 0x7a, 0x16, 0xec, 0x0c, 0x7a, 0x97, 0xc4, 0xb1, 0xde, + 0x91, 0x4b, 0x9f, 0xd3, 0xe6, 0x59, 0x9c, 0x08, 0xfa, 0x28, 0xcb, 0x1b, 0xc5, 0x86, 0x6d, 0xf1, + 0xf2, 0x46, 0x35, 0x0b, 0xa5, 0xcb, 0x6f, 0x98, 0x93, 0xa7, 0xdf, 0x9e, 0xac, 0x52, 0x2d, 0x94, + 0x1a, 0xa5, 0x72, 0x6c, 0x7e, 0x8d, 0xe7, 0x72, 0xbc, 0xd2, 0xe3, 0xea, 0x95, 0x1b, 0xdb, 0x2f, + 0xe2, 0x6c, 0xb3, 0xd2, 0x5e, 0xfc, 0x09, 0x00, 0x00, 0xff, 0xff, 0xdd, 0x23, 0x8f, 0x51, 0x3c, + 0x07, 0x00, 0x00, } diff --git a/go/vt/proto/vschema/vschema.pb.go b/go/vt/proto/vschema/vschema.pb.go index 4a3aa16b0d1..3cf3feeadad 100644 --- a/go/vt/proto/vschema/vschema.pb.go +++ b/go/vt/proto/vschema/vschema.pb.go @@ -19,6 +19,95 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package +// RoutingRules specify the high level routing rules for the VSchema. +type RoutingRules struct { + // rules should ideally be a map. However protos dont't allow + // repeated fields as elements of a map. So, we use a list + // instead. + Rules []*RoutingRule `protobuf:"bytes,1,rep,name=rules,proto3" json:"rules,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RoutingRules) Reset() { *m = RoutingRules{} } +func (m *RoutingRules) String() string { return proto.CompactTextString(m) } +func (*RoutingRules) ProtoMessage() {} +func (*RoutingRules) Descriptor() ([]byte, []int) { + return fileDescriptor_vschema_ddae95e7e0992d00, []int{0} +} +func (m *RoutingRules) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RoutingRules.Unmarshal(m, b) +} +func (m *RoutingRules) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RoutingRules.Marshal(b, m, deterministic) +} +func (dst *RoutingRules) XXX_Merge(src proto.Message) { + xxx_messageInfo_RoutingRules.Merge(dst, src) +} +func (m *RoutingRules) XXX_Size() int { + return xxx_messageInfo_RoutingRules.Size(m) +} +func (m *RoutingRules) XXX_DiscardUnknown() { + xxx_messageInfo_RoutingRules.DiscardUnknown(m) +} + +var xxx_messageInfo_RoutingRules proto.InternalMessageInfo + +func (m *RoutingRules) GetRules() []*RoutingRule { + if m != nil { + return m.Rules + } + return nil +} + +// RoutingRule specifies a routing rule. +type RoutingRule struct { + FromTable string `protobuf:"bytes,1,opt,name=from_table,json=fromTable,proto3" json:"from_table,omitempty"` + ToTables []string `protobuf:"bytes,2,rep,name=to_tables,json=toTables,proto3" json:"to_tables,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RoutingRule) Reset() { *m = RoutingRule{} } +func (m *RoutingRule) String() string { return proto.CompactTextString(m) } +func (*RoutingRule) ProtoMessage() {} +func (*RoutingRule) Descriptor() ([]byte, []int) { + return fileDescriptor_vschema_ddae95e7e0992d00, []int{1} +} +func (m *RoutingRule) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_RoutingRule.Unmarshal(m, b) +} +func (m *RoutingRule) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_RoutingRule.Marshal(b, m, deterministic) +} +func (dst *RoutingRule) XXX_Merge(src proto.Message) { + xxx_messageInfo_RoutingRule.Merge(dst, src) +} +func (m *RoutingRule) XXX_Size() int { + return xxx_messageInfo_RoutingRule.Size(m) +} +func (m *RoutingRule) XXX_DiscardUnknown() { + xxx_messageInfo_RoutingRule.DiscardUnknown(m) +} + +var xxx_messageInfo_RoutingRule proto.InternalMessageInfo + +func (m *RoutingRule) GetFromTable() string { + if m != nil { + return m.FromTable + } + return "" +} + +func (m *RoutingRule) GetToTables() []string { + if m != nil { + return m.ToTables + } + return nil +} + // Keyspace is the vschema for a keyspace. type Keyspace struct { // If sharded is false, vindexes and tables are ignored. @@ -34,7 +123,7 @@ func (m *Keyspace) Reset() { *m = Keyspace{} } func (m *Keyspace) String() string { return proto.CompactTextString(m) } func (*Keyspace) ProtoMessage() {} func (*Keyspace) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{0} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{2} } func (m *Keyspace) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Keyspace.Unmarshal(m, b) @@ -99,7 +188,7 @@ func (m *Vindex) Reset() { *m = Vindex{} } func (m *Vindex) String() string { return proto.CompactTextString(m) } func (*Vindex) ProtoMessage() {} func (*Vindex) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{1} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{3} } func (m *Vindex) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Vindex.Unmarshal(m, b) @@ -170,7 +259,7 @@ func (m *Table) Reset() { *m = Table{} } func (m *Table) String() string { return proto.CompactTextString(m) } func (*Table) ProtoMessage() {} func (*Table) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{2} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{4} } func (m *Table) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Table.Unmarshal(m, b) @@ -249,7 +338,7 @@ func (m *ColumnVindex) Reset() { *m = ColumnVindex{} } func (m *ColumnVindex) String() string { return proto.CompactTextString(m) } func (*ColumnVindex) ProtoMessage() {} func (*ColumnVindex) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{3} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{5} } func (m *ColumnVindex) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ColumnVindex.Unmarshal(m, b) @@ -304,7 +393,7 @@ func (m *AutoIncrement) Reset() { *m = AutoIncrement{} } func (m *AutoIncrement) String() string { return proto.CompactTextString(m) } func (*AutoIncrement) ProtoMessage() {} func (*AutoIncrement) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{4} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{6} } func (m *AutoIncrement) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_AutoIncrement.Unmarshal(m, b) @@ -351,7 +440,7 @@ func (m *Column) Reset() { *m = Column{} } func (m *Column) String() string { return proto.CompactTextString(m) } func (*Column) ProtoMessage() {} func (*Column) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{5} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{7} } func (m *Column) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Column.Unmarshal(m, b) @@ -389,6 +478,7 @@ func (m *Column) GetType() query.Type { type SrvVSchema struct { // keyspaces is a map of keyspace name -> Keyspace object. Keyspaces map[string]*Keyspace `protobuf:"bytes,1,rep,name=keyspaces,proto3" json:"keyspaces,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + RoutingRules *RoutingRules `protobuf:"bytes,2,opt,name=routing_rules,json=routingRules,proto3" json:"routing_rules,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -398,7 +488,7 @@ func (m *SrvVSchema) Reset() { *m = SrvVSchema{} } func (m *SrvVSchema) String() string { return proto.CompactTextString(m) } func (*SrvVSchema) ProtoMessage() {} func (*SrvVSchema) Descriptor() ([]byte, []int) { - return fileDescriptor_vschema_58a865bec489dd60, []int{6} + return fileDescriptor_vschema_ddae95e7e0992d00, []int{8} } func (m *SrvVSchema) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_SrvVSchema.Unmarshal(m, b) @@ -425,7 +515,16 @@ func (m *SrvVSchema) GetKeyspaces() map[string]*Keyspace { return nil } +func (m *SrvVSchema) GetRoutingRules() *RoutingRules { + if m != nil { + return m.RoutingRules + } + return nil +} + func init() { + proto.RegisterType((*RoutingRules)(nil), "vschema.RoutingRules") + proto.RegisterType((*RoutingRule)(nil), "vschema.RoutingRule") proto.RegisterType((*Keyspace)(nil), "vschema.Keyspace") proto.RegisterMapType((map[string]*Table)(nil), "vschema.Keyspace.TablesEntry") proto.RegisterMapType((map[string]*Vindex)(nil), "vschema.Keyspace.VindexesEntry") @@ -439,44 +538,49 @@ func init() { proto.RegisterMapType((map[string]*Keyspace)(nil), "vschema.SrvVSchema.KeyspacesEntry") } -func init() { proto.RegisterFile("vschema.proto", fileDescriptor_vschema_58a865bec489dd60) } - -var fileDescriptor_vschema_58a865bec489dd60 = []byte{ - // 562 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x54, 0x41, 0x6f, 0xd3, 0x4c, - 0x10, 0x95, 0x93, 0xc6, 0x4d, 0xc6, 0x5f, 0xd2, 0x8f, 0x55, 0x29, 0xc6, 0x08, 0x35, 0xb2, 0x0a, - 0x84, 0x8b, 0x23, 0xa5, 0x42, 0x82, 0xa2, 0x22, 0x4a, 0xc4, 0xa1, 0xa2, 0x12, 0xc8, 0x8d, 0x7a, - 0xe0, 0x12, 0x6d, 0x9d, 0x11, 0xb1, 0x9a, 0xd8, 0xae, 0x77, 0x6d, 0xf0, 0x4f, 0xe1, 0x84, 0xc4, - 0x3f, 0xe0, 0x1f, 0x22, 0xef, 0xae, 0xdd, 0x75, 0x1a, 0x6e, 0xfb, 0x3c, 0xf3, 0xde, 0xbc, 0x9d, - 0x9d, 0x31, 0xf4, 0x73, 0x16, 0x2c, 0x71, 0x4d, 0xbd, 0x24, 0x8d, 0x79, 0x4c, 0x76, 0x15, 0x74, - 0xac, 0xdb, 0x0c, 0xd3, 0x42, 0x7e, 0x75, 0xff, 0xb4, 0xa0, 0xfb, 0x09, 0x0b, 0x96, 0xd0, 0x00, - 0x89, 0x0d, 0xbb, 0x6c, 0x49, 0xd3, 0x05, 0x2e, 0x6c, 0x63, 0x68, 0x8c, 0xba, 0x7e, 0x05, 0xc9, - 0x5b, 0xe8, 0xe6, 0x61, 0xb4, 0xc0, 0x1f, 0xc8, 0xec, 0xd6, 0xb0, 0x3d, 0xb2, 0x26, 0x87, 0x5e, - 0x25, 0x5f, 0xd1, 0xbd, 0x2b, 0x95, 0xf1, 0x31, 0xe2, 0x69, 0xe1, 0xd7, 0x04, 0xf2, 0x0a, 0x4c, - 0x4e, 0xaf, 0x57, 0xc8, 0xec, 0xb6, 0xa0, 0x3e, 0xbd, 0x4f, 0x9d, 0x89, 0xb8, 0x24, 0xaa, 0x64, - 0xe7, 0x02, 0xfa, 0x0d, 0x45, 0xf2, 0x3f, 0xb4, 0x6f, 0xb0, 0x10, 0xd6, 0x7a, 0x7e, 0x79, 0x24, - 0xcf, 0xa0, 0x93, 0xd3, 0x55, 0x86, 0x76, 0x6b, 0x68, 0x8c, 0xac, 0xc9, 0x5e, 0x2d, 0x2c, 0x89, - 0xbe, 0x8c, 0x9e, 0xb4, 0x5e, 0x1b, 0xce, 0x39, 0x58, 0x5a, 0x91, 0x2d, 0x5a, 0x47, 0x4d, 0xad, - 0x41, 0xad, 0x25, 0x68, 0x9a, 0x94, 0xfb, 0xdb, 0x00, 0x53, 0x16, 0x20, 0x04, 0x76, 0x78, 0x91, - 0xa0, 0xd2, 0x11, 0x67, 0x72, 0x0c, 0x66, 0x42, 0x53, 0xba, 0xae, 0x3a, 0xf5, 0x64, 0xc3, 0x95, - 0xf7, 0x45, 0x44, 0xd5, 0x65, 0x65, 0x2a, 0xd9, 0x87, 0x4e, 0xfc, 0x3d, 0xc2, 0xd4, 0x6e, 0x0b, - 0x25, 0x09, 0x9c, 0x37, 0x60, 0x69, 0xc9, 0x5b, 0x4c, 0xef, 0xeb, 0xa6, 0x7b, 0xba, 0xc9, 0x9f, - 0x2d, 0xe8, 0x08, 0xe7, 0x5b, 0x3d, 0xbe, 0x83, 0xbd, 0x20, 0x5e, 0x65, 0xeb, 0x68, 0xbe, 0xf1, - 0xac, 0x0f, 0x6b, 0xb3, 0x53, 0x11, 0x57, 0x8d, 0x1c, 0x04, 0x1a, 0x42, 0x46, 0x4e, 0x61, 0x40, - 0x33, 0x1e, 0xcf, 0xc3, 0x28, 0x48, 0x71, 0x8d, 0x11, 0x17, 0xbe, 0xad, 0xc9, 0x41, 0x4d, 0x3f, - 0xcb, 0x78, 0x7c, 0x5e, 0x45, 0xfd, 0x3e, 0xd5, 0x21, 0x79, 0x09, 0xbb, 0x52, 0x90, 0xd9, 0x3b, - 0xa2, 0xec, 0xde, 0x46, 0x59, 0xbf, 0x8a, 0x93, 0x03, 0x30, 0x93, 0x30, 0x8a, 0x70, 0x61, 0x77, - 0x84, 0x7f, 0x85, 0xc8, 0x09, 0x3c, 0x56, 0x37, 0x58, 0x85, 0x8c, 0xcf, 0x69, 0xc6, 0x97, 0x71, - 0x1a, 0x72, 0xca, 0xc3, 0x1c, 0x6d, 0x53, 0x4c, 0xef, 0x23, 0x99, 0x70, 0x11, 0x32, 0x7e, 0xa6, - 0x87, 0xdd, 0x19, 0xfc, 0xa7, 0xdf, 0xae, 0xac, 0x21, 0x53, 0x55, 0x8f, 0x14, 0x2a, 0x3b, 0x17, - 0xd1, 0x75, 0xd5, 0x5c, 0x71, 0x2e, 0x77, 0xa4, 0xb2, 0x5e, 0x4e, 0x73, 0xaf, 0x76, 0xea, 0x4e, - 0xa1, 0xdf, 0xb8, 0xf4, 0x3f, 0x65, 0x1d, 0xe8, 0x32, 0xbc, 0xcd, 0x30, 0x0a, 0x2a, 0xe9, 0x1a, - 0xbb, 0xa7, 0x60, 0x4e, 0x9b, 0xc5, 0x0d, 0xad, 0xf8, 0xa1, 0x7a, 0xca, 0x92, 0x35, 0x98, 0x58, - 0x9e, 0xdc, 0xe4, 0x59, 0x91, 0xa0, 0x7c, 0x57, 0xf7, 0x97, 0x01, 0x70, 0x99, 0xe6, 0x57, 0x97, - 0xa2, 0x99, 0xe4, 0x3d, 0xf4, 0x6e, 0xd4, 0x8a, 0x31, 0xdb, 0x10, 0x9d, 0x76, 0xeb, 0x4e, 0xdf, - 0xe5, 0xd5, 0x7b, 0xa8, 0x86, 0xf2, 0x8e, 0xe4, 0x7c, 0x86, 0x41, 0x33, 0xb8, 0x65, 0x08, 0x5f, - 0x34, 0x37, 0xe7, 0xc1, 0xbd, 0xf5, 0xd6, 0xe6, 0xf2, 0xc3, 0xf3, 0xaf, 0x47, 0x79, 0xc8, 0x91, - 0x31, 0x2f, 0x8c, 0xc7, 0xf2, 0x34, 0xfe, 0x16, 0x8f, 0x73, 0x3e, 0x16, 0x3f, 0xa4, 0xb1, 0xe2, - 0x5e, 0x9b, 0x02, 0x1e, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x41, 0x18, 0x64, 0xd4, 0xc6, 0x04, - 0x00, 0x00, +func init() { proto.RegisterFile("vschema.proto", fileDescriptor_vschema_ddae95e7e0992d00) } + +var fileDescriptor_vschema_ddae95e7e0992d00 = []byte{ + // 643 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x54, 0xdd, 0x6e, 0xd3, 0x30, + 0x14, 0x56, 0xda, 0x35, 0x6b, 0x4f, 0xd6, 0x0e, 0xac, 0x6d, 0x84, 0x4e, 0xd3, 0xaa, 0x68, 0x40, + 0xe1, 0xa2, 0x95, 0x3a, 0x21, 0x41, 0xd1, 0x10, 0x63, 0xe2, 0x62, 0x62, 0x12, 0x28, 0x9b, 0x76, + 0xc1, 0x4d, 0xe5, 0xb5, 0x66, 0x8b, 0xd6, 0xc6, 0x99, 0xed, 0x04, 0xf2, 0x28, 0xdc, 0xf2, 0x06, + 0x3c, 0x0f, 0x2f, 0x83, 0xe2, 0x9f, 0xcc, 0xe9, 0xca, 0x9d, 0x3f, 0x9f, 0xf3, 0x7d, 0xe7, 0xf3, + 0xb1, 0x7d, 0xa0, 0x9d, 0xf1, 0xe9, 0x0d, 0x59, 0xe0, 0x41, 0xc2, 0xa8, 0xa0, 0x68, 0x5d, 0xc3, + 0xae, 0x77, 0x97, 0x12, 0x96, 0xab, 0xdd, 0x60, 0x0c, 0x1b, 0x21, 0x4d, 0x45, 0x14, 0x5f, 0x87, + 0xe9, 0x9c, 0x70, 0xf4, 0x0a, 0x1a, 0xac, 0x58, 0xf8, 0x4e, 0xaf, 0xde, 0xf7, 0x46, 0x5b, 0x03, + 0x23, 0x62, 0x65, 0x85, 0x2a, 0x25, 0x38, 0x05, 0xcf, 0xda, 0x45, 0x7b, 0x00, 0xdf, 0x19, 0x5d, + 0x4c, 0x04, 0xbe, 0x9a, 0x13, 0xdf, 0xe9, 0x39, 0xfd, 0x56, 0xd8, 0x2a, 0x76, 0x2e, 0x8a, 0x0d, + 0xb4, 0x0b, 0x2d, 0x41, 0x55, 0x90, 0xfb, 0xb5, 0x5e, 0xbd, 0xdf, 0x0a, 0x9b, 0x82, 0xca, 0x18, + 0x0f, 0xfe, 0xd4, 0xa0, 0xf9, 0x99, 0xe4, 0x3c, 0xc1, 0x53, 0x82, 0x7c, 0x58, 0xe7, 0x37, 0x98, + 0xcd, 0xc8, 0x4c, 0xaa, 0x34, 0x43, 0x03, 0xd1, 0x3b, 0x68, 0x66, 0x51, 0x3c, 0x23, 0x3f, 0xb5, + 0x84, 0x37, 0xda, 0x2f, 0x0d, 0x1a, 0xfa, 0xe0, 0x52, 0x67, 0x7c, 0x8a, 0x05, 0xcb, 0xc3, 0x92, + 0x80, 0x5e, 0x83, 0xab, 0xab, 0xd7, 0x25, 0x75, 0xef, 0x21, 0x55, 0xb9, 0x51, 0x44, 0x9d, 0xdc, + 0x3d, 0x83, 0x76, 0x45, 0x11, 0x3d, 0x82, 0xfa, 0x2d, 0xc9, 0xf5, 0x01, 0x8b, 0x25, 0x7a, 0x06, + 0x8d, 0x0c, 0xcf, 0x53, 0xe2, 0xd7, 0x7a, 0x4e, 0xdf, 0x1b, 0x6d, 0x96, 0xc2, 0x8a, 0x18, 0xaa, + 0xe8, 0xb8, 0xf6, 0xc6, 0xe9, 0x9e, 0x82, 0x67, 0x15, 0x59, 0xa1, 0x75, 0x50, 0xd5, 0xea, 0x94, + 0x5a, 0x92, 0x66, 0x49, 0x05, 0xbf, 0x1d, 0x70, 0x55, 0x01, 0x84, 0x60, 0x4d, 0xe4, 0x89, 0x69, + 0xba, 0x5c, 0xa3, 0x43, 0x70, 0x13, 0xcc, 0xf0, 0xc2, 0x74, 0x6a, 0x77, 0xc9, 0xd5, 0xe0, 0xab, + 0x8c, 0xea, 0xc3, 0xaa, 0x54, 0xb4, 0x05, 0x0d, 0xfa, 0x23, 0x26, 0xcc, 0xaf, 0x4b, 0x25, 0x05, + 0xba, 0x6f, 0xc1, 0xb3, 0x92, 0x57, 0x98, 0xde, 0xb2, 0x4d, 0xb7, 0x6c, 0x93, 0xbf, 0x6a, 0xd0, + 0x50, 0xf7, 0xbf, 0xca, 0xe3, 0x7b, 0xd8, 0x9c, 0xd2, 0x79, 0xba, 0x88, 0x27, 0x4b, 0xd7, 0xba, + 0x5d, 0x9a, 0x3d, 0x91, 0x71, 0xdd, 0xc8, 0xce, 0xd4, 0x42, 0x84, 0xa3, 0x23, 0xe8, 0xe0, 0x54, + 0xd0, 0x49, 0x14, 0x4f, 0x19, 0x59, 0x90, 0x58, 0x48, 0xdf, 0xde, 0x68, 0xa7, 0xa4, 0x1f, 0xa7, + 0x82, 0x9e, 0x9a, 0x68, 0xd8, 0xc6, 0x36, 0x44, 0x2f, 0x61, 0x5d, 0x09, 0x72, 0x7f, 0x4d, 0x96, + 0xdd, 0x5c, 0x2a, 0x1b, 0x9a, 0x38, 0xda, 0x01, 0x37, 0x89, 0xe2, 0x98, 0xcc, 0xfc, 0x86, 0xf4, + 0xaf, 0x11, 0x1a, 0xc3, 0x53, 0x7d, 0x82, 0x79, 0xc4, 0xc5, 0x04, 0xa7, 0xe2, 0x86, 0xb2, 0x48, + 0x60, 0x11, 0x65, 0xc4, 0x77, 0xe5, 0xeb, 0x7d, 0xa2, 0x12, 0xce, 0x22, 0x2e, 0x8e, 0xed, 0x70, + 0x70, 0x01, 0x1b, 0xf6, 0xe9, 0x8a, 0x1a, 0x2a, 0x55, 0xf7, 0x48, 0xa3, 0xa2, 0x73, 0x31, 0x5e, + 0x98, 0xe6, 0xca, 0x75, 0xf1, 0x47, 0x8c, 0xf5, 0xba, 0xfc, 0x4b, 0x06, 0x06, 0x27, 0xd0, 0xae, + 0x1c, 0xfa, 0xbf, 0xb2, 0x5d, 0x68, 0x72, 0x72, 0x97, 0x92, 0x78, 0x6a, 0xa4, 0x4b, 0x1c, 0x1c, + 0x81, 0x7b, 0x52, 0x2d, 0xee, 0x58, 0xc5, 0xf7, 0xf5, 0x55, 0x16, 0xac, 0xce, 0xc8, 0x1b, 0xa8, + 0x81, 0x72, 0x91, 0x27, 0x44, 0xdd, 0x6b, 0xf0, 0xd7, 0x01, 0x38, 0x67, 0xd9, 0xe5, 0xb9, 0x6c, + 0x26, 0xfa, 0x00, 0xad, 0x5b, 0xfd, 0xc5, 0xcc, 0x60, 0x09, 0xca, 0x4e, 0xdf, 0xe7, 0x95, 0xff, + 0x50, 0x3f, 0xca, 0x7b, 0x12, 0x1a, 0x43, 0x9b, 0xa9, 0x51, 0x33, 0x51, 0xe3, 0x49, 0xfd, 0x8e, + 0xed, 0x55, 0xe3, 0x89, 0x87, 0x1b, 0xcc, 0x42, 0xdd, 0x2f, 0xd0, 0xa9, 0x0a, 0xaf, 0x78, 0xc0, + 0x2f, 0xaa, 0xbf, 0xee, 0xf1, 0x83, 0xd1, 0x60, 0xbd, 0xe9, 0x8f, 0xcf, 0xbf, 0x1d, 0x64, 0x91, + 0x20, 0x9c, 0x0f, 0x22, 0x3a, 0x54, 0xab, 0xe1, 0x35, 0x1d, 0x66, 0x62, 0x28, 0x67, 0xea, 0x50, + 0x73, 0xaf, 0x5c, 0x09, 0x0f, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xc8, 0x0d, 0x93, 0x48, 0x89, + 0x05, 0x00, 0x00, } diff --git a/go/vt/proto/vtgateservice/vtgateservice.pb.go b/go/vt/proto/vtgateservice/vtgateservice.pb.go index 3a0bc073da8..921b5403fdc 100644 --- a/go/vt/proto/vtgateservice/vtgateservice.pb.go +++ b/go/vt/proto/vtgateservice/vtgateservice.pb.go @@ -39,17 +39,17 @@ const _ = grpc.SupportPackageIsVersion4 type VitessClient interface { // Execute tries to route the query to the right shard. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 Execute(ctx context.Context, in *vtgate.ExecuteRequest, opts ...grpc.CallOption) (*vtgate.ExecuteResponse, error) // ExecuteBatch tries to route the list of queries on the right shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 ExecuteBatch(ctx context.Context, in *vtgate.ExecuteBatchRequest, opts ...grpc.CallOption) (*vtgate.ExecuteBatchResponse, error) // StreamExecute executes a streaming query based on shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // Use this method if the query returns a large number of rows. // API group: v3 StreamExecute(ctx context.Context, in *vtgate.StreamExecuteRequest, opts ...grpc.CallOption) (Vitess_StreamExecuteClient, error) @@ -467,17 +467,17 @@ func (x *vitessUpdateStreamClient) Recv() (*vtgate.UpdateStreamResponse, error) type VitessServer interface { // Execute tries to route the query to the right shard. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 Execute(context.Context, *vtgate.ExecuteRequest) (*vtgate.ExecuteResponse, error) // ExecuteBatch tries to route the list of queries on the right shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 ExecuteBatch(context.Context, *vtgate.ExecuteBatchRequest) (*vtgate.ExecuteBatchResponse, error) // StreamExecute executes a streaming query based on shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // Use this method if the query returns a large number of rows. // API group: v3 StreamExecute(*vtgate.StreamExecuteRequest, Vitess_StreamExecuteServer) error diff --git a/go/vt/schemamanager/schemaswap/schema_swap.go b/go/vt/schemamanager/schemaswap/schema_swap.go index b32b0e0c146..8a947319087 100644 --- a/go/vt/schemamanager/schemaswap/schema_swap.go +++ b/go/vt/schemamanager/schemaswap/schema_swap.go @@ -39,6 +39,7 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" workflowpb "vitess.io/vitess/go/vt/proto/workflow" "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/vtctl" "vitess.io/vitess/go/vt/vttablet/tmclient" "vitess.io/vitess/go/vt/workflow" @@ -55,8 +56,8 @@ var ( reparentTimeout = flag.Duration("schema_swap_reparent_timeout", 30*time.Second, "timeout to wait for slaves when doing reparent during schema swap") - errOnlyMasterLeft = errors.New("Only master is left to swap schema") - errNoBackupWithSwap = errors.New("Restore from backup cannot pick up new schema") + errOnlyMasterLeft = errors.New("only master is left to swap schema") + errNoBackupWithSwap = errors.New("restore from backup cannot pick up new schema") ) const ( @@ -187,7 +188,7 @@ func (*SwapWorkflowFactory) Init(_ *workflow.Manager, workflowProto *workflowpb. return err } if *keyspace == "" || *sql == "" { - return fmt.Errorf("Keyspace name and SQL query must be provided for schema swap") + return fmt.Errorf("keyspace name and SQL query must be provided for schema swap") } workflowProto.Name = fmt.Sprintf("Schema swap on keyspace %s", *keyspace) @@ -283,7 +284,7 @@ func (schemaSwap *Swap) closeRetryChannel() { // from UI. func (schemaSwap *Swap) Action(ctx context.Context, path, name string) error { if name != "Retry" { - return fmt.Errorf("Unknown action on schema swap: %v", name) + return fmt.Errorf("unknown action on schema swap: %v", name) } schemaSwap.closeRetryChannel() schemaSwap.rootUINode.BroadcastChanges(false /* updateChildren */) @@ -477,12 +478,12 @@ func (schemaSwap *Swap) initializeSwap() error { recorder.RecordError(metadata.err) } else if metadata.lastStartedSwap < metadata.lastFinishedSwap || metadata.lastStartedSwap > metadata.lastFinishedSwap+1 { recorder.RecordError(fmt.Errorf( - "Bad swap metadata on shard %v: LastFinishedSchemaSwap=%v, LastStartedSchemaSwap=%v", + "bad swap metadata on shard %v: LastFinishedSchemaSwap=%v, LastStartedSchemaSwap=%v", schemaSwap.allShards[i].shardName, metadata.lastFinishedSwap, metadata.lastStartedSwap)) } else if metadata.lastStartedSwap != metadata.lastFinishedSwap { if metadata.currentSQL != schemaSwap.sql { recorder.RecordError(fmt.Errorf( - "Shard %v has an already started schema swap with a different set of SQL statements", + "shard %v has an already started schema swap with a different set of SQL statements", schemaSwap.allShards[i].shardName)) } } @@ -501,7 +502,7 @@ func (schemaSwap *Swap) initializeSwap() error { // The shard doesn't have schema swap started yet or it's already finished. if schemaSwap.swapID != metadata.lastFinishedSwap && schemaSwap.swapID != metadata.lastFinishedSwap+1 { recorder.RecordError(fmt.Errorf( - "Shard %v has last finished swap id euqal to %v which doesn't align with swap id for the keyspace equal to %v", + "shard %v has last finished swap id euqal to %v which doesn't align with swap id for the keyspace equal to %v", schemaSwap.allShards[i].shardName, metadata.lastFinishedSwap, schemaSwap.swapID)) } else if schemaSwap.swapID == metadata.lastFinishedSwap { haveFinishedSwap = true @@ -510,7 +511,7 @@ func (schemaSwap *Swap) initializeSwap() error { } } else if schemaSwap.swapID != metadata.lastStartedSwap { recorder.RecordError(fmt.Errorf( - "Shard %v has an already started schema swap with an id %v, while for the keyspace it should be equal to %v", + "shard %v has an already started schema swap with an id %v, while for the keyspace it should be equal to %v", schemaSwap.allShards[i].shardName, metadata.lastStartedSwap, schemaSwap.swapID)) } } @@ -603,8 +604,8 @@ func (shardSwap *shardSchemaSwap) readShardMetadata(metadata *shardSwapMetadata, return } query := fmt.Sprintf( - "SELECT name, value FROM _vt.shard_metadata WHERE name in ('%s', '%s', '%s')", - lastStartedMetadataName, lastFinishedMetadataName, currentSQLMetadataName) + "SELECT name, value FROM _vt.shard_metadata WHERE db_name = '%s' and name in ('%s', '%s', '%s')", + topoproto.TabletDbName(tablet), lastStartedMetadataName, lastFinishedMetadataName, currentSQLMetadataName) queryResult, err := shardSwap.executeAdminQuery(tablet, query, 3 /* maxRows */) if err != nil { metadata.err = err @@ -640,7 +641,9 @@ func (shardSwap *shardSchemaSwap) writeStartedSwap() error { return err } queryBuf := bytes.Buffer{} - queryBuf.WriteString("INSERT INTO _vt.shard_metadata (name, value) VALUES ('") + queryBuf.WriteString("INSERT INTO _vt.shard_metadata (db_name, name, value) VALUES ('") + queryBuf.WriteString(topoproto.TabletDbName(tablet)) + queryBuf.WriteString("',") queryBuf.WriteString(currentSQLMetadataName) queryBuf.WriteString("',") sqlValue := sqltypes.NewVarChar(shardSwap.parent.sql) @@ -666,13 +669,13 @@ func (shardSwap *shardSchemaSwap) writeFinishedSwap() error { return err } query := fmt.Sprintf( - "INSERT INTO _vt.shard_metadata (name, value) VALUES ('%s', '%d') ON DUPLICATE KEY UPDATE value = '%d'", - lastFinishedMetadataName, shardSwap.parent.swapID, shardSwap.parent.swapID) + "INSERT INTO _vt.shard_metadata (db_name, name, value) VALUES ('%s', '%s', '%d') ON DUPLICATE KEY UPDATE value = '%d'", + topoproto.TabletDbName(tablet), lastFinishedMetadataName, shardSwap.parent.swapID, shardSwap.parent.swapID) _, err = shardSwap.executeAdminQuery(tablet, query, 0 /* maxRows */) if err != nil { return err } - query = fmt.Sprintf("DELETE FROM _vt.shard_metadata WHERE name = '%s'", currentSQLMetadataName) + query = fmt.Sprintf("DELETE FROM _vt.shard_metadata WHERE db_name = '%s' AND name = '%s'", topoproto.TabletDbName(tablet), currentSQLMetadataName) _, err = shardSwap.executeAdminQuery(tablet, query, 0 /* maxRows */) return err } @@ -761,7 +764,7 @@ func (shardSwap *shardSchemaSwap) startWaitingOnUnhealthyTablet(tablet *topodata tabletKey := discovery.TabletToMapKey(tablet) tabletStats, tabletFound := shardSwap.allTablets[tabletKey] if !tabletFound { - return nil, fmt.Errorf("Tablet %v has disappeared while doing schema swap", tablet.Alias) + return nil, fmt.Errorf("tablet %v has disappeared while doing schema swap", tablet.Alias) } if isTabletHealthy(tabletStats) { return nil, nil @@ -896,7 +899,7 @@ func (shardSwap *shardSchemaSwap) executeAdminQuery(tablet *topodatapb.Tablet, q func (shardSwap *shardSchemaSwap) isSwapApplied(tablet *topodatapb.Tablet) (bool, error) { swapIDResult, err := shardSwap.executeAdminQuery( tablet, - fmt.Sprintf("SELECT value FROM _vt.local_metadata WHERE name = '%s'", lastAppliedMetadataName), + fmt.Sprintf("SELECT value FROM _vt.local_metadata WHERE db_name = '%s' AND name = '%s'", topoproto.TabletDbName(tablet), lastAppliedMetadataName), 1 /* maxRows */) if err != nil { return false, err @@ -1036,8 +1039,8 @@ func (shardSwap *shardSchemaSwap) applySeedSchemaChange() (err error) { return err } updateAppliedSwapQuery := fmt.Sprintf( - "INSERT INTO _vt.local_metadata (name, value) VALUES ('%s', '%d') ON DUPLICATE KEY UPDATE value = '%d'", - lastAppliedMetadataName, shardSwap.parent.swapID, shardSwap.parent.swapID) + "INSERT INTO _vt.local_metadata (db_name, name, value) VALUES ('%s', '%s', '%d') ON DUPLICATE KEY UPDATE value = '%d'", + topoproto.TabletDbName(seedTablet), lastAppliedMetadataName, shardSwap.parent.swapID, shardSwap.parent.swapID) _, err = shardSwap.parent.tabletClient.ExecuteFetchAsDba( shardSwap.parent.ctx, seedTablet, @@ -1286,7 +1289,7 @@ func (shardSwap *shardSchemaSwap) reparentFromMaster(masterTablet *topodatapb.Ta return err } if hookResult.ExitStatus != hook.HOOK_SUCCESS { - return fmt.Errorf("Error executing 'reparent_away' hook: %v", hookResult.String()) + return fmt.Errorf("error executing 'reparent_away' hook: %v", hookResult.String()) } } else { wr := wrangler.New(logutil.NewConsoleLogger(), shardSwap.parent.topoServer, shardSwap.parent.tabletClient) diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go index fc7c4b45e8d..44d47cfcdcb 100644 --- a/go/vt/sqlparser/ast.go +++ b/go/vt/sqlparser/ast.go @@ -532,7 +532,7 @@ func (node *Stream) walkSubtree(visit Visit) error { // the row and re-inserts with new values. For that reason we keep it as an Insert struct. // Replaces are currently disallowed in sharded schemas because // of the implications the deletion part may have on vindexes. -// If you add fields here, consider adding them to calls to validateSubquerySamePlan. +// If you add fields here, consider adding them to calls to validateUnshardedRoute. type Insert struct { Action string Comments Comments @@ -584,7 +584,7 @@ func (Values) iInsertRows() {} func (*ParenSelect) iInsertRows() {} // Update represents an UPDATE statement. -// If you add fields here, consider adding them to calls to validateSubquerySamePlan. +// If you add fields here, consider adding them to calls to validateUnshardedRoute. type Update struct { Comments Comments Ignore string @@ -618,7 +618,7 @@ func (node *Update) walkSubtree(visit Visit) error { } // Delete represents a DELETE statement. -// If you add fields here, consider adding them to calls to validateSubquerySamePlan. +// If you add fields here, consider adding them to calls to validateUnshardedRoute. type Delete struct { Comments Comments Targets TableNames diff --git a/go/vt/sqlparser/normalizer.go b/go/vt/sqlparser/normalizer.go index a2ac08337ab..ac223477e27 100644 --- a/go/vt/sqlparser/normalizer.go +++ b/go/vt/sqlparser/normalizer.go @@ -88,6 +88,9 @@ func (nz *normalizer) WalkSelect(node SQLNode) (bool, error) { // Common node types that never contain SQLVals or ListArgs but create a lot of object // allocations. return false, nil + case OrderBy, GroupBy: + // do not make a bind var for order by column_position + return false, nil } return true, nil } diff --git a/go/vt/sqlparser/normalizer_test.go b/go/vt/sqlparser/normalizer_test.go index 7aa24814d55..b102b0ec2ad 100644 --- a/go/vt/sqlparser/normalizer_test.go +++ b/go/vt/sqlparser/normalizer_test.go @@ -127,6 +127,16 @@ func TestNormalize(t *testing.T) { in: "update a set v1 = b'11'", outstmt: "update a set v1 = B'11'", outbv: map[string]*querypb.BindVariable{}, + }, { + // ORDER BY column_position + in: "select a, b from t order by 1 asc", + outstmt: "select a, b from t order by 1 asc", + outbv: map[string]*querypb.BindVariable{}, + }, { + // ORDER BY variable + in: "select a, b from t order by c asc", + outstmt: "select a, b from t order by c asc", + outbv: map[string]*querypb.BindVariable{}, }, { // Values up to len 256 will reuse. in: fmt.Sprintf("select * from t where v1 = '%256s' and v2 = '%256s'", "a", "a"), diff --git a/go/vt/sqlparser/parse_test.go b/go/vt/sqlparser/parse_test.go index 4ba5574c629..a189a51cbb3 100644 --- a/go/vt/sqlparser/parse_test.go +++ b/go/vt/sqlparser/parse_test.go @@ -1120,6 +1120,9 @@ var ( }, { input: "show databases", output: "show databases", + }, { + input: "show schemas", + output: "show schemas", }, { input: "show engine INNODB", output: "show engine", diff --git a/go/vt/sqlparser/parsed_query.go b/go/vt/sqlparser/parsed_query.go index 8fcce8c98aa..2f6c4e705e7 100644 --- a/go/vt/sqlparser/parsed_query.go +++ b/go/vt/sqlparser/parsed_query.go @@ -53,23 +53,31 @@ func (pq *ParsedQuery) GenerateQuery(bindVariables map[string]*querypb.BindVaria } var buf strings.Builder buf.Grow(len(pq.Query)) + if err := pq.Append(&buf, bindVariables, extras); err != nil { + return "", err + } + return buf.String(), nil +} + +// Append appends the generated query to the provided buffer. +func (pq *ParsedQuery) Append(buf *strings.Builder, bindVariables map[string]*querypb.BindVariable, extras map[string]Encodable) error { current := 0 for _, loc := range pq.bindLocations { buf.WriteString(pq.Query[current:loc.offset]) name := pq.Query[loc.offset : loc.offset+loc.length] if encodable, ok := extras[name[1:]]; ok { - encodable.EncodeSQL(&buf) + encodable.EncodeSQL(buf) } else { supplied, _, err := FetchBindVar(name, bindVariables) if err != nil { - return "", err + return err } - EncodeValue(&buf, supplied) + EncodeValue(buf, supplied) } current = loc.offset + loc.length } buf.WriteString(pq.Query[current:]) - return buf.String(), nil + return nil } // MarshalJSON is a custom JSON marshaler for ParsedQuery. diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index 65cb1ea8f66..3de30b98c80 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -302,58 +302,59 @@ const UNSIGNED = 57537 const ZEROFILL = 57538 const COLLATION = 57539 const DATABASES = 57540 -const TABLES = 57541 -const VITESS_KEYSPACES = 57542 -const VITESS_SHARDS = 57543 -const VITESS_TABLETS = 57544 -const VSCHEMA = 57545 -const VSCHEMA_TABLES = 57546 -const VITESS_TARGET = 57547 -const FULL = 57548 -const PROCESSLIST = 57549 -const COLUMNS = 57550 -const FIELDS = 57551 -const ENGINES = 57552 -const PLUGINS = 57553 -const NAMES = 57554 -const CHARSET = 57555 -const GLOBAL = 57556 -const SESSION = 57557 -const ISOLATION = 57558 -const LEVEL = 57559 -const READ = 57560 -const WRITE = 57561 -const ONLY = 57562 -const REPEATABLE = 57563 -const COMMITTED = 57564 -const UNCOMMITTED = 57565 -const SERIALIZABLE = 57566 -const CURRENT_TIMESTAMP = 57567 -const DATABASE = 57568 -const CURRENT_DATE = 57569 -const CURRENT_TIME = 57570 -const LOCALTIME = 57571 -const LOCALTIMESTAMP = 57572 -const UTC_DATE = 57573 -const UTC_TIME = 57574 -const UTC_TIMESTAMP = 57575 -const REPLACE = 57576 -const CONVERT = 57577 -const CAST = 57578 -const SUBSTR = 57579 -const SUBSTRING = 57580 -const GROUP_CONCAT = 57581 -const SEPARATOR = 57582 -const TIMESTAMPADD = 57583 -const TIMESTAMPDIFF = 57584 -const MATCH = 57585 -const AGAINST = 57586 -const BOOLEAN = 57587 -const LANGUAGE = 57588 -const WITH = 57589 -const QUERY = 57590 -const EXPANSION = 57591 -const UNUSED = 57592 +const SCHEMAS = 57541 +const TABLES = 57542 +const VITESS_KEYSPACES = 57543 +const VITESS_SHARDS = 57544 +const VITESS_TABLETS = 57545 +const VSCHEMA = 57546 +const VSCHEMA_TABLES = 57547 +const VITESS_TARGET = 57548 +const FULL = 57549 +const PROCESSLIST = 57550 +const COLUMNS = 57551 +const FIELDS = 57552 +const ENGINES = 57553 +const PLUGINS = 57554 +const NAMES = 57555 +const CHARSET = 57556 +const GLOBAL = 57557 +const SESSION = 57558 +const ISOLATION = 57559 +const LEVEL = 57560 +const READ = 57561 +const WRITE = 57562 +const ONLY = 57563 +const REPEATABLE = 57564 +const COMMITTED = 57565 +const UNCOMMITTED = 57566 +const SERIALIZABLE = 57567 +const CURRENT_TIMESTAMP = 57568 +const DATABASE = 57569 +const CURRENT_DATE = 57570 +const CURRENT_TIME = 57571 +const LOCALTIME = 57572 +const LOCALTIMESTAMP = 57573 +const UTC_DATE = 57574 +const UTC_TIME = 57575 +const UTC_TIMESTAMP = 57576 +const REPLACE = 57577 +const CONVERT = 57578 +const CAST = 57579 +const SUBSTR = 57580 +const SUBSTRING = 57581 +const GROUP_CONCAT = 57582 +const SEPARATOR = 57583 +const TIMESTAMPADD = 57584 +const TIMESTAMPDIFF = 57585 +const MATCH = 57586 +const AGAINST = 57587 +const BOOLEAN = 57588 +const LANGUAGE = 57589 +const WITH = 57590 +const QUERY = 57591 +const EXPANSION = 57592 +const UNUSED = 57593 var yyToknames = [...]string{ "$end", @@ -571,6 +572,7 @@ var yyToknames = [...]string{ "ZEROFILL", "COLLATION", "DATABASES", + "SCHEMAS", "TABLES", "VITESS_KEYSPACES", "VITESS_SHARDS", @@ -640,1361 +642,1394 @@ var yyExca = [...]int{ 5, 29, -2, 4, -1, 37, - 159, 295, - 160, 295, - -2, 285, - -1, 268, - 112, 640, - -2, 636, - -1, 269, + 159, 296, + 160, 296, + -2, 286, + -1, 270, 112, 641, -2, 637, - -1, 334, - 82, 814, + -1, 271, + 112, 642, + -2, 638, + -1, 336, + 82, 816, -2, 60, - -1, 335, - 82, 771, + -1, 337, + 82, 772, -2, 61, - -1, 340, - 82, 750, - -2, 602, -1, 342, - 82, 792, - -2, 604, - -1, 611, - 1, 347, - 5, 347, - 12, 347, - 13, 347, - 14, 347, - 15, 347, - 17, 347, - 19, 347, - 30, 347, - 31, 347, - 42, 347, - 43, 347, - 44, 347, - 45, 347, - 46, 347, - 48, 347, - 49, 347, - 52, 347, - 53, 347, - 55, 347, - 56, 347, - 268, 347, - -2, 365, + 82, 751, + -2, 603, + -1, 344, + 82, 793, + -2, 605, -1, 614, + 1, 348, + 5, 348, + 12, 348, + 13, 348, + 14, 348, + 15, 348, + 17, 348, + 19, 348, + 30, 348, + 31, 348, + 42, 348, + 43, 348, + 44, 348, + 45, 348, + 46, 348, + 48, 348, + 49, 348, + 52, 348, + 53, 348, + 55, 348, + 56, 348, + 269, 348, + -2, 366, + -1, 617, 53, 43, 55, 43, -2, 45, - -1, 756, - 112, 643, - -2, 639, - -1, 977, + -1, 759, + 112, 644, + -2, 640, + -1, 980, 5, 30, - -2, 431, - -1, 1009, + -2, 432, + -1, 1012, 5, 29, - -2, 576, - -1, 1261, - 5, 30, -2, 577, - -1, 1317, - 5, 29, - -2, 579, - -1, 1398, + -1, 1264, 5, 30, + -2, 578, + -1, 1320, + 5, 29, -2, 580, + -1, 1401, + 5, 30, + -2, 581, } const yyPrivate = 57344 -const yyLast = 12688 +const yyLast = 13012 var yyAct = [...]int{ - 269, 1432, 1422, 1386, 1101, 1012, 1216, 1281, 567, 1294, - 1329, 299, 1030, 1156, 286, 273, 1190, 863, 840, 247, - 57, 906, 942, 1013, 1157, 1153, 1055, 862, 1169, 1163, - 1036, 238, 81, 872, 1124, 892, 209, 339, 969, 209, - 791, 788, 859, 1081, 471, 300, 51, 781, 721, 1072, - 838, 624, 842, 827, 566, 3, 607, 876, 758, 809, - 499, 790, 505, 902, 440, 820, 333, 209, 81, 623, - 271, 951, 209, 608, 209, 511, 239, 240, 241, 242, - 519, 328, 245, 256, 330, 311, 582, 317, 318, 315, - 316, 314, 313, 312, 56, 1425, 1409, 51, 1420, 1396, - 1417, 319, 320, 1217, 260, 252, 246, 581, 1408, 1395, - 1146, 1253, 336, 445, 625, 473, 626, 61, 886, 1361, - 532, 531, 541, 542, 534, 535, 536, 537, 538, 539, - 540, 533, 1185, 1186, 543, 854, 855, 925, 204, 200, - 201, 202, 1184, 63, 64, 65, 66, 67, 489, 1043, - 853, 924, 1042, 1125, 244, 1044, 490, 487, 488, 196, - 493, 198, 243, 1063, 885, 1284, 1300, 893, 1236, 1234, - 237, 469, 482, 483, 696, 458, 1104, 1103, 694, 929, - 1419, 475, 1416, 477, 1387, 877, 1100, 821, 923, 1379, - 1127, 1330, 1440, 1338, 459, 447, 198, 209, 1105, 700, - 209, 1088, 1031, 1033, 1332, 687, 209, 1179, 695, 1436, - 1178, 1177, 209, 474, 476, 81, 879, 81, 492, 81, - 81, 879, 81, 1129, 81, 1133, 443, 1128, 697, 1126, - 1086, 1097, 1368, 81, 1131, 450, 211, 1099, 920, 917, - 918, 199, 916, 1130, 936, 1056, 1264, 935, 555, 556, - 727, 1111, 987, 963, 197, 730, 1132, 1134, 523, 203, - 470, 465, 470, 81, 470, 470, 860, 470, 1202, 470, - 543, 507, 1331, 927, 930, 495, 496, 455, 470, 1032, - 531, 541, 542, 534, 535, 536, 537, 538, 539, 540, - 533, 1362, 722, 543, 1339, 1337, 893, 1087, 51, 518, - 70, 472, 1092, 1089, 1082, 1090, 1085, 508, 944, 922, - 1083, 1084, 878, 552, 275, 1434, 554, 878, 1435, 1203, - 1433, 1377, 1394, 516, 1091, 1347, 209, 209, 209, 1167, - 627, 921, 81, 1098, 533, 1096, 71, 543, 81, 518, - 452, 1148, 453, 810, 565, 454, 569, 570, 571, 572, - 573, 574, 575, 576, 577, 1441, 580, 583, 583, 583, - 589, 583, 583, 589, 583, 597, 598, 599, 600, 601, - 602, 509, 612, 723, 926, 441, 336, 606, 461, 462, - 463, 810, 765, 996, 689, 879, 943, 446, 478, 928, - 479, 480, 1061, 481, 1442, 484, 763, 764, 762, 585, - 587, 1382, 591, 593, 494, 596, 733, 734, 439, 615, - 513, 748, 750, 751, 441, 1400, 621, 749, 1290, 812, - 584, 586, 588, 590, 592, 594, 595, 532, 531, 541, - 542, 534, 535, 536, 537, 538, 539, 540, 533, 54, - 209, 543, 982, 517, 516, 81, 517, 516, 1289, 761, - 209, 209, 81, 1150, 517, 516, 209, 1076, 1075, 209, - 518, 882, 209, 518, 448, 449, 209, 883, 81, 81, - 729, 518, 1064, 81, 81, 81, 209, 81, 81, 22, - 970, 878, 1402, 81, 81, 1378, 875, 873, 1311, 874, - 470, 1287, 517, 516, 871, 877, 1108, 470, 298, 960, - 961, 962, 1073, 709, 1249, 498, 498, 728, 782, 518, - 783, 81, 1375, 470, 470, 209, 195, 1219, 470, 470, - 470, 81, 470, 470, 517, 516, 1056, 735, 470, 470, - 79, 1246, 498, 54, 1045, 1051, 1046, 707, 759, 251, - 701, 518, 532, 531, 541, 542, 534, 535, 536, 537, - 538, 539, 540, 533, 1335, 1418, 543, 1404, 498, 1335, - 1390, 756, 1335, 498, 1344, 81, 338, 784, 755, 532, - 531, 541, 542, 534, 535, 536, 537, 538, 539, 540, - 533, 325, 326, 543, 981, 553, 980, 1335, 1369, 1343, - 800, 804, 706, 737, 705, 754, 811, 690, 81, 81, - 51, 752, 688, 517, 516, 209, 1335, 1334, 1279, 1278, - 1266, 498, 1199, 209, 209, 569, 686, 209, 209, 685, - 518, 81, 467, 693, 795, 460, 785, 786, 1263, 498, - 1209, 1208, 796, 797, 81, 1205, 1206, 266, 806, 710, - 711, 611, 817, 58, 712, 713, 714, 1037, 716, 717, - 1205, 1204, 880, 807, 718, 719, 975, 498, 839, 824, - 498, 816, 612, 818, 819, 336, 497, 848, 618, 534, - 535, 536, 537, 538, 539, 540, 533, 823, 864, 543, - 846, 793, 498, 975, 894, 895, 896, 850, 209, 81, - 824, 81, 851, 634, 633, 81, 81, 209, 209, 867, - 209, 209, 824, 1166, 209, 81, 908, 1410, 1154, 24, - 619, 1166, 617, 338, 793, 338, 1114, 338, 338, 24, - 338, 209, 338, 209, 209, 991, 209, 1259, 847, 1037, - 617, 338, 989, 1007, 470, 1346, 470, 1008, 824, 1207, - 1047, 852, 1001, 1243, 498, 904, 905, 1316, 1000, 986, - 470, 536, 537, 538, 539, 540, 533, 54, 984, 543, - 975, 521, 888, 889, 890, 891, 975, 54, 756, 990, - 1240, 498, 1166, 617, 759, 755, 988, 620, 899, 900, - 901, 532, 531, 541, 542, 534, 535, 536, 537, 538, - 539, 540, 533, 985, 952, 543, 953, 24, 731, 699, - 1296, 964, 983, 887, 959, 1271, 907, 1195, 532, 531, - 541, 542, 534, 535, 536, 537, 538, 539, 540, 533, - 1050, 965, 543, 1170, 1171, 209, 209, 209, 209, 209, - 338, 903, 898, 897, 253, 1102, 629, 209, 910, 1014, - 209, 760, 1427, 743, 209, 54, 1423, 1197, 209, 1173, - 1154, 974, 829, 832, 833, 834, 830, 1077, 831, 835, - 912, 725, 914, 81, 703, 1176, 1010, 1011, 995, 993, - 612, 612, 612, 612, 612, 1009, 940, 1175, 1024, 1039, - 1048, 1015, 54, 1025, 1018, 839, 1022, 1034, 1021, 1027, - 1020, 1023, 1414, 612, 795, 1407, 1035, 1110, 1038, 1016, - 1017, 948, 1019, 1057, 1040, 1412, 1026, 864, 833, 834, - 958, 81, 81, 957, 557, 558, 559, 560, 561, 562, - 563, 564, 257, 258, 1065, 1066, 512, 611, 736, 1068, - 500, 611, 1053, 1054, 632, 1067, 468, 1069, 1070, 1071, - 81, 510, 501, 338, 1060, 1384, 1383, 1080, 1074, 1314, - 338, 1058, 1052, 1257, 1292, 209, 498, 470, 913, 702, - 837, 1093, 254, 255, 81, 512, 338, 338, 248, 956, - 1351, 338, 338, 338, 58, 338, 338, 955, 249, 1350, - 1298, 338, 338, 1037, 491, 470, 792, 794, 1429, 1428, - 1107, 720, 514, 532, 531, 541, 542, 534, 535, 536, - 537, 538, 539, 540, 533, 1118, 1117, 543, 1116, 739, - 1429, 81, 81, 1155, 1147, 1365, 1123, 1136, 1285, 521, - 1158, 726, 338, 1014, 60, 756, 62, 616, 55, 1135, - 1, 1421, 1141, 1218, 1293, 81, 289, 288, 291, 292, - 293, 294, 919, 1165, 1385, 290, 295, 1174, 81, 1328, - 81, 81, 1189, 870, 1159, 861, 51, 1181, 69, 438, - 68, 1376, 869, 787, 1180, 1160, 868, 1188, 1336, 1183, - 1283, 881, 1062, 802, 802, 1187, 884, 760, 209, 802, - 1193, 1194, 1196, 1079, 1059, 1192, 1381, 640, 638, 639, - 637, 642, 864, 641, 864, 209, 814, 815, 636, 222, - 331, 81, 836, 628, 81, 81, 209, 909, 1200, 1201, - 515, 1106, 81, 1211, 72, 209, 1095, 1094, 915, 338, - 485, 486, 224, 551, 954, 1212, 1041, 1214, 337, 1161, - 1223, 732, 338, 504, 1349, 1297, 994, 578, 808, 611, - 611, 611, 611, 611, 1224, 274, 747, 287, 284, 285, - 738, 1232, 1006, 525, 611, 272, 1116, 264, 610, 603, - 612, 828, 611, 757, 826, 1225, 766, 767, 768, 769, - 770, 771, 772, 773, 774, 775, 776, 777, 778, 779, - 780, 1258, 1014, 825, 1172, 1168, 1267, 338, 81, 338, - 1268, 609, 1251, 931, 932, 1113, 81, 1252, 1360, 742, - 26, 1277, 59, 338, 972, 1048, 259, 19, 973, 18, - 17, 81, 20, 16, 15, 977, 978, 979, 81, 14, - 456, 813, 30, 1273, 1274, 1275, 992, 21, 13, 338, - 12, 998, 864, 999, 11, 10, 1002, 1003, 1004, 1005, - 1286, 9, 1288, 8, 1229, 1230, 7, 1231, 6, 5, - 1233, 4, 1235, 250, 23, 2, 470, 0, 1029, 0, - 0, 0, 1295, 0, 81, 81, 1299, 81, 0, 0, - 0, 1158, 81, 0, 81, 81, 81, 209, 0, 1323, - 81, 1324, 1325, 1326, 1315, 262, 0, 1322, 0, 0, - 0, 0, 0, 0, 0, 1327, 1333, 81, 0, 0, - 1340, 0, 0, 0, 1348, 1159, 0, 0, 1318, 0, - 1280, 0, 0, 0, 0, 0, 0, 1317, 0, 0, - 0, 0, 802, 0, 0, 0, 0, 0, 1158, 1341, - 1366, 1342, 0, 0, 0, 0, 81, 0, 1345, 1374, - 1373, 0, 0, 0, 0, 0, 0, 81, 81, 0, - 0, 0, 0, 0, 0, 0, 1388, 0, 0, 0, - 1392, 338, 1159, 0, 51, 1389, 0, 0, 0, 81, - 0, 1397, 0, 1367, 0, 0, 503, 0, 0, 0, - 209, 1014, 1291, 0, 0, 0, 0, 0, 81, 0, - 0, 1295, 864, 0, 1122, 1406, 0, 0, 0, 0, - 0, 966, 967, 968, 0, 0, 0, 0, 1411, 1078, - 338, 81, 207, 1413, 0, 236, 0, 0, 1415, 0, - 0, 0, 0, 0, 1426, 0, 0, 0, 1256, 611, - 0, 1437, 0, 0, 0, 0, 0, 1250, 338, 0, - 263, 0, 0, 207, 0, 0, 0, 0, 207, 0, - 207, 1255, 541, 542, 534, 535, 536, 537, 538, 539, - 540, 533, 338, 219, 543, 1424, 532, 531, 541, 542, - 534, 535, 536, 537, 538, 539, 540, 533, 0, 0, - 543, 0, 0, 0, 0, 0, 338, 232, 0, 532, - 531, 541, 542, 534, 535, 536, 537, 538, 539, 540, - 533, 0, 0, 543, 0, 0, 802, 0, 0, 1162, - 1164, 532, 531, 541, 542, 534, 535, 536, 537, 538, - 539, 540, 533, 0, 0, 543, 0, 0, 0, 0, - 0, 0, 0, 1164, 502, 506, 0, 1226, 212, 0, - 0, 0, 0, 0, 1228, 215, 338, 0, 338, 1191, - 0, 524, 0, 223, 218, 1237, 1238, 1239, 0, 1242, - 0, 0, 1245, 0, 1248, 0, 0, 0, 0, 613, - 0, 0, 0, 207, 0, 0, 207, 0, 0, 1260, - 1261, 1262, 207, 1265, 0, 221, 568, 0, 207, 0, - 0, 231, 0, 0, 0, 579, 0, 0, 0, 1215, - 1276, 0, 1220, 1221, 0, 206, 0, 0, 1120, 1121, - 338, 0, 0, 0, 0, 0, 0, 213, 0, 0, - 0, 1137, 1138, 1139, 1140, 0, 1142, 1143, 1144, 1145, - 0, 0, 0, 0, 0, 0, 329, 0, 1151, 1152, - 0, 442, 0, 444, 225, 216, 0, 226, 227, 228, - 230, 0, 229, 235, 1119, 0, 0, 217, 220, 0, - 214, 234, 233, 0, 0, 802, 0, 0, 0, 0, - 0, 0, 1310, 0, 532, 531, 541, 542, 534, 535, - 536, 537, 538, 539, 540, 533, 338, 0, 543, 829, - 832, 833, 834, 830, 1282, 831, 835, 0, 1198, 1170, - 1171, 0, 207, 207, 207, 0, 0, 0, 0, 338, - 0, 0, 0, 0, 0, 0, 338, 0, 0, 0, - 0, 1352, 1353, 1354, 1355, 1356, 1357, 1358, 1359, 0, - 0, 0, 1363, 1364, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1370, 1371, 1372, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1227, 0, 0, - 0, 0, 1319, 1320, 0, 1321, 451, 0, 0, 457, - 1282, 0, 1282, 1282, 1282, 464, 0, 724, 1191, 0, - 1393, 466, 0, 0, 0, 0, 0, 1398, 0, 0, - 0, 0, 0, 0, 0, 1282, 0, 0, 0, 0, - 0, 0, 745, 746, 0, 1403, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 207, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 207, 207, 0, 0, - 0, 0, 207, 0, 1380, 207, 0, 0, 207, 0, - 0, 0, 708, 0, 0, 338, 338, 0, 0, 0, - 1438, 1439, 207, 0, 0, 568, 0, 0, 798, 799, - 0, 0, 0, 0, 802, 0, 0, 1399, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1301, 1302, - 0, 1303, 1304, 0, 1305, 1306, 1405, 1307, 1308, 1309, - 0, 207, 0, 1312, 1313, 605, 0, 614, 0, 0, - 708, 0, 0, 0, 0, 0, 527, 0, 530, 1282, - 0, 0, 0, 858, 544, 545, 546, 547, 548, 549, - 550, 0, 528, 529, 526, 532, 531, 541, 542, 534, - 535, 536, 537, 538, 539, 540, 533, 0, 0, 543, - 971, 0, 263, 0, 0, 0, 0, 263, 263, 0, - 0, 803, 803, 263, 0, 0, 0, 803, 1247, 0, - 532, 531, 541, 542, 534, 535, 536, 537, 538, 539, - 540, 533, 0, 0, 543, 657, 263, 263, 263, 263, - 0, 207, 0, 0, 0, 0, 0, 0, 0, 207, - 844, 0, 0, 207, 207, 0, 532, 531, 541, 542, - 534, 535, 536, 537, 538, 539, 540, 533, 0, 635, - 543, 0, 0, 0, 949, 950, 0, 506, 0, 691, - 692, 0, 0, 0, 0, 698, 0, 0, 329, 0, - 0, 704, 532, 531, 541, 542, 534, 535, 536, 537, - 538, 539, 540, 533, 0, 715, 543, 0, 0, 0, - 0, 0, 0, 645, 0, 0, 0, 0, 0, 0, - 0, 0, 1430, 0, 207, 0, 0, 0, 0, 0, - 0, 0, 0, 207, 207, 0, 207, 207, 0, 976, - 207, 0, 0, 0, 744, 0, 0, 0, 0, 0, - 658, 0, 0, 0, 0, 0, 997, 207, 0, 945, - 946, 0, 207, 0, 0, 0, 0, 708, 0, 0, - 0, 0, 671, 674, 675, 676, 677, 678, 679, 263, - 680, 681, 682, 683, 684, 659, 660, 661, 662, 643, - 644, 672, 0, 646, 0, 647, 648, 649, 650, 651, - 652, 653, 654, 655, 656, 663, 664, 665, 666, 667, - 668, 669, 670, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 263, 1244, 0, 0, - 0, 0, 0, 0, 822, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 263, 0, 0, 849, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 803, 207, 207, 207, 207, 207, 0, 673, 0, 0, - 0, 0, 0, 1028, 0, 0, 207, 0, 0, 0, - 844, 0, 1241, 0, 207, 0, 0, 0, 0, 0, - 1109, 0, 0, 0, 0, 24, 25, 52, 27, 28, - 0, 532, 531, 541, 542, 534, 535, 536, 537, 538, - 539, 540, 533, 0, 43, 543, 0, 911, 0, 29, - 48, 49, 0, 0, 0, 0, 933, 934, 0, 937, - 938, 0, 0, 939, 0, 0, 0, 0, 0, 38, - 0, 0, 1149, 54, 0, 0, 0, 0, 0, 0, - 941, 0, 0, 0, 0, 947, 532, 531, 541, 542, - 534, 535, 536, 537, 538, 539, 540, 533, 0, 0, - 543, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1182, 0, 0, 0, 0, 0, 0, - 0, 207, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 263, 0, 0, 31, 32, 34, 33, 36, 0, - 50, 0, 263, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 708, 0, 0, 0, 0, 0, - 0, 37, 44, 45, 0, 0, 46, 47, 35, 0, - 0, 0, 0, 0, 803, 0, 0, 0, 0, 0, - 0, 39, 40, 0, 41, 42, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1254, 0, 0, 0, - 0, 0, 0, 0, 0, 568, 0, 0, 0, 0, - 0, 0, 0, 1269, 207, 0, 1270, 0, 0, 1272, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 207, 0, 0, 0, 0, 0, 0, 0, 0, - 53, 0, 207, 0, 0, 0, 0, 0, 0, 0, - 0, 207, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1112, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 803, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1391, 568, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1210, 0, 0, - 0, 0, 0, 844, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1213, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1222, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 426, 415, 0, 386, - 429, 364, 378, 437, 379, 380, 408, 350, 394, 139, - 376, 0, 367, 345, 373, 346, 365, 388, 103, 391, - 363, 417, 397, 428, 120, 435, 122, 402, 0, 159, - 131, 0, 0, 390, 419, 392, 413, 385, 409, 355, - 401, 430, 377, 406, 431, 0, 0, 0, 80, 0, - 865, 866, 803, 0, 0, 0, 0, 96, 0, 404, - 425, 375, 405, 407, 344, 403, 207, 348, 351, 436, - 421, 370, 371, 1049, 0, 0, 0, 0, 0, 0, - 389, 393, 410, 383, 0, 0, 0, 0, 0, 0, - 0, 0, 368, 0, 400, 0, 0, 0, 352, 349, - 0, 0, 387, 0, 0, 0, 354, 0, 369, 411, - 0, 343, 109, 414, 420, 384, 210, 424, 382, 381, - 427, 146, 0, 162, 111, 119, 83, 90, 0, 110, - 137, 151, 155, 418, 366, 374, 99, 372, 153, 141, - 175, 399, 142, 152, 123, 167, 147, 174, 182, 183, - 164, 181, 190, 84, 163, 173, 97, 156, 86, 171, - 161, 129, 115, 116, 85, 0, 150, 102, 107, 101, - 138, 168, 169, 100, 193, 91, 180, 88, 92, 179, - 136, 166, 172, 130, 127, 87, 170, 128, 126, 118, + 271, 1435, 1425, 1219, 1389, 1104, 570, 275, 1015, 1284, + 1297, 866, 1332, 301, 1033, 1159, 288, 843, 1193, 249, + 57, 1156, 1016, 909, 865, 1166, 1160, 945, 1172, 895, + 569, 3, 81, 1058, 875, 341, 210, 841, 1127, 210, + 972, 794, 1039, 784, 791, 1075, 879, 627, 616, 845, + 724, 830, 812, 761, 610, 502, 264, 508, 905, 442, + 626, 335, 823, 862, 514, 954, 273, 210, 81, 611, + 522, 258, 210, 332, 210, 1084, 584, 56, 1428, 240, + 1412, 330, 248, 313, 207, 319, 320, 317, 318, 316, + 315, 314, 1423, 1399, 928, 1420, 1220, 585, 1411, 321, + 322, 1398, 1256, 1149, 447, 262, 475, 1187, 927, 1188, + 1189, 889, 628, 61, 629, 331, 205, 201, 202, 203, + 444, 856, 446, 496, 241, 242, 243, 244, 1046, 246, + 247, 1045, 857, 858, 1047, 245, 932, 1066, 888, 63, + 64, 65, 66, 67, 1287, 926, 1364, 535, 534, 544, + 545, 537, 538, 539, 540, 541, 542, 543, 536, 492, + 197, 546, 199, 896, 1303, 1239, 460, 493, 490, 491, + 1237, 239, 477, 1422, 479, 485, 486, 1107, 1128, 1106, + 699, 697, 495, 473, 471, 1419, 1382, 1390, 1103, 824, + 1333, 880, 461, 1443, 1091, 923, 920, 921, 210, 919, + 1439, 210, 449, 1335, 476, 478, 1108, 210, 199, 703, + 793, 1341, 690, 210, 698, 1130, 81, 1182, 81, 81, + 882, 81, 81, 1089, 81, 1181, 81, 1180, 1034, 1036, + 930, 933, 445, 700, 452, 81, 212, 1100, 204, 200, + 1371, 882, 1267, 1102, 939, 221, 453, 938, 1132, 459, + 1136, 338, 1131, 1114, 1129, 466, 198, 882, 990, 1134, + 966, 468, 558, 559, 733, 81, 925, 526, 1133, 234, + 443, 1334, 467, 510, 863, 539, 540, 541, 542, 543, + 536, 1135, 1137, 546, 546, 511, 1059, 1205, 924, 457, + 1090, 730, 521, 474, 896, 1095, 1092, 1085, 1093, 1088, + 736, 737, 1380, 1086, 1087, 1035, 1437, 505, 509, 1438, + 947, 1436, 1342, 1340, 1350, 1397, 881, 1094, 536, 1365, + 213, 546, 443, 725, 527, 498, 499, 216, 210, 210, + 210, 1170, 929, 630, 81, 225, 220, 881, 1206, 1101, + 81, 1099, 878, 876, 1151, 877, 768, 931, 520, 519, + 874, 880, 454, 881, 455, 441, 813, 456, 519, 571, + 766, 767, 765, 692, 1064, 521, 609, 223, 582, 512, + 463, 464, 465, 233, 521, 813, 608, 999, 617, 537, + 538, 539, 540, 541, 542, 543, 536, 70, 946, 546, + 1444, 587, 589, 591, 593, 595, 597, 598, 1385, 214, + 985, 516, 480, 481, 726, 482, 483, 624, 484, 22, + 487, 618, 588, 590, 448, 594, 596, 1403, 599, 497, + 1293, 520, 519, 71, 54, 1292, 227, 217, 218, 1445, + 228, 229, 230, 232, 764, 231, 237, 984, 521, 983, + 219, 222, 210, 215, 236, 235, 785, 81, 786, 885, + 520, 519, 210, 210, 81, 886, 520, 519, 210, 1079, + 1078, 210, 196, 1067, 210, 732, 1405, 521, 210, 253, + 81, 81, 1381, 521, 1378, 81, 81, 81, 210, 81, + 81, 520, 519, 751, 753, 754, 81, 81, 1153, 752, + 638, 450, 451, 963, 964, 965, 1314, 1048, 521, 1049, + 694, 695, 731, 712, 1290, 1111, 701, 1076, 1222, 331, + 1338, 1421, 707, 1059, 81, 1407, 501, 338, 210, 520, + 519, 1338, 1393, 501, 81, 1054, 718, 327, 328, 787, + 738, 1338, 501, 710, 1338, 1372, 521, 709, 277, 1338, + 1337, 1347, 704, 762, 1282, 1281, 1269, 501, 1266, 501, + 1346, 727, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 759, 708, 546, 693, 747, 691, 81, 291, + 290, 293, 294, 295, 296, 688, 748, 749, 292, 297, + 1212, 1211, 1208, 1209, 740, 803, 807, 1208, 1207, 978, + 501, 814, 757, 469, 755, 827, 501, 796, 501, 637, + 636, 81, 81, 798, 462, 1202, 1157, 883, 210, 1169, + 1169, 850, 58, 620, 826, 1040, 210, 210, 796, 1040, + 210, 210, 1262, 1349, 81, 788, 789, 827, 1210, 571, + 621, 689, 801, 802, 1117, 994, 24, 81, 696, 827, + 832, 835, 836, 837, 833, 810, 834, 838, 820, 992, + 1173, 1174, 978, 54, 713, 714, 825, 851, 827, 715, + 716, 717, 1169, 719, 720, 302, 51, 24, 1050, 852, + 721, 722, 622, 855, 620, 989, 987, 1004, 978, 993, + 849, 897, 898, 899, 54, 853, 854, 861, 24, 1003, + 978, 210, 81, 991, 81, 1319, 620, 870, 81, 81, + 210, 210, 623, 210, 210, 734, 702, 210, 81, 255, + 758, 911, 1010, 1413, 1299, 54, 1011, 51, 1430, 988, + 986, 890, 1274, 910, 210, 254, 210, 210, 1198, 210, + 1053, 832, 835, 836, 837, 833, 54, 834, 838, 914, + 1173, 1174, 1426, 907, 908, 906, 901, 900, 936, 937, + 1105, 940, 941, 913, 1417, 942, 1200, 54, 891, 892, + 893, 894, 1176, 1157, 1080, 728, 706, 746, 1027, 759, + 1179, 1178, 944, 1028, 902, 903, 904, 950, 1025, 762, + 1024, 1023, 1410, 1026, 799, 800, 259, 260, 952, 953, + 809, 509, 955, 1113, 956, 535, 534, 544, 545, 537, + 538, 539, 540, 541, 542, 543, 536, 338, 951, 546, + 515, 556, 1415, 819, 961, 821, 822, 960, 1071, 968, + 867, 1029, 635, 836, 837, 513, 503, 470, 210, 210, + 210, 210, 210, 1063, 1017, 916, 1387, 1386, 504, 1317, + 210, 1061, 1055, 210, 1260, 1295, 705, 210, 973, 840, + 515, 210, 959, 979, 1012, 256, 257, 250, 1354, 251, + 958, 300, 58, 1353, 998, 1301, 81, 614, 1040, 494, + 1000, 1432, 1431, 798, 723, 517, 915, 1051, 917, 1432, + 1368, 472, 1018, 472, 472, 1021, 472, 472, 1041, 472, + 1030, 472, 943, 79, 1042, 1288, 1038, 729, 1019, 1020, + 472, 1022, 60, 1043, 62, 619, 55, 1, 1424, 1221, + 1296, 922, 1388, 1060, 81, 81, 1331, 758, 1192, 873, + 51, 1068, 1069, 864, 1056, 1057, 69, 440, 68, 340, + 815, 1379, 872, 871, 1339, 555, 1286, 884, 557, 1065, + 887, 1199, 1062, 81, 1384, 643, 641, 1077, 642, 640, + 645, 644, 639, 224, 333, 839, 962, 631, 210, 1070, + 912, 1072, 1073, 1074, 518, 72, 568, 81, 572, 573, + 574, 575, 576, 577, 578, 579, 580, 1098, 583, 586, + 586, 586, 592, 586, 586, 592, 586, 600, 601, 602, + 603, 604, 605, 1097, 615, 1110, 1096, 918, 1083, 488, + 489, 226, 554, 977, 1112, 957, 1115, 1044, 339, 1164, + 1121, 735, 1120, 507, 81, 81, 1352, 1150, 1017, 1158, + 1126, 996, 1300, 997, 1139, 1161, 759, 581, 1138, 811, + 276, 750, 289, 286, 287, 741, 1009, 528, 81, 274, + 266, 613, 606, 831, 1163, 829, 828, 1175, 1171, 867, + 1177, 81, 612, 81, 81, 1116, 1152, 1255, 1168, 1363, + 1184, 745, 26, 1183, 1191, 59, 261, 19, 763, 18, + 17, 20, 16, 15, 14, 1190, 458, 340, 30, 340, + 340, 210, 340, 340, 1186, 340, 21, 340, 13, 1195, + 1196, 1197, 12, 11, 10, 9, 340, 1185, 210, 1082, + 8, 7, 6, 5, 81, 4, 252, 81, 81, 210, + 23, 2, 472, 0, 0, 81, 0, 0, 210, 472, + 0, 0, 0, 0, 0, 0, 524, 1109, 0, 1213, + 0, 0, 1203, 1204, 0, 472, 472, 0, 1226, 0, + 472, 472, 472, 0, 472, 472, 1216, 0, 0, 0, + 1119, 472, 472, 0, 614, 1228, 1235, 1225, 614, 1227, + 0, 0, 0, 0, 1214, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1144, 0, 1215, 1017, 1217, 500, + 1261, 0, 0, 0, 0, 0, 0, 0, 0, 1270, + 0, 81, 1271, 0, 0, 340, 0, 0, 0, 81, + 0, 632, 1051, 0, 1280, 0, 0, 0, 0, 0, + 1257, 0, 1252, 501, 81, 0, 0, 0, 0, 571, + 0, 81, 0, 51, 0, 0, 0, 1272, 0, 0, + 1273, 0, 0, 1275, 867, 0, 867, 1289, 572, 1291, + 1232, 1233, 1253, 1234, 0, 0, 1236, 0, 1238, 0, + 535, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 1302, 546, 0, 0, 81, 81, 0, + 81, 0, 0, 0, 0, 81, 1161, 81, 81, 81, + 210, 842, 0, 81, 1326, 615, 1327, 1328, 1329, 1318, + 1325, 0, 0, 0, 0, 0, 1320, 1330, 1119, 0, + 81, 1336, 0, 0, 763, 1343, 1283, 0, 340, 1351, + 0, 0, 0, 0, 0, 340, 535, 534, 544, 545, + 537, 538, 539, 540, 541, 542, 543, 536, 0, 0, + 546, 340, 340, 1161, 0, 1369, 340, 340, 340, 81, + 340, 340, 0, 0, 1377, 1376, 0, 340, 340, 268, + 81, 81, 1370, 1344, 0, 1345, 0, 472, 0, 472, + 1391, 1395, 1392, 0, 0, 0, 614, 614, 614, 614, + 614, 0, 81, 472, 867, 742, 1017, 1400, 0, 0, + 0, 614, 0, 210, 0, 524, 0, 0, 340, 614, + 0, 81, 0, 0, 0, 0, 0, 0, 1294, 1409, + 0, 0, 0, 0, 1298, 0, 0, 0, 1249, 501, + 1394, 571, 1414, 1416, 81, 0, 0, 0, 0, 0, + 0, 0, 0, 1418, 967, 0, 0, 1429, 0, 790, + 0, 1404, 1246, 501, 1440, 0, 0, 0, 0, 805, + 805, 0, 739, 0, 0, 805, 535, 534, 544, 545, + 537, 538, 539, 540, 541, 542, 543, 536, 0, 0, + 546, 0, 817, 818, 0, 0, 0, 0, 0, 0, + 535, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 1259, 546, 340, 0, 0, 0, 1013, + 1014, 0, 0, 615, 615, 615, 615, 615, 340, 660, + 795, 797, 1243, 501, 0, 0, 0, 0, 842, 0, + 1037, 0, 0, 0, 0, 0, 615, 0, 0, 0, + 0, 535, 534, 544, 545, 537, 538, 539, 540, 541, + 542, 543, 536, 1298, 867, 546, 0, 0, 0, 0, + 535, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 340, 546, 340, 0, 0, 1258, 934, + 935, 0, 0, 0, 0, 0, 0, 0, 0, 340, + 0, 0, 0, 0, 0, 0, 0, 648, 0, 0, + 472, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 0, 546, 340, 535, 534, 544, 545, + 537, 538, 539, 540, 541, 542, 543, 536, 472, 0, + 546, 0, 0, 0, 661, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 560, 561, + 562, 563, 564, 565, 566, 567, 674, 677, 678, 679, + 680, 681, 682, 0, 683, 684, 685, 686, 687, 662, + 663, 664, 665, 646, 647, 675, 614, 649, 0, 650, + 651, 652, 653, 654, 655, 656, 657, 658, 659, 666, + 667, 668, 669, 670, 671, 672, 673, 1162, 0, 51, + 24, 25, 52, 27, 28, 0, 0, 0, 805, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 43, + 0, 0, 0, 0, 29, 48, 49, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 501, 975, 0, + 0, 0, 976, 0, 38, 0, 0, 340, 54, 980, + 981, 982, 676, 0, 0, 0, 0, 0, 0, 0, + 995, 0, 0, 0, 0, 1001, 0, 1002, 0, 0, + 1005, 1006, 1007, 1008, 535, 534, 544, 545, 537, 538, + 539, 540, 541, 542, 543, 536, 0, 0, 546, 0, + 0, 0, 1032, 0, 0, 1081, 340, 0, 0, 1250, + 0, 0, 0, 615, 0, 0, 0, 0, 0, 31, + 32, 34, 33, 36, 0, 50, 0, 0, 0, 0, + 0, 0, 0, 0, 340, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1254, 37, 44, 45, 0, + 0, 46, 47, 35, 0, 0, 0, 0, 340, 0, + 0, 0, 0, 0, 0, 0, 39, 40, 0, 41, + 42, 0, 0, 0, 0, 0, 1276, 1277, 1278, 0, + 0, 0, 340, 535, 534, 544, 545, 537, 538, 539, + 540, 541, 542, 543, 536, 0, 0, 546, 0, 0, + 0, 1247, 805, 0, 0, 1165, 1167, 0, 760, 472, + 0, 769, 770, 771, 772, 773, 774, 775, 776, 777, + 778, 779, 780, 781, 782, 783, 0, 0, 0, 1167, + 0, 0, 0, 0, 0, 0, 0, 0, 1125, 0, + 0, 0, 340, 0, 340, 1194, 1244, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 53, 0, 1162, 0, + 0, 1321, 0, 0, 0, 0, 816, 0, 0, 0, + 0, 0, 0, 0, 506, 535, 534, 544, 545, 537, + 538, 539, 540, 541, 542, 543, 536, 0, 0, 546, + 0, 1348, 0, 0, 0, 1218, 0, 0, 1223, 1224, + 0, 0, 0, 0, 0, 0, 340, 0, 0, 0, + 208, 0, 0, 238, 0, 1162, 0, 51, 0, 0, + 535, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 0, 546, 0, 0, 0, 265, 0, + 0, 208, 1122, 0, 0, 0, 208, 0, 208, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 805, 535, 534, 544, 545, 537, 538, 539, 540, + 541, 542, 543, 536, 0, 0, 546, 0, 0, 0, + 0, 1229, 340, 0, 0, 0, 0, 0, 1231, 0, + 1285, 0, 0, 0, 0, 0, 0, 0, 0, 1240, + 1241, 1242, 0, 1245, 0, 340, 1248, 0, 1251, 974, + 0, 0, 340, 0, 0, 0, 0, 0, 1427, 0, + 0, 0, 0, 1263, 1264, 1265, 0, 1268, 0, 535, + 534, 544, 545, 537, 538, 539, 540, 541, 542, 543, + 536, 0, 0, 546, 1279, 0, 969, 970, 971, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1322, 1323, + 0, 1324, 0, 0, 0, 0, 1285, 0, 1285, 1285, + 1285, 0, 208, 0, 1194, 208, 0, 0, 0, 0, + 0, 208, 0, 0, 530, 0, 533, 208, 0, 0, + 0, 1285, 547, 548, 549, 550, 551, 552, 553, 0, + 531, 532, 529, 535, 534, 544, 545, 537, 538, 539, + 540, 541, 542, 543, 536, 0, 1313, 546, 535, 534, + 544, 545, 537, 538, 539, 540, 541, 542, 543, 536, + 1383, 0, 546, 0, 0, 0, 0, 0, 0, 0, + 0, 340, 340, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 805, 0, 0, 1402, 0, 1355, 1356, 1357, 1358, 1359, + 1360, 1361, 1362, 0, 0, 0, 1366, 1367, 0, 0, + 0, 0, 1408, 0, 0, 0, 0, 0, 1373, 1374, + 1375, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 208, 208, 208, 1285, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1396, 0, 0, 0, 0, 0, + 0, 1401, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1406, + 0, 0, 0, 1123, 1124, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1140, 1141, 1142, 1143, + 0, 1145, 1146, 1147, 1148, 0, 0, 0, 0, 0, + 0, 0, 0, 1154, 1155, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1441, 1442, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 208, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 208, 208, 0, 0, + 0, 0, 208, 0, 0, 208, 0, 0, 208, 0, + 0, 0, 711, 1201, 0, 0, 0, 0, 0, 0, + 0, 0, 208, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 208, 0, 0, 0, 0, 0, 0, 0, + 0, 711, 1230, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 265, 0, 0, 0, 0, 265, 265, + 0, 0, 806, 806, 265, 0, 0, 0, 806, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 265, 265, 265, + 265, 0, 208, 0, 0, 0, 0, 0, 0, 0, + 208, 847, 0, 0, 208, 208, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1304, 1305, 0, 1306, 1307, 0, 1308, + 1309, 0, 1310, 1311, 1312, 0, 0, 0, 1315, 1316, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 208, 0, 0, 0, 0, + 0, 0, 0, 0, 208, 208, 0, 208, 208, 0, + 0, 208, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 208, 0, + 948, 949, 0, 208, 0, 0, 0, 0, 711, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 265, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 265, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 265, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 806, 208, 208, 208, 208, 208, 1433, 0, 0, + 0, 0, 0, 0, 1031, 0, 0, 208, 0, 0, + 0, 847, 0, 0, 0, 208, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 208, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 265, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 265, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 711, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 806, 0, 0, 139, 0, + 0, 0, 0, 272, 0, 0, 0, 103, 0, 269, + 0, 0, 0, 120, 312, 122, 0, 0, 160, 131, + 0, 0, 0, 0, 303, 304, 0, 0, 0, 0, + 0, 0, 859, 0, 54, 0, 0, 270, 291, 290, + 293, 294, 295, 296, 0, 0, 96, 292, 297, 298, + 299, 860, 0, 0, 267, 284, 0, 311, 0, 0, + 0, 0, 0, 0, 0, 208, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 281, 282, 0, + 0, 0, 208, 325, 0, 283, 0, 0, 278, 279, + 280, 285, 0, 208, 0, 0, 0, 0, 0, 0, + 0, 109, 208, 0, 0, 211, 0, 0, 323, 0, + 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, + 151, 155, 0, 0, 0, 99, 0, 153, 141, 176, + 0, 142, 152, 123, 168, 147, 175, 183, 184, 165, + 182, 191, 84, 164, 174, 97, 156, 86, 172, 162, + 129, 115, 116, 85, 806, 150, 102, 107, 101, 138, + 169, 170, 100, 194, 91, 181, 88, 92, 180, 136, + 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, + 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, + 0, 161, 178, 195, 94, 0, 157, 166, 185, 186, + 187, 188, 189, 190, 0, 0, 95, 108, 104, 143, + 135, 93, 114, 158, 117, 124, 149, 193, 140, 154, + 98, 177, 159, 313, 324, 319, 320, 317, 318, 316, + 315, 314, 326, 305, 306, 307, 308, 310, 0, 321, + 322, 309, 82, 89, 121, 192, 148, 106, 179, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 428, 417, 847, 388, 431, 366, 380, 439, + 381, 382, 410, 352, 396, 139, 378, 0, 369, 347, + 375, 348, 367, 390, 103, 393, 365, 419, 399, 430, + 120, 437, 122, 404, 0, 160, 131, 0, 0, 392, + 421, 394, 415, 387, 411, 357, 403, 432, 379, 408, + 433, 0, 0, 0, 80, 0, 868, 869, 0, 0, + 0, 0, 0, 96, 0, 406, 427, 377, 407, 409, + 346, 405, 0, 350, 353, 438, 423, 372, 373, 1052, + 0, 0, 0, 0, 0, 0, 391, 395, 412, 385, + 0, 0, 0, 806, 0, 0, 0, 0, 370, 0, + 402, 0, 0, 0, 354, 351, 0, 208, 389, 0, + 0, 0, 356, 0, 371, 413, 0, 345, 109, 416, + 422, 386, 211, 426, 384, 383, 429, 146, 0, 163, + 111, 119, 83, 90, 0, 110, 137, 151, 155, 420, + 368, 376, 99, 374, 153, 141, 176, 401, 142, 152, + 123, 168, 147, 175, 183, 184, 165, 182, 191, 84, + 164, 174, 97, 156, 86, 172, 162, 129, 115, 116, + 85, 0, 150, 102, 107, 101, 138, 169, 170, 100, + 194, 91, 181, 88, 92, 180, 136, 167, 173, 130, + 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, + 145, 113, 133, 132, 134, 0, 349, 0, 161, 178, + 195, 94, 364, 157, 166, 185, 186, 187, 188, 189, + 190, 0, 0, 95, 108, 104, 143, 135, 93, 114, + 158, 117, 124, 149, 193, 140, 154, 98, 177, 159, + 360, 363, 358, 359, 397, 398, 434, 435, 436, 414, + 355, 0, 361, 362, 0, 418, 424, 425, 400, 82, + 89, 121, 192, 148, 106, 179, 428, 417, 0, 388, + 431, 366, 380, 439, 381, 382, 410, 352, 396, 139, + 378, 0, 369, 347, 375, 348, 367, 390, 103, 393, + 365, 419, 399, 430, 120, 437, 122, 404, 0, 160, + 131, 0, 0, 392, 421, 394, 415, 387, 411, 357, + 403, 432, 379, 408, 433, 0, 0, 0, 80, 0, + 868, 869, 0, 0, 0, 0, 0, 96, 0, 406, + 427, 377, 407, 409, 346, 405, 0, 350, 353, 438, + 423, 372, 373, 0, 0, 0, 0, 0, 0, 0, + 391, 395, 412, 385, 0, 0, 0, 0, 0, 0, + 0, 0, 370, 0, 402, 0, 0, 0, 354, 351, + 0, 0, 389, 0, 0, 0, 356, 0, 371, 413, + 0, 345, 109, 416, 422, 386, 211, 426, 384, 383, + 429, 146, 0, 163, 111, 119, 83, 90, 0, 110, + 137, 151, 155, 420, 368, 376, 99, 374, 153, 141, + 176, 401, 142, 152, 123, 168, 147, 175, 183, 184, + 165, 182, 191, 84, 164, 174, 97, 156, 86, 172, + 162, 129, 115, 116, 85, 0, 150, 102, 107, 101, + 138, 169, 170, 100, 194, 91, 181, 88, 92, 180, + 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, 0, - 347, 0, 160, 177, 194, 94, 362, 165, 184, 185, - 186, 187, 188, 189, 0, 0, 95, 108, 104, 143, - 135, 93, 114, 157, 117, 124, 149, 192, 140, 154, - 98, 176, 158, 358, 361, 356, 357, 395, 396, 432, - 433, 434, 412, 353, 0, 359, 360, 0, 416, 422, - 423, 398, 82, 89, 121, 191, 148, 106, 178, 1401, - 426, 415, 0, 386, 429, 364, 378, 437, 379, 380, - 408, 350, 394, 139, 376, 0, 367, 345, 373, 346, - 365, 388, 103, 391, 363, 417, 397, 428, 120, 435, - 122, 402, 0, 159, 131, 0, 0, 390, 419, 392, - 413, 385, 409, 355, 401, 430, 377, 406, 431, 0, - 0, 0, 80, 0, 865, 866, 0, 0, 0, 0, - 0, 96, 0, 404, 425, 375, 405, 407, 344, 403, - 0, 348, 351, 436, 421, 370, 371, 0, 0, 0, - 0, 0, 0, 0, 389, 393, 410, 383, 0, 0, - 0, 0, 0, 0, 0, 0, 368, 0, 400, 0, - 0, 0, 352, 349, 0, 0, 387, 0, 0, 0, - 354, 0, 369, 411, 0, 343, 109, 414, 420, 384, - 210, 424, 382, 381, 427, 146, 0, 162, 111, 119, - 83, 90, 0, 110, 137, 151, 155, 418, 366, 374, - 99, 372, 153, 141, 175, 399, 142, 152, 123, 167, - 147, 174, 182, 183, 164, 181, 190, 84, 163, 173, - 97, 156, 86, 171, 161, 129, 115, 116, 85, 0, - 150, 102, 107, 101, 138, 168, 169, 100, 193, 91, - 180, 88, 92, 179, 136, 166, 172, 130, 127, 87, - 170, 128, 126, 118, 105, 112, 144, 125, 145, 113, - 133, 132, 134, 0, 347, 0, 160, 177, 194, 94, - 362, 165, 184, 185, 186, 187, 188, 189, 0, 0, - 95, 108, 104, 143, 135, 93, 114, 157, 117, 124, - 149, 192, 140, 154, 98, 176, 158, 358, 361, 356, - 357, 395, 396, 432, 433, 434, 412, 353, 0, 359, - 360, 0, 416, 422, 423, 398, 82, 89, 121, 191, - 148, 106, 178, 426, 415, 0, 386, 429, 364, 378, - 437, 379, 380, 408, 350, 394, 139, 376, 0, 367, - 345, 373, 346, 365, 388, 103, 391, 363, 417, 397, - 428, 120, 435, 122, 402, 0, 159, 131, 0, 0, - 390, 419, 392, 413, 385, 409, 355, 401, 430, 377, - 406, 431, 54, 0, 0, 80, 0, 0, 0, 0, - 0, 0, 0, 0, 96, 0, 404, 425, 375, 405, - 407, 344, 403, 0, 348, 351, 436, 421, 370, 371, - 0, 0, 0, 0, 0, 0, 0, 389, 393, 410, - 383, 0, 0, 0, 0, 0, 0, 0, 0, 368, - 0, 400, 0, 0, 0, 352, 349, 0, 0, 387, - 0, 0, 0, 354, 0, 369, 411, 0, 343, 109, - 414, 420, 384, 210, 424, 382, 381, 427, 146, 0, - 162, 111, 119, 83, 90, 0, 110, 137, 151, 155, - 418, 366, 374, 99, 372, 153, 141, 175, 399, 142, - 152, 123, 167, 147, 174, 182, 183, 164, 181, 190, - 84, 163, 173, 97, 156, 86, 171, 161, 129, 115, - 116, 85, 0, 150, 102, 107, 101, 138, 168, 169, - 100, 193, 91, 180, 88, 92, 179, 136, 166, 172, - 130, 127, 87, 170, 128, 126, 118, 105, 112, 144, - 125, 145, 113, 133, 132, 134, 0, 347, 0, 160, - 177, 194, 94, 362, 165, 184, 185, 186, 187, 188, - 189, 0, 0, 95, 108, 104, 143, 135, 93, 114, - 157, 117, 124, 149, 192, 140, 154, 98, 176, 158, - 358, 361, 356, 357, 395, 396, 432, 433, 434, 412, - 353, 0, 359, 360, 0, 416, 422, 423, 398, 82, - 89, 121, 191, 148, 106, 178, 426, 415, 0, 386, - 429, 364, 378, 437, 379, 380, 408, 350, 394, 139, - 376, 0, 367, 345, 373, 346, 365, 388, 103, 391, - 363, 417, 397, 428, 120, 435, 122, 402, 0, 159, - 131, 0, 0, 390, 419, 392, 413, 385, 409, 355, - 401, 430, 377, 406, 431, 0, 0, 0, 80, 0, - 0, 0, 0, 0, 0, 0, 0, 96, 0, 404, - 425, 375, 405, 407, 344, 403, 0, 348, 351, 436, - 421, 370, 371, 0, 0, 0, 0, 0, 0, 0, - 389, 393, 410, 383, 0, 0, 0, 0, 0, 0, - 1115, 0, 368, 0, 400, 0, 0, 0, 352, 349, - 0, 0, 387, 0, 0, 0, 354, 0, 369, 411, - 0, 343, 109, 414, 420, 384, 210, 424, 382, 381, - 427, 146, 0, 162, 111, 119, 83, 90, 0, 110, - 137, 151, 155, 418, 366, 374, 99, 372, 153, 141, - 175, 399, 142, 152, 123, 167, 147, 174, 182, 183, - 164, 181, 190, 84, 163, 173, 97, 156, 86, 171, - 161, 129, 115, 116, 85, 0, 150, 102, 107, 101, - 138, 168, 169, 100, 193, 91, 180, 88, 92, 179, - 136, 166, 172, 130, 127, 87, 170, 128, 126, 118, + 349, 0, 161, 178, 195, 94, 364, 157, 166, 185, + 186, 187, 188, 189, 190, 0, 0, 95, 108, 104, + 143, 135, 93, 114, 158, 117, 124, 149, 193, 140, + 154, 98, 177, 159, 360, 363, 358, 359, 397, 398, + 434, 435, 436, 414, 355, 0, 361, 362, 0, 418, + 424, 425, 400, 82, 89, 121, 192, 148, 106, 179, + 428, 417, 0, 388, 431, 366, 380, 439, 381, 382, + 410, 352, 396, 139, 378, 0, 369, 347, 375, 348, + 367, 390, 103, 393, 365, 419, 399, 430, 120, 437, + 122, 404, 0, 160, 131, 0, 0, 392, 421, 394, + 415, 387, 411, 357, 403, 432, 379, 408, 433, 54, + 0, 0, 80, 0, 0, 0, 0, 0, 0, 0, + 0, 96, 0, 406, 427, 377, 407, 409, 346, 405, + 0, 350, 353, 438, 423, 372, 373, 0, 0, 0, + 0, 0, 0, 0, 391, 395, 412, 385, 0, 0, + 0, 0, 0, 0, 0, 0, 370, 0, 402, 0, + 0, 0, 354, 351, 0, 0, 389, 0, 0, 0, + 356, 0, 371, 413, 0, 345, 109, 416, 422, 386, + 211, 426, 384, 383, 429, 146, 0, 163, 111, 119, + 83, 90, 0, 110, 137, 151, 155, 420, 368, 376, + 99, 374, 153, 141, 176, 401, 142, 152, 123, 168, + 147, 175, 183, 184, 165, 182, 191, 84, 164, 174, + 97, 156, 86, 172, 162, 129, 115, 116, 85, 0, + 150, 102, 107, 101, 138, 169, 170, 100, 194, 91, + 181, 88, 92, 180, 136, 167, 173, 130, 127, 87, + 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, + 133, 132, 134, 0, 349, 0, 161, 178, 195, 94, + 364, 157, 166, 185, 186, 187, 188, 189, 190, 0, + 0, 95, 108, 104, 143, 135, 93, 114, 158, 117, + 124, 149, 193, 140, 154, 98, 177, 159, 360, 363, + 358, 359, 397, 398, 434, 435, 436, 414, 355, 0, + 361, 362, 0, 418, 424, 425, 400, 82, 89, 121, + 192, 148, 106, 179, 428, 417, 0, 388, 431, 366, + 380, 439, 381, 382, 410, 352, 396, 139, 378, 0, + 369, 347, 375, 348, 367, 390, 103, 393, 365, 419, + 399, 430, 120, 437, 122, 404, 0, 160, 131, 0, + 0, 392, 421, 394, 415, 387, 411, 357, 403, 432, + 379, 408, 433, 0, 0, 0, 80, 0, 0, 0, + 0, 0, 0, 0, 0, 96, 0, 406, 427, 377, + 407, 409, 346, 405, 0, 350, 353, 438, 423, 372, + 373, 0, 0, 0, 0, 0, 0, 0, 391, 395, + 412, 385, 0, 0, 0, 0, 0, 0, 1118, 0, + 370, 0, 402, 0, 0, 0, 354, 351, 0, 0, + 389, 0, 0, 0, 356, 0, 371, 413, 0, 345, + 109, 416, 422, 386, 211, 426, 384, 383, 429, 146, + 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, + 155, 420, 368, 376, 99, 374, 153, 141, 176, 401, + 142, 152, 123, 168, 147, 175, 183, 184, 165, 182, + 191, 84, 164, 174, 97, 156, 86, 172, 162, 129, + 115, 116, 85, 0, 150, 102, 107, 101, 138, 169, + 170, 100, 194, 91, 181, 88, 92, 180, 136, 167, + 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, + 144, 125, 145, 113, 133, 132, 134, 0, 349, 0, + 161, 178, 195, 94, 364, 157, 166, 185, 186, 187, + 188, 189, 190, 0, 0, 95, 108, 104, 143, 135, + 93, 114, 158, 117, 124, 149, 193, 140, 154, 98, + 177, 159, 360, 363, 358, 359, 397, 398, 434, 435, + 436, 414, 355, 0, 361, 362, 0, 418, 424, 425, + 400, 82, 89, 121, 192, 148, 106, 179, 428, 417, + 0, 388, 431, 366, 380, 439, 381, 382, 410, 352, + 396, 139, 378, 0, 369, 347, 375, 348, 367, 390, + 103, 393, 365, 419, 399, 430, 120, 437, 122, 404, + 0, 160, 131, 0, 0, 392, 421, 394, 415, 387, + 411, 357, 403, 432, 379, 408, 433, 0, 0, 0, + 270, 0, 0, 0, 0, 0, 0, 0, 0, 96, + 0, 406, 427, 377, 407, 409, 346, 405, 0, 350, + 353, 438, 423, 372, 373, 0, 0, 0, 0, 0, + 0, 0, 391, 395, 412, 385, 0, 0, 0, 0, + 0, 0, 756, 0, 370, 0, 402, 0, 0, 0, + 354, 351, 0, 0, 389, 0, 0, 0, 356, 0, + 371, 413, 0, 345, 109, 416, 422, 386, 211, 426, + 384, 383, 429, 146, 0, 163, 111, 119, 83, 90, + 0, 110, 137, 151, 155, 420, 368, 376, 99, 374, + 153, 141, 176, 401, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 174, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 92, 180, 136, 167, 173, 130, 127, 87, 171, 128, + 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, + 134, 0, 349, 0, 161, 178, 195, 94, 364, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 135, 93, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 360, 363, 358, 359, + 397, 398, 434, 435, 436, 414, 355, 0, 361, 362, + 0, 418, 424, 425, 400, 82, 89, 121, 192, 148, + 106, 179, 428, 417, 0, 388, 431, 366, 380, 439, + 381, 382, 410, 352, 396, 139, 378, 0, 369, 347, + 375, 348, 367, 390, 103, 393, 365, 419, 399, 430, + 120, 437, 122, 404, 0, 160, 131, 0, 0, 392, + 421, 394, 415, 387, 411, 357, 403, 432, 379, 408, + 433, 0, 0, 0, 80, 0, 0, 0, 0, 0, + 0, 0, 0, 96, 0, 406, 427, 377, 407, 409, + 346, 405, 0, 350, 353, 438, 423, 372, 373, 0, + 0, 0, 0, 0, 0, 0, 391, 395, 412, 385, + 0, 0, 0, 0, 0, 0, 0, 0, 370, 0, + 402, 0, 0, 0, 354, 351, 0, 0, 389, 0, + 0, 0, 356, 0, 371, 413, 0, 345, 109, 416, + 422, 386, 211, 426, 384, 383, 429, 146, 0, 163, + 111, 119, 83, 90, 0, 110, 137, 151, 155, 420, + 368, 376, 99, 374, 153, 141, 176, 401, 142, 152, + 123, 168, 147, 175, 183, 184, 165, 182, 191, 84, + 164, 174, 97, 156, 86, 172, 162, 129, 115, 116, + 85, 0, 150, 102, 107, 101, 138, 169, 170, 100, + 194, 91, 181, 88, 92, 180, 136, 167, 173, 130, + 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, + 145, 113, 133, 132, 134, 0, 349, 0, 161, 178, + 195, 94, 364, 157, 166, 185, 186, 187, 188, 189, + 190, 0, 0, 95, 108, 104, 143, 135, 93, 114, + 158, 117, 124, 149, 193, 140, 154, 98, 177, 159, + 360, 363, 358, 359, 397, 398, 434, 435, 436, 414, + 355, 0, 361, 362, 0, 418, 424, 425, 400, 82, + 89, 121, 192, 148, 106, 179, 428, 417, 0, 388, + 431, 366, 380, 439, 381, 382, 410, 352, 396, 139, + 378, 0, 369, 347, 375, 348, 367, 390, 103, 393, + 365, 419, 399, 430, 120, 437, 122, 404, 0, 160, + 131, 0, 0, 392, 421, 394, 415, 387, 411, 357, + 403, 432, 379, 408, 433, 0, 0, 0, 270, 0, + 0, 0, 0, 0, 0, 0, 0, 96, 0, 406, + 427, 377, 407, 409, 346, 405, 0, 350, 353, 438, + 423, 372, 373, 0, 0, 0, 0, 0, 0, 0, + 391, 395, 412, 385, 0, 0, 0, 0, 0, 0, + 0, 0, 370, 0, 402, 0, 0, 0, 354, 351, + 0, 0, 389, 0, 0, 0, 356, 0, 371, 413, + 0, 345, 109, 416, 422, 386, 211, 426, 384, 383, + 429, 146, 0, 163, 111, 119, 83, 90, 0, 110, + 137, 151, 155, 420, 368, 376, 99, 374, 153, 141, + 176, 401, 142, 152, 123, 168, 147, 175, 183, 184, + 165, 182, 191, 84, 164, 174, 97, 156, 86, 172, + 162, 129, 115, 116, 85, 0, 150, 102, 107, 101, + 138, 169, 170, 100, 194, 91, 181, 88, 92, 180, + 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, 0, - 347, 0, 160, 177, 194, 94, 362, 165, 184, 185, - 186, 187, 188, 189, 0, 0, 95, 108, 104, 143, - 135, 93, 114, 157, 117, 124, 149, 192, 140, 154, - 98, 176, 158, 358, 361, 356, 357, 395, 396, 432, - 433, 434, 412, 353, 0, 359, 360, 0, 416, 422, - 423, 398, 82, 89, 121, 191, 148, 106, 178, 426, - 415, 0, 386, 429, 364, 378, 437, 379, 380, 408, - 350, 394, 139, 376, 0, 367, 345, 373, 346, 365, - 388, 103, 391, 363, 417, 397, 428, 120, 435, 122, - 402, 0, 159, 131, 0, 0, 390, 419, 392, 413, - 385, 409, 355, 401, 430, 377, 406, 431, 0, 0, - 0, 268, 0, 0, 0, 0, 0, 0, 0, 0, - 96, 0, 404, 425, 375, 405, 407, 344, 403, 0, - 348, 351, 436, 421, 370, 371, 0, 0, 0, 0, - 0, 0, 0, 389, 393, 410, 383, 0, 0, 0, - 0, 0, 0, 753, 0, 368, 0, 400, 0, 0, - 0, 352, 349, 0, 0, 387, 0, 0, 0, 354, - 0, 369, 411, 0, 343, 109, 414, 420, 384, 210, - 424, 382, 381, 427, 146, 0, 162, 111, 119, 83, - 90, 0, 110, 137, 151, 155, 418, 366, 374, 99, - 372, 153, 141, 175, 399, 142, 152, 123, 167, 147, - 174, 182, 183, 164, 181, 190, 84, 163, 173, 97, - 156, 86, 171, 161, 129, 115, 116, 85, 0, 150, - 102, 107, 101, 138, 168, 169, 100, 193, 91, 180, - 88, 92, 179, 136, 166, 172, 130, 127, 87, 170, - 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, - 132, 134, 0, 347, 0, 160, 177, 194, 94, 362, - 165, 184, 185, 186, 187, 188, 189, 0, 0, 95, - 108, 104, 143, 135, 93, 114, 157, 117, 124, 149, - 192, 140, 154, 98, 176, 158, 358, 361, 356, 357, - 395, 396, 432, 433, 434, 412, 353, 0, 359, 360, - 0, 416, 422, 423, 398, 82, 89, 121, 191, 148, - 106, 178, 426, 415, 0, 386, 429, 364, 378, 437, - 379, 380, 408, 350, 394, 139, 376, 0, 367, 345, - 373, 346, 365, 388, 103, 391, 363, 417, 397, 428, - 120, 435, 122, 402, 0, 159, 131, 0, 0, 390, - 419, 392, 413, 385, 409, 355, 401, 430, 377, 406, - 431, 0, 0, 0, 80, 0, 0, 0, 0, 0, - 0, 0, 0, 96, 0, 404, 425, 375, 405, 407, - 344, 403, 0, 348, 351, 436, 421, 370, 371, 0, - 0, 0, 0, 0, 0, 0, 389, 393, 410, 383, - 0, 0, 0, 0, 0, 0, 0, 0, 368, 0, - 400, 0, 0, 0, 352, 349, 0, 0, 387, 0, - 0, 0, 354, 0, 369, 411, 0, 343, 109, 414, - 420, 384, 210, 424, 382, 381, 427, 146, 0, 162, - 111, 119, 83, 90, 0, 110, 137, 151, 155, 418, - 366, 374, 99, 372, 153, 141, 175, 399, 142, 152, - 123, 167, 147, 174, 182, 183, 164, 181, 190, 84, - 163, 173, 97, 156, 86, 171, 161, 129, 115, 116, - 85, 0, 150, 102, 107, 101, 138, 168, 169, 100, - 193, 91, 180, 88, 92, 179, 136, 166, 172, 130, - 127, 87, 170, 128, 126, 118, 105, 112, 144, 125, - 145, 113, 133, 132, 134, 0, 347, 0, 160, 177, - 194, 94, 362, 165, 184, 185, 186, 187, 188, 189, - 0, 0, 95, 108, 104, 143, 135, 93, 114, 157, - 117, 124, 149, 192, 140, 154, 98, 176, 158, 358, - 361, 356, 357, 395, 396, 432, 433, 434, 412, 353, - 0, 359, 360, 0, 416, 422, 423, 398, 82, 89, - 121, 191, 148, 106, 178, 426, 415, 0, 386, 429, - 364, 378, 437, 379, 380, 408, 350, 394, 139, 376, - 0, 367, 345, 373, 346, 365, 388, 103, 391, 363, - 417, 397, 428, 120, 435, 122, 402, 0, 159, 131, - 0, 0, 390, 419, 392, 413, 385, 409, 355, 401, - 430, 377, 406, 431, 0, 0, 0, 268, 0, 0, - 0, 0, 0, 0, 0, 0, 96, 0, 404, 425, - 375, 405, 407, 344, 403, 0, 348, 351, 436, 421, - 370, 371, 0, 0, 0, 0, 0, 0, 0, 389, - 393, 410, 383, 0, 0, 0, 0, 0, 0, 0, - 0, 368, 0, 400, 0, 0, 0, 352, 349, 0, - 0, 387, 0, 0, 0, 354, 0, 369, 411, 0, - 343, 109, 414, 420, 384, 210, 424, 382, 381, 427, - 146, 0, 162, 111, 119, 83, 90, 0, 110, 137, - 151, 155, 418, 366, 374, 99, 372, 153, 141, 175, - 399, 142, 152, 123, 167, 147, 174, 182, 183, 164, - 181, 190, 84, 163, 173, 97, 156, 86, 171, 161, - 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, - 168, 169, 100, 193, 91, 180, 88, 92, 179, 136, - 166, 172, 130, 127, 87, 170, 128, 126, 118, 105, - 112, 144, 125, 145, 113, 133, 132, 134, 0, 347, - 0, 160, 177, 194, 94, 362, 165, 184, 185, 186, - 187, 188, 189, 0, 0, 95, 108, 104, 143, 135, - 93, 114, 157, 117, 124, 149, 192, 140, 154, 98, - 176, 158, 358, 361, 356, 357, 395, 396, 432, 433, - 434, 412, 353, 0, 359, 360, 0, 416, 422, 423, - 398, 82, 89, 121, 191, 148, 106, 178, 426, 415, - 0, 386, 429, 364, 378, 437, 379, 380, 408, 350, - 394, 139, 376, 0, 367, 345, 373, 346, 365, 388, - 103, 391, 363, 417, 397, 428, 120, 435, 122, 402, - 0, 159, 131, 0, 0, 390, 419, 392, 413, 385, - 409, 355, 401, 430, 377, 406, 431, 0, 0, 0, + 349, 0, 161, 178, 195, 94, 364, 157, 166, 185, + 186, 187, 188, 189, 190, 0, 0, 95, 108, 104, + 143, 135, 93, 114, 158, 117, 124, 149, 193, 140, + 154, 98, 177, 159, 360, 363, 358, 359, 397, 398, + 434, 435, 436, 414, 355, 0, 361, 362, 0, 418, + 424, 425, 400, 82, 89, 121, 192, 148, 106, 179, + 428, 417, 0, 388, 431, 366, 380, 439, 381, 382, + 410, 352, 396, 139, 378, 0, 369, 347, 375, 348, + 367, 390, 103, 393, 365, 419, 399, 430, 120, 437, + 122, 404, 0, 160, 131, 0, 0, 392, 421, 394, + 415, 387, 411, 357, 403, 432, 379, 408, 433, 0, + 0, 0, 80, 0, 0, 0, 0, 0, 0, 0, + 0, 96, 0, 406, 427, 377, 407, 409, 346, 405, + 0, 350, 353, 438, 423, 372, 373, 0, 0, 0, + 0, 0, 0, 0, 391, 395, 412, 385, 0, 0, + 0, 0, 0, 0, 0, 0, 370, 0, 402, 0, + 0, 0, 354, 351, 0, 0, 389, 0, 0, 0, + 356, 0, 371, 413, 0, 345, 109, 416, 422, 386, + 211, 426, 384, 383, 429, 146, 0, 163, 111, 119, + 83, 90, 0, 110, 137, 151, 155, 420, 368, 376, + 99, 374, 153, 141, 176, 401, 142, 152, 123, 168, + 147, 175, 183, 184, 165, 182, 191, 84, 164, 174, + 97, 156, 86, 172, 162, 129, 115, 116, 85, 0, + 150, 102, 107, 101, 138, 169, 170, 100, 194, 91, + 181, 88, 343, 180, 136, 167, 173, 130, 127, 87, + 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, + 133, 132, 134, 0, 349, 0, 161, 178, 195, 94, + 364, 157, 166, 185, 186, 187, 188, 189, 190, 0, + 0, 95, 108, 104, 143, 344, 342, 114, 158, 117, + 124, 149, 193, 140, 154, 98, 177, 159, 360, 363, + 358, 359, 397, 398, 434, 435, 436, 414, 355, 0, + 361, 362, 0, 418, 424, 425, 400, 82, 89, 121, + 192, 148, 106, 179, 428, 417, 0, 388, 431, 366, + 380, 439, 381, 382, 410, 352, 396, 139, 378, 0, + 369, 347, 375, 348, 367, 390, 103, 393, 365, 419, + 399, 430, 120, 437, 122, 404, 0, 160, 131, 0, + 0, 392, 421, 394, 415, 387, 411, 357, 403, 432, + 379, 408, 433, 0, 0, 0, 209, 0, 0, 0, + 0, 0, 0, 0, 0, 96, 0, 406, 427, 377, + 407, 409, 346, 405, 0, 350, 353, 438, 423, 372, + 373, 0, 0, 0, 0, 0, 0, 0, 391, 395, + 412, 385, 0, 0, 0, 0, 0, 0, 0, 0, + 370, 0, 402, 0, 0, 0, 354, 351, 0, 0, + 389, 0, 0, 0, 356, 0, 371, 413, 0, 345, + 109, 416, 422, 386, 211, 426, 384, 383, 429, 146, + 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, + 155, 420, 368, 376, 99, 374, 153, 141, 176, 401, + 142, 152, 123, 168, 147, 175, 183, 184, 165, 182, + 191, 84, 164, 174, 97, 156, 86, 172, 162, 129, + 115, 116, 85, 0, 150, 102, 107, 101, 138, 169, + 170, 100, 194, 91, 181, 88, 92, 180, 136, 167, + 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, + 144, 125, 145, 113, 133, 132, 134, 0, 349, 0, + 161, 178, 195, 94, 364, 157, 166, 185, 186, 187, + 188, 189, 190, 0, 0, 95, 108, 104, 143, 135, + 93, 114, 158, 117, 124, 149, 193, 140, 154, 98, + 177, 159, 360, 363, 358, 359, 397, 398, 434, 435, + 436, 414, 355, 0, 361, 362, 0, 418, 424, 425, + 400, 82, 89, 121, 192, 148, 106, 179, 428, 417, + 0, 388, 431, 366, 380, 439, 381, 382, 410, 352, + 396, 139, 378, 0, 369, 347, 375, 348, 367, 390, + 103, 393, 365, 419, 399, 430, 120, 437, 122, 404, + 0, 160, 131, 0, 0, 392, 421, 394, 415, 387, + 411, 357, 403, 432, 379, 408, 433, 0, 0, 0, 80, 0, 0, 0, 0, 0, 0, 0, 0, 96, - 0, 404, 425, 375, 405, 407, 344, 403, 0, 348, - 351, 436, 421, 370, 371, 0, 0, 0, 0, 0, - 0, 0, 389, 393, 410, 383, 0, 0, 0, 0, - 0, 0, 0, 0, 368, 0, 400, 0, 0, 0, - 352, 349, 0, 0, 387, 0, 0, 0, 354, 0, - 369, 411, 0, 343, 109, 414, 420, 384, 210, 424, - 382, 381, 427, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 418, 366, 374, 99, 372, - 153, 141, 175, 399, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 341, 179, 136, 166, 172, 130, 127, 87, 170, 128, + 0, 406, 427, 377, 407, 409, 346, 405, 0, 350, + 353, 438, 423, 372, 373, 0, 0, 0, 0, 0, + 0, 0, 391, 395, 412, 385, 0, 0, 0, 0, + 0, 0, 0, 0, 370, 0, 402, 0, 0, 0, + 354, 351, 0, 0, 389, 0, 0, 0, 356, 0, + 371, 413, 0, 345, 109, 416, 422, 386, 211, 426, + 384, 383, 429, 146, 0, 163, 111, 119, 83, 90, + 0, 110, 137, 151, 155, 420, 368, 376, 99, 374, + 153, 141, 176, 401, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 625, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 343, 180, 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 347, 0, 160, 177, 194, 94, 362, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 342, 340, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 358, 361, 356, 357, 395, - 396, 432, 433, 434, 412, 353, 0, 359, 360, 0, - 416, 422, 423, 398, 82, 89, 121, 191, 148, 106, - 178, 426, 415, 0, 386, 429, 364, 378, 437, 379, - 380, 408, 350, 394, 139, 376, 0, 367, 345, 373, - 346, 365, 388, 103, 391, 363, 417, 397, 428, 120, - 435, 122, 402, 0, 159, 131, 0, 0, 390, 419, - 392, 413, 385, 409, 355, 401, 430, 377, 406, 431, - 0, 0, 0, 208, 0, 0, 0, 0, 0, 0, - 0, 0, 96, 0, 404, 425, 375, 405, 407, 344, - 403, 0, 348, 351, 436, 421, 370, 371, 0, 0, - 0, 0, 0, 0, 0, 389, 393, 410, 383, 0, - 0, 0, 0, 0, 0, 0, 0, 368, 0, 400, - 0, 0, 0, 352, 349, 0, 0, 387, 0, 0, - 0, 354, 0, 369, 411, 0, 343, 109, 414, 420, - 384, 210, 424, 382, 381, 427, 146, 0, 162, 111, - 119, 83, 90, 0, 110, 137, 151, 155, 418, 366, - 374, 99, 372, 153, 141, 175, 399, 142, 152, 123, - 167, 147, 174, 182, 183, 164, 181, 190, 84, 163, - 173, 97, 156, 86, 171, 161, 129, 115, 116, 85, - 0, 150, 102, 107, 101, 138, 168, 169, 100, 193, - 91, 180, 88, 92, 179, 136, 166, 172, 130, 127, - 87, 170, 128, 126, 118, 105, 112, 144, 125, 145, - 113, 133, 132, 134, 0, 347, 0, 160, 177, 194, - 94, 362, 165, 184, 185, 186, 187, 188, 189, 0, - 0, 95, 108, 104, 143, 135, 93, 114, 157, 117, - 124, 149, 192, 140, 154, 98, 176, 158, 358, 361, - 356, 357, 395, 396, 432, 433, 434, 412, 353, 0, - 359, 360, 0, 416, 422, 423, 398, 82, 89, 121, - 191, 148, 106, 178, 426, 415, 0, 386, 429, 364, - 378, 437, 379, 380, 408, 350, 394, 139, 376, 0, - 367, 345, 373, 346, 365, 388, 103, 391, 363, 417, - 397, 428, 120, 435, 122, 402, 0, 159, 131, 0, - 0, 390, 419, 392, 413, 385, 409, 355, 401, 430, - 377, 406, 431, 0, 0, 0, 80, 0, 0, 0, - 0, 0, 0, 0, 0, 96, 0, 404, 425, 375, - 405, 407, 344, 403, 0, 348, 351, 436, 421, 370, - 371, 0, 0, 0, 0, 0, 0, 0, 389, 393, - 410, 383, 0, 0, 0, 0, 0, 0, 0, 0, - 368, 0, 400, 0, 0, 0, 352, 349, 0, 0, - 387, 0, 0, 0, 354, 0, 369, 411, 0, 343, - 109, 414, 420, 384, 210, 424, 382, 381, 427, 146, - 0, 162, 111, 119, 83, 90, 0, 110, 137, 151, - 155, 418, 366, 374, 99, 372, 153, 141, 175, 399, - 142, 152, 123, 167, 147, 174, 182, 183, 164, 181, - 190, 84, 163, 622, 97, 156, 86, 171, 161, 129, - 115, 116, 85, 0, 150, 102, 107, 101, 138, 168, - 169, 100, 193, 91, 180, 88, 341, 179, 136, 166, - 172, 130, 127, 87, 170, 128, 126, 118, 105, 112, - 144, 125, 145, 113, 133, 132, 134, 0, 347, 0, - 160, 177, 194, 94, 362, 165, 184, 185, 186, 187, - 188, 189, 0, 0, 95, 108, 104, 143, 342, 340, - 114, 157, 117, 124, 149, 192, 140, 154, 98, 176, - 158, 358, 361, 356, 357, 395, 396, 432, 433, 434, - 412, 353, 0, 359, 360, 0, 416, 422, 423, 398, - 82, 89, 121, 191, 148, 106, 178, 426, 415, 0, - 386, 429, 364, 378, 437, 379, 380, 408, 350, 394, - 139, 376, 0, 367, 345, 373, 346, 365, 388, 103, - 391, 363, 417, 397, 428, 120, 435, 122, 402, 0, - 159, 131, 0, 0, 390, 419, 392, 413, 385, 409, - 355, 401, 430, 377, 406, 431, 0, 0, 0, 80, - 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, - 404, 425, 375, 405, 407, 344, 403, 0, 348, 351, - 436, 421, 370, 371, 0, 0, 0, 0, 0, 0, - 0, 389, 393, 410, 383, 0, 0, 0, 0, 0, - 0, 0, 0, 368, 0, 400, 0, 0, 0, 352, - 349, 0, 0, 387, 0, 0, 0, 354, 0, 369, - 411, 0, 343, 109, 414, 420, 384, 210, 424, 382, - 381, 427, 146, 0, 162, 111, 119, 83, 90, 0, - 110, 137, 151, 155, 418, 366, 374, 99, 372, 153, - 141, 175, 399, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 332, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 341, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, - 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 347, 0, 160, 177, 194, 94, 362, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 342, 340, 335, 334, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 358, 361, 356, 357, 395, 396, - 432, 433, 434, 412, 353, 0, 359, 360, 0, 416, - 422, 423, 398, 82, 89, 121, 191, 148, 106, 178, - 139, 0, 0, 0, 0, 270, 0, 0, 0, 103, - 0, 267, 0, 0, 0, 120, 310, 122, 0, 0, - 159, 131, 0, 0, 0, 0, 301, 302, 0, 0, - 0, 0, 0, 0, 856, 0, 54, 0, 0, 268, - 289, 288, 291, 292, 293, 294, 0, 0, 96, 290, - 295, 296, 297, 857, 0, 0, 265, 282, 0, 309, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 279, - 280, 0, 0, 0, 0, 323, 0, 281, 0, 0, - 276, 277, 278, 283, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 0, 0, 0, 210, 0, 0, - 321, 0, 146, 0, 162, 111, 119, 83, 90, 0, - 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, - 141, 175, 0, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 173, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 92, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, - 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 0, 0, 160, 177, 194, 94, 0, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 135, 93, 114, 157, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 311, 322, 317, 318, 315, 316, - 314, 313, 312, 324, 303, 304, 305, 306, 308, 0, - 319, 320, 307, 82, 89, 121, 191, 148, 106, 178, - 139, 0, 0, 789, 0, 270, 0, 0, 0, 103, - 0, 267, 0, 0, 0, 120, 310, 122, 0, 0, - 159, 131, 0, 0, 0, 0, 301, 302, 0, 0, - 0, 0, 0, 0, 0, 0, 54, 0, 0, 268, - 289, 288, 291, 292, 293, 294, 0, 0, 96, 290, - 295, 296, 297, 0, 0, 0, 265, 282, 0, 309, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 279, - 280, 261, 0, 0, 0, 323, 0, 281, 0, 0, - 276, 277, 278, 283, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 0, 0, 0, 210, 0, 0, - 321, 0, 146, 0, 162, 111, 119, 83, 90, 0, - 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, - 141, 175, 0, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 173, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 92, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, - 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 0, 0, 160, 177, 194, 94, 0, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 135, 93, 114, 157, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 311, 322, 317, 318, 315, 316, - 314, 313, 312, 324, 303, 304, 305, 306, 308, 0, - 319, 320, 307, 82, 89, 121, 191, 148, 106, 178, - 139, 0, 0, 0, 0, 270, 0, 0, 0, 103, - 0, 267, 0, 0, 0, 120, 310, 122, 0, 0, - 159, 131, 0, 0, 0, 0, 301, 302, 0, 0, - 0, 0, 0, 0, 0, 0, 54, 0, 498, 268, - 289, 288, 291, 292, 293, 294, 0, 0, 96, 290, - 295, 296, 297, 0, 0, 0, 265, 282, 0, 309, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 279, - 280, 0, 0, 0, 0, 323, 0, 281, 0, 0, - 276, 277, 278, 283, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 0, 0, 0, 210, 0, 0, - 321, 0, 146, 0, 162, 111, 119, 83, 90, 0, - 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, - 141, 175, 0, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 173, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 92, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, - 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 0, 0, 160, 177, 194, 94, 0, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 135, 93, 114, 157, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 311, 322, 317, 318, 315, 316, - 314, 313, 312, 324, 303, 304, 305, 306, 308, 0, - 319, 320, 307, 82, 89, 121, 191, 148, 106, 178, - 139, 0, 0, 0, 0, 270, 0, 0, 0, 103, - 0, 267, 0, 0, 0, 120, 310, 122, 0, 0, - 159, 131, 0, 0, 0, 0, 301, 302, 0, 0, - 0, 0, 0, 0, 0, 0, 54, 0, 0, 268, - 289, 288, 291, 292, 293, 294, 0, 0, 96, 290, - 295, 296, 297, 0, 0, 0, 265, 282, 0, 309, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 279, - 280, 261, 0, 0, 0, 323, 0, 281, 0, 0, - 276, 277, 278, 283, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 0, 0, 0, 210, 0, 0, - 321, 0, 146, 0, 162, 111, 119, 83, 90, 0, + 134, 0, 349, 0, 161, 178, 195, 94, 364, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 344, 342, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 360, 363, 358, 359, + 397, 398, 434, 435, 436, 414, 355, 0, 361, 362, + 0, 418, 424, 425, 400, 82, 89, 121, 192, 148, + 106, 179, 428, 417, 0, 388, 431, 366, 380, 439, + 381, 382, 410, 352, 396, 139, 378, 0, 369, 347, + 375, 348, 367, 390, 103, 393, 365, 419, 399, 430, + 120, 437, 122, 404, 0, 160, 131, 0, 0, 392, + 421, 394, 415, 387, 411, 357, 403, 432, 379, 408, + 433, 0, 0, 0, 80, 0, 0, 0, 0, 0, + 0, 0, 0, 96, 0, 406, 427, 377, 407, 409, + 346, 405, 0, 350, 353, 438, 423, 372, 373, 0, + 0, 0, 0, 0, 0, 0, 391, 395, 412, 385, + 0, 0, 0, 0, 0, 0, 0, 0, 370, 0, + 402, 0, 0, 0, 354, 351, 0, 0, 389, 0, + 0, 0, 356, 0, 371, 413, 0, 345, 109, 416, + 422, 386, 211, 426, 384, 383, 429, 146, 0, 163, + 111, 119, 83, 90, 0, 110, 137, 151, 155, 420, + 368, 376, 99, 374, 153, 141, 176, 401, 142, 152, + 123, 168, 147, 175, 183, 184, 165, 182, 191, 84, + 164, 334, 97, 156, 86, 172, 162, 129, 115, 116, + 85, 0, 150, 102, 107, 101, 138, 169, 170, 100, + 194, 91, 181, 88, 343, 180, 136, 167, 173, 130, + 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, + 145, 113, 133, 132, 134, 0, 349, 0, 161, 178, + 195, 94, 364, 157, 166, 185, 186, 187, 188, 189, + 190, 0, 0, 95, 108, 104, 143, 344, 342, 337, + 336, 117, 124, 149, 193, 140, 154, 98, 177, 159, + 360, 363, 358, 359, 397, 398, 434, 435, 436, 414, + 355, 0, 361, 362, 0, 418, 424, 425, 400, 82, + 89, 121, 192, 148, 106, 179, 139, 0, 0, 792, + 0, 272, 0, 0, 0, 103, 0, 269, 0, 0, + 0, 120, 312, 122, 0, 0, 160, 131, 0, 0, + 0, 0, 303, 304, 0, 0, 0, 0, 0, 0, + 0, 0, 54, 0, 0, 270, 291, 290, 293, 294, + 295, 296, 0, 0, 96, 292, 297, 298, 299, 0, + 0, 0, 267, 284, 0, 311, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 281, 282, 263, 0, 0, + 0, 325, 0, 283, 0, 0, 278, 279, 280, 285, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, + 0, 0, 0, 211, 0, 0, 323, 0, 146, 0, + 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, + 0, 0, 0, 99, 0, 153, 141, 176, 0, 142, + 152, 123, 168, 147, 175, 183, 184, 165, 182, 191, + 84, 164, 174, 97, 156, 86, 172, 162, 129, 115, + 116, 85, 0, 150, 102, 107, 101, 138, 169, 170, + 100, 194, 91, 181, 88, 92, 180, 136, 167, 173, + 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, + 125, 145, 113, 133, 132, 134, 0, 0, 0, 161, + 178, 195, 94, 0, 157, 166, 185, 186, 187, 188, + 189, 190, 0, 0, 95, 108, 104, 143, 135, 93, + 114, 158, 117, 124, 149, 193, 140, 154, 98, 177, + 159, 313, 324, 319, 320, 317, 318, 316, 315, 314, + 326, 305, 306, 307, 308, 310, 0, 321, 322, 309, + 82, 89, 121, 192, 148, 106, 179, 139, 0, 0, + 0, 0, 272, 0, 0, 0, 103, 0, 269, 0, + 0, 0, 120, 312, 122, 0, 0, 160, 131, 0, + 0, 0, 0, 303, 304, 0, 0, 0, 0, 0, + 0, 0, 0, 54, 0, 501, 270, 291, 290, 293, + 294, 295, 296, 0, 0, 96, 292, 297, 298, 299, + 0, 0, 0, 267, 284, 0, 311, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 281, 282, 0, 0, + 0, 0, 325, 0, 283, 0, 0, 278, 279, 280, + 285, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 109, 0, 0, 0, 211, 0, 0, 323, 0, 146, + 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, + 155, 0, 0, 0, 99, 0, 153, 141, 176, 0, + 142, 152, 123, 168, 147, 175, 183, 184, 165, 182, + 191, 84, 164, 174, 97, 156, 86, 172, 162, 129, + 115, 116, 85, 0, 150, 102, 107, 101, 138, 169, + 170, 100, 194, 91, 181, 88, 92, 180, 136, 167, + 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, + 144, 125, 145, 113, 133, 132, 134, 0, 0, 0, + 161, 178, 195, 94, 0, 157, 166, 185, 186, 187, + 188, 189, 190, 0, 0, 95, 108, 104, 143, 135, + 93, 114, 158, 117, 124, 149, 193, 140, 154, 98, + 177, 159, 313, 324, 319, 320, 317, 318, 316, 315, + 314, 326, 305, 306, 307, 308, 310, 0, 321, 322, + 309, 82, 89, 121, 192, 148, 106, 179, 139, 0, + 0, 0, 0, 272, 0, 0, 0, 103, 0, 269, + 0, 0, 0, 120, 312, 122, 0, 0, 160, 131, + 0, 0, 0, 0, 303, 304, 0, 0, 0, 0, + 0, 0, 0, 0, 54, 0, 0, 270, 291, 290, + 293, 294, 295, 296, 0, 0, 96, 292, 297, 298, + 299, 0, 0, 0, 267, 284, 0, 311, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 281, 282, 263, + 0, 0, 0, 325, 0, 283, 0, 0, 278, 279, + 280, 285, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 109, 0, 0, 0, 211, 0, 0, 323, 0, + 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, + 151, 155, 0, 0, 0, 99, 0, 153, 141, 176, + 0, 142, 152, 123, 168, 147, 175, 183, 184, 165, + 182, 191, 84, 164, 174, 97, 156, 86, 172, 162, + 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, + 169, 170, 100, 194, 91, 181, 88, 92, 180, 136, + 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, + 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, + 0, 161, 178, 195, 94, 0, 157, 166, 185, 186, + 187, 188, 189, 190, 0, 0, 95, 108, 104, 143, + 135, 93, 114, 158, 117, 124, 149, 193, 140, 154, + 98, 177, 159, 313, 324, 319, 320, 317, 318, 316, + 315, 314, 326, 305, 306, 307, 308, 310, 24, 321, + 322, 309, 82, 89, 121, 192, 148, 106, 179, 0, + 139, 0, 0, 0, 0, 272, 0, 0, 0, 103, + 0, 269, 0, 0, 0, 120, 312, 122, 0, 0, + 160, 131, 0, 0, 0, 0, 303, 304, 0, 0, + 0, 0, 0, 0, 0, 0, 54, 0, 0, 270, + 291, 290, 293, 294, 295, 296, 0, 0, 96, 292, + 297, 298, 299, 0, 0, 0, 267, 284, 0, 311, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 281, + 282, 0, 0, 0, 0, 325, 0, 283, 0, 0, + 278, 279, 280, 285, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 109, 0, 0, 0, 211, 0, 0, + 323, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, - 141, 175, 0, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 173, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 92, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, + 141, 176, 0, 142, 152, 123, 168, 147, 175, 183, + 184, 165, 182, 191, 84, 164, 174, 97, 156, 86, + 172, 162, 129, 115, 116, 85, 0, 150, 102, 107, + 101, 138, 169, 170, 100, 194, 91, 181, 88, 92, + 180, 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 0, 0, 160, 177, 194, 94, 0, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 135, 93, 114, 157, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 311, 322, 317, 318, 315, 316, - 314, 313, 312, 324, 303, 304, 305, 306, 308, 24, - 319, 320, 307, 82, 89, 121, 191, 148, 106, 178, - 0, 139, 0, 0, 0, 0, 270, 0, 0, 0, - 103, 0, 267, 0, 0, 0, 120, 310, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 301, 302, 0, - 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, - 268, 289, 288, 291, 292, 293, 294, 0, 0, 96, - 290, 295, 296, 297, 0, 0, 0, 265, 282, 0, - 309, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 279, 280, 0, 0, 0, 0, 323, 0, 281, 0, - 0, 276, 277, 278, 283, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 321, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 311, 322, 317, 318, 315, - 316, 314, 313, 312, 324, 303, 304, 305, 306, 308, - 0, 319, 320, 307, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 0, 270, 0, 0, 0, - 103, 0, 267, 0, 0, 0, 120, 310, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 301, 302, 0, - 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, - 268, 289, 288, 291, 292, 293, 294, 0, 0, 96, - 290, 295, 296, 297, 0, 0, 0, 265, 282, 0, - 309, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 279, 280, 0, 0, 0, 0, 323, 0, 281, 0, - 0, 276, 277, 278, 283, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 321, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 311, 322, 317, 318, 315, - 316, 314, 313, 312, 324, 303, 304, 305, 306, 308, - 0, 319, 320, 307, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 0, 0, 0, 0, 0, - 103, 0, 0, 0, 0, 0, 120, 310, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 301, 302, 0, - 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, - 268, 289, 288, 291, 292, 293, 294, 0, 0, 96, - 290, 295, 296, 297, 0, 0, 0, 0, 282, 0, - 309, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 279, 280, 0, 0, 0, 0, 323, 0, 281, 0, - 0, 276, 277, 278, 283, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 321, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 1431, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 311, 322, 317, 318, 315, - 316, 314, 313, 312, 324, 303, 304, 305, 306, 308, - 0, 319, 320, 307, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 0, 0, 0, 0, 0, - 103, 0, 0, 0, 0, 0, 120, 310, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 301, 302, 0, - 0, 0, 0, 0, 0, 0, 0, 54, 0, 498, - 268, 289, 288, 291, 292, 293, 294, 0, 0, 96, - 290, 295, 296, 297, 0, 0, 0, 0, 282, 0, - 309, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 279, 280, 0, 0, 0, 0, 323, 0, 281, 0, - 0, 276, 277, 278, 283, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 321, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 311, 322, 317, 318, 315, - 316, 314, 313, 312, 324, 303, 304, 305, 306, 308, - 0, 319, 320, 307, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 0, 0, 0, 0, 0, - 103, 0, 0, 0, 0, 0, 120, 310, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 301, 302, 0, + 0, 0, 0, 161, 178, 195, 94, 0, 157, 166, + 185, 186, 187, 188, 189, 190, 0, 0, 95, 108, + 104, 143, 135, 93, 114, 158, 117, 124, 149, 193, + 140, 154, 98, 177, 159, 313, 324, 319, 320, 317, + 318, 316, 315, 314, 326, 305, 306, 307, 308, 310, + 0, 321, 322, 309, 82, 89, 121, 192, 148, 106, + 179, 139, 0, 0, 0, 0, 272, 0, 0, 0, + 103, 0, 269, 0, 0, 0, 120, 312, 122, 0, + 0, 160, 131, 0, 0, 0, 0, 303, 304, 0, 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, - 268, 289, 288, 291, 292, 293, 294, 0, 0, 96, - 290, 295, 296, 297, 0, 0, 0, 0, 282, 0, - 309, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 279, 280, 0, 0, 0, 0, 323, 0, 281, 0, - 0, 276, 277, 278, 283, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 321, 0, 146, 0, 162, 111, 119, 83, 90, + 270, 291, 290, 293, 294, 295, 296, 0, 0, 96, + 292, 297, 298, 299, 0, 0, 0, 267, 284, 0, + 311, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 281, 282, 0, 0, 0, 0, 325, 0, 283, 0, + 0, 278, 279, 280, 285, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 109, 0, 0, 0, 211, 0, + 0, 323, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, + 153, 141, 176, 0, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 174, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 92, 180, 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 311, 322, 317, 318, 315, - 316, 314, 313, 312, 324, 303, 304, 305, 306, 308, - 0, 319, 320, 307, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 0, 0, 0, 0, 0, - 103, 0, 0, 0, 0, 0, 120, 0, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 0, 0, 0, + 134, 0, 0, 0, 161, 178, 195, 94, 0, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 135, 93, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 313, 324, 319, 320, + 317, 318, 316, 315, 314, 326, 305, 306, 307, 308, + 310, 0, 321, 322, 309, 82, 89, 121, 192, 148, + 106, 179, 139, 0, 0, 0, 0, 0, 0, 0, + 0, 103, 0, 0, 0, 0, 0, 120, 312, 122, + 0, 0, 160, 131, 0, 0, 0, 0, 303, 304, + 0, 0, 0, 0, 0, 0, 0, 0, 54, 0, + 0, 270, 291, 290, 293, 294, 295, 296, 0, 0, + 96, 292, 297, 298, 299, 0, 0, 0, 0, 284, + 0, 311, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 281, 282, 0, 0, 0, 0, 325, 0, 283, + 0, 0, 278, 279, 280, 285, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 109, 0, 0, 0, 211, + 0, 0, 323, 0, 146, 0, 163, 111, 119, 83, + 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, + 0, 153, 141, 176, 1434, 142, 152, 123, 168, 147, + 175, 183, 184, 165, 182, 191, 84, 164, 174, 97, + 156, 86, 172, 162, 129, 115, 116, 85, 0, 150, + 102, 107, 101, 138, 169, 170, 100, 194, 91, 181, + 88, 92, 180, 136, 167, 173, 130, 127, 87, 171, + 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, + 132, 134, 0, 0, 0, 161, 178, 195, 94, 0, + 157, 166, 185, 186, 187, 188, 189, 190, 0, 0, + 95, 108, 104, 143, 135, 93, 114, 158, 117, 124, + 149, 193, 140, 154, 98, 177, 159, 313, 324, 319, + 320, 317, 318, 316, 315, 314, 326, 305, 306, 307, + 308, 310, 0, 321, 322, 309, 82, 89, 121, 192, + 148, 106, 179, 139, 0, 0, 0, 0, 0, 0, + 0, 0, 103, 0, 0, 0, 0, 0, 120, 312, + 122, 0, 0, 160, 131, 0, 0, 0, 0, 303, + 304, 0, 0, 0, 0, 0, 0, 0, 0, 54, + 0, 501, 270, 291, 290, 293, 294, 295, 296, 0, + 0, 96, 292, 297, 298, 299, 0, 0, 0, 0, + 284, 0, 311, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 281, 282, 0, 0, 0, 0, 325, 0, + 283, 0, 0, 278, 279, 280, 285, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 109, 0, 0, 0, + 211, 0, 0, 323, 0, 146, 0, 163, 111, 119, + 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, + 99, 0, 153, 141, 176, 0, 142, 152, 123, 168, + 147, 175, 183, 184, 165, 182, 191, 84, 164, 174, + 97, 156, 86, 172, 162, 129, 115, 116, 85, 0, + 150, 102, 107, 101, 138, 169, 170, 100, 194, 91, + 181, 88, 92, 180, 136, 167, 173, 130, 127, 87, + 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, + 133, 132, 134, 0, 0, 0, 161, 178, 195, 94, + 0, 157, 166, 185, 186, 187, 188, 189, 190, 0, + 0, 95, 108, 104, 143, 135, 93, 114, 158, 117, + 124, 149, 193, 140, 154, 98, 177, 159, 313, 324, + 319, 320, 317, 318, 316, 315, 314, 326, 305, 306, + 307, 308, 310, 0, 321, 322, 309, 82, 89, 121, + 192, 148, 106, 179, 139, 0, 0, 0, 0, 0, + 0, 0, 0, 103, 0, 0, 0, 0, 0, 120, + 312, 122, 0, 0, 160, 131, 0, 0, 0, 0, + 303, 304, 0, 0, 0, 0, 0, 0, 0, 0, + 54, 0, 0, 270, 291, 290, 293, 294, 295, 296, + 0, 0, 96, 292, 297, 298, 299, 0, 0, 0, + 0, 284, 0, 311, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 281, 282, 0, 0, 0, 0, 325, + 0, 283, 0, 0, 278, 279, 280, 285, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, + 0, 211, 0, 0, 323, 0, 146, 0, 163, 111, + 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, + 0, 99, 0, 153, 141, 176, 0, 142, 152, 123, + 168, 147, 175, 183, 184, 165, 182, 191, 84, 164, + 174, 97, 156, 86, 172, 162, 129, 115, 116, 85, + 0, 150, 102, 107, 101, 138, 169, 170, 100, 194, + 91, 181, 88, 92, 180, 136, 167, 173, 130, 127, + 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, + 113, 133, 132, 134, 0, 0, 0, 161, 178, 195, + 94, 0, 157, 166, 185, 186, 187, 188, 189, 190, + 0, 0, 95, 108, 104, 143, 135, 93, 114, 158, + 117, 124, 149, 193, 140, 154, 98, 177, 159, 313, + 324, 319, 320, 317, 318, 316, 315, 314, 326, 305, + 306, 307, 308, 310, 0, 321, 322, 309, 82, 89, + 121, 192, 148, 106, 179, 139, 0, 0, 0, 0, + 0, 0, 0, 0, 103, 0, 0, 0, 0, 0, + 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 80, 0, 0, 0, 0, 0, + 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 80, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 535, 534, 544, 545, 537, 538, 539, 540, 541, 542, + 543, 536, 0, 0, 546, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, + 0, 0, 211, 0, 0, 0, 0, 146, 0, 163, + 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, + 0, 0, 99, 0, 153, 141, 176, 0, 142, 152, + 123, 168, 147, 175, 183, 184, 165, 182, 191, 84, + 164, 174, 97, 156, 86, 172, 162, 129, 115, 116, + 85, 0, 150, 102, 107, 101, 138, 169, 170, 100, + 194, 91, 181, 88, 92, 180, 136, 167, 173, 130, + 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, + 145, 113, 133, 132, 134, 0, 0, 0, 161, 178, + 195, 94, 0, 157, 166, 185, 186, 187, 188, 189, + 190, 0, 0, 95, 108, 104, 143, 135, 93, 114, + 158, 117, 124, 149, 193, 140, 154, 98, 177, 159, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 82, + 89, 121, 192, 148, 106, 179, 139, 0, 0, 0, + 523, 0, 0, 0, 0, 103, 0, 0, 0, 0, + 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 80, 0, 525, 0, 0, + 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, + 520, 519, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 521, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 532, 531, 541, 542, - 534, 535, 536, 537, 538, 539, 540, 533, 0, 0, - 543, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 0, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 82, 89, 121, 191, 148, 106, - 178, 139, 0, 0, 0, 520, 0, 0, 0, 0, - 103, 0, 0, 0, 0, 0, 120, 0, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, + 0, 0, 0, 211, 0, 0, 0, 0, 146, 0, + 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, + 0, 0, 0, 99, 0, 153, 141, 176, 0, 142, + 152, 123, 168, 147, 175, 183, 184, 165, 182, 191, + 84, 164, 174, 97, 156, 86, 172, 162, 129, 115, + 116, 85, 0, 150, 102, 107, 101, 138, 169, 170, + 100, 194, 91, 181, 88, 92, 180, 136, 167, 173, + 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, + 125, 145, 113, 133, 132, 134, 0, 0, 0, 161, + 178, 195, 94, 0, 157, 166, 185, 186, 187, 188, + 189, 190, 0, 0, 95, 108, 104, 143, 135, 93, + 114, 158, 117, 124, 149, 193, 140, 154, 98, 177, + 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, + 82, 89, 121, 192, 148, 106, 179, 103, 0, 0, + 0, 0, 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 80, 0, 522, 0, 0, 0, 0, 0, 0, 96, - 0, 0, 0, 0, 0, 517, 516, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 80, 0, 0, + 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, + 0, 0, 74, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 518, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 0, 0, 146, 0, 162, 111, 119, 83, 90, - 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, - 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 139, 0, 82, 89, 121, 191, 148, 106, - 178, 103, 0, 0, 0, 0, 0, 120, 0, 122, - 0, 0, 159, 131, 0, 0, 0, 0, 0, 0, + 0, 109, 76, 77, 0, 73, 0, 0, 0, 78, + 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, + 151, 155, 0, 0, 0, 99, 0, 153, 141, 176, + 0, 142, 152, 123, 168, 147, 175, 183, 184, 165, + 182, 191, 84, 164, 174, 97, 156, 86, 172, 162, + 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, + 169, 170, 100, 194, 91, 181, 88, 92, 180, 136, + 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, + 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, + 0, 161, 178, 195, 94, 0, 157, 166, 185, 186, + 187, 188, 189, 190, 0, 0, 95, 108, 104, 143, + 135, 93, 114, 158, 117, 124, 149, 193, 140, 154, + 98, 177, 159, 0, 75, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 82, 89, 121, 192, 148, 106, 179, 139, + 0, 0, 0, 846, 0, 0, 0, 0, 103, 0, + 0, 0, 0, 0, 120, 0, 122, 0, 0, 160, + 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 209, 0, + 848, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 80, 0, 0, 0, 0, 0, 0, 0, 0, - 96, 0, 0, 0, 0, 0, 74, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 109, 76, 77, 0, 73, - 0, 0, 0, 78, 146, 0, 162, 111, 119, 83, - 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, - 0, 153, 141, 175, 0, 142, 152, 123, 167, 147, - 174, 182, 183, 164, 181, 190, 84, 163, 173, 97, - 156, 86, 171, 161, 129, 115, 116, 85, 0, 150, - 102, 107, 101, 138, 168, 169, 100, 193, 91, 180, - 88, 92, 179, 136, 166, 172, 130, 127, 87, 170, - 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, - 132, 134, 0, 0, 0, 160, 177, 194, 94, 0, - 165, 184, 185, 186, 187, 188, 189, 0, 0, 95, - 108, 104, 143, 135, 93, 114, 157, 117, 124, 149, - 192, 140, 154, 98, 176, 158, 0, 75, 0, 0, + 0, 0, 109, 0, 0, 0, 211, 0, 0, 0, + 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, + 137, 151, 155, 0, 0, 0, 99, 0, 153, 141, + 176, 0, 142, 152, 123, 168, 147, 175, 183, 184, + 165, 182, 191, 84, 164, 174, 97, 156, 86, 172, + 162, 129, 115, 116, 85, 0, 150, 102, 107, 101, + 138, 169, 170, 100, 194, 91, 181, 88, 92, 180, + 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, + 105, 112, 144, 125, 145, 113, 133, 132, 134, 0, + 0, 0, 161, 178, 195, 94, 0, 157, 166, 185, + 186, 187, 188, 189, 190, 0, 0, 95, 108, 104, + 143, 135, 93, 114, 158, 117, 124, 149, 193, 140, + 154, 98, 177, 159, 0, 0, 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 82, 89, 121, 191, 148, - 106, 178, 139, 0, 0, 0, 843, 0, 0, 0, - 0, 103, 0, 0, 0, 0, 0, 120, 0, 122, - 0, 0, 159, 131, 0, 0, 0, 0, 0, 0, + 0, 139, 0, 82, 89, 121, 192, 148, 106, 179, + 103, 0, 0, 0, 0, 0, 120, 0, 122, 0, + 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, + 80, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 208, 0, 845, 0, 0, 0, 0, 0, 0, - 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 109, 0, 0, 0, 210, - 0, 0, 0, 0, 146, 0, 162, 111, 119, 83, - 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, - 0, 153, 141, 175, 0, 142, 152, 123, 167, 147, - 174, 182, 183, 164, 181, 190, 84, 163, 173, 97, - 156, 86, 171, 161, 129, 115, 116, 85, 0, 150, - 102, 107, 101, 138, 168, 169, 100, 193, 91, 180, - 88, 92, 179, 136, 166, 172, 130, 127, 87, 170, - 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, - 132, 134, 0, 0, 0, 160, 177, 194, 94, 0, - 165, 184, 185, 186, 187, 188, 189, 0, 0, 95, - 108, 104, 143, 135, 93, 114, 157, 117, 124, 149, - 192, 140, 154, 98, 176, 158, 0, 0, 0, 0, + 0, 0, 0, 0, 109, 0, 0, 0, 211, 0, + 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, + 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, + 153, 141, 176, 0, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 174, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 92, 180, 136, 167, 173, 130, 127, 87, 171, 128, + 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, + 134, 0, 0, 0, 161, 178, 195, 94, 0, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 135, 93, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 139, 0, 82, 89, 121, 191, 148, - 106, 178, 103, 0, 0, 0, 0, 0, 120, 0, - 122, 0, 0, 159, 131, 0, 0, 0, 0, 0, + 0, 0, 0, 139, 0, 82, 89, 121, 192, 148, + 106, 179, 103, 0, 0, 0, 0, 0, 120, 0, + 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 54, - 0, 0, 80, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 209, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, 0, - 210, 0, 0, 0, 0, 146, 0, 162, 111, 119, + 211, 0, 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, - 99, 0, 153, 141, 175, 0, 142, 152, 123, 167, - 147, 174, 182, 183, 164, 181, 190, 84, 163, 173, - 97, 156, 86, 171, 161, 129, 115, 116, 85, 0, - 150, 102, 107, 101, 138, 168, 169, 100, 193, 91, - 180, 88, 92, 179, 136, 166, 172, 130, 127, 87, - 170, 128, 126, 118, 105, 112, 144, 125, 145, 113, - 133, 132, 134, 0, 0, 0, 160, 177, 194, 94, - 0, 165, 184, 185, 186, 187, 188, 189, 0, 0, - 95, 108, 104, 143, 135, 93, 114, 157, 117, 124, - 149, 192, 140, 154, 98, 176, 158, 0, 0, 0, - 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 139, 0, 82, 89, 121, 191, - 148, 106, 178, 103, 0, 0, 0, 0, 0, 120, - 0, 122, 0, 0, 159, 131, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 54, 0, 0, 208, 0, 0, 0, 0, 0, 0, + 99, 0, 153, 141, 176, 0, 142, 152, 123, 168, + 147, 175, 183, 184, 165, 182, 191, 84, 164, 174, + 97, 156, 86, 172, 162, 129, 115, 116, 85, 0, + 150, 102, 107, 101, 138, 169, 170, 100, 194, 91, + 181, 88, 92, 180, 136, 167, 173, 130, 127, 87, + 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, + 133, 132, 134, 0, 0, 0, 161, 178, 195, 94, + 0, 157, 166, 185, 186, 187, 188, 189, 190, 0, + 0, 95, 108, 104, 143, 135, 93, 114, 158, 117, + 124, 149, 193, 140, 154, 98, 177, 159, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 82, 89, 121, + 192, 148, 106, 179, 139, 0, 0, 0, 846, 0, + 0, 0, 0, 103, 0, 0, 0, 0, 0, 120, + 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 209, 0, 848, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, - 0, 210, 0, 0, 0, 0, 146, 0, 162, 111, + 0, 211, 0, 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, - 0, 99, 0, 153, 141, 175, 0, 142, 152, 123, - 167, 147, 174, 182, 183, 164, 181, 190, 84, 163, - 173, 97, 156, 86, 171, 161, 129, 115, 116, 85, - 0, 150, 102, 107, 101, 138, 168, 169, 100, 193, - 91, 180, 88, 92, 179, 136, 166, 172, 130, 127, - 87, 170, 128, 126, 118, 105, 112, 144, 125, 145, - 113, 133, 132, 134, 0, 0, 0, 160, 177, 194, - 94, 0, 165, 184, 185, 186, 187, 188, 189, 0, - 0, 95, 108, 104, 143, 135, 93, 114, 157, 117, - 124, 149, 192, 140, 154, 98, 176, 158, 0, 0, + 0, 99, 0, 153, 141, 176, 0, 844, 152, 123, + 168, 147, 175, 183, 184, 165, 182, 191, 84, 164, + 174, 97, 156, 86, 172, 162, 129, 115, 116, 85, + 0, 150, 102, 107, 101, 138, 169, 170, 100, 194, + 91, 181, 88, 92, 180, 136, 167, 173, 130, 127, + 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, + 113, 133, 132, 134, 0, 0, 0, 161, 178, 195, + 94, 0, 157, 166, 185, 186, 187, 188, 189, 190, + 0, 0, 95, 108, 104, 143, 135, 93, 114, 158, + 117, 124, 149, 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 82, 89, 121, - 191, 148, 106, 178, 139, 0, 0, 0, 843, 0, - 0, 0, 0, 103, 0, 0, 0, 0, 0, 120, - 0, 122, 0, 0, 159, 131, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 139, 0, 82, 89, + 121, 192, 148, 106, 179, 103, 0, 0, 0, 0, + 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 208, 0, 845, 0, 0, 0, 0, - 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 80, 0, 0, 743, 0, + 0, 744, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, - 0, 210, 0, 0, 0, 0, 146, 0, 162, 111, - 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, - 0, 99, 0, 153, 141, 175, 0, 841, 152, 123, - 167, 147, 174, 182, 183, 164, 181, 190, 84, 163, - 173, 97, 156, 86, 171, 161, 129, 115, 116, 85, - 0, 150, 102, 107, 101, 138, 168, 169, 100, 193, - 91, 180, 88, 92, 179, 136, 166, 172, 130, 127, - 87, 170, 128, 126, 118, 105, 112, 144, 125, 145, - 113, 133, 132, 134, 0, 0, 0, 160, 177, 194, - 94, 0, 165, 184, 185, 186, 187, 188, 189, 0, - 0, 95, 108, 104, 143, 135, 93, 114, 157, 117, - 124, 149, 192, 140, 154, 98, 176, 158, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, + 0, 0, 0, 211, 0, 0, 0, 0, 146, 0, + 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, + 0, 0, 0, 99, 0, 153, 141, 176, 0, 142, + 152, 123, 168, 147, 175, 183, 184, 165, 182, 191, + 84, 164, 174, 97, 156, 86, 172, 162, 129, 115, + 116, 85, 0, 150, 102, 107, 101, 138, 169, 170, + 100, 194, 91, 181, 88, 92, 180, 136, 167, 173, + 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, + 125, 145, 113, 133, 132, 134, 0, 0, 0, 161, + 178, 195, 94, 0, 157, 166, 185, 186, 187, 188, + 189, 190, 0, 0, 95, 108, 104, 143, 135, 93, + 114, 158, 117, 124, 149, 193, 140, 154, 98, 177, + 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, + 82, 89, 121, 192, 148, 106, 179, 103, 0, 634, + 0, 0, 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 139, 0, 82, 89, 121, - 191, 148, 106, 178, 103, 0, 0, 0, 0, 0, - 120, 0, 122, 0, 0, 159, 131, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 80, 0, 633, + 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 80, 0, 0, 740, 0, 0, - 741, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, - 0, 0, 210, 0, 0, 0, 0, 146, 0, 162, - 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, - 0, 0, 99, 0, 153, 141, 175, 0, 142, 152, - 123, 167, 147, 174, 182, 183, 164, 181, 190, 84, - 163, 173, 97, 156, 86, 171, 161, 129, 115, 116, - 85, 0, 150, 102, 107, 101, 138, 168, 169, 100, - 193, 91, 180, 88, 92, 179, 136, 166, 172, 130, - 127, 87, 170, 128, 126, 118, 105, 112, 144, 125, - 145, 113, 133, 132, 134, 0, 0, 0, 160, 177, - 194, 94, 0, 165, 184, 185, 186, 187, 188, 189, - 0, 0, 95, 108, 104, 143, 135, 93, 114, 157, - 117, 124, 149, 192, 140, 154, 98, 176, 158, 0, + 0, 109, 0, 0, 0, 211, 0, 0, 0, 0, + 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, + 151, 155, 0, 0, 0, 99, 0, 153, 141, 176, + 0, 142, 152, 123, 168, 147, 175, 183, 184, 165, + 182, 191, 84, 164, 174, 97, 156, 86, 172, 162, + 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, + 169, 170, 100, 194, 91, 181, 88, 92, 180, 136, + 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, + 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, + 0, 161, 178, 195, 94, 0, 157, 166, 185, 186, + 187, 188, 189, 190, 0, 0, 95, 108, 104, 143, + 135, 93, 114, 158, 117, 124, 149, 193, 140, 154, + 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 139, 0, 82, 89, - 121, 191, 148, 106, 178, 103, 0, 631, 0, 0, - 0, 120, 0, 122, 0, 0, 159, 131, 0, 0, + 139, 0, 82, 89, 121, 192, 148, 106, 179, 103, + 0, 0, 0, 0, 0, 120, 0, 122, 0, 0, + 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 54, 0, 0, 209, + 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 80, 0, 630, 0, 0, - 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, - 0, 0, 0, 210, 0, 0, 0, 0, 146, 0, - 162, 111, 119, 83, 90, 0, 110, 137, 151, 155, - 0, 0, 0, 99, 0, 153, 141, 175, 0, 142, - 152, 123, 167, 147, 174, 182, 183, 164, 181, 190, - 84, 163, 173, 97, 156, 86, 171, 161, 129, 115, - 116, 85, 0, 150, 102, 107, 101, 138, 168, 169, - 100, 193, 91, 180, 88, 92, 179, 136, 166, 172, - 130, 127, 87, 170, 128, 126, 118, 105, 112, 144, - 125, 145, 113, 133, 132, 134, 0, 0, 0, 160, - 177, 194, 94, 0, 165, 184, 185, 186, 187, 188, - 189, 0, 0, 95, 108, 104, 143, 135, 93, 114, - 157, 117, 124, 149, 192, 140, 154, 98, 176, 158, + 0, 0, 0, 109, 0, 0, 0, 211, 0, 0, + 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, + 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, + 141, 176, 0, 142, 152, 123, 168, 147, 175, 183, + 184, 165, 182, 191, 84, 164, 174, 97, 156, 86, + 172, 162, 129, 115, 116, 85, 0, 150, 102, 107, + 101, 138, 169, 170, 100, 194, 91, 181, 88, 92, + 180, 136, 167, 173, 130, 127, 87, 171, 128, 126, + 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, + 0, 0, 0, 161, 178, 195, 94, 0, 157, 166, + 185, 186, 187, 188, 189, 190, 0, 0, 95, 108, + 104, 143, 135, 93, 114, 158, 117, 124, 149, 193, + 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 139, 0, 82, - 89, 121, 191, 148, 106, 178, 103, 0, 0, 0, - 0, 0, 120, 0, 122, 0, 0, 159, 131, 0, + 0, 0, 139, 0, 82, 89, 121, 192, 148, 106, + 179, 103, 0, 0, 0, 0, 0, 120, 0, 122, + 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 54, 0, 0, 208, 0, 0, 0, - 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, + 0, 209, 0, 848, 0, 0, 0, 0, 0, 0, + 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 109, 0, 0, 0, 211, + 0, 0, 0, 0, 146, 0, 163, 111, 119, 83, + 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, + 0, 153, 141, 176, 0, 142, 152, 123, 168, 147, + 175, 183, 184, 165, 182, 191, 84, 164, 174, 97, + 156, 86, 172, 162, 129, 115, 116, 85, 0, 150, + 102, 107, 101, 138, 169, 170, 100, 194, 91, 181, + 88, 92, 180, 136, 167, 173, 130, 127, 87, 171, + 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, + 132, 134, 0, 0, 0, 161, 178, 195, 94, 0, + 157, 166, 185, 186, 187, 188, 189, 190, 0, 0, + 95, 108, 104, 143, 135, 93, 114, 158, 117, 124, + 149, 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 109, 0, 0, 0, 210, 0, 0, 0, 0, 146, - 0, 162, 111, 119, 83, 90, 0, 110, 137, 151, - 155, 0, 0, 0, 99, 0, 153, 141, 175, 0, - 142, 152, 123, 167, 147, 174, 182, 183, 164, 181, - 190, 84, 163, 173, 97, 156, 86, 171, 161, 129, - 115, 116, 85, 0, 150, 102, 107, 101, 138, 168, - 169, 100, 193, 91, 180, 88, 92, 179, 136, 166, - 172, 130, 127, 87, 170, 128, 126, 118, 105, 112, - 144, 125, 145, 113, 133, 132, 134, 0, 0, 0, - 160, 177, 194, 94, 0, 165, 184, 185, 186, 187, - 188, 189, 0, 0, 95, 108, 104, 143, 135, 93, - 114, 157, 117, 124, 149, 192, 140, 154, 98, 176, - 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, - 82, 89, 121, 191, 148, 106, 178, 103, 0, 0, - 0, 0, 0, 120, 0, 122, 0, 0, 159, 131, + 0, 0, 0, 0, 139, 0, 82, 89, 121, 192, + 148, 106, 179, 103, 0, 0, 0, 0, 0, 120, + 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 208, 0, 845, - 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, + 0, 0, 0, 270, 0, 808, 0, 0, 0, 0, + 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, + 0, 211, 0, 0, 0, 0, 146, 0, 163, 111, + 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, + 0, 99, 0, 153, 141, 176, 0, 142, 152, 123, + 168, 147, 175, 183, 184, 165, 182, 191, 84, 164, + 174, 97, 156, 86, 172, 162, 129, 115, 116, 85, + 0, 150, 102, 107, 101, 138, 169, 170, 100, 194, + 91, 181, 88, 92, 180, 136, 167, 173, 130, 127, + 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, + 113, 133, 132, 134, 0, 0, 0, 161, 178, 195, + 94, 0, 157, 166, 185, 186, 187, 188, 189, 190, + 0, 0, 95, 108, 104, 143, 135, 93, 114, 158, + 117, 124, 149, 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 109, 0, 0, 0, 210, 0, 0, 0, 0, - 146, 0, 162, 111, 119, 83, 90, 0, 110, 137, - 151, 155, 0, 0, 0, 99, 0, 153, 141, 175, - 0, 142, 152, 123, 167, 147, 174, 182, 183, 164, - 181, 190, 84, 163, 173, 97, 156, 86, 171, 161, - 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, - 168, 169, 100, 193, 91, 180, 88, 92, 179, 136, - 166, 172, 130, 127, 87, 170, 128, 126, 118, 105, - 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, - 0, 160, 177, 194, 94, 0, 165, 184, 185, 186, - 187, 188, 189, 0, 0, 95, 108, 104, 143, 135, - 93, 114, 157, 117, 124, 149, 192, 140, 154, 98, - 176, 158, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 139, - 0, 82, 89, 121, 191, 148, 106, 178, 103, 0, - 0, 0, 0, 0, 120, 0, 122, 0, 0, 159, - 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 268, 0, - 805, 0, 0, 0, 0, 0, 0, 96, 0, 0, + 0, 0, 0, 0, 0, 0, 139, 0, 82, 89, + 121, 192, 148, 106, 179, 103, 0, 0, 0, 0, + 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 270, 0, 804, 0, 0, + 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 109, 0, 0, 0, 210, 0, 0, 0, - 0, 146, 0, 162, 111, 119, 83, 90, 0, 110, - 137, 151, 155, 0, 0, 0, 99, 0, 153, 141, - 175, 0, 142, 152, 123, 167, 147, 174, 182, 183, - 164, 181, 190, 84, 163, 173, 97, 156, 86, 171, - 161, 129, 115, 116, 85, 0, 150, 102, 107, 101, - 138, 168, 169, 100, 193, 91, 180, 88, 92, 179, - 136, 166, 172, 130, 127, 87, 170, 128, 126, 118, - 105, 112, 144, 125, 145, 113, 133, 132, 134, 0, - 0, 0, 160, 177, 194, 94, 0, 165, 184, 185, - 186, 187, 188, 189, 0, 0, 95, 108, 104, 143, - 135, 93, 114, 157, 117, 124, 149, 192, 140, 154, - 98, 176, 158, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, + 0, 0, 0, 211, 0, 0, 0, 0, 146, 0, + 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, + 0, 0, 0, 99, 0, 153, 141, 176, 0, 142, + 152, 123, 168, 147, 175, 183, 184, 165, 182, 191, + 84, 164, 174, 97, 156, 86, 172, 162, 129, 115, + 116, 85, 0, 150, 102, 107, 101, 138, 169, 170, + 100, 194, 91, 181, 88, 92, 180, 136, 167, 173, + 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, + 125, 145, 113, 133, 132, 134, 0, 0, 0, 161, + 178, 195, 94, 0, 157, 166, 185, 186, 187, 188, + 189, 190, 0, 0, 95, 108, 104, 143, 135, 93, + 114, 158, 117, 124, 149, 193, 140, 154, 98, 177, + 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, + 82, 89, 121, 192, 148, 106, 179, 103, 0, 0, + 0, 0, 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 139, 0, 82, 89, 121, 191, 148, 106, 178, 103, - 0, 0, 0, 0, 0, 120, 0, 122, 0, 0, - 159, 131, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 268, - 0, 801, 0, 0, 0, 0, 0, 0, 96, 0, + 0, 0, 0, 0, 0, 0, 0, 80, 0, 525, + 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 109, 0, 0, 0, 210, 0, 0, - 0, 0, 146, 0, 162, 111, 119, 83, 90, 0, - 110, 137, 151, 155, 0, 0, 0, 99, 0, 153, - 141, 175, 0, 142, 152, 123, 167, 147, 174, 182, - 183, 164, 181, 190, 84, 163, 173, 97, 156, 86, - 171, 161, 129, 115, 116, 85, 0, 150, 102, 107, - 101, 138, 168, 169, 100, 193, 91, 180, 88, 92, - 179, 136, 166, 172, 130, 127, 87, 170, 128, 126, - 118, 105, 112, 144, 125, 145, 113, 133, 132, 134, - 0, 0, 0, 160, 177, 194, 94, 0, 165, 184, - 185, 186, 187, 188, 189, 0, 0, 95, 108, 104, - 143, 135, 93, 114, 157, 117, 124, 149, 192, 140, - 154, 98, 176, 158, 0, 0, 0, 0, 0, 0, + 0, 109, 0, 0, 0, 211, 0, 0, 0, 0, + 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, + 151, 155, 0, 0, 0, 99, 0, 153, 141, 176, + 0, 142, 152, 123, 168, 147, 175, 183, 184, 165, + 182, 191, 84, 164, 174, 97, 156, 86, 172, 162, + 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, + 169, 170, 100, 194, 91, 181, 88, 92, 180, 136, + 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, + 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, + 0, 161, 178, 195, 94, 0, 157, 166, 185, 186, + 187, 188, 189, 190, 0, 0, 95, 108, 104, 143, + 135, 93, 114, 158, 117, 124, 149, 193, 140, 154, + 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 139, 0, 82, 89, 121, 191, 148, 106, 178, + 0, 139, 82, 89, 121, 192, 148, 106, 179, 607, 103, 0, 0, 0, 0, 0, 120, 0, 122, 0, - 0, 159, 131, 0, 0, 0, 0, 0, 0, 0, + 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 80, 0, 522, 0, 0, 0, 0, 0, 0, 96, + 209, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 109, 0, 0, 0, 210, 0, - 0, 0, 0, 146, 0, 162, 111, 119, 83, 90, + 0, 0, 0, 0, 109, 0, 0, 0, 211, 0, + 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, - 153, 141, 175, 0, 142, 152, 123, 167, 147, 174, - 182, 183, 164, 181, 190, 84, 163, 173, 97, 156, - 86, 171, 161, 129, 115, 116, 85, 0, 150, 102, - 107, 101, 138, 168, 169, 100, 193, 91, 180, 88, - 92, 179, 136, 166, 172, 130, 127, 87, 170, 128, + 153, 141, 176, 0, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 174, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 92, 180, 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, - 134, 0, 0, 0, 160, 177, 194, 94, 0, 165, - 184, 185, 186, 187, 188, 189, 0, 0, 95, 108, - 104, 143, 135, 93, 114, 157, 117, 124, 149, 192, - 140, 154, 98, 176, 158, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 139, 82, 89, 121, 191, 148, 106, - 178, 604, 103, 0, 0, 0, 0, 0, 120, 0, - 122, 0, 0, 159, 131, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 208, 0, 0, 0, 0, 0, 0, 0, + 134, 0, 0, 0, 161, 178, 195, 94, 0, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 135, 93, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 329, 0, 0, 0, + 0, 0, 0, 139, 0, 82, 89, 121, 192, 148, + 106, 179, 103, 0, 0, 0, 0, 0, 120, 0, + 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 209, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, 0, - 210, 0, 0, 0, 0, 146, 0, 162, 111, 119, + 211, 0, 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, 0, - 99, 0, 153, 141, 175, 0, 142, 152, 123, 167, - 147, 174, 182, 183, 164, 181, 190, 84, 163, 173, - 97, 156, 86, 171, 161, 129, 115, 116, 85, 0, - 150, 102, 107, 101, 138, 168, 169, 100, 193, 91, - 180, 88, 92, 179, 136, 166, 172, 130, 127, 87, - 170, 128, 126, 118, 105, 112, 144, 125, 145, 113, - 133, 132, 134, 0, 0, 0, 160, 177, 194, 94, - 0, 165, 184, 185, 186, 187, 188, 189, 0, 0, - 95, 108, 104, 143, 135, 93, 114, 157, 117, 124, - 149, 192, 140, 154, 98, 176, 158, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 327, 0, 0, - 0, 0, 0, 0, 139, 0, 82, 89, 121, 191, - 148, 106, 178, 103, 0, 0, 0, 0, 0, 120, - 0, 122, 0, 0, 159, 131, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 208, 0, 0, 0, 0, 0, 0, - 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, - 0, 210, 0, 0, 0, 0, 146, 0, 162, 111, - 119, 83, 90, 0, 110, 137, 151, 155, 0, 0, - 0, 99, 0, 153, 141, 175, 0, 142, 152, 123, - 167, 147, 174, 182, 183, 164, 181, 190, 84, 163, - 173, 97, 156, 86, 171, 161, 129, 115, 116, 85, - 0, 150, 102, 107, 101, 138, 168, 169, 100, 193, - 91, 180, 88, 92, 179, 136, 166, 172, 130, 127, - 87, 170, 128, 126, 118, 105, 112, 144, 125, 145, - 113, 133, 132, 134, 0, 0, 0, 160, 177, 194, - 94, 0, 165, 184, 185, 186, 187, 188, 189, 0, - 0, 95, 108, 104, 143, 135, 93, 114, 157, 117, - 124, 149, 192, 140, 154, 98, 176, 158, 0, 0, + 99, 0, 153, 141, 176, 0, 142, 152, 123, 168, + 147, 175, 183, 184, 165, 182, 191, 84, 164, 174, + 97, 156, 86, 172, 162, 129, 115, 116, 85, 0, + 150, 102, 107, 101, 138, 169, 170, 100, 194, 91, + 181, 88, 92, 180, 136, 167, 173, 130, 127, 87, + 171, 128, 126, 118, 105, 112, 144, 125, 145, 113, + 133, 132, 134, 0, 0, 0, 161, 178, 195, 94, + 0, 157, 166, 185, 186, 187, 188, 189, 190, 0, + 0, 95, 108, 104, 143, 135, 93, 114, 158, 117, + 124, 149, 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, 82, 89, 121, - 191, 148, 106, 178, 103, 0, 0, 0, 0, 0, - 120, 0, 122, 0, 0, 159, 131, 0, 0, 0, + 192, 148, 106, 179, 103, 0, 0, 0, 0, 0, + 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 208, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 209, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, 0, - 205, 0, 210, 0, 0, 0, 0, 146, 0, 162, + 206, 0, 211, 0, 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, 155, 0, - 0, 0, 99, 0, 153, 141, 175, 0, 142, 152, - 123, 167, 147, 174, 182, 183, 164, 181, 190, 84, - 163, 173, 97, 156, 86, 171, 161, 129, 115, 116, - 85, 0, 150, 102, 107, 101, 138, 168, 169, 100, - 193, 91, 180, 88, 92, 179, 136, 166, 172, 130, - 127, 87, 170, 128, 126, 118, 105, 112, 144, 125, - 145, 113, 133, 132, 134, 0, 0, 0, 160, 177, - 194, 94, 0, 165, 184, 185, 186, 187, 188, 189, - 0, 0, 95, 108, 104, 143, 135, 93, 114, 157, - 117, 124, 149, 192, 140, 154, 98, 176, 158, 0, + 0, 0, 99, 0, 153, 141, 176, 0, 142, 152, + 123, 168, 147, 175, 183, 184, 165, 182, 191, 84, + 164, 174, 97, 156, 86, 172, 162, 129, 115, 116, + 85, 0, 150, 102, 107, 101, 138, 169, 170, 100, + 194, 91, 181, 88, 92, 180, 136, 167, 173, 130, + 127, 87, 171, 128, 126, 118, 105, 112, 144, 125, + 145, 113, 133, 132, 134, 0, 0, 0, 161, 178, + 195, 94, 0, 157, 166, 185, 186, 187, 188, 189, + 190, 0, 0, 95, 108, 104, 143, 135, 93, 114, + 158, 117, 124, 149, 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 139, 0, 82, 89, - 121, 191, 148, 106, 178, 103, 0, 0, 0, 0, - 0, 120, 0, 122, 0, 0, 159, 131, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 139, 0, 82, + 89, 121, 192, 148, 106, 179, 103, 0, 0, 0, + 0, 0, 120, 0, 122, 0, 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 80, 0, 0, 0, 0, - 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 80, 0, 0, 0, + 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 109, - 0, 0, 0, 210, 0, 0, 0, 0, 146, 0, - 162, 111, 119, 83, 90, 0, 110, 137, 151, 155, - 0, 0, 0, 99, 0, 153, 141, 175, 0, 142, - 152, 123, 167, 147, 174, 182, 183, 164, 181, 190, - 84, 163, 173, 97, 156, 86, 171, 161, 129, 115, - 116, 85, 0, 150, 102, 107, 101, 138, 168, 169, - 100, 193, 91, 180, 88, 92, 179, 136, 166, 172, - 130, 127, 87, 170, 128, 126, 118, 105, 112, 144, - 125, 145, 113, 133, 132, 134, 0, 0, 0, 160, - 177, 194, 94, 0, 165, 184, 185, 186, 187, 188, - 189, 0, 0, 95, 108, 104, 143, 135, 93, 114, - 157, 117, 124, 149, 192, 140, 154, 98, 176, 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 139, 0, 82, - 89, 121, 191, 148, 106, 178, 103, 0, 0, 0, - 0, 0, 120, 0, 122, 0, 0, 159, 131, 0, + 109, 0, 0, 0, 211, 0, 0, 0, 0, 146, + 0, 163, 111, 119, 83, 90, 0, 110, 137, 151, + 155, 0, 0, 0, 99, 0, 153, 141, 176, 0, + 142, 152, 123, 168, 147, 175, 183, 184, 165, 182, + 191, 84, 164, 174, 97, 156, 86, 172, 162, 129, + 115, 116, 85, 0, 150, 102, 107, 101, 138, 169, + 170, 100, 194, 91, 181, 88, 92, 180, 136, 167, + 173, 130, 127, 87, 171, 128, 126, 118, 105, 112, + 144, 125, 145, 113, 133, 132, 134, 0, 0, 0, + 161, 178, 195, 94, 0, 157, 166, 185, 186, 187, + 188, 189, 190, 0, 0, 95, 108, 104, 143, 135, + 93, 114, 158, 117, 124, 149, 193, 140, 154, 98, + 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 139, + 0, 82, 89, 121, 192, 148, 106, 179, 103, 0, + 0, 0, 0, 0, 120, 0, 122, 0, 0, 160, + 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 209, 0, + 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 208, 0, 0, 0, - 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 109, 0, 0, 0, 211, 0, 0, 0, + 0, 146, 0, 163, 111, 119, 83, 90, 0, 110, + 137, 151, 155, 0, 0, 0, 99, 0, 153, 141, + 176, 0, 142, 152, 123, 168, 147, 175, 183, 184, + 165, 182, 191, 84, 164, 174, 97, 156, 86, 172, + 162, 129, 115, 116, 85, 0, 150, 102, 107, 101, + 138, 169, 170, 100, 194, 91, 181, 88, 92, 180, + 136, 167, 173, 130, 127, 87, 171, 128, 126, 118, + 105, 112, 144, 125, 145, 113, 133, 132, 134, 0, + 0, 0, 161, 178, 195, 94, 0, 157, 166, 185, + 186, 187, 188, 189, 190, 0, 0, 95, 108, 104, + 143, 135, 93, 114, 158, 117, 124, 149, 193, 140, + 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 109, 0, 0, 0, 210, 0, 0, 0, 0, 146, - 0, 162, 111, 119, 83, 90, 0, 110, 137, 151, - 155, 0, 0, 0, 99, 0, 153, 141, 175, 0, - 142, 152, 123, 167, 147, 174, 182, 183, 164, 181, - 190, 84, 163, 173, 97, 156, 86, 171, 161, 129, - 115, 116, 85, 0, 150, 102, 107, 101, 138, 168, - 169, 100, 193, 91, 180, 88, 92, 179, 136, 166, - 172, 130, 127, 87, 170, 128, 126, 118, 105, 112, - 144, 125, 145, 113, 133, 132, 134, 0, 0, 0, - 160, 177, 194, 94, 0, 165, 184, 185, 186, 187, - 188, 189, 0, 0, 95, 108, 104, 143, 135, 93, - 114, 157, 117, 124, 149, 192, 140, 154, 98, 176, - 158, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 139, 0, - 82, 89, 121, 191, 148, 106, 178, 103, 0, 0, - 0, 0, 0, 120, 0, 122, 0, 0, 159, 131, + 0, 139, 0, 82, 89, 121, 192, 148, 106, 179, + 103, 0, 0, 0, 0, 0, 120, 0, 122, 0, + 0, 160, 131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 268, 0, 0, - 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, + 270, 0, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 109, 0, 0, 0, 210, 0, 0, 0, 0, - 146, 0, 162, 111, 119, 83, 90, 0, 110, 137, - 151, 155, 0, 0, 0, 99, 0, 153, 141, 175, - 0, 142, 152, 123, 167, 147, 174, 182, 183, 164, - 181, 190, 84, 163, 173, 97, 156, 86, 171, 161, - 129, 115, 116, 85, 0, 150, 102, 107, 101, 138, - 168, 169, 100, 193, 91, 180, 88, 92, 179, 136, - 166, 172, 130, 127, 87, 170, 128, 126, 118, 105, - 112, 144, 125, 145, 113, 133, 132, 134, 0, 0, - 0, 160, 177, 194, 94, 0, 165, 184, 185, 186, - 187, 188, 189, 0, 0, 95, 108, 104, 143, 135, - 93, 114, 157, 117, 124, 149, 192, 140, 154, 98, - 176, 158, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 109, 0, 0, 0, 211, 0, + 0, 0, 0, 146, 0, 163, 111, 119, 83, 90, + 0, 110, 137, 151, 155, 0, 0, 0, 99, 0, + 153, 141, 176, 0, 142, 152, 123, 168, 147, 175, + 183, 184, 165, 182, 191, 84, 164, 174, 97, 156, + 86, 172, 162, 129, 115, 116, 85, 0, 150, 102, + 107, 101, 138, 169, 170, 100, 194, 91, 181, 88, + 92, 180, 136, 167, 173, 130, 127, 87, 171, 128, + 126, 118, 105, 112, 144, 125, 145, 113, 133, 132, + 134, 0, 0, 0, 161, 178, 195, 94, 0, 157, + 166, 185, 186, 187, 188, 189, 190, 0, 0, 95, + 108, 104, 143, 135, 93, 114, 158, 117, 124, 149, + 193, 140, 154, 98, 177, 159, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 82, 89, 121, 191, 148, 106, 178, + 0, 0, 0, 0, 0, 82, 89, 121, 192, 148, + 106, 179, } var yyPact = [...]int{ - 2229, -1000, -174, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1674, -1000, -192, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 959, 1019, -1000, -1000, -1000, -1000, -1000, -1000, - 246, 8304, 34, 118, 16, 11697, 113, 1430, 12179, -1000, - 6, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -54, -62, - -1000, 791, -1000, -1000, -1000, -1000, -1000, 951, 962, 828, - 942, 882, -1000, 6312, 69, 69, 11456, 5312, -1000, -1000, - 318, 12179, 102, 12179, -138, 67, 67, 67, -1000, -1000, + -1000, -1000, 847, 897, -1000, -1000, -1000, -1000, -1000, -1000, + 333, 8610, 35, 116, -6, 12017, 113, 212, 12501, -1000, + 7, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -82, -88, + -1000, 630, -1000, -1000, -1000, -1000, -1000, 840, 843, 703, + 835, 746, -1000, 6610, 81, 81, 11775, 5857, -1000, -1000, + 265, 12501, 108, 12501, -148, 74, 74, 74, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -2006,23 +2041,22 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 112, 12179, 225, -1000, 12179, - 66, 568, 66, 66, 66, 12179, -1000, 149, -1000, -1000, - -1000, 12179, 565, 906, 3208, 58, 3208, -1000, 3208, 3208, - -1000, 3208, 13, 3208, -68, 972, -1000, -1000, -1000, -1000, - 2, -1000, 3208, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 450, 911, 6813, 6813, - 959, -1000, 791, -1000, -1000, -1000, 905, -1000, -1000, 345, - 981, -1000, 8063, 146, -1000, 6813, 1832, 479, -1000, -1000, - 479, -1000, -1000, 135, -1000, -1000, 7563, 7563, 7563, 7563, - 7563, 7563, 7563, 7563, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 479, -1000, - 6563, 479, 479, 479, 479, 479, 479, 479, 479, 6813, - 479, 479, 479, 479, 479, 479, 479, 479, 479, 479, - 479, 479, 479, 479, 479, 11215, 10009, 12179, 657, -1000, - 722, 5049, -119, -1000, -1000, -1000, 248, 9768, -1000, -1000, - -1000, 904, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 111, 12501, 237, -1000, + 12501, 64, 547, 64, 64, 64, 12501, -1000, 160, -1000, + -1000, -1000, 12501, 536, 797, 3745, 49, 3745, 3745, -1000, + 3745, 3745, -1000, 3745, 16, 3745, -58, 857, -1000, -1000, + -1000, -1000, -35, -1000, 3745, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 467, 807, + 7113, 7113, 847, -1000, 630, -1000, -1000, -1000, 789, -1000, + -1000, 336, 864, -1000, 8368, 155, -1000, 7113, 2080, 599, + -1000, -1000, 599, -1000, -1000, 149, -1000, -1000, 7866, 7866, + 7866, 7866, 7866, 7866, 7866, 7866, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 599, -1000, 6862, 599, 599, 599, 599, 599, 599, 599, + 599, 7113, 599, 599, 599, 599, 599, 599, 599, 599, + 599, 599, 599, 599, 599, 599, 599, 11533, 10322, 12501, + 619, -1000, 647, 5593, -122, -1000, -1000, -1000, 251, 10080, + -1000, -1000, -1000, 792, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -2030,131 +2064,132 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 638, 12179, - -1000, 1945, -1000, 562, 3208, 80, 545, 310, 540, 12179, - 12179, 3208, 21, 51, 105, 12179, 744, 73, 12179, 936, - 812, 12179, 537, 535, -1000, 4786, -1000, 3208, 3208, -1000, - -1000, -1000, 3208, 3208, 3208, 12179, 3208, 3208, -1000, -1000, - -1000, -1000, 3208, 3208, -1000, 980, 281, -1000, -1000, -1000, - -1000, 6813, -1000, 809, -1000, -1000, -1000, -1000, -1000, -1000, - 1012, 158, 452, 143, 743, -1000, 382, 951, 450, 882, - 9527, 800, -1000, -1000, 12179, -1000, 6813, 6813, 343, -1000, - 10973, -1000, -1000, 3734, 210, 7563, 385, 306, 7563, 7563, - 7563, 7563, 7563, 7563, 7563, 7563, 7563, 7563, 7563, 7563, - 7563, 7563, 7563, 451, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 510, -1000, 791, 978, 978, 163, 163, 163, - 163, 163, 163, 163, 7813, 5812, 450, 626, 371, 6563, - 6312, 6312, 6813, 6813, 10732, 10491, 6312, 944, 265, 371, - 12420, -1000, -1000, 7313, -1000, -1000, -1000, -1000, -1000, 450, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 11938, 11938, 6312, - 6312, 6312, 6312, 36, 12179, -1000, 647, 810, -1000, -1000, - -1000, 938, 9036, 9286, 36, 675, 10009, 12179, -1000, -1000, - 4523, 722, -119, 686, -1000, -84, -101, 5562, 159, -1000, - -1000, -1000, -1000, 2945, 357, 596, 393, -48, -1000, -1000, - -1000, 749, -1000, 749, 749, 749, 749, -20, -20, -20, - -20, -1000, -1000, -1000, -1000, -1000, 779, 778, -1000, 749, - 749, 749, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 777, 777, 777, 752, 752, 785, -1000, 12179, 3208, 935, - 3208, -1000, 122, -1000, 11938, 11938, 12179, 12179, 127, 12179, - 12179, 718, -1000, 12179, 3208, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 12179, 296, 12179, 12179, 371, 12179, -1000, 863, 6813, 6813, - 4260, 6813, -1000, -1000, -1000, 911, -1000, 944, 958, -1000, - 879, 876, 6312, -1000, -1000, 210, 250, -1000, -1000, 431, - -1000, -1000, -1000, -1000, 141, 479, -1000, 1903, -1000, -1000, - -1000, -1000, 385, 7563, 7563, 7563, 334, 1903, 1867, 1357, - 186, 163, 652, 652, 230, 230, 230, 230, 230, 572, - 572, -1000, -1000, -1000, 450, -1000, -1000, -1000, 450, 6312, - 711, -1000, -1000, 6813, -1000, 450, 601, 601, 531, 420, - 747, 738, -1000, 140, 721, 714, 601, 6312, 303, -1000, - 6813, 450, -1000, 900, 693, 687, 601, 450, 601, 601, - 703, 479, -1000, 12420, 10009, 10009, 10009, 10009, 10009, -1000, - 848, 846, -1000, 844, 836, 864, 12179, -1000, 604, 9036, - 152, 479, -1000, 10250, -1000, -1000, 971, 10009, 635, -1000, - -1000, 686, -119, -86, -1000, -1000, -1000, -1000, 371, -1000, - 477, 685, 2681, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 766, 478, -1000, 924, 193, 188, 469, 923, -1000, -1000, - -1000, 915, -1000, 324, -50, -1000, -1000, 412, -20, -20, - -1000, -1000, 159, 899, 159, 159, 159, 443, 443, -1000, - -1000, -1000, -1000, 398, -1000, -1000, -1000, 397, -1000, 805, - 11938, 3208, -1000, -1000, -1000, -1000, 173, 173, 209, -1000, + 544, 12501, -1000, 1469, -1000, 518, 3745, 87, 510, 289, + 508, 12501, 12501, 3745, 24, 57, 110, 12501, 651, 83, + 12501, 823, 714, 12501, 506, 480, -1000, 5329, -1000, 3745, + 3745, -1000, -1000, -1000, 3745, 3745, 3745, 12501, 3745, 3745, + -1000, -1000, -1000, -1000, -1000, 3745, 3745, -1000, 863, 312, + -1000, -1000, -1000, -1000, 7113, -1000, 713, -1000, -1000, -1000, + -1000, -1000, -1000, 888, 199, 447, 152, 650, -1000, 276, + 840, 467, 746, 9838, 724, -1000, -1000, 12501, -1000, 7113, + 7113, 415, -1000, 11290, -1000, -1000, 4273, 203, 7866, 370, + 270, 7866, 7866, 7866, 7866, 7866, 7866, 7866, 7866, 7866, + 7866, 7866, 7866, 7866, 7866, 7866, 389, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 472, -1000, 630, 511, 511, + 177, 177, 177, 177, 177, 177, 177, 8117, 6108, 467, + 542, 349, 6862, 6610, 6610, 7113, 7113, 11048, 10806, 6610, + 829, 278, 349, 12743, -1000, -1000, 7615, -1000, -1000, -1000, + -1000, -1000, 467, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 12259, 12259, 6610, 6610, 6610, 6610, 38, 12501, -1000, 584, + 689, -1000, -1000, -1000, 827, 9345, 9596, 38, 558, 10322, + 12501, -1000, -1000, 5065, 647, -122, 618, -1000, -114, -105, + 2940, 167, -1000, -1000, -1000, -1000, 3481, 213, 551, 381, + -74, -1000, -1000, -1000, 667, -1000, 667, 667, 667, 667, + -24, -24, -24, -24, -1000, -1000, -1000, -1000, -1000, 693, + 692, -1000, 667, 667, 667, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 35, 782, -1000, -1000, -1000, 20, 19, 72, -1000, 3208, - -1000, 281, -1000, 437, 6813, -1000, -1000, -1000, 858, 371, - 371, 139, -1000, -1000, 12179, -1000, -1000, -1000, -1000, 705, - -1000, -1000, -1000, 3471, 6312, -1000, 334, 1903, 1581, -1000, - 7563, 7563, -1000, -1000, 601, 6312, 371, -1000, -1000, -1000, - 45, 451, 45, 7563, 7563, 7563, 7563, 4260, 7563, 7563, - 7563, 7563, -151, 628, 260, -1000, 6813, 374, -1000, -1000, - 7563, 7563, -1000, -1000, -1000, -1000, 798, 12420, 479, -1000, - 8795, 11938, 717, -1000, 247, 810, 771, 797, 1647, -1000, - -1000, -1000, -1000, 835, -1000, 823, -1000, -1000, -1000, -1000, - -1000, 87, 86, 83, 11938, -1000, 959, 6813, 635, -1000, - -1000, -1000, -93, -107, -1000, -1000, -1000, 2945, -1000, 2945, - 11938, 47, -1000, 469, 469, -1000, -1000, -1000, 753, 795, - 7563, -1000, -1000, -1000, 556, 159, 159, -1000, 211, -1000, - -1000, -1000, 595, -1000, 580, 684, 575, 12179, -1000, -1000, + -1000, -1000, -1000, 691, 691, 691, 669, 669, 700, -1000, + 12501, 3745, 812, 3745, -1000, 79, -1000, 12259, 12259, 12501, + 12501, 127, 12501, 12501, 641, -1000, 12501, 3745, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, 12179, -1000, -1000, -1000, -1000, -1000, - 11938, -161, 460, 11938, 11938, 12179, -1000, 296, -1000, 371, - -1000, 3997, -1000, 971, 10009, -1000, -1000, 450, -1000, 7563, - 1903, 1903, -1000, -1000, 450, 749, 749, -1000, 749, 752, - -1000, 749, -3, 749, -4, 450, 450, 715, 2203, 688, - 2148, -1000, 476, 1939, 449, 1418, 479, -146, -1000, 371, - 6813, 1396, 1373, -1000, 926, 656, 672, -1000, -1000, 6062, - 450, 573, 134, 555, -1000, 959, 12420, 6813, -1000, -1000, - 6813, 751, -1000, 6813, -1000, -1000, -1000, 479, 479, 479, - 555, 951, 371, -1000, -1000, -1000, -1000, 2681, -1000, 553, - -1000, 749, -1000, -1000, -1000, 11938, -44, 1009, 1903, -1000, - -1000, -1000, -1000, -1000, -20, 432, -20, 388, -1000, 358, - 3208, -1000, -1000, -1000, -1000, 928, -1000, 3997, -1000, -1000, - 746, -1000, -1000, -1000, 967, 683, -1000, 1903, -1000, -1000, - 109, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 7563, 7563, -1000, 7563, 7563, -1000, 7563, 7563, -1000, 7563, - 7563, 7563, 450, 429, 371, 7563, 7563, 921, -1000, 479, - -1000, -1000, 713, 11938, 11938, -1000, 11938, 951, -1000, 371, - 371, 11938, 371, 11938, 11938, 11938, 8554, -1000, 138, 11938, - -1000, 551, -1000, 165, -1000, -157, 159, -1000, 159, 533, - 508, -1000, 479, 680, -1000, 243, 11938, 965, 954, -1000, - -1000, 900, 900, 900, 900, 900, 900, 900, 900, 27, - -1000, -1000, 900, 900, 1006, -1000, 479, -1000, 791, 120, - -1000, -1000, -1000, 532, 507, 507, 507, 152, 138, -1000, - 455, 239, 426, -1000, 49, 11938, 335, 918, -1000, 917, - -1000, -1000, -1000, -1000, -1000, 33, 3997, 2945, 504, -1000, - 6813, 6813, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 450, 60, -166, -1000, -1000, 12420, 672, 450, 11938, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 355, -1000, -1000, 12179, - -1000, -1000, 423, -1000, -1000, 502, -1000, 11938, -1000, -1000, - 782, 371, 659, -1000, 856, -155, -170, 648, -1000, -1000, - -1000, 653, -1000, -1000, 33, 871, -161, -1000, 853, -1000, - 11938, -1000, 29, -1000, -164, 499, 26, -167, 794, 479, - -171, 790, -1000, 979, 7063, -1000, -1000, 1001, 179, 179, - 900, 450, -1000, -1000, -1000, 57, 326, -1000, -1000, -1000, - -1000, -1000, -1000, + -1000, -1000, -1000, 12501, 298, 12501, 12501, 349, 12501, -1000, + 770, 7113, 7113, 4801, 7113, -1000, -1000, -1000, 807, -1000, + 829, 841, -1000, 783, 780, 6610, -1000, -1000, 203, 285, + -1000, -1000, 425, -1000, -1000, -1000, -1000, 148, 599, -1000, + 2095, -1000, -1000, -1000, -1000, 370, 7866, 7866, 7866, 702, + 2095, 2006, 457, 1487, 177, 176, 176, 214, 214, 214, + 214, 214, 282, 282, -1000, -1000, -1000, 467, -1000, -1000, + -1000, 467, 6610, 635, -1000, -1000, 7113, -1000, 467, 534, + 534, 384, 378, 665, 664, -1000, 146, 638, 624, 534, + 6610, 297, -1000, 7113, 467, -1000, 1661, 634, 622, 534, + 467, 534, 534, 682, 599, -1000, 12743, 10322, 10322, 10322, + 10322, 10322, -1000, 739, 738, -1000, 736, 726, 779, 12501, + -1000, 540, 9345, 178, 599, -1000, 10564, -1000, -1000, 856, + 10322, 603, -1000, -1000, 618, -122, -108, -1000, -1000, -1000, + -1000, 349, -1000, 440, 613, 3217, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 676, 468, -1000, 814, 192, 229, 456, + 813, -1000, -1000, -1000, 804, -1000, 296, -76, -1000, -1000, + 403, -24, -24, -1000, -1000, 167, 788, 167, 167, 167, + 448, 448, -1000, -1000, -1000, -1000, 400, -1000, -1000, -1000, + 399, -1000, 712, 12259, 3745, -1000, -1000, -1000, -1000, 166, + 166, 215, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 37, 697, -1000, -1000, -1000, 22, 20, + 80, -1000, 3745, -1000, 312, -1000, 446, 7113, -1000, -1000, + -1000, 754, 349, 349, 141, -1000, -1000, 12501, -1000, -1000, + -1000, -1000, 623, -1000, -1000, -1000, 4009, 6610, -1000, 702, + 2095, 1939, -1000, 7866, 7866, -1000, -1000, 534, 6610, 349, + -1000, -1000, -1000, 70, 389, 70, 7866, 7866, 7866, 7866, + 4801, 7866, 7866, 7866, 7866, -159, 597, 263, -1000, 7113, + 409, -1000, -1000, 7866, 7866, -1000, -1000, -1000, -1000, 711, + 12743, 599, -1000, 9103, 12259, 607, -1000, 249, 689, 688, + 710, 598, -1000, -1000, -1000, -1000, 729, -1000, 728, -1000, + -1000, -1000, -1000, -1000, 103, 101, 93, 12259, -1000, 847, + 7113, 603, -1000, -1000, -1000, -129, -131, -1000, -1000, -1000, + 3481, -1000, 3481, 12259, 53, -1000, 456, 456, -1000, -1000, + -1000, 674, 704, 7866, -1000, -1000, -1000, 549, 167, 167, + -1000, 230, -1000, -1000, -1000, 532, -1000, 527, 573, 525, + 12501, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 12501, -1000, -1000, + -1000, -1000, -1000, 12259, -169, 451, 12259, 12259, 12501, -1000, + 298, -1000, 349, -1000, 4537, -1000, 856, 10322, -1000, -1000, + 467, -1000, 7866, 2095, 2095, -1000, -1000, 467, 667, 667, + -1000, 667, 669, -1000, 667, -2, 667, -7, 467, 467, + 1447, 1897, 1377, 1852, -1000, 1353, 1760, 1157, 1223, 599, + -156, -1000, 349, 7113, 1503, 1428, -1000, 817, 554, 567, + -1000, -1000, 6359, 467, 493, 130, 491, -1000, 847, 12743, + 7113, -1000, -1000, 7113, 668, -1000, 7113, -1000, -1000, -1000, + 599, 599, 599, 491, 840, 349, -1000, -1000, -1000, -1000, + 3217, -1000, 489, -1000, 667, -1000, -1000, -1000, 12259, -65, + 886, 2095, -1000, -1000, -1000, -1000, -1000, -24, 445, -24, + 365, -1000, 360, 3745, -1000, -1000, -1000, -1000, 819, -1000, + 4537, -1000, -1000, 660, -1000, -1000, -1000, 852, 572, -1000, + 2095, -1000, -1000, 107, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 7866, 7866, -1000, 7866, 7866, -1000, 7866, + 7866, -1000, 7866, 7866, 7866, 467, 437, 349, 7866, 7866, + 811, -1000, 599, -1000, -1000, 661, 12259, 12259, -1000, 12259, + 840, -1000, 349, 349, 12259, 349, 12259, 12259, 12259, 8861, + -1000, 137, 12259, -1000, 484, -1000, 183, -1000, -160, 167, + -1000, 167, 494, 485, -1000, 599, 568, -1000, 232, 12259, + 849, 842, -1000, -1000, 1661, 1661, 1661, 1661, 1661, 1661, + 1661, 1661, 54, -1000, -1000, 1661, 1661, 871, -1000, 599, + -1000, 630, 128, -1000, -1000, -1000, 479, 476, 476, 476, + 178, 137, -1000, 417, 220, 413, -1000, 46, 12259, 332, + 809, -1000, 808, -1000, -1000, -1000, -1000, -1000, 36, 4537, + 3481, 466, -1000, 7113, 7113, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 467, 52, -173, -1000, -1000, 12743, 567, + 467, 12259, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 357, + -1000, -1000, 12501, -1000, -1000, 407, -1000, -1000, 460, -1000, + 12259, -1000, -1000, 697, 349, 563, -1000, 743, -166, -187, + 555, -1000, -1000, -1000, 659, -1000, -1000, 36, 778, -169, + -1000, 715, -1000, 12259, -1000, 32, -1000, -170, 455, 19, + -174, 690, 599, -189, 666, -1000, 862, 7364, -1000, -1000, + 870, 170, 170, 1661, 467, -1000, -1000, -1000, 58, 361, + -1000, -1000, -1000, -1000, -1000, -1000, } var yyPgo = [...]int{ - 0, 1255, 54, 479, 1254, 1253, 1251, 1249, 1248, 1246, - 1243, 1241, 1235, 1234, 1230, 1228, 1227, 1222, 1220, 1219, - 1214, 1213, 1212, 1210, 1209, 1207, 117, 1206, 1202, 1200, - 75, 1199, 83, 1198, 1197, 38, 61, 41, 40, 1285, - 1195, 50, 56, 73, 1191, 28, 1185, 1184, 81, 1183, - 53, 1164, 1161, 1569, 1159, 1158, 12, 30, 1157, 1155, - 1153, 1152, 70, 637, 1150, 1149, 14, 1148, 1147, 86, - 1146, 58, 8, 13, 11, 24, 1145, 314, 15, 1138, - 59, 1137, 1136, 1135, 1134, 20, 1133, 62, 1131, 19, - 60, 1129, 7, 65, 29, 25, 5, 84, 69, 1128, - 23, 66, 51, 1126, 1124, 516, 1123, 1122, 48, 1121, - 1120, 22, 175, 387, 1118, 1117, 1116, 1114, 37, 0, - 498, 44, 80, 1110, 1107, 1103, 1376, 71, 52, 18, - 1102, 31, 171, 47, 1100, 1099, 34, 1098, 1093, 1091, - 1090, 1089, 1088, 1087, 118, 1086, 1084, 1082, 35, 42, - 1076, 1072, 63, 21, 1071, 1070, 1068, 49, 64, 1066, - 1062, 57, 26, 1061, 1060, 1059, 1058, 1055, 27, 17, - 1053, 16, 1052, 10, 1049, 33, 1044, 3, 1042, 9, - 1034, 6, 1033, 4, 43, 1, 1031, 2, 1030, 1028, - 45, 419, 1027, 1026, 107, + 0, 1111, 30, 409, 1110, 1106, 1105, 1103, 1102, 1101, + 1100, 1095, 1094, 1093, 1092, 1088, 1086, 1078, 1076, 1074, + 1073, 1072, 1071, 1070, 1069, 1067, 113, 1066, 1065, 1062, + 64, 1061, 71, 1059, 1057, 40, 210, 44, 41, 56, + 1055, 37, 54, 69, 1052, 28, 1048, 1047, 81, 1046, + 51, 1045, 1043, 48, 1042, 1041, 14, 42, 1040, 1039, + 1037, 1036, 66, 1349, 1035, 1034, 16, 1033, 1032, 97, + 1031, 53, 6, 15, 13, 26, 1030, 538, 7, 1029, + 52, 1027, 1023, 1022, 1016, 20, 1013, 57, 1011, 19, + 55, 1009, 9, 62, 25, 21, 8, 73, 60, 1008, + 22, 61, 47, 1007, 1005, 462, 1002, 1001, 50, 1000, + 999, 27, 166, 414, 997, 993, 977, 965, 35, 0, + 861, 183, 70, 964, 960, 957, 1944, 65, 49, 17, + 955, 79, 184, 43, 954, 953, 38, 952, 951, 950, + 949, 948, 946, 945, 111, 944, 942, 941, 29, 63, + 940, 939, 58, 23, 937, 936, 934, 45, 59, 933, + 932, 46, 33, 931, 928, 927, 926, 923, 24, 11, + 919, 18, 918, 12, 916, 34, 912, 4, 911, 10, + 910, 3, 909, 5, 75, 1, 908, 2, 907, 906, + 665, 930, 905, 904, 76, } var yyR1 = [...]int{ @@ -2186,43 +2221,44 @@ var yyR1 = [...]int{ 15, 19, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, - 20, 20, 20, 109, 109, 107, 107, 110, 110, 108, - 108, 108, 111, 111, 111, 135, 135, 135, 21, 21, - 23, 23, 24, 25, 22, 22, 22, 22, 22, 22, - 22, 16, 193, 26, 27, 27, 28, 28, 28, 32, - 32, 32, 30, 30, 31, 31, 37, 37, 36, 36, - 38, 38, 38, 38, 123, 123, 123, 122, 122, 40, - 40, 41, 41, 42, 42, 43, 43, 43, 43, 55, - 55, 92, 92, 94, 94, 44, 44, 44, 44, 45, - 45, 46, 46, 47, 47, 130, 130, 129, 129, 129, - 128, 128, 49, 49, 49, 51, 50, 50, 50, 50, - 52, 52, 54, 54, 53, 53, 56, 56, 56, 56, - 57, 57, 39, 39, 39, 39, 39, 39, 39, 106, - 106, 59, 59, 58, 58, 58, 58, 58, 58, 58, - 58, 58, 58, 70, 70, 70, 70, 70, 70, 60, - 60, 60, 60, 60, 60, 60, 35, 35, 71, 71, - 71, 77, 72, 72, 63, 63, 63, 63, 63, 63, + 20, 20, 20, 20, 109, 109, 107, 107, 110, 110, + 108, 108, 108, 111, 111, 111, 135, 135, 135, 21, + 21, 23, 23, 24, 25, 22, 22, 22, 22, 22, + 22, 22, 16, 193, 26, 27, 27, 28, 28, 28, + 32, 32, 32, 30, 30, 31, 31, 37, 37, 36, + 36, 38, 38, 38, 38, 123, 123, 123, 122, 122, + 40, 40, 41, 41, 42, 42, 43, 43, 43, 43, + 55, 55, 92, 92, 94, 94, 44, 44, 44, 44, + 45, 45, 46, 46, 47, 47, 130, 130, 129, 129, + 129, 128, 128, 49, 49, 49, 51, 50, 50, 50, + 50, 52, 52, 54, 54, 53, 53, 56, 56, 56, + 56, 57, 57, 39, 39, 39, 39, 39, 39, 39, + 106, 106, 59, 59, 58, 58, 58, 58, 58, 58, + 58, 58, 58, 58, 70, 70, 70, 70, 70, 70, + 60, 60, 60, 60, 60, 60, 60, 35, 35, 71, + 71, 71, 77, 72, 72, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, - 63, 63, 63, 63, 63, 63, 67, 67, 67, 65, + 63, 63, 63, 63, 63, 63, 63, 67, 67, 67, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, - 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, - 66, 66, 66, 66, 66, 66, 194, 194, 69, 68, - 68, 68, 68, 33, 33, 33, 33, 33, 133, 133, - 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, - 136, 136, 136, 81, 81, 34, 34, 79, 79, 80, - 82, 82, 78, 78, 78, 62, 62, 62, 62, 62, - 62, 62, 62, 64, 64, 64, 83, 83, 84, 84, - 85, 85, 86, 86, 87, 88, 88, 88, 89, 89, - 89, 89, 90, 90, 90, 61, 61, 61, 61, 61, - 61, 91, 91, 91, 91, 95, 95, 73, 73, 75, - 75, 74, 76, 96, 96, 100, 97, 97, 101, 101, - 101, 101, 99, 99, 99, 125, 125, 125, 104, 104, - 112, 112, 113, 113, 105, 105, 114, 114, 114, 114, - 114, 114, 114, 114, 114, 114, 115, 115, 115, 116, - 116, 117, 117, 117, 124, 124, 120, 120, 121, 121, - 126, 126, 127, 127, 118, 118, 118, 118, 118, 118, + 65, 66, 66, 66, 66, 66, 66, 66, 66, 66, + 66, 66, 66, 66, 66, 66, 66, 194, 194, 69, + 68, 68, 68, 68, 33, 33, 33, 33, 33, 133, + 133, 136, 136, 136, 136, 136, 136, 136, 136, 136, + 136, 136, 136, 136, 81, 81, 34, 34, 79, 79, + 80, 82, 82, 78, 78, 78, 62, 62, 62, 62, + 62, 62, 62, 62, 64, 64, 64, 83, 83, 84, + 84, 85, 85, 86, 86, 87, 88, 88, 88, 89, + 89, 89, 89, 90, 90, 90, 61, 61, 61, 61, + 61, 61, 91, 91, 91, 91, 95, 95, 73, 73, + 75, 75, 74, 76, 96, 96, 100, 97, 97, 101, + 101, 101, 101, 99, 99, 99, 125, 125, 125, 104, + 104, 112, 112, 113, 113, 105, 105, 114, 114, 114, + 114, 114, 114, 114, 114, 114, 114, 115, 115, 115, + 116, 116, 117, 117, 117, 124, 124, 120, 120, 121, + 121, 126, 126, 127, 127, 118, 118, 118, 118, 118, + 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, @@ -2231,7 +2267,6 @@ var yyR1 = [...]int{ 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, - 118, 118, 118, 118, 118, 118, 118, 118, 118, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, @@ -2243,7 +2278,8 @@ var yyR1 = [...]int{ 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, - 119, 119, 119, 190, 191, 131, 132, 132, 132, + 119, 119, 119, 119, 119, 190, 191, 131, 132, 132, + 132, } var yyR2 = [...]int{ @@ -2273,44 +2309,44 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 7, 1, 3, 8, 8, 3, 3, 5, 4, 6, 5, 4, 4, 3, 2, 3, 4, 4, 3, 4, 4, 4, 4, 4, - 4, 3, 2, 3, 3, 2, 3, 4, 3, 7, - 5, 4, 2, 4, 2, 2, 2, 2, 3, 3, - 5, 2, 3, 1, 1, 0, 1, 1, 1, 0, - 2, 2, 0, 2, 2, 0, 1, 1, 2, 1, - 1, 2, 1, 1, 2, 2, 2, 2, 2, 3, - 3, 2, 0, 2, 0, 2, 1, 2, 2, 0, - 1, 1, 0, 1, 0, 1, 0, 1, 1, 3, - 1, 2, 3, 5, 0, 1, 2, 1, 1, 0, - 2, 1, 3, 1, 1, 1, 3, 1, 3, 3, - 7, 1, 3, 1, 3, 4, 4, 4, 3, 2, - 4, 0, 1, 0, 2, 0, 1, 0, 1, 2, - 1, 1, 1, 2, 2, 1, 2, 3, 2, 3, - 2, 2, 2, 1, 1, 3, 0, 5, 5, 5, - 0, 2, 1, 3, 3, 2, 3, 1, 2, 0, - 3, 1, 1, 3, 3, 4, 4, 5, 3, 4, - 5, 6, 2, 1, 2, 1, 2, 1, 2, 1, - 1, 1, 1, 1, 1, 1, 0, 2, 1, 1, - 1, 3, 1, 3, 1, 1, 1, 1, 1, 3, + 4, 3, 3, 2, 3, 3, 2, 3, 4, 3, + 7, 5, 4, 2, 4, 2, 2, 2, 2, 3, + 3, 5, 2, 3, 1, 1, 0, 1, 1, 1, + 0, 2, 2, 0, 2, 2, 0, 1, 1, 2, + 1, 1, 2, 1, 1, 2, 2, 2, 2, 2, + 3, 3, 2, 0, 2, 0, 2, 1, 2, 2, + 0, 1, 1, 0, 1, 0, 1, 0, 1, 1, + 3, 1, 2, 3, 5, 0, 1, 2, 1, 1, + 0, 2, 1, 3, 1, 1, 1, 3, 1, 3, + 3, 7, 1, 3, 1, 3, 4, 4, 4, 3, + 2, 4, 0, 1, 0, 2, 0, 1, 0, 1, + 2, 1, 1, 1, 2, 2, 1, 2, 3, 2, + 3, 2, 2, 2, 1, 1, 3, 0, 5, 5, + 5, 0, 2, 1, 3, 3, 2, 3, 1, 2, + 0, 3, 1, 1, 3, 3, 4, 4, 5, 3, + 4, 5, 6, 2, 1, 2, 1, 2, 1, 2, + 1, 1, 1, 1, 1, 1, 1, 0, 2, 1, + 1, 1, 3, 1, 3, 1, 1, 1, 1, 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 2, 2, 2, 2, 2, 2, - 2, 3, 1, 1, 1, 1, 4, 5, 6, 4, - 4, 6, 6, 6, 6, 8, 8, 6, 8, 8, - 6, 8, 8, 6, 8, 8, 9, 7, 5, 4, - 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 8, 8, 0, 2, 3, 4, - 4, 4, 4, 0, 3, 4, 7, 3, 1, 1, - 2, 3, 3, 1, 2, 2, 1, 2, 1, 2, - 2, 1, 2, 0, 1, 0, 2, 1, 2, 4, - 0, 2, 1, 3, 5, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 2, 2, 0, 3, 0, 2, - 0, 3, 1, 3, 2, 0, 1, 1, 0, 2, - 4, 4, 0, 2, 4, 2, 1, 3, 5, 4, - 6, 1, 3, 3, 5, 0, 5, 1, 3, 1, - 2, 3, 1, 1, 3, 3, 1, 3, 3, 3, - 3, 3, 1, 2, 1, 1, 1, 1, 1, 1, - 0, 2, 0, 3, 0, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, - 1, 0, 1, 1, 0, 2, 1, 1, 1, 1, + 3, 3, 3, 3, 3, 2, 2, 2, 2, 2, + 2, 2, 3, 1, 1, 1, 1, 4, 5, 6, + 4, 4, 6, 6, 6, 6, 8, 8, 6, 8, + 8, 6, 8, 8, 6, 8, 8, 9, 7, 5, + 4, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 8, 8, 0, 2, 3, + 4, 4, 4, 4, 0, 3, 4, 7, 3, 1, + 1, 2, 3, 3, 1, 2, 2, 1, 2, 1, + 2, 2, 1, 2, 0, 1, 0, 2, 1, 2, + 4, 0, 2, 1, 3, 5, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 2, 2, 0, 3, 0, + 2, 0, 3, 1, 3, 2, 0, 1, 1, 0, + 2, 4, 4, 0, 2, 4, 2, 1, 3, 5, + 4, 6, 1, 3, 3, 5, 0, 5, 1, 3, + 1, 2, 3, 1, 1, 3, 3, 1, 3, 3, + 3, 3, 3, 1, 2, 1, 1, 1, 1, 1, + 1, 0, 2, 0, 3, 0, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, + 1, 1, 0, 1, 1, 0, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, @@ -2332,7 +2368,8 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 0, 0, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 0, 0, 1, + 1, } var yyChk = [...]int{ @@ -2341,294 +2378,294 @@ var yyChk = [...]int{ -22, -16, -3, -4, 6, 7, -29, 9, 10, 30, -17, 115, 116, 118, 117, 149, 119, 142, 50, 162, 163, 165, 166, 25, 143, 144, 147, 148, 31, 32, - 121, -190, 8, 251, 54, -189, 268, -85, 15, -28, + 121, -190, 8, 252, 54, -189, 269, -85, 15, -28, 5, -26, -193, -26, -26, -26, -26, -26, -164, -166, - 54, 90, -117, 125, 72, 243, 122, 123, 129, -120, - 57, -119, 261, 135, 162, 173, 167, 194, 186, 262, - 136, 184, 187, 230, 214, 225, 66, 165, 239, 145, - 182, 178, 176, 27, 227, 199, 266, 177, 226, 121, - 138, 133, 200, 204, 231, 171, 172, 233, 198, 134, - 33, 263, 35, 153, 234, 202, 197, 193, 196, 170, - 192, 39, 206, 205, 207, 229, 189, 139, 179, 18, - 237, 148, 151, 228, 201, 203, 130, 155, 265, 235, - 175, 140, 152, 147, 238, 141, 166, 232, 241, 38, - 211, 169, 132, 163, 159, 216, 190, 154, 180, 181, - 195, 168, 191, 164, 156, 149, 240, 212, 267, 188, - 185, 160, 157, 158, 217, 218, 219, 220, 221, 222, - 161, 264, 236, 183, 213, -105, 125, 220, 127, 123, - 123, 124, 125, 243, 122, 123, -53, -126, 57, -119, - 125, 123, 108, 187, 230, 115, 215, 227, 124, 33, - 228, 155, -135, 123, -107, 214, 217, 218, 219, 222, - 220, 161, 57, 232, 231, 223, -126, 164, -131, -131, - -131, -131, -131, 216, 216, -131, -2, -89, 17, 16, - -5, -3, -190, 6, 20, 21, -32, 40, 41, -27, - -38, 99, -39, -126, -58, 74, -63, 29, 57, -119, - 23, -62, -59, -78, -76, -77, 108, 109, 110, 97, - 98, 105, 75, 111, -67, -65, -66, -68, 59, 58, - 67, 60, 61, 62, 63, 68, 69, 70, -120, -74, - -190, 44, 45, 252, 253, 254, 255, 260, 256, 77, - 34, 242, 250, 249, 248, 246, 247, 244, 245, 258, - 259, 128, 243, 103, 251, -105, -105, 11, -48, -53, - -97, -134, 164, -101, 232, 231, -121, -99, -120, -118, - 230, 187, 229, 120, 73, 22, 24, 209, 76, 108, - 16, 77, 107, 252, 115, 48, 244, 245, 242, 254, - 255, 243, 215, 29, 10, 25, 143, 21, 101, 117, - 80, 81, 146, 23, 144, 70, 19, 51, 11, 13, - 14, 128, 127, 92, 124, 46, 8, 111, 26, 89, - 42, 28, 44, 90, 17, 246, 247, 31, 260, 150, - 103, 49, 36, 74, 68, 71, 52, 72, 15, 47, - 91, 118, 251, 45, 122, 6, 257, 30, 142, 43, - 123, 79, 258, 259, 126, 69, 5, 129, 32, 9, - 50, 53, 248, 249, 250, 34, 78, 12, -165, 90, - -158, 57, -53, 124, -53, 251, -113, 128, -113, -113, - 123, -53, 115, 117, 120, 52, -18, -53, -112, 128, - 57, -112, -112, -112, -53, 112, -53, 57, 30, -132, - -190, -121, 243, 57, 155, 123, 156, 125, -132, -132, - -132, -132, 159, 160, -132, -110, -109, 225, 226, 216, - 224, 12, 216, 158, -132, -131, -131, -191, 56, -90, - 19, 31, -39, -126, -86, -87, -39, -85, -2, -26, - 36, -30, 21, 65, 11, -123, 73, 72, 89, -122, - 22, -120, 59, 112, -39, -60, 92, 74, 90, 91, - 76, 94, 93, 104, 97, 98, 99, 100, 101, 102, - 103, 95, 96, 107, 82, 83, 84, 85, 86, 87, - 88, -106, -190, -77, -190, 113, 114, -63, -63, -63, - -63, -63, -63, -63, -63, -190, -2, -72, -39, -190, - -190, -190, -190, -190, -190, -190, -190, -190, -81, -39, - -190, -194, -69, -190, -194, -69, -194, -69, -194, -190, - -194, -69, -194, -69, -194, -194, -69, -190, -190, -190, - -190, -190, -190, -54, 26, -53, -41, -42, -43, -44, - -55, -77, -190, -53, -53, -48, -192, 55, 11, 53, - 55, -97, 164, -98, -102, 233, 235, 82, -125, -120, - 59, 29, 30, 56, 55, -53, -137, -140, -142, -141, - -143, -138, -139, 184, 185, 108, 188, 190, 191, 192, - 193, 194, 195, 196, 197, 198, 199, 30, 145, 180, - 181, 182, 183, 200, 201, 202, 203, 204, 205, 206, - 207, 167, 186, 262, 168, 169, 170, 171, 172, 173, - 175, 176, 177, 178, 179, 57, -132, 125, 57, 74, - 57, -53, -53, -132, 157, 157, 123, 123, -53, 55, - 126, -48, 23, 52, -53, 57, 57, -127, -126, -118, - -132, -132, -132, -132, -132, -53, -132, -132, -132, -132, - 11, -108, 11, 92, -39, 52, 9, 92, 55, 18, - 112, 55, -88, 24, 25, -89, -191, -32, -64, -120, - 60, 63, -31, 43, -53, -39, -39, -70, 68, 74, - 69, 70, -122, 99, -127, -121, -118, -63, -71, -74, - -77, 64, 92, 90, 91, 76, -63, -63, -63, -63, + 54, 90, -117, 125, 72, 244, 122, 123, 129, -120, + 57, -119, 262, 135, 162, 173, 167, 194, 186, 263, + 136, 184, 187, 231, 214, 226, 66, 165, 240, 145, + 182, 178, 176, 27, 228, 199, 267, 177, 227, 121, + 138, 133, 200, 204, 232, 171, 172, 234, 198, 134, + 33, 264, 35, 153, 235, 202, 197, 193, 196, 170, + 192, 39, 206, 205, 207, 230, 189, 139, 179, 18, + 238, 148, 151, 229, 201, 203, 130, 155, 266, 236, + 175, 140, 152, 147, 239, 141, 166, 216, 233, 242, + 38, 211, 169, 132, 163, 159, 217, 190, 154, 180, + 181, 195, 168, 191, 164, 156, 149, 241, 212, 268, + 188, 185, 160, 157, 158, 218, 219, 220, 221, 222, + 223, 161, 265, 237, 183, 213, -105, 125, 221, 127, + 123, 123, 124, 125, 244, 122, 123, -53, -126, 57, + -119, 125, 123, 108, 187, 231, 115, 215, 216, 228, + 124, 33, 229, 155, -135, 123, -107, 214, 218, 219, + 220, 223, 221, 161, 57, 233, 232, 224, -126, 164, + -131, -131, -131, -131, -131, 217, 217, -131, -2, -89, + 17, 16, -5, -3, -190, 6, 20, 21, -32, 40, + 41, -27, -38, 99, -39, -126, -58, 74, -63, 29, + 57, -119, 23, -62, -59, -78, -76, -77, 108, 109, + 110, 97, 98, 105, 75, 111, -67, -65, -66, -68, + 59, 58, 67, 60, 61, 62, 63, 68, 69, 70, + -120, -74, -190, 44, 45, 253, 254, 255, 256, 261, + 257, 77, 34, 243, 251, 250, 249, 247, 248, 245, + 246, 259, 260, 128, 244, 103, 252, -105, -105, 11, + -48, -53, -97, -134, 164, -101, 233, 232, -121, -99, + -120, -118, 231, 187, 230, 120, 73, 22, 24, 209, + 76, 108, 16, 77, 107, 253, 115, 48, 245, 246, + 243, 255, 256, 244, 215, 29, 10, 25, 143, 21, + 101, 117, 80, 81, 146, 23, 144, 70, 19, 51, + 11, 13, 14, 128, 127, 92, 124, 46, 8, 111, + 26, 89, 42, 28, 44, 90, 17, 247, 248, 31, + 261, 150, 103, 49, 36, 74, 68, 71, 52, 72, + 15, 47, 91, 118, 252, 45, 122, 6, 258, 30, + 142, 43, 123, 79, 259, 260, 126, 69, 5, 129, + 32, 9, 50, 53, 249, 250, 251, 34, 78, 12, + -165, 90, -158, 57, -53, 124, -53, 252, -113, 128, + -113, -113, 123, -53, 115, 117, 120, 52, -18, -53, + -112, 128, 57, -112, -112, -112, -53, 112, -53, 57, + 30, -132, -190, -121, 244, 57, 155, 123, 156, 125, + -132, -132, -132, -132, -132, 159, 160, -132, -110, -109, + 226, 227, 217, 225, 12, 217, 158, -132, -131, -131, + -191, 56, -90, 19, 31, -39, -126, -86, -87, -39, + -85, -2, -26, 36, -30, 21, 65, 11, -123, 73, + 72, 89, -122, 22, -120, 59, 112, -39, -60, 92, + 74, 90, 91, 76, 94, 93, 104, 97, 98, 99, + 100, 101, 102, 103, 95, 96, 107, 82, 83, 84, + 85, 86, 87, 88, -106, -190, -77, -190, 113, 114, + -63, -63, -63, -63, -63, -63, -63, -63, -190, -2, + -72, -39, -190, -190, -190, -190, -190, -190, -190, -190, + -190, -81, -39, -190, -194, -69, -190, -194, -69, -194, + -69, -194, -190, -194, -69, -194, -69, -194, -194, -69, + -190, -190, -190, -190, -190, -190, -54, 26, -53, -41, + -42, -43, -44, -55, -77, -190, -53, -53, -48, -192, + 55, 11, 53, 55, -97, 164, -98, -102, 234, 236, + 82, -125, -120, 59, 29, 30, 56, 55, -53, -137, + -140, -142, -141, -143, -138, -139, 184, 185, 108, 188, + 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, + 30, 145, 180, 181, 182, 183, 200, 201, 202, 203, + 204, 205, 206, 207, 167, 186, 263, 168, 169, 170, + 171, 172, 173, 175, 176, 177, 178, 179, 57, -132, + 125, 57, 74, 57, -53, -53, -132, 157, 157, 123, + 123, -53, 55, 126, -48, 23, 52, -53, 57, 57, + -127, -126, -118, -132, -132, -132, -132, -132, -53, -132, + -132, -132, -132, 11, -108, 11, 92, -39, 52, 9, + 92, 55, 18, 112, 55, -88, 24, 25, -89, -191, + -32, -64, -120, 60, 63, -31, 43, -53, -39, -39, + -70, 68, 74, 69, 70, -122, 99, -127, -121, -118, + -63, -71, -74, -77, 64, 92, 90, 91, 76, -63, -63, -63, -63, -63, -63, -63, -63, -63, -63, -63, - -63, -133, 57, 59, 57, -62, -62, -120, -37, 21, - -36, -38, -191, 55, -191, -2, -36, -36, -39, -39, - -78, 59, -120, -126, -78, 59, -36, -30, -79, -80, - 78, -78, -191, -63, -120, -120, -36, -37, -36, -36, - -93, 151, -53, 30, 55, -49, -51, -50, -52, 42, - 46, 48, 43, 44, 45, 49, -130, 22, -41, -190, - -129, 151, -128, 22, -126, 59, -93, 53, -41, -53, - -101, -98, 55, 234, 236, 237, 52, 71, -39, -149, - 107, -167, -168, -169, -121, 59, 60, -158, -159, -160, - -170, 137, -175, 130, 132, 129, -161, 138, 124, 28, - 56, -154, 68, 74, -150, 212, -144, 54, -144, -144, - -144, -144, -148, 187, -148, -148, -148, 54, 54, -144, - -144, -144, -152, 54, -152, -152, -153, 54, -153, -124, - 53, -53, -132, 23, -132, -114, 120, 117, 118, -178, - 116, 209, 187, 66, 29, 15, 252, 151, 267, 57, - 152, -120, -120, -53, -53, 120, 117, -53, -53, -53, - -132, -53, -111, 90, 12, -126, -126, -53, 38, -39, - -39, -127, -87, -90, -104, 19, 11, 34, 34, -36, - 68, 69, 70, 112, -190, -71, -63, -63, -63, -35, - 146, 73, -191, -191, -36, 55, -39, -191, -191, -191, - 55, 53, 22, 55, 11, 55, 11, 112, 55, 11, - 55, 11, -191, -36, -82, -80, 80, -39, -191, -191, - 55, 55, -191, -191, -191, -191, -61, 30, 34, -2, - -190, -190, -96, -100, -78, -42, -43, -43, -42, -43, - 42, 42, 42, 47, 42, 47, 42, -50, -126, -191, - -56, 50, 127, 51, -190, -128, -57, 12, -41, -57, - -102, -103, 238, 235, 241, 57, 59, 55, -169, 82, - 54, 57, 28, -161, -161, -162, 57, -162, 28, -146, - 29, 68, -151, 213, 60, -148, -148, -149, 30, -149, - -149, -149, -157, 59, -157, 60, 60, 52, -120, -132, - -131, -184, 131, 137, 138, 133, 57, 124, 28, 130, - 132, 151, 129, -184, -115, -116, 126, 22, 124, 28, - 151, -183, 53, 157, 157, 126, -132, -108, 59, -39, - 39, 112, -53, -40, 11, 99, -121, -37, -35, 73, - -63, -63, -191, -38, -136, 108, 184, 145, 182, 178, - 198, 189, 211, 180, 212, -133, -136, -63, -63, -63, - -63, -121, -63, -63, -63, -63, 261, -85, 81, -39, - 79, -63, -63, -95, 52, -96, -73, -75, -74, -190, - -2, -91, -120, -94, -120, -57, 55, 82, -46, -45, - 52, 53, -47, 52, -45, 42, 42, 124, 124, 124, - -94, -85, -39, -57, 235, 239, 240, -168, -169, -172, - -171, -120, -175, -162, -162, 54, -147, 52, -63, 56, - -149, -149, 57, 108, 56, 55, 56, 55, 56, 55, - -53, -131, -131, -53, -131, -120, -181, 264, -182, 57, - -120, -120, -53, -111, -57, -41, -191, -63, -191, -144, - -144, -144, -153, -144, 172, -144, 172, -191, -191, -191, - 55, 19, -191, 55, 19, -191, 55, 19, -191, 55, - 19, -190, -34, 257, -39, 55, 55, 27, -95, 55, - -191, -191, -191, 55, 112, -191, 55, -85, -100, -39, - -39, 54, -39, -190, -190, -190, -191, -89, 56, 55, - -144, -92, -120, -155, 209, 9, -148, 59, -148, 60, - 60, -132, 26, -180, -179, -121, 54, -83, 13, -148, - 57, -63, -63, -63, -63, -63, -63, -63, -63, -63, - -191, 59, -63, -63, 28, -75, 34, -2, -190, -120, - -120, -120, -89, -92, -92, -92, -92, -129, -174, -173, - 53, 134, 66, -171, 56, 55, -156, 130, 28, 129, - -66, -149, -149, 56, 56, -190, 55, 82, -92, -84, - 14, 16, -191, -191, -191, -191, -191, -191, -191, -191, - -33, 92, 264, -191, -191, 9, -73, -2, 112, 56, - -191, -191, -191, -56, -173, 57, -163, 82, 59, 140, - -120, -145, 66, 28, 28, -176, -177, 151, -179, -169, - 56, -39, -72, -191, 262, 49, 265, -96, -191, -120, - 60, -53, 59, -191, 55, -120, -183, 39, 263, 266, - 54, -177, 34, -181, 39, -92, 153, 264, 56, 154, - 265, -186, -187, 52, -190, 266, -187, 52, 10, 9, - -63, 150, -185, 141, 136, 139, 30, -185, -191, -191, - 135, 29, 68, + -63, -63, -63, -63, -133, 57, 59, 57, -62, -62, + -120, -37, 21, -36, -38, -191, 55, -191, -2, -36, + -36, -39, -39, -78, 59, -120, -126, -78, 59, -36, + -30, -79, -80, 78, -78, -191, -63, -120, -120, -36, + -37, -36, -36, -93, 151, -53, 30, 55, -49, -51, + -50, -52, 42, 46, 48, 43, 44, 45, 49, -130, + 22, -41, -190, -129, 151, -128, 22, -126, 59, -93, + 53, -41, -53, -101, -98, 55, 235, 237, 238, 52, + 71, -39, -149, 107, -167, -168, -169, -121, 59, 60, + -158, -159, -160, -170, 137, -175, 130, 132, 129, -161, + 138, 124, 28, 56, -154, 68, 74, -150, 212, -144, + 54, -144, -144, -144, -144, -148, 187, -148, -148, -148, + 54, 54, -144, -144, -144, -152, 54, -152, -152, -153, + 54, -153, -124, 53, -53, -132, 23, -132, -114, 120, + 117, 118, -178, 116, 209, 187, 66, 29, 15, 253, + 151, 268, 57, 152, -120, -120, -53, -53, 120, 117, + -53, -53, -53, -132, -53, -111, 90, 12, -126, -126, + -53, 38, -39, -39, -127, -87, -90, -104, 19, 11, + 34, 34, -36, 68, 69, 70, 112, -190, -71, -63, + -63, -63, -35, 146, 73, -191, -191, -36, 55, -39, + -191, -191, -191, 55, 53, 22, 55, 11, 55, 11, + 112, 55, 11, 55, 11, -191, -36, -82, -80, 80, + -39, -191, -191, 55, 55, -191, -191, -191, -191, -61, + 30, 34, -2, -190, -190, -96, -100, -78, -42, -43, + -43, -42, -43, 42, 42, 42, 47, 42, 47, 42, + -50, -126, -191, -56, 50, 127, 51, -190, -128, -57, + 12, -41, -57, -102, -103, 239, 236, 242, 57, 59, + 55, -169, 82, 54, 57, 28, -161, -161, -162, 57, + -162, 28, -146, 29, 68, -151, 213, 60, -148, -148, + -149, 30, -149, -149, -149, -157, 59, -157, 60, 60, + 52, -120, -132, -131, -184, 131, 137, 138, 133, 57, + 124, 28, 130, 132, 151, 129, -184, -115, -116, 126, + 22, 124, 28, 151, -183, 53, 157, 157, 126, -132, + -108, 59, -39, 39, 112, -53, -40, 11, 99, -121, + -37, -35, 73, -63, -63, -191, -38, -136, 108, 184, + 145, 182, 178, 198, 189, 211, 180, 212, -133, -136, + -63, -63, -63, -63, -121, -63, -63, -63, -63, 262, + -85, 81, -39, 79, -63, -63, -95, 52, -96, -73, + -75, -74, -190, -2, -91, -120, -94, -120, -57, 55, + 82, -46, -45, 52, 53, -47, 52, -45, 42, 42, + 124, 124, 124, -94, -85, -39, -57, 236, 240, 241, + -168, -169, -172, -171, -120, -175, -162, -162, 54, -147, + 52, -63, 56, -149, -149, 57, 108, 56, 55, 56, + 55, 56, 55, -53, -131, -131, -53, -131, -120, -181, + 265, -182, 57, -120, -120, -53, -111, -57, -41, -191, + -63, -191, -144, -144, -144, -153, -144, 172, -144, 172, + -191, -191, -191, 55, 19, -191, 55, 19, -191, 55, + 19, -191, 55, 19, -190, -34, 258, -39, 55, 55, + 27, -95, 55, -191, -191, -191, 55, 112, -191, 55, + -85, -100, -39, -39, 54, -39, -190, -190, -190, -191, + -89, 56, 55, -144, -92, -120, -155, 209, 9, -148, + 59, -148, 60, 60, -132, 26, -180, -179, -121, 54, + -83, 13, -148, 57, -63, -63, -63, -63, -63, -63, + -63, -63, -63, -191, 59, -63, -63, 28, -75, 34, + -2, -190, -120, -120, -120, -89, -92, -92, -92, -92, + -129, -174, -173, 53, 134, 66, -171, 56, 55, -156, + 130, 28, 129, -66, -149, -149, 56, 56, -190, 55, + 82, -92, -84, 14, 16, -191, -191, -191, -191, -191, + -191, -191, -191, -33, 92, 265, -191, -191, 9, -73, + -2, 112, 56, -191, -191, -191, -56, -173, 57, -163, + 82, 59, 140, -120, -145, 66, 28, 28, -176, -177, + 151, -179, -169, 56, -39, -72, -191, 263, 49, 266, + -96, -191, -120, 60, -53, 59, -191, 55, -120, -183, + 39, 264, 267, 54, -177, 34, -181, 39, -92, 153, + 265, 56, 154, 266, -186, -187, 52, -190, 267, -187, + 52, 10, 9, -63, 150, -185, 141, 136, 139, 30, + -185, -191, -191, 135, 29, 68, } var yyDef = [...]int{ 23, -2, 2, -2, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, - 21, 22, 560, 0, 312, 312, 312, 312, 312, 312, - 0, 631, 614, 0, 0, 0, 0, -2, 299, 300, - 0, 302, 303, 855, 855, 855, 855, 855, 0, 0, - 855, 0, 35, 36, 853, 1, 3, 568, 0, 0, - 316, 319, 314, 0, 614, 614, 0, 0, 62, 63, - 0, 0, 0, 839, 0, 612, 612, 612, 632, 633, - 636, 637, 739, 740, 741, 742, 743, 744, 745, 746, - 747, 748, 749, 750, 751, 752, 753, 754, 755, 756, - 757, 758, 759, 760, 761, 762, 763, 764, 765, 766, - 767, 768, 769, 770, 771, 772, 773, 774, 775, 776, - 777, 778, 779, 780, 781, 782, 783, 784, 785, 786, - 787, 788, 789, 790, 791, 792, 793, 794, 795, 796, - 797, 798, 799, 800, 801, 802, 803, 804, 805, 806, - 807, 808, 809, 810, 811, 812, 813, 814, 815, 816, - 817, 818, 819, 820, 821, 822, 823, 824, 825, 826, - 827, 828, 829, 830, 831, 832, 833, 834, 835, 836, - 837, 838, 840, 841, 842, 843, 844, 845, 846, 847, - 848, 849, 850, 851, 852, 0, 0, 0, 615, 0, - 610, 0, 610, 610, 610, 0, 250, 384, 640, 641, - 839, 0, 0, 0, 856, 0, 856, 262, 856, 856, - 265, 856, 0, 856, 0, 272, 274, 275, 276, 277, - 0, 281, 856, 296, 297, 286, 298, 301, 304, 305, - 306, 307, 308, 855, 855, 311, 29, 572, 0, 0, - 560, 31, 0, 312, 317, 318, 322, 320, 321, 313, - 0, 330, 334, 0, 392, 0, 397, 399, -2, -2, - 0, 434, 435, 436, 437, 438, 0, 0, 0, 0, - 0, 0, 0, 0, 462, 463, 464, 465, 545, 546, - 547, 548, 549, 550, 551, 552, 401, 402, 542, 592, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 533, - 0, 506, 506, 506, 506, 506, 506, 506, 506, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 43, - 47, 0, 830, 596, -2, -2, 0, 0, 638, 639, - -2, 749, -2, 644, 645, 646, 647, 648, 649, 650, - 651, 652, 653, 654, 655, 656, 657, 658, 659, 660, - 661, 662, 663, 664, 665, 666, 667, 668, 669, 670, - 671, 672, 673, 674, 675, 676, 677, 678, 679, 680, - 681, 682, 683, 684, 685, 686, 687, 688, 689, 690, - 691, 692, 693, 694, 695, 696, 697, 698, 699, 700, - 701, 702, 703, 704, 705, 706, 707, 708, 709, 710, - 711, 712, 713, 714, 715, 716, 717, 718, 719, 720, - 721, 722, 723, 724, 725, 726, 727, 728, 729, 730, - 731, 732, 733, 734, 735, 736, 737, 738, 0, 0, - 81, 0, 79, 0, 856, 0, 0, 0, 0, 0, - 0, 856, 0, 0, 0, 0, 241, 0, 0, 0, - 0, 0, 0, 0, 249, 0, 251, 856, 856, 254, - 857, 858, 856, 856, 856, 0, 856, 856, 261, 263, - 264, 266, 856, 856, 268, 0, 289, 287, 288, 283, - 284, 0, 278, 279, 282, 309, 310, 30, 854, 24, - 0, 0, 569, 0, 561, 562, 565, 568, 29, 319, - 0, 324, 323, 315, 0, 331, 0, 0, 0, 335, - 0, 337, 338, 0, 395, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 419, 420, 421, 422, 423, 424, - 425, 398, 0, 412, 0, 0, 0, 454, 455, 456, - 457, 458, 459, 460, 0, 326, 29, 0, 432, 0, - 0, 0, 0, 0, 0, 0, 0, 322, 0, 534, - 0, 490, 498, 0, 491, 499, 492, 500, 493, 0, - 494, 501, 495, 502, 496, 497, 503, 0, 0, 0, - 326, 0, 0, 45, 0, 383, 0, 341, 343, 344, - 345, -2, 0, 367, -2, 0, 0, 0, 41, 42, - 0, 48, 830, 50, 51, 0, 0, 0, 159, 605, - 606, 607, 603, 203, 0, 0, 147, 143, 87, 88, - 89, 136, 91, 136, 136, 136, 136, 156, 156, 156, - 156, 119, 120, 121, 122, 123, 0, 0, 106, 136, - 136, 136, 110, 126, 127, 128, 129, 130, 131, 132, - 133, 92, 93, 94, 95, 96, 97, 98, 99, 100, - 138, 138, 138, 140, 140, 634, 65, 0, 856, 0, - 856, 77, 0, 217, 0, 0, 0, 0, 0, 0, - 0, 244, 611, 0, 856, 247, 248, 385, 642, 643, - 252, 253, 255, 256, 257, 258, 259, 260, 267, 271, - 0, 292, 0, 0, 273, 0, 573, 0, 0, 0, - 0, 0, 564, 566, 567, 572, 32, 322, 0, 553, - 0, 0, 0, 325, 27, 393, 394, 396, 413, 0, - 415, 417, 336, 332, 0, 543, -2, 403, 404, 428, - 429, 430, 0, 0, 0, 0, 426, 408, 0, 439, - 440, 441, 442, 443, 444, 445, 446, 447, 448, 449, - 450, 453, 518, 519, 0, 451, 452, 461, 0, 0, - 327, 328, 431, 0, 591, 29, 0, 0, 0, 0, - 0, 0, 542, 0, 0, 0, 0, 0, 540, 537, - 0, 0, 507, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 382, 0, 0, 0, 0, 0, 0, 372, - 0, 0, 375, 0, 0, 0, 0, 366, 0, 0, - 386, 799, 368, 0, 370, 371, 390, 0, 390, 44, - 597, 49, 0, 0, 54, 55, 598, 599, 600, 601, - 0, 78, 204, 206, 209, 210, 211, 82, 83, 84, - 0, 0, 191, 0, 0, 185, 185, 0, 183, 184, - 80, 150, 148, 0, 145, 144, 90, 0, 156, 156, - 113, 114, 159, 0, 159, 159, 159, 0, 0, 107, - 108, 109, 101, 0, 102, 103, 104, 0, 105, 0, - 0, 856, 67, 613, 68, 855, 0, 0, 626, 218, - 616, 617, 618, 619, 620, 621, 622, 623, 624, 625, - 0, 69, 220, 222, 221, 0, 0, 0, 242, 856, - 246, 289, 270, 0, 0, 290, 291, 280, 0, 570, - 571, 0, 563, 25, 0, 608, 609, 554, 555, 339, - 414, 416, 418, 0, 326, 405, 426, 409, 0, 406, - 0, 0, 400, 466, 0, 0, 433, -2, 469, 470, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 560, 0, 538, 0, 0, 489, 508, - 0, 0, 509, 510, 511, 512, 585, 0, 0, -2, - 0, 0, 390, 593, 0, 342, 361, 363, 0, 358, - 373, 374, 376, 0, 378, 0, 380, 381, 346, 348, - 349, 0, 0, 0, 0, 369, 560, 0, 390, 40, - 52, 53, 0, 0, 59, 160, 161, 0, 207, 0, - 0, 0, 178, 185, 185, 181, 186, 182, 0, 152, - 0, 149, 86, 146, 0, 159, 159, 115, 0, 116, - 117, 118, 0, 134, 0, 0, 0, 0, 635, 66, - 212, 855, 225, 226, 227, 228, 229, 230, 231, 232, - 233, 234, 235, 855, 0, 855, 627, 628, 629, 630, - 0, 72, 0, 0, 0, 0, 245, 292, 293, 294, - 574, 0, 26, 390, 0, 333, 544, 0, 407, 0, - 427, 410, 467, 329, 0, 136, 136, 523, 136, 140, - 526, 136, 528, 136, 531, 0, 0, 0, 0, 0, - 0, 543, 0, 0, 0, 0, 0, 535, 488, 541, - 0, 0, 0, 33, 0, 585, 575, 587, 589, 0, - 29, 0, 581, 0, 353, 560, 0, 0, 355, 362, - 0, 0, 356, 0, 357, 377, 379, 0, 0, 0, - 0, 568, 391, 39, 56, 57, 58, 205, 208, 0, - 187, 136, 190, 179, 180, 0, 154, 0, 151, 137, - 111, 112, 157, 158, 156, 0, 156, 0, 141, 0, - 856, 213, 214, 215, 216, 0, 219, 0, 70, 71, - 0, 224, 243, 269, 556, 340, 468, 411, 471, 520, - 156, 524, 525, 527, 529, 530, 532, 473, 472, 474, - 0, 0, 480, 0, 0, 477, 0, 0, 483, 0, - 0, 0, 0, 0, 539, 0, 0, 0, 34, 0, - 590, -2, 0, 0, 0, 46, 0, 568, 594, 595, - 359, 0, 364, 0, 0, 0, 367, 38, 170, 0, - 189, 0, 351, 162, 155, 0, 159, 135, 159, 0, - 0, 64, 0, 73, 74, 0, 0, 558, 0, 521, - 522, 0, 0, 0, 0, 0, 0, 0, 0, 513, - 487, 536, 0, 0, 0, 588, 0, -2, 0, 583, - 582, 354, 37, 0, 0, 0, 0, 386, 169, 171, - 0, 176, 0, 188, 0, 0, 167, 0, 164, 166, - 153, 124, 125, 139, 142, 0, 0, 0, 0, 28, - 0, 0, 475, 476, 481, 482, 478, 479, 484, 485, - 0, 0, 0, 504, 505, 0, 578, 29, 0, 360, - 387, 388, 389, 350, 172, 173, 0, 177, 175, 0, - 352, 85, 0, 163, 165, 0, 237, 0, 75, 76, - 69, 559, 557, 486, 0, 0, 0, 586, -2, 584, - 174, 0, 168, 236, 0, 0, 72, 514, 0, 517, - 0, 238, 0, 223, 515, 0, 0, 0, 192, 0, - 0, 193, 194, 0, 0, 516, 195, 0, 0, 0, - 0, 0, 196, 198, 199, 0, 0, 197, 239, 240, - 200, 201, 202, + 21, 22, 561, 0, 313, 313, 313, 313, 313, 313, + 0, 632, 615, 0, 0, 0, 0, -2, 300, 301, + 0, 303, 304, 857, 857, 857, 857, 857, 0, 0, + 857, 0, 35, 36, 855, 1, 3, 569, 0, 0, + 317, 320, 315, 0, 615, 615, 0, 0, 62, 63, + 0, 0, 0, 841, 0, 613, 613, 613, 633, 634, + 637, 638, 740, 741, 742, 743, 744, 745, 746, 747, + 748, 749, 750, 751, 752, 753, 754, 755, 756, 757, + 758, 759, 760, 761, 762, 763, 764, 765, 766, 767, + 768, 769, 770, 771, 772, 773, 774, 775, 776, 777, + 778, 779, 780, 781, 782, 783, 784, 785, 786, 787, + 788, 789, 790, 791, 792, 793, 794, 795, 796, 797, + 798, 799, 800, 801, 802, 803, 804, 805, 806, 807, + 808, 809, 810, 811, 812, 813, 814, 815, 816, 817, + 818, 819, 820, 821, 822, 823, 824, 825, 826, 827, + 828, 829, 830, 831, 832, 833, 834, 835, 836, 837, + 838, 839, 840, 842, 843, 844, 845, 846, 847, 848, + 849, 850, 851, 852, 853, 854, 0, 0, 0, 616, + 0, 611, 0, 611, 611, 611, 0, 250, 385, 641, + 642, 841, 0, 0, 0, 858, 0, 858, 858, 263, + 858, 858, 266, 858, 0, 858, 0, 273, 275, 276, + 277, 278, 0, 282, 858, 297, 298, 287, 299, 302, + 305, 306, 307, 308, 309, 857, 857, 312, 29, 573, + 0, 0, 561, 31, 0, 313, 318, 319, 323, 321, + 322, 314, 0, 331, 335, 0, 393, 0, 398, 400, + -2, -2, 0, 435, 436, 437, 438, 439, 0, 0, + 0, 0, 0, 0, 0, 0, 463, 464, 465, 466, + 546, 547, 548, 549, 550, 551, 552, 553, 402, 403, + 543, 593, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 534, 0, 507, 507, 507, 507, 507, 507, 507, + 507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 43, 47, 0, 832, 597, -2, -2, 0, 0, + 639, 640, -2, 750, -2, 645, 646, 647, 648, 649, + 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, + 660, 661, 662, 663, 664, 665, 666, 667, 668, 669, + 670, 671, 672, 673, 674, 675, 676, 677, 678, 679, + 680, 681, 682, 683, 684, 685, 686, 687, 688, 689, + 690, 691, 692, 693, 694, 695, 696, 697, 698, 699, + 700, 701, 702, 703, 704, 705, 706, 707, 708, 709, + 710, 711, 712, 713, 714, 715, 716, 717, 718, 719, + 720, 721, 722, 723, 724, 725, 726, 727, 728, 729, + 730, 731, 732, 733, 734, 735, 736, 737, 738, 739, + 0, 0, 81, 0, 79, 0, 858, 0, 0, 0, + 0, 0, 0, 858, 0, 0, 0, 0, 241, 0, + 0, 0, 0, 0, 0, 0, 249, 0, 251, 858, + 858, 254, 859, 860, 858, 858, 858, 0, 858, 858, + 261, 262, 264, 265, 267, 858, 858, 269, 0, 290, + 288, 289, 284, 285, 0, 279, 280, 283, 310, 311, + 30, 856, 24, 0, 0, 570, 0, 562, 563, 566, + 569, 29, 320, 0, 325, 324, 316, 0, 332, 0, + 0, 0, 336, 0, 338, 339, 0, 396, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 420, 421, 422, + 423, 424, 425, 426, 399, 0, 413, 0, 0, 0, + 455, 456, 457, 458, 459, 460, 461, 0, 327, 29, + 0, 433, 0, 0, 0, 0, 0, 0, 0, 0, + 323, 0, 535, 0, 491, 499, 0, 492, 500, 493, + 501, 494, 0, 495, 502, 496, 503, 497, 498, 504, + 0, 0, 0, 327, 0, 0, 45, 0, 384, 0, + 342, 344, 345, 346, -2, 0, 368, -2, 0, 0, + 0, 41, 42, 0, 48, 832, 50, 51, 0, 0, + 0, 159, 606, 607, 608, 604, 203, 0, 0, 147, + 143, 87, 88, 89, 136, 91, 136, 136, 136, 136, + 156, 156, 156, 156, 119, 120, 121, 122, 123, 0, + 0, 106, 136, 136, 136, 110, 126, 127, 128, 129, + 130, 131, 132, 133, 92, 93, 94, 95, 96, 97, + 98, 99, 100, 138, 138, 138, 140, 140, 635, 65, + 0, 858, 0, 858, 77, 0, 217, 0, 0, 0, + 0, 0, 0, 0, 244, 612, 0, 858, 247, 248, + 386, 643, 644, 252, 253, 255, 256, 257, 258, 259, + 260, 268, 272, 0, 293, 0, 0, 274, 0, 574, + 0, 0, 0, 0, 0, 565, 567, 568, 573, 32, + 323, 0, 554, 0, 0, 0, 326, 27, 394, 395, + 397, 414, 0, 416, 418, 337, 333, 0, 544, -2, + 404, 405, 429, 430, 431, 0, 0, 0, 0, 427, + 409, 0, 440, 441, 442, 443, 444, 445, 446, 447, + 448, 449, 450, 451, 454, 519, 520, 0, 452, 453, + 462, 0, 0, 328, 329, 432, 0, 592, 29, 0, + 0, 0, 0, 0, 0, 543, 0, 0, 0, 0, + 0, 541, 538, 0, 0, 508, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 383, 0, 0, 0, 0, + 0, 0, 373, 0, 0, 376, 0, 0, 0, 0, + 367, 0, 0, 387, 800, 369, 0, 371, 372, 391, + 0, 391, 44, 598, 49, 0, 0, 54, 55, 599, + 600, 601, 602, 0, 78, 204, 206, 209, 210, 211, + 82, 83, 84, 0, 0, 191, 0, 0, 185, 185, + 0, 183, 184, 80, 150, 148, 0, 145, 144, 90, + 0, 156, 156, 113, 114, 159, 0, 159, 159, 159, + 0, 0, 107, 108, 109, 101, 0, 102, 103, 104, + 0, 105, 0, 0, 858, 67, 614, 68, 857, 0, + 0, 627, 218, 617, 618, 619, 620, 621, 622, 623, + 624, 625, 626, 0, 69, 220, 222, 221, 0, 0, + 0, 242, 858, 246, 290, 271, 0, 0, 291, 292, + 281, 0, 571, 572, 0, 564, 25, 0, 609, 610, + 555, 556, 340, 415, 417, 419, 0, 327, 406, 427, + 410, 0, 407, 0, 0, 401, 467, 0, 0, 434, + -2, 470, 471, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 561, 0, 539, 0, + 0, 490, 509, 0, 0, 510, 511, 512, 513, 586, + 0, 0, -2, 0, 0, 391, 594, 0, 343, 362, + 364, 0, 359, 374, 375, 377, 0, 379, 0, 381, + 382, 347, 349, 350, 0, 0, 0, 0, 370, 561, + 0, 391, 40, 52, 53, 0, 0, 59, 160, 161, + 0, 207, 0, 0, 0, 178, 185, 185, 181, 186, + 182, 0, 152, 0, 149, 86, 146, 0, 159, 159, + 115, 0, 116, 117, 118, 0, 134, 0, 0, 0, + 0, 636, 66, 212, 857, 225, 226, 227, 228, 229, + 230, 231, 232, 233, 234, 235, 857, 0, 857, 628, + 629, 630, 631, 0, 72, 0, 0, 0, 0, 245, + 293, 294, 295, 575, 0, 26, 391, 0, 334, 545, + 0, 408, 0, 428, 411, 468, 330, 0, 136, 136, + 524, 136, 140, 527, 136, 529, 136, 532, 0, 0, + 0, 0, 0, 0, 544, 0, 0, 0, 0, 0, + 536, 489, 542, 0, 0, 0, 33, 0, 586, 576, + 588, 590, 0, 29, 0, 582, 0, 354, 561, 0, + 0, 356, 363, 0, 0, 357, 0, 358, 378, 380, + 0, 0, 0, 0, 569, 392, 39, 56, 57, 58, + 205, 208, 0, 187, 136, 190, 179, 180, 0, 154, + 0, 151, 137, 111, 112, 157, 158, 156, 0, 156, + 0, 141, 0, 858, 213, 214, 215, 216, 0, 219, + 0, 70, 71, 0, 224, 243, 270, 557, 341, 469, + 412, 472, 521, 156, 525, 526, 528, 530, 531, 533, + 474, 473, 475, 0, 0, 481, 0, 0, 478, 0, + 0, 484, 0, 0, 0, 0, 0, 540, 0, 0, + 0, 34, 0, 591, -2, 0, 0, 0, 46, 0, + 569, 595, 596, 360, 0, 365, 0, 0, 0, 368, + 38, 170, 0, 189, 0, 352, 162, 155, 0, 159, + 135, 159, 0, 0, 64, 0, 73, 74, 0, 0, + 559, 0, 522, 523, 0, 0, 0, 0, 0, 0, + 0, 0, 514, 488, 537, 0, 0, 0, 589, 0, + -2, 0, 584, 583, 355, 37, 0, 0, 0, 0, + 387, 169, 171, 0, 176, 0, 188, 0, 0, 167, + 0, 164, 166, 153, 124, 125, 139, 142, 0, 0, + 0, 0, 28, 0, 0, 476, 477, 482, 483, 479, + 480, 485, 486, 0, 0, 0, 505, 506, 0, 579, + 29, 0, 361, 388, 389, 390, 351, 172, 173, 0, + 177, 175, 0, 353, 85, 0, 163, 165, 0, 237, + 0, 75, 76, 69, 560, 558, 487, 0, 0, 0, + 587, -2, 585, 174, 0, 168, 236, 0, 0, 72, + 515, 0, 518, 0, 238, 0, 223, 516, 0, 0, + 0, 192, 0, 0, 193, 194, 0, 0, 517, 195, + 0, 0, 0, 0, 0, 196, 198, 199, 0, 0, + 197, 239, 240, 200, 201, 202, } var yyTok1 = [...]int{ @@ -2637,7 +2674,7 @@ var yyTok1 = [...]int{ 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 75, 3, 3, 3, 102, 94, 3, 54, 56, 99, 97, 55, 98, 112, 100, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 268, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 269, 83, 82, 84, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, @@ -2672,7 +2709,7 @@ var yyTok2 = [...]int{ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, 258, - 259, 260, 261, 262, 263, 264, 265, 266, 267, + 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, } var yyTok3 = [...]int{ 0, @@ -4476,13 +4513,13 @@ yydefault: yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 262: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:1489 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 263: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1493 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} @@ -4494,39 +4531,45 @@ yydefault: yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 265: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:1501 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 266: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1505 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 267: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:1509 { - yyVAL.statement = &Show{Scope: yyDollar[2].str, Type: string(yyDollar[3].bytes)} + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 268: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:1513 { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} + yyVAL.statement = &Show{Scope: yyDollar[2].str, Type: string(yyDollar[3].bytes)} } case 269: - yyDollar = yyS[yypt-7 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:1517 + { + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} + } + case 270: + yyDollar = yyS[yypt-7 : yypt+1] +//line sql.y:1521 { showTablesOpt := &ShowTablesOpt{Full: yyDollar[2].str, DbName: yyDollar[6].str, Filter: yyDollar[7].showFilter} yyVAL.statement = &Show{Type: string(yyDollar[3].str), ShowTablesOpt: showTablesOpt, OnTable: yyDollar[5].tableName} } - case 270: + case 271: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:1522 +//line sql.y:1526 { // this is ugly, but I couldn't find a better way for now if yyDollar[3].str == "processlist" { @@ -4536,32 +4579,26 @@ yydefault: yyVAL.statement = &Show{Type: yyDollar[3].str, ShowTablesOpt: showTablesOpt} } } - case 271: + case 272: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1532 +//line sql.y:1536 { yyVAL.statement = &Show{Scope: yyDollar[2].str, Type: string(yyDollar[3].bytes)} } - case 272: + case 273: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1536 +//line sql.y:1540 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } - case 273: + case 274: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1540 +//line sql.y:1544 { // Cannot dereference $4 directly, or else the parser stackcannot be pooled. See yyParsePooled showCollationFilterOpt := yyDollar[4].expr yyVAL.statement = &Show{Type: string(yyDollar[2].bytes), ShowCollationFilterOpt: &showCollationFilterOpt} } - case 274: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1546 - { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} - } case 275: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1550 @@ -4581,10 +4618,10 @@ yydefault: yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 278: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1562 { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes) + " " + string(yyDollar[3].bytes)} + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 279: yyDollar = yyS[yypt-3 : yypt+1] @@ -4593,28 +4630,28 @@ yydefault: yyVAL.statement = &Show{Type: string(yyDollar[2].bytes) + " " + string(yyDollar[3].bytes)} } case 280: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:1570 { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes) + " " + string(yyDollar[3].bytes), OnTable: yyDollar[5].tableName} + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes) + " " + string(yyDollar[3].bytes)} } case 281: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] //line sql.y:1574 { - yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes) + " " + string(yyDollar[3].bytes), OnTable: yyDollar[5].tableName} } case 282: - yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1584 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:1578 { yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 283: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1590 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:1588 { - yyVAL.str = string(yyDollar[1].bytes) + yyVAL.statement = &Show{Type: string(yyDollar[2].bytes)} } case 284: yyDollar = yyS[yypt-1 : yypt+1] @@ -4623,22 +4660,22 @@ yydefault: yyVAL.str = string(yyDollar[1].bytes) } case 285: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1600 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:1598 { - yyVAL.str = "" + yyVAL.str = string(yyDollar[1].bytes) } case 286: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:1604 { - yyVAL.str = "full " + yyVAL.str = "" } case 287: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1610 +//line sql.y:1608 { - yyVAL.str = string(yyDollar[1].bytes) + yyVAL.str = "full " } case 288: yyDollar = yyS[yypt-1 : yypt+1] @@ -4647,16 +4684,16 @@ yydefault: yyVAL.str = string(yyDollar[1].bytes) } case 289: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1620 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:1618 { - yyVAL.str = "" + yyVAL.str = string(yyDollar[1].bytes) } case 290: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:1624 { - yyVAL.str = yyDollar[2].tableIdent.v + yyVAL.str = "" } case 291: yyDollar = yyS[yypt-2 : yypt+1] @@ -4665,82 +4702,82 @@ yydefault: yyVAL.str = yyDollar[2].tableIdent.v } case 292: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1634 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:1632 { - yyVAL.showFilter = nil + yyVAL.str = yyDollar[2].tableIdent.v } case 293: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:1638 { - yyVAL.showFilter = &ShowFilter{Like: string(yyDollar[2].bytes)} + yyVAL.showFilter = nil } case 294: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1642 { - yyVAL.showFilter = &ShowFilter{Filter: yyDollar[2].expr} + yyVAL.showFilter = &ShowFilter{Like: string(yyDollar[2].bytes)} } case 295: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1648 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:1646 { - yyVAL.str = "" + yyVAL.showFilter = &ShowFilter{Filter: yyDollar[2].expr} } case 296: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:1652 { - yyVAL.str = SessionStr + yyVAL.str = "" } case 297: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:1656 { - yyVAL.str = GlobalStr + yyVAL.str = SessionStr } case 298: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1662 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:1660 { - yyVAL.statement = &Use{DBName: yyDollar[2].tableIdent} + yyVAL.str = GlobalStr } case 299: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1666 { - yyVAL.statement = &Use{DBName: TableIdent{v: ""}} + yyVAL.statement = &Use{DBName: yyDollar[2].tableIdent} } case 300: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1672 +//line sql.y:1670 { - yyVAL.statement = &Begin{} + yyVAL.statement = &Use{DBName: TableIdent{v: ""}} } case 301: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:1676 { yyVAL.statement = &Begin{} } case 302: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1682 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:1680 { - yyVAL.statement = &Commit{} + yyVAL.statement = &Begin{} } case 303: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1688 +//line sql.y:1686 { - yyVAL.statement = &Rollback{} + yyVAL.statement = &Commit{} } case 304: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1694 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:1692 { - yyVAL.statement = &OtherRead{} + yyVAL.statement = &Rollback{} } case 305: yyDollar = yyS[yypt-2 : yypt+1] @@ -4758,7 +4795,7 @@ yydefault: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1706 { - yyVAL.statement = &OtherAdmin{} + yyVAL.statement = &OtherRead{} } case 308: yyDollar = yyS[yypt-2 : yypt+1] @@ -4767,7 +4804,7 @@ yydefault: yyVAL.statement = &OtherAdmin{} } case 309: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:1714 { yyVAL.statement = &OtherAdmin{} @@ -4779,260 +4816,260 @@ yydefault: yyVAL.statement = &OtherAdmin{} } case 311: + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:1722 + { + yyVAL.statement = &OtherAdmin{} + } + case 312: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1724 +//line sql.y:1728 { yyVAL.statement = &DDL{Action: FlushStr} } - case 312: + case 313: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1728 +//line sql.y:1732 { setAllowComments(yylex, true) } - case 313: + case 314: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1732 +//line sql.y:1736 { yyVAL.bytes2 = yyDollar[2].bytes2 setAllowComments(yylex, false) } - case 314: + case 315: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1738 +//line sql.y:1742 { yyVAL.bytes2 = nil } - case 315: + case 316: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1742 +//line sql.y:1746 { yyVAL.bytes2 = append(yyDollar[1].bytes2, yyDollar[2].bytes) } - case 316: + case 317: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1748 +//line sql.y:1752 { yyVAL.str = UnionStr } - case 317: + case 318: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1752 +//line sql.y:1756 { yyVAL.str = UnionAllStr } - case 318: + case 319: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1756 +//line sql.y:1760 { yyVAL.str = UnionDistinctStr } - case 319: + case 320: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1761 +//line sql.y:1765 { yyVAL.str = "" } - case 320: + case 321: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1765 +//line sql.y:1769 { yyVAL.str = SQLNoCacheStr } - case 321: + case 322: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1769 +//line sql.y:1773 { yyVAL.str = SQLCacheStr } - case 322: + case 323: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1774 +//line sql.y:1778 { yyVAL.str = "" } - case 323: + case 324: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1778 +//line sql.y:1782 { yyVAL.str = DistinctStr } - case 324: + case 325: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1783 +//line sql.y:1787 { yyVAL.str = "" } - case 325: + case 326: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1787 +//line sql.y:1791 { yyVAL.str = StraightJoinHint } - case 326: + case 327: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1792 +//line sql.y:1796 { yyVAL.selectExprs = nil } - case 327: + case 328: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1796 +//line sql.y:1800 { yyVAL.selectExprs = yyDollar[1].selectExprs } - case 328: + case 329: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1802 +//line sql.y:1806 { yyVAL.selectExprs = SelectExprs{yyDollar[1].selectExpr} } - case 329: + case 330: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1806 +//line sql.y:1810 { yyVAL.selectExprs = append(yyVAL.selectExprs, yyDollar[3].selectExpr) } - case 330: + case 331: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1812 +//line sql.y:1816 { yyVAL.selectExpr = &StarExpr{} } - case 331: + case 332: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1816 +//line sql.y:1820 { yyVAL.selectExpr = &AliasedExpr{Expr: yyDollar[1].expr, As: yyDollar[2].colIdent} } - case 332: + case 333: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1820 +//line sql.y:1824 { yyVAL.selectExpr = &StarExpr{TableName: TableName{Name: yyDollar[1].tableIdent}} } - case 333: + case 334: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:1824 +//line sql.y:1828 { yyVAL.selectExpr = &StarExpr{TableName: TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent}} } - case 334: + case 335: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1829 +//line sql.y:1833 { yyVAL.colIdent = ColIdent{} } - case 335: + case 336: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1833 +//line sql.y:1837 { yyVAL.colIdent = yyDollar[1].colIdent } - case 336: + case 337: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1837 +//line sql.y:1841 { yyVAL.colIdent = yyDollar[2].colIdent } - case 338: + case 339: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1844 +//line sql.y:1848 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 339: + case 340: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1849 +//line sql.y:1853 { yyVAL.tableExprs = TableExprs{&AliasedTableExpr{Expr: TableName{Name: NewTableIdent("dual")}}} } - case 340: + case 341: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1853 +//line sql.y:1857 { yyVAL.tableExprs = yyDollar[2].tableExprs } - case 341: + case 342: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1859 +//line sql.y:1863 { yyVAL.tableExprs = TableExprs{yyDollar[1].tableExpr} } - case 342: + case 343: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1863 +//line sql.y:1867 { yyVAL.tableExprs = append(yyVAL.tableExprs, yyDollar[3].tableExpr) } - case 345: + case 346: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1873 +//line sql.y:1877 { yyVAL.tableExpr = yyDollar[1].aliasedTableName } - case 346: + case 347: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1877 +//line sql.y:1881 { yyVAL.tableExpr = &AliasedTableExpr{Expr: yyDollar[1].subquery, As: yyDollar[3].tableIdent} } - case 347: + case 348: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1881 +//line sql.y:1885 { // missed alias for subquery yylex.Error("Every derived table must have its own alias") return 1 } - case 348: + case 349: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1887 +//line sql.y:1891 { yyVAL.tableExpr = &ParenTableExpr{Exprs: yyDollar[2].tableExprs} } - case 349: + case 350: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1893 +//line sql.y:1897 { yyVAL.aliasedTableName = &AliasedTableExpr{Expr: yyDollar[1].tableName, As: yyDollar[2].tableIdent, Hints: yyDollar[3].indexHints} } - case 350: + case 351: yyDollar = yyS[yypt-7 : yypt+1] -//line sql.y:1897 +//line sql.y:1901 { yyVAL.aliasedTableName = &AliasedTableExpr{Expr: yyDollar[1].tableName, Partitions: yyDollar[4].partitions, As: yyDollar[6].tableIdent, Hints: yyDollar[7].indexHints} } - case 351: + case 352: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1903 +//line sql.y:1907 { yyVAL.columns = Columns{yyDollar[1].colIdent} } - case 352: + case 353: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1907 +//line sql.y:1911 { yyVAL.columns = append(yyVAL.columns, yyDollar[3].colIdent) } - case 353: + case 354: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1913 +//line sql.y:1917 { yyVAL.partitions = Partitions{yyDollar[1].colIdent} } - case 354: + case 355: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1917 +//line sql.y:1921 { yyVAL.partitions = append(yyVAL.partitions, yyDollar[3].colIdent) } - case 355: - yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1930 - { - yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} - } case 356: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:1934 @@ -5046,91 +5083,91 @@ yydefault: yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} } case 358: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:1942 { - yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr} + yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr, Condition: yyDollar[4].joinCondition} } case 359: + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:1946 + { + yyVAL.tableExpr = &JoinTableExpr{LeftExpr: yyDollar[1].tableExpr, Join: yyDollar[2].str, RightExpr: yyDollar[3].tableExpr} + } + case 360: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1948 +//line sql.y:1952 { yyVAL.joinCondition = JoinCondition{On: yyDollar[2].expr} } - case 360: + case 361: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1950 +//line sql.y:1954 { yyVAL.joinCondition = JoinCondition{Using: yyDollar[3].columns} } - case 361: + case 362: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1954 +//line sql.y:1958 { yyVAL.joinCondition = JoinCondition{} } - case 362: + case 363: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1956 +//line sql.y:1960 { yyVAL.joinCondition = yyDollar[1].joinCondition } - case 363: + case 364: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1960 +//line sql.y:1964 { yyVAL.joinCondition = JoinCondition{} } - case 364: + case 365: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1962 +//line sql.y:1966 { yyVAL.joinCondition = JoinCondition{On: yyDollar[2].expr} } - case 365: + case 366: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1965 +//line sql.y:1969 { yyVAL.empty = struct{}{} } - case 366: + case 367: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1967 +//line sql.y:1971 { yyVAL.empty = struct{}{} } - case 367: + case 368: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1970 +//line sql.y:1974 { yyVAL.tableIdent = NewTableIdent("") } - case 368: + case 369: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1974 +//line sql.y:1978 { yyVAL.tableIdent = yyDollar[1].tableIdent } - case 369: + case 370: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1978 +//line sql.y:1982 { yyVAL.tableIdent = yyDollar[2].tableIdent } - case 371: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1985 - { - yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) - } case 372: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1991 +//line sql.y:1989 { - yyVAL.str = JoinStr + yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } case 373: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:1995 { yyVAL.str = JoinStr @@ -5142,44 +5179,50 @@ yydefault: yyVAL.str = JoinStr } case 375: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2005 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2003 { - yyVAL.str = StraightJoinStr + yyVAL.str = JoinStr } case 376: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2011 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2009 { - yyVAL.str = LeftJoinStr + yyVAL.str = StraightJoinStr } case 377: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2015 { yyVAL.str = LeftJoinStr } case 378: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2019 { - yyVAL.str = RightJoinStr + yyVAL.str = LeftJoinStr } case 379: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2023 { yyVAL.str = RightJoinStr } case 380: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2029 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:2027 { - yyVAL.str = NaturalJoinStr + yyVAL.str = RightJoinStr } case 381: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2033 + { + yyVAL.str = NaturalJoinStr + } + case 382: + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2037 { if yyDollar[2].str == LeftJoinStr { yyVAL.str = NaturalLeftJoinStr @@ -5187,401 +5230,395 @@ yydefault: yyVAL.str = NaturalRightJoinStr } } - case 382: + case 383: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2043 +//line sql.y:2047 { yyVAL.tableName = yyDollar[2].tableName } - case 383: + case 384: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2047 +//line sql.y:2051 { yyVAL.tableName = yyDollar[1].tableName } - case 384: + case 385: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2053 +//line sql.y:2057 { yyVAL.tableName = TableName{Name: yyDollar[1].tableIdent} } - case 385: + case 386: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2057 +//line sql.y:2061 { yyVAL.tableName = TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent} } - case 386: + case 387: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2062 +//line sql.y:2066 { yyVAL.indexHints = nil } - case 387: + case 388: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2066 +//line sql.y:2070 { yyVAL.indexHints = &IndexHints{Type: UseStr, Indexes: yyDollar[4].columns} } - case 388: + case 389: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2070 +//line sql.y:2074 { yyVAL.indexHints = &IndexHints{Type: IgnoreStr, Indexes: yyDollar[4].columns} } - case 389: + case 390: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2074 +//line sql.y:2078 { yyVAL.indexHints = &IndexHints{Type: ForceStr, Indexes: yyDollar[4].columns} } - case 390: + case 391: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2079 +//line sql.y:2083 { yyVAL.expr = nil } - case 391: + case 392: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2083 +//line sql.y:2087 { yyVAL.expr = yyDollar[2].expr } - case 392: + case 393: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2089 +//line sql.y:2093 { yyVAL.expr = yyDollar[1].expr } - case 393: + case 394: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2093 +//line sql.y:2097 { yyVAL.expr = &AndExpr{Left: yyDollar[1].expr, Right: yyDollar[3].expr} } - case 394: + case 395: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2097 +//line sql.y:2101 { yyVAL.expr = &OrExpr{Left: yyDollar[1].expr, Right: yyDollar[3].expr} } - case 395: + case 396: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2101 +//line sql.y:2105 { yyVAL.expr = &NotExpr{Expr: yyDollar[2].expr} } - case 396: + case 397: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2105 +//line sql.y:2109 { yyVAL.expr = &IsExpr{Operator: yyDollar[3].str, Expr: yyDollar[1].expr} } - case 397: + case 398: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2109 +//line sql.y:2113 { yyVAL.expr = yyDollar[1].expr } - case 398: + case 399: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2113 +//line sql.y:2117 { yyVAL.expr = &Default{ColName: yyDollar[2].str} } - case 399: + case 400: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2119 +//line sql.y:2123 { yyVAL.str = "" } - case 400: + case 401: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2123 +//line sql.y:2127 { yyVAL.str = string(yyDollar[2].bytes) } - case 401: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2129 - { - yyVAL.boolVal = BoolVal(true) - } case 402: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2133 { - yyVAL.boolVal = BoolVal(false) + yyVAL.boolVal = BoolVal(true) } case 403: - yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2139 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2137 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: yyDollar[2].str, Right: yyDollar[3].expr} + yyVAL.boolVal = BoolVal(false) } case 404: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2143 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: InStr, Right: yyDollar[3].colTuple} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: yyDollar[2].str, Right: yyDollar[3].expr} } case 405: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2147 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotInStr, Right: yyDollar[4].colTuple} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: InStr, Right: yyDollar[3].colTuple} } case 406: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2151 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: LikeStr, Right: yyDollar[3].expr, Escape: yyDollar[4].expr} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotInStr, Right: yyDollar[4].colTuple} } case 407: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2155 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotLikeStr, Right: yyDollar[4].expr, Escape: yyDollar[5].expr} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: LikeStr, Right: yyDollar[3].expr, Escape: yyDollar[4].expr} } case 408: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] //line sql.y:2159 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: RegexpStr, Right: yyDollar[3].expr} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotLikeStr, Right: yyDollar[4].expr, Escape: yyDollar[5].expr} } case 409: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2163 { - yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotRegexpStr, Right: yyDollar[4].expr} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: RegexpStr, Right: yyDollar[3].expr} } case 410: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2167 { - yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: BetweenStr, From: yyDollar[3].expr, To: yyDollar[5].expr} + yyVAL.expr = &ComparisonExpr{Left: yyDollar[1].expr, Operator: NotRegexpStr, Right: yyDollar[4].expr} } case 411: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] //line sql.y:2171 { - yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: NotBetweenStr, From: yyDollar[4].expr, To: yyDollar[6].expr} + yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: BetweenStr, From: yyDollar[3].expr, To: yyDollar[5].expr} } case 412: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2175 { - yyVAL.expr = &ExistsExpr{Subquery: yyDollar[2].subquery} + yyVAL.expr = &RangeCond{Left: yyDollar[1].expr, Operator: NotBetweenStr, From: yyDollar[4].expr, To: yyDollar[6].expr} } case 413: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2181 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2179 { - yyVAL.str = IsNullStr + yyVAL.expr = &ExistsExpr{Subquery: yyDollar[2].subquery} } case 414: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2185 { - yyVAL.str = IsNotNullStr + yyVAL.str = IsNullStr } case 415: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2189 { - yyVAL.str = IsTrueStr + yyVAL.str = IsNotNullStr } case 416: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2193 { - yyVAL.str = IsNotTrueStr + yyVAL.str = IsTrueStr } case 417: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2197 { - yyVAL.str = IsFalseStr + yyVAL.str = IsNotTrueStr } case 418: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2201 { - yyVAL.str = IsNotFalseStr + yyVAL.str = IsFalseStr } case 419: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2207 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2205 { - yyVAL.str = EqualStr + yyVAL.str = IsNotFalseStr } case 420: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2211 { - yyVAL.str = LessThanStr + yyVAL.str = EqualStr } case 421: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2215 { - yyVAL.str = GreaterThanStr + yyVAL.str = LessThanStr } case 422: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2219 { - yyVAL.str = LessEqualStr + yyVAL.str = GreaterThanStr } case 423: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2223 { - yyVAL.str = GreaterEqualStr + yyVAL.str = LessEqualStr } case 424: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2227 { - yyVAL.str = NotEqualStr + yyVAL.str = GreaterEqualStr } case 425: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2231 { - yyVAL.str = NullSafeEqualStr + yyVAL.str = NotEqualStr } case 426: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2236 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2235 { - yyVAL.expr = nil + yyVAL.str = NullSafeEqualStr } case 427: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:2240 { - yyVAL.expr = yyDollar[2].expr + yyVAL.expr = nil } case 428: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2246 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2244 { - yyVAL.colTuple = yyDollar[1].valTuple + yyVAL.expr = yyDollar[2].expr } case 429: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2250 { - yyVAL.colTuple = yyDollar[1].subquery + yyVAL.colTuple = yyDollar[1].valTuple } case 430: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2254 { - yyVAL.colTuple = ListArg(yyDollar[1].bytes) + yyVAL.colTuple = yyDollar[1].subquery } case 431: - yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2260 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2258 { - yyVAL.subquery = &Subquery{yyDollar[2].selStmt} + yyVAL.colTuple = ListArg(yyDollar[1].bytes) } case 432: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2266 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:2264 { - yyVAL.exprs = Exprs{yyDollar[1].expr} + yyVAL.subquery = &Subquery{yyDollar[2].selStmt} } case 433: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2270 { - yyVAL.exprs = append(yyDollar[1].exprs, yyDollar[3].expr) + yyVAL.exprs = Exprs{yyDollar[1].expr} } case 434: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2276 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:2274 { - yyVAL.expr = yyDollar[1].expr + yyVAL.exprs = append(yyDollar[1].exprs, yyDollar[3].expr) } case 435: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2280 { - yyVAL.expr = yyDollar[1].boolVal + yyVAL.expr = yyDollar[1].expr } case 436: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2284 { - yyVAL.expr = yyDollar[1].colName + yyVAL.expr = yyDollar[1].boolVal } case 437: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2288 { - yyVAL.expr = yyDollar[1].expr + yyVAL.expr = yyDollar[1].colName } case 438: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2292 { - yyVAL.expr = yyDollar[1].subquery + yyVAL.expr = yyDollar[1].expr } case 439: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2296 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitAndStr, Right: yyDollar[3].expr} + yyVAL.expr = yyDollar[1].subquery } case 440: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2300 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitOrStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitAndStr, Right: yyDollar[3].expr} } case 441: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2304 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitXorStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitOrStr, Right: yyDollar[3].expr} } case 442: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2308 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: PlusStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: BitXorStr, Right: yyDollar[3].expr} } case 443: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2312 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MinusStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: PlusStr, Right: yyDollar[3].expr} } case 444: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2316 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MultStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MinusStr, Right: yyDollar[3].expr} } case 445: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2320 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: DivStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: MultStr, Right: yyDollar[3].expr} } case 446: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2324 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: IntDivStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: DivStr, Right: yyDollar[3].expr} } case 447: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2328 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ModStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: IntDivStr, Right: yyDollar[3].expr} } case 448: yyDollar = yyS[yypt-3 : yypt+1] @@ -5593,53 +5630,59 @@ yydefault: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2336 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftLeftStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ModStr, Right: yyDollar[3].expr} } case 450: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2340 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftRightStr, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftLeftStr, Right: yyDollar[3].expr} } case 451: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2344 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONExtractOp, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].expr, Operator: ShiftRightStr, Right: yyDollar[3].expr} } case 452: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2348 { - yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONUnquoteExtractOp, Right: yyDollar[3].expr} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONExtractOp, Right: yyDollar[3].expr} } case 453: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2352 { - yyVAL.expr = &CollateExpr{Expr: yyDollar[1].expr, Charset: yyDollar[3].str} + yyVAL.expr = &BinaryExpr{Left: yyDollar[1].colName, Operator: JSONUnquoteExtractOp, Right: yyDollar[3].expr} } case 454: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2356 { - yyVAL.expr = &UnaryExpr{Operator: BinaryStr, Expr: yyDollar[2].expr} + yyVAL.expr = &CollateExpr{Expr: yyDollar[1].expr, Charset: yyDollar[3].str} } case 455: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2360 { - yyVAL.expr = &UnaryExpr{Operator: UBinaryStr, Expr: yyDollar[2].expr} + yyVAL.expr = &UnaryExpr{Operator: BinaryStr, Expr: yyDollar[2].expr} } case 456: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2364 { - yyVAL.expr = &UnaryExpr{Operator: Utf8mb4Str, Expr: yyDollar[2].expr} + yyVAL.expr = &UnaryExpr{Operator: UBinaryStr, Expr: yyDollar[2].expr} } case 457: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2368 + { + yyVAL.expr = &UnaryExpr{Operator: Utf8mb4Str, Expr: yyDollar[2].expr} + } + case 458: + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2372 { if num, ok := yyDollar[2].expr.(*SQLVal); ok && num.Type == IntVal { yyVAL.expr = num @@ -5647,9 +5690,9 @@ yydefault: yyVAL.expr = &UnaryExpr{Operator: UPlusStr, Expr: yyDollar[2].expr} } } - case 458: + case 459: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2376 +//line sql.y:2380 { if num, ok := yyDollar[2].expr.(*SQLVal); ok && num.Type == IntVal { // Handle double negative @@ -5663,21 +5706,21 @@ yydefault: yyVAL.expr = &UnaryExpr{Operator: UMinusStr, Expr: yyDollar[2].expr} } } - case 459: + case 460: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2390 +//line sql.y:2394 { yyVAL.expr = &UnaryExpr{Operator: TildaStr, Expr: yyDollar[2].expr} } - case 460: + case 461: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2394 +//line sql.y:2398 { yyVAL.expr = &UnaryExpr{Operator: BangStr, Expr: yyDollar[2].expr} } - case 461: + case 462: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2398 +//line sql.y:2402 { // This rule prevents the usage of INTERVAL // as a function. If support is needed for that, @@ -5685,41 +5728,35 @@ yydefault: // will be non-trivial because of grammar conflicts. yyVAL.expr = &IntervalExpr{Expr: yyDollar[2].expr, Unit: yyDollar[3].colIdent.String()} } - case 466: + case 467: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2416 +//line sql.y:2420 { yyVAL.expr = &FuncExpr{Name: yyDollar[1].colIdent, Exprs: yyDollar[3].selectExprs} } - case 467: + case 468: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2420 +//line sql.y:2424 { yyVAL.expr = &FuncExpr{Name: yyDollar[1].colIdent, Distinct: true, Exprs: yyDollar[4].selectExprs} } - case 468: + case 469: yyDollar = yyS[yypt-6 : yypt+1] -//line sql.y:2424 +//line sql.y:2428 { yyVAL.expr = &FuncExpr{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].colIdent, Exprs: yyDollar[5].selectExprs} } - case 469: - yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2434 - { - yyVAL.expr = &FuncExpr{Name: NewColIdent("left"), Exprs: yyDollar[3].selectExprs} - } case 470: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2438 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("right"), Exprs: yyDollar[3].selectExprs} + yyVAL.expr = &FuncExpr{Name: NewColIdent("left"), Exprs: yyDollar[3].selectExprs} } case 471: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2442 { - yyVAL.expr = &ConvertExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].convertType} + yyVAL.expr = &FuncExpr{Name: NewColIdent("right"), Exprs: yyDollar[3].selectExprs} } case 472: yyDollar = yyS[yypt-6 : yypt+1] @@ -5731,19 +5768,19 @@ yydefault: yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2450 { - yyVAL.expr = &ConvertUsingExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].str} + yyVAL.expr = &ConvertExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].convertType} } case 474: yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2454 { - yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} + yyVAL.expr = &ConvertUsingExpr{Expr: yyDollar[3].expr, Type: yyDollar[5].str} } case 475: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2458 { - yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} + yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} } case 476: yyDollar = yyS[yypt-8 : yypt+1] @@ -5752,16 +5789,16 @@ yydefault: yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } case 477: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line sql.y:2466 { - yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} + yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } case 478: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2470 { - yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} + yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: nil} } case 479: yyDollar = yyS[yypt-8 : yypt+1] @@ -5770,16 +5807,16 @@ yydefault: yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } case 480: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line sql.y:2478 { - yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: nil} + yyVAL.expr = &SubstrExpr{Name: yyDollar[3].colName, From: yyDollar[5].expr, To: yyDollar[7].expr} } case 481: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2482 { - yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} + yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: nil} } case 482: yyDollar = yyS[yypt-8 : yypt+1] @@ -5788,16 +5825,16 @@ yydefault: yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} } case 483: - yyDollar = yyS[yypt-6 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line sql.y:2490 { - yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: nil} + yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} } case 484: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] //line sql.y:2494 { - yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} + yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: nil} } case 485: yyDollar = yyS[yypt-8 : yypt+1] @@ -5806,190 +5843,190 @@ yydefault: yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} } case 486: - yyDollar = yyS[yypt-9 : yypt+1] + yyDollar = yyS[yypt-8 : yypt+1] //line sql.y:2502 { - yyVAL.expr = &MatchExpr{Columns: yyDollar[3].selectExprs, Expr: yyDollar[7].expr, Option: yyDollar[8].str} + yyVAL.expr = &SubstrExpr{StrVal: NewStrVal(yyDollar[3].bytes), From: yyDollar[5].expr, To: yyDollar[7].expr} } case 487: - yyDollar = yyS[yypt-7 : yypt+1] + yyDollar = yyS[yypt-9 : yypt+1] //line sql.y:2506 { - yyVAL.expr = &GroupConcatExpr{Distinct: yyDollar[3].str, Exprs: yyDollar[4].selectExprs, OrderBy: yyDollar[5].orderBy, Separator: yyDollar[6].str} + yyVAL.expr = &MatchExpr{Columns: yyDollar[3].selectExprs, Expr: yyDollar[7].expr, Option: yyDollar[8].str} } case 488: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-7 : yypt+1] //line sql.y:2510 { - yyVAL.expr = &CaseExpr{Expr: yyDollar[2].expr, Whens: yyDollar[3].whens, Else: yyDollar[4].expr} + yyVAL.expr = &GroupConcatExpr{Distinct: yyDollar[3].str, Exprs: yyDollar[4].selectExprs, OrderBy: yyDollar[5].orderBy, Separator: yyDollar[6].str} } case 489: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] //line sql.y:2514 { - yyVAL.expr = &ValuesFuncExpr{Name: yyDollar[3].colName} + yyVAL.expr = &CaseExpr{Expr: yyDollar[2].expr, Whens: yyDollar[3].whens, Else: yyDollar[4].expr} } case 490: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2524 + yyDollar = yyS[yypt-4 : yypt+1] +//line sql.y:2518 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("current_timestamp")} + yyVAL.expr = &ValuesFuncExpr{Name: yyDollar[3].colName} } case 491: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2528 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_timestamp")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("current_timestamp")} } case 492: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2532 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_time")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_timestamp")} } case 493: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2537 +//line sql.y:2536 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_date")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_time")} } case 494: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2542 +//line sql.y:2541 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("localtime")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("utc_date")} } case 495: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2547 +//line sql.y:2546 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("localtimestamp")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("localtime")} } case 496: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2553 +//line sql.y:2551 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("current_date")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("localtimestamp")} } case 497: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2558 +//line sql.y:2557 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("current_time")} + yyVAL.expr = &FuncExpr{Name: NewColIdent("current_date")} } case 498: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2563 +//line sql.y:2562 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("current_timestamp"), Fsp: yyDollar[2].expr} + yyVAL.expr = &FuncExpr{Name: NewColIdent("current_time")} } case 499: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2567 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("utc_timestamp"), Fsp: yyDollar[2].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("current_timestamp"), Fsp: yyDollar[2].expr} } case 500: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2571 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("utc_time"), Fsp: yyDollar[2].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("utc_timestamp"), Fsp: yyDollar[2].expr} } case 501: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2576 +//line sql.y:2575 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("localtime"), Fsp: yyDollar[2].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("utc_time"), Fsp: yyDollar[2].expr} } case 502: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2581 +//line sql.y:2580 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("localtimestamp"), Fsp: yyDollar[2].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("localtime"), Fsp: yyDollar[2].expr} } case 503: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2586 +//line sql.y:2585 { - yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("current_time"), Fsp: yyDollar[2].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("localtimestamp"), Fsp: yyDollar[2].expr} } case 504: - yyDollar = yyS[yypt-8 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2590 { - yyVAL.expr = &TimestampFuncExpr{Name: string("timestampadd"), Unit: yyDollar[3].colIdent.String(), Expr1: yyDollar[5].expr, Expr2: yyDollar[7].expr} + yyVAL.expr = &CurTimeFuncExpr{Name: NewColIdent("current_time"), Fsp: yyDollar[2].expr} } case 505: yyDollar = yyS[yypt-8 : yypt+1] //line sql.y:2594 { - yyVAL.expr = &TimestampFuncExpr{Name: string("timestampdiff"), Unit: yyDollar[3].colIdent.String(), Expr1: yyDollar[5].expr, Expr2: yyDollar[7].expr} + yyVAL.expr = &TimestampFuncExpr{Name: string("timestampadd"), Unit: yyDollar[3].colIdent.String(), Expr1: yyDollar[5].expr, Expr2: yyDollar[7].expr} } - case 508: - yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2604 + case 506: + yyDollar = yyS[yypt-8 : yypt+1] +//line sql.y:2598 { - yyVAL.expr = yyDollar[2].expr + yyVAL.expr = &TimestampFuncExpr{Name: string("timestampdiff"), Unit: yyDollar[3].colIdent.String(), Expr1: yyDollar[5].expr, Expr2: yyDollar[7].expr} } case 509: - yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2614 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:2608 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("if"), Exprs: yyDollar[3].selectExprs} + yyVAL.expr = yyDollar[2].expr } case 510: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2618 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("database"), Exprs: yyDollar[3].selectExprs} + yyVAL.expr = &FuncExpr{Name: NewColIdent("if"), Exprs: yyDollar[3].selectExprs} } case 511: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2622 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("mod"), Exprs: yyDollar[3].selectExprs} + yyVAL.expr = &FuncExpr{Name: NewColIdent("database"), Exprs: yyDollar[3].selectExprs} } case 512: yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2626 { - yyVAL.expr = &FuncExpr{Name: NewColIdent("replace"), Exprs: yyDollar[3].selectExprs} + yyVAL.expr = &FuncExpr{Name: NewColIdent("mod"), Exprs: yyDollar[3].selectExprs} } case 513: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2632 + yyDollar = yyS[yypt-4 : yypt+1] +//line sql.y:2630 { - yyVAL.str = "" + yyVAL.expr = &FuncExpr{Name: NewColIdent("replace"), Exprs: yyDollar[3].selectExprs} } case 514: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-0 : yypt+1] //line sql.y:2636 { - yyVAL.str = BooleanModeStr + yyVAL.str = "" } case 515: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2640 { - yyVAL.str = NaturalLanguageModeStr + yyVAL.str = BooleanModeStr } case 516: - yyDollar = yyS[yypt-7 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line sql.y:2644 { - yyVAL.str = NaturalLanguageModeWithQueryExpansionStr + yyVAL.str = NaturalLanguageModeStr } case 517: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-7 : yypt+1] //line sql.y:2648 { - yyVAL.str = QueryExpansionStr + yyVAL.str = NaturalLanguageModeWithQueryExpansionStr } case 518: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2654 + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:2652 { - yyVAL.str = string(yyDollar[1].bytes) + yyVAL.str = QueryExpansionStr } case 519: yyDollar = yyS[yypt-1 : yypt+1] @@ -5998,63 +6035,63 @@ yydefault: yyVAL.str = string(yyDollar[1].bytes) } case 520: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2664 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2662 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} + yyVAL.str = string(yyDollar[1].bytes) } case 521: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2668 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal, Charset: yyDollar[3].str, Operator: CharacterSetStr} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} } case 522: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2672 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal, Charset: string(yyDollar[3].bytes)} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal, Charset: yyDollar[3].str, Operator: CharacterSetStr} } case 523: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2676 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal, Charset: string(yyDollar[3].bytes)} } case 524: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2680 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } case 525: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2684 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} - yyVAL.convertType.Length = yyDollar[2].LengthScaleOption.Length - yyVAL.convertType.Scale = yyDollar[2].LengthScaleOption.Scale + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} } case 526: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2690 + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2688 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} + yyVAL.convertType.Length = yyDollar[2].LengthScaleOption.Length + yyVAL.convertType.Scale = yyDollar[2].LengthScaleOption.Scale } case 527: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2694 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } case 528: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2698 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} } case 529: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2702 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} @@ -6063,143 +6100,149 @@ yydefault: yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2706 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } case 531: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:2710 { - yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes), Length: yyDollar[2].sqlVal} } case 532: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2714 { yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} } case 533: + yyDollar = yyS[yypt-2 : yypt+1] +//line sql.y:2718 + { + yyVAL.convertType = &ConvertType{Type: string(yyDollar[1].bytes)} + } + case 534: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2719 +//line sql.y:2723 { yyVAL.expr = nil } - case 534: + case 535: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2723 +//line sql.y:2727 { yyVAL.expr = yyDollar[1].expr } - case 535: + case 536: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2728 +//line sql.y:2732 { yyVAL.str = string("") } - case 536: + case 537: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2732 +//line sql.y:2736 { yyVAL.str = " separator '" + string(yyDollar[2].bytes) + "'" } - case 537: + case 538: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2738 +//line sql.y:2742 { yyVAL.whens = []*When{yyDollar[1].when} } - case 538: + case 539: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2742 +//line sql.y:2746 { yyVAL.whens = append(yyDollar[1].whens, yyDollar[2].when) } - case 539: + case 540: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2748 +//line sql.y:2752 { yyVAL.when = &When{Cond: yyDollar[2].expr, Val: yyDollar[4].expr} } - case 540: + case 541: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2753 +//line sql.y:2757 { yyVAL.expr = nil } - case 541: + case 542: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2757 +//line sql.y:2761 { yyVAL.expr = yyDollar[2].expr } - case 542: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2763 - { - yyVAL.colName = &ColName{Name: yyDollar[1].colIdent} - } case 543: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2767 { - yyVAL.colName = &ColName{Qualifier: TableName{Name: yyDollar[1].tableIdent}, Name: yyDollar[3].colIdent} + yyVAL.colName = &ColName{Name: yyDollar[1].colIdent} } case 544: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:2771 { - yyVAL.colName = &ColName{Qualifier: TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent}, Name: yyDollar[5].colIdent} + yyVAL.colName = &ColName{Qualifier: TableName{Name: yyDollar[1].tableIdent}, Name: yyDollar[3].colIdent} } case 545: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2777 + yyDollar = yyS[yypt-5 : yypt+1] +//line sql.y:2775 { - yyVAL.expr = NewStrVal(yyDollar[1].bytes) + yyVAL.colName = &ColName{Qualifier: TableName{Qualifier: yyDollar[1].tableIdent, Name: yyDollar[3].tableIdent}, Name: yyDollar[5].colIdent} } case 546: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2781 { - yyVAL.expr = NewHexVal(yyDollar[1].bytes) + yyVAL.expr = NewStrVal(yyDollar[1].bytes) } case 547: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2785 { - yyVAL.expr = NewBitVal(yyDollar[1].bytes) + yyVAL.expr = NewHexVal(yyDollar[1].bytes) } case 548: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2789 { - yyVAL.expr = NewIntVal(yyDollar[1].bytes) + yyVAL.expr = NewBitVal(yyDollar[1].bytes) } case 549: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2793 { - yyVAL.expr = NewFloatVal(yyDollar[1].bytes) + yyVAL.expr = NewIntVal(yyDollar[1].bytes) } case 550: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2797 { - yyVAL.expr = NewHexNum(yyDollar[1].bytes) + yyVAL.expr = NewFloatVal(yyDollar[1].bytes) } case 551: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2801 { - yyVAL.expr = NewValArg(yyDollar[1].bytes) + yyVAL.expr = NewHexNum(yyDollar[1].bytes) } case 552: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:2805 { - yyVAL.expr = &NullVal{} + yyVAL.expr = NewValArg(yyDollar[1].bytes) } case 553: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2811 +//line sql.y:2809 + { + yyVAL.expr = &NullVal{} + } + case 554: + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:2815 { // TODO(sougou): Deprecate this construct. if yyDollar[1].colIdent.Lowered() != "value" { @@ -6208,239 +6251,239 @@ yydefault: } yyVAL.expr = NewIntVal([]byte("1")) } - case 554: + case 555: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2820 +//line sql.y:2824 { yyVAL.expr = NewIntVal(yyDollar[1].bytes) } - case 555: + case 556: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2824 +//line sql.y:2828 { yyVAL.expr = NewValArg(yyDollar[1].bytes) } - case 556: + case 557: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2829 +//line sql.y:2833 { yyVAL.exprs = nil } - case 557: + case 558: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2833 +//line sql.y:2837 { yyVAL.exprs = yyDollar[3].exprs } - case 558: + case 559: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2838 +//line sql.y:2842 { yyVAL.expr = nil } - case 559: + case 560: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2842 +//line sql.y:2846 { yyVAL.expr = yyDollar[2].expr } - case 560: + case 561: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2847 +//line sql.y:2851 { yyVAL.orderBy = nil } - case 561: + case 562: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2851 +//line sql.y:2855 { yyVAL.orderBy = yyDollar[3].orderBy } - case 562: + case 563: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2857 +//line sql.y:2861 { yyVAL.orderBy = OrderBy{yyDollar[1].order} } - case 563: + case 564: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2861 +//line sql.y:2865 { yyVAL.orderBy = append(yyDollar[1].orderBy, yyDollar[3].order) } - case 564: + case 565: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2867 +//line sql.y:2871 { yyVAL.order = &Order{Expr: yyDollar[1].expr, Direction: yyDollar[2].str} } - case 565: + case 566: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2872 +//line sql.y:2876 { yyVAL.str = AscScr } - case 566: + case 567: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2876 +//line sql.y:2880 { yyVAL.str = AscScr } - case 567: + case 568: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2880 +//line sql.y:2884 { yyVAL.str = DescScr } - case 568: + case 569: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2885 +//line sql.y:2889 { yyVAL.limit = nil } - case 569: + case 570: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2889 +//line sql.y:2893 { yyVAL.limit = &Limit{Rowcount: yyDollar[2].expr} } - case 570: + case 571: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2893 +//line sql.y:2897 { yyVAL.limit = &Limit{Offset: yyDollar[2].expr, Rowcount: yyDollar[4].expr} } - case 571: + case 572: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2897 +//line sql.y:2901 { yyVAL.limit = &Limit{Offset: yyDollar[4].expr, Rowcount: yyDollar[2].expr} } - case 572: + case 573: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2902 +//line sql.y:2906 { yyVAL.str = "" } - case 573: + case 574: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2906 +//line sql.y:2910 { yyVAL.str = ForUpdateStr } - case 574: + case 575: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2910 +//line sql.y:2914 { yyVAL.str = ShareModeStr } - case 575: + case 576: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2923 +//line sql.y:2927 { yyVAL.ins = &Insert{Rows: yyDollar[2].values} } - case 576: + case 577: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2927 +//line sql.y:2931 { yyVAL.ins = &Insert{Rows: yyDollar[1].selStmt} } - case 577: + case 578: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2931 +//line sql.y:2935 { // Drop the redundant parenthesis. yyVAL.ins = &Insert{Rows: yyDollar[2].selStmt} } - case 578: + case 579: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2936 +//line sql.y:2940 { yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[5].values} } - case 579: + case 580: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:2940 +//line sql.y:2944 { yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[4].selStmt} } - case 580: + case 581: yyDollar = yyS[yypt-6 : yypt+1] -//line sql.y:2944 +//line sql.y:2948 { // Drop the redundant parenthesis. yyVAL.ins = &Insert{Columns: yyDollar[2].columns, Rows: yyDollar[5].selStmt} } - case 581: + case 582: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2951 +//line sql.y:2955 { yyVAL.columns = Columns{yyDollar[1].colIdent} } - case 582: + case 583: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2955 +//line sql.y:2959 { yyVAL.columns = Columns{yyDollar[3].colIdent} } - case 583: + case 584: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2959 +//line sql.y:2963 { yyVAL.columns = append(yyVAL.columns, yyDollar[3].colIdent) } - case 584: + case 585: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2963 +//line sql.y:2967 { yyVAL.columns = append(yyVAL.columns, yyDollar[5].colIdent) } - case 585: + case 586: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2968 +//line sql.y:2972 { yyVAL.updateExprs = nil } - case 586: + case 587: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2972 +//line sql.y:2976 { yyVAL.updateExprs = yyDollar[5].updateExprs } - case 587: + case 588: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2978 +//line sql.y:2982 { yyVAL.values = Values{yyDollar[1].valTuple} } - case 588: + case 589: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2982 +//line sql.y:2986 { yyVAL.values = append(yyDollar[1].values, yyDollar[3].valTuple) } - case 589: + case 590: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2988 +//line sql.y:2992 { yyVAL.valTuple = yyDollar[1].valTuple } - case 590: + case 591: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2992 +//line sql.y:2996 { yyVAL.valTuple = ValTuple{} } - case 591: + case 592: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2998 +//line sql.y:3002 { yyVAL.valTuple = ValTuple(yyDollar[2].exprs) } - case 592: + case 593: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3004 +//line sql.y:3008 { if len(yyDollar[1].valTuple) == 1 { yyVAL.expr = &ParenExpr{yyDollar[1].valTuple[0]} @@ -6448,195 +6491,189 @@ yydefault: yyVAL.expr = yyDollar[1].valTuple } } - case 593: + case 594: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3014 +//line sql.y:3018 { yyVAL.updateExprs = UpdateExprs{yyDollar[1].updateExpr} } - case 594: + case 595: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3018 +//line sql.y:3022 { yyVAL.updateExprs = append(yyDollar[1].updateExprs, yyDollar[3].updateExpr) } - case 595: + case 596: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3024 +//line sql.y:3028 { yyVAL.updateExpr = &UpdateExpr{Name: yyDollar[1].colName, Expr: yyDollar[3].expr} } - case 596: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3030 - { - yyVAL.setExprs = SetExprs{yyDollar[1].setExpr} - } case 597: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:3034 { - yyVAL.setExprs = append(yyDollar[1].setExprs, yyDollar[3].setExpr) + yyVAL.setExprs = SetExprs{yyDollar[1].setExpr} } case 598: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3040 +//line sql.y:3038 { - yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: NewStrVal([]byte("on"))} + yyVAL.setExprs = append(yyDollar[1].setExprs, yyDollar[3].setExpr) } case 599: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:3044 { - yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: NewStrVal([]byte("off"))} + yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: NewStrVal([]byte("on"))} } case 600: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:3048 { - yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: yyDollar[3].expr} + yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: NewStrVal([]byte("off"))} } case 601: yyDollar = yyS[yypt-3 : yypt+1] //line sql.y:3052 + { + yyVAL.setExpr = &SetExpr{Name: yyDollar[1].colIdent, Expr: yyDollar[3].expr} + } + case 602: + yyDollar = yyS[yypt-3 : yypt+1] +//line sql.y:3056 { yyVAL.setExpr = &SetExpr{Name: NewColIdent(string(yyDollar[1].bytes)), Expr: yyDollar[2].expr} } - case 603: + case 604: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:3059 +//line sql.y:3063 { yyVAL.bytes = []byte("charset") } - case 605: + case 606: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3066 +//line sql.y:3070 { yyVAL.expr = NewStrVal([]byte(yyDollar[1].colIdent.String())) } - case 606: + case 607: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3070 +//line sql.y:3074 { yyVAL.expr = NewStrVal(yyDollar[1].bytes) } - case 607: + case 608: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3074 +//line sql.y:3078 { yyVAL.expr = &Default{} } - case 610: + case 611: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3083 +//line sql.y:3087 { yyVAL.byt = 0 } - case 611: + case 612: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:3085 +//line sql.y:3089 { yyVAL.byt = 1 } - case 612: + case 613: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3088 +//line sql.y:3092 { yyVAL.empty = struct{}{} } - case 613: + case 614: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3090 +//line sql.y:3094 { yyVAL.empty = struct{}{} } - case 614: + case 615: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3093 +//line sql.y:3097 { yyVAL.str = "" } - case 615: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3095 - { - yyVAL.str = IgnoreStr - } case 616: yyDollar = yyS[yypt-1 : yypt+1] //line sql.y:3099 { - yyVAL.empty = struct{}{} + yyVAL.str = IgnoreStr } case 617: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3101 +//line sql.y:3103 { yyVAL.empty = struct{}{} } case 618: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3103 +//line sql.y:3105 { yyVAL.empty = struct{}{} } case 619: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3105 +//line sql.y:3107 { yyVAL.empty = struct{}{} } case 620: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3107 +//line sql.y:3109 { yyVAL.empty = struct{}{} } case 621: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3109 +//line sql.y:3111 { yyVAL.empty = struct{}{} } case 622: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3111 +//line sql.y:3113 { yyVAL.empty = struct{}{} } case 623: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3113 +//line sql.y:3115 { yyVAL.empty = struct{}{} } case 624: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3115 +//line sql.y:3117 { yyVAL.empty = struct{}{} } case 625: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3117 +//line sql.y:3119 { yyVAL.empty = struct{}{} } case 626: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3120 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:3121 { yyVAL.empty = struct{}{} } case 627: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3122 + yyDollar = yyS[yypt-0 : yypt+1] +//line sql.y:3124 { yyVAL.empty = struct{}{} } case 628: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3124 +//line sql.y:3126 { yyVAL.empty = struct{}{} } @@ -6648,45 +6685,45 @@ yydefault: } case 630: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3130 +//line sql.y:3132 { yyVAL.empty = struct{}{} } case 631: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3133 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:3134 { yyVAL.empty = struct{}{} } case 632: - yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3135 + yyDollar = yyS[yypt-0 : yypt+1] +//line sql.y:3137 { yyVAL.empty = struct{}{} } case 633: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3137 +//line sql.y:3139 { yyVAL.empty = struct{}{} } case 634: - yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3140 + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:3141 { - yyVAL.colIdent = ColIdent{} + yyVAL.empty = struct{}{} } case 635: - yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:3142 + yyDollar = yyS[yypt-0 : yypt+1] +//line sql.y:3144 { - yyVAL.colIdent = yyDollar[2].colIdent + yyVAL.colIdent = ColIdent{} } case 636: - yyDollar = yyS[yypt-1 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] //line sql.y:3146 { - yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) + yyVAL.colIdent = yyDollar[2].colIdent } case 637: yyDollar = yyS[yypt-1 : yypt+1] @@ -6694,17 +6731,17 @@ yydefault: { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } - case 639: + case 638: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3157 +//line sql.y:3154 { yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } case 640: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3163 +//line sql.y:3161 { - yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) + yyVAL.colIdent = NewColIdent(string(yyDollar[1].bytes)) } case 641: yyDollar = yyS[yypt-1 : yypt+1] @@ -6712,48 +6749,54 @@ yydefault: { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 643: + case 642: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3174 +//line sql.y:3171 { yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) } - case 853: + case 644: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3409 +//line sql.y:3178 + { + yyVAL.tableIdent = NewTableIdent(string(yyDollar[1].bytes)) + } + case 855: + yyDollar = yyS[yypt-1 : yypt+1] +//line sql.y:3414 { if incNesting(yylex) { yylex.Error("max nesting level reached") return 1 } } - case 854: + case 856: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3418 +//line sql.y:3423 { decNesting(yylex) } - case 855: + case 857: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3423 +//line sql.y:3428 { skipToEnd(yylex) } - case 856: + case 858: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:3428 +//line sql.y:3433 { skipToEnd(yylex) } - case 857: + case 859: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3432 +//line sql.y:3437 { skipToEnd(yylex) } - case 858: + case 860: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3436 +//line sql.y:3441 { skipToEnd(yylex) } diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index f657ab8326a..a14d032ae0f 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -181,7 +181,7 @@ func skipToEnd(yylex interface{}) { %token NULLX AUTO_INCREMENT APPROXNUM SIGNED UNSIGNED ZEROFILL // Supported SHOW tokens -%token COLLATION DATABASES TABLES VITESS_KEYSPACES VITESS_SHARDS VITESS_TABLETS VSCHEMA VSCHEMA_TABLES VITESS_TARGET FULL PROCESSLIST COLUMNS FIELDS ENGINES PLUGINS +%token COLLATION DATABASES SCHEMAS TABLES VITESS_KEYSPACES VITESS_SHARDS VITESS_TABLETS VSCHEMA VSCHEMA_TABLES VITESS_TARGET FULL PROCESSLIST COLUMNS FIELDS ENGINES PLUGINS // SET tokens %token NAMES CHARSET GLOBAL SESSION ISOLATION LEVEL READ WRITE ONLY REPEATABLE COMMITTED UNCOMMITTED SERIALIZABLE @@ -1485,6 +1485,10 @@ show_statement: { $$ = &Show{Type: string($2)} } +| SHOW SCHEMAS ddl_skip_to_end + { + $$ = &Show{Type: string($2)} + } | SHOW ENGINES { $$ = &Show{Type: string($2)} @@ -3364,6 +3368,7 @@ non_reserved_keyword: | REPEATABLE | RESTRICT | ROLLBACK +| SCHEMAS | SESSION | SERIALIZABLE | SHARE diff --git a/go/vt/sqlparser/token.go b/go/vt/sqlparser/token.go index dee429af007..bc6a38adf37 100644 --- a/go/vt/sqlparser/token.go +++ b/go/vt/sqlparser/token.go @@ -318,7 +318,7 @@ var keywords = map[string]int{ "rlike": REGEXP, "rollback": ROLLBACK, "schema": SCHEMA, - "schemas": UNUSED, + "schemas": SCHEMAS, "second_microsecond": UNUSED, "select": SELECT, "sensitive": UNUSED, diff --git a/go/vt/topo/etcd2topo/election.go b/go/vt/topo/etcd2topo/election.go index 1311546b7dc..354c5e49932 100644 --- a/go/vt/topo/etcd2topo/election.go +++ b/go/vt/topo/etcd2topo/election.go @@ -75,16 +75,14 @@ func (mp *etcdMasterParticipation) WaitForMastership() (context.Context, error) // we just cancel that context. lockCtx, lockCancel := context.WithCancel(context.Background()) go func() { - select { - case <-mp.stop: - if ld != nil { - if err := ld.Unlock(context.Background()); err != nil { - log.Errorf("failed to unlock electionPath %v: %v", electionPath, err) - } + <-mp.stop + if ld != nil { + if err := ld.Unlock(context.Background()); err != nil { + log.Errorf("failed to unlock electionPath %v: %v", electionPath, err) } - lockCancel() - close(mp.done) } + lockCancel() + close(mp.done) }() // Try to get the mastership, by getting a lock. diff --git a/go/vt/topo/etcd2topo/server_test.go b/go/vt/topo/etcd2topo/server_test.go index a6d1fc28c67..beccd612b67 100644 --- a/go/vt/topo/etcd2topo/server_test.go +++ b/go/vt/topo/etcd2topo/server_test.go @@ -80,7 +80,7 @@ func startEtcd(t *testing.T) (*exec.Cmd, string, string) { if _, err := cli.Get(ctx, "/"); err == nil { break } - if time.Now().Sub(start) > 10*time.Second { + if time.Since(start) > 10*time.Second { t.Fatalf("Failed to start etcd daemon in time") } time.Sleep(10 * time.Millisecond) diff --git a/go/vt/topo/helpers/compare.go b/go/vt/topo/helpers/compare.go index a30777c8d6e..f14844f01c5 100644 --- a/go/vt/topo/helpers/compare.go +++ b/go/vt/topo/helpers/compare.go @@ -21,6 +21,7 @@ package helpers import ( "reflect" + "github.com/golang/protobuf/proto" "golang.org/x/net/context" "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/topo" @@ -179,3 +180,19 @@ func CompareShardReplications(ctx context.Context, fromTS, toTS *topo.Server) er } return nil } + +// CompareRoutingRules will compare the routing rules in the destination topo. +func CompareRoutingRules(ctx context.Context, fromTS, toTS *topo.Server) error { + rrFrom, err := fromTS.GetRoutingRules(ctx) + if err != nil { + return vterrors.Wrapf(err, "GetKeyspace(from)") + } + rrTo, err := toTS.GetRoutingRules(ctx) + if err != nil { + return vterrors.Wrapf(err, "GetKeyspace(to)") + } + if !proto.Equal(rrFrom, rrTo) { + return vterrors.Errorf(vtrpc.Code_FAILED_PRECONDITION, "routing rules: %v does not match %v", rrFrom, rrTo) + } + return nil +} diff --git a/go/vt/topo/helpers/compare_test.go b/go/vt/topo/helpers/compare_test.go index cda8f325a8d..45ea73adab1 100644 --- a/go/vt/topo/helpers/compare_test.go +++ b/go/vt/topo/helpers/compare_test.go @@ -77,4 +77,16 @@ func TestBasicCompare(t *testing.T) { if err != nil { t.Fatalf("Compare tablets failed: %v", err) } + + err = CompareRoutingRules(ctx, fromTS, toTS) + if err == nil { + t.Fatalf("Compare routing rules is not failing when topos are not in sync") + } + + CopyRoutingRules(ctx, fromTS, toTS) + + err = CompareRoutingRules(ctx, fromTS, toTS) + if err != nil { + t.Fatalf("Compare routing rules failed: %v", err) + } } diff --git a/go/vt/topo/helpers/copy.go b/go/vt/topo/helpers/copy.go index 0e6f2825f24..f23101a79c7 100644 --- a/go/vt/topo/helpers/copy.go +++ b/go/vt/topo/helpers/copy.go @@ -175,3 +175,14 @@ func CopyShardReplications(ctx context.Context, fromTS, toTS *topo.Server) { } } } + +// CopyRoutingRules will create the routing rules in the destination topo. +func CopyRoutingRules(ctx context.Context, fromTS, toTS *topo.Server) { + rr, err := fromTS.GetRoutingRules(ctx) + if err != nil { + log.Fatalf("GetRoutingRules: %v", err) + } + if err := toTS.SaveRoutingRules(ctx, rr); err != nil { + log.Errorf("SaveRoutingRules(%v): %v", rr, err) + } +} diff --git a/go/vt/topo/helpers/copy_test.go b/go/vt/topo/helpers/copy_test.go index 218249fe5e8..870ad2d3847 100644 --- a/go/vt/topo/helpers/copy_test.go +++ b/go/vt/topo/helpers/copy_test.go @@ -26,6 +26,7 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" ) func createSetup(ctx context.Context, t *testing.T) (*topo.Server, *topo.Server) { @@ -86,6 +87,16 @@ func createSetup(ctx context.Context, t *testing.T) (*topo.Server, *topo.Server) t.Fatalf("cannot create slave tablet: %v", err) } + rr := &vschemapb.RoutingRules{ + Rules: []*vschemapb.RoutingRule{{ + FromTable: "t1", + ToTables: []string{"t2", "t3"}, + }}, + } + if err := fromTS.SaveRoutingRules(ctx, rr); err != nil { + t.Fatalf("cannot save routing rules: %v", err) + } + return fromTS, toTS } @@ -116,12 +127,12 @@ func TestBasic(t *testing.T) { CopyShards(ctx, fromTS, toTS) // check ShardReplication copy - sr, err := fromTS.GetShardReplication(ctx, "test_cell", "test_keyspace", "0") + _, err = fromTS.GetShardReplication(ctx, "test_cell", "test_keyspace", "0") if err != nil { t.Fatalf("fromTS.GetShardReplication failed: %v", err) } CopyShardReplications(ctx, fromTS, toTS) - sr, err = toTS.GetShardReplication(ctx, "test_cell", "test_keyspace", "0") + sr, err := toTS.GetShardReplication(ctx, "test_cell", "test_keyspace", "0") if err != nil { t.Fatalf("toTS.GetShardReplication failed: %v", err) } diff --git a/go/vt/topo/keyspace.go b/go/vt/topo/keyspace.go index 96a58fd970b..3519c1ebc92 100644 --- a/go/vt/topo/keyspace.go +++ b/go/vt/topo/keyspace.go @@ -255,6 +255,20 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string) return result, nil } +// GetOnlyShard returns the single ShardInfo of an unsharded keyspace. +func (ts *Server) GetOnlyShard(ctx context.Context, keyspace string) (*ShardInfo, error) { + allShards, err := ts.FindAllShardsInKeyspace(ctx, keyspace) + if err != nil { + return nil, err + } + if len(allShards) == 1 { + for _, s := range allShards { + return s, nil + } + } + return nil, vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "keyspace %s must have one and only one shard: %v", keyspace, allShards) +} + // DeleteKeyspace wraps the underlying Conn.Delete // and dispatches the event. func (ts *Server) DeleteKeyspace(ctx context.Context, keyspace string) error { diff --git a/go/vt/topo/locks.go b/go/vt/topo/locks.go index a52ed2ef254..574208e1b1a 100644 --- a/go/vt/topo/locks.go +++ b/go/vt/topo/locks.go @@ -45,7 +45,7 @@ var ( // Deprecated // LockTimeout is the command line flag that introduces a shorter // timeout for locking topology structures. - deprecatedLockTimeout = flag.Duration("lock_timeout", defaultLockTimeout, "deprecated: timeout for acquiring topology locks, use remote_operation_timeout") + _ = flag.Duration("lock_timeout", defaultLockTimeout, "deprecated: timeout for acquiring topology locks, use remote_operation_timeout") // RemoteOperationTimeout is used for operations where we have to // call out to another process. diff --git a/go/vt/topo/memorytopo/election.go b/go/vt/topo/memorytopo/election.go index d778e7792a6..f66154a02de 100644 --- a/go/vt/topo/memorytopo/election.go +++ b/go/vt/topo/memorytopo/election.go @@ -83,16 +83,14 @@ func (mp *cMasterParticipation) WaitForMastership() (context.Context, error) { // we just cancel that context. lockCtx, lockCancel := context.WithCancel(context.Background()) go func() { - select { - case <-mp.stop: - if ld != nil { - if err := ld.Unlock(context.Background()); err != nil { - log.Errorf("failed to unlock LockDescriptor %v: %v", electionPath, err) - } + <-mp.stop + if ld != nil { + if err := ld.Unlock(context.Background()); err != nil { + log.Errorf("failed to unlock LockDescriptor %v: %v", electionPath, err) } - lockCancel() - close(mp.done) } + lockCancel() + close(mp.done) }() // Try to get the mastership, by getting a lock. diff --git a/go/vt/topo/memorytopo/file.go b/go/vt/topo/memorytopo/file.go index d1322f8c4da..807d725b204 100644 --- a/go/vt/topo/memorytopo/file.go +++ b/go/vt/topo/memorytopo/file.go @@ -167,6 +167,7 @@ func (c *Conn) Delete(ctx context.Context, filePath string, version topo.Version // Check if it's a directory. if n.isDirectory() { + //lint:ignore ST1005 Delete is a function name return fmt.Errorf("Delete(%v, %v) failed: it's a directory", c.cell, filePath) } diff --git a/go/vt/topo/server.go b/go/vt/topo/server.go index 5b390e8673b..cdaa2446838 100644 --- a/go/vt/topo/server.go +++ b/go/vt/topo/server.go @@ -74,6 +74,7 @@ const ( TabletFile = "Tablet" SrvVSchemaFile = "SrvVSchema" SrvKeyspaceFile = "SrvKeyspace" + RoutingRulesFile = "RoutingRules" ) // Path for all object types. diff --git a/go/vt/topo/srv_keyspace.go b/go/vt/topo/srv_keyspace.go index 7f895741d0d..5acd78064d1 100644 --- a/go/vt/topo/srv_keyspace.go +++ b/go/vt/topo/srv_keyspace.go @@ -676,7 +676,7 @@ func OrderAndCheckPartitions(cell string, srvKeyspace *topodatapb.SrvKeyspace) e // this is the custom sharding case, all KeyRanges must be nil continue } - if bytes.Compare(currShard.KeyRange.End, nextShard.KeyRange.Start) != 0 { + if !bytes.Equal(currShard.KeyRange.End, nextShard.KeyRange.Start) { return fmt.Errorf("non-contiguous KeyRange values for %v in cell %v at shard %v to %v: %v != %v", tabletType, cell, i, i+1, hex.EncodeToString(currShard.KeyRange.End), hex.EncodeToString(nextShard.KeyRange.Start)) } } diff --git a/go/vt/topo/test/file.go b/go/vt/topo/test/file.go index 68ea7df5b2b..bdbc2749c00 100644 --- a/go/vt/topo/test/file.go +++ b/go/vt/topo/test/file.go @@ -60,13 +60,13 @@ func checkFileInCell(t *testing.T, conn topo.Conn, hasCells bool) { checkListDir(ctx, t, conn, "/", expected) // Get with no file -> ErrNoNode. - contents, version, err := conn.Get(ctx, "/myfile") + _, _, err := conn.Get(ctx, "/myfile") if !topo.IsErrType(err, topo.NoNode) { t.Errorf("Get(non-existent) didn't return ErrNoNode but: %v", err) } // Create a file. - version, err = conn.Create(ctx, "/myfile", []byte{'a'}) + version, err := conn.Create(ctx, "/myfile", []byte{'a'}) if err != nil { t.Fatalf("Create('/myfile') failed: %v", err) } diff --git a/go/vt/topo/test/lock.go b/go/vt/topo/test/lock.go index f7e27c58740..315653ff000 100644 --- a/go/vt/topo/test/lock.go +++ b/go/vt/topo/test/lock.go @@ -142,10 +142,10 @@ func checkLockUnblocks(ctx context.Context, t *testing.T, conn topo.Conn) { <-unblock lockDescriptor, err := conn.Lock(ctx, keyspacePath, "unblocks") if err != nil { - t.Fatalf("Lock(test_keyspace) failed: %v", err) + t.Errorf("Lock(test_keyspace) failed: %v", err) } if err = lockDescriptor.Unlock(ctx); err != nil { - t.Fatalf("Unlock(test_keyspace): %v", err) + t.Errorf("Unlock(test_keyspace): %v", err) } close(finished) }() diff --git a/go/vt/topo/test/testing.go b/go/vt/topo/test/testing.go index 1bf5df48f57..9cc2f035ec0 100644 --- a/go/vt/topo/test/testing.go +++ b/go/vt/topo/test/testing.go @@ -86,6 +86,11 @@ func TopoServerTestSuite(t *testing.T, factory func() *topo.Server) { checkVSchema(t, ts) ts.Close() + t.Log("=== checkRoutingRules") + ts = factory() + checkRoutingRules(t, ts) + ts.Close() + t.Log("=== checkElection") ts = factory() checkElection(t, ts) diff --git a/go/vt/topo/test/vschema.go b/go/vt/topo/test/vschema.go index 51f7478a8b2..220f12c5888 100644 --- a/go/vt/topo/test/vschema.go +++ b/go/vt/topo/test/vschema.go @@ -38,8 +38,7 @@ func checkVSchema(t *testing.T, ts *topo.Server) { t.Fatalf("CreateShard: %v", err) } - got, err := ts.GetVSchema(ctx, "test_keyspace") - want := &vschemapb.Keyspace{} + _, err := ts.GetVSchema(ctx, "test_keyspace") if !topo.IsErrType(err, topo.NoNode) { t.Error(err) } @@ -53,11 +52,11 @@ func checkVSchema(t *testing.T, ts *topo.Server) { t.Fatal(err) } - got, err = ts.GetVSchema(ctx, "test_keyspace") + got, err := ts.GetVSchema(ctx, "test_keyspace") if err != nil { t.Error(err) } - want = &vschemapb.Keyspace{ + want := &vschemapb.Keyspace{ Tables: map[string]*vschemapb.Table{ "unsharded": {}, }, @@ -94,3 +93,30 @@ func checkVSchema(t *testing.T, ts *topo.Server) { t.Errorf(`GetShardNames: want [ "b0-c0" ], got %v`, shards) } } + +// checkRoutingRules runs the tests on the routing rules part of the API +func checkRoutingRules(t *testing.T, ts *topo.Server) { + ctx := context.Background() + + if _, err := ts.GetRoutingRules(ctx); err != nil { + t.Fatal(err) + } + + want := &vschemapb.RoutingRules{ + Rules: []*vschemapb.RoutingRule{{ + FromTable: "t1", + ToTables: []string{"t2", "t3"}, + }}, + } + if err := ts.SaveRoutingRules(ctx, want); err != nil { + t.Fatal(err) + } + + got, err := ts.GetRoutingRules(ctx) + if err != nil { + t.Error(err) + } + if !proto.Equal(got, want) { + t.Errorf("GetRoutingRules: %v, want %v", got, want) + } +} diff --git a/go/vt/topo/test/watch.go b/go/vt/topo/test/watch.go index afa18e61952..30aab3ecaae 100644 --- a/go/vt/topo/test/watch.go +++ b/go/vt/topo/test/watch.go @@ -39,7 +39,7 @@ func waitForInitialValue(t *testing.T, conn topo.Conn, srvKeyspace *topodatapb.S current, changes, cancel = conn.Watch(ctx, "keyspaces/test_keyspace/SrvKeyspace") if topo.IsErrType(current.Err, topo.NoNode) { // hasn't appeared yet - if time.Now().Sub(start) > 10*time.Second { + if time.Since(start) > 10*time.Second { t.Fatalf("time out waiting for file to appear") } time.Sleep(10 * time.Millisecond) @@ -72,7 +72,7 @@ func checkWatch(t *testing.T, ts *topo.Server) { } // start watching something that doesn't exist -> error - current, changes, cancel := conn.Watch(ctx, "keyspaces/test_keyspace/SrvKeyspace") + current, changes, _ := conn.Watch(ctx, "keyspaces/test_keyspace/SrvKeyspace") if !topo.IsErrType(current.Err, topo.NoNode) { t.Errorf("watch on missing node didn't return ErrNoNode: %v %v", current, changes) } @@ -86,7 +86,7 @@ func checkWatch(t *testing.T, ts *topo.Server) { } // start watching again, it should work - changes, cancel = waitForInitialValue(t, conn, srvKeyspace) + changes, cancel := waitForInitialValue(t, conn, srvKeyspace) defer cancel() // change the data diff --git a/go/vt/topo/topoproto/shard.go b/go/vt/topo/topoproto/shard.go index d7d837019be..47c629ebd6b 100644 --- a/go/vt/topo/topoproto/shard.go +++ b/go/vt/topo/topoproto/shard.go @@ -37,7 +37,7 @@ func KeyspaceShardString(keyspace, shard string) string { func ParseKeyspaceShard(param string) (string, string, error) { keySpaceShard := strings.Split(param, "/") if len(keySpaceShard) != 2 { - return "", "", fmt.Errorf("Invalid shard path: %v", param) + return "", "", fmt.Errorf("invalid shard path: %v", param) } return keySpaceShard[0], keySpaceShard[1], nil } diff --git a/go/vt/topo/topotests/srv_keyspace_test.go b/go/vt/topo/topotests/srv_keyspace_test.go index 6468493a390..31c0106701f 100644 --- a/go/vt/topo/topotests/srv_keyspace_test.go +++ b/go/vt/topo/topotests/srv_keyspace_test.go @@ -44,7 +44,7 @@ func waitForInitialSrvKeyspace(t *testing.T, ts *topo.Server, cell, keyspace str switch { case topo.IsErrType(current.Err, topo.NoNode): // hasn't appeared yet - if time.Now().Sub(start) > 10*time.Second { + if time.Since(start) > 10*time.Second { t.Fatalf("time out waiting for file to appear") } time.Sleep(10 * time.Millisecond) @@ -141,7 +141,7 @@ func TestWatchSrvKeyspace(t *testing.T) { cancel() // Bad data in topo, setting the watch should now fail. - current, changes, cancel = ts.WatchSrvKeyspace(ctx, cell, keyspace) + current, _, _ = ts.WatchSrvKeyspace(ctx, cell, keyspace) if current.Err == nil || !strings.Contains(current.Err.Error(), "error unpacking initial SrvKeyspace object") { t.Fatalf("expected an initial error setting watch on bad content, but got: %v", current.Err) } @@ -152,11 +152,11 @@ func TestWatchSrvKeyspace(t *testing.T) { } start := time.Now() for { - current, changes, cancel = ts.WatchSrvKeyspace(ctx, cell, keyspace) + current, changes, _ = ts.WatchSrvKeyspace(ctx, cell, keyspace) if current.Err != nil { if strings.Contains(current.Err.Error(), "error unpacking initial SrvKeyspace object") { // hasn't changed yet - if time.Now().Sub(start) > 10*time.Second { + if time.Since(start) > 10*time.Second { t.Fatalf("time out waiting for file to appear") } time.Sleep(10 * time.Millisecond) @@ -199,7 +199,7 @@ func TestWatchSrvKeyspaceCancel(t *testing.T) { ts := memorytopo.NewServer(cell) // No SrvKeyspace -> ErrNoNode - current, changes, cancel := ts.WatchSrvKeyspace(ctx, cell, keyspace) + current, _, _ := ts.WatchSrvKeyspace(ctx, cell, keyspace) if !topo.IsErrType(current.Err, topo.NoNode) { t.Errorf("Got invalid result from WatchSrvKeyspace(not there): %v", current.Err) } @@ -213,7 +213,7 @@ func TestWatchSrvKeyspaceCancel(t *testing.T) { } // Starting the watch should now work. - current, changes, cancel = waitForInitialSrvKeyspace(t, ts, cell, keyspace) + current, changes, cancel := waitForInitialSrvKeyspace(t, ts, cell, keyspace) if !proto.Equal(current.Value, wanted) { t.Fatalf("got bad data: %v expected: %v", current.Value, wanted) } @@ -255,10 +255,10 @@ func TestUpdateSrvKeyspacePartitions(t *testing.T) { // Create initial value initial := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: keyRange[0], }, @@ -292,18 +292,18 @@ func TestUpdateSrvKeyspacePartitions(t *testing.T) { targetKs := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: keyRange[0], }, - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, @@ -349,10 +349,10 @@ func TestUpdateSrvKeyspacePartitions(t *testing.T) { // removing works targetKs = &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: keyRange[0], }, @@ -387,23 +387,23 @@ func TestUpdateSrvKeyspacePartitions(t *testing.T) { // You can add to partitions that do not exist targetKs = &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: keyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, @@ -466,6 +466,16 @@ func TestUpdateSrvKeyspacePartitions(t *testing.T) { t.Fatalf("GetSrvKeyspace() failed: %v", err) } + got, err = json2.MarshalPB(srvKeyspace) + if err != nil { + t.Fatalf("MarshalPB() failed: %v", err) + } + + want, err = json2.MarshalPB(targetKs) + if err != nil { + t.Fatalf("MarshalPB() failed: %v", err) + } + if string(got) != string(want) { t.Errorf("GetSrvKeyspace() failure. Got %v, want: %v", string(got), string(want)) } @@ -491,14 +501,14 @@ func TestUpdateUpdateDisableQueryService(t *testing.T) { // Create initial value initial := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, @@ -522,25 +532,25 @@ func TestUpdateUpdateDisableQueryService(t *testing.T) { targetKs := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-80", KeyRange: leftKeyRange[0], QueryServiceDisabled: true, }, - &topodatapb.ShardTabletControl{ + { Name: "80-", KeyRange: rightKeyRange[0], QueryServiceDisabled: true, @@ -607,25 +617,25 @@ func TestUpdateUpdateDisableQueryService(t *testing.T) { // You can enable query service targetKs = &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-80", KeyRange: leftKeyRange[0], QueryServiceDisabled: false, }, - &topodatapb.ShardTabletControl{ + { Name: "80-", KeyRange: rightKeyRange[0], QueryServiceDisabled: false, @@ -683,32 +693,32 @@ func TestGetShardServingTypes(t *testing.T) { // Create initial value initial := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_RDONLY, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, @@ -756,7 +766,7 @@ func TestGetShardServingTypes(t *testing.T) { t.Errorf("GetShardServingTypes() failure. Got %v, want: %v", got, want) } - keyRange, err := key.ParseShardingSpec("-") + keyRange, _ := key.ParseShardingSpec("-") shardInfo = topo.NewShardInfo(keyspace, "-", &topodatapb.Shard{KeyRange: keyRange[0]}, nil) @@ -791,32 +801,32 @@ func TestGetShardServingCells(t *testing.T) { // Create initial value initial := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_RDONLY, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, @@ -900,42 +910,42 @@ func TestMasterMigrateServedType(t *testing.T) { // Create initial value initial := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-", KeyRange: initialKeyRange[0], QueryServiceDisabled: true, }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-", KeyRange: initialKeyRange[0], QueryServiceDisabled: true, }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_RDONLY, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, @@ -981,46 +991,46 @@ func TestMasterMigrateServedType(t *testing.T) { targetKs := &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-", KeyRange: initialKeyRange[0], QueryServiceDisabled: true, }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, }, ShardTabletControls: []*topodatapb.ShardTabletControl{ - &topodatapb.ShardTabletControl{ + { Name: "-", KeyRange: initialKeyRange[0], QueryServiceDisabled: true, }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_RDONLY, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, @@ -1097,36 +1107,36 @@ func TestMasterMigrateServedType(t *testing.T) { targetKs = &topodatapb.SrvKeyspace{ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{ - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_MASTER, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_REPLICA, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-", KeyRange: initialKeyRange[0], }, }, }, - &topodatapb.SrvKeyspace_KeyspacePartition{ + { ServedType: topodatapb.TabletType_RDONLY, ShardReferences: []*topodatapb.ShardReference{ - &topodatapb.ShardReference{ + { Name: "-80", KeyRange: leftKeyRange[0], }, - &topodatapb.ShardReference{ + { Name: "80-", KeyRange: rightKeyRange[0], }, diff --git a/go/vt/topo/vschema.go b/go/vt/topo/vschema.go index 76799c35629..a2103ff10ea 100644 --- a/go/vt/topo/vschema.go +++ b/go/vt/topo/vschema.go @@ -64,3 +64,36 @@ func (ts *Server) GetVSchema(ctx context.Context, keyspace string) (*vschemapb.K } return &vs, nil } + +// SaveRoutingRules saves the routing rules into the topo. +func (ts *Server) SaveRoutingRules(ctx context.Context, routingRules *vschemapb.RoutingRules) error { + data, err := proto.Marshal(routingRules) + if err != nil { + return err + } + + if len(data) == 0 { + // No vschema, remove it. So we can remove the keyspace. + return ts.globalCell.Delete(ctx, RoutingRulesFile, nil) + } + + _, err = ts.globalCell.Update(ctx, RoutingRulesFile, data, nil) + return err +} + +// GetRoutingRules fetches the routing rules from the topo. +func (ts *Server) GetRoutingRules(ctx context.Context) (*vschemapb.RoutingRules, error) { + rr := &vschemapb.RoutingRules{} + data, _, err := ts.globalCell.Get(ctx, RoutingRulesFile) + if err != nil { + if IsErrType(err, NoNode) { + return rr, nil + } + return nil, err + } + err = proto.Unmarshal(data, rr) + if err != nil { + return nil, vterrors.Wrapf(err, "bad routing rules data: %q", data) + } + return rr, nil +} diff --git a/go/vt/topo/zk2topo/file.go b/go/vt/topo/zk2topo/file.go index 67e603e8c37..876f2c23ea7 100644 --- a/go/vt/topo/zk2topo/file.go +++ b/go/vt/topo/zk2topo/file.go @@ -44,7 +44,7 @@ func (zs *Server) Create(ctx context.Context, filePath string, contents []byte) if err != nil { return nil, convertError(err, zkPath) } - if bytes.Compare(data, contents) != 0 { + if !bytes.Equal(data, contents) { return nil, fmt.Errorf("file contents changed between zk.Create and zk.Get") } diff --git a/go/vt/topotools/rebuild_vschema.go b/go/vt/topotools/rebuild_vschema.go index 1b356319361..a5726056d5d 100644 --- a/go/vt/topotools/rebuild_vschema.go +++ b/go/vt/topotools/rebuild_vschema.go @@ -79,6 +79,12 @@ func RebuildVSchema(ctx context.Context, log logutil.Logger, ts *topo.Server, ce return finalErr } + rr, err := ts.GetRoutingRules(ctx) + if err != nil { + return fmt.Errorf("GetRoutingRules failed: %v", err) + } + srvVSchema.RoutingRules = rr + // now save the SrvVSchema in all cells in parallel for _, cell := range cells { wg.Add(1) diff --git a/go/vt/topotools/rebuild_vschema_test.go b/go/vt/topotools/rebuild_vschema_test.go index 052c7d17013..68a13defbc5 100644 --- a/go/vt/topotools/rebuild_vschema_test.go +++ b/go/vt/topotools/rebuild_vschema_test.go @@ -32,7 +32,9 @@ import ( func TestRebuildVSchema(t *testing.T) { ctx := context.Background() logger := logutil.NewConsoleLogger() - emptySrvVSchema := &vschemapb.SrvVSchema{} + emptySrvVSchema := &vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{}, + } // Set up topology. cells := []string{"cell1", "cell2"} @@ -50,6 +52,7 @@ func TestRebuildVSchema(t *testing.T) { // create a keyspace, rebuild, should see an empty entry emptyKs1SrvVSchema := &vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{}, Keyspaces: map[string]*vschemapb.Keyspace{ "ks1": {}, }, @@ -77,6 +80,7 @@ func TestRebuildVSchema(t *testing.T) { t.Errorf("RebuildVSchema failed: %v", err) } wanted1 := &vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{}, Keyspaces: map[string]*vschemapb.Keyspace{ "ks1": keyspace1, }, @@ -100,7 +104,6 @@ func TestRebuildVSchema(t *testing.T) { }, Tables: map[string]*vschemapb.Table{ "table1": { - Type: "sequence", ColumnVindexes: []*vschemapb.ColumnVindex{ { Column: "column1", @@ -117,6 +120,7 @@ func TestRebuildVSchema(t *testing.T) { t.Errorf("RebuildVSchema failed: %v", err) } wanted2 := &vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{}, Keyspaces: map[string]*vschemapb.Keyspace{ "ks1": keyspace1, "ks2": keyspace2, @@ -139,6 +143,35 @@ func TestRebuildVSchema(t *testing.T) { } } + rr := &vschemapb.RoutingRules{ + Rules: []*vschemapb.RoutingRule{{ + FromTable: "t1", + ToTables: []string{"t2", "t3"}, + }}, + } + + if err := ts.SaveRoutingRules(ctx, rr); err != nil { + t.Fatalf("SaveRoutingRules() failed: %v", err) + } + if err := RebuildVSchema(ctx, logger, ts, nil); err != nil { + t.Errorf("RebuildVSchema failed: %v", err) + } + wanted3 := &vschemapb.SrvVSchema{ + RoutingRules: rr, + Keyspaces: map[string]*vschemapb.Keyspace{ + "ks1": keyspace1, + "ks2": keyspace2, + }, + } + for _, cell := range cells { + if v, err := ts.GetSrvVSchema(ctx, cell); err != nil || !proto.Equal(v, wanted3) { + t.Errorf("unexpected GetSrvVSchema(%v) result: %v %v", cell, v, err) + } + } + + wanted4 := wanted1 + wanted4.RoutingRules = rr + // Delete a keyspace, checks vschema entry in map goes away. if err := ts.SaveVSchema(ctx, "ks2", &vschemapb.Keyspace{}); err != nil { t.Fatalf("SaveVSchema(ks1) failed: %v", err) @@ -150,7 +183,7 @@ func TestRebuildVSchema(t *testing.T) { t.Errorf("RebuildVSchema failed: %v", err) } for _, cell := range cells { - if v, err := ts.GetSrvVSchema(ctx, cell); err != nil || !proto.Equal(v, wanted1) { + if v, err := ts.GetSrvVSchema(ctx, cell); err != nil || !proto.Equal(v, wanted4) { t.Errorf("unexpected GetSrvVSchema(%v) result: %v %v", cell, v, err) } } diff --git a/go/vt/vtaclcheck/vtaclcheck.go b/go/vt/vtaclcheck/vtaclcheck.go index 8574ac43f72..f9a4f7ab74f 100644 --- a/go/vt/vtaclcheck/vtaclcheck.go +++ b/go/vt/vtaclcheck/vtaclcheck.go @@ -45,7 +45,7 @@ func Init(opts *Options) error { } // Verify options if opts.ACLFile == "" && opts.StaticAuthFile == "" { - return fmt.Errorf("No options specified") + return fmt.Errorf("no options specified") } options = opts diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 28a6ae52dca..861eff8f1bf 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -483,6 +483,12 @@ func (itc *internalTabletConn) VStream(ctx context.Context, target *querypb.Targ return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) } +// VStreamRows is part of the QueryService interface. +func (itc *internalTabletConn) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + err := itc.tablet.qsc.QueryService().VStreamRows(ctx, target, query, lastpk, send) + return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) +} + // // TabletManagerClient implementation // diff --git a/go/vt/vtctl/cells_aliases.go b/go/vt/vtctl/cells_aliases.go index 5ef2a78252b..4ffd67b14bd 100644 --- a/go/vt/vtctl/cells_aliases.go +++ b/go/vt/vtctl/cells_aliases.go @@ -39,13 +39,13 @@ func init() { "AddCellsAlias", commandAddCellsAlias, "[-cells ] ", - "Registers a local topology service in a new cell by creating the CellInfo with the provided parameters. The address will be used to connect to the topology service, and we'll put Vitess data starting at the provided root."}) + "Registers a local topology service in a new cell by creating the CellsAlias with the provided parameters. An alis provides a group cells that replica/rdonly can route. By default, vitess won't route traffic cross cells for replica/rdonly tablets. Aliases provide a way to create groups where this is allowed."}) addCommand(cellsAliasesGroupName, command{ "UpdateCellsAlias", commandUpdateCellsAlias, "[-cells ] ", - "Updates the content of a CellInfo with the provided parameters. If a value is empty, it is not updated. The CellInfo will be created if it doesn't exist."}) + "Updates the content of a CellsAlias with the provided parameters. If a value is empty, it is not updated. The CellsAlias will be created if it doesn't exist."}) addCommand(cellsAliasesGroupName, command{ "DeleteCellsAlias", @@ -57,7 +57,7 @@ func init() { "GetCellsAliases", commandGetCellsAliases, "", - "Lists all the cells for which we have a CellInfo object, meaning we have a local topology service registered."}) + "Lists all the cells for which we have a CellsAlias object."}) } func commandAddCellsAlias(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { diff --git a/go/vt/vtctl/fakevtctlclient/fake_loggerevent_streamingclient.go b/go/vt/vtctl/fakevtctlclient/fake_loggerevent_streamingclient.go index 7d7872316ff..15689624e76 100644 --- a/go/vt/vtctl/fakevtctlclient/fake_loggerevent_streamingclient.go +++ b/go/vt/vtctl/fakevtctlclient/fake_loggerevent_streamingclient.go @@ -83,7 +83,7 @@ func (f *FakeLoggerEventStreamingClient) RegisterResultForAddr(addr string, args result.count++ return nil } - return fmt.Errorf("A different result (%v) is already registered for command: %v", result, args) + return fmt.Errorf("a different result (%v) is already registered for command: %v", result, args) } f.results[k] = &v return nil diff --git a/go/vt/vtctl/throttler.go b/go/vt/vtctl/throttler.go index 9418b7d1518..1e8452e577b 100644 --- a/go/vt/vtctl/throttler.go +++ b/go/vt/vtctl/throttler.go @@ -35,12 +35,14 @@ import ( throttlerdatapb "vitess.io/vitess/go/vt/proto/throttlerdata" ) +const ( + throttlerGroupName = "Resharding Throttler" + shortTimeout = 15 * time.Second +) + // This file contains the commands to control the throttler which is used during // resharding (vtworker) and by filtered replication (vttablet). -const throttlerGroupName = "Resharding Throttler" -const shortTimeout = 15 * time.Second - func init() { addCommandGroup(throttlerGroupName) diff --git a/go/vt/vtctl/topo.go b/go/vt/vtctl/topo.go index 547a84baa13..6797a72e4d3 100644 --- a/go/vt/vtctl/topo.go +++ b/go/vt/vtctl/topo.go @@ -17,8 +17,10 @@ limitations under the License. package vtctl import ( + "encoding/json" "flag" "fmt" + "github.com/golang/protobuf/jsonpb" "io/ioutil" "path" @@ -42,7 +44,7 @@ func init() { addCommand(topoGroupName, command{ "TopoCat", commandTopoCat, - "[-cell ] [-decode_proto] [-long] [...]", + "[-cell ] [-decode_proto] [-decode_proto_json] [-long] [...]", "Retrieves the file(s) at from the topo service, and displays it. It can resolve wildcards, and decode the proto-encoded data."}) addCommand(topoGroupName, command{ @@ -54,7 +56,7 @@ func init() { // DecodeContent uses the filename to imply a type, and proto-decodes // the right object, then echoes it as a string. -func DecodeContent(filename string, data []byte) (string, error) { +func DecodeContent(filename string, data []byte, json bool) (string, error) { name := path.Base(filename) var p proto.Message @@ -75,19 +77,31 @@ func DecodeContent(filename string, data []byte) (string, error) { p = new(vschemapb.SrvVSchema) case topo.SrvKeyspaceFile: p = new(topodatapb.SrvKeyspace) + case topo.RoutingRulesFile: + p = new(vschemapb.RoutingRules) default: - return string(data), nil + if json { + return "", fmt.Errorf("unknown topo protobuf type for %v", name) + } else { + return string(data), nil + } } if err := proto.Unmarshal(data, p); err != nil { return string(data), err } - return proto.MarshalTextString(p), nil + + if json { + return new(jsonpb.Marshaler).MarshalToString(p) + } else { + return proto.MarshalTextString(p), nil + } } func commandTopoCat(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { cell := subFlags.String("cell", topo.GlobalCell, "topology cell to cat the file from. Defaults to global cell.") long := subFlags.Bool("long", false, "long listing.") + decodeProtoJson := subFlags.Bool("decode_proto_json", false, "decode proto files and display them as json") decodeProto := subFlags.Bool("decode_proto", false, "decode proto files and display them as text") subFlags.Parse(args) if subFlags.NArg() == 0 { @@ -106,38 +120,18 @@ func commandTopoCat(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.F if err != nil { return err } - hasError := false - for _, arg := range resolved { - data, version, err := conn.Get(ctx, arg) - if err != nil { - hasError = true - wr.Logger().Printf("TopoCat: Get(%v) failed: %v\n", arg, err) - continue - } - if *long { - wr.Logger().Printf("path=%v version=%v\n", arg, version) - } - decoded := "" - if *decodeProto { - decoded, err = DecodeContent(arg, data) - if err != nil { - wr.Logger().Warningf("TopoCat: cannot proto decode %v: %v", arg, err) - decoded = string(data) - } - } else { - decoded = string(data) - } - wr.Logger().Printf(decoded) - if len(decoded) > 0 && decoded[len(decoded)-1] != '\n' && *long { - wr.Logger().Printf("\n") - } - } - if hasError { - return fmt.Errorf("TopoCat: some paths had errors") + var topologyDecoder TopologyDecoder + switch { + case *decodeProtoJson: + topologyDecoder = JsonTopologyDecoder{} + case *decodeProto: + topologyDecoder = ProtoTopologyDecoder{} + default: + topologyDecoder = PlainTopologyDecoder{} } - return nil + return topologyDecoder.decode(resolved, conn, ctx, wr, *long) } func commandTopoCp(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { @@ -179,3 +173,115 @@ func copyFileToTopo(ctx context.Context, ts *topo.Server, cell, from, to string) _, err = conn.Update(ctx, to, data, nil) return err } + +type TopologyDecoder interface { + decode([]string, topo.Conn, context.Context, *wrangler.Wrangler, bool) error +} + +type ProtoTopologyDecoder struct{} +type PlainTopologyDecoder struct{} +type JsonTopologyDecoder struct{} + +func (d ProtoTopologyDecoder) decode(topoPaths []string, conn topo.Conn, ctx context.Context, wr *wrangler.Wrangler, long bool) error { + hasError := false + for _, topoPath := range topoPaths { + data, version, err := conn.Get(ctx, topoPath) + if err != nil { + hasError = true + wr.Logger().Printf("TopoCat: Get(%v) failed: %v\n", topoPath, err) + continue + } + + if long { + wr.Logger().Printf("path=%v version=%v\n", topoPath, version) + } + + decoded, err := DecodeContent(topoPath, data, false) + if err != nil { + wr.Logger().Warningf("TopoCat: cannot proto decode %v: %v", topoPath, err) + decoded = string(data) + } + + wr.Logger().Printf(decoded) + if len(decoded) > 0 && decoded[len(decoded)-1] != '\n' && long { + wr.Logger().Printf("\n") + } + } + + if hasError { + return fmt.Errorf("TopoCat: some paths had errors") + } + return nil +} + +func (d PlainTopologyDecoder) decode(topoPaths []string, conn topo.Conn, ctx context.Context, wr *wrangler.Wrangler, long bool) error { + hasError := false + for _, topoPath := range topoPaths { + data, version, err := conn.Get(ctx, topoPath) + if err != nil { + hasError = true + wr.Logger().Printf("TopoCat: Get(%v) failed: %v\n", topoPath, err) + continue + } + + if long { + wr.Logger().Printf("path=%v version=%v\n", topoPath, version) + } + decoded := string(data) + wr.Logger().Printf(decoded) + if len(decoded) > 0 && decoded[len(decoded)-1] != '\n' && long { + wr.Logger().Printf("\n") + } + } + + if hasError { + return fmt.Errorf("TopoCat: some paths had errors") + } + return nil +} + +func (d JsonTopologyDecoder) decode(topoPaths []string, conn topo.Conn, ctx context.Context, wr *wrangler.Wrangler, long bool) error { + hasError := false + var jsonData []interface{} + for _, topoPath := range topoPaths { + data, version, err := conn.Get(ctx, topoPath) + if err != nil { + hasError = true + wr.Logger().Printf("TopoCat: Get(%v) failed: %v\n", topoPath, err) + continue + } + + decoded, err := DecodeContent(topoPath, data, true) + if err != nil { + hasError = true + wr.Logger().Printf("TopoCat: cannot proto decode %v: %v", topoPath, err) + continue + } + + var jsonDatum map[string]interface{} + if err = json.Unmarshal([]byte(decoded), &jsonDatum); err != nil { + hasError = true + wr.Logger().Printf("TopoCat: cannot json Unmarshal %v: %v", topoPath, err) + continue + } + + if long { + jsonDatum["__path"] = topoPath + jsonDatum["__version"] = version.String() + } + jsonData = append(jsonData, jsonDatum) + } + + jsonBytes, err := json.Marshal(jsonData) + if err != nil { + hasError = true + wr.Logger().Printf("TopoCat: cannot json Marshal: %v", err) + } else { + wr.Logger().Printf(string(jsonBytes) + "\n") + } + + if hasError { + return fmt.Errorf("TopoCat: some paths had errors") + } + return nil +} diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index 80fbedc9859..d96e47434de 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -307,6 +307,12 @@ var commands = []commandGroup{ {"ValidateKeyspace", commandValidateKeyspace, "[-ping-tablets] ", "Validates that all nodes reachable from the specified keyspace are consistent."}, + {"SplitClone", commandSplitClone, + " ", + "Start the SplitClone process to perform horizontal resharding. Example: SplitClone ks '0' '-80,80-'"}, + {"VerticalSplitClone", commandVerticalSplitClone, + " ", + "Start the VerticalSplitClone process to perform vertical resharding. Example: SplitClone from_ks to_ks 'a,/b.*/'"}, {"MigrateServedTypes", commandMigrateServedTypes, "[-cells=c1,c2,...] [-reverse] [-skip-refresh-state] ", "Migrates a serving type from the source shard to the shards that it replicates to. This command also rebuilds the serving graph. The argument can specify any of the shards involved in the migration."}, @@ -395,6 +401,12 @@ var commands = []commandGroup{ {"ApplyVSchema", commandApplyVSchema, "{-vschema= || -vschema_file= || -sql= || -sql_file=} [-cells=c1,c2,...] [-skip_rebuild] [-dry-run] ", "Applies the VTGate routing schema to the provided keyspace. Shows the result after application."}, + {"GetRoutingRules", commandGetRoutingRules, + "", + "Displays the VSchema routing rules."}, + {"ApplyRoutingRules", commandApplyRoutingRules, + "{-rules= || -rules_file=} [-cells=c1,c2,...] [-skip_rebuild] [-dry-run]", + "Applies the VSchema routing rules."}, {"RebuildVSchemaGraph", commandRebuildVSchemaGraph, "[-cells=c1,c2,...]", "Rebuilds the cell-specific SrvVSchema from the global VSchema objects in the provided cells (or all cells if none provided)."}, @@ -1718,6 +1730,32 @@ func commandValidateKeyspace(ctx context.Context, wr *wrangler.Wrangler, subFlag return wr.ValidateKeyspace(ctx, keyspace, *pingTablets) } +func commandSplitClone(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 3 { + return fmt.Errorf("three arguments are required: keyspace, from_shards, to_shards") + } + keyspace := subFlags.Arg(0) + from := strings.Split(subFlags.Arg(1), ",") + to := strings.Split(subFlags.Arg(2), ",") + return wr.SplitClone(ctx, keyspace, from, to) +} + +func commandVerticalSplitClone(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 3 { + return fmt.Errorf("three arguments are required: from_keyspace, to_keyspace, tables") + } + fromKeyspace := subFlags.Arg(0) + toKeyspace := subFlags.Arg(1) + tables := strings.Split(subFlags.Arg(2), ",") + return wr.VerticalSplitClone(ctx, fromKeyspace, toKeyspace, tables) +} + func commandMigrateServedTypes(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { cellsStr := subFlags.String("cells", "", "Specifies a comma-separated list of cells to update") reverse := subFlags.Bool("reverse", false, "Moves the served tablet type backward instead of forward. Use in case of trouble") @@ -2156,6 +2194,20 @@ func commandGetVSchema(ctx context.Context, wr *wrangler.Wrangler, subFlags *fla return nil } +func commandGetRoutingRules(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + rr, err := wr.TopoServer().GetRoutingRules(ctx) + if err != nil { + return err + } + b, err := json2.MarshalIndentPB(rr, " ") + if err != nil { + wr.Logger().Printf("%v\n", err) + return err + } + wr.Logger().Printf("%s\n", b) + return nil +} + func commandRebuildVSchemaGraph(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { var cells flagutil.StringListValue subFlags.Var(&cells, "cells", "Specifies a comma-separated list of cells to look for tablets") @@ -2277,6 +2329,54 @@ func commandApplyVSchema(ctx context.Context, wr *wrangler.Wrangler, subFlags *f return topotools.RebuildVSchema(ctx, wr.Logger(), wr.TopoServer(), cells) } +func commandApplyRoutingRules(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + routingRules := subFlags.String("rules", "", "Specify rules as a string") + routingRulesFile := subFlags.String("vschema_file", "", "Specify rules in a file") + skipRebuild := subFlags.Bool("skip_rebuild", false, "If set, do no rebuild the SrvSchema objects.") + var cells flagutil.StringListValue + subFlags.Var(&cells, "cells", "If specified, limits the rebuild to the cells, after upload. Ignored if skipRebuild is set.") + + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 0 { + return fmt.Errorf("ApplyRoutingRules doesn't take any arguments") + } + + var rulesBytes []byte + if *routingRulesFile != "" { + var err error + rulesBytes, err = ioutil.ReadFile(*routingRulesFile) + if err != nil { + return err + } + } else { + rulesBytes = []byte(*routingRules) + } + + rr := &vschemapb.RoutingRules{} + if err := json2.Unmarshal(rulesBytes, rr); err != nil { + return err + } + + b, err := json2.MarshalIndentPB(rr, " ") + if err != nil { + wr.Logger().Errorf2(err, "Failed to marshal RoutingRules for display") + } else { + wr.Logger().Printf("New RoutingRules object:\n%s\nIf this is not what you expected, check the input data (as JSON parsing will skip unexpected fields).\n", b) + } + + if err := wr.TopoServer().SaveRoutingRules(ctx, rr); err != nil { + return err + } + + if *skipRebuild { + wr.Logger().Warningf("Skipping rebuild of SrvVSchema, will need to run RebuildVSchemaGraph for changes to take effect") + return nil + } + return topotools.RebuildVSchema(ctx, wr.Logger(), wr.TopoServer(), cells) +} + func commandGetSrvKeyspaceNames(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { if err := subFlags.Parse(args); err != nil { return err diff --git a/go/vt/vtctl/vtctlclient/wrapper.go b/go/vt/vtctl/vtctlclient/wrapper.go index 3c78b007209..cc1b9500652 100644 --- a/go/vt/vtctl/vtctlclient/wrapper.go +++ b/go/vt/vtctl/vtctlclient/wrapper.go @@ -36,12 +36,12 @@ var ( // have to be consumed by the caller who has to specify a "recv" function. func RunCommandAndWait(ctx context.Context, server string, args []string, recv func(*logutilpb.Event)) error { if recv == nil { - return errors.New("No function closure for Event stream specified") + return errors.New("no function closure for Event stream specified") } // create the client client, err := New(server) if err != nil { - return fmt.Errorf("Cannot dial to server %v: %v", server, err) + return fmt.Errorf("cannot dial to server %v: %v", server, err) } defer client.Close() @@ -53,7 +53,7 @@ func RunCommandAndWait(ctx context.Context, server string, args []string, recv f } stream, err := client.ExecuteVtctlCommand(ctx, args, timeout) if err != nil { - return fmt.Errorf("Cannot execute remote command: %v", err) + return fmt.Errorf("cannot execute remote command: %v", err) } // stream the result @@ -65,7 +65,7 @@ func RunCommandAndWait(ctx context.Context, server string, args []string, recv f case io.EOF: return nil default: - return fmt.Errorf("Remote error: %v", err) + return fmt.Errorf("remote error: %v", err) } } } diff --git a/go/vt/vtctl/vtctlclienttest/client.go b/go/vt/vtctl/vtctlclienttest/client.go index 6c1ec3b2485..ca43317996f 100644 --- a/go/vt/vtctl/vtctlclienttest/client.go +++ b/go/vt/vtctl/vtctlclienttest/client.go @@ -104,7 +104,7 @@ func TestSuite(t *testing.T, ts *topo.Server, client vtctlclient.VtctlClient) { t.Fatalf("Remote error: %v", err) } - got, err = stream.Recv() + _, err = stream.Recv() expected = "node doesn't exist" if err == nil || !strings.Contains(err.Error(), expected) { t.Fatalf("Unexpected remote error, got: '%v' was expecting to find '%v'", err, expected) @@ -116,7 +116,7 @@ func TestSuite(t *testing.T, ts *topo.Server, client vtctlclient.VtctlClient) { t.Fatalf("Remote error: %v", err) } - got, err = stream.Recv() + _, err = stream.Recv() expected1 := "this command panics on purpose" expected2 := "uncaught vtctl panic" if err == nil || !strings.Contains(err.Error(), expected1) || !strings.Contains(err.Error(), expected2) { diff --git a/go/vt/vtctld/explorer.go b/go/vt/vtctld/explorer.go index 003bba14575..2dd4275563d 100644 --- a/go/vt/vtctld/explorer.go +++ b/go/vt/vtctld/explorer.go @@ -89,7 +89,7 @@ func (ex *backendExplorer) HandlePath(nodePath string, r *http.Request) *Result case nil: if len(data) > 0 { // It has contents, we just use it if possible. - decoded, err := vtctl.DecodeContent(relativePath, data) + decoded, err := vtctl.DecodeContent(relativePath, data, false) if err != nil { result.Error = err.Error() } else { diff --git a/go/vt/vtctld/tablet_data_test.go b/go/vt/vtctld/tablet_data_test.go index 153bc68fb11..edc69d985a1 100644 --- a/go/vt/vtctld/tablet_data_test.go +++ b/go/vt/vtctld/tablet_data_test.go @@ -22,10 +22,11 @@ import ( "testing" "time" - "golang.org/x/net/context" - "github.com/golang/protobuf/proto" + "golang.org/x/net/context" "vitess.io/vitess/go/vt/logutil" + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vttablet/grpcqueryservice" "vitess.io/vitess/go/vt/vttablet/queryservice" @@ -33,9 +34,6 @@ import ( "vitess.io/vitess/go/vt/vttablet/tmclient" "vitess.io/vitess/go/vt/wrangler" "vitess.io/vitess/go/vt/wrangler/testlib" - - querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) // streamHealthTabletServer is a local QueryService implementation to support the tests @@ -91,7 +89,7 @@ func (s *streamHealthTabletServer) streamHealthUnregister(id int) error { } // BroadcastHealth will broadcast the current health to all listeners -func (s *streamHealthTabletServer) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats) { +func (s *streamHealthTabletServer) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats, maxCache time.Duration) { shr := &querypb.StreamHealthResponse{ TabletExternallyReparentedTimestamp: terTimestamp, RealtimeStats: stats, @@ -137,7 +135,7 @@ func TestTabletData(t *testing.T) { case <-stop: return default: - shsq.BroadcastHealth(42, stats) + shsq.BroadcastHealth(42, stats, time.Minute) } } }() diff --git a/go/vt/vtctld/tablet_stats_cache.go b/go/vt/vtctld/tablet_stats_cache.go index 3ee0df82abc..26323010895 100644 --- a/go/vt/vtctld/tablet_stats_cache.go +++ b/go/vt/vtctld/tablet_stats_cache.go @@ -27,6 +27,18 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) +const ( + // tabletMissing represents a missing/non-existent tablet for any metric. + tabletMissing = -1 + // These values represent the threshold for replication lag. + lagThresholdDegraded = 60 + lagThresholdUnhealthy = 120 + // These values represent the health of the tablet - 1 is healthy, 2 is degraded, 3 is unhealthy + tabletHealthy = 0 + tabletDegraded = 1 + tabletUnhealthy = 2 +) + // yLabel is used to keep track of the cell and type labels of the heatmap. type yLabel struct { CellLabel label @@ -60,18 +72,6 @@ func (a byTabletUID) Len() int { return len(a) } func (a byTabletUID) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a byTabletUID) Less(i, j int) bool { return a[i].Tablet.Alias.Uid < a[j].Tablet.Alias.Uid } -// tabletMissing represents a missing/non-existent tablet for any metric. -const tabletMissing = -1 - -// These values represent the threshold for replication lag. -const lagThresholdDegraded = 60 -const lagThresholdUnhealthy = 120 - -// These values represent the health of the tablet - 1 is healthy, 2 is degraded, 3 is unhealthy -const tabletHealthy = 0 -const tabletDegraded = 1 -const tabletUnhealthy = 2 - // availableTabletTypes is an array of tabletTypes that are being considered to display on the heatmap. // Note: this list must always be sorted by the order they should appear (i.e. MASTER first, then REPLICA, then RDONLY) var availableTabletTypes = []topodatapb.TabletType{topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY} diff --git a/go/vt/vterrors/errors_test.go b/go/vt/vterrors/errors_test.go index 772b7d90251..f0498ee3306 100644 --- a/go/vt/vterrors/errors_test.go +++ b/go/vt/vterrors/errors_test.go @@ -172,15 +172,16 @@ func TestStackFormat(t *testing.T) { err := outer() got := fmt.Sprintf("%v", err) - assertStringContains(t, got, "innerMost") - assertStringContains(t, got, "middle") - assertStringContains(t, got, "outer") -} + assertContains(t, got, "innerMost", false) + assertContains(t, got, "middle", false) + assertContains(t, got, "outer", false) -func assertStringContains(t *testing.T, s, substring string) { - if !strings.Contains(s, substring) { - t.Errorf("string did not contain `%v`: \n %v", substring, s) - } + LogErrStacks = true + defer func() { LogErrStacks = false }() + got = fmt.Sprintf("%v", err) + assertContains(t, got, "innerMost", true) + assertContains(t, got, "middle", true) + assertContains(t, got, "outer", true) } // errors.New, etc values are not expected to be compared by value @@ -256,18 +257,29 @@ func TestWrapping(t *testing.T) { err1 := Errorf(vtrpcpb.Code_UNAVAILABLE, "foo") err2 := Wrapf(err1, "bar") err3 := Wrapf(err2, "baz") + errorWithoutStack := fmt.Sprintf("%v", err3) + + LogErrStacks = true errorWithStack := fmt.Sprintf("%v", err3) + LogErrStacks = false assertEquals(t, err3.Error(), "baz: bar: foo") - assertContains(t, errorWithStack, "foo") - assertContains(t, errorWithStack, "bar") - assertContains(t, errorWithStack, "baz") - assertContains(t, errorWithStack, "TestWrapping") + assertContains(t, errorWithoutStack, "foo", true) + assertContains(t, errorWithoutStack, "bar", true) + assertContains(t, errorWithoutStack, "baz", true) + assertContains(t, errorWithoutStack, "TestWrapping", false) + + assertContains(t, errorWithStack, "foo", true) + assertContains(t, errorWithStack, "bar", true) + assertContains(t, errorWithStack, "baz", true) + assertContains(t, errorWithStack, "TestWrapping", true) + } -func assertContains(t *testing.T, s, substring string) { - if !strings.Contains(s, substring) { - t.Fatalf("expected string that contains [%s] but got [%s]", substring, s) +func assertContains(t *testing.T, s, substring string, contains bool) { + t.Helper() + if doesContain := strings.Contains(s, substring); doesContain != contains { + t.Errorf("string `%v` contains `%v`: %v, want %v", s, substring, doesContain, contains) } } @@ -275,4 +287,4 @@ func assertEquals(t *testing.T, a, b interface{}) { if a != b { t.Fatalf("expected [%s] to be equal to [%s]", a, b) } -} \ No newline at end of file +} diff --git a/go/vt/vterrors/vterrors.go b/go/vt/vterrors/vterrors.go index c798fa8ffb7..95af7967544 100644 --- a/go/vt/vterrors/vterrors.go +++ b/go/vt/vterrors/vterrors.go @@ -70,6 +70,7 @@ package vterrors import ( + "flag" "fmt" "io" @@ -77,6 +78,14 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) +// LogErrStacks controls whether or not printing errors includes the +// embedded stack trace in the output. +var LogErrStacks bool + +func init() { + flag.BoolVar(&LogErrStacks, "LogErrStacks", false, "log stack traces in errors") +} + // New returns an error with the supplied message. // New also records the stack trace at the point it was called. func New(code vtrpcpb.Code, message string) error { @@ -122,7 +131,9 @@ func (f *fundamental) Format(s fmt.State, verb rune) { case 'v': panicIfError(io.WriteString(s, "Code: "+f.code.String()+"\n")) panicIfError(io.WriteString(s, f.msg+"\n")) - f.stack.Format(s, verb) + if LogErrStacks { + f.stack.Format(s, verb) + } return case 's': panicIfError(io.WriteString(s, f.msg)) @@ -198,7 +209,9 @@ func (w *wrapping) Format(s fmt.State, verb rune) { if rune('v') == verb { panicIfError(fmt.Fprintf(s, "%v\n", w.Cause())) panicIfError(io.WriteString(s, w.msg)) - w.stack.Format(s, verb) + if LogErrStacks { + w.stack.Format(s, verb) + } return } diff --git a/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt b/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt index 6e5a18f3fec..e999a0dcaa6 100644 --- a/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt +++ b/go/vt/vtexplain/testdata/multi-output/updatesharded-output.txt @@ -56,3 +56,76 @@ update user set name='alicia' where name='alice' 9 ks_sharded/c0-: commit ---------------------------------------------------------------------- +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: 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: 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: 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-: commit + +---------------------------------------------------------------------- +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 */ +3 ks_sharded/40-80: commit +4 ks_sharded/-40: commit + +---------------------------------------------------------------------- +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 */ + +---------------------------------------------------------------------- +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 */ + +---------------------------------------------------------------------- +commit + +3 ks_sharded/-40: commit + +---------------------------------------------------------------------- +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 */ + +---------------------------------------------------------------------- +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 */ + +---------------------------------------------------------------------- +commit + +3 ks_sharded/-40: commit +4 ks_sharded/40-80: commit + +---------------------------------------------------------------------- diff --git a/go/vt/vtexplain/testdata/updatesharded-queries.sql b/go/vt/vtexplain/testdata/updatesharded-queries.sql index ef1b9764115..f2e1ecdab37 100644 --- a/go/vt/vtexplain/testdata/updatesharded-queries.sql +++ b/go/vt/vtexplain/testdata/updatesharded-queries.sql @@ -6,5 +6,23 @@ update user set pet='fido' where id=1; update user set name='alicia' where id=1; update user set name='alicia' where name='alice'; -/* not supported - multi-shard update */ --- update user set pet='rover' where name='alice'; +/* scatter update -- supported but with nondeterministic output */ +/* update name_info set has_nickname=1 where nickname != ''; */ + +/* scatter update autocommit */ +update /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ name_info set has_nickname=1 where nickname != ''; + +/* multi-shard update by secondary vindex */ +update user set pet='rover' where name='alice'; + +/* update in a transaction on one shard */ +begin; +update user set nickname='alice' where id=1; +update user set nickname='bob' where id=1; +commit; + +/* update in a transaction on multiple shards */ +begin; +update user set nickname='alice' where id=1; +update user set nickname='bob' where id=3; +commit; diff --git a/go/vt/vtexplain/vtexplain.go b/go/vt/vtexplain/vtexplain.go index 8c7d6e6438b..0023dc4acc1 100644 --- a/go/vt/vtexplain/vtexplain.go +++ b/go/vt/vtexplain/vtexplain.go @@ -44,6 +44,8 @@ var ( type ExecutorMode string const ( + vtexplainCell = "explainCell" + // ModeMulti is the default mode with autocommit implemented at vtgate ModeMulti = "multi" @@ -141,10 +143,6 @@ type Explain struct { TabletActions map[string]*TabletActions } -const ( - vtexplainCell = "explainCell" -) - // Init sets up the fake execution environment func Init(vSchemaStr, sqlSchema string, opts *Options) error { // Verify options @@ -256,8 +254,11 @@ func Run(sql string) ([]*Explain, error) { } if sql != "" { - // Reset the global time simulator for each query - batchTime = sync2.NewBatcher(*batchInterval) + // Reset the global time simulator unless there's an open transaction + // in the session from the previous staement. + if vtgateSession == nil || !vtgateSession.GetInTransaction() { + batchTime = sync2.NewBatcher(*batchInterval) + } log.V(100).Infof("explain %s", sql) e, err := explain(sql) if err != nil { diff --git a/go/vt/vtexplain/vtexplain_vttablet.go b/go/vt/vtexplain/vtexplain_vttablet.go index 421c6e4be65..7c00864f2c8 100644 --- a/go/vt/vtexplain/vtexplain_vttablet.go +++ b/go/vt/vtexplain/vtexplain_vttablet.go @@ -73,7 +73,7 @@ type explainTablet struct { func newTablet(opts *Options, t *topodatapb.Tablet) *explainTablet { db := fakesqldb.New(nil) - config := tabletenv.DefaultQsConfig + config := tabletenv.Config if opts.ExecutionMode == ModeTwoPC { config.TwoPCCoordinatorAddress = "XXX" config.TwoPCAbandonAge = 1.0 @@ -117,6 +117,11 @@ var _ queryservice.QueryService = (*explainTablet)(nil) // compile-time interfac func (t *explainTablet) Begin(ctx context.Context, target *querypb.Target, options *querypb.ExecuteOptions) (int64, error) { t.mu.Lock() t.currentTime = batchTime.Wait() + t.tabletQueries = append(t.tabletQueries, &TabletQuery{ + Time: t.currentTime, + SQL: "begin", + }) + t.mu.Unlock() return t.tsv.Begin(ctx, target, options) @@ -126,7 +131,12 @@ func (t *explainTablet) Begin(ctx context.Context, target *querypb.Target, optio func (t *explainTablet) Commit(ctx context.Context, target *querypb.Target, transactionID int64) error { t.mu.Lock() t.currentTime = batchTime.Wait() + t.tabletQueries = append(t.tabletQueries, &TabletQuery{ + Time: t.currentTime, + SQL: "commit", + }) t.mu.Unlock() + return t.tsv.Commit(ctx, target, transactionID) } diff --git a/go/vt/vtgate/buffer/flags_test.go b/go/vt/vtgate/buffer/flags_test.go index b6f1673dd95..f0d6efff794 100644 --- a/go/vt/vtgate/buffer/flags_test.go +++ b/go/vt/vtgate/buffer/flags_test.go @@ -41,7 +41,7 @@ func TestVerifyFlags(t *testing.T) { resetFlagsForTesting() flag.Set("enable_buffer", "true") flag.Set("buffer_keyspace_shards", "ks1//0") - if err := verifyFlags(); err == nil || !strings.Contains(err.Error(), "Invalid shard path") { + if err := verifyFlags(); err == nil || !strings.Contains(err.Error(), "invalid shard path") { t.Fatalf("Invalid shard names are not allowed. err: %v", err) } diff --git a/go/vt/vtgate/buffer/timeout_thread.go b/go/vt/vtgate/buffer/timeout_thread.go index 9d0d4891603..69b3974f111 100644 --- a/go/vt/vtgate/buffer/timeout_thread.go +++ b/go/vt/vtgate/buffer/timeout_thread.go @@ -95,7 +95,7 @@ func (tt *timeoutThread) run() { // waitForEntry blocks until "e" exceeds its buffering window or buffering stops // in general. It returns true if the timeout thread should stop. func (tt *timeoutThread) waitForEntry(e *entry) bool { - windowExceeded := time.NewTimer(e.deadline.Sub(time.Now())) + windowExceeded := time.NewTimer(time.Until(e.deadline)) defer windowExceeded.Stop() select { diff --git a/go/vt/vtgate/buffer/variables.go b/go/vt/vtgate/buffer/variables.go index c718f9ba8bf..6adfbda7755 100644 --- a/go/vt/vtgate/buffer/variables.go +++ b/go/vt/vtgate/buffer/variables.go @@ -109,8 +109,8 @@ var stopReasons = []stopReason{stopFailoverEndDetected, stopMaxFailoverDurationE const ( stopFailoverEndDetected stopReason = "NewMasterSeen" - stopMaxFailoverDurationExceeded = "MaxDurationExceeded" - stopShutdown = "Shutdown" + stopMaxFailoverDurationExceeded stopReason = "MaxDurationExceeded" + stopShutdown stopReason = "Shutdown" ) // evictedReason is used in "requestsEvicted" as "Reason" label. @@ -119,9 +119,10 @@ type evictedReason string var evictReasons = []evictedReason{evictedContextDone, evictedBufferFull, evictedWindowExceeded} const ( - evictedContextDone evictedReason = "ContextDone" - evictedBufferFull = "BufferFull" - evictedWindowExceeded = "WindowExceeded" + evictedContextDone evictedReason = "ContextDone" + //lint: ignore SA9004 ok not to use explicit type here because implicit type string is correct + evictedBufferFull = "BufferFull" + evictedWindowExceeded = "WindowExceeded" ) // skippedReason is used in "requestsSkipped" as "Reason" label. diff --git a/go/vt/vtgate/engine/delete.go b/go/vt/vtgate/engine/delete.go index 4930d0933fc..53041050d4e 100644 --- a/go/vt/vtgate/engine/delete.go +++ b/go/vt/vtgate/engine/delete.go @@ -19,6 +19,7 @@ package engine import ( "encoding/json" "fmt" + "time" "vitess.io/vitess/go/jsonutil" "vitess.io/vitess/go/sqltypes" @@ -63,6 +64,9 @@ type Delete struct { // Option to override the standard behavior and allow a multi-shard delete // to use single round trip autocommit. MultiShardAutocommit bool + + // QueryTimeout contains the optional timeout (in milliseconds) to apply to this query + QueryTimeout int } // MarshalJSON serializes the Delete into a JSON representation. @@ -84,6 +88,7 @@ func (del *Delete) MarshalJSON() ([]byte, error) { Table string `json:",omitempty"` OwnedVindexQuery string `json:",omitempty"` MultiShardAutocommit bool `json:",omitempty"` + QueryTimeout int `json:",omitempty"` }{ Opcode: del.Opcode, Keyspace: del.Keyspace, @@ -93,6 +98,7 @@ func (del *Delete) MarshalJSON() ([]byte, error) { Table: tname, OwnedVindexQuery: del.OwnedVindexQuery, MultiShardAutocommit: del.MultiShardAutocommit, + QueryTimeout: del.QueryTimeout, } return jsonutil.MarshalNoEscape(marshalDelete) } @@ -141,6 +147,11 @@ func (del *Delete) RouteType() string { // Execute performs a non-streaming exec. func (del *Delete) Execute(vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { + if del.QueryTimeout != 0 { + cancel := vcursor.SetContextTimeout(time.Duration(del.QueryTimeout) * time.Millisecond) + defer cancel() + } + switch del.Opcode { case DeleteUnsharded: return del.execDeleteUnsharded(vcursor, bindVars) diff --git a/go/vt/vtgate/engine/insert.go b/go/vt/vtgate/engine/insert.go index a1ead406736..f19377f1f38 100644 --- a/go/vt/vtgate/engine/insert.go +++ b/go/vt/vtgate/engine/insert.go @@ -21,6 +21,7 @@ import ( "fmt" "strconv" "strings" + "time" "vitess.io/vitess/go/jsonutil" "vitess.io/vitess/go/sqltypes" @@ -76,6 +77,9 @@ type Insert struct { // However some application use cases would prefer that the statement partially // succeed in order to get the performance benefits of autocommit. MultiShardAutocommit bool + + // QueryTimeout contains the optional timeout (in milliseconds) to apply to this query + QueryTimeout int } // NewQueryInsert creates an Insert with a query string. @@ -127,6 +131,7 @@ func (ins *Insert) MarshalJSON() ([]byte, error) { Mid []string `json:",omitempty"` Suffix string `json:",omitempty"` MultiShardAutocommit bool `json:",omitempty"` + QueryTimeout int `json:",omitempty"` }{ Opcode: ins.Opcode, Keyspace: ins.Keyspace, @@ -138,6 +143,7 @@ func (ins *Insert) MarshalJSON() ([]byte, error) { Mid: ins.Mid, Suffix: ins.Suffix, MultiShardAutocommit: ins.MultiShardAutocommit, + QueryTimeout: ins.QueryTimeout, } return jsonutil.MarshalNoEscape(marshalInsert) } @@ -191,6 +197,11 @@ func (ins *Insert) RouteType() string { // Execute performs a non-streaming exec. func (ins *Insert) Execute(vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { + if ins.QueryTimeout != 0 { + cancel := vcursor.SetContextTimeout(time.Duration(ins.QueryTimeout) * time.Millisecond) + defer cancel() + } + switch ins.Opcode { case InsertUnsharded: return ins.execInsertUnsharded(vcursor, bindVars) @@ -445,9 +456,7 @@ func (ins *Insert) processPrimary(vcursor VCursor, vindexKeys [][]sqltypes.Value var flattenedVindexKeys []sqltypes.Value // TODO: @rafael - this will change once vindex Primary keys also support multicolumns for _, val := range vindexKeys { - for _, internalVal := range val { - flattenedVindexKeys = append(flattenedVindexKeys, internalVal) - } + flattenedVindexKeys = append(flattenedVindexKeys, val...) } destinations, err := colVindex.Vindex.Map(vcursor, flattenedVindexKeys) diff --git a/go/vt/vtgate/engine/limit_test.go b/go/vt/vtgate/engine/limit_test.go index 029c548b073..890458fde53 100644 --- a/go/vt/vtgate/engine/limit_test.go +++ b/go/vt/vtgate/engine/limit_test.go @@ -90,12 +90,6 @@ func TestLimitExecute(t *testing.T) { } // Test with limit higher than input. - wantResult = sqltypes.MakeTestResult( - fields, - "a|1", - "b|2", - "c|3", - ) inputResult = sqltypes.MakeTestResult( fields, "a|1", @@ -114,8 +108,8 @@ func TestLimitExecute(t *testing.T) { if err != nil { t.Error(err) } - if !reflect.DeepEqual(result, inputResult) { - t.Errorf("l.Execute:\n%v, want\n%v", result, inputResult) + if !reflect.DeepEqual(result, wantResult) { + t.Errorf("l.Execute:\n%v, want\n%v", result, wantResult) } // Test with bind vars. diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index 6760c42dd62..77536a61c17 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -29,13 +29,14 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" ) -// SeqVarName is a reserved bind var name for sequence values. -const SeqVarName = "__seq" - -// ListVarName is a reserved bind var name for list vars. -// This is used for sending different IN clause values -// to different shards. -const ListVarName = "__vals" +const ( + // SeqVarName is a reserved bind var name for sequence values. + SeqVarName = "__seq" + // ListVarName is a reserved bind var name for list vars. + // This is used for sending different IN clause values + // to different shards. + ListVarName = "__vals" +) // VCursor defines the interface the engine will use // to execute routes. diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index fb6678bd23c..f24dd6f2e4b 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -143,7 +143,8 @@ func (route *Route) MarshalJSON() ([]byte, error) { // RouteOpcode is a number representing the opcode // for the Route primitve. Adding new opcodes here -// will require review of the join code in planbuilder. +// will require review of the join code and +// the finalizeOptions code in planbuilder. type RouteOpcode int // This is the list of RouteOpcode values. @@ -170,6 +171,8 @@ const ( SelectNext // SelectDBA is for executing a DBA statement. SelectDBA + // SelectReference is for fetching from a reference table. + SelectReference ) var routeName = map[RouteOpcode]string{ @@ -180,6 +183,7 @@ var routeName = map[RouteOpcode]string{ SelectScatter: "SelectScatter", SelectNext: "SelectNext", SelectDBA: "SelectDBA", + SelectReference: "SelectReference", } var ( @@ -211,16 +215,13 @@ func (route *Route) Execute(vcursor VCursor, bindVars map[string]*querypb.BindVa } func (route *Route) execute(vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { - switch route.Opcode { - case SelectNext, SelectDBA: - return execAnyShard(vcursor, route.Query, bindVars, route.Keyspace) - } - var rss []*srvtopo.ResolvedShard var bvs []map[string]*querypb.BindVariable var err error switch route.Opcode { - case SelectUnsharded, SelectScatter: + case SelectUnsharded, SelectNext, SelectDBA, SelectReference: + rss, bvs, err = route.paramsAnyShard(vcursor, bindVars) + case SelectScatter: rss, bvs, err = route.paramsAllShards(vcursor, bindVars) case SelectEqual, SelectEqualUnique: rss, bvs, err = route.paramsSelectEqual(vcursor, bindVars) @@ -277,7 +278,9 @@ func (route *Route) StreamExecute(vcursor VCursor, bindVars map[string]*querypb. defer cancel() } switch route.Opcode { - case SelectUnsharded, SelectScatter: + case SelectUnsharded, SelectNext, SelectDBA, SelectReference: + rss, bvs, err = route.paramsAnyShard(vcursor, bindVars) + case SelectScatter: rss, bvs, err = route.paramsAllShards(vcursor, bindVars) case SelectEqual, SelectEqualUnique: rss, bvs, err = route.paramsSelectEqual(vcursor, bindVars) @@ -321,7 +324,7 @@ func (route *Route) GetFields(vcursor VCursor, bindVars map[string]*querypb.Bind } if len(rss) != 1 { // This code is unreachable. It's just a sanity check. - return nil, fmt.Errorf("No shards for keyspace: %s", route.Keyspace.Name) + return nil, fmt.Errorf("no shards for keyspace: %s", route.Keyspace.Name) } qr, err := execShard(vcursor, route.FieldQuery, bindVars, rss[0], false /* isDML */, false /* canAutocommit */) if err != nil { @@ -342,6 +345,18 @@ func (route *Route) paramsAllShards(vcursor VCursor, bindVars map[string]*queryp return rss, multiBindVars, nil } +func (route *Route) paramsAnyShard(vcursor VCursor, bindVars map[string]*querypb.BindVariable) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { + rss, _, err := vcursor.ResolveDestinations(route.Keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) + if err != nil { + return nil, nil, vterrors.Wrap(err, "paramsAnyShard") + } + multiBindVars := make([]map[string]*querypb.BindVariable, len(rss)) + for i := range multiBindVars { + multiBindVars[i] = bindVars + } + return rss, multiBindVars, nil +} + func (route *Route) paramsSelectEqual(vcursor VCursor, bindVars map[string]*querypb.BindVariable) ([]*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, error) { key, err := route.Values[0].ResolveValue(bindVars) if err != nil { @@ -452,20 +467,6 @@ func resolveSingleShard(vcursor VCursor, vindex vindexes.Vindex, keyspace *vinde return rss[0], ksid, nil } -func execAnyShard(vcursor VCursor, query string, bindVars map[string]*querypb.BindVariable, keyspace *vindexes.Keyspace) (*sqltypes.Result, error) { - rss, _, err := vcursor.ResolveDestinations(keyspace.Name, nil, []key.Destination{key.DestinationAnyShard{}}) - if err != nil { - // TODO(alainjobart): this eats the error code. Use vterrors.Wrapf instead. - // And audit the entire file for it. - return nil, fmt.Errorf("execAnyShard: %v", err) - } - if len(rss) != 1 { - // This code is unreachable. It's just a sanity check. - return nil, fmt.Errorf("No shards for keyspace: %s", keyspace.Name) - } - return vcursor.ExecuteStandalone(query, bindVars, rss[0]) -} - func execShard(vcursor VCursor, query string, bindVars map[string]*querypb.BindVariable, rs *srvtopo.ResolvedShard, isDML, canAutocommit bool) (*sqltypes.Result, error) { autocommit := canAutocommit && vcursor.AutocommitApproval() result, errs := vcursor.ExecuteMultiShard([]*srvtopo.ResolvedShard{rs}, []*querypb.BoundQuery{ diff --git a/go/vt/vtgate/engine/route_test.go b/go/vt/vtgate/engine/route_test.go index 682e6f0d788..ca403fbffcf 100644 --- a/go/vt/vtgate/engine/route_test.go +++ b/go/vt/vtgate/engine/route_test.go @@ -54,7 +54,7 @@ func TestSelectUnsharded(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `ExecuteMultiShard ks.0: dummy_select {} false false`, }) expectResult(t, "sel.Execute", result, defaultSelectResult) @@ -65,7 +65,7 @@ func TestSelectUnsharded(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `StreamExecuteMulti dummy_select ks.0: {} `, }) expectResult(t, "sel.StreamExecute", result, defaultSelectResult) @@ -446,13 +446,17 @@ func TestSelectNext(t *testing.T) { } vc.ExpectLog(t, []string{ `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, - `ExecuteStandalone dummy_select ks -20`, + `ExecuteMultiShard ks.-20: dummy_select {} false false`, }) expectResult(t, "sel.Execute", result, defaultSelectResult) vc.Rewind() - _, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) - expectError(t, "sel.StreamExecute", err, `query "dummy_select" cannot be used for streaming`) + result, _ = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, + `StreamExecuteMulti dummy_select ks.-20: {} `, + }) + expectResult(t, "sel.StreamExecute", result, defaultSelectResult) } func TestSelectDBA(t *testing.T) { @@ -476,13 +480,51 @@ func TestSelectDBA(t *testing.T) { } vc.ExpectLog(t, []string{ `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, - `ExecuteStandalone dummy_select ks -20`, + `ExecuteMultiShard ks.-20: dummy_select {} false false`, }) expectResult(t, "sel.Execute", result, defaultSelectResult) vc.Rewind() - _, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) - expectError(t, "sel.StreamExecute", err, `query "dummy_select" cannot be used for streaming`) + result, _ = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, + `StreamExecuteMulti dummy_select ks.-20: {} `, + }) + expectResult(t, "sel.StreamExecute", result, defaultSelectResult) +} + +func TestSelectReference(t *testing.T) { + sel := NewRoute( + SelectReference, + &vindexes.Keyspace{ + Name: "ks", + Sharded: true, + }, + "dummy_select", + "dummy_select_field", + ) + + vc := &loggingVCursor{ + shards: []string{"-20", "20-"}, + results: []*sqltypes.Result{defaultSelectResult}, + } + result, err := sel.Execute(vc, map[string]*querypb.BindVariable{}, false) + if err != nil { + t.Fatal(err) + } + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, + `ExecuteMultiShard ks.-20: dummy_select {} false false`, + }) + expectResult(t, "sel.Execute", result, defaultSelectResult) + + vc.Rewind() + result, _ = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) + vc.ExpectLog(t, []string{ + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, + `StreamExecuteMulti dummy_select ks.-20: {} `, + }) + expectResult(t, "sel.StreamExecute", result, defaultSelectResult) } func TestRouteGetFields(t *testing.T) { @@ -564,7 +606,7 @@ func TestRouteSort(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `ExecuteMultiShard ks.0: dummy_select {} false false`, }) wantResult := sqltypes.MakeTestResult( @@ -650,7 +692,7 @@ func TestRouteSortTruncate(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `ExecuteMultiShard ks.0: dummy_select {} false false`, }) wantResult := sqltypes.MakeTestResult( @@ -696,7 +738,7 @@ func TestRouteStreamTruncate(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `ExecuteMultiShard ks.0: dummy_select {} false false`, }) wantResult := sqltypes.MakeTestResult( @@ -743,7 +785,7 @@ func TestRouteStreamSortTruncate(t *testing.T) { t.Fatal(err) } vc.ExpectLog(t, []string{ - `ResolveDestinations ks [] Destinations:DestinationAllShards()`, + `ResolveDestinations ks [] Destinations:DestinationAnyShard()`, `StreamExecuteMulti dummy_select ks.0: {} `, }) @@ -774,11 +816,11 @@ func TestParamsFail(t *testing.T) { vc := &loggingVCursor{shardErr: errors.New("shard error")} _, err := sel.Execute(vc, map[string]*querypb.BindVariable{}, false) - expectError(t, "sel.Execute err", err, "paramsAllShards: shard error") + expectError(t, "sel.Execute err", err, "paramsAnyShard: shard error") vc.Rewind() _, err = wrapStreamExecute(sel, vc, map[string]*querypb.BindVariable{}, false) - expectError(t, "sel.StreamExecute err", err, "paramsAllShards: shard error") + expectError(t, "sel.StreamExecute err", err, "paramsAnyShard: shard error") } func TestExecFail(t *testing.T) { diff --git a/go/vt/vtgate/engine/update.go b/go/vt/vtgate/engine/update.go index 83a48c32f31..81857997947 100644 --- a/go/vt/vtgate/engine/update.go +++ b/go/vt/vtgate/engine/update.go @@ -19,6 +19,7 @@ package engine import ( "encoding/json" "fmt" + "time" "vitess.io/vitess/go/jsonutil" "vitess.io/vitess/go/sqltypes" @@ -66,6 +67,9 @@ type Update struct { // Option to override the standard behavior and allow a multi-shard update // to use single round trip autocommit. MultiShardAutocommit bool + + // QueryTimeout contains the optional timeout (in milliseconds) to apply to this query + QueryTimeout int } // MarshalJSON serializes the Update into a JSON representation. @@ -88,6 +92,7 @@ func (upd *Update) MarshalJSON() ([]byte, error) { Table string `json:",omitempty"` OwnedVindexQuery string `json:",omitempty"` MultiShardAutocommit bool `json:",omitempty"` + QueryTimeout int `json:",omitempty"` }{ Opcode: upd.Opcode, Keyspace: upd.Keyspace, @@ -98,6 +103,7 @@ func (upd *Update) MarshalJSON() ([]byte, error) { Table: tname, OwnedVindexQuery: upd.OwnedVindexQuery, MultiShardAutocommit: upd.MultiShardAutocommit, + QueryTimeout: upd.QueryTimeout, } return jsonutil.MarshalNoEscape(marshalUpdate) } @@ -145,6 +151,11 @@ func (upd *Update) RouteType() string { // Execute performs a non-streaming exec. func (upd *Update) Execute(vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) { + if upd.QueryTimeout != 0 { + cancel := vcursor.SetContextTimeout(time.Duration(upd.QueryTimeout) * time.Millisecond) + defer cancel() + } + switch upd.Opcode { case UpdateUnsharded: return upd.execUpdateUnsharded(vcursor, bindVars) diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 8ebddca2055..6e7f5428a88 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -775,7 +775,7 @@ func (e *Executor) handleShow(ctx context.Context, safeSession *SafeSession, sql show.ShowTablesOpt.DbName = "" } sql = sqlparser.String(show) - case sqlparser.KeywordString(sqlparser.DATABASES), sqlparser.KeywordString(sqlparser.VITESS_KEYSPACES): + case sqlparser.KeywordString(sqlparser.DATABASES), sqlparser.KeywordString(sqlparser.SCHEMAS), sqlparser.KeywordString(sqlparser.VITESS_KEYSPACES): keyspaces, err := e.resolver.resolver.GetAllKeyspaces(ctx) if err != nil { return nil, err diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index 26f86cbae6f..1f522d839ba 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -267,6 +267,11 @@ var unshardedVSchema = ` } ` +const ( + testBufferSize = 10 + testCacheSize = int64(10) +) + type DestinationAnyShardPickerFirstShard struct{} func (dp DestinationAnyShardPickerFirstShard) PickShard(shardCount int) int { @@ -328,9 +333,6 @@ func init() { vindexes.Register("keyrange_lookuper_unique", newKeyRangeLookuperUnique) } -const testBufferSize = 10 -const testCacheSize = int64(10) - func createExecutorEnv() (executor *Executor, sbc1, sbc2, sbclookup *sandboxconn.SandboxConn) { cell := "aa" hc := discovery.NewFakeHealthCheck() diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 365bfd55d52..afa03b54e70 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -599,7 +599,7 @@ func TestExecutorShow(t *testing.T) { executor, _, _, sbclookup := createExecutorEnv() session := NewSafeSession(&vtgatepb.Session{TargetString: "@master"}) - for _, query := range []string{"show databases", "show vitess_keyspaces"} { + for _, query := range []string{"show databases", "show schemas", "show vitess_keyspaces"} { qr, err := executor.Execute(context.Background(), "TestExecute", session, query, nil) if err != nil { t.Error(err) @@ -674,7 +674,7 @@ func TestExecutorShow(t *testing.T) { t.Errorf("%v:\n%+v, want\n%+v", query, qr, wantqr) } - qr, err = executor.Execute(context.Background(), "TestExecute", session, "show create table unknown_table", nil) + _, err = executor.Execute(context.Background(), "TestExecute", session, "show create table unknown_table", nil) if err != errNoKeyspace { t.Errorf("Got: %v. Want: %v", err, errNoKeyspace) } diff --git a/go/vt/vtgate/fakerpcvtgateconn/conn.go b/go/vt/vtgate/fakerpcvtgateconn/conn.go index bb8a03f1804..19e57f932d2 100644 --- a/go/vt/vtgate/fakerpcvtgateconn/conn.go +++ b/go/vt/vtgate/fakerpcvtgateconn/conn.go @@ -184,8 +184,7 @@ func (conn *FakeVTGateConn) Execute(ctx context.Context, session *vtgatepb.Sessi return nil, nil, fmt.Errorf( "Execute: %+v, want %+v", query, response.execQuery) } - var reply sqltypes.Result - reply = *response.reply + reply := *response.reply s := newSession(true, "test_keyspace", []string{}, topodatapb.TabletType_MASTER) return s, &reply, nil } @@ -254,8 +253,7 @@ func (conn *FakeVTGateConn) ExecuteShards(ctx context.Context, sql string, keysp return nil, nil, fmt.Errorf( "ExecuteShards: %+v, want %+v", query, response.shardQuery) } - var reply sqltypes.Result - reply = *response.reply + reply := *response.reply if s != nil { s = newSession(true, keyspace, shards, tabletType) } diff --git a/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go b/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go index 64db444a4ec..c115bf0484d 100644 --- a/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go +++ b/go/vt/vtgate/grpcvtgateconn/conn_rpc_test.go @@ -144,7 +144,7 @@ func TestGRPCVTGateConnAuth(t *testing.T) { t.Fatalf("dial failed: %v", err) } vtgateconntest.RegisterTestDialProtocol(client) - conn, err := vtgateconn.DialProtocol(context.Background(), "test", "") + conn, _ := vtgateconn.DialProtocol(context.Background(), "test", "") // run the test suite _, err = conn.Begin(context.Background()) want := "rpc error: code = Unauthenticated desc = username and password must be provided" diff --git a/go/vt/vtgate/grpcvtgateservice/server.go b/go/vt/vtgate/grpcvtgateservice/server.go index 5afc5e1b934..c56384739cd 100644 --- a/go/vt/vtgate/grpcvtgateservice/server.go +++ b/go/vt/vtgate/grpcvtgateservice/server.go @@ -125,7 +125,6 @@ func (vtg *VTGate) Execute(ctx context.Context, request *vtgatepb.ExecuteRequest func (vtg *VTGate) ExecuteBatch(ctx context.Context, request *vtgatepb.ExecuteBatchRequest) (response *vtgatepb.ExecuteBatchResponse, err error) { defer vtg.server.HandlePanic(&err) ctx = withCallerIDContext(ctx, request.CallerId) - results := make([]sqltypes.QueryResponse, len(request.Queries)) sqlQueries := make([]string, len(request.Queries)) bindVars := make([]map[string]*querypb.BindVariable, len(request.Queries)) for queryNum, query := range request.Queries { @@ -143,7 +142,7 @@ func (vtg *VTGate) ExecuteBatch(ctx context.Context, request *vtgatepb.ExecuteBa if session.Options == nil { session.Options = request.Options } - session, results, err = vtg.server.ExecuteBatch(ctx, session, sqlQueries, bindVars) + session, results, err := vtg.server.ExecuteBatch(ctx, session, sqlQueries, bindVars) return &vtgatepb.ExecuteBatchResponse{ Results: sqltypes.QueryResponsesToProto3(results), Session: session, diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go index 221151d892f..63ada48de8e 100644 --- a/go/vt/vtgate/planbuilder/builder.go +++ b/go/vt/vtgate/planbuilder/builder.go @@ -31,7 +31,7 @@ import ( // builder defines the interface that a primitive must // satisfy. type builder interface { - // Order is the execution order of the primitve. If there are subprimitves, + // Order is the execution order of the primitive. If there are subprimitives, // the order is one above the order of the subprimitives. // This is because the primitive executes its subprimitives first and // processes their results to generate its own values. @@ -48,9 +48,6 @@ type builder interface { // execute before this one. Reorder(int) - // Primitve returns the underlying primitive. - Primitive() engine.Primitive - // First returns the first builder of the tree, // which is usually the left most. First() builder @@ -103,13 +100,17 @@ type builder interface { // resultColumn, whereas PushSelect guarantees the addition of a new // result column and returns a distinct symbol for it. SupplyCol(col *sqlparser.ColName) (rc *resultColumn, colnum int) + + // Primitive returns the underlying primitive. + // This function should only be called after Wireup is finished. + Primitive() engine.Primitive } // ContextVSchema defines the interface for this package to fetch // info about tables. type ContextVSchema interface { FindTable(tablename sqlparser.TableName) (*vindexes.Table, string, topodatapb.TabletType, key.Destination, error) - FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) + FindTablesOrVindex(tablename sqlparser.TableName) ([]*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) DefaultKeyspace() (*vindexes.Keyspace, error) TargetString() string } diff --git a/go/vt/vtgate/planbuilder/delete.go b/go/vt/vtgate/planbuilder/delete.go index 0d20ae4556c..6fd807c0dc7 100644 --- a/go/vt/vtgate/planbuilder/delete.go +++ b/go/vt/vtgate/planbuilder/delete.go @@ -30,53 +30,48 @@ import ( // buildDeletePlan builds the instructions for a DELETE statement. func buildDeletePlan(del *sqlparser.Delete, vschema ContextVSchema) (*engine.Delete, error) { - edel := &engine.Delete{ - Query: generateQuery(del), - } + edel := &engine.Delete{} pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(del))) - if err := pb.processTableExprs(del.TableExprs); err != nil { + ro, err := pb.processDMLTable(del.TableExprs) + if err != nil { return nil, err } - rb, ok := pb.bldr.(*route) - if !ok { - return nil, errors.New("unsupported: multi-table/vindex delete statement in sharded keyspace") - } - edel.Keyspace = rb.ERoute.Keyspace + edel.Query = generateQuery(del) + edel.Keyspace = ro.eroute.Keyspace if !edel.Keyspace.Sharded { // We only validate non-table subexpressions because the previous analysis has already validated them. - if !pb.validateSubquerySamePlan(del.Targets, del.Where, del.OrderBy, del.Limit) { + if !pb.finalizeUnshardedDMLSubqueries(del.Targets, del.Where, del.OrderBy, del.Limit) { return nil, errors.New("unsupported: sharded subqueries in DML") } edel.Opcode = engine.DeleteUnsharded + // Generate query after all the analysis. Otherwise table name substitutions for + // routed tables won't happen. + edel.Query = generateQuery(del) return edel, nil } - if del.Targets != nil || len(pb.st.tables) != 1 { + if del.Targets != nil || ro.vschemaTable == nil { return nil, errors.New("unsupported: multi-table delete statement in sharded keyspace") } if hasSubquery(del) { return nil, errors.New("unsupported: subqueries in sharded DML") } - var vindexTable *vindexes.Table - for _, tval := range pb.st.tables { - vindexTable = tval.vindexTable - } - edel.Table = vindexTable - if edel.Table == nil { - return nil, errors.New("internal error: table.vindexTable is mysteriously nil") - } - var err error + edel.Table = ro.vschemaTable + // Generate query after all the analysis. Otherwise table name substitutions for + // routed tables won't happen. + edel.Query = generateQuery(del) directives := sqlparser.ExtractCommentDirectives(del.Comments) if directives.IsSet(sqlparser.DirectiveMultiShardAutocommit) { edel.MultiShardAutocommit = true } - if rb.ERoute.TargetDestination != nil { - if rb.ERoute.TargetTabletType != topodatapb.TabletType_MASTER { + edel.QueryTimeout = queryTimeout(directives) + if ro.eroute.TargetDestination != nil { + if ro.eroute.TargetTabletType != topodatapb.TabletType_MASTER { return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unsupported: DELETE statement with a replica target") } edel.Opcode = engine.DeleteByDestination - edel.TargetDestination = rb.ERoute.TargetDestination + edel.TargetDestination = ro.eroute.TargetDestination return edel, nil } edel.Vindex, edel.Values, err = getDMLRouting(del.Where, edel.Table) diff --git a/go/vt/vtgate/planbuilder/expr.go b/go/vt/vtgate/planbuilder/expr.go index 84ee7b9646a..d37cbec543c 100644 --- a/go/vt/vtgate/planbuilder/expr.go +++ b/go/vt/vtgate/planbuilder/expr.go @@ -154,7 +154,7 @@ func (pb *primitiveBuilder) findOrigin(expr sqlparser.Expr) (pullouts []*pullout switch { // If it's last_insert_id, ensure it's a single unsharded route. case node.Name.EqualString("last_insert_id"): - if rb, isRoute := pb.bldr.(*route); !isRoute || rb.ERoute.Keyspace.Sharded { + if rb, isRoute := pb.bldr.(*route); !isRoute || !rb.removeShardedOptions() { return false, errors.New("unsupported: LAST_INSERT_ID is only allowed for unsharded keyspaces") } } @@ -169,8 +169,7 @@ func (pb *primitiveBuilder) findOrigin(expr sqlparser.Expr) (pullouts []*pullout highestRoute, _ := highestOrigin.(*route) for _, sqi := range subqueries { subroute, _ := sqi.bldr.(*route) - if highestRoute != nil && subroute != nil && highestRoute.SubqueryCanMerge(pb, subroute) { - subroute.Redirect = highestRoute + if highestRoute != nil && subroute != nil && highestRoute.MergeSubquery(pb, subroute) { continue } if sqi.origin != nil { @@ -241,14 +240,17 @@ func hasSubquery(node sqlparser.SQLNode) bool { return has } -func (pb *primitiveBuilder) validateSubquerySamePlan(nodes ...sqlparser.SQLNode) bool { +func (pb *primitiveBuilder) finalizeUnshardedDMLSubqueries(nodes ...sqlparser.SQLNode) bool { var keyspace string if rb, ok := pb.bldr.(*route); ok { - keyspace = rb.ERoute.Keyspace.Name + keyspace = rb.routeOptions[0].eroute.Keyspace.Name + } else { + // This code is unreachable because the caller checks. + return false } - samePlan := true for _, node := range nodes { + samePlan := true inSubQuery := false _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { switch nodeType := node.(type) { @@ -269,10 +271,13 @@ func (pb *primitiveBuilder) validateSubquerySamePlan(nodes ...sqlparser.SQLNode) samePlan = false return false, errors.New("dummy") } - if innerRoute.ERoute.Keyspace.Name != keyspace { + if !innerRoute.removeOptionsWithUnmatchedKeyspace(keyspace) { samePlan = false return false, errors.New("dummy") } + for _, sub := range innerRoute.routeOptions[0].substitutions { + *sub.oldExpr = *sub.newExpr + } case *sqlparser.Union: if !inSubQuery { return true, nil @@ -287,7 +292,7 @@ func (pb *primitiveBuilder) validateSubquerySamePlan(nodes ...sqlparser.SQLNode) samePlan = false return false, errors.New("dummy") } - if innerRoute.ERoute.Keyspace.Name != keyspace { + if !innerRoute.removeOptionsWithUnmatchedKeyspace(keyspace) { samePlan = false return false, errors.New("dummy") } diff --git a/go/vt/vtgate/planbuilder/from.go b/go/vt/vtgate/planbuilder/from.go index dd598cc9510..d7f467890fc 100644 --- a/go/vt/vtgate/planbuilder/from.go +++ b/go/vt/vtgate/planbuilder/from.go @@ -17,6 +17,7 @@ limitations under the License. package planbuilder import ( + "errors" "fmt" "vitess.io/vitess/go/sqltypes" @@ -27,6 +28,22 @@ import ( // This file has functions to analyze the FROM clause. +// processDMLTable analyzes the FROM clause for DMLs and returns a routeOption. +func (pb *primitiveBuilder) processDMLTable(tableExprs sqlparser.TableExprs) (*routeOption, error) { + if err := pb.processTableExprs(tableExprs); err != nil { + return nil, err + } + rb, ok := pb.bldr.(*route) + if !ok { + return nil, errors.New("unsupported: multi-shard or vindex write statement") + } + ro := rb.routeOptions[0] + for _, sub := range ro.substitutions { + *sub.oldExpr = *sub.newExpr + } + return ro, nil +} + // processTableExprs analyzes the FROM clause. It produces a builder // with all the routes identified. func (pb *primitiveBuilder) processTableExprs(tableExprs sqlparser.TableExprs) error { @@ -102,32 +119,47 @@ func (pb *primitiveBuilder) processAliasedTable(tableExpr *sqlparser.AliasedTabl // build a route primitive that has the subquery in its // FROM clause. This allows for other constructs to be // later pushed into it. - table := &vindexes.Table{ - Keyspace: subroute.ERoute.Keyspace, - } - for _, rc := range subroute.ResultColumns() { - if rc.column.Vindex == nil { - continue + rb, st := newRoute(&sqlparser.Select{From: sqlparser.TableExprs([]sqlparser.TableExpr{tableExpr})}) + + // The subquery needs to be represented as a new logical table in the symtab. + // The new route will inherit the routeOptions of the underlying subquery. + // For this, we first build new vschema tables based on the columns returned + // by the subquery, and re-expose possible vindexes. When added to the symtab, + // a new set of column references will be generated against the new tables, + // and those vindex maps will be returned. They have to replace the old vindex + // maps of the inherited route options. + vschemaTables := make([]*vindexes.Table, 0, len(subroute.routeOptions)) + for _, ro := range subroute.routeOptions { + vst := &vindexes.Table{ + Keyspace: ro.eroute.Keyspace, } - // Check if a colvindex of the same name already exists. - // Dups are not allowed in subqueries in this situation. - for _, colVindex := range table.ColumnVindexes { - if colVindex.Columns[0].Equal(rc.alias) { - return fmt.Errorf("duplicate column aliases: %v", rc.alias) + vschemaTables = append(vschemaTables, vst) + for _, rc := range subroute.ResultColumns() { + vindex, ok := ro.vindexMap[rc.column] + if !ok { + continue } + // Check if a colvindex of the same name already exists. + // Dups are not allowed in subqueries in this situation. + for _, colVindex := range vst.ColumnVindexes { + if colVindex.Columns[0].Equal(rc.alias) { + return fmt.Errorf("duplicate column aliases: %v", rc.alias) + } + } + vst.ColumnVindexes = append(vst.ColumnVindexes, &vindexes.ColumnVindex{ + Columns: []sqlparser.ColIdent{rc.alias}, + Vindex: vindex, + }) } - table.ColumnVindexes = append(table.ColumnVindexes, &vindexes.ColumnVindex{ - Columns: []sqlparser.ColIdent{rc.alias}, - Vindex: rc.column.Vindex, - }) } - rb, st := newRoute( - &sqlparser.Select{From: sqlparser.TableExprs([]sqlparser.TableExpr{tableExpr})}, - subroute.ERoute, - subroute.condition, - ) - // AddVindexTable can never fail because symtab is empty. - _ = st.AddVindexTable(sqlparser.TableName{Name: tableExpr.As}, table, rb) + vindexMaps, err := st.AddVSchemaTable(sqlparser.TableName{Name: tableExpr.As}, vschemaTables, rb) + if err != nil { + return err + } + for i, ro := range subroute.routeOptions { + ro.SubqueryToTable(rb, vindexMaps[i]) + } + rb.routeOptions = subroute.routeOptions subroute.Redirect = rb pb.bldr, pb.st = rb, st return nil @@ -148,13 +180,13 @@ func (pb *primitiveBuilder) buildTablePrimitive(tableExpr *sqlparser.AliasedTabl if err != nil { return err } - rb, st := newRoute(sel, nil, nil) - rb.ERoute = engine.NewSimpleRoute(engine.SelectDBA, ks) + rb, st := newRoute(sel) + rb.routeOptions = []*routeOption{newSimpleRouteOption(rb, engine.NewSimpleRoute(engine.SelectDBA, ks))} pb.bldr, pb.st = rb, st return nil } - table, vindex, _, destTableType, destTarget, err := pb.vschema.FindTableOrVindex(tableName) + vschemaTables, vindex, _, destTableType, destTarget, err := pb.vschema.FindTablesOrVindex(tableName) if err != nil { return err } @@ -163,29 +195,56 @@ func (pb *primitiveBuilder) buildTablePrimitive(tableExpr *sqlparser.AliasedTabl return nil } - rb, st := newRoute(sel, nil, nil) + rb, st := newRoute(sel) pb.bldr, pb.st = rb, st - // AddVindexTable can never fail because symtab is empty. - _ = st.AddVindexTable(alias, table, rb) - - if !table.Keyspace.Sharded { - rb.ERoute = engine.NewSimpleRoute(engine.SelectUnsharded, table.Keyspace) - return nil + vindexMaps, err := st.AddVSchemaTable(alias, vschemaTables, rb) + if err != nil { + return err } - if table.Pinned == nil { - rb.ERoute = engine.NewSimpleRoute(engine.SelectScatter, table.Keyspace) - rb.ERoute.TargetDestination = destTarget - rb.ERoute.TargetTabletType = destTableType - - return nil + for i, vst := range vschemaTables { + sub := &tableSubstitution{ + oldExpr: tableExpr, + } + if tableExpr.As.IsEmpty() { + if tableName.Name != vst.Name { + // Table name does not match. Change and alias it to old name. + sub.newExpr = &sqlparser.AliasedTableExpr{ + Expr: sqlparser.TableName{Name: vst.Name}, + As: tableName.Name, + } + } + } else { + // Table is already aliased. + if tableName.Name != vst.Name { + // Table name does not match. Change it and reuse existing alias. + sub.newExpr = &sqlparser.AliasedTableExpr{ + Expr: sqlparser.TableName{Name: vst.Name}, + As: tableExpr.As, + } + } + } + var eroute *engine.Route + switch { + case vst.Type == vindexes.TypeSequence: + eroute = engine.NewSimpleRoute(engine.SelectNext, vst.Keyspace) + case vst.Type == vindexes.TypeReference: + eroute = engine.NewSimpleRoute(engine.SelectReference, vst.Keyspace) + case !vst.Keyspace.Sharded: + eroute = engine.NewSimpleRoute(engine.SelectUnsharded, vst.Keyspace) + case vst.Pinned == nil: + eroute = engine.NewSimpleRoute(engine.SelectScatter, vst.Keyspace) + eroute.TargetDestination = destTarget + eroute.TargetTabletType = destTableType + default: + // Pinned tables have their keyspace ids already assigned. + // Use the Binary vindex, which is the identity function + // for keyspace id. Currently only dual tables are pinned. + eroute = engine.NewSimpleRoute(engine.SelectEqualUnique, vst.Keyspace) + eroute.Vindex, _ = vindexes.NewBinary("binary", nil) + eroute.Values = []sqltypes.PlanValue{{Value: sqltypes.MakeTrusted(sqltypes.VarBinary, vst.Pinned)}} + } + rb.routeOptions = append(rb.routeOptions, newRouteOption(rb, vst, sub, vindexMaps[i], eroute)) } - // Pinned tables have their keyspace ids already assigned. - // Use the Binary vindex, which is the identity function - // for keyspace id. Currently only dual tables are pinned. - eRoute := engine.NewSimpleRoute(engine.SelectEqualUnique, table.Keyspace) - eRoute.Vindex, _ = vindexes.NewBinary("binary", nil) - eRoute.Values = []sqltypes.PlanValue{{Value: sqltypes.MakeTrusted(sqltypes.VarBinary, table.Pinned)}} - rb.ERoute = eRoute return nil } @@ -225,49 +284,38 @@ func convertToLeftJoin(ajoin *sqlparser.JoinTableExpr) { } func (pb *primitiveBuilder) join(rpb *primitiveBuilder, ajoin *sqlparser.JoinTableExpr) error { + // Merge the symbol tables. In the case of a left join, we have to + // ideally create new symbols that originate from the join primitive. + // However, this is not worth it for now, because the Push functions + // verify that only valid constructs are passed through in case of left join. + err := pb.st.Merge(rpb.st) + if err != nil { + return err + } + lRoute, leftIsRoute := pb.bldr.(*route) rRoute, rightIsRoute := rpb.bldr.(*route) - if leftIsRoute && rightIsRoute { - // If both are routes, they have an opportunity - // to merge into one. - if lRoute.ERoute.Keyspace.Name != rRoute.ERoute.Keyspace.Name { - goto nomerge - } - // We don't have to check on SelectNext because the syntax - // doesn't allow joins. - switch lRoute.ERoute.Opcode { - case engine.SelectUnsharded: - if rRoute.ERoute.Opcode == engine.SelectUnsharded { - return pb.mergeRoutes(rpb, ajoin) - } - return errIntermixingUnsupported - case engine.SelectDBA: - if rRoute.ERoute.Opcode == engine.SelectDBA { - return pb.mergeRoutes(rpb, ajoin) - } - return errIntermixingUnsupported - } - - // Both route are sharded routes. For ',' joins (ajoin==nil), don't - // analyze mergeability. - if ajoin == nil { - goto nomerge - } + if !leftIsRoute || !rightIsRoute { + return newJoin(pb, rpb, ajoin) + } - // Both route are sharded routes. Analyze join condition for merging. - for _, filter := range splitAndExpression(nil, ajoin.Condition.On) { - if pb.isSameRoute(rpb, filter) { - return pb.mergeRoutes(rpb, ajoin) + // Try merging the routes. + isLeftJoin := ajoin != nil && ajoin.Join == sqlparser.LeftJoinStr + var mergedRouteOptions []*routeOption +outer: + for _, lro := range lRoute.routeOptions { + for _, rro := range rRoute.routeOptions { + if lro.JoinCanMerge(pb, rro, ajoin) { + lro.MergeJoin(rro, isLeftJoin) + mergedRouteOptions = append(mergedRouteOptions, lro) + continue outer } } - - // Both l & r routes point to the same shard. - if lRoute.isSameShardedRoute(rRoute) == nil { - return pb.mergeRoutes(rpb, ajoin) - } + } + if len(mergedRouteOptions) != 0 { + return pb.mergeRoutes(rpb, mergedRouteOptions, ajoin) } -nomerge: return newJoin(pb, rpb, ajoin) } @@ -275,7 +323,7 @@ nomerge: // see if the primitive can be improved. The operation can fail if // the expression contains a non-pushable subquery. ajoin can be nil // if the join is on a ',' operator. -func (pb *primitiveBuilder) mergeRoutes(rpb *primitiveBuilder, ajoin *sqlparser.JoinTableExpr) error { +func (pb *primitiveBuilder) mergeRoutes(rpb *primitiveBuilder, routeOptions []*routeOption, ajoin *sqlparser.JoinTableExpr) error { lRoute := pb.bldr.(*route) rRoute := rpb.bldr.(*route) sel := lRoute.Select.(*sqlparser.Select) @@ -285,17 +333,12 @@ func (pb *primitiveBuilder) mergeRoutes(rpb *primitiveBuilder, ajoin *sqlparser. sel.From = append(sel.From, rhsSel.From...) } else { sel.From = sqlparser.TableExprs{ajoin} - if ajoin.Join == sqlparser.LeftJoinStr { - rpb.st.ClearVindexes() - } } - // Redirect before merging the symtabs. Merge will use Redirect - // to check if rRoute matches lRoute. rRoute.Redirect = lRoute - err := pb.st.Merge(rpb.st) - if err != nil { - return err - } + // Since the routes have merged, set st.singleRoute to point at + // the merged route. + pb.st.singleRoute = lRoute + lRoute.routeOptions = routeOptions if ajoin == nil { return nil } @@ -306,42 +349,7 @@ func (pb *primitiveBuilder) mergeRoutes(rpb *primitiveBuilder, ajoin *sqlparser. ajoin.Condition.On = expr pb.addPullouts(pullouts) for _, filter := range splitAndExpression(nil, ajoin.Condition.On) { - lRoute.UpdatePlan(pb, filter) + lRoute.UpdatePlans(pb, filter) } return nil } - -// isSameRoute returns true if the join constraint makes the routes -// mergeable by unique vindex. The constraint has to be an equality -// like a.id = b.id where both columns have the same unique vindex. -func (pb *primitiveBuilder) isSameRoute(rpb *primitiveBuilder, filter sqlparser.Expr) bool { - lRoute := pb.bldr.(*route) - rRoute := rpb.bldr.(*route) - - filter = skipParenthesis(filter) - comparison, ok := filter.(*sqlparser.ComparisonExpr) - if !ok { - return false - } - if comparison.Operator != sqlparser.EqualStr { - return false - } - left := comparison.Left - right := comparison.Right - lVindex := pb.st.Vindex(left, lRoute) - if lVindex == nil { - left, right = right, left - lVindex = pb.st.Vindex(left, lRoute) - } - if lVindex == nil || !lVindex.IsUnique() { - return false - } - rVindex := rpb.st.Vindex(right, rRoute) - if rVindex == nil { - return false - } - if rVindex != lVindex { - return false - } - return true -} diff --git a/go/vt/vtgate/planbuilder/insert.go b/go/vt/vtgate/planbuilder/insert.go index 7bacfac0aee..9ea0f21a6ed 100644 --- a/go/vt/vtgate/planbuilder/insert.go +++ b/go/vt/vtgate/planbuilder/insert.go @@ -31,32 +31,26 @@ import ( // buildInsertPlan builds the route for an INSERT statement. func buildInsertPlan(ins *sqlparser.Insert, vschema ContextVSchema) (*engine.Insert, error) { pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(ins))) - aliased := &sqlparser.AliasedTableExpr{Expr: ins.Table} - if err := pb.processAliasedTable(aliased); err != nil { + exprs := sqlparser.TableExprs{&sqlparser.AliasedTableExpr{Expr: ins.Table}} + ro, err := pb.processDMLTable(exprs) + if err != nil { return nil, err } - rb, ok := pb.bldr.(*route) - if !ok { - // This can happen only for vindexes right now. - return nil, fmt.Errorf("inserting into a vindex not allowed: %s", sqlparser.String(ins.Table)) - } - if rb.ERoute.TargetDestination != nil { + // The table might have been routed to a different one. + ins.Table = exprs[0].(*sqlparser.AliasedTableExpr).Expr.(sqlparser.TableName) + if ro.eroute.TargetDestination != nil { return nil, errors.New("unsupported: INSERT with a target destination") } - var table *vindexes.Table - for _, tval := range pb.st.tables { - table = tval.vindexTable - } - if !table.Keyspace.Sharded { - if !pb.validateSubquerySamePlan(ins) { + if !ro.vschemaTable.Keyspace.Sharded { + if !pb.finalizeUnshardedDMLSubqueries(ins) { return nil, errors.New("unsupported: sharded subquery in insert values") } - return buildInsertUnshardedPlan(ins, table, vschema) + return buildInsertUnshardedPlan(ins, ro.vschemaTable, vschema) } if ins.Action == sqlparser.ReplaceStr { return nil, errors.New("unsupported: REPLACE INTO with sharded schema") } - return buildInsertShardedPlan(ins, table) + return buildInsertShardedPlan(ins, ro.vschemaTable) } func buildInsertUnshardedPlan(ins *sqlparser.Insert, table *vindexes.Table, vschema ContextVSchema) (*engine.Insert, error) { @@ -123,6 +117,8 @@ func buildInsertShardedPlan(ins *sqlparser.Insert, table *vindexes.Table) (*engi eins.MultiShardAutocommit = true } + eins.QueryTimeout = queryTimeout(directives) + var rows sqlparser.Values switch insertValues := ins.Rows.(type) { case *sqlparser.Select, *sqlparser.Union: diff --git a/go/vt/vtgate/planbuilder/join.go b/go/vt/vtgate/planbuilder/join.go index ba3f4e4de14..ffbf9cba9eb 100644 --- a/go/vt/vtgate/planbuilder/join.go +++ b/go/vt/vtgate/planbuilder/join.go @@ -87,6 +87,8 @@ func newJoin(lpb, rpb *primitiveBuilder, ajoin *sqlparser.JoinTableExpr) error { // we mark the LHS symtab as outer scope to the RHS, just like // a subquery. This make the RHS treat the LHS symbols as external. // This will prevent constructs from escaping out of the rpb scope. + // At this point, the LHS symtab also contains symbols of the RHS. + // But the RHS will hide those, as intended. rpb.st.Outer = lpb.st if err := rpb.pushFilter(ajoin.Condition.On, sqlparser.WhereStr); err != nil { return err @@ -95,20 +97,11 @@ func newJoin(lpb, rpb *primitiveBuilder, ajoin *sqlparser.JoinTableExpr) error { return errors.New("unsupported: join with USING(column_list) clause") } } - // Merge the symbol tables. In the case of a left join, we have to - // ideally create new symbols that originate from the join primitive. - // However, this is not worth it for now, because the Push functions - // verify that only valid constructs are passed through in case of left join. - if err := lpb.st.Merge(rpb.st); err != nil { - return err - } lpb.bldr = &join{ Left: lpb.bldr, Right: rpb.bldr, ejoin: &engine.Join{ Opcode: opcode, - Left: lpb.bldr.Primitive(), - Right: rpb.bldr.Primitive(), Vars: make(map[string]int), }, } @@ -134,6 +127,8 @@ func (jb *join) Reorder(order int) { // Primitive satisfies the builder interface. func (jb *join) Primitive() engine.Primitive { + jb.ejoin.Left = jb.Left.Primitive() + jb.ejoin.Right = jb.Right.Primitive() return jb.ejoin } diff --git a/go/vt/vtgate/planbuilder/ordered_aggregate.go b/go/vt/vtgate/planbuilder/ordered_aggregate.go index 30c4ec28317..f87ae1ba67b 100644 --- a/go/vt/vtgate/planbuilder/ordered_aggregate.go +++ b/go/vt/vtgate/planbuilder/ordered_aggregate.go @@ -66,7 +66,7 @@ type orderedAggregate struct { // can handle. func (pb *primitiveBuilder) checkAggregates(sel *sqlparser.Select) (groupByHandler, error) { rb, isRoute := pb.bldr.(*route) - if isRoute && rb.IsSingle() { + if isRoute && rb.removeMultishardOptions() { return rb, nil } @@ -102,14 +102,20 @@ func (pb *primitiveBuilder) checkAggregates(sel *sqlparser.Select) (groupByHandl // shards, which will require us to perform the grouping // at the vtgate level. if sel.Distinct != "" { - for _, selectExpr := range sel.SelectExprs { - switch selectExpr := selectExpr.(type) { - case *sqlparser.AliasedExpr: - vindex := pb.st.Vindex(selectExpr.Expr, rb) - if vindex != nil && vindex.IsUnique() { - return rb, nil + success := rb.removeOptions(func(ro *routeOption) bool { + for _, selectExpr := range sel.SelectExprs { + switch selectExpr := selectExpr.(type) { + case *sqlparser.AliasedExpr: + vindex := ro.FindVindex(pb, selectExpr.Expr) + if vindex != nil && vindex.IsUnique() { + return true + } } } + return false + }) + if success { + return rb, nil } } @@ -176,40 +182,42 @@ func nodeHasAggregates(node sqlparser.SQLNode) bool { // error conditions are treated as no match for simplicity; They will be // subsequently caught downstream. func (pb *primitiveBuilder) groupByHasUniqueVindex(sel *sqlparser.Select, rb *route) bool { - for _, expr := range sel.GroupBy { - var matchedExpr sqlparser.Expr - switch node := expr.(type) { - case *sqlparser.ColName: - if expr := findAlias(node, sel.SelectExprs); expr != nil { - matchedExpr = expr - } else { - matchedExpr = node - } - case *sqlparser.SQLVal: - if node.Type != sqlparser.IntVal { - continue - } - num, err := strconv.ParseInt(string(node.Val), 0, 64) - if err != nil { - continue - } - if num < 1 || num > int64(len(sel.SelectExprs)) { + return rb.removeOptions(func(ro *routeOption) bool { + for _, expr := range sel.GroupBy { + var matchedExpr sqlparser.Expr + switch node := expr.(type) { + case *sqlparser.ColName: + if expr := findAlias(node, sel.SelectExprs); expr != nil { + matchedExpr = expr + } else { + matchedExpr = node + } + case *sqlparser.SQLVal: + if node.Type != sqlparser.IntVal { + continue + } + num, err := strconv.ParseInt(string(node.Val), 0, 64) + if err != nil { + continue + } + if num < 1 || num > int64(len(sel.SelectExprs)) { + continue + } + expr, ok := sel.SelectExprs[num-1].(*sqlparser.AliasedExpr) + if !ok { + continue + } + matchedExpr = expr.Expr + default: continue } - expr, ok := sel.SelectExprs[num-1].(*sqlparser.AliasedExpr) - if !ok { - continue + vindex := ro.FindVindex(pb, matchedExpr) + if vindex != nil && vindex.IsUnique() { + return true } - matchedExpr = expr.Expr - default: - continue } - vindex := pb.st.Vindex(matchedExpr, rb) - if vindex != nil && vindex.IsUnique() { - return true - } - } - return false + return false + }) } func findAlias(colname *sqlparser.ColName, selects sqlparser.SelectExprs) sqlparser.Expr { diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go index 31f1228f900..fe3a5ed1572 100644 --- a/go/vt/vtgate/planbuilder/plan_test.go +++ b/go/vt/vtgate/planbuilder/plan_test.go @@ -173,6 +173,11 @@ func loadSchema(t *testing.T, filename string) *vindexes.VSchema { if err != nil { t.Fatal(err) } + for _, ks := range vschema.Keyspaces { + if ks.Error != nil { + t.Fatal(ks.Error) + } + } return vschema } @@ -192,16 +197,16 @@ func (vw *vschemaWrapper) FindTable(tab sqlparser.TableName) (*vindexes.Table, s return table, destKeyspace, destTabletType, destTarget, nil } -func (vw *vschemaWrapper) FindTableOrVindex(tab sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +func (vw *vschemaWrapper) FindTablesOrVindex(tab sqlparser.TableName) ([]*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { destKeyspace, destTabletType, destTarget, err := topoproto.ParseDestination(tab.Qualifier.String(), topodatapb.TabletType_MASTER) if err != nil { return nil, nil, destKeyspace, destTabletType, destTarget, err } - table, vindex, err := vw.v.FindTableOrVindex(destKeyspace, tab.Name.String()) + tables, vindex, err := vw.v.FindTablesOrVindex(destKeyspace, tab.Name.String()) if err != nil { return nil, nil, destKeyspace, destTabletType, destTarget, err } - return table, vindex, destKeyspace, destTabletType, destTarget, nil + return tables, vindex, destKeyspace, destTabletType, destTarget, nil } func (vw *vschemaWrapper) DefaultKeyspace() (*vindexes.Keyspace, error) { diff --git a/go/vt/vtgate/planbuilder/postprocess.go b/go/vt/vtgate/planbuilder/postprocess.go index c30447e7370..78d20ce2ae9 100644 --- a/go/vt/vtgate/planbuilder/postprocess.go +++ b/go/vt/vtgate/planbuilder/postprocess.go @@ -135,7 +135,7 @@ func (pb *primitiveBuilder) pushLimit(limit *sqlparser.Limit) error { return nil } rb, ok := pb.bldr.(*route) - if ok && rb.IsSingle() { + if ok && rb.removeMultishardOptions() { rb.SetLimit(limit) return nil } diff --git a/go/vt/vtgate/planbuilder/route.go b/go/vt/vtgate/planbuilder/route.go index 74084b8d028..df4d47c5a1d 100644 --- a/go/vt/vtgate/planbuilder/route.go +++ b/go/vt/vtgate/planbuilder/route.go @@ -24,18 +24,19 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/engine" - "vitess.io/vitess/go/vt/vtgate/vindexes" ) var _ builder = (*route)(nil) -var errIntermixingUnsupported = errors.New("unsupported: intermixing of information_schema and regular tables") - // route is used to build a Route primitive. // It's used to build one of the Select routes like // SelectScatter, etc. Portions of the original Select AST // are moved into this node, which will be used to build // the final SQL for this route. +// A route can have multiple routeOptions. They are kept +// up-to-date as the route improves. Those that don't +// qualify are continuously removed from the options. +// A single best route is chosen before the Wireup phase. type route struct { order int @@ -51,26 +52,19 @@ type route struct { // resultColumns represent the columns returned by this route. resultColumns []*resultColumn - // condition stores the AST condition that will be used - // to resolve the ERoute Values field. - condition sqlparser.Expr - // weight_string keeps track of the weight_string expressions // that were added additionally for each column. These expressions // are added to be used for collation of text columns. weightStrings map[*resultColumn]int - // ERoute is the primitive being built. - ERoute *engine.Route + routeOptions []*routeOption } -func newRoute(stmt sqlparser.SelectStatement, eroute *engine.Route, condition sqlparser.Expr) (*route, *symtab) { +func newRoute(stmt sqlparser.SelectStatement) (*route, *symtab) { rb := &route{ Select: stmt, order: 1, - condition: condition, weightStrings: make(map[*resultColumn]int), - ERoute: eroute, } return rb, newSymtabWithRoute(rb) } @@ -96,7 +90,7 @@ func (rb *route) Reorder(order int) { // Primitive satisfies the builder interface. func (rb *route) Primitive() engine.Primitive { - return rb.ERoute + return rb.routeOptions[0].eroute } // First satisfies the builder interface. @@ -119,122 +113,14 @@ func (rb *route) PushFilter(pb *primitiveBuilder, filter sqlparser.Expr, whereTy case sqlparser.HavingStr: sel.AddHaving(filter) } - rb.UpdatePlan(pb, filter) + rb.UpdatePlans(pb, filter) return nil } -// UpdatePlan evaluates the primitive against the specified -// filter. If it's an improvement, the primitive is updated. -// We assume that the filter has already been pushed into -// the route. This function should only be used when merging -// routes, where the ON clause gets implicitly pushed into -// the merged route. -func (rb *route) UpdatePlan(pb *primitiveBuilder, filter sqlparser.Expr) { - opcode, vindex, values := rb.computePlan(pb, filter) - if opcode == engine.SelectScatter { - return - } - switch rb.ERoute.Opcode { - case engine.SelectEqualUnique: - if opcode == engine.SelectEqualUnique && vindex.Cost() < rb.ERoute.Vindex.Cost() { - rb.updateRoute(opcode, vindex, values) - } - case engine.SelectEqual: - switch opcode { - case engine.SelectEqualUnique: - rb.updateRoute(opcode, vindex, values) - case engine.SelectEqual: - if vindex.Cost() < rb.ERoute.Vindex.Cost() { - rb.updateRoute(opcode, vindex, values) - } - } - case engine.SelectIN: - switch opcode { - case engine.SelectEqualUnique, engine.SelectEqual: - rb.updateRoute(opcode, vindex, values) - case engine.SelectIN: - if vindex.Cost() < rb.ERoute.Vindex.Cost() { - rb.updateRoute(opcode, vindex, values) - } - } - case engine.SelectScatter: - switch opcode { - case engine.SelectEqualUnique, engine.SelectEqual, engine.SelectIN: - rb.updateRoute(opcode, vindex, values) - } - } -} - -func (rb *route) updateRoute(opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { - rb.ERoute.Opcode = opcode - rb.ERoute.Vindex = vindex - rb.condition = condition -} - -// computePlan computes the plan for the specified filter. -func (rb *route) computePlan(pb *primitiveBuilder, filter sqlparser.Expr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { - switch node := filter.(type) { - case *sqlparser.ComparisonExpr: - switch node.Operator { - case sqlparser.EqualStr: - return rb.computeEqualPlan(pb, node) - case sqlparser.InStr: - return rb.computeINPlan(pb, node) - } - case *sqlparser.ParenExpr: - return rb.computePlan(pb, node.Expr) - } - return engine.SelectScatter, nil, nil -} - -// computeEqualPlan computes the plan for an equality constraint. -func (rb *route) computeEqualPlan(pb *primitiveBuilder, comparison *sqlparser.ComparisonExpr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { - left := comparison.Left - right := comparison.Right - vindex = pb.st.Vindex(left, rb) - if vindex == nil { - left, right = right, left - vindex = pb.st.Vindex(left, rb) - if vindex == nil { - return engine.SelectScatter, nil, nil - } - } - if !rb.exprIsValue(right) { - return engine.SelectScatter, nil, nil - } - if vindex.IsUnique() { - return engine.SelectEqualUnique, vindex, right - } - return engine.SelectEqual, vindex, right -} - -// computeINPlan computes the plan for an IN constraint. -func (rb *route) computeINPlan(pb *primitiveBuilder, comparison *sqlparser.ComparisonExpr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { - vindex = pb.st.Vindex(comparison.Left, rb) - if vindex == nil { - return engine.SelectScatter, nil, nil - } - switch node := comparison.Right.(type) { - case sqlparser.ValTuple: - for _, n := range node { - if !rb.exprIsValue(n) { - return engine.SelectScatter, nil, nil - } - } - return engine.SelectIN, vindex, comparison - case sqlparser.ListArg: - return engine.SelectIN, vindex, comparison - } - return engine.SelectScatter, nil, nil -} - -// exprIsValue returns true if the expression can be treated as a value -// for the route. External references are treated as value. -func (rb *route) exprIsValue(expr sqlparser.Expr) bool { - if node, ok := expr.(*sqlparser.ColName); ok { - return node.Metadata.(*column).Origin() != rb +func (rb *route) UpdatePlans(pb *primitiveBuilder, filter sqlparser.Expr) { + for _, ro := range rb.routeOptions { + ro.UpdatePlan(pb, filter) } - return sqlparser.IsValue(expr) } // PushSelect satisfies the builder interface. @@ -277,7 +163,10 @@ func (rb *route) SetGroupBy(groupBy sqlparser.GroupBy) error { // PushOrderBy sets the order by for the route. func (rb *route) PushOrderBy(order *sqlparser.Order) error { - if rb.IsSingle() { + // By this time, if any single shard options were already present, + // multi-sharded options would have already been removed. So, this + // call is only for checking if the route has single shard options. + if rb.removeMultishardOptions() { rb.Select.AddOrder(order) return nil } @@ -306,10 +195,13 @@ func (rb *route) PushOrderBy(order *sqlparser.Order) error { if colnum == -1 { return fmt.Errorf("unsupported: in scatter query: order by must reference a column in the select list: %s", sqlparser.String(order)) } - rb.ERoute.OrderBy = append(rb.ERoute.OrderBy, engine.OrderbyParams{ + ob := engine.OrderbyParams{ Col: colnum, Desc: order.Direction == sqlparser.DescScr, - }) + } + for _, ro := range rb.routeOptions { + ro.eroute.OrderBy = append(ro.eroute.OrderBy, ob) + } rb.Select.AddOrder(order) return nil @@ -347,16 +239,19 @@ func (rb *route) PushMisc(sel *sqlparser.Select) { // Wireup satisfies the builder interface. func (rb *route) Wireup(bldr builder, jt *jointab) error { + rb.finalizeOptions() + // Precaution: update ERoute.Values only if it's not set already. - if rb.ERoute.Values == nil { + ro := rb.routeOptions[0] + if ro.eroute.Values == nil { // Resolve values stored in the builder. - switch vals := rb.condition.(type) { + switch vals := ro.condition.(type) { case *sqlparser.ComparisonExpr: pv, err := rb.procureValues(bldr, jt, vals.Right) if err != nil { return err } - rb.ERoute.Values = []sqltypes.PlanValue{pv} + ro.eroute.Values = []sqltypes.PlanValue{pv} vals.Right = sqlparser.ListArg("::" + engine.ListVarName) case nil: // no-op. @@ -365,7 +260,7 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { if err != nil { return err } - rb.ERoute.Values = []sqltypes.PlanValue{pv} + ro.eroute.Values = []sqltypes.PlanValue{pv} } } @@ -373,18 +268,18 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { // we have to request the corresponding weight_string from mysql // and use that value instead. This is because we cannot mimic // mysql's collation behavior yet. - for i, orderby := range rb.ERoute.OrderBy { + for i, orderby := range ro.eroute.OrderBy { rc := rb.resultColumns[orderby.Col] if sqltypes.IsText(rc.column.typ) { // If a weight string was previously requested (by OrderedAggregator), // reuse it. if colnum, ok := rb.weightStrings[rc]; ok { - rb.ERoute.OrderBy[i].Col = colnum + ro.eroute.OrderBy[i].Col = colnum continue } // len(rb.resultColumns) does not change. No harm using the value multiple times. - rb.ERoute.TruncateColumnCount = len(rb.resultColumns) + ro.eroute.TruncateColumnCount = len(rb.resultColumns) // This code is partially duplicated from SupplyWeightString and PushSelect. // We should not update resultColumns because it's not returned in the result. @@ -399,7 +294,7 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { } sel := rb.Select.(*sqlparser.Select) sel.SelectExprs = append(sel.SelectExprs, expr) - rb.ERoute.OrderBy[i].Col = len(sel.SelectExprs) - 1 + ro.eroute.OrderBy[i].Col = len(sel.SelectExprs) - 1 // We don't really have to update weightStrings, but we're doing it // for good measure. rb.weightStrings[rc] = len(sel.SelectExprs) - 1 @@ -419,7 +314,7 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { } case *sqlparser.ComparisonExpr: if node.Operator == sqlparser.EqualStr { - if rb.exprIsValue(node.Left) && !rb.exprIsValue(node.Right) { + if ro.exprIsValue(node.Left) && !ro.exprIsValue(node.Right) { node.Left, node.Right = node.Right, node.Left } } @@ -427,6 +322,11 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { return true, nil }, rb.Select) + // Substitute table names + for _, sub := range ro.substitutions { + *sub.oldExpr = *sub.newExpr + } + // Generate query while simultaneously resolving values. varFormatter := func(buf *sqlparser.TrackedBuffer, node sqlparser.SQLNode) { switch node := node.(type) { @@ -448,8 +348,8 @@ func (rb *route) Wireup(bldr builder, jt *jointab) error { } buf := sqlparser.NewTrackedBuffer(varFormatter) varFormatter(buf, rb.Select) - rb.ERoute.Query = buf.ParsedQuery().Query - rb.ERoute.FieldQuery = rb.generateFieldQuery(rb.Select, jt) + ro.eroute.Query = buf.ParsedQuery().Query + ro.eroute.FieldQuery = rb.generateFieldQuery(rb.Select, jt) return nil } @@ -460,6 +360,16 @@ func systemTable(qualifier string) bool { strings.EqualFold(qualifier, "mysql") } +func (rb *route) finalizeOptions() { + bestOption := rb.routeOptions[0] + for i := 1; i < len(rb.routeOptions); i++ { + if cur := rb.routeOptions[i]; cur.isBetterThan(bestOption) { + bestOption = cur + } + } + rb.routeOptions = []*routeOption{bestOption} +} + // procureValues procures and converts the input into // the expected types for rb.Values. func (rb *route) procureValues(bldr builder, jt *jointab, val sqlparser.Expr) (sqltypes.PlanValue, error) { @@ -576,107 +486,110 @@ func (rb *route) BuildColName(index int) (*sqlparser.ColName, error) { }, nil } -// IsSingle returns true if the route targets only one database. -func (rb *route) IsSingle() bool { - switch rb.ERoute.Opcode { - // Even thought SelectNext is a single-shard query, we don't - // include it here because it can't be combined with any other construct. - case engine.SelectUnsharded, engine.SelectDBA, engine.SelectEqualUnique: +// MergeSubquery returns true if the subquery route could successfully be merged +// with the outer route. +func (rb *route) MergeSubquery(pb *primitiveBuilder, inner *route) bool { + var mergedRouteOptions []*routeOption +outer: + for _, lro := range rb.routeOptions { + for _, rro := range inner.routeOptions { + if lro.SubqueryCanMerge(pb, rro) { + lro.MergeSubquery(rro) + mergedRouteOptions = append(mergedRouteOptions, lro) + continue outer + } + } + } + if len(mergedRouteOptions) != 0 { + rb.routeOptions = mergedRouteOptions + inner.Redirect = rb return true } return false } -// SubqueryCanMerge returns nil if the supplied route that represents -// a subquery can be merged with the outer route. If not, it -// returns an appropriate error. -func (rb *route) SubqueryCanMerge(pb *primitiveBuilder, inner *route) bool { - if rb.ERoute.Keyspace.Name != inner.ERoute.Keyspace.Name { - return false - } - switch inner.ERoute.Opcode { - case engine.SelectUnsharded: - if rb.ERoute.Opcode == engine.SelectUnsharded { - return true - } - return false - case engine.SelectDBA: - if rb.ERoute.Opcode == engine.SelectDBA { - return true - } - return false - case engine.SelectNext: - return false - case engine.SelectEqualUnique: - // This checks for the case where the subquery is dependent - // on the vindex column of the outer query: - // select ... from a where a.id = 5 ... (select ... from b where b.id = a.id). - // If b.id and a.id have the same vindex, it becomes a single-shard - // query: the subquery can merge with the outer query. - switch vals := inner.condition.(type) { - case *sqlparser.ColName: - if pb.st.Vindex(vals, rb) == inner.ERoute.Vindex { - return true +// MergeUnion returns true if the rhs route could successfully be merged +// with the rb route. +func (rb *route) MergeUnion(right *route) bool { + var mergedRouteOptions []*routeOption +outer: + for _, lro := range rb.routeOptions { + for _, rro := range right.routeOptions { + if lro.UnionCanMerge(rro) { + lro.MergeUnion(rro) + mergedRouteOptions = append(mergedRouteOptions, lro) + continue outer } } - default: - return false } - return rb.isSameShardedRoute(inner) == nil + if len(mergedRouteOptions) != 0 { + rb.routeOptions = mergedRouteOptions + right.Redirect = rb + return true + } + return false } -// UnionCanMerge returns nil if the supplied route that represents -// the RHS of a union can be merged with the current route. If not, it -// returns an appropriate error. -func (rb *route) UnionCanMerge(right *route) error { - if rb.ERoute.Opcode == engine.SelectNext || right.ERoute.Opcode == engine.SelectNext { - return errors.New("unsupported: UNION on sequence tables") - } - if rb.ERoute.Keyspace.Name != right.ERoute.Keyspace.Name { - return errors.New("unsupported: UNION on different keyspaces") - } - switch rb.ERoute.Opcode { - case engine.SelectUnsharded: - if right.ERoute.Opcode == engine.SelectUnsharded { - return nil +// removeMultishardOptions removes all multi-shard options from the +// route. It returns false if no such options exist. +func (rb *route) removeMultishardOptions() bool { + return rb.removeOptions(func(ro *routeOption) bool { + switch ro.eroute.Opcode { + case engine.SelectUnsharded, engine.SelectDBA, engine.SelectNext, engine.SelectEqualUnique, engine.SelectReference: + return true } - return errIntermixingUnsupported - case engine.SelectDBA: - if right.ERoute.Opcode == engine.SelectDBA { - return nil + return false + }) +} + +// removeShardedOptions removes all sharded options from the +// route. It returns false if no such options exist. +// This is used for constructs that are only supported for unsharded +// keyspaces like last_insert_id. +func (rb *route) removeShardedOptions() bool { + return rb.removeOptions(func(ro *routeOption) bool { + return ro.eroute.Opcode == engine.SelectUnsharded + }) +} + +// removeOptionsWithUnmatchedKeyspace removes all options that don't match +// the specified keyspace. It returns false if no such options exist. +func (rb *route) removeOptionsWithUnmatchedKeyspace(keyspace string) bool { + return rb.removeOptions(func(ro *routeOption) bool { + return ro.eroute.Keyspace.Name == keyspace + }) +} + +// removeOptions removes all options that don't match on +// the criteria function. It returns false if no such options exist. +func (rb *route) removeOptions(match func(*routeOption) bool) bool { + var newOptions []*routeOption + for _, ro := range rb.routeOptions { + if match(ro) { + newOptions = append(newOptions, ro) } - return errIntermixingUnsupported } - return rb.isSameShardedRoute(right) + if len(newOptions) == 0 { + return false + } + rb.routeOptions = newOptions + return true } -// isSameShardedRoute returns nil if the supplied route has -// the same single shard target as the current route. If not, it -// returns an appropriate error. -func (rb *route) isSameShardedRoute(right *route) error { - if rb.ERoute.Opcode != engine.SelectEqualUnique || right.ERoute.Opcode != engine.SelectEqualUnique { - return errors.New("unsupported: UNION or subquery containing multi-shard queries") - } - if rb.ERoute.Vindex != right.ERoute.Vindex { - return errors.New("unsupported: UNION or subquery on different shards: vindexes are different") +// queryTimeout returns DirectiveQueryTimeout value if set, otherwise returns 0. +func queryTimeout(d sqlparser.CommentDirectives) int { + if d == nil { + return 0 } - if !valEqual(rb.condition, right.condition) { - return errors.New("unsupported: UNION or subquery on different shards: vindex values are different") + + val, ok := d[sqlparser.DirectiveQueryTimeout] + if !ok { + return 0 } - return nil -} -// SetOpcode changes the opcode to the specified value. -// If the change is not allowed, it returns an error. -func (rb *route) SetOpcode(code engine.RouteOpcode) error { - switch code { - case engine.SelectNext: - if rb.ERoute.Opcode != engine.SelectUnsharded { - return errors.New("NEXT used on a sharded table") - } - default: - panic(fmt.Sprintf("BUG: unrecognized transition: %v", code)) + intVal, ok := val.(int) + if ok { + return intVal } - rb.ERoute.Opcode = code - return nil + return 0 } diff --git a/go/vt/vtgate/planbuilder/route_option.go b/go/vt/vtgate/planbuilder/route_option.go new file mode 100644 index 00000000000..5a4f45f60a7 --- /dev/null +++ b/go/vt/vtgate/planbuilder/route_option.go @@ -0,0 +1,351 @@ +/* +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/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/vindexes" +) + +// routeOption contains all the information for one route option. +// A route can have multiple options. +type routeOption struct { + rb *route + + // vschemaTable is set only if a single table is referenced + // in the from clause. It's used only for DMLs. + vschemaTable *vindexes.Table + + // substitutions contain the list of table expressions that + // have to be substituted in the route's query. + substitutions []*tableSubstitution + + // vindexMap is a map of all vindexMap that can be used + // for the routeOption. + vindexMap map[*column]vindexes.Vindex + + // condition stores the AST condition that will be used + // to resolve the ERoute Values field. + condition sqlparser.Expr + + // eroute is the primitive being built. + eroute *engine.Route +} + +type tableSubstitution struct { + newExpr, oldExpr *sqlparser.AliasedTableExpr +} + +func newSimpleRouteOption(rb *route, eroute *engine.Route) *routeOption { + return &routeOption{ + rb: rb, + eroute: eroute, + } +} + +func newRouteOption(rb *route, vst *vindexes.Table, sub *tableSubstitution, vindexMap map[*column]vindexes.Vindex, eroute *engine.Route) *routeOption { + var subs []*tableSubstitution + if sub != nil && sub.newExpr != nil { + subs = []*tableSubstitution{sub} + } + return &routeOption{ + rb: rb, + vschemaTable: vst, + substitutions: subs, + vindexMap: vindexMap, + eroute: eroute, + } +} + +func (ro *routeOption) JoinCanMerge(pb *primitiveBuilder, rro *routeOption, ajoin *sqlparser.JoinTableExpr) bool { + return ro.canMerge(rro, func() bool { + if ajoin == nil { + return false + } + for _, filter := range splitAndExpression(nil, ajoin.Condition.On) { + if ro.canMergeOnFilter(pb, rro, filter) { + return true + } + } + return false + }) +} + +func (ro *routeOption) MergeJoin(rro *routeOption, isLeftJoin bool) { + ro.vschemaTable = nil + ro.substitutions = append(ro.substitutions, rro.substitutions...) + if isLeftJoin { + return + } + // Add RHS vindexes only if it's not a left join. + for c, v := range rro.vindexMap { + if ro.vindexMap == nil { + ro.vindexMap = make(map[*column]vindexes.Vindex) + } + ro.vindexMap[c] = v + } +} + +func (ro *routeOption) SubqueryCanMerge(pb *primitiveBuilder, inner *routeOption) bool { + return ro.canMerge(inner, func() bool { + switch vals := inner.condition.(type) { + case *sqlparser.ColName: + if ro.FindVindex(pb, vals) == inner.eroute.Vindex { + return true + } + } + return false + }) +} + +func (ro *routeOption) MergeSubquery(subqueryOption *routeOption) { + ro.substitutions = append(ro.substitutions, subqueryOption.substitutions...) +} + +func (ro *routeOption) UnionCanMerge(rro *routeOption) bool { + return ro.canMerge(rro, func() bool { return false }) +} + +func (ro *routeOption) MergeUnion(rro *routeOption) { + ro.vschemaTable = nil + ro.substitutions = append(ro.substitutions, rro.substitutions...) +} + +func (ro *routeOption) SubqueryToTable(rb *route, vindexMap map[*column]vindexes.Vindex) { + ro.rb = rb + ro.vschemaTable = nil + ro.vindexMap = vindexMap +} + +func (ro *routeOption) canMerge(rro *routeOption, customCheck func() bool) bool { + if ro.eroute.Keyspace.Name != rro.eroute.Keyspace.Name { + return false + } + if rro.eroute.Opcode == engine.SelectReference { + // Any opcode can join with a reference table. + return true + } + switch ro.eroute.Opcode { + case engine.SelectUnsharded, engine.SelectDBA: + return ro.eroute.Opcode == rro.eroute.Opcode + case engine.SelectEqualUnique: + // Check if they target the same shard. + if rro.eroute.Opcode == engine.SelectEqualUnique && ro.eroute.Vindex == rro.eroute.Vindex && valEqual(ro.condition, rro.condition) { + return true + } + case engine.SelectReference: + // TODO(sougou): this can be changed to true, but we'll have + // to merge against rro insteal of ro. + return false + case engine.SelectNext: + return false + } + if customCheck != nil { + return customCheck() + } + return false +} + +// canMergeOnFilter returns true if the join constraint makes the routes +// mergeable by unique vindex. The constraint has to be an equality +// like a.id = b.id where both columns have the same unique vindex. +func (ro *routeOption) canMergeOnFilter(pb *primitiveBuilder, rro *routeOption, filter sqlparser.Expr) bool { + filter = skipParenthesis(filter) + comparison, ok := filter.(*sqlparser.ComparisonExpr) + if !ok { + return false + } + if comparison.Operator != sqlparser.EqualStr { + return false + } + left := comparison.Left + right := comparison.Right + lVindex := ro.FindVindex(pb, left) + if lVindex == nil { + left, right = right, left + lVindex = ro.FindVindex(pb, left) + } + if lVindex == nil || !lVindex.IsUnique() { + return false + } + rVindex := rro.FindVindex(pb, right) + if rVindex == nil { + return false + } + return rVindex == lVindex +} + +// UpdatePlan evaluates the primitive against the specified +// filter. If it's an improvement, the primitive is updated. +// We assume that the filter has already been pushed into +// the route. +func (ro *routeOption) UpdatePlan(pb *primitiveBuilder, filter sqlparser.Expr) { + switch ro.eroute.Opcode { + case engine.SelectUnsharded, engine.SelectNext, engine.SelectDBA, engine.SelectReference: + return + } + opcode, vindex, values := ro.computePlan(pb, filter) + if opcode == engine.SelectScatter { + return + } + switch ro.eroute.Opcode { + case engine.SelectEqualUnique: + if opcode == engine.SelectEqualUnique && vindex.Cost() < ro.eroute.Vindex.Cost() { + ro.updateRoute(opcode, vindex, values) + } + case engine.SelectEqual: + switch opcode { + case engine.SelectEqualUnique: + ro.updateRoute(opcode, vindex, values) + case engine.SelectEqual: + if vindex.Cost() < ro.eroute.Vindex.Cost() { + ro.updateRoute(opcode, vindex, values) + } + } + case engine.SelectIN: + switch opcode { + case engine.SelectEqualUnique, engine.SelectEqual: + ro.updateRoute(opcode, vindex, values) + case engine.SelectIN: + if vindex.Cost() < ro.eroute.Vindex.Cost() { + ro.updateRoute(opcode, vindex, values) + } + } + case engine.SelectScatter: + switch opcode { + case engine.SelectEqualUnique, engine.SelectEqual, engine.SelectIN: + ro.updateRoute(opcode, vindex, values) + } + } +} + +func (ro *routeOption) updateRoute(opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { + ro.eroute.Opcode = opcode + ro.eroute.Vindex = vindex + ro.condition = condition +} + +// computePlan computes the plan for the specified filter. +func (ro *routeOption) computePlan(pb *primitiveBuilder, filter sqlparser.Expr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { + switch node := filter.(type) { + case *sqlparser.ComparisonExpr: + switch node.Operator { + case sqlparser.EqualStr: + return ro.computeEqualPlan(pb, node) + case sqlparser.InStr: + return ro.computeINPlan(pb, node) + } + case *sqlparser.ParenExpr: + return ro.computePlan(pb, node.Expr) + } + return engine.SelectScatter, nil, nil +} + +// computeEqualPlan computes the plan for an equality constraint. +func (ro *routeOption) computeEqualPlan(pb *primitiveBuilder, comparison *sqlparser.ComparisonExpr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { + left := comparison.Left + right := comparison.Right + vindex = ro.FindVindex(pb, left) + if vindex == nil { + left, right = right, left + vindex = ro.FindVindex(pb, left) + if vindex == nil { + return engine.SelectScatter, nil, nil + } + } + if !ro.exprIsValue(right) { + return engine.SelectScatter, nil, nil + } + if vindex.IsUnique() { + return engine.SelectEqualUnique, vindex, right + } + return engine.SelectEqual, vindex, right +} + +// computeINPlan computes the plan for an IN constraint. +func (ro *routeOption) computeINPlan(pb *primitiveBuilder, comparison *sqlparser.ComparisonExpr) (opcode engine.RouteOpcode, vindex vindexes.Vindex, condition sqlparser.Expr) { + vindex = ro.FindVindex(pb, comparison.Left) + if vindex == nil { + return engine.SelectScatter, nil, nil + } + switch node := comparison.Right.(type) { + case sqlparser.ValTuple: + for _, n := range node { + if !ro.exprIsValue(n) { + return engine.SelectScatter, nil, nil + } + } + return engine.SelectIN, vindex, comparison + case sqlparser.ListArg: + return engine.SelectIN, vindex, comparison + } + return engine.SelectScatter, nil, nil +} + +var planCost = map[engine.RouteOpcode]int{ + engine.SelectUnsharded: 0, + engine.SelectNext: 0, + engine.SelectDBA: 0, + engine.SelectReference: 0, + engine.SelectEqualUnique: 1, + engine.SelectIN: 2, + engine.SelectEqual: 3, + engine.SelectScatter: 4, +} + +func (ro *routeOption) isBetterThan(other *routeOption) bool { + ropc := planCost[ro.eroute.Opcode] + otherpc := planCost[other.eroute.Opcode] + if ropc < otherpc { + return true + } + if ropc == otherpc { + switch other.eroute.Opcode { + case engine.SelectEqualUnique, engine.SelectIN, engine.SelectEqual: + return ro.eroute.Vindex.Cost() < other.eroute.Vindex.Cost() + } + } + return false +} + +func (ro *routeOption) FindVindex(pb *primitiveBuilder, expr sqlparser.Expr) vindexes.Vindex { + col, ok := expr.(*sqlparser.ColName) + if !ok { + return nil + } + if col.Metadata == nil { + // Find will set the Metadata. + if _, _, err := pb.st.Find(col); err != nil { + return nil + } + } + c := col.Metadata.(*column) + if c.Origin() != ro.rb { + return nil + } + return ro.vindexMap[c] +} + +// exprIsValue returns true if the expression can be treated as a value +// for the routeOption. External references are treated as value. +func (ro *routeOption) exprIsValue(expr sqlparser.Expr) bool { + if node, ok := expr.(*sqlparser.ColName); ok { + return node.Metadata.(*column).Origin() != ro.rb + } + return sqlparser.IsValue(expr) +} diff --git a/go/vt/vtgate/planbuilder/route_option_test.go b/go/vt/vtgate/planbuilder/route_option_test.go new file mode 100644 index 00000000000..425aa3d34f5 --- /dev/null +++ b/go/vt/vtgate/planbuilder/route_option_test.go @@ -0,0 +1,186 @@ +/* +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 ( + "testing" + + "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/vindexes" +) + +func TestIsBetterThan(t *testing.T) { + // All combinations aren't worth it. + testcases := []struct { + left, right engine.RouteOpcode + leftcost, rightcost int + out bool + }{{ + left: engine.SelectUnsharded, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectUnsharded, + right: engine.SelectNext, + out: false, + }, { + left: engine.SelectUnsharded, + right: engine.SelectDBA, + out: false, + }, { + left: engine.SelectUnsharded, + right: engine.SelectReference, + out: false, + }, { + left: engine.SelectUnsharded, + right: engine.SelectEqualUnique, + out: true, + }, { + left: engine.SelectUnsharded, + right: engine.SelectIN, + out: true, + }, { + left: engine.SelectUnsharded, + right: engine.SelectEqual, + out: true, + }, { + left: engine.SelectUnsharded, + right: engine.SelectScatter, + out: true, + }, { + left: engine.SelectNext, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectDBA, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectReference, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectEqualUnique, + leftcost: 2, + rightcost: 1, + out: false, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectEqualUnique, + leftcost: 1, + rightcost: 1, + out: false, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectEqualUnique, + leftcost: 1, + rightcost: 2, + out: true, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectIN, + out: true, + }, { + left: engine.SelectEqualUnique, + right: engine.SelectIN, + out: true, + }, { + left: engine.SelectIN, + right: engine.SelectIN, + leftcost: 2, + rightcost: 1, + out: false, + }, { + left: engine.SelectIN, + right: engine.SelectIN, + leftcost: 1, + rightcost: 1, + out: false, + }, { + left: engine.SelectIN, + right: engine.SelectIN, + leftcost: 1, + rightcost: 2, + out: true, + }, { + left: engine.SelectIN, + right: engine.SelectEqual, + out: true, + }, { + left: engine.SelectEqual, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectEqual, + right: engine.SelectEqual, + leftcost: 2, + rightcost: 1, + out: false, + }, { + left: engine.SelectEqual, + right: engine.SelectEqual, + leftcost: 1, + rightcost: 1, + out: false, + }, { + left: engine.SelectEqual, + right: engine.SelectEqual, + leftcost: 1, + rightcost: 2, + out: true, + }, { + left: engine.SelectEqual, + right: engine.SelectScatter, + out: true, + }, { + left: engine.SelectScatter, + right: engine.SelectUnsharded, + out: false, + }, { + left: engine.SelectScatter, + right: engine.SelectScatter, + out: false, + }} + buildOption := func(opt engine.RouteOpcode, cost int) *routeOption { + var v vindexes.Vindex + switch cost { + case 1: + v, _ = newHashIndex("", nil) + case 2: + v, _ = newLookupIndex("", nil) + } + return &routeOption{ + eroute: &engine.Route{ + Opcode: opt, + Vindex: v, + }, + } + } + for _, tcase := range testcases { + left := buildOption(tcase.left, tcase.leftcost) + right := buildOption(tcase.right, tcase.rightcost) + got := left.isBetterThan(right) + if got != tcase.out { + t.Errorf("isBetterThan(%v, %v, %v, %v): %v, want %v", tcase.left, tcase.leftcost, tcase.right, tcase.rightcost, got, tcase.out) + } + } +} diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go index 8814f328b1b..dce56b4ec4b 100644 --- a/go/vt/vtgate/planbuilder/select.go +++ b/go/vt/vtgate/planbuilder/select.go @@ -77,14 +77,17 @@ func (pb *primitiveBuilder) processSelect(sel *sqlparser.Select, outer *symtab) } if rb, ok := pb.bldr.(*route); ok { - directives := sqlparser.ExtractCommentDirectives(sel.Comments) - rb.ERoute.QueryTimeout = queryTimeout(directives) - if rb.ERoute.TargetDestination != nil { - return errors.New("unsupported: SELECT with a target destination") - } + // TODO(sougou): this can probably be improved. + for _, ro := range rb.routeOptions { + directives := sqlparser.ExtractCommentDirectives(sel.Comments) + ro.eroute.QueryTimeout = queryTimeout(directives) + if ro.eroute.TargetDestination != nil { + return errors.New("unsupported: SELECT with a target destination") + } - if directives.IsSet(sqlparser.DirectiveScatterErrorsAsWarnings) { - rb.ERoute.ScatterErrorsAsWarnings = true + if directives.IsSet(sqlparser.DirectiveScatterErrorsAsWarnings) { + ro.eroute.ScatterErrorsAsWarnings = true + } } } @@ -214,10 +217,8 @@ func (pb *primitiveBuilder) pushSelectRoutes(selectExprs sqlparser.SelectExprs) } // Validate keyspace reference if any. if !node.TableName.IsEmpty() { - if qual := node.TableName.Qualifier; !qual.IsEmpty() { - if qual.String() != rb.ERoute.Keyspace.Name { - return nil, fmt.Errorf("cannot resolve %s to keyspace %s", sqlparser.String(node), rb.ERoute.Keyspace.Name) - } + if _, err := pb.st.FindTable(node.TableName); err != nil { + return nil, err } } resultColumns = append(resultColumns, rb.PushAnonymous(node)) @@ -227,8 +228,11 @@ func (pb *primitiveBuilder) pushSelectRoutes(selectExprs sqlparser.SelectExprs) // This code is unreachable because the parser doesn't allow joins for next val statements. return nil, errors.New("unsupported: SELECT NEXT query in cross-shard query") } - if err := rb.SetOpcode(engine.SelectNext); err != nil { - return nil, err + for _, ro := range rb.routeOptions { + if ro.eroute.Opcode != engine.SelectNext { + return nil, errors.New("NEXT used on a non-sequence table") + } + ro.eroute.Opcode = engine.SelectNext } resultColumns = append(resultColumns, rb.PushAnonymous(node)) default: @@ -285,7 +289,7 @@ func (pb *primitiveBuilder) expandStar(inrcs []*resultColumn, expr *sqlparser.St As: col, } } - rc, _, err := pb.bldr.PushSelect(expr, t.origin) + rc, _, err := pb.bldr.PushSelect(expr, t.Origin()) if err != nil { // Unreachable because PushSelect won't fail on ColName. return inrcs, false, err @@ -312,7 +316,7 @@ func (pb *primitiveBuilder) expandStar(inrcs []*resultColumn, expr *sqlparser.St Qualifier: expr.TableName, }, } - rc, _, err := pb.bldr.PushSelect(expr, t.origin) + rc, _, err := pb.bldr.PushSelect(expr, t.Origin()) if err != nil { // Unreachable because PushSelect won't fail on ColName. return inrcs, false, err @@ -321,21 +325,3 @@ func (pb *primitiveBuilder) expandStar(inrcs []*resultColumn, expr *sqlparser.St } return inrcs, true, nil } - -// queryTimeout returns DirectiveQueryTimeout value if set, otherwise returns 0. -func queryTimeout(d sqlparser.CommentDirectives) int { - if d == nil { - return 0 - } - - val, ok := d[sqlparser.DirectiveQueryTimeout] - if !ok { - return 0 - } - - intVal, ok := val.(int) - if ok { - return intVal - } - return 0 -} diff --git a/go/vt/vtgate/planbuilder/symtab.go b/go/vt/vtgate/planbuilder/symtab.go index ac4caae56dd..59c5f28e2c1 100644 --- a/go/vt/vtgate/planbuilder/symtab.go +++ b/go/vt/vtgate/planbuilder/symtab.go @@ -59,12 +59,6 @@ type symtab struct { // singleRoute is set only if all the symbols in // the symbol table are part of the same route. - // The route is set at creation time to be the - // the same as the route it was built for. As - // symbols are added through Merge, the route - // is validated against the newer symbols. If any - // of them have a different route, the value is - // set to nil. singleRoute *route ResultColumns []*resultColumn @@ -90,54 +84,67 @@ func newSymtabWithRoute(rb *route) *symtab { } } -// AddVindexTable creates a table from a vindex table -// and adds it to symtab. -func (st *symtab) AddVindexTable(alias sqlparser.TableName, vindexTable *vindexes.Table, rb *route) error { +// AddVSchemaTable takes a list of vschema tables as input and +// creates a table with multiple route options. It returns a +// list of vindex maps, one for each input. +func (st *symtab) AddVSchemaTable(alias sqlparser.TableName, vschemaTables []*vindexes.Table, rb *route) (vindexMaps []map[*column]vindexes.Vindex, err error) { t := &table{ - alias: alias, - origin: rb, - vindexTable: vindexTable, - isAuthoritative: vindexTable.ColumnListAuthoritative, + alias: alias, + origin: rb, } - for _, col := range vindexTable.Columns { - t.addColumn(col.Name, &column{ - origin: rb, - st: st, - typ: col.Type, - }) - } + vindexMaps = make([]map[*column]vindexes.Vindex, len(vschemaTables)) + for i, vst := range vschemaTables { + // If any input is authoritative, we make the table authoritative. + // TODO(sougou): vschema builder should validate that authoritative columns match. + if vst.ColumnListAuthoritative { + t.isAuthoritative = true + } - for _, cv := range vindexTable.ColumnVindexes { - for i, cvcol := range cv.Columns { - var vindex vindexes.Vindex - if i == 0 { - // For now, only the first column is used for vindex Map functions. - vindex = cv.Vindex - } - lowered := cvcol.Lowered() - if col, ok := t.columns[lowered]; ok { - col.Vindex = vindex - continue - } - t.addColumn(cvcol, &column{ + for _, col := range vst.Columns { + t.addColumn(col.Name, &column{ origin: rb, st: st, - Vindex: vindex, + typ: col.Type, }) } - } - if ai := vindexTable.AutoIncrement; ai != nil { - lowered := ai.Column.Lowered() - if _, ok := t.columns[lowered]; !ok { - t.addColumn(ai.Column, &column{ - origin: rb, - st: st, - }) + var vindexMap map[*column]vindexes.Vindex + for _, cv := range vst.ColumnVindexes { + for i, cvcol := range cv.Columns { + col, ok := t.columns[cvcol.Lowered()] + if !ok { + col = &column{ + origin: rb, + st: st, + } + t.addColumn(cvcol, col) + } + if i == 0 { + // For now, only the first column is used for vindex Map functions. + if vindexMap == nil { + vindexMap = make(map[*column]vindexes.Vindex) + } + vindexMap[col] = cv.Vindex + } + } + } + vindexMaps[i] = vindexMap + + if ai := vst.AutoIncrement; ai != nil { + if _, ok := t.columns[ai.Column.Lowered()]; !ok { + t.addColumn(ai.Column, &column{ + origin: rb, + st: st, + }) + } } + + } + if err := st.AddTable(t); err != nil { + return nil, err } - return st.AddTable(t) + return vindexMaps, nil } // Merge merges the new symtab into the current one. @@ -218,17 +225,6 @@ func (st *symtab) FindTable(tname sqlparser.TableName) (*table, error) { return t, nil } -// ClearVindexes removes the Column Vindexes from the aliases signifying -// that they cannot be used to make routing improvements. This is -// called if a primitive is in the RHS of a LEFT JOIN. -func (st *symtab) ClearVindexes() { - for _, t := range st.tables { - for _, c := range t.columns { - c.Vindex = nil - } - } -} - // SetResultColumns sets the result columns. func (st *symtab) SetResultColumns(rcs []*resultColumn) { for _, rc := range rcs { @@ -374,7 +370,7 @@ func (st *symtab) searchTables(col *sqlparser.ColName) (*column, error) { return nil, fmt.Errorf("symbol %s not found in table or subquery", sqlparser.String(col)) } c = &column{ - origin: t.origin, + origin: t.Origin(), st: st, } t.addColumn(col.Name, c) @@ -398,26 +394,6 @@ func ResultFromNumber(rcs []*resultColumn, val *sqlparser.SQLVal) (int, error) { return int(num - 1), nil } -// Vindex returns the vindex if the expression is a plain column reference -// that is part of the specified route, and has an associated vindex. -func (st *symtab) Vindex(expr sqlparser.Expr, scope *route) vindexes.Vindex { - col, ok := expr.(*sqlparser.ColName) - if !ok { - return nil - } - if col.Metadata == nil { - // Find will set the Metadata. - if _, _, err := st.Find(col); err != nil { - return nil - } - } - c := col.Metadata.(*column) - if c.Origin() != scope { - return nil - } - return c.Vindex -} - // ResolveSymbols resolves all column references against symtab. // This makes sure that they all have their Metadata initialized. // If a symbol cannot be resolved or if the expression contains @@ -445,7 +421,6 @@ type table struct { columnNames []sqlparser.ColIdent isAuthoritative bool origin builder - vindexTable *vindexes.Table } func (t *table) addColumn(alias sqlparser.ColIdent, c *column) { @@ -461,10 +436,17 @@ func (t *table) addColumn(alias sqlparser.ColIdent, c *column) { t.columnNames = append(t.columnNames, alias) } +// Origin returns the route that originates the table. +func (t *table) Origin() builder { + // If it's a route, we have to resolve it. + if rb, ok := t.origin.(*route); ok { + return rb.Resolve() + } + return t.origin +} + // column represents a unique symbol in the query that other -// parts can refer to. If a column originates from a sharded -// table, and is tied to a vindex, then its Vindex field is -// set, which can be used to improve a route's plan. +// parts can refer to. // Every column contains the builder it originates from. // // Two columns are equal if their pointer values match. @@ -474,7 +456,6 @@ func (t *table) addColumn(alias sqlparser.ColIdent, c *column) { type column struct { origin builder st *symtab - Vindex vindexes.Vindex typ querypb.Type colnum int } diff --git a/go/vt/vtgate/planbuilder/symtab_test.go b/go/vt/vtgate/planbuilder/symtab_test.go index f10a8cc1e58..80a0bc68c4b 100644 --- a/go/vt/vtgate/planbuilder/symtab_test.go +++ b/go/vt/vtgate/planbuilder/symtab_test.go @@ -29,20 +29,22 @@ func TestSymtabAddVindexTable(t *testing.T) { rb := &route{} tcases := []struct { - in *vindexes.Table - out []string + in []*vindexes.Table + authoritative bool + vindexes [][]string }{{ - in: &vindexes.Table{ + in: []*vindexes.Table{{ Columns: []vindexes.Column{{ Name: sqlparser.NewColIdent("C1"), }, { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ ColumnVindexes: []*vindexes.ColumnVindex{{ Columns: []sqlparser.ColIdent{sqlparser.NewColIdent("C1")}, }}, @@ -52,10 +54,11 @@ func TestSymtabAddVindexTable(t *testing.T) { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{"c1"}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ ColumnVindexes: []*vindexes.ColumnVindex{{ Columns: []sqlparser.ColIdent{ sqlparser.NewColIdent("C1"), @@ -68,10 +71,11 @@ func TestSymtabAddVindexTable(t *testing.T) { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{"c1"}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ AutoIncrement: &vindexes.AutoIncrement{ Column: sqlparser.NewColIdent("C1"), }, @@ -81,10 +85,11 @@ func TestSymtabAddVindexTable(t *testing.T) { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ ColumnVindexes: []*vindexes.ColumnVindex{{ Columns: []sqlparser.ColIdent{sqlparser.NewColIdent("C1")}, }}, @@ -92,20 +97,22 @@ func TestSymtabAddVindexTable(t *testing.T) { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{"c1"}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ ColumnVindexes: []*vindexes.ColumnVindex{{ Columns: []sqlparser.ColIdent{ sqlparser.NewColIdent("C1"), sqlparser.NewColIdent("C2"), }, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{"c1"}}, }, { - in: &vindexes.Table{ + in: []*vindexes.Table{{ AutoIncrement: &vindexes.AutoIncrement{ Column: sqlparser.NewColIdent("C1"), }, @@ -113,22 +120,97 @@ func TestSymtabAddVindexTable(t *testing.T) { Name: sqlparser.NewColIdent("C2"), Type: sqltypes.VarChar, }}, - }, - out: []string{"c1", "c2"}, + }}, + authoritative: false, + vindexes: [][]string{{}}, + }, { + in: []*vindexes.Table{{ + Columns: []vindexes.Column{{ + Name: sqlparser.NewColIdent("C2"), + Type: sqltypes.VarChar, + }}, + }, { + Columns: []vindexes.Column{{ + Name: sqlparser.NewColIdent("C1"), + Type: sqltypes.VarChar, + }}, + }}, + authoritative: false, + vindexes: [][]string{{}, {}}, + }, { + in: []*vindexes.Table{{ + Columns: []vindexes.Column{{ + Name: sqlparser.NewColIdent("C2"), + Type: sqltypes.VarChar, + }}, + }, { + Columns: []vindexes.Column{{ + Name: sqlparser.NewColIdent("C1"), + Type: sqltypes.VarChar, + }}, + ColumnListAuthoritative: true, + }}, + authoritative: true, + vindexes: [][]string{{}, {}}, + }, { + in: []*vindexes.Table{{ + ColumnVindexes: []*vindexes.ColumnVindex{{ + Columns: []sqlparser.ColIdent{ + sqlparser.NewColIdent("C1"), + }, + }}, + }, { + ColumnVindexes: []*vindexes.ColumnVindex{{ + Columns: []sqlparser.ColIdent{ + sqlparser.NewColIdent("C2"), + }, + }}, + }}, + authoritative: false, + vindexes: [][]string{{"c1"}, {"c2"}}, + }, { + in: []*vindexes.Table{{ + ColumnVindexes: []*vindexes.ColumnVindex{{ + Columns: []sqlparser.ColIdent{ + sqlparser.NewColIdent("C1"), + }, + }, { + Columns: []sqlparser.ColIdent{ + sqlparser.NewColIdent("C2"), + }, + }}, + }}, + authoritative: false, + vindexes: [][]string{{"c1", "c2"}, {}}, }} + out := []string{"c1", "c2"} for _, tcase := range tcases { st := newSymtab() - err := st.AddVindexTable(tname, tcase.in, rb) + vindexMaps, err := st.AddVSchemaTable(tname, tcase.in, rb) if err != nil { t.Error(err) continue } tab := st.tables[tname] - for _, col := range tcase.out { + for _, col := range out { if tab.columns[col] == nil { - t.Errorf("st.AddVindexTable(%+v): column %s not found", tcase.in, col) + t.Errorf("st.AddVSchemaTable(%+v): column %s not found", tcase.in, col) } } + for i, cols := range tcase.vindexes { + for _, col := range cols { + c := tab.columns[col] + if c == nil { + t.Errorf("st.AddVSchemaTable(%+v): column %s not found", tcase.in, col) + } + if _, ok := vindexMaps[i][c]; !ok { + t.Errorf("st.AddVSchemaTable(%+v).vindexMap[%d]: column %s not found", tcase.in, i, col) + } + } + } + if tab.isAuthoritative != tcase.authoritative { + t.Errorf("st.AddVSchemaTable(%+v).authoritative: %v want %v", tcase.in, tab.isAuthoritative, tcase.authoritative) + } } } diff --git a/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt b/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt index 07b85698b7c..18ac18ddf28 100644 --- a/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/aggr_cases.txt @@ -978,6 +978,51 @@ } } +# routing rules for aggregates where sharded and unsharded match. Unsharded always wins. +"select id, count(*) from route2 group by id" +{ + "Original": "select id, count(*) from route2 group by id", + "Instructions": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select id, count(*) from unsharded as route2 group by id", + "FieldQuery": "select id, count(*) from unsharded as route2 where 1 != 1 group by id" + } +} + +# routing rules for select distinct where sharded and unsharded match. Unsharded always wins. +"select distinct id, a from route2" +{ + "Original": "select distinct id, a from route2", + "Instructions": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select distinct id, a from unsharded as route2", + "FieldQuery": "select id, a from unsharded as route2 where 1 != 1" + } +} + +# oreder by on a reference table +"select col from ref order by col" +{ + "Original": "select col from ref order by col", + "Instructions": { + "Opcode": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from ref order by col asc", + "FieldQuery": "select col from ref where 1 != 1" + } +} + # Group by invalid column number (code is duplicated from symab). "select id from user group by 1.1" "column number is not an int" diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases.txt b/go/vt/vtgate/planbuilder/testdata/dml_cases.txt index 8de0a71d9c2..60eb32bfa4e 100644 --- a/go/vt/vtgate/planbuilder/testdata/dml_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/dml_cases.txt @@ -90,6 +90,39 @@ } } +# routing rules: updated of a routed table +"update route1 set a=1 where id=1" +{ + "Original": "update route1 set a=1 where id=1", + "Instructions": { + "Opcode": "UpdateEqual", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "update user as route1 set a = 1 where id = 1", + "Vindex": "user_index", + "Values": [ + 1 + ], + "Table": "user" + } +} + +# routing rules: subquery chooses unsharded route for update +"update unsharded_a set a=(select a from route2)" +{ + "Original": "update unsharded_a set a=(select a from route2)", + "Instructions": { + "Opcode": "UpdateUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "update unsharded_a set a = (select a from unsharded as route2)" + } +} + # delete unsharded "delete from unsharded" { @@ -339,6 +372,40 @@ } } +# routing rules: deleted from a routed table +"delete from route1 where id = 1" +{ + "Original": "delete from route1 where id = 1", + "Instructions": { + "Opcode": "DeleteEqual", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "delete from user as route1 where id = 1", + "Vindex": "user_index", + "Values": [ + 1 + ], + "Table": "user", + "OwnedVindexQuery": "select Name, Costly from user where id = 1 for update" + } +} + +# routing rules: subquery chooses unsharded route for delete +"delete from unsharded_a where a=(select a from route2)" +{ + "Original": "delete from unsharded_a where a=(select a from route2)", + "Instructions": { + "Opcode": "DeleteUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "delete from unsharded_a where a = (select a from unsharded as route2)" + } +} + # update by lookup "update music set val = 1 where id = 1" { @@ -682,6 +749,52 @@ } } +# insert into a routed table +"insert into route1(id) values (1)" +{ + "Original": "insert into route1(id) values (1)", + "Instructions": { + "Opcode": "InsertSharded", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "insert into user(id, Name, Costly) values (:_Id0, :_Name0, :_Costly0)", + "Values": [ + [ + [ + ":__seq0" + ] + ], + [ + [ + null + ] + ], + [ + [ + null + ] + ] + ], + "Table": "user", + "Generate": { + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select next :n values from seq", + "Values": [ + 1 + ] + }, + "Prefix": "insert into user(id, Name, Costly) values ", + "Mid": [ + "(:_Id0, :_Name0, :_Costly0)" + ] + } +} + # insert no column list "insert into user values(1, 2, 3)" "no column list" @@ -988,6 +1101,33 @@ } } +# insert with query timeout +"insert /*vt+ QUERY_TIMEOUT_MS=1 */ into user(id) values (1), (2)" +{ + "Original": "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into user(id) values (1), (2)", + "Instructions": { + "Opcode": "InsertSharded", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into user(id, Name, Costly) values (:_Id0, :_Name0, :_Costly0), (:_Id1, :_Name1, :_Costly1)", + "Values": [[[":__seq0",":__seq1"]],[[null,null]],[[null,null]]], + "Table": "user", + "Generate": { + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select next :n values from seq", + "Values": [1,2] + }, + "Prefix": "insert /*vt+ QUERY_TIMEOUT_MS=1 */ into user(id, Name, Costly) values ", + "Mid": ["(:_Id0, :_Name0, :_Costly0)","(:_Id1, :_Name1, :_Costly1)"], + "QueryTimeout": 1 + } +} + # insert with multiple rows - multi-shard autocommit "insert /*vt+ MULTI_SHARD_AUTOCOMMIT=1 */ into user(id) values (1), (2)" { @@ -1017,7 +1157,7 @@ # insert into a vindex not allowed "insert into user_index(id) values(1)" -"inserting into a vindex not allowed: user_index" +"unsupported: multi-shard or vindex write statement" # simple replace unsharded "replace into unsharded values(1, 2)" @@ -1337,6 +1477,22 @@ } } +# update with no primary vindex on where clause (scatter update) - query timeout +"update /*vt+ QUERY_TIMEOUT_MS=1 */ user_extra set val = 1" +{ + "Original": "update /*vt+ QUERY_TIMEOUT_MS=1 */ user_extra set val = 1", + "Instructions": { + "Opcode": "UpdateScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "update /*vt+ QUERY_TIMEOUT_MS=1 */ user_extra set val = 1", + "Table": "user_extra", + "QueryTimeout": 1 + } +} + # update with non-comparison expr "update user_extra set val = 1 where id between 1 and 2" { @@ -1489,6 +1645,22 @@ } } +# delete from with no index match - query timeout +"delete /*vt+ QUERY_TIMEOUT_MS=1 */ from user_extra where name = 'jose'" +{ + "Original": "delete /*vt+ QUERY_TIMEOUT_MS=1 */ from user_extra where name = 'jose'", + "Instructions": { + "Opcode": "DeleteScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "delete /*vt+ QUERY_TIMEOUT_MS=1 */ from user_extra where name = 'jose'", + "Table": "user_extra", + "QueryTimeout": 1 + } +} + # delete from with primary id in through IN clause "delete from user_extra where user_id in (1, 2)" { diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt index b9b9da4a4f5..6a8eaaf80a2 100644 --- a/go/vt/vtgate/planbuilder/testdata/filter_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.txt @@ -524,6 +524,44 @@ } } +# routing rules: choose first table due to better plan +"select col from route1 where id = 1" +{ + "Original": "select col from route1 where id = 1", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user as route1 where id = 1", + "FieldQuery": "select col from user as route1 where 1 != 1", + "Vindex": "user_index", + "Values": [ + 1 + ] + } +} + +# routing rules: choose second table due to better plan +"select col from route1 where email = 'aaa'" +{ + "Original": "select col from route1 where email = 'aaa'", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user_metadata as route1 where email = 'aaa'", + "FieldQuery": "select col from user_metadata as route1 where 1 != 1", + "Vindex": "email_user_map", + "Values": [ + "aaa" + ] + } +} + # subquery of information_schema with itself "select * from information_schema.a where id in (select * from information_schema.b)" { @@ -902,6 +940,54 @@ } } +# routing rules subquery merge +"select col from user where id = (select id from route2 where route2.id = user.id)" +{ + "Original": "select col from user where id = (select id from route2 where route2.id = user.id)", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user where id = (select id from user as route2 where route2.id = user.id)", + "FieldQuery": "select col from user where 1 != 1" + } +} + +# routing rules subquery pullout +"select col from user where id = (select id from route2)" +{ + "Original": "select col from user where id = (select id from route2)", + "Instructions": { + "Opcode": "PulloutValue", + "SubqueryResult": "__sq1", + "HasValues": "__sq_has_values1", + "Subquery": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select id from unsharded as route2", + "FieldQuery": "select id from unsharded as route2 where 1 != 1" + }, + "Underlying": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user where id = :__sq1", + "FieldQuery": "select col from user where 1 != 1", + "Vindex": "user_index", + "Values": [ + ":__sq1" + ] + } + } +} + # Case preservation test "select user_extra.Id from user join user_extra on user.iD = user_extra.User_Id where user.Id = 5" { diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.txt b/go/vt/vtgate/planbuilder/testdata/from_cases.txt index bc71a4c047f..0925a54a109 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.txt @@ -28,6 +28,36 @@ } } +# Select from sequence +"select next 2 values from seq" +{ + "Original": "select next 2 values from seq", + "Instructions": { + "Opcode": "SelectNext", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select next 2 values from seq", + "FieldQuery": "select next 2 values from seq where 1 != 1" + } +} + +# Select from reference +"select * from ref" +{ + "Original": "select * from ref", + "Instructions": { + "Opcode": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select * from ref", + "FieldQuery": "select * from ref where 1 != 1" + } +} + # Single information_schema query "select col from information_schema.foo" { @@ -88,6 +118,66 @@ } } +# routing rules where table name matches, and there's no alias. +"select * from second_user.user" +{ + "Original": "select * from second_user.user", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select * from user", + "FieldQuery": "select * from user where 1 != 1" + } +} + +# routing rules where table name matches, and there's an alias. +"select * from second_user.user as a" +{ + "Original": "select * from second_user.user as a", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select * from user as a", + "FieldQuery": "select * from user as a where 1 != 1" + } +} + +# routing rules where table name does not match, and there's no alias. +"select * from route1" +{ + "Original": "select * from route1", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select * from user as route1", + "FieldQuery": "select * from user as route1 where 1 != 1" + } +} + +# routing rules where table name does not match, and there's an alias. +"select * from route1 as a" +{ + "Original": "select * from route1 as a", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select * from user as a", + "FieldQuery": "select * from user as a where 1 != 1" + } +} + # ',' join "select music.col from user, music" { @@ -730,6 +820,111 @@ } } +# join with reference table +"select user.col from user join ref" +{ + "Original": "select user.col from user join ref", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select user.col from user join ref", + "FieldQuery": "select user.col from user join ref where 1 != 1" + } +} + +# reference table self-join +"select r1.col from ref r1 join ref" +{ + "Original": "select r1.col from ref r1 join ref", + "Instructions": { + "Opcode": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select r1.col from ref as r1 join ref", + "FieldQuery": "select r1.col from ref as r1 join ref where 1 != 1" + } +} + +# reference table doesn't merge with other opcodes yet. +"select ref.col from ref join user" +{ + "Original": "select ref.col from ref join user", + "Instructions": { + "Opcode": "Join", + "Left": { + "Opcode": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select ref.col from ref", + "FieldQuery": "select ref.col from ref where 1 != 1" + }, + "Right": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select 1 from user", + "FieldQuery": "select 1 from user where 1 != 1" + }, + "Cols": [ + -1 + ] + } +} + +# routing rules for join, merged route wins +"select route2.col from route2 join user_extra on route2.id = user_extra.user_id" +{ + "Original": "select route2.col from route2 join user_extra on route2.id = user_extra.user_id", + "Instructions": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select route2.col from user as route2 join user_extra on route2.id = user_extra.user_id", + "FieldQuery": "select route2.col from user as route2 join user_extra on route2.id = user_extra.user_id where 1 != 1" + } +} + +# routing rules for join, unsharded route wins if we can't find a merged route +"select route2.col from route2 join user_extra" +{ + "Original": "select route2.col from route2 join user_extra", + "Instructions": { + "Opcode": "Join", + "Left": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select route2.col from unsharded as route2", + "FieldQuery": "select route2.col from unsharded as route2 where 1 != 1" + }, + "Right": { + "Opcode": "SelectScatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select 1 from user_extra", + "FieldQuery": "select 1 from user_extra where 1 != 1" + }, + "Cols": [ + -1 + ] + } +} + # subquery "select id from (select id, col from user where id = 5) as t" { @@ -835,6 +1030,61 @@ } } +# routing rules for subquery where the first route wins +"select id from (select id, col from route1 where id = 5) as t" +{ + "Original": "select id from (select id, col from route1 where id = 5) as t", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select id from (select id, col from user as route1 where id = 5) as t", + "FieldQuery": "select id from (select id, col from user as route1 where 1 != 1) as t where 1 != 1", + "Vindex": "user_index", + "Values": [5] + } +} + +# routing rules for subquery where the second route wins +"select id from (select id, col from route1 where email = 'aaa') as t" +{ + "Original": "select id from (select id, col from route1 where email = 'aaa') as t", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select id from (select id, col from user_metadata as route1 where email = 'aaa') as t", + "FieldQuery": "select id from (select id, col from user_metadata as route1 where 1 != 1) as t where 1 != 1", + "Vindex": "email_user_map", + "Values": [ + "aaa" + ] + } +} + +# routing rules for subquery where the constraint is in the outer query +"select id from (select id, col from route1) as t where id = 5" +{ + "Original": "select id from (select id, col from route1) as t where id = 5", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select id from (select id, col from user as route1) as t where id = 5", + "FieldQuery": "select id from (select id, col from user as route1 where 1 != 1) as t where 1 != 1", + "Vindex": "user_index", + "Values": [ + 5 + ] + } +} + # merge subqueries with single-shard routes "select u.col, e.col from (select col from user where id = 5) as u join (select col from user_extra where user_id = 5) as e" { @@ -852,6 +1102,66 @@ } } +# join of information_schema with normal table +"select foo from information_schema.a join unsharded" +{ + "Original": "select foo from information_schema.a join unsharded", + "Instructions": { + "Opcode": "Join", + "Left": { + "Opcode": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select foo from information_schema.a", + "FieldQuery": "select foo from information_schema.a where 1 != 1" + }, + "Right": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select 1 from unsharded", + "FieldQuery": "select 1 from unsharded where 1 != 1" + }, + "Cols": [ + -1 + ] + } +} + +# join of normal table with information_schema +"select foo from unsharded join information_schema.a" +{ + "Original": "select foo from unsharded join information_schema.a", + "Instructions": { + "Opcode": "Join", + "Left": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select foo from unsharded", + "FieldQuery": "select foo from unsharded where 1 != 1" + }, + "Right": { + "Opcode": "SelectDBA", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select 1 from information_schema.a", + "FieldQuery": "select 1 from information_schema.a where 1 != 1" + }, + "Cols": [ + -1 + ] + } +} + # wire-up on join with cross-shard subquery "select t.col1 from (select user.id, user.col1 from user join user_extra) as t join unsharded on unsharded.col1 = t.col1 and unsharded.id = t.id" { diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt index 594deb8e290..63f0237b561 100644 --- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.txt @@ -774,6 +774,44 @@ } } +# routing rules: order by test pushed to all options. table 1 is chosen. +"select col from route1 where id = 1 order by col" +{ + "Original": "select col from route1 where id = 1 order by col", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user as route1 where id = 1 order by col asc", + "FieldQuery": "select col from user as route1 where 1 != 1", + "Vindex": "user_index", + "Values": [ + 1 + ] + } +} + +# routing rules: order by test pushed to all options. table 2 is chosen. +"select col from route1 where email = 'aaa' order by col" +{ + "Original": "select col from route1 where email = 'aaa' order by col", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from user_metadata as route1 where email = 'aaa' order by col asc", + "FieldQuery": "select col from user_metadata as route1 where 1 != 1", + "Vindex": "email_user_map", + "Values": [ + "aaa" + ] + } +} + # LIMIT "select col1 from user where id = 1 limit 1" { @@ -921,6 +959,21 @@ } } +# limit on reference table +"select col from ref limit 1" +{ + "Original": "select col from ref limit 1", + "Instructions": { + "Opcode": "SelectReference", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select col from ref limit 1", + "FieldQuery": "select col from ref where 1 != 1" + } +} + # invalid limit expression "select id from user limit 1+1" "unexpected expression in LIMIT: limit 1 + 1" diff --git a/go/vt/vtgate/planbuilder/testdata/schema_test.json b/go/vt/vtgate/planbuilder/testdata/schema_test.json index 01b19f5c2c2..b70892b23d0 100644 --- a/go/vt/vtgate/planbuilder/testdata/schema_test.json +++ b/go/vt/vtgate/planbuilder/testdata/schema_test.json @@ -1,4 +1,16 @@ { + "routing_rules": { + "rules": [{ + "from_table": "route1", + "to_tables": ["user.user", "user.user_metadata"] + }, { + "from_table": "route2", + "to_tables": ["user.user", "main.unsharded"] + }, { + "from_table": "second_user.user", + "to_tables": ["user.user"] + }] + }, "keyspaces": { "user": { "sharded": true, @@ -177,6 +189,9 @@ } ] }, + "ref": { + "type": "reference" + }, "pin_test": { "pinned": "80" }, diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.txt b/go/vt/vtgate/planbuilder/testdata/select_cases.txt index 3380f73e25a..0f331bf423a 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.txt @@ -344,22 +344,7 @@ # nextval for simple route "select next value from user" -"NEXT used on a sharded table" - -# next n values for simple route -"select next 2 values from seq" -{ - "Original": "select next 2 values from seq", - "Instructions": { - "Opcode": "SelectNext", - "Keyspace": { - "Name": "main", - "Sharded": false - }, - "Query": "select next 2 values from seq", - "FieldQuery": "select next 2 values from seq where 1 != 1" - } -} +"NEXT used on a non-sequence table" # last_insert_id for unsharded route "select last_insert_id() from main.dual" @@ -1078,3 +1063,38 @@ "FieldQuery": "(select id from unsharded where 1 != 1) union (select id from unsharded where 1 != 1)" } } + +# routing rules: union finds the correct option to merge, and renames the table +"select a from route1 where email = 'aaa' union select b from route1 where email = 'aaa'" +{ + "Original": "select a from route1 where email = 'aaa' union select b from route1 where email = 'aaa'", + "Instructions": { + "Opcode": "SelectEqualUnique", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "Query": "select a from user_metadata as route1 where email = 'aaa' union select b from user_metadata as route1 where email = 'aaa'", + "FieldQuery": "select a from user_metadata as route1 where 1 != 1 union select b from user_metadata as route1 where 1 != 1", + "Vindex": "email_user_map", + "Values": [ + "aaa" + ] + } +} + +# routing rules: ensure directives are not lost +"select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from route2" +{ + "Original": "select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from route2", + "Instructions": { + "Opcode": "SelectUnsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "Query": "select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from unsharded as route2", + "FieldQuery": "select * from unsharded as route2 where 1 != 1", + "QueryTimeout": 1000 + } +} diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt index d360aa0f368..ea443d7eaa7 100644 --- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt +++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.txt @@ -1,6 +1,6 @@ # Unions "select * from user union select * from user_extra" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" # SET "set a=1" @@ -29,11 +29,11 @@ # union operations in subqueries (FROM) "select * from (select * from user union all select * from user_extra) as t" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" # union operations in subqueries (expressions) "select * from user where id in (select * from user union select * from user_extra)" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" # TODO: Implement support for select with a target destination "select * from `user[-]`.user_metadata" @@ -75,14 +75,6 @@ "select last_insert_id() from user" "unsupported: LAST_INSERT_ID is only allowed for unsharded keyspaces" -# join of information_schema with normal table -"select foo from information_schema.a join unsharded" -"unsupported: intermixing of information_schema and regular tables" - -# join of normal table with information_schema -"select foo from unsharded join information_schema.a" -"unsupported: intermixing of information_schema and regular tables" - # natural join "select * from user natural join user_extra" "unsupported: natural join" @@ -257,7 +249,7 @@ # multi delete multi table "delete user from user join user_extra on user.id = user_extra.id where user.name = 'foo'" -"unsupported: multi-table/vindex delete statement in sharded keyspace" +"unsupported: multi-shard or vindex write statement" # scatter delete with owned lookup vindex "delete from user" @@ -293,11 +285,11 @@ # join in update tables "update user join user_extra on user.id = user_extra.id set user.name = 'foo'" -"unsupported: multi-table/vindex update statement in sharded keyspace" +"unsupported: multi-shard or vindex write statement" # multiple tables in update "update user as u, user_extra as ue set u.name = 'foo' where u.id = ue.id" -"unsupported: multi-table/vindex update statement in sharded keyspace" +"unsupported: multi-shard or vindex write statement" # unsharded insert with cross-shard join" "insert into unsharded select u.col from user u join user u1" @@ -377,45 +369,45 @@ # union of information_schema with normal table "select * from information_schema.a union select * from unsharded" -"unsupported: intermixing of information_schema and regular tables" +"unsupported: UNION cannot be executed as a single route" # union of information_schema with normal table "select * from unsharded union select * from information_schema.a" -"unsupported: intermixing of information_schema and regular tables" +"unsupported: UNION cannot be executed as a single route" # multi-shard union "(select id from user union select id from music) union select 1 from dual" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" # multi-shard union "select 1 from music union (select id from user union all select name from unsharded)" -"unsupported: UNION on different keyspaces" +"unsupported: UNION cannot be executed as a single route" # multi-shard union "select 1 from music union (select id from user union select name from unsharded)" -"unsupported: UNION on different keyspaces" +"unsupported: UNION cannot be executed as a single route" # multi-shard union "select id from user union all select id from music" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" # union with the same target shard because of vindex "select * from music where id = 1 union select * from user where id = 1" -"unsupported: UNION or subquery on different shards: vindexes are different" +"unsupported: UNION cannot be executed as a single route" # union with different target shards "select 1 from music where id = 1 union select 1 from music where id = 2" -"unsupported: UNION or subquery on different shards: vindex values are different" +"unsupported: UNION cannot be executed as a single route" # Union all "select col1, col2 from user union all select col1, col2 from user_extra" -"unsupported: UNION or subquery containing multi-shard queries" +"unsupported: UNION cannot be executed as a single route" "(select user.id, user.name from user join user_extra where user_extra.extra = 'asdf') union select 'b','c' from user" -"unsupported construct: SELECT of UNION is non-trivial" +"unsupported: SELECT of UNION is non-trivial" "select 'b','c' from user union (select user.id, user.name from user join user_extra where user_extra.extra = 'asdf')" -"unsupported construct: SELECT of UNION is non-trivial" +"unsupported: SELECT of UNION is non-trivial" "select keyspace_id from user_index where id = 1 and id = 2" "unsupported: where clause for vindex function must be of the form id = (multiple filters)" @@ -440,3 +432,8 @@ "select func(keyspace_id) from user_index where id = :id" "unsupported: expression on results of a vindex function" + +# Multi-table unique vindex constraint on left table of left join, two levels of join, simple aggregation +# This should work, but doesn't. See https://github.com/vitessio/vitess/issues/4772 +"select user.id, count(*) from user left join user_extra ue1 on user.id = ue1.user_id left join user_extra ue2 on ue1.user_id = ue2.user_id group by user.id" +"unsupported: cross-shard query with aggregates" diff --git a/go/vt/vtgate/planbuilder/union.go b/go/vt/vtgate/planbuilder/union.go index b7faada2402..e1945cc0846 100644 --- a/go/vt/vtgate/planbuilder/union.go +++ b/go/vt/vtgate/planbuilder/union.go @@ -37,8 +37,7 @@ func buildUnionPlan(union *sqlparser.Union, vschema ContextVSchema) (primitive e } func (pb *primitiveBuilder) processUnion(union *sqlparser.Union, outer *symtab) error { - lpb := newPrimitiveBuilder(pb.vschema, pb.jt) - if err := lpb.processPart(union.Left, outer); err != nil { + if err := pb.processPart(union.Left, outer); err != nil { return err } rpb := newPrimitiveBuilder(pb.vschema, pb.jt) @@ -46,9 +45,7 @@ func (pb *primitiveBuilder) processUnion(union *sqlparser.Union, outer *symtab) return err } - var err error - pb.bldr, pb.st, err = unionRouteMerge(union, lpb.bldr, rpb.bldr) - if err != nil { + if err := unionRouteMerge(union, pb.bldr, rpb.bldr); err != nil { return err } pb.st.Outer = outer @@ -71,24 +68,18 @@ func (pb *primitiveBuilder) processPart(part sqlparser.SelectStatement, outer *s panic(fmt.Sprintf("BUG: unexpected SELECT type: %T", part)) } -func unionRouteMerge(union *sqlparser.Union, left, right builder) (builder, *symtab, error) { +func unionRouteMerge(union *sqlparser.Union, left, right builder) error { lroute, ok := left.(*route) if !ok { - return nil, nil, errors.New("unsupported construct: SELECT of UNION is non-trivial") + return errors.New("unsupported: SELECT of UNION is non-trivial") } rroute, ok := right.(*route) if !ok { - return nil, nil, errors.New("unsupported construct: SELECT of UNION is non-trivial") + return errors.New("unsupported: SELECT of UNION is non-trivial") } - if err := lroute.UnionCanMerge(rroute); err != nil { - return nil, nil, err + if !lroute.MergeUnion(rroute) { + return errors.New("unsupported: UNION cannot be executed as a single route") } - rb, st := newRoute( - &sqlparser.Union{Type: union.Type, Left: union.Left, Right: union.Right, Lock: union.Lock}, - lroute.ERoute, - lroute.condition, - ) - lroute.Redirect = rb - rroute.Redirect = rb - return rb, st, nil + lroute.Select = &sqlparser.Union{Type: union.Type, Left: union.Left, Right: union.Right, Lock: union.Lock} + return nil } diff --git a/go/vt/vtgate/planbuilder/update.go b/go/vt/vtgate/planbuilder/update.go index 85a9c174307..e537e4f88f3 100644 --- a/go/vt/vtgate/planbuilder/update.go +++ b/go/vt/vtgate/planbuilder/update.go @@ -32,24 +32,23 @@ import ( // buildUpdatePlan builds the instructions for an UPDATE statement. func buildUpdatePlan(upd *sqlparser.Update, vschema ContextVSchema) (*engine.Update, error) { eupd := &engine.Update{ - Query: generateQuery(upd), ChangedVindexValues: make(map[string][]sqltypes.PlanValue), } pb := newPrimitiveBuilder(vschema, newJointab(sqlparser.GetBindvars(upd))) - if err := pb.processTableExprs(upd.TableExprs); err != nil { + ro, err := pb.processDMLTable(upd.TableExprs) + if err != nil { return nil, err } - rb, ok := pb.bldr.(*route) - if !ok { - return nil, errors.New("unsupported: multi-table/vindex update statement in sharded keyspace") - } - eupd.Keyspace = rb.ERoute.Keyspace + eupd.Keyspace = ro.eroute.Keyspace if !eupd.Keyspace.Sharded { // We only validate non-table subexpressions because the previous analysis has already validated them. - if !pb.validateSubquerySamePlan(upd.Exprs, upd.Where, upd.OrderBy, upd.Limit) { + if !pb.finalizeUnshardedDMLSubqueries(upd.Exprs, upd.Where, upd.OrderBy, upd.Limit) { return nil, errors.New("unsupported: sharded subqueries in DML") } eupd.Opcode = engine.UpdateUnsharded + // Generate query after all the analysis. Otherwise table name substitutions for + // routed tables won't happen. + eupd.Query = generateQuery(upd) return eupd, nil } @@ -60,27 +59,27 @@ func buildUpdatePlan(upd *sqlparser.Update, vschema ContextVSchema) (*engine.Upd return nil, errors.New("unsupported: multi-table update statement in sharded keyspace") } + // Generate query after all the analysis. Otherwise table name substitutions for + // routed tables won't happen. + eupd.Query = generateQuery(upd) + directives := sqlparser.ExtractCommentDirectives(upd.Comments) if directives.IsSet(sqlparser.DirectiveMultiShardAutocommit) { eupd.MultiShardAutocommit = true } - var vindexTable *vindexes.Table - for _, tval := range pb.st.tables { - vindexTable = tval.vindexTable - } - eupd.Table = vindexTable + eupd.QueryTimeout = queryTimeout(directives) + eupd.Table = ro.vschemaTable if eupd.Table == nil { return nil, errors.New("internal error: table.vindexTable is mysteriously nil") } - var err error - if rb.ERoute.TargetDestination != nil { - if rb.ERoute.TargetTabletType != topodatapb.TabletType_MASTER { + if ro.eroute.TargetDestination != nil { + if ro.eroute.TargetTabletType != topodatapb.TabletType_MASTER { return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unsupported: UPDATE statement with a replica target") } eupd.Opcode = engine.UpdateByDestination - eupd.TargetDestination = rb.ERoute.TargetDestination + eupd.TargetDestination = ro.eroute.TargetDestination return eupd, nil } diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 5ae671037f9..3a9022bacfe 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -103,8 +103,8 @@ func (vc *vcursorImpl) FindTable(name sqlparser.TableName) (*vindexes.Table, str return table, destKeyspace, destTabletType, dest, err } -// FindTableOrVindex finds the specified table or vindex. -func (vc *vcursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { +// FindTablesOrVindex finds the specified table or vindex. +func (vc *vcursorImpl) FindTablesOrVindex(name sqlparser.TableName) ([]*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) { destKeyspace, destTabletType, dest, err := vc.executor.ParseDestinationTarget(name.Qualifier.String()) if err != nil { return nil, nil, "", destTabletType, nil, err @@ -112,11 +112,11 @@ func (vc *vcursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.Ta if destKeyspace == "" { destKeyspace = vc.keyspace } - table, vindex, err := vc.executor.VSchema().FindTableOrVindex(destKeyspace, name.Name.String()) + tables, vindex, err := vc.executor.VSchema().FindTablesOrVindex(destKeyspace, name.Name.String()) if err != nil { return nil, nil, "", destTabletType, nil, err } - return table, vindex, destKeyspace, destTabletType, dest, nil + return tables, vindex, destKeyspace, destTabletType, dest, nil } // DefaultKeyspace returns the default keyspace of the current request diff --git a/go/vt/vtgate/vindexes/binary.go b/go/vt/vtgate/vindexes/binary.go index 952c08f4017..2d437835bc1 100644 --- a/go/vt/vtgate/vindexes/binary.go +++ b/go/vt/vtgate/vindexes/binary.go @@ -63,7 +63,7 @@ func (vind *Binary) IsFunctional() bool { func (vind *Binary) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]byte) ([]bool, error) { out := make([]bool, len(ids)) for i := range ids { - out[i] = (bytes.Compare(ids[i].ToBytes(), ksids[i]) == 0) + out[i] = bytes.Equal(ids[i].ToBytes(), ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/binary_test.go b/go/vt/vtgate/vindexes/binary_test.go index 9a7e1d0c325..15b5d517a3e 100644 --- a/go/vt/vtgate/vindexes/binary_test.go +++ b/go/vt/vtgate/vindexes/binary_test.go @@ -61,7 +61,7 @@ func TestBinaryMap(t *testing.T) { t.Error(err) } out := []byte(got[0].(key.DestinationKeyspaceID)) - if bytes.Compare(tcase.out, out) != 0 { + if !bytes.Equal(tcase.out, out) { t.Errorf("Map(%#v): %#v, want %#v", tcase.in, out, tcase.out) } } diff --git a/go/vt/vtgate/vindexes/binarymd5.go b/go/vt/vtgate/vindexes/binarymd5.go index f741aa41c58..b3cb1f95e39 100644 --- a/go/vt/vtgate/vindexes/binarymd5.go +++ b/go/vt/vtgate/vindexes/binarymd5.go @@ -62,7 +62,7 @@ func (vind *BinaryMD5) IsFunctional() bool { func (vind *BinaryMD5) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]byte) ([]bool, error) { out := make([]bool, len(ids)) for i := range ids { - out[i] = (bytes.Compare(binHash(ids[i].ToBytes()), ksids[i]) == 0) + out[i] = bytes.Equal(binHash(ids[i].ToBytes()), ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/hash.go b/go/vt/vtgate/vindexes/hash.go index 7c03640ea53..edd47e101ff 100644 --- a/go/vt/vtgate/vindexes/hash.go +++ b/go/vt/vtgate/vindexes/hash.go @@ -101,7 +101,7 @@ func (vind *Hash) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]byte) ([]boo if err != nil { return nil, vterrors.Wrap(err, "hash.Verify") } - out[i] = bytes.Compare(vhash(num), ksids[i]) == 0 + out[i] = bytes.Equal(vhash(num), ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/lookup_hash_test.go b/go/vt/vtgate/vindexes/lookup_hash_test.go index 7cd9692d3e2..f717d9d9c75 100644 --- a/go/vt/vtgate/vindexes/lookup_hash_test.go +++ b/go/vt/vtgate/vindexes/lookup_hash_test.go @@ -37,7 +37,7 @@ func TestLookupHashNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - l, err := CreateVindex("lookup_hash", "lookup_hash", map[string]string{ + _, err := CreateVindex("lookup_hash", "lookup_hash", map[string]string{ "table": "t", "from": "fromc", "to": "toc", diff --git a/go/vt/vtgate/vindexes/lookup_hash_unique_test.go b/go/vt/vtgate/vindexes/lookup_hash_unique_test.go index f194c82ac24..536e4a7d4a9 100644 --- a/go/vt/vtgate/vindexes/lookup_hash_unique_test.go +++ b/go/vt/vtgate/vindexes/lookup_hash_unique_test.go @@ -31,7 +31,7 @@ func TestLookupHashUniqueNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - l, err := CreateVindex("lookup_hash_unique", "lookup_hash_unique", map[string]string{ + l, _ = CreateVindex("lookup_hash_unique", "lookup_hash_unique", map[string]string{ "table": "t", "from": "fromc", "to": "toc", @@ -41,7 +41,7 @@ func TestLookupHashUniqueNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - _, err = CreateVindex("lookup_hash_unique", "lookup_hash_unique", map[string]string{ + _, err := CreateVindex("lookup_hash_unique", "lookup_hash_unique", map[string]string{ "table": "t", "from": "fromc", "to": "toc", diff --git a/go/vt/vtgate/vindexes/lookup_test.go b/go/vt/vtgate/vindexes/lookup_test.go index d8d441f8a4b..cea040d387a 100644 --- a/go/vt/vtgate/vindexes/lookup_test.go +++ b/go/vt/vtgate/vindexes/lookup_test.go @@ -92,7 +92,7 @@ func TestLookupNonUniqueNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - l, err := CreateVindex("lookup", "lookup", map[string]string{ + _, err := CreateVindex("lookup", "lookup", map[string]string{ "table": "t", "from": "fromc", "to": "toc", @@ -484,7 +484,7 @@ func TestLookupNonUniqueDelete(t *testing.T) { } func TestLookupNonUniqueDeleteAutocommit(t *testing.T) { - lookupNonUnique, err := CreateVindex("lookup", "lookup", map[string]string{ + lookupNonUnique, _ := CreateVindex("lookup", "lookup", map[string]string{ "table": "t", "from": "fromc", "to": "toc", @@ -492,7 +492,7 @@ func TestLookupNonUniqueDeleteAutocommit(t *testing.T) { }) vc := &vcursor{} - err = lookupNonUnique.(Lookup).Delete(vc, [][]sqltypes.Value{{sqltypes.NewInt64(1)}, {sqltypes.NewInt64(2)}}, []byte("test")) + err := lookupNonUnique.(Lookup).Delete(vc, [][]sqltypes.Value{{sqltypes.NewInt64(1)}, {sqltypes.NewInt64(2)}}, []byte("test")) if err != nil { t.Error(err) } diff --git a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go index 54eeb5f047c..f5414e6bcd4 100644 --- a/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go +++ b/go/vt/vtgate/vindexes/lookup_unicodeloosemd5_hash_test.go @@ -27,10 +27,12 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -const hashed10 uint64 = 17563797831108199066 -const hashed20 uint64 = 8729390916138266389 -const hashed30 uint64 = 1472608112194674795 -const hashed40 uint64 = 16576388050845489136 +const ( + hashed10 uint64 = 17563797831108199066 + hashed20 uint64 = 8729390916138266389 + hashed30 uint64 = 1472608112194674795 + hashed40 uint64 = 16576388050845489136 +) func TestLookupUnicodeLooseMD5HashMap(t *testing.T) { lookup := createLookup(t, "lookup_unicodeloosemd5_hash", false) @@ -408,7 +410,7 @@ func TestLookupUnicodeLooseMD5HashDelete(t *testing.T) { } func TestLookupUnicodeLooseMD5HashDeleteAutocommit(t *testing.T) { - lookupNonUnique, err := CreateVindex("lookup_unicodeloosemd5_hash", "lookup", map[string]string{ + lookupNonUnique, _ := CreateVindex("lookup_unicodeloosemd5_hash", "lookup", map[string]string{ "table": "t", "from": "fromc", "to": "toc", @@ -416,7 +418,7 @@ func TestLookupUnicodeLooseMD5HashDeleteAutocommit(t *testing.T) { }) vc := &vcursor{} - err = lookupNonUnique.(Lookup).Delete(vc, [][]sqltypes.Value{{sqltypes.NewInt64(10)}, {sqltypes.NewInt64(20)}}, []byte("\x16k@\xb4J\xbaK\xd6")) + err := lookupNonUnique.(Lookup).Delete(vc, [][]sqltypes.Value{{sqltypes.NewInt64(10)}, {sqltypes.NewInt64(20)}}, []byte("\x16k@\xb4J\xbaK\xd6")) if err != nil { t.Error(err) } diff --git a/go/vt/vtgate/vindexes/lookup_unique_test.go b/go/vt/vtgate/vindexes/lookup_unique_test.go index 41326591ad7..7c15e240bc7 100644 --- a/go/vt/vtgate/vindexes/lookup_unique_test.go +++ b/go/vt/vtgate/vindexes/lookup_unique_test.go @@ -33,7 +33,7 @@ func TestLookupUniqueNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - l, err := CreateVindex("lookup_unique", "lookup_unique", map[string]string{ + l, _ = CreateVindex("lookup_unique", "lookup_unique", map[string]string{ "table": "t", "from": "fromc", "to": "toc", @@ -43,7 +43,7 @@ func TestLookupUniqueNew(t *testing.T) { t.Errorf("Create(lookup, false): %v, want %v", got, want) } - _, err = CreateVindex("lookup_unique", "lookup_unique", map[string]string{ + _, err := CreateVindex("lookup_unique", "lookup_unique", map[string]string{ "table": "t", "from": "fromc", "to": "toc", diff --git a/go/vt/vtgate/vindexes/null.go b/go/vt/vtgate/vindexes/null.go index a726d111e8c..ade93ae474f 100644 --- a/go/vt/vtgate/vindexes/null.go +++ b/go/vt/vtgate/vindexes/null.go @@ -76,7 +76,7 @@ func (vind *Null) Map(cursor VCursor, ids []sqltypes.Value) ([]key.Destination, func (vind *Null) Verify(cursor VCursor, ids []sqltypes.Value, ksids [][]byte) ([]bool, error) { out := make([]bool, len(ids)) for i := range ids { - out[i] = bytes.Compare(nullksid, ksids[i]) == 0 + out[i] = bytes.Equal(nullksid, ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/numeric.go b/go/vt/vtgate/vindexes/numeric.go index ec48cb05d0c..66ffc5c92de 100644 --- a/go/vt/vtgate/vindexes/numeric.go +++ b/go/vt/vtgate/vindexes/numeric.go @@ -72,7 +72,7 @@ func (*Numeric) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]byte) ([]bool, return nil, vterrors.Wrap(err, "Numeric.Verify") } binary.BigEndian.PutUint64(keybytes[:], num) - out[i] = bytes.Compare(keybytes[:], ksids[i]) == 0 + out[i] = bytes.Equal(keybytes[:], ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/numeric_static_map.go b/go/vt/vtgate/vindexes/numeric_static_map.go index e0d6198a9e1..42f49f2d780 100644 --- a/go/vt/vtgate/vindexes/numeric_static_map.go +++ b/go/vt/vtgate/vindexes/numeric_static_map.go @@ -99,7 +99,7 @@ func (vind *NumericStaticMap) Verify(_ VCursor, ids []sqltypes.Value, ksids [][] num = lookupNum } binary.BigEndian.PutUint64(keybytes[:], num) - out[i] = bytes.Compare(keybytes[:], ksids[i]) == 0 + out[i] = bytes.Equal(keybytes[:], ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/reverse_bits.go b/go/vt/vtgate/vindexes/reverse_bits.go index eb193935367..fcc661d294d 100644 --- a/go/vt/vtgate/vindexes/reverse_bits.go +++ b/go/vt/vtgate/vindexes/reverse_bits.go @@ -86,7 +86,7 @@ func (vind *ReverseBits) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]byte) if err != nil { return nil, vterrors.Wrap(err, "reverseBits.Verify") } - out[i] = bytes.Compare(reverse(num), ksids[i]) == 0 + out[i] = bytes.Equal(reverse(num), ksids[i]) } return out, nil } diff --git a/go/vt/vtgate/vindexes/unicodeloosemd5.go b/go/vt/vtgate/vindexes/unicodeloosemd5.go index e7c93b3a2d2..1a0c63f4151 100644 --- a/go/vt/vtgate/vindexes/unicodeloosemd5.go +++ b/go/vt/vtgate/vindexes/unicodeloosemd5.go @@ -75,7 +75,7 @@ func (vind *UnicodeLooseMD5) Verify(_ VCursor, ids []sqltypes.Value, ksids [][]b if err != nil { return nil, fmt.Errorf("UnicodeLooseMD5.Verify: %v", err) } - out[i] = (bytes.Compare(data, ksids[i]) == 0) + out[i] = bytes.Equal(data, ksids[i]) } return out, nil } @@ -94,7 +94,7 @@ func (vind *UnicodeLooseMD5) Map(cursor VCursor, ids []sqltypes.Value) ([]key.De } func unicodeHash(key sqltypes.Value) ([]byte, error) { - collator := collatorPool.Get().(pooledCollator) + collator := collatorPool.Get().(*pooledCollator) defer collatorPool.Put(collator) norm, err := normalize(collator.col, collator.buf, key.ToBytes()) @@ -144,7 +144,7 @@ func newPooledCollator() interface{} { // way to verify this. // Also, the locale differences are not an issue for level 1, // because the conservative comparison makes them all equal. - return pooledCollator{ + return &pooledCollator{ col: collate.New(language.English, collate.Loose), buf: new(collate.Buffer), } diff --git a/go/vt/vtgate/vindexes/unicodeloosemd5_test.go b/go/vt/vtgate/vindexes/unicodeloosemd5_test.go index d95254d9f79..a5b614dddff 100644 --- a/go/vt/vtgate/vindexes/unicodeloosemd5_test.go +++ b/go/vt/vtgate/vindexes/unicodeloosemd5_test.go @@ -137,7 +137,7 @@ func TestNormalization(t *testing.T) { in: "T", out: "\x18\x16", }} - collator := newPooledCollator().(pooledCollator) + collator := newPooledCollator().(*pooledCollator) for _, tcase := range tcases { norm, err := normalize(collator.col, collator.buf, []byte(tcase.in)) if err != nil { @@ -158,7 +158,7 @@ func TestInvalidUnicodeNormalization(t *testing.T) { "\x8a[\xdf,\u007fĄE\x92\xd2W+\xcd\x06h\xd2", } wantErr := "invalid UTF-8" - collator := newPooledCollator().(pooledCollator) + collator := newPooledCollator().(*pooledCollator) for _, in := range inputs { // We've observed that infinite looping is a possible failure mode for the @@ -190,7 +190,7 @@ func BenchmarkNormalizeSafe(b *testing.B) { input := []byte("testing") for i := 0; i < b.N; i++ { - collator := newPooledCollator().(pooledCollator) + collator := newPooledCollator().(*pooledCollator) normalize(collator.col, collator.buf, input) } } @@ -199,7 +199,7 @@ func BenchmarkNormalizeSafe(b *testing.B) { // are shared between iterations, assuming no concurrency. func BenchmarkNormalizeShared(b *testing.B) { input := []byte("testing") - collator := newPooledCollator().(pooledCollator) + collator := newPooledCollator().(*pooledCollator) for i := 0; i < b.N; i++ { normalize(collator.col, collator.buf, input) @@ -212,7 +212,7 @@ func BenchmarkNormalizePooled(b *testing.B) { input := []byte("testing") for i := 0; i < b.N; i++ { - collator := collatorPool.Get().(pooledCollator) + collator := collatorPool.Get().(*pooledCollator) normalize(collator.col, collator.buf, input) collatorPool.Put(collator) } diff --git a/go/vt/vtgate/vindexes/vschema.go b/go/vt/vtgate/vindexes/vschema.go index ece6f1e58b1..a7bc43036a9 100644 --- a/go/vt/vtgate/vindexes/vschema.go +++ b/go/vt/vtgate/vindexes/vschema.go @@ -31,17 +31,42 @@ import ( vschemapb "vitess.io/vitess/go/vt/proto/vschema" ) +// The following constants represent table types. +const ( + TypeSequence = "sequence" + TypeReference = "reference" +) + // VSchema represents the denormalized version of SrvVSchema, // used for building routing plans. type VSchema struct { + RoutingRules map[string]*RoutingRule `json:"routing_rules"` uniqueTables map[string]*Table uniqueVindexes map[string]Vindex Keyspaces map[string]*KeyspaceSchema `json:"keyspaces"` } +// RoutingRule represents one routing rule. +type RoutingRule struct { + Tables []*Table + Error error +} + +// MarshalJSON returns a JSON representation of Column. +func (rr *RoutingRule) MarshalJSON() ([]byte, error) { + if rr.Error != nil { + return json.Marshal(rr.Error.Error()) + } + tables := make([]string, 0, len(rr.Tables)) + for _, t := range rr.Tables { + tables = append(tables, t.Keyspace.Name+"."+t.Name.String()) + } + return json.Marshal(tables) +} + // Table represents a table in VSchema. type Table struct { - IsSequence bool `json:"is_sequence,omitempty"` + Type string `json:"type,omitempty"` Name sqlparser.TableIdent `json:"name"` Keyspace *Keyspace `json:"-"` ColumnVindexes []*ColumnVindex `json:"column_vindexes,omitempty"` @@ -122,14 +147,15 @@ type AutoIncrement struct { // BuildVSchema builds a VSchema from a SrvVSchema. func BuildVSchema(source *vschemapb.SrvVSchema) (vschema *VSchema, err error) { vschema = &VSchema{ + RoutingRules: make(map[string]*RoutingRule), uniqueTables: make(map[string]*Table), uniqueVindexes: make(map[string]Vindex), Keyspaces: make(map[string]*KeyspaceSchema), } buildKeyspaces(source, vschema) - buildTables(source, vschema) resolveAutoIncrement(source, vschema) addDual(vschema) + buildRoutingRule(source, vschema) return vschema, nil } @@ -151,7 +177,6 @@ func BuildKeyspaceSchema(input *vschemapb.Keyspace, keyspace string) (*KeyspaceS Keyspaces: make(map[string]*KeyspaceSchema), } buildKeyspaces(formal, vschema) - buildTables(formal, vschema) err := vschema.Keyspaces[keyspace].Error return vschema.Keyspaces[keyspace], err } @@ -165,7 +190,7 @@ func ValidateKeyspace(input *vschemapb.Keyspace) error { func buildKeyspaces(source *vschemapb.SrvVSchema, vschema *VSchema) { for ksname, ks := range source.Keyspaces { - vschema.Keyspaces[ksname] = &KeyspaceSchema{ + ksvschema := &KeyspaceSchema{ Keyspace: &Keyspace{ Name: ksname, Sharded: ks.Sharded, @@ -173,120 +198,123 @@ func buildKeyspaces(source *vschemapb.SrvVSchema, vschema *VSchema) { Tables: make(map[string]*Table), Vindexes: make(map[string]Vindex), } + vschema.Keyspaces[ksname] = ksvschema + ksvschema.Error = buildTables(ks, vschema, ksvschema) } } -func buildTables(source *vschemapb.SrvVSchema, vschema *VSchema) { -outer: - for ksname, ks := range source.Keyspaces { - ksvschema := vschema.Keyspaces[ksname] - keyspace := ksvschema.Keyspace - for vname, vindexInfo := range ks.Vindexes { - vindex, err := CreateVindex(vindexInfo.Type, vname, vindexInfo.Params) - if err != nil { - ksvschema.Error = err - continue outer - } - if _, ok := vschema.uniqueVindexes[vname]; ok { - vschema.uniqueVindexes[vname] = nil - } else { - vschema.uniqueVindexes[vname] = vindex - } - vschema.Keyspaces[ksname].Vindexes[vname] = vindex +func buildTables(ks *vschemapb.Keyspace, vschema *VSchema, ksvschema *KeyspaceSchema) error { + keyspace := ksvschema.Keyspace + for vname, vindexInfo := range ks.Vindexes { + vindex, err := CreateVindex(vindexInfo.Type, vname, vindexInfo.Params) + if err != nil { + return err } - for tname, table := range ks.Tables { - t := &Table{ - Name: sqlparser.NewTableIdent(tname), - Keyspace: keyspace, - ColumnListAuthoritative: table.ColumnListAuthoritative, - } - if _, ok := vschema.uniqueTables[tname]; ok { - vschema.uniqueTables[tname] = nil - } else { - vschema.uniqueTables[tname] = t - } - vschema.Keyspaces[ksname].Tables[tname] = t - if table.Type == "sequence" { - t.IsSequence = true + if _, ok := vschema.uniqueVindexes[vname]; ok { + vschema.uniqueVindexes[vname] = nil + } else { + vschema.uniqueVindexes[vname] = vindex + } + ksvschema.Vindexes[vname] = vindex + } + for tname, table := range ks.Tables { + t := &Table{ + Name: sqlparser.NewTableIdent(tname), + Keyspace: keyspace, + ColumnListAuthoritative: table.ColumnListAuthoritative, + } + switch table.Type { + case "", TypeReference: + t.Type = table.Type + case TypeSequence: + if keyspace.Sharded && table.Pinned == "" { + return fmt.Errorf("sequence table has to be in an unsharded keyspace or must be pinned: %s", tname) } - if table.Pinned != "" { - decoded, err := hex.DecodeString(table.Pinned) - if err != nil { - ksvschema.Error = fmt.Errorf("could not decode the keyspace id for pin: %v", err) - continue outer - } - t.Pinned = decoded - } else if keyspace.Sharded && len(table.ColumnVindexes) == 0 { - ksvschema.Error = fmt.Errorf("missing primary col vindex for table: %s", tname) - continue outer + t.Type = table.Type + default: + return fmt.Errorf("unidentified table type %s", table.Type) + } + if table.Pinned != "" { + decoded, err := hex.DecodeString(table.Pinned) + if err != nil { + return fmt.Errorf("could not decode the keyspace id for pin: %v", err) } + t.Pinned = decoded + } - // Initialize Columns. - colNames := make(map[string]bool) - for _, col := range table.Columns { - name := sqlparser.NewColIdent(col.Name) - if colNames[name.Lowered()] { - ksvschema.Error = fmt.Errorf("duplicate column name '%v' for table: %s", name, tname) - continue outer - } - colNames[name.Lowered()] = true - t.Columns = append(t.Columns, Column{Name: name, Type: col.Type}) + // If keyspace is sharded, then any table that's not a reference or pinned must have vindexes. + if keyspace.Sharded && t.Type != TypeReference && table.Pinned == "" && len(table.ColumnVindexes) == 0 { + return fmt.Errorf("missing primary col vindex for table: %s", tname) + } + + // Initialize Columns. + colNames := make(map[string]bool) + for _, col := range table.Columns { + name := sqlparser.NewColIdent(col.Name) + if colNames[name.Lowered()] { + return fmt.Errorf("duplicate column name '%v' for table: %s", name, tname) } + colNames[name.Lowered()] = true + t.Columns = append(t.Columns, Column{Name: name, Type: col.Type}) + } - // Initialize ColumnVindexes. - for i, ind := range table.ColumnVindexes { - vindexInfo, ok := ks.Vindexes[ind.Name] - if !ok { - ksvschema.Error = fmt.Errorf("vindex %s not found for table %s", ind.Name, tname) - continue outer - } - vindex := vschema.Keyspaces[ksname].Vindexes[ind.Name] - owned := false - if _, ok := vindex.(Lookup); ok && vindexInfo.Owner == tname { - owned = true + // Initialize ColumnVindexes. + for i, ind := range table.ColumnVindexes { + vindexInfo, ok := ks.Vindexes[ind.Name] + if !ok { + return fmt.Errorf("vindex %s not found for table %s", ind.Name, tname) + } + vindex := ksvschema.Vindexes[ind.Name] + owned := false + if _, ok := vindex.(Lookup); ok && vindexInfo.Owner == tname { + owned = true + } + var columns []sqlparser.ColIdent + if ind.Column != "" { + if len(ind.Columns) > 0 { + return fmt.Errorf("can't use column and columns at the same time in vindex (%s) and table (%s)", ind.Name, tname) } - var columns []sqlparser.ColIdent - if ind.Column != "" { - if len(ind.Columns) > 0 { - ksvschema.Error = fmt.Errorf("can't use column and columns at the same time in vindex (%s) and table (%s)", ind.Name, tname) - continue outer - } - columns = []sqlparser.ColIdent{sqlparser.NewColIdent(ind.Column)} - } else { - if len(ind.Columns) == 0 { - ksvschema.Error = fmt.Errorf("must specify at least one column for vindex (%s) and table (%s)", ind.Name, tname) - continue outer - } - for _, indCol := range ind.Columns { - columns = append(columns, sqlparser.NewColIdent(indCol)) - } + columns = []sqlparser.ColIdent{sqlparser.NewColIdent(ind.Column)} + } else { + if len(ind.Columns) == 0 { + return fmt.Errorf("must specify at least one column for vindex (%s) and table (%s)", ind.Name, tname) } - columnVindex := &ColumnVindex{ - Columns: columns, - Type: vindexInfo.Type, - Name: ind.Name, - Owned: owned, - Vindex: vindex, + for _, indCol := range ind.Columns { + columns = append(columns, sqlparser.NewColIdent(indCol)) } - if i == 0 { - // Perform Primary vindex check. - if !columnVindex.Vindex.IsUnique() { - ksvschema.Error = fmt.Errorf("primary vindex %s is not Unique for table %s", ind.Name, tname) - continue outer - } - if owned { - ksvschema.Error = fmt.Errorf("primary vindex %s cannot be owned for table %s", ind.Name, tname) - continue outer - } + } + columnVindex := &ColumnVindex{ + Columns: columns, + Type: vindexInfo.Type, + Name: ind.Name, + Owned: owned, + Vindex: vindex, + } + if i == 0 { + // Perform Primary vindex check. + if !columnVindex.Vindex.IsUnique() { + return fmt.Errorf("primary vindex %s is not Unique for table %s", ind.Name, tname) } - t.ColumnVindexes = append(t.ColumnVindexes, columnVindex) if owned { - t.Owned = append(t.Owned, columnVindex) + return fmt.Errorf("primary vindex %s cannot be owned for table %s", ind.Name, tname) } } - t.Ordered = colVindexSorted(t.ColumnVindexes) + t.ColumnVindexes = append(t.ColumnVindexes, columnVindex) + if owned { + t.Owned = append(t.Owned, columnVindex) + } + } + t.Ordered = colVindexSorted(t.ColumnVindexes) + + // Add the table to the map entries. + if _, ok := vschema.uniqueTables[tname]; ok { + vschema.uniqueTables[tname] = nil + } else { + vschema.uniqueTables[tname] = t } + ksvschema.Tables[tname] = t } + return nil } func resolveAutoIncrement(source *vschemapb.SrvVSchema, vschema *VSchema) { @@ -294,7 +322,7 @@ func resolveAutoIncrement(source *vschemapb.SrvVSchema, vschema *VSchema) { ksvschema := vschema.Keyspaces[ksname] for tname, table := range ks.Tables { t := ksvschema.Tables[tname] - if table.AutoIncrement == nil { + if t == nil || table.AutoIncrement == nil { continue } t.AutoIncrement = &AutoIncrement{Column: sqlparser.NewColIdent(table.AutoIncrement.Column)} @@ -332,6 +360,48 @@ func addDual(vschema *VSchema) { } } +func buildRoutingRule(source *vschemapb.SrvVSchema, vschema *VSchema) { + if source.RoutingRules == nil { + return + } +outer: + for _, rule := range source.RoutingRules.Rules { + rr := &RoutingRule{} + for _, toTable := range rule.ToTables { + if _, ok := vschema.RoutingRules[rule.FromTable]; ok { + vschema.RoutingRules[rule.FromTable] = &RoutingRule{ + Error: fmt.Errorf("duplicate rule for entry %s", rule.FromTable), + } + continue outer + } + parts := strings.Split(toTable, ".") + if len(parts) != 2 { + vschema.RoutingRules[rule.FromTable] = &RoutingRule{ + Error: fmt.Errorf("table %s must be qualified", toTable), + } + continue outer + } + t, err := vschema.FindTable(parts[0], parts[1]) + if err != nil { + vschema.RoutingRules[rule.FromTable] = &RoutingRule{ + Error: err, + } + continue outer + } + for _, existing := range rr.Tables { + if existing == t { + vschema.RoutingRules[rule.FromTable] = &RoutingRule{ + Error: fmt.Errorf("table %s specified more than once", toTable), + } + continue outer + } + } + rr.Tables = append(rr.Tables, t) + } + vschema.RoutingRules[rule.FromTable] = rr + } +} + // findQualified finds a table t or k.t. func (vschema *VSchema) findQualified(name string) (*Table, error) { splits := strings.Split(name, ".") @@ -351,6 +421,7 @@ func (vschema *VSchema) findQualified(name string) (*Table, error) { // only if its name is unique across all keyspaces. If there is only one // keyspace in the vschema, and it's unsharded, then all table requests are considered // valid and belonging to that keyspace. +// FindTable bypasses routing rules and returns at most one table. func (vschema *VSchema) FindTable(keyspace, tablename string) (*Table, error) { t, err := vschema.findTable(keyspace, tablename) if err != nil { @@ -397,14 +468,36 @@ func (vschema *VSchema) findTable(keyspace, tablename string) (*Table, error) { return table, nil } -// FindTableOrVindex finds a table or a Vindex by name using Find and FindVindex. -func (vschema *VSchema) FindTableOrVindex(keyspace, name string) (*Table, Vindex, error) { - t, err := vschema.findTable(keyspace, name) +func (vschema *VSchema) findTables(keyspace, tablename string) ([]*Table, error) { + qualified := tablename + if keyspace != "" { + qualified = keyspace + "." + tablename + } + rr, ok := vschema.RoutingRules[qualified] + if ok { + if len(rr.Tables) == 0 { + return nil, fmt.Errorf("table %s has been disabled", tablename) + } + return rr.Tables, nil + } + t, err := vschema.findTable(keyspace, tablename) + if err != nil { + return nil, err + } + if t == nil { + return nil, nil + } + return []*Table{t}, nil +} + +// FindTablesOrVindex finds a table or a Vindex by name using Find and FindVindex. +func (vschema *VSchema) FindTablesOrVindex(keyspace, name string) ([]*Table, Vindex, error) { + tables, err := vschema.findTables(keyspace, name) if err != nil { return nil, nil, err } - if t != nil { - return t, nil, nil + if tables != nil { + return tables, nil, nil } v, err := vschema.FindVindex(keyspace, name) if err != nil { @@ -444,9 +537,7 @@ func (bc ByCost) Swap(i, j int) { bc[i], bc[j] = bc[j], bc[i] } func (bc ByCost) Less(i, j int) bool { return bc[i].Vindex.Cost() < bc[j].Vindex.Cost() } func colVindexSorted(cvs []*ColumnVindex) (sorted []*ColumnVindex) { - for _, cv := range cvs { - sorted = append(sorted, cv) - } + sorted = append(sorted, cvs...) sort.Sort(ByCost(sorted)) return sorted } diff --git a/go/vt/vtgate/vindexes/vschema_test.go b/go/vt/vtgate/vindexes/vschema_test.go index 19606e634a9..8b5caf8eacb 100644 --- a/go/vt/vtgate/vindexes/vschema_test.go +++ b/go/vt/vtgate/vindexes/vschema_test.go @@ -18,6 +18,7 @@ package vindexes import ( "encoding/json" + "errors" "reflect" "strings" "testing" @@ -131,6 +132,7 @@ func TestUnshardedVSchema(t *testing.T) { Keyspace: ks, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -193,6 +195,7 @@ func TestVSchemaColumns(t *testing.T) { Keyspace: ks, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -258,6 +261,7 @@ func TestVSchemaColumnListAuthoritative(t *testing.T) { Keyspace: ks, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -338,6 +342,7 @@ func TestVSchemaPinned(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -441,6 +446,7 @@ func TestShardedVSchemaOwned(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -468,7 +474,158 @@ func TestShardedVSchemaOwned(t *testing.T) { wantjson, _ := json.Marshal(want) t.Errorf("BuildVSchema:\n%s, want\n%s", gotjson, wantjson) } +} +func TestVSchemaRoutingRules(t *testing.T) { + input := vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{ + Rules: []*vschemapb.RoutingRule{{ + FromTable: "rt1", + ToTables: []string{"ks1.t1", "ks2.t2"}, + }, { + FromTable: "rt2", + ToTables: []string{"ks2.t2"}, + }, { + FromTable: "dup", + ToTables: []string{"ks1.t1"}, + }, { + FromTable: "dup", + ToTables: []string{"ks1.t1"}, + }, { + FromTable: "unqualified", + ToTables: []string{"t1"}, + }, { + FromTable: "badkeyspace", + ToTables: []string{"ks3.t1"}, + }, { + FromTable: "notfound", + ToTables: []string{"ks1.t2"}, + }, { + FromTable: "doubletable", + ToTables: []string{"ks1.t1", "ks1.t1"}, + }}, + }, + Keyspaces: map[string]*vschemapb.Keyspace{ + "ks1": { + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "stfu1": { + Type: "stfu", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{ + { + Column: "c1", + Name: "stfu1", + }, + }, + }, + }, + }, + "ks2": { + Tables: map[string]*vschemapb.Table{ + "t2": {}, + }, + }, + }, + } + got, _ := BuildVSchema(&input) + ks1 := &Keyspace{ + Name: "ks1", + Sharded: true, + } + ks2 := &Keyspace{ + Name: "ks2", + } + vindex1 := &stFU{ + name: "stfu1", + } + t1 := &Table{ + Name: sqlparser.NewTableIdent("t1"), + Keyspace: ks1, + ColumnVindexes: []*ColumnVindex{{ + Columns: []sqlparser.ColIdent{sqlparser.NewColIdent("c1")}, + Type: "stfu", + Name: "stfu1", + Vindex: vindex1, + }}, + } + t1.Ordered = []*ColumnVindex{ + t1.ColumnVindexes[0], + } + t2 := &Table{ + Name: sqlparser.NewTableIdent("t2"), + Keyspace: ks2, + } + dual1 := &Table{ + Name: sqlparser.NewTableIdent("dual"), + Keyspace: ks1, + Pinned: []byte{0}, + } + dual2 := &Table{ + Name: sqlparser.NewTableIdent("dual"), + Keyspace: ks2, + } + want := &VSchema{ + RoutingRules: map[string]*RoutingRule{ + "rt1": { + Tables: []*Table{t1, t2}, + }, + "rt2": { + Tables: []*Table{t2}, + }, + "dup": { + Error: errors.New("duplicate rule for entry dup"), + }, + "unqualified": { + Error: errors.New("table t1 must be qualified"), + }, + "badkeyspace": { + Error: errors.New("keyspace ks3 not found in vschema"), + }, + "notfound": { + Error: errors.New("table t2 not found"), + }, + "doubletable": { + Error: errors.New("table ks1.t1 specified more than once"), + }, + }, + uniqueTables: map[string]*Table{ + "t1": t1, + "t2": t2, + "dual": dual1, + }, + uniqueVindexes: map[string]Vindex{ + "stfu1": vindex1, + }, + Keyspaces: map[string]*KeyspaceSchema{ + "ks1": { + Keyspace: ks1, + Tables: map[string]*Table{ + "t1": t1, + "dual": dual1, + }, + Vindexes: map[string]Vindex{ + "stfu1": vindex1, + }, + }, + "ks2": { + Keyspace: ks2, + Tables: map[string]*Table{ + "t2": t2, + "dual": dual2, + }, + Vindexes: map[string]Vindex{}, + }, + }, + } + if !reflect.DeepEqual(got, want) { + gotb, _ := json.Marshal(got) + wantb, _ := json.Marshal(want) + t.Errorf("BuildVSchema:\n%s, want\n%s", gotb, wantb) + } } func TestFindVindexForSharding(t *testing.T) { @@ -650,6 +807,7 @@ func TestShardedVSchemaMultiColumnVindex(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -749,6 +907,7 @@ func TestShardedVSchemaNotOwned(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": t1, "dual": dual, @@ -860,14 +1019,14 @@ func TestBuildVSchemaDupSeq(t *testing.T) { } got, _ := BuildVSchema(&good) t1a := &Table{ - Name: sqlparser.NewTableIdent("t1"), - Keyspace: ksa, - IsSequence: true, + Name: sqlparser.NewTableIdent("t1"), + Keyspace: ksa, + Type: "sequence", } t1b := &Table{ - Name: sqlparser.NewTableIdent("t1"), - Keyspace: ksb, - IsSequence: true, + Name: sqlparser.NewTableIdent("t1"), + Keyspace: ksb, + Type: "sequence", } duala := &Table{ Name: sqlparser.NewTableIdent("dual"), @@ -878,6 +1037,7 @@ func TestBuildVSchemaDupSeq(t *testing.T) { Keyspace: ksb, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": nil, "dual": duala, @@ -948,6 +1108,7 @@ func TestBuildVSchemaDupTable(t *testing.T) { Keyspace: ksb, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": nil, "dual": duala, @@ -1083,6 +1244,7 @@ func TestBuildVSchemaDupVindex(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "t1": nil, "dual": duala, @@ -1342,9 +1504,9 @@ func TestSequence(t *testing.T) { Sharded: true, } seq := &Table{ - Name: sqlparser.NewTableIdent("seq"), - Keyspace: ksu, - IsSequence: true, + Name: sqlparser.NewTableIdent("seq"), + Keyspace: ksu, + Type: "sequence", } vindex1 := &stFU{ name: "stfu1", @@ -1400,6 +1562,7 @@ func TestSequence(t *testing.T) { Pinned: []byte{0}, } want := &VSchema{ + RoutingRules: map[string]*RoutingRule{}, uniqueTables: map[string]*Table{ "seq": seq, "t1": t1, @@ -1542,6 +1705,27 @@ func TestBadSequenceName(t *testing.T) { } } +func TestBadShardedSequence(t *testing.T) { + bad := vschemapb.SrvVSchema{ + Keyspaces: map[string]*vschemapb.Keyspace{ + "sharded": { + Sharded: true, + Tables: map[string]*vschemapb.Table{ + "t1": { + Type: "sequence", + }, + }, + }, + }, + } + got, _ := BuildVSchema(&bad) + err := got.Keyspaces["sharded"].Error + want := "sequence table has to be in an unsharded keyspace or must be pinned: t1" + if err == nil || err.Error() != want { + t.Errorf("BuildVSchema: %v, want %v", err, want) + } +} + func TestFindTable(t *testing.T) { input := vschemapb.SrvVSchema{ Keyspaces: map[string]*vschemapb.Keyspace{ @@ -1607,7 +1791,7 @@ func TestFindTable(t *testing.T) { if !reflect.DeepEqual(got, ta) { t.Errorf("FindTable(\"t1a\"): %+v, want %+v", got, ta) } - got, err = vschema.FindTable("ksa", "ta") + got, _ = vschema.FindTable("ksa", "ta") if !reflect.DeepEqual(got, ta) { t.Errorf("FindTable(\"t1a\"): %+v, want %+v", got, ta) } @@ -1617,7 +1801,7 @@ func TestFindTable(t *testing.T) { Name: "ksa", }, } - got, err = vschema.FindTable("ksa", "none") + got, _ = vschema.FindTable("ksa", "none") if !reflect.DeepEqual(got, none) { t.Errorf("FindTable(\"t1a\"): %+v, want %+v", got, none) } @@ -1633,8 +1817,20 @@ func TestFindTable(t *testing.T) { } } -func TestFindTableOrVindex(t *testing.T) { +func TestFindTablesOrVindex(t *testing.T) { input := vschemapb.SrvVSchema{ + RoutingRules: &vschemapb.RoutingRules{ + Rules: []*vschemapb.RoutingRule{{ + FromTable: "unqualified", + ToTables: []string{"ksa.ta", "ksb.t1"}, + }, { + FromTable: "newks.qualified", + ToTables: []string{"ksa.ta"}, + }, { + FromTable: "notarget", + ToTables: []string{}, + }}, + }, Keyspaces: map[string]*vschemapb.Keyspace{ "ksa": { Tables: map[string]*vschemapb.Table{ @@ -1678,34 +1874,30 @@ func TestFindTableOrVindex(t *testing.T) { }, } vschema, _ := BuildVSchema(&input) + ta := vschema.Keyspaces["ksa"].Tables["ta"] + t1 := vschema.Keyspaces["ksb"].Tables["t1"] - _, _, err := vschema.FindTableOrVindex("", "t1") + _, _, err := vschema.FindTablesOrVindex("", "t1") wantErr := "ambiguous table reference: t1" if err == nil || err.Error() != wantErr { - t.Errorf("FindTableOrVindex(\"\"): %v, want %s", err, wantErr) + t.Errorf("FindTablesOrVindex(\"\"): %v, want %s", err, wantErr) } - _, _, err = vschema.FindTableOrVindex("", "none") + _, _, err = vschema.FindTablesOrVindex("", "none") wantErr = "table none not found" if err == nil || err.Error() != wantErr { - t.Errorf("FindTableOrVindex(\"\"): %v, want %s", err, wantErr) + t.Errorf("FindTablesOrVindex(\"\"): %v, want %s", err, wantErr) } - got, _, err := vschema.FindTableOrVindex("", "ta") + got, _, err := vschema.FindTablesOrVindex("", "ta") if err != nil { t.Fatal(err) } - ta := &Table{ - Name: sqlparser.NewTableIdent("ta"), - Keyspace: &Keyspace{ - Name: "ksa", - }, - } - if !reflect.DeepEqual(got, ta) { - t.Errorf("FindTableOrVindex(\"t1a\"): %+v, want %+v", got, ta) + if !reflect.DeepEqual(got, []*Table{ta}) { + t.Errorf("FindTablesOrVindex(\"t1a\"): %+v, want %+v", got, ta) } - _, vindex, err := vschema.FindTableOrVindex("", "stfu1") + _, vindex, err := vschema.FindTablesOrVindex("", "stfu1") if err != nil { t.Fatal(err) } @@ -1713,10 +1905,10 @@ func TestFindTableOrVindex(t *testing.T) { name: "stfu1", } if !reflect.DeepEqual(vindex, wantVindex) { - t.Errorf("FindTableOrVindex(\"stfu1\"): %+v, want %+v", vindex, wantVindex) + t.Errorf("FindTablesOrVindex(\"stfu1\"): %+v, want %+v", vindex, wantVindex) } - _, vindex, err = vschema.FindTableOrVindex("ksc", "ta") + _, vindex, err = vschema.FindTablesOrVindex("ksc", "ta") if err != nil { t.Fatal(err) } @@ -1724,13 +1916,35 @@ func TestFindTableOrVindex(t *testing.T) { name: "ta", } if !reflect.DeepEqual(vindex, wantVindex) { - t.Errorf("FindTableOrVindex(\"stfu1\"): %+v, want %+v", vindex, wantVindex) + t.Errorf("FindTablesOrVindex(\"stfu1\"): %+v, want %+v", vindex, wantVindex) } - _, _, err = vschema.FindTableOrVindex("", "dup") + _, _, err = vschema.FindTablesOrVindex("", "dup") wantErr = "ambiguous vindex reference: dup" if err == nil || err.Error() != wantErr { - t.Errorf("FindTableOrVindex(\"\"): %v, want %s", err, wantErr) + t.Errorf("FindTablesOrVindex(\"\"): %v, want %s", err, wantErr) + } + + got, _, err = vschema.FindTablesOrVindex("", "unqualified") + if err != nil { + t.Fatal(err) + } + if want := []*Table{ta, t1}; !reflect.DeepEqual(got, want) { + t.Errorf("FindTablesOrVindex(unqualified): %+v, want %+v", got, want) + } + + got, _, err = vschema.FindTablesOrVindex("newks", "qualified") + if err != nil { + t.Fatal(err) + } + if want := []*Table{ta}; !reflect.DeepEqual(got, want) { + t.Errorf("FindTablesOrVindex(unqualified): %+v, want %+v", got, want) + } + + _, _, err = vschema.FindTablesOrVindex("", "notarget") + wantErr = "table notarget has been disabled" + if err == nil || err.Error() != wantErr { + t.Errorf("FindTablesOrVindex(\"\"): %v, want %s", err, wantErr) } } @@ -1897,8 +2111,8 @@ func TestVSchemaJSON(t *testing.T) { }}, }, "t2": { - IsSequence: true, - Name: sqlparser.NewTableIdent("n2"), + Type: "sequence", + Name: sqlparser.NewTableIdent("n2"), }, }, }, @@ -1968,7 +2182,7 @@ func TestVSchemaJSON(t *testing.T) { ] }, "t2": { - "is_sequence": true, + "type": "sequence", "name": "n2" } } diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 2b2e7b81d8c..e17f70374b0 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -59,7 +59,6 @@ var ( streamBufferSize = flag.Int("stream_buffer_size", 32*1024, "the number of bytes sent from vtgate for each stream call. It's recommended to keep this value in sync with vttablet's query-server-config-stream-buffer-size.") queryPlanCacheSize = flag.Int64("gate_query_cache_size", 10000, "gate server query cache size, maximum number of queries to be cached. vtgate 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.") legacyAutocommit = flag.Bool("legacy_autocommit", false, "DEPRECATED: set this flag to true to get the legacy behavior: all transactions will need an explicit begin, and DMLs outside transactions will return an error.") - enableForwarding = flag.Bool("enable_forwarding", false, "if specified, this process will also expose a QueryService interface that allows other vtgates to talk through this vtgate to the underlying tablets.") disableLocalGateway = flag.Bool("disable_local_gateway", false, "if specified, this process will not route any queries to local tablets in the local cell") ) diff --git a/go/vt/vtgate/vtgate_test.go b/go/vt/vtgate/vtgate_test.go index 5f11adadb4e..8a80936666b 100644 --- a/go/vt/vtgate/vtgate_test.go +++ b/go/vt/vtgate/vtgate_test.go @@ -63,18 +63,18 @@ func init() { } ` getSandbox(KsTestBadVSchema).VSchema = ` - { +{ "sharded": true, "tables": { "t2": { - "auto_increment": { - "column": "id", - "sequence": "id_seq" - } + "auto_increment": { + "column": "id", + "sequence": "id_seq" + } } - } } - ` +} +` hcVTGateTest = discovery.NewFakeHealthCheck() *transactionMode = "MULTI" // The topo.Server is used to start watching the cells described diff --git a/go/vt/vtgate/vtgateconn/vtgateconn_test.go b/go/vt/vtgate/vtgateconn/vtgateconn_test.go index 0c3489fedc1..a8eb7b34a12 100644 --- a/go/vt/vtgate/vtgateconn/vtgateconn_test.go +++ b/go/vt/vtgate/vtgateconn/vtgateconn_test.go @@ -32,14 +32,14 @@ func TestRegisterDialer(t *testing.T) { func TestGetDialerWithProtocol(t *testing.T) { protocol := "test2" - c, err := DialProtocol(context.Background(), protocol, "") + _, err := DialProtocol(context.Background(), protocol, "") if err == nil || err.Error() != "no dialer registered for VTGate protocol "+protocol { t.Fatalf("protocol: %s is not registered, should return error: %v", protocol, err) } RegisterDialer(protocol, func(context.Context, string) (Impl, error) { return nil, nil }) - c, err = DialProtocol(context.Background(), protocol, "") + c, err := DialProtocol(context.Background(), protocol, "") if err != nil || c == nil { t.Fatalf("dialerFunc has been registered, should not get nil: %v %v", err, c) } diff --git a/go/vt/vtgate/vtgateconntest/client.go b/go/vt/vtgate/vtgateconntest/client.go index 17f117ce9ab..98cfabc479b 100644 --- a/go/vt/vtgate/vtgateconntest/client.go +++ b/go/vt/vtgate/vtgateconntest/client.go @@ -53,8 +53,10 @@ type fakeVTGateService struct { errorWait chan struct{} } -const expectedErrMatch string = "test vtgate error" -const expectedCode vtrpcpb.Code = vtrpcpb.Code_INVALID_ARGUMENT +const ( + expectedErrMatch string = "test vtgate error" + expectedCode vtrpcpb.Code = vtrpcpb.Code_INVALID_ARGUMENT +) var errTestVtGateError = vterrors.New(expectedCode, expectedErrMatch) @@ -1452,7 +1454,7 @@ func testStreamExecuteError(t *testing.T, session *vtgateconn.VTGateSession, fak // signal to the server that the first result has been received close(fake.errorWait) // After 1 result, we expect to get an error (no more results). - qr, err = stream.Recv() + _, err = stream.Recv() if err == nil { t.Fatalf("StreamExecute channel wasn't closed") } @@ -1533,7 +1535,7 @@ func testStreamExecuteShardsError(t *testing.T, conn *vtgateconn.VTGateConn, fak // signal to the server that the first result has been received close(fake.errorWait) // After 1 result, we expect to get an error (no more results). - qr, err = stream.Recv() + _, err = stream.Recv() if err == nil { t.Fatalf("StreamExecuteShards channel wasn't closed") } @@ -1614,7 +1616,7 @@ func testStreamExecuteKeyRangesError(t *testing.T, conn *vtgateconn.VTGateConn, // signal to the server that the first result has been received close(fake.errorWait) // After 1 result, we expect to get an error (no more results). - qr, err = stream.Recv() + _, err = stream.Recv() if err == nil { t.Fatalf("StreamExecuteKeyRanges channel wasn't closed") } @@ -1695,7 +1697,7 @@ func testStreamExecuteKeyspaceIdsError(t *testing.T, conn *vtgateconn.VTGateConn // signal to the server that the first result has been received close(fake.errorWait) // After 1 result, we expect to get an error (no more results). - qr, err = stream.Recv() + _, err = stream.Recv() if err == nil { t.Fatalf("StreamExecuteKeyspaceIds channel wasn't closed") } @@ -2166,7 +2168,7 @@ func testUpdateStreamError(t *testing.T, conn *vtgateconn.VTGateConn, fake *fake // signal to the server that the first result has been received close(fake.errorWait) // After 1 result, we expect to get an error (no more results). - qr, _, err = stream.Recv() + _, _, err = stream.Recv() if err == nil { t.Fatalf("UpdateStream channel wasn't closed") } diff --git a/go/vt/vttablet/endtoend/misc_test.go b/go/vt/vttablet/endtoend/misc_test.go index 4544ac0bc31..2f12bf0a063 100644 --- a/go/vt/vttablet/endtoend/misc_test.go +++ b/go/vt/vttablet/endtoend/misc_test.go @@ -29,17 +29,15 @@ import ( "github.com/golang/protobuf/proto" "golang.org/x/net/context" - "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/callerid" "vitess.io/vitess/go/vt/log" + querypb "vitess.io/vitess/go/vt/proto/query" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vttablet/endtoend/framework" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" - - querypb "vitess.io/vitess/go/vt/proto/query" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) func TestSimpleRead(t *testing.T) { @@ -453,7 +451,7 @@ func TestHealth(t *testing.T) { func TestStreamHealth(t *testing.T) { var health *querypb.StreamHealthResponse - framework.Server.BroadcastHealth(0, nil) + framework.Server.BroadcastHealth(0, nil, time.Minute) if err := framework.Server.StreamHealth(context.Background(), func(shr *querypb.StreamHealthResponse) error { health = shr return io.EOF @@ -465,6 +463,23 @@ func TestStreamHealth(t *testing.T) { } } +func TestStreamHealth_Expired(t *testing.T) { + var health *querypb.StreamHealthResponse + framework.Server.BroadcastHealth(0, nil, time.Millisecond) + time.Sleep(5 * time.Millisecond) + ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond * 100) + defer cancel() + if err := framework.Server.StreamHealth(ctx, func(shr *querypb.StreamHealthResponse) error { + health = shr + return io.EOF + }); err != nil { + t.Fatal(err) + } + if health != nil { + t.Errorf("Health: %v, want %v", health, nil) + } +} + func TestQueryStats(t *testing.T) { client := framework.NewClient() vstart := framework.DebugVars() diff --git a/go/vt/vttablet/grpcqueryservice/server.go b/go/vt/vttablet/grpcqueryservice/server.go index 36a776ae540..d0f55351516 100644 --- a/go/vt/vttablet/grpcqueryservice/server.go +++ b/go/vt/vttablet/grpcqueryservice/server.go @@ -76,14 +76,12 @@ func (q *query) StreamExecute(request *querypb.StreamExecuteRequest, stream quer request.EffectiveCallerId, request.ImmediateCallerId, ) - if err := q.server.StreamExecute(ctx, request.Target, request.Query.Sql, request.Query.BindVariables, request.TransactionId, request.Options, func(reply *sqltypes.Result) error { + err = q.server.StreamExecute(ctx, request.Target, request.Query.Sql, request.Query.BindVariables, request.TransactionId, request.Options, func(reply *sqltypes.Result) error { return stream.Send(&querypb.StreamExecuteResponse{ Result: sqltypes.ResultToProto3(reply), }) - }); err != nil { - return vterrors.ToGRPC(err) - } - return nil + }) + return vterrors.ToGRPC(err) } // Begin is part of the queryservice.QueryServer interface @@ -300,14 +298,12 @@ func (q *query) MessageStream(request *querypb.MessageStreamRequest, stream quer request.EffectiveCallerId, request.ImmediateCallerId, ) - if err := q.server.MessageStream(ctx, request.Target, request.Name, func(qr *sqltypes.Result) error { + err = q.server.MessageStream(ctx, request.Target, request.Name, func(qr *sqltypes.Result) error { return stream.Send(&querypb.MessageStreamResponse{ Result: sqltypes.ResultToProto3(qr), }) - }); err != nil { - return vterrors.ToGRPC(err) - } - return nil + }) + return vterrors.ToGRPC(err) } // MessageAck is part of the queryservice.QueryServer interface @@ -352,10 +348,8 @@ func (q *query) SplitQuery(ctx context.Context, request *querypb.SplitQueryReque // StreamHealth is part of the queryservice.QueryServer interface func (q *query) StreamHealth(request *querypb.StreamHealthRequest, stream queryservicepb.Query_StreamHealthServer) (err error) { defer q.server.HandlePanic(&err) - if err = q.server.StreamHealth(stream.Context(), stream.Send); err != nil { - return vterrors.ToGRPC(err) - } - return nil + err = q.server.StreamHealth(stream.Context(), stream.Send) + return vterrors.ToGRPC(err) } // UpdateStream is part of the queryservice.QueryServer interface @@ -365,14 +359,12 @@ func (q *query) UpdateStream(request *querypb.UpdateStreamRequest, stream querys request.EffectiveCallerId, request.ImmediateCallerId, ) - if err := q.server.UpdateStream(ctx, request.Target, request.Position, request.Timestamp, func(reply *querypb.StreamEvent) error { + err = q.server.UpdateStream(ctx, request.Target, request.Position, request.Timestamp, func(reply *querypb.StreamEvent) error { return stream.Send(&querypb.UpdateStreamResponse{ Event: reply, }) - }); err != nil { - return vterrors.ToGRPC(err) - } - return nil + }) + return vterrors.ToGRPC(err) } // VStream is part of the queryservice.QueryServer interface @@ -382,14 +374,23 @@ func (q *query) VStream(request *binlogdatapb.VStreamRequest, stream queryservic request.EffectiveCallerId, request.ImmediateCallerId, ) - if err := q.server.VStream(ctx, request.Target, request.Position, request.Filter, func(events []*binlogdatapb.VEvent) error { + err = q.server.VStream(ctx, request.Target, request.Position, request.Filter, func(events []*binlogdatapb.VEvent) error { return stream.Send(&binlogdatapb.VStreamResponse{ Events: events, }) - }); err != nil { - return vterrors.ToGRPC(err) - } - return nil + }) + return vterrors.ToGRPC(err) +} + +// VStreamRows is part of the queryservice.QueryServer interface +func (q *query) VStreamRows(request *binlogdatapb.VStreamRowsRequest, stream queryservicepb.Query_VStreamRowsServer) (err error) { + defer q.server.HandlePanic(&err) + ctx := callerid.NewContext(callinfo.GRPCCallInfo(stream.Context()), + request.EffectiveCallerId, + request.ImmediateCallerId, + ) + err = q.server.VStreamRows(ctx, request.Target, request.Query, request.Lastpk, stream.Send) + return vterrors.ToGRPC(err) } // Register registers the implementation on the provide gRPC Server. diff --git a/go/vt/vttablet/grpctabletconn/conn.go b/go/vt/vttablet/grpctabletconn/conn.go index 56667e72513..aae4da11ac8 100644 --- a/go/vt/vttablet/grpctabletconn/conn.go +++ b/go/vt/vttablet/grpctabletconn/conn.go @@ -714,6 +714,47 @@ func (conn *gRPCQueryClient) VStream(ctx context.Context, target *querypb.Target } } +// VStreamRows streams rows of a query from the specified starting point. +func (conn *gRPCQueryClient) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + stream, err := func() (queryservicepb.Query_VStreamRowsClient, error) { + conn.mu.RLock() + defer conn.mu.RUnlock() + if conn.cc == nil { + return nil, tabletconn.ConnClosed + } + + req := &binlogdatapb.VStreamRowsRequest{ + Target: target, + EffectiveCallerId: callerid.EffectiveCallerIDFromContext(ctx), + ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx), + Query: query, + Lastpk: lastpk, + } + stream, err := conn.c.VStreamRows(ctx, req) + if err != nil { + return nil, tabletconn.ErrorFromGRPC(err) + } + return stream, nil + }() + if err != nil { + return err + } + for { + r, err := stream.Recv() + if err != nil { + return tabletconn.ErrorFromGRPC(err) + } + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + if err := send(r); err != nil { + return err + } + } +} + // HandlePanic is a no-op. func (conn *gRPCQueryClient) HandlePanic(err *error) { } diff --git a/go/vt/vttablet/queryservice/queryservice.go b/go/vt/vttablet/queryservice/queryservice.go index 52153a2941b..7dbdd8a59cb 100644 --- a/go/vt/vttablet/queryservice/queryservice.go +++ b/go/vt/vttablet/queryservice/queryservice.go @@ -103,6 +103,9 @@ type QueryService interface { // VStream streams VReplication events based on the specified filter. VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + // VStreamRows streams rows of a table from the specified starting point. + VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error + // StreamHealth streams health status. StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error diff --git a/go/vt/vttablet/queryservice/wrapped.go b/go/vt/vttablet/queryservice/wrapped.go index 9d25d42dbd0..2d9a434314c 100644 --- a/go/vt/vttablet/queryservice/wrapped.go +++ b/go/vt/vttablet/queryservice/wrapped.go @@ -252,9 +252,16 @@ func (ws *wrappedService) UpdateStream(ctx context.Context, target *querypb.Targ } func (ws *wrappedService) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - return ws.wrapper(ctx, target, ws.impl, "UpdateStream", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { + return ws.wrapper(ctx, target, ws.impl, "VStream", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { innerErr := conn.VStream(ctx, target, startPos, filter, send) - return canRetry(ctx, innerErr), innerErr + return false, innerErr + }) +} + +func (ws *wrappedService) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + return ws.wrapper(ctx, target, ws.impl, "VStreamRows", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { + innerErr := conn.VStreamRows(ctx, target, query, lastpk, send) + return false, innerErr }) } diff --git a/go/vt/vttablet/sandboxconn/sandboxconn.go b/go/vt/vttablet/sandboxconn/sandboxconn.go index 36caa6ee668..3504d8f2218 100644 --- a/go/vt/vttablet/sandboxconn/sandboxconn.go +++ b/go/vt/vttablet/sandboxconn/sandboxconn.go @@ -350,17 +350,22 @@ func (sbc *SandboxConn) SplitQuery( // StreamHealth is not implemented. func (sbc *SandboxConn) StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error { - return fmt.Errorf("Not implemented in test") + return fmt.Errorf("not implemented in test") } // UpdateStream is part of the QueryService interface. func (sbc *SandboxConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, callback func(*querypb.StreamEvent) error) error { - return fmt.Errorf("Not implemented in test") + return fmt.Errorf("not implemented in test") } // VStream is part of the QueryService interface. func (sbc *SandboxConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - return fmt.Errorf("Not implemented in test") + return fmt.Errorf("not implemented in test") +} + +// VStreamRows is part of the QueryService interface. +func (sbc *SandboxConn) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + return fmt.Errorf("not implemented in test") } // HandlePanic is part of the QueryService interface. diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go index 51ae53f8b33..1d01be25e4c 100644 --- a/go/vt/vttablet/tabletconntest/fakequeryservice.go +++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go @@ -782,11 +782,13 @@ func (f *FakeQueryService) StreamHealth(ctx context.Context, callback func(*quer return nil } -// UpdateStreamPosition is a test update stream position. -const UpdateStreamPosition = "update stream position" +const ( + // UpdateStreamPosition is a test update stream position. + UpdateStreamPosition = "update stream position" -// UpdateStreamTimestamp is a test update stream timestamp. -const UpdateStreamTimestamp = 123654 + // UpdateStreamTimestamp is a test update stream timestamp. + UpdateStreamTimestamp = 123654 +) // UpdateStreamStreamEvent1 is a test update stream event. var UpdateStreamStreamEvent1 = querypb.StreamEvent{ @@ -856,6 +858,11 @@ func (f *FakeQueryService) VStream(ctx context.Context, target *querypb.Target, panic("not implemented") } +// VStreamRows is part of the QueryService interface. +func (f *FakeQueryService) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + panic("not implemented") +} + // CreateFakeServer returns the fake server for the tests func CreateFakeServer(t *testing.T) *FakeQueryService { return &FakeQueryService{ diff --git a/go/vt/vttablet/tabletmanager/action_agent.go b/go/vt/vttablet/tabletmanager/action_agent.go index e55377e7993..98072227b75 100644 --- a/go/vt/vttablet/tabletmanager/action_agent.go +++ b/go/vt/vttablet/tabletmanager/action_agent.go @@ -263,13 +263,6 @@ func NewActionAgent( agent.statsTabletType = stats.NewString("TabletType") agent.statsTabletTypeCount = stats.NewCountersWithSingleLabel("TabletTypeCount", "Number of times the tablet changed to the labeled type", "type") - // The db name will get set by the Start function called below, before - // VREngine gets to invoke the FilteredWithDB call. - agent.VREngine = vreplication.NewEngine(ts, tabletAlias.Cell, mysqld, func() binlogplayer.DBClient { - return binlogplayer.NewDBClient(agent.DBConfigs.FilteredWithDB()) - }) - servenv.OnTerm(agent.VREngine.Close) - var mysqlHost string var mysqlPort int32 if appConfig := dbcfgs.AppWithDB(); appConfig.Host != "" { @@ -289,6 +282,12 @@ func NewActionAgent( return nil, err } + // The db name is set by the Start function called above + agent.VREngine = vreplication.NewEngine(ts, tabletAlias.Cell, mysqld, func() binlogplayer.DBClient { + return binlogplayer.NewDBClient(agent.DBConfigs.FilteredWithDB()) + }, agent.DBConfigs.FilteredWithDB().DbName) + servenv.OnTerm(agent.VREngine.Close) + // Run a background task to rebuild the SrvKeyspace in our cell/keyspace // if it doesn't exist yet. go agent.maybeRebuildKeyspace(agent.initialTablet.Alias.Cell, agent.initialTablet.Keyspace) @@ -340,6 +339,10 @@ func NewActionAgent( // NewTestActionAgent creates an agent for test purposes. Only a // subset of features are supported now, but we'll add more over time. func NewTestActionAgent(batchCtx context.Context, ts *topo.Server, tabletAlias *topodatapb.TabletAlias, vtPort, grpcPort int32, mysqlDaemon mysqlctl.MysqlDaemon, preStart func(*ActionAgent)) *ActionAgent { + ti, err := ts.GetTablet(batchCtx, tabletAlias) + if err != nil { + panic(vterrors.Wrap(err, "failed reading tablet")) + } agent := &ActionAgent{ QueryServiceControl: tabletservermock.NewController(), UpdateStream: binlog.NewUpdateStreamControlMock(), @@ -350,7 +353,7 @@ func NewTestActionAgent(batchCtx context.Context, ts *topo.Server, tabletAlias * Cnf: nil, MysqlDaemon: mysqlDaemon, DBConfigs: &dbconfigs.DBConfigs{}, - VREngine: vreplication.NewEngine(ts, tabletAlias.Cell, mysqlDaemon, binlogplayer.NewFakeDBClient), + VREngine: vreplication.NewEngine(ts, tabletAlias.Cell, mysqlDaemon, binlogplayer.NewFakeDBClient, ti.DbName()), History: history.New(historyLength), _healthy: fmt.Errorf("healthcheck not run yet"), } @@ -389,7 +392,7 @@ func NewComboActionAgent(batchCtx context.Context, ts *topo.Server, tabletAlias Cnf: nil, MysqlDaemon: mysqlDaemon, DBConfigs: dbcfgs, - VREngine: vreplication.NewEngine(nil, "", nil, nil), + VREngine: vreplication.NewEngine(nil, "", nil, nil, ""), gotMysqlPort: true, History: history.New(historyLength), _healthy: fmt.Errorf("healthcheck not run yet"), diff --git a/go/vt/vttablet/tabletmanager/init_tablet.go b/go/vt/vttablet/tabletmanager/init_tablet.go index ba4ed6e8a53..47e5c00917f 100644 --- a/go/vt/vttablet/tabletmanager/init_tablet.go +++ b/go/vt/vttablet/tabletmanager/init_tablet.go @@ -131,7 +131,8 @@ func (agent *ActionAgent) InitTablet(port, gRPCPort int32) error { case err == nil: // NOOP case topo.IsErrType(err, topo.NoNode): - err = topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), agent.TopoServer, *initKeyspace, []string{agent.TabletAlias.Cell}) + // try to RebuildKeyspace here but ignore errors if it fails + topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), agent.TopoServer, *initKeyspace, []string{agent.TabletAlias.Cell}) default: return vterrors.Wrap(err, "InitTablet failed to read srvKeyspace") } @@ -209,7 +210,7 @@ func (agent *ActionAgent) InitTablet(port, gRPCPort int32) error { if *initPopulateMetadata { agent.setTablet(tablet) localMetadata := agent.getLocalMetadataValues(tablet.Type) - err := mysqlctl.PopulateMetadataTables(agent.MysqlDaemon, localMetadata) + err := mysqlctl.PopulateMetadataTables(agent.MysqlDaemon, localMetadata, topoproto.TabletDbName(tablet)) if err != nil { return vterrors.Wrap(err, "failed to -init_populate_metadata") } diff --git a/go/vt/vttablet/tabletmanager/init_tablet_test.go b/go/vt/vttablet/tabletmanager/init_tablet_test.go index 27c057c9680..b19db6af3a1 100644 --- a/go/vt/vttablet/tabletmanager/init_tablet_test.go +++ b/go/vt/vttablet/tabletmanager/init_tablet_test.go @@ -173,7 +173,7 @@ func TestInitTablet(t *testing.T) { } db := fakesqldb.New(t) defer db.Close() - db.AddQueryPattern(`(SET|CREATE|BEGIN|INSERT|COMMIT)\b.*`, &sqltypes.Result{}) + db.AddQueryPattern(`(SET|CREATE|BEGIN|INSERT|COMMIT|ALTER|UPDATE)\b.*`, &sqltypes.Result{}) /* db.AddQuery("SET @@session.sql_log_bin = 0", &sqltypes.Result{}) db.AddQuery("CREATE DATABASE IF NOT EXISTS _vt", &sqltypes.Result{}) @@ -194,7 +194,7 @@ func TestInitTablet(t *testing.T) { TabletAlias: tabletAlias, MysqlDaemon: mysqlDaemon, DBConfigs: &dbconfigs.DBConfigs{}, - VREngine: vreplication.NewEngine(nil, "", nil, nil), + VREngine: vreplication.NewEngine(nil, "", nil, nil, ""), batchCtx: ctx, History: history.New(historyLength), _healthy: fmt.Errorf("healthcheck not run yet"), diff --git a/go/vt/vttablet/tabletmanager/orchestrator.go b/go/vt/vttablet/tabletmanager/orchestrator.go index 95e37d2ccde..66d8222d221 100644 --- a/go/vt/vttablet/tabletmanager/orchestrator.go +++ b/go/vt/vttablet/tabletmanager/orchestrator.go @@ -139,13 +139,13 @@ func (orc *orcClient) InActiveShardRecovery(tablet *topodatapb.Tablet) (bool, er resp, err := orc.apiGet("audit-recovery", "alias", alias) if err != nil { - return false, err + return false, fmt.Errorf("error calling Orchestrator API: %v", err) } var r []map[string]interface{} if err := json.Unmarshal(resp, &r); err != nil { - return false, err + return false, fmt.Errorf("error parsing JSON response from Orchestrator: %v; response: %q", err, string(resp)) } // Orchestrator returns a 0-length response when it has no history of recovery on this cluster. diff --git a/go/vt/vttablet/tabletmanager/rpc_backup.go b/go/vt/vttablet/tabletmanager/rpc_backup.go index fd9b4480979..f9a3d78fe3c 100644 --- a/go/vt/vttablet/tabletmanager/rpc_backup.go +++ b/go/vt/vttablet/tabletmanager/rpc_backup.go @@ -25,6 +25,7 @@ import ( "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vterrors" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -58,16 +59,22 @@ func (agent *ActionAgent) Backup(ctx context.Context, concurrency int, logger lo } originalType := tablet.Type - // update our type to BACKUP - if _, err := topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, topodatapb.TabletType_BACKUP); err != nil { - return err + engine, err := mysqlctl.GetBackupEngine() + if err != nil { + return vterrors.Wrap(err, "failed to find backup engine") } + builtin, _ := engine.(*mysqlctl.BuiltinBackupEngine) + if builtin != nil { + // update our type to BACKUP + if _, err := topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, topodatapb.TabletType_BACKUP); err != nil { + return err + } - // let's update our internal state (stop query service and other things) - if err := agent.refreshTablet(ctx, "before backup"); err != nil { - return err + // let's update our internal state (stop query service and other things) + if err := agent.refreshTablet(ctx, "before backup"); err != nil { + return err + } } - // create the loggers: tee to console and source l := logutil.NewTeeLogger(logutil.NewConsoleLogger(), logger) @@ -76,22 +83,29 @@ func (agent *ActionAgent) Backup(ctx context.Context, concurrency int, logger lo name := fmt.Sprintf("%v.%v", time.Now().UTC().Format("2006-01-02.150405"), topoproto.TabletAliasString(tablet.Alias)) returnErr := mysqlctl.Backup(ctx, agent.Cnf, agent.MysqlDaemon, l, dir, name, concurrency, agent.hookExtraEnv()) - // change our type back to the original value - _, err = topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, originalType) - if err != nil { - // failure in changing the topology type is probably worse, - // so returning that (we logged the snapshot error anyway) - if returnErr != nil { - l.Errorf("mysql backup command returned error: %v", returnErr) + if builtin != nil { + + bgCtx := context.Background() + // Starting from here we won't be able to recover if we get stopped by a cancelled + // context. It is also possible that the context already timed out during the + // above call to Backup. Thus we use the background context to get through to the finish. + + // change our type back to the original value + _, err = topotools.ChangeType(bgCtx, agent.TopoServer, tablet.Alias, originalType) + if err != nil { + // failure in changing the topology type is probably worse, + // so returning that (we logged the snapshot error anyway) + if returnErr != nil { + l.Errorf("mysql backup command returned error: %v", returnErr) + } + returnErr = err } - returnErr = err - } - // let's update our internal state (start query service and other things) - if err := agent.refreshTablet(ctx, "after backup"); err != nil { - return err + // let's update our internal state (start query service and other things) + if err := agent.refreshTablet(bgCtx, "after backup"); err != nil { + return err + } } - // and re-run health check to be sure to capture any replication delay agent.runHealthCheckLocked() diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go index 802d6141e1e..49413f44b16 100644 --- a/go/vt/vttablet/tabletmanager/rpc_replication.go +++ b/go/vt/vttablet/tabletmanager/rpc_replication.go @@ -37,7 +37,8 @@ import ( ) var ( - enableSemiSync = flag.Bool("enable_semi_sync", false, "Enable semi-sync when configuring replication, on master and replica tablets only (rdonly tablets will not ack).") + enableSemiSync = flag.Bool("enable_semi_sync", false, "Enable semi-sync when configuring replication, on master and replica tablets only (rdonly tablets will not ack).") + setSuperReadOnly = flag.Bool("use_super_read_only", false, "Set super_read_only flag when performing planned failover.") ) // SlaveStatus returns the replication status @@ -331,17 +332,29 @@ func (agent *ActionAgent) DemoteMaster(ctx context.Context) (string, error) { // Now, set the server read-only. Note all active connections are not // affected. - if err := agent.MysqlDaemon.SetReadOnly(true); err != nil { - // if this failed, revert the change to serving - if _ /* state changed */, err1 := agent.QueryServiceControl.SetServingType(tablet.Type, true, nil); err1 != nil { - log.Warningf("SetServingType(serving=true) failed after failed SetReadOnly %v", err1) + if *setSuperReadOnly { + // Setting super_read_only also sets read_only + if err := agent.MysqlDaemon.SetSuperReadOnly(true); err != nil { + // if this failed, revert the change to serving + if _ /* state changed */, err1 := agent.QueryServiceControl.SetServingType(tablet.Type, true, nil); err1 != nil { + log.Warningf("SetServingType(serving=true) failed after failed SetSuperReadOnly %v", err1) + } + return "", err + } + } else { + if err := agent.MysqlDaemon.SetReadOnly(true); err != nil { + // if this failed, revert the change to serving + if _ /* state changed */, err1 := agent.QueryServiceControl.SetServingType(tablet.Type, true, nil); err1 != nil { + log.Warningf("SetServingType(serving=true) failed after failed SetReadOnly %v", err1) + } + return "", err } - return "", err } // If using semi-sync, we need to disable master-side. if err := agent.fixSemiSync(topodatapb.TabletType_REPLICA); err != nil { // if this failed, set server read-only back to false, set tablet back to serving + // setting read_only OFF will also set super_read_only OFF if it was set if err1 := agent.MysqlDaemon.SetReadOnly(false); err1 != nil { log.Warningf("SetReadOnly(false) failed after failed fixSemiSync %v", err1) } @@ -355,6 +368,7 @@ func (agent *ActionAgent) DemoteMaster(ctx context.Context) (string, error) { if err != nil { // if DemoteMaster failed, undo all the steps before // 1. set server back to read-only false + // setting read_only OFF will also set super_read_only OFF if it was set if err1 := agent.MysqlDaemon.SetReadOnly(false); err1 != nil { log.Warningf("SetReadOnly(false) failed after failed DemoteMaster %v", err1) } diff --git a/go/vt/vttablet/tabletmanager/state_change.go b/go/vt/vttablet/tabletmanager/state_change.go index e2fa8dd6838..2e6a1a4076f 100644 --- a/go/vt/vttablet/tabletmanager/state_change.go +++ b/go/vt/vttablet/tabletmanager/state_change.go @@ -24,24 +24,21 @@ import ( "strings" "time" - "vitess.io/vitess/go/vt/vterrors" - "golang.org/x/net/context" - "vitess.io/vitess/go/event" "vitess.io/vitess/go/trace" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/mysqlctl" + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/tabletmanager/events" "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" "vitess.io/vitess/go/vt/vttablet/tabletserver/rules" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" - - querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) var ( @@ -105,6 +102,7 @@ func (agent *ActionAgent) broadcastHealth() { replicationDelay := agent._replicationDelay healthError := agent._healthy terTime := agent._tabletExternallyReparentedTime + healthyTime := agent._healthyTime agent.mutex.Unlock() // send it to our observers @@ -116,12 +114,17 @@ func (agent *ActionAgent) broadcastHealth() { stats.Qps = tabletenv.QPSRates.TotalRate() if healthError != nil { stats.HealthError = healthError.Error() + } else { + timeSinceLastCheck := time.Since(healthyTime) + if timeSinceLastCheck > *healthCheckInterval*3 { + stats.HealthError = fmt.Sprintf("last health check is too old: %s > %s", timeSinceLastCheck, *healthCheckInterval*3) + } } var ts int64 if !terTime.IsZero() { ts = terTime.Unix() } - go agent.QueryServiceControl.BroadcastHealth(ts, stats) + go agent.QueryServiceControl.BroadcastHealth(ts, stats, *healthCheckInterval*3) } // refreshTablet needs to be run after an action may have changed the current diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 75d1c7c32de..9d3a356d8f0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -189,12 +189,13 @@ func (ct *controller) runBlp(ctx context.Context) (err error) { player := binlogplayer.NewBinlogPlayerKeyRange(dbClient, tablet, ct.source.KeyRange, ct.id, ct.blpStats) return player.ApplyBinlogEvents(ctx) case ct.source.Filter != nil: - // VPlayer requires the timezone to be UTC. + // Timestamp fields from binlogs are always sent as UTC. + // So, we should set the timezone to be UTC for those values to be correctly inserted. if _, err := dbClient.ExecuteFetch("set @@session.time_zone = '+00:00'", 10000); err != nil { return err } - vplayer := newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) - return vplayer.Play(ctx) + vreplicator := newVReplicator(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) + return vreplicator.Replicate(ctx) } return fmt.Errorf("missing source") } diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go index e261444ee6c..ba858cf04d3 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go @@ -44,6 +44,7 @@ var ( sqltypes.NULL, // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } @@ -64,7 +65,7 @@ func TestControllerKeyRange(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -99,7 +100,7 @@ func TestControllerTables(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -191,7 +192,7 @@ func TestControllerOverrides(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -255,10 +256,10 @@ func TestControllerRetry(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", nil, errors.New("(expected error)")) - dbClient.ExpectRequest("update _vt.vreplication set state='Error', message='error (expected error) in selecting vreplication settings select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1' where id=1", testDMLResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", nil, errors.New("(expected error)")) + dbClient.ExpectRequest("update _vt.vreplication set state='Error', message='error (expected error) in selecting vreplication settings select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1' where id=1", testDMLResponse, nil) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -298,10 +299,11 @@ func TestControllerStopPosition(t *testing.T) { sqltypes.NewVarBinary("MariaDB/0-1-1235"), // stop_pos sqltypes.NewVarBinary("9223372036854775807"), // max_tps sqltypes.NewVarBinary("9223372036854775807"), // max_replication_lag + sqltypes.NewVarBinary("Running"), // state }, }, } - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", withStop, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", withStop, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 3577508abdc..7aa5c542ba8 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -61,17 +61,19 @@ type Engine struct { cell string mysqld mysqlctl.MysqlDaemon dbClientFactory func() binlogplayer.DBClient + dbName string } // NewEngine creates a new Engine. // A nil ts means that the Engine is disabled. -func NewEngine(ts *topo.Server, cell string, mysqld mysqlctl.MysqlDaemon, dbClientFactory func() binlogplayer.DBClient) *Engine { +func NewEngine(ts *topo.Server, cell string, mysqld mysqlctl.MysqlDaemon, dbClientFactory func() binlogplayer.DBClient, dbName string) *Engine { vre := &Engine{ controllers: make(map[int]*controller), ts: ts, cell: cell, mysqld: mysqld, dbClientFactory: dbClientFactory, + dbName: dbName, } return vre } @@ -111,18 +113,31 @@ func (vre *Engine) executeFetchMaybeCreateTable(dbClient binlogplayer.DBClient, // If it's a bad table or db, it could be because _vt.vreplication wasn't created. // In that case we can try creating it again. merr, isSQLErr := err.(*mysql.SQLError) - if !isSQLErr || !(merr.Num == mysql.ERNoSuchTable || merr.Num == mysql.ERBadDb) { + if !isSQLErr || !(merr.Num == mysql.ERNoSuchTable || merr.Num == mysql.ERBadDb || merr.Num == mysql.ERBadFieldError) { return qr, err } log.Info("Looks like _vt.vreplication table may not exist. Trying to recreate... ") - for _, query := range binlogplayer.CreateVReplicationTable() { - if _, merr := dbClient.ExecuteFetch(query, 0); merr != nil { - log.Warningf("Failed to ensure _vt.vreplication table exists: %v", merr) - return nil, err + if merr.Num == mysql.ERNoSuchTable || merr.Num == mysql.ERBadDb { + for _, query := range binlogplayer.CreateVReplicationTable() { + if _, merr := dbClient.ExecuteFetch(query, 0); merr != nil { + log.Warningf("Failed to ensure _vt.vreplication table exists: %v", merr) + return nil, err + } + } + } + if merr.Num == mysql.ERBadFieldError { + log.Info("Adding column to table _vt.vreplication") + for _, query := range binlogplayer.AlterVReplicationTable() { + if _, merr := dbClient.ExecuteFetch(query, 0); merr != nil { + merr, isSQLErr := err.(*mysql.SQLError) + if !isSQLErr || !(merr.Num == mysql.ERDupFieldName) { + log.Warningf("Failed to alter _vt.vreplication table: %v", merr) + return nil, err + } + } } } - return dbClient.ExecuteFetch(query, maxrows) } @@ -133,13 +148,18 @@ func (vre *Engine) initAll() error { } defer dbClient.Close() - rows, err := readAllRows(dbClient) + rows, err := readAllRows(dbClient, vre.dbName) if err != nil { // Handle Table not found. if merr, ok := err.(*mysql.SQLError); ok && merr.Num == mysql.ERNoSuchTable { log.Info("_vt.vreplication table not found. Will create it later if needed") return nil } + // Handle missing field + if merr, ok := err.(*mysql.SQLError); ok && merr.Num == mysql.ERBadFieldError { + log.Info("_vt.vreplication table found but is missing field db_name. Will add it later if needed") + return nil + } return err } for _, row := range rows { @@ -347,8 +367,8 @@ func (vre *Engine) updateStats() { } } -func readAllRows(dbClient binlogplayer.DBClient) ([]map[string]string, error) { - qr, err := dbClient.ExecuteFetch("select * from _vt.vreplication", 10000) +func readAllRows(dbClient binlogplayer.DBClient, dbName string) ([]map[string]string, error) { + qr, err := dbClient.ExecuteFetch(fmt.Sprintf("select * from _vt.vreplication where db_name=%v", encodeString(dbName)), 10000) if err != nil { return nil, err } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index eb106370b56..a4ac882dd10 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -41,12 +41,12 @@ func TestEngineOpen(t *testing.T) { // Test Insert - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) if vre.IsOpen() { t.Errorf("IsOpen: %v, want false", vre.IsOpen()) } - dbClient.ExpectRequest("select * from _vt.vreplication", sqltypes.MakeTestResult( + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", sqltypes.MakeTestResult( sqltypes.MakeTestFields( "id|state|source", "int64|varchar|varchar", @@ -54,7 +54,7 @@ func TestEngineOpen(t *testing.T) { fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), nil) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -89,9 +89,9 @@ func TestEngineExec(t *testing.T) { // Test Insert - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -107,7 +107,7 @@ func TestEngineExec(t *testing.T) { fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), nil) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -147,7 +147,7 @@ func TestEngineExec(t *testing.T) { fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), nil) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) @@ -212,9 +212,9 @@ func TestEngineBadInsert(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -242,9 +242,9 @@ func TestEngineSelect(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -277,9 +277,9 @@ func TestWaitForPos(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -307,7 +307,7 @@ func TestWaitForPosError(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) err := vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") want := `vreplication engine is closed` @@ -315,7 +315,7 @@ func TestWaitForPosError(t *testing.T) { t.Errorf("WaitForPos: %v, want %v", err, want) } - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -349,9 +349,9 @@ func TestWaitForPosCancel(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -396,10 +396,10 @@ func TestCreateDBAndTable(t *testing.T) { // Test Insert - vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory, dbClient.DBName()) tableNotFound := mysql.SQLError{Num: 1146, Message: "table not found"} - dbClient.ExpectRequest("select * from _vt.vreplication", nil, &tableNotFound) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", nil, &tableNotFound) if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } @@ -412,7 +412,6 @@ func TestCreateDBAndTable(t *testing.T) { dbClient.ExpectRequest("CREATE DATABASE IF NOT EXISTS _vt", &sqltypes.Result{}, nil) dbClient.ExpectRequest("DROP TABLE IF EXISTS _vt.blp_checkpoint", &sqltypes.Result{}, nil) dbClient.ExpectRequestRE("CREATE TABLE IF NOT EXISTS _vt.vreplication.*", &sqltypes.Result{}, nil) - dbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) // Non-recoverable error. @@ -439,7 +438,7 @@ func TestCreateDBAndTable(t *testing.T) { fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), nil) dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil) - dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=1", testSettingsResponse, nil) + dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state from _vt.vreplication where id=1", testSettingsResponse, nil) dbClient.ExpectRequest("begin", nil, nil) dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil) dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 23c019bd638..97a2ee97884 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -70,10 +70,6 @@ func init() { func TestMain(m *testing.M) { flag.Parse() // Do not remove this comment, import into google3 depends on it - if testing.Short() { - os.Exit(m.Run()) - } - exitCode := func() int { var err error env, err = testenv.Init() @@ -100,13 +96,23 @@ func TestMain(m *testing.M) { return 1 } - playerEngine = NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) + playerEngine = NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory, vrepldb) if err := playerEngine.Open(context.Background()); err != nil { fmt.Fprintf(os.Stderr, "%v", err) return 1 } defer playerEngine.Close() + if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), binlogplayer.CreateVReplicationTable()); err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } + + if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), CreateCopyState); err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } + return m.Run() }() os.Exit(exitCode) @@ -129,6 +135,8 @@ func addTablet(id int, shard string, tabletType topodatapb.TabletType, serving, func deleteTablet(t *topodatapb.Tablet) { env.TopoServ.DeleteTablet(context.Background(), t.Alias) + // This is not automatically removed from shard replication, which results in log spam. + topo.DeleteTabletReplicationData(context.Background(), env.TopoServ, t) } func newTablet(id int, shard string, tabletType topodatapb.TabletType, serving, healthy bool) *topodatapb.Tablet { @@ -198,6 +206,25 @@ func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, return streamerEngine.Stream(ctx, startPos, filter, send) } +// streamRowsHook allows you to do work just before VStreamRows is dispatched. +var streamRowsHook func(ctx context.Context) + +// VStreamRows directly calls into the pre-initialized engine. +func (ftc *fakeTabletConn) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + if streamRowsHook != nil { + streamRowsHook(ctx) + } + var row []sqltypes.Value + if lastpk != nil { + r := sqltypes.Proto3ToResult(lastpk) + if len(r.Rows) != 1 { + return fmt.Errorf("unexpected lastpk input: %v", lastpk) + } + row = r.Rows[0] + } + return streamerEngine.StreamRows(ctx, query, row, send) +} + //-------------------------------------- // Binlog Client to TabletManager @@ -372,10 +399,64 @@ func expectDBClientQueries(t *testing.T, queries []string) { } } +// expectNontxQueries disregards transactional statements like begin and commit. +// It also disregards updates to _vt.vreplication. +func expectNontxQueries(t *testing.T, queries []string) { + t.Helper() + failed := false + for i, query := range queries { + if failed { + t.Errorf("no query received, expecting %s", query) + continue + } + var got string + retry: + select { + case got = <-globalDBQueries: + if got == "begin" || got == "commit" || strings.Contains(got, "_vt.vreplication") { + goto retry + } + var match bool + if query[0] == '/' { + result, err := regexp.MatchString(query[1:], got) + if err != nil { + panic(err) + } + match = result + } else { + match = (got == query) + } + if !match { + t.Errorf("query:\n%q, does not match query %d:\n%q", got, i, query) + } + case <-time.After(5 * time.Second): + t.Errorf("no query received, expecting %s", query) + failed = true + } + } + for { + select { + case got := <-globalDBQueries: + if got == "begin" || got == "commit" || got == "rollback" || strings.Contains(got, "_vt.vreplication") { + continue + } + t.Errorf("unexpected query: %s", got) + default: + return + } + } +} + func expectData(t *testing.T, table string, values [][]string) { t.Helper() - qr, err := env.Mysqld.FetchSuperQuery(context.Background(), fmt.Sprintf("select * from %s.%s", vrepldb, table)) + var query string + if len(strings.Split(table, ".")) == 1 { + query = fmt.Sprintf("select * from %s.%s", vrepldb, table) + } else { + query = fmt.Sprintf("select * from %s", table) + } + qr, err := env.Mysqld.FetchSuperQuery(context.Background(), query) if err != nil { t.Error(err) return diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go deleted file mode 100644 index dc7bebf084e..00000000000 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go +++ /dev/null @@ -1,125 +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 vreplication - -import ( - "vitess.io/vitess/go/sqltypes" - "vitess.io/vitess/go/vt/sqlparser" - - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" -) - -// PlayerPlan is the execution plan for a player stream. -type PlayerPlan struct { - VStreamFilter *binlogdatapb.Filter - TablePlans map[string]*TablePlan -} - -// TablePlan is the execution plan for a table within a player stream. -type TablePlan struct { - Name string - PKReferences []string `json:",omitempty"` - Insert *sqlparser.ParsedQuery `json:",omitempty"` - Update *sqlparser.ParsedQuery `json:",omitempty"` - Delete *sqlparser.ParsedQuery `json:",omitempty"` - Fields []*querypb.Field `json:",omitempty"` -} - -func (tp *TablePlan) generateStatements(rowChange *binlogdatapb.RowChange) ([]string, error) { - // MakeRowTrusted is needed here because Proto3ToResult is not convenient. - var before, after bool - bindvars := make(map[string]*querypb.BindVariable) - if rowChange.Before != nil { - before = true - vals := sqltypes.MakeRowTrusted(tp.Fields, rowChange.Before) - for i, field := range tp.Fields { - bindvars["b_"+field.Name] = sqltypes.ValueBindVariable(vals[i]) - } - } - if rowChange.After != nil { - after = true - vals := sqltypes.MakeRowTrusted(tp.Fields, rowChange.After) - for i, field := range tp.Fields { - bindvars["a_"+field.Name] = sqltypes.ValueBindVariable(vals[i]) - } - } - switch { - case !before && after: - query, err := tp.Insert.GenerateQuery(bindvars, nil) - if err != nil { - return nil, err - } - return []string{query}, nil - case before && !after: - if tp.Delete == nil { - return nil, nil - } - query, err := tp.Delete.GenerateQuery(bindvars, nil) - if err != nil { - return nil, err - } - return []string{query}, nil - case before && after: - if !tp.pkChanged(bindvars) { - query, err := tp.Update.GenerateQuery(bindvars, nil) - if err != nil { - return nil, err - } - return []string{query}, nil - } - - queries := make([]string, 0, 2) - if tp.Delete != nil { - query, err := tp.Delete.GenerateQuery(bindvars, nil) - if err != nil { - return nil, err - } - queries = append(queries, query) - } - query, err := tp.Insert.GenerateQuery(bindvars, nil) - if err != nil { - return nil, err - } - queries = append(queries, query) - return queries, nil - } - return nil, nil -} - -func (tp *TablePlan) pkChanged(bindvars map[string]*querypb.BindVariable) bool { - for _, pkref := range tp.PKReferences { - v1, _ := sqltypes.BindVariableToValue(bindvars["b_"+pkref]) - v2, _ := sqltypes.BindVariableToValue(bindvars["a_"+pkref]) - if !valsEqual(v1, v2) { - return true - } - } - return false -} - -func valsEqual(v1, v2 sqltypes.Value) bool { - if v1.IsNull() && v2.IsNull() { - return true - } - // If any one of them is null, something has changed. - if v1.IsNull() || v2.IsNull() { - return false - } - // Compare content only if none are null. - return v1.ToString() == v2.ToString() -} diff --git a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go new file mode 100644 index 00000000000..1eb7d3799a7 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go @@ -0,0 +1,267 @@ +/* +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 vreplication + +import ( + "encoding/json" + "fmt" + "sort" + "strings" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/sqlparser" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" +) + +// ReplicatorPlan is the execution plan for the replicator. +// The constructor for this is buildReplicatorPlan in table_plan_builder.go +// The initial build identifies the tables that need to be replicated, +// and builds partial TablePlan objects for them. The partial plan is used +// to send streaming requests. As the responses return field info, this +// information is used to build the final execution plan (buildExecutionPlan). +type ReplicatorPlan struct { + VStreamFilter *binlogdatapb.Filter + TargetTables map[string]*TablePlan + TablePlans map[string]*TablePlan + tableKeys map[string][]string +} + +// buildExecution plan uses the field info as input and the partially built +// TablePlan for that table to build a full plan. +func (rp *ReplicatorPlan) buildExecutionPlan(fieldEvent *binlogdatapb.FieldEvent) (*TablePlan, error) { + prelim := rp.TablePlans[fieldEvent.TableName] + if prelim == nil { + // Unreachable code. + return nil, fmt.Errorf("plan not found for %s", fieldEvent.TableName) + } + if prelim.Insert != nil { + tplanv := *prelim + tplanv.Fields = fieldEvent.Fields + return &tplanv, nil + } + // select * construct was used. We need to use the field names. + tplan, err := rp.buildFromFields(prelim.TargetName, fieldEvent.Fields) + if err != nil { + return nil, err + } + tplan.Fields = fieldEvent.Fields + return tplan, nil +} + +// buildFromFields builds a full TablePlan, but uses the field info as the +// full column list. This happens when the query used was a 'select *', which +// requires us to wait for the field info sent by the source. +func (rp *ReplicatorPlan) buildFromFields(tableName string, fields []*querypb.Field) (*TablePlan, error) { + tpb := &tablePlanBuilder{ + name: sqlparser.NewTableIdent(tableName), + } + for _, field := range fields { + colName := sqlparser.NewColIdent(field.Name) + cexpr := &colExpr{ + colName: colName, + expr: &sqlparser.ColName{ + Name: colName, + }, + references: map[string]bool{ + field.Name: true, + }, + } + tpb.colExprs = append(tpb.colExprs, cexpr) + } + if err := tpb.analyzePK(rp.tableKeys); err != nil { + return nil, err + } + return tpb.generate(rp.tableKeys), nil +} + +// MarshalJSON performs a custom JSON Marshalling. +func (rp *ReplicatorPlan) MarshalJSON() ([]byte, error) { + var targets []string + for k := range rp.TargetTables { + targets = append(targets, k) + } + sort.Strings(targets) + v := struct { + VStreamFilter *binlogdatapb.Filter + TargetTables []string + TablePlans map[string]*TablePlan + }{ + VStreamFilter: rp.VStreamFilter, + TargetTables: targets, + TablePlans: rp.TablePlans, + } + return json.Marshal(&v) +} + +// TablePlan is the execution plan for a table within a player stream. +// The ParsedQuery objects assume that a map of before and after values +// will be built based on the streaming rows. Before image values will +// be prefixed with a "b_", and after image values will be prefixed +// with a "a_". +type TablePlan struct { + TargetName string + SendRule *binlogdatapb.Rule + PKReferences []string + // BulkInsertFront, BulkInsertValues and BulkInsertOnDup are used + // by vcopier. + BulkInsertFront *sqlparser.ParsedQuery + BulkInsertValues *sqlparser.ParsedQuery + BulkInsertOnDup *sqlparser.ParsedQuery + // Insert, Update and Delete are used by vplayer. + // If the plan is an insertIgnore type, then Insert + // and Update contain 'insert ignore' statements and + // Delete is nil. + Insert *sqlparser.ParsedQuery + Update *sqlparser.ParsedQuery + Delete *sqlparser.ParsedQuery + Fields []*querypb.Field +} + +// MarshalJSON performs a custom JSON Marshalling. +func (tp *TablePlan) MarshalJSON() ([]byte, error) { + v := struct { + TargetName string + SendRule string + PKReferences []string `json:",omitempty"` + InsertFront *sqlparser.ParsedQuery `json:",omitempty"` + InsertValues *sqlparser.ParsedQuery `json:",omitempty"` + InsertOnDup *sqlparser.ParsedQuery `json:",omitempty"` + Insert *sqlparser.ParsedQuery `json:",omitempty"` + Update *sqlparser.ParsedQuery `json:",omitempty"` + Delete *sqlparser.ParsedQuery `json:",omitempty"` + }{ + TargetName: tp.TargetName, + SendRule: tp.SendRule.Match, + PKReferences: tp.PKReferences, + InsertFront: tp.BulkInsertFront, + InsertValues: tp.BulkInsertValues, + InsertOnDup: tp.BulkInsertOnDup, + Insert: tp.Insert, + Update: tp.Update, + Delete: tp.Delete, + } + return json.Marshal(&v) +} + +func (tp *TablePlan) generateBulkInsert(rows *binlogdatapb.VStreamRowsResponse) (string, error) { + bindvars := make(map[string]*querypb.BindVariable, len(tp.Fields)) + var buf strings.Builder + if err := tp.BulkInsertFront.Append(&buf, nil, nil); err != nil { + return "", err + } + buf.WriteString(" values ") + separator := "" + for _, row := range rows.Rows { + vals := sqltypes.MakeRowTrusted(tp.Fields, row) + for i, field := range tp.Fields { + bindvars["a_"+field.Name] = sqltypes.ValueBindVariable(vals[i]) + } + buf.WriteString(separator) + separator = ", " + tp.BulkInsertValues.Append(&buf, bindvars, nil) + } + if tp.BulkInsertOnDup != nil { + tp.BulkInsertOnDup.Append(&buf, nil, nil) + } + return buf.String(), nil +} + +func (tp *TablePlan) generateStatements(rowChange *binlogdatapb.RowChange) ([]string, error) { + // MakeRowTrusted is needed here because Proto3ToResult is not convenient. + var before, after bool + bindvars := make(map[string]*querypb.BindVariable, len(tp.Fields)) + if rowChange.Before != nil { + before = true + vals := sqltypes.MakeRowTrusted(tp.Fields, rowChange.Before) + for i, field := range tp.Fields { + bindvars["b_"+field.Name] = sqltypes.ValueBindVariable(vals[i]) + } + } + if rowChange.After != nil { + after = true + vals := sqltypes.MakeRowTrusted(tp.Fields, rowChange.After) + for i, field := range tp.Fields { + bindvars["a_"+field.Name] = sqltypes.ValueBindVariable(vals[i]) + } + } + switch { + case !before && after: + query, err := tp.Insert.GenerateQuery(bindvars, nil) + if err != nil { + return nil, err + } + return []string{query}, nil + case before && !after: + if tp.Delete == nil { + return nil, nil + } + query, err := tp.Delete.GenerateQuery(bindvars, nil) + if err != nil { + return nil, err + } + return []string{query}, nil + case before && after: + if !tp.pkChanged(bindvars) { + query, err := tp.Update.GenerateQuery(bindvars, nil) + if err != nil { + return nil, err + } + return []string{query}, nil + } + + queries := make([]string, 0, 2) + if tp.Delete != nil { + query, err := tp.Delete.GenerateQuery(bindvars, nil) + if err != nil { + return nil, err + } + queries = append(queries, query) + } + query, err := tp.Insert.GenerateQuery(bindvars, nil) + if err != nil { + return nil, err + } + queries = append(queries, query) + return queries, nil + } + return nil, nil +} + +func (tp *TablePlan) pkChanged(bindvars map[string]*querypb.BindVariable) bool { + for _, pkref := range tp.PKReferences { + v1, _ := sqltypes.BindVariableToValue(bindvars["b_"+pkref]) + v2, _ := sqltypes.BindVariableToValue(bindvars["a_"+pkref]) + if !valsEqual(v1, v2) { + return true + } + } + return false +} + +func valsEqual(v1, v2 sqltypes.Value) bool { + if v1.IsNull() && v2.IsNull() { + return true + } + // If any one of them is null, something has changed. + if v1.IsNull() || v2.IsNull() { + return false + } + // Compare content only if none are null. + return v1.ToString() == v2.ToString() +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go similarity index 52% rename from go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go rename to go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go index 2baa5cc9599..c12c26f830e 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go @@ -20,17 +20,23 @@ import ( "encoding/json" "testing" + "vitess.io/vitess/go/sqltypes" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) -type TestPlayerPlan struct { +type TestReplicatorPlan struct { VStreamFilter *binlogdatapb.Filter + TargetTables []string TablePlans map[string]*TestTablePlan } type TestTablePlan struct { - Name string + TargetName string + SendRule string PKReferences []string `json:",omitempty"` + InsertFront string `json:",omitempty"` + InsertValues string `json:",omitempty"` + InsertOnDup string `json:",omitempty"` Insert string `json:",omitempty"` Update string `json:",omitempty"` Delete string `json:",omitempty"` @@ -38,9 +44,10 @@ type TestTablePlan struct { func TestBuildPlayerPlan(t *testing.T) { testcases := []struct { - input *binlogdatapb.Filter - plan *TestPlayerPlan - err string + input *binlogdatapb.Filter + plan *TestReplicatorPlan + planpk *TestReplicatorPlan + err string }{{ // Regular expression input: &binlogdatapb.Filter{ @@ -48,13 +55,35 @@ func TestBuildPlayerPlan(t *testing.T) { Match: "/.*", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ - Match: "/.*", + Match: "t1", + Filter: "select * from t1", }}, }, - TablePlans: map[string]*TestTablePlan{}, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t1": { + TargetName: "t1", + SendRule: "t1", + }, + }, + }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t1": { + TargetName: "t1", + SendRule: "t1", + }, + }, }, }, { // '*' expression @@ -64,16 +93,33 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select * from t2", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select * from t2", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t2": { + TargetName: "t1", + SendRule: "t2", + }, + }, + }, + planpk: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t2", Filter: "select * from t2", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t2": { - Name: "t1", + TargetName: "t1", + SendRule: "t2", }, }, }, @@ -85,23 +131,48 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select c1, c2 from t2", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t2", Filter: "select c1, c2 from t2", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t2": { - Name: "t1", + TargetName: "t1", + SendRule: "t2", PKReferences: []string{"c1"}, - Insert: "insert into t1 set c1=:a_c1, c2=:a_c2", + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(:a_c1,:a_c2)", + Insert: "insert into t1(c1,c2) values (:a_c1,:a_c2)", Update: "update t1 set c2=:a_c2 where c1=:b_c1", Delete: "delete from t1 where c1=:b_c1", }, }, }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2, pk1, pk2 from t2", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t2": { + TargetName: "t1", + SendRule: "t2", + PKReferences: []string{"c1", "pk1", "pk2"}, + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(:a_c1,:a_c2)", + Insert: "insert into t1(c1,c2) select :a_c1, :a_c2 where (:a_pk1,:a_pk2) <= (1,'aaa')", + Update: "update t1 set c2=:a_c2 where c1=:b_c1 and (:b_pk1,:b_pk2) <= (1,'aaa')", + Delete: "delete from t1 where c1=:b_c1 and (:b_pk1,:b_pk2) <= (1,'aaa')", + }, + }, + }, }, { // partial group by input: &binlogdatapb.Filter{ @@ -110,23 +181,50 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select c1, c2, c3 from t2 group by c3, c1", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t2", Filter: "select c1, c2, c3 from t2", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t2": { - Name: "t1", + TargetName: "t1", + SendRule: "t2", PKReferences: []string{"c1"}, - Insert: "insert into t1 set c1=:a_c1, c2=:a_c2, c3=:a_c3 on duplicate key update c2=:a_c2", + InsertFront: "insert into t1(c1,c2,c3)", + InsertValues: "(:a_c1,:a_c2,:a_c3)", + InsertOnDup: "on duplicate key update c2=values(c2)", + Insert: "insert into t1(c1,c2,c3) values (:a_c1,:a_c2,:a_c3) on duplicate key update c2=values(c2)", Update: "update t1 set c2=:a_c2 where c1=:b_c1", Delete: "update t1 set c2=null where c1=:b_c1", }, }, }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2, c3, pk1, pk2 from t2", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t2": { + TargetName: "t1", + SendRule: "t2", + PKReferences: []string{"c1", "pk1", "pk2"}, + InsertFront: "insert into t1(c1,c2,c3)", + InsertValues: "(:a_c1,:a_c2,:a_c3)", + InsertOnDup: "on duplicate key update c2=values(c2)", + Insert: "insert into t1(c1,c2,c3) select :a_c1, :a_c2, :a_c3 where (:a_pk1,:a_pk2) <= (1,'aaa') on duplicate key update c2=values(c2)", + Update: "update t1 set c2=:a_c2 where c1=:b_c1 and (:b_pk1,:b_pk2) <= (1,'aaa')", + Delete: "update t1 set c2=null where c1=:b_c1 and (:b_pk1,:b_pk2) <= (1,'aaa')", + }, + }, + }, }, { // full group by input: &binlogdatapb.Filter{ @@ -135,19 +233,43 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select c1, c2, c3 from t2 group by c3, c1, c2", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t2", Filter: "select c1, c2, c3 from t2", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t2": { - Name: "t1", + TargetName: "t1", + SendRule: "t2", PKReferences: []string{"c1"}, - Insert: "insert ignore into t1 set c1=:a_c1, c2=:a_c2, c3=:a_c3", - Update: "insert ignore into t1 set c1=:a_c1, c2=:a_c2, c3=:a_c3", + InsertFront: "insert ignore into t1(c1,c2,c3)", + InsertValues: "(:a_c1,:a_c2,:a_c3)", + Insert: "insert ignore into t1(c1,c2,c3) values (:a_c1,:a_c2,:a_c3)", + Update: "insert ignore into t1(c1,c2,c3) values (:a_c1,:a_c2,:a_c3)", + }, + }, + }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2, c3, pk1, pk2 from t2", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t2": { + TargetName: "t1", + SendRule: "t2", + PKReferences: []string{"c1", "pk1", "pk2"}, + InsertFront: "insert ignore into t1(c1,c2,c3)", + InsertValues: "(:a_c1,:a_c2,:a_c3)", + Insert: "insert ignore into t1(c1,c2,c3) select :a_c1, :a_c2, :a_c3 where (:a_pk1,:a_pk2) <= (1,'aaa')", + Update: "insert ignore into t1(c1,c2,c3) select :a_c1, :a_c2, :a_c3 where (:a_pk1,:a_pk2) <= (1,'aaa')", }, }, }, @@ -158,23 +280,48 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select foo(a) as c1, b c2 from t1", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t1", Filter: "select a, b from t1", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t1": { - Name: "t1", + TargetName: "t1", + SendRule: "t1", PKReferences: []string{"a"}, - Insert: "insert into t1 set c1=foo(:a_a), c2=:a_b", + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(foo(:a_a),:a_b)", + Insert: "insert into t1(c1,c2) values (foo(:a_a),:a_b)", Update: "update t1 set c2=:a_b where c1=(foo(:b_a))", Delete: "delete from t1 where c1=(foo(:b_a))", }, }, }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select a, b, pk1, pk2 from t1", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t1": { + TargetName: "t1", + SendRule: "t1", + PKReferences: []string{"a", "pk1", "pk2"}, + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(foo(:a_a),:a_b)", + Insert: "insert into t1(c1,c2) select foo(:a_a), :a_b where (:a_pk1,:a_pk2) <= (1,'aaa')", + Update: "update t1 set c2=:a_b where c1=(foo(:b_a)) and (:b_pk1,:b_pk2) <= (1,'aaa')", + Delete: "delete from t1 where c1=(foo(:b_a)) and (:b_pk1,:b_pk2) <= (1,'aaa')", + }, + }, + }, }, { input: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ @@ -182,23 +329,48 @@ func TestBuildPlayerPlan(t *testing.T) { Filter: "select a + b as c1, c as c2 from t1", }}, }, - plan: &TestPlayerPlan{ + plan: &TestReplicatorPlan{ VStreamFilter: &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ Match: "t1", Filter: "select a, b, c from t1", }}, }, + TargetTables: []string{"t1"}, TablePlans: map[string]*TestTablePlan{ "t1": { - Name: "t1", + TargetName: "t1", + SendRule: "t1", PKReferences: []string{"a", "b"}, - Insert: "insert into t1 set c1=:a_a + :a_b, c2=:a_c", + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(:a_a + :a_b,:a_c)", + Insert: "insert into t1(c1,c2) values (:a_a + :a_b,:a_c)", Update: "update t1 set c2=:a_c where c1=(:b_a + :b_b)", Delete: "delete from t1 where c1=(:b_a + :b_b)", }, }, }, + planpk: &TestReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select a, b, c, pk1, pk2 from t1", + }}, + }, + TargetTables: []string{"t1"}, + TablePlans: map[string]*TestTablePlan{ + "t1": { + TargetName: "t1", + SendRule: "t1", + PKReferences: []string{"a", "b", "pk1", "pk2"}, + InsertFront: "insert into t1(c1,c2)", + InsertValues: "(:a_a + :a_b,:a_c)", + Insert: "insert into t1(c1,c2) select :a_a + :a_b, :a_c where (:a_pk1,:a_pk2) <= (1,'aaa')", + Update: "update t1 set c2=:a_c where c1=(:b_a + :b_b) and (:b_pk1,:b_pk2) <= (1,'aaa')", + Delete: "delete from t1 where c1=(:b_a + :b_b) and (:b_pk1,:b_pk2) <= (1,'aaa')", + }, + }, + }, }, { // syntax error input: &binlogdatapb.Filter{ @@ -349,8 +521,18 @@ func TestBuildPlayerPlan(t *testing.T) { "t1": {"c1"}, } + copyState := map[string]*sqltypes.Result{ + "t1": sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "pk1|pk2", + "int64|varchar", + ), + "1|aaa", + ), + } + for _, tcase := range testcases { - plan, err := buildPlayerPlan(tcase.input, tableKeys) + plan, err := buildReplicatorPlan(tcase.input, tableKeys, nil) gotPlan, _ := json.Marshal(plan) wantPlan, _ := json.Marshal(tcase.plan) if string(gotPlan) != string(wantPlan) { @@ -363,5 +545,15 @@ func TestBuildPlayerPlan(t *testing.T) { if gotErr != tcase.err { t.Errorf("Filter err(%v): %s, want %v", tcase.input, gotErr, tcase.err) } + + plan, err = buildReplicatorPlan(tcase.input, tableKeys, copyState) + if err != nil { + continue + } + gotPlan, _ = json.Marshal(plan) + wantPlan, _ = json.Marshal(tcase.planpk) + if string(gotPlan) != string(wantPlan) { + t.Errorf("Filter(%v,copyState):\n%s, want\n%s", tcase.input, gotPlan, wantPlan) + } } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go index 4f6464cde9c..c913f656a35 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go +++ b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go @@ -18,13 +18,14 @@ package vreplication import ( "fmt" + "regexp" "sort" "strings" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" ) type tablePlanBuilder struct { @@ -34,6 +35,7 @@ type tablePlanBuilder struct { colExprs []*colExpr onInsert insertType pkCols []*colExpr + lastpk *sqltypes.Result } // colExpr describes the processing to be performed to @@ -68,36 +70,92 @@ type insertType int // The following values are the various insert types. const ( insertNormal = insertType(iota) - insertOndup + insertOnDup insertIgnore ) -func buildPlayerPlan(filter *binlogdatapb.Filter, tableKeys map[string][]string) (*PlayerPlan, error) { - plan := &PlayerPlan{ - VStreamFilter: &binlogdatapb.Filter{ - Rules: make([]*binlogdatapb.Rule, len(filter.Rules)), - }, - TablePlans: make(map[string]*TablePlan), - } - for i, rule := range filter.Rules { - if strings.HasPrefix(rule.Match, "/") { - plan.VStreamFilter.Rules[i] = rule +// buildReplicatorPlan builds a ReplicatorPlan for the tables that match the filter. +// The filter is matched against the target schema. For every table matched, +// a table-specific rule is built to be sent to the source. We don't send the +// original rule to the source because it may not match the same tables as the +// target. +// The TablePlan built is a partial plan. The full plan for a table is built +// when we receive field information from events or rows sent by the source. +// buildExecutionPlan is the function that builds the full plan. +func buildReplicatorPlan(filter *binlogdatapb.Filter, tableKeys map[string][]string, copyState map[string]*sqltypes.Result) (*ReplicatorPlan, error) { + plan := &ReplicatorPlan{ + VStreamFilter: &binlogdatapb.Filter{}, + TargetTables: make(map[string]*TablePlan), + TablePlans: make(map[string]*TablePlan), + tableKeys: tableKeys, + } +nextTable: + for tableName := range tableKeys { + lastpk, ok := copyState[tableName] + if ok && lastpk == nil { + // Don't replicate uncopied tables. continue } - sendRule, tablePlan, err := buildTablePlan(rule, tableKeys) - if err != nil { - return nil, err + for _, rule := range filter.Rules { + switch { + case strings.HasPrefix(rule.Match, "/"): + expr := strings.Trim(rule.Match, "/") + result, err := regexp.MatchString(expr, tableName) + if err != nil { + return nil, err + } + if !result { + continue + } + sendRule := &binlogdatapb.Rule{ + Match: tableName, + Filter: buildQuery(tableName, rule.Filter), + } + plan.VStreamFilter.Rules = append(plan.VStreamFilter.Rules, sendRule) + tablePlan := &TablePlan{ + TargetName: tableName, + SendRule: sendRule, + } + plan.TargetTables[tableName] = tablePlan + plan.TablePlans[tableName] = tablePlan + continue nextTable + case rule.Match == tableName: + tablePlan, err := buildTablePlan(rule, tableKeys, lastpk) + if err != nil { + return nil, err + } + if _, ok := plan.TablePlans[tablePlan.SendRule.Match]; ok { + continue + } + plan.VStreamFilter.Rules = append(plan.VStreamFilter.Rules, tablePlan.SendRule) + plan.TargetTables[tableName] = tablePlan + plan.TablePlans[tablePlan.SendRule.Match] = tablePlan + continue nextTable + } } - plan.VStreamFilter.Rules[i] = sendRule - plan.TablePlans[sendRule.Match] = tablePlan } return plan, nil } -func buildTablePlan(rule *binlogdatapb.Rule, tableKeys map[string][]string) (*binlogdatapb.Rule, *TablePlan, error) { - sel, fromTable, err := analyzeSelectFrom(rule.Filter) +func buildQuery(tableName, filter string) string { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select * from %v", sqlparser.NewTableIdent(tableName)) + if filter != "" { + buf.Myprintf(" where in_keyrange(%v)", sqlparser.NewStrVal([]byte(filter))) + } + return buf.String() +} + +func buildTablePlan(rule *binlogdatapb.Rule, tableKeys map[string][]string, lastpk *sqltypes.Result) (*TablePlan, error) { + query := rule.Filter + if query == "" { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select * from %v", sqlparser.NewTableIdent(rule.Match)) + query = buf.String() + } + sel, fromTable, err := analyzeSelectFrom(query) if err != nil { - return nil, nil, err + return nil, err } sendRule := &binlogdatapb.Rule{ Match: fromTable, @@ -105,13 +163,17 @@ func buildTablePlan(rule *binlogdatapb.Rule, tableKeys map[string][]string) (*bi if expr, ok := sel.SelectExprs[0].(*sqlparser.StarExpr); ok { if len(sel.SelectExprs) != 1 { - return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + return nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) } if !expr.TableName.IsEmpty() { - return nil, nil, fmt.Errorf("unsupported qualifier for '*' expression: %v", sqlparser.String(expr)) + return nil, fmt.Errorf("unsupported qualifier for '*' expression: %v", sqlparser.String(expr)) } - sendRule.Filter = rule.Filter - return sendRule, &TablePlan{Name: rule.Match}, nil + sendRule.Filter = query + tablePlan := &TablePlan{ + TargetName: rule.Match, + SendRule: sendRule, + } + return tablePlan, nil } tpb := &tablePlanBuilder{ @@ -121,44 +183,28 @@ func buildTablePlan(rule *binlogdatapb.Rule, tableKeys map[string][]string) (*bi Where: sel.Where, }, selColumns: make(map[string]bool), + lastpk: lastpk, } if err := tpb.analyzeExprs(sel.SelectExprs); err != nil { - return nil, nil, err + return nil, err + } + if tpb.lastpk != nil { + for _, f := range tpb.lastpk.Fields { + tpb.addCol(sqlparser.NewColIdent(f.Name)) + } } if err := tpb.analyzeGroupBy(sel.GroupBy); err != nil { - return nil, nil, err + return nil, err } if err := tpb.analyzePK(tableKeys); err != nil { - return nil, nil, err + return nil, err } sendRule.Filter = sqlparser.String(tpb.sendSelect) tablePlan := tpb.generate(tableKeys) - return sendRule, tablePlan, nil -} - -func buildTablePlanFromFields(tableName string, fields []*querypb.Field, tableKeys map[string][]string) (*TablePlan, error) { - tpb := &tablePlanBuilder{ - name: sqlparser.NewTableIdent(tableName), - } - for _, field := range fields { - colName := sqlparser.NewColIdent(field.Name) - cexpr := &colExpr{ - colName: colName, - expr: &sqlparser.ColName{ - Name: colName, - }, - references: map[string]bool{ - field.Name: true, - }, - } - tpb.colExprs = append(tpb.colExprs, cexpr) - } - if err := tpb.analyzePK(tableKeys); err != nil { - return nil, err - } - return tpb.generate(tableKeys), nil + tablePlan.SendRule = sendRule + return tablePlan, nil } func (tpb *tablePlanBuilder) generate(tableKeys map[string][]string) *TablePlan { @@ -168,17 +214,28 @@ func (tpb *tablePlanBuilder) generate(tableKeys map[string][]string) *TablePlan refmap[k] = true } } + if tpb.lastpk != nil { + for _, f := range tpb.lastpk.Fields { + refmap[f.Name] = true + } + } pkrefs := make([]string, 0, len(refmap)) for k := range refmap { pkrefs = append(pkrefs, k) } sort.Strings(pkrefs) + + bvf := &bindvarFormatter{} + return &TablePlan{ - Name: tpb.name.String(), - PKReferences: pkrefs, - Insert: tpb.generateInsertStatement(), - Update: tpb.generateUpdateStatement(), - Delete: tpb.generateDeleteStatement(), + TargetName: tpb.name.String(), + PKReferences: pkrefs, + BulkInsertFront: tpb.generateInsertPart(sqlparser.NewTrackedBuffer(bvf.formatter)), + BulkInsertValues: tpb.generateValuesPart(sqlparser.NewTrackedBuffer(bvf.formatter), bvf), + BulkInsertOnDup: tpb.generateOnDupPart(sqlparser.NewTrackedBuffer(bvf.formatter)), + Insert: tpb.generateInsertStatement(), + Update: tpb.generateUpdateStatement(), + Delete: tpb.generateDeleteStatement(), } } @@ -320,7 +377,7 @@ func (tpb *tablePlanBuilder) analyzeGroupBy(groupBy sqlparser.GroupBy) error { tpb.onInsert = insertIgnore for _, cExpr := range tpb.colExprs { if !cExpr.isGrouped { - tpb.onInsert = insertOndup + tpb.onInsert = insertOnDup break } } @@ -358,53 +415,59 @@ func (tpb *tablePlanBuilder) findCol(name sqlparser.ColIdent) *colExpr { func (tpb *tablePlanBuilder) generateInsertStatement() *sqlparser.ParsedQuery { bvf := &bindvarFormatter{} buf := sqlparser.NewTrackedBuffer(bvf.formatter) - if tpb.onInsert == insertIgnore { - buf.Myprintf("insert ignore into %v set ", tpb.name) + + tpb.generateInsertPart(buf) + if tpb.lastpk == nil { + buf.Myprintf(" values ", tpb.name) + tpb.generateValuesPart(buf, bvf) } else { - buf.Myprintf("insert into %v set ", tpb.name) - } - tpb.generateInsertValues(buf, bvf) - if tpb.onInsert == insertOndup { - buf.Myprintf(" on duplicate key update ") - tpb.generateUpdate(buf, bvf, false /* before */, true /* after */) + tpb.generateSelectPart(buf, bvf) } + tpb.generateOnDupPart(buf) + return buf.ParsedQuery() } -func (tpb *tablePlanBuilder) generateUpdateStatement() *sqlparser.ParsedQuery { +func (tpb *tablePlanBuilder) generateInsertPart(buf *sqlparser.TrackedBuffer) *sqlparser.ParsedQuery { if tpb.onInsert == insertIgnore { - return tpb.generateInsertStatement() + buf.Myprintf("insert ignore into %v(", tpb.name) + } else { + buf.Myprintf("insert into %v(", tpb.name) } - bvf := &bindvarFormatter{} - buf := sqlparser.NewTrackedBuffer(bvf.formatter) - buf.Myprintf("update %v set ", tpb.name) - tpb.generateUpdate(buf, bvf, true /* before */, true /* after */) - tpb.generateWhere(buf, bvf) + separator := "" + for _, cexpr := range tpb.colExprs { + buf.Myprintf("%s%s", separator, cexpr.colName.String()) + separator = "," + } + buf.Myprintf(")", tpb.name) return buf.ParsedQuery() } -func (tpb *tablePlanBuilder) generateDeleteStatement() *sqlparser.ParsedQuery { - bvf := &bindvarFormatter{} - buf := sqlparser.NewTrackedBuffer(bvf.formatter) - switch tpb.onInsert { - case insertNormal: - buf.Myprintf("delete from %v", tpb.name) - tpb.generateWhere(buf, bvf) - case insertOndup: - buf.Myprintf("update %v set ", tpb.name) - tpb.generateUpdate(buf, bvf, true /* before */, false /* after */) - tpb.generateWhere(buf, bvf) - case insertIgnore: - return nil +func (tpb *tablePlanBuilder) generateValuesPart(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter) *sqlparser.ParsedQuery { + bvf.mode = bvAfter + separator := "(" + for _, cexpr := range tpb.colExprs { + buf.Myprintf("%s", separator) + separator = "," + switch cexpr.operation { + case opExpr: + buf.Myprintf("%v", cexpr.expr) + case opCount: + buf.WriteString("1") + case opSum: + buf.Myprintf("ifnull(%v, 0)", cexpr.expr) + } } + buf.Myprintf(")") return buf.ParsedQuery() } -func (tpb *tablePlanBuilder) generateInsertValues(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter) { +func (tpb *tablePlanBuilder) generateSelectPart(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter) *sqlparser.ParsedQuery { bvf.mode = bvAfter + buf.WriteString(" select ") separator := "" for _, cexpr := range tpb.colExprs { - buf.Myprintf("%s%s=", separator, cexpr.colName.String()) + buf.Myprintf("%s", separator) separator = ", " switch cexpr.operation { case opExpr: @@ -415,9 +478,16 @@ func (tpb *tablePlanBuilder) generateInsertValues(buf *sqlparser.TrackedBuffer, buf.Myprintf("ifnull(%v, 0)", cexpr.expr) } } + buf.WriteString(" where ") + tpb.generatePKConstraint(buf, bvf) + return buf.ParsedQuery() } -func (tpb *tablePlanBuilder) generateUpdate(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter, before, after bool) { +func (tpb *tablePlanBuilder) generateOnDupPart(buf *sqlparser.TrackedBuffer) *sqlparser.ParsedQuery { + if tpb.onInsert != insertOnDup { + return nil + } + buf.Myprintf(" on duplicate key update ") separator := "" for _, cexpr := range tpb.colExprs { if cexpr.isGrouped || cexpr.isPK { @@ -427,33 +497,80 @@ func (tpb *tablePlanBuilder) generateUpdate(buf *sqlparser.TrackedBuffer, bvf *b separator = ", " switch cexpr.operation { case opExpr: - if after { - bvf.mode = bvAfter - buf.Myprintf("%v", cexpr.expr) - } else { - buf.WriteString("null") - } + buf.Myprintf("values(%s)", cexpr.colName.String()) case opCount: - switch { - case before && after: - buf.Myprintf("%s", cexpr.colName.String()) - case before: - buf.Myprintf("%s-1", cexpr.colName.String()) - case after: - buf.Myprintf("%s+1", cexpr.colName.String()) - } + buf.Myprintf("%s+1", cexpr.colName.String()) case opSum: buf.Myprintf("%s", cexpr.colName.String()) - if before { - bvf.mode = bvBefore - buf.Myprintf("-ifnull(%v, 0)", cexpr.expr) + buf.Myprintf("+ifnull(values(%s), 0)", cexpr.colName.String()) + } + } + return buf.ParsedQuery() +} + +func (tpb *tablePlanBuilder) generateUpdateStatement() *sqlparser.ParsedQuery { + if tpb.onInsert == insertIgnore { + return tpb.generateInsertStatement() + } + bvf := &bindvarFormatter{} + buf := sqlparser.NewTrackedBuffer(bvf.formatter) + buf.Myprintf("update %v set ", tpb.name) + separator := "" + for _, cexpr := range tpb.colExprs { + if cexpr.isGrouped || cexpr.isPK { + continue + } + buf.Myprintf("%s%s=", separator, cexpr.colName.String()) + separator = ", " + switch cexpr.operation { + case opExpr: + bvf.mode = bvAfter + buf.Myprintf("%v", cexpr.expr) + case opCount: + buf.Myprintf("%s", cexpr.colName.String()) + case opSum: + buf.Myprintf("%s", cexpr.colName.String()) + bvf.mode = bvBefore + buf.Myprintf("-ifnull(%v, 0)", cexpr.expr) + bvf.mode = bvAfter + buf.Myprintf("+ifnull(%v, 0)", cexpr.expr) + } + } + tpb.generateWhere(buf, bvf) + return buf.ParsedQuery() +} + +func (tpb *tablePlanBuilder) generateDeleteStatement() *sqlparser.ParsedQuery { + bvf := &bindvarFormatter{} + buf := sqlparser.NewTrackedBuffer(bvf.formatter) + switch tpb.onInsert { + case insertNormal: + buf.Myprintf("delete from %v", tpb.name) + tpb.generateWhere(buf, bvf) + case insertOnDup: + bvf.mode = bvBefore + buf.Myprintf("update %v set ", tpb.name) + separator := "" + for _, cexpr := range tpb.colExprs { + if cexpr.isGrouped || cexpr.isPK { + continue } - if after { - bvf.mode = bvAfter - buf.Myprintf("+ifnull(%v, 0)", cexpr.expr) + buf.Myprintf("%s%s=", separator, cexpr.colName.String()) + separator = ", " + switch cexpr.operation { + case opExpr: + buf.WriteString("null") + case opCount: + buf.Myprintf("%s-1", cexpr.colName.String()) + case opSum: + buf.Myprintf("%s-ifnull(%v, 0)", cexpr.colName.String(), cexpr.expr) } } + tpb.generateWhere(buf, bvf) + case insertIgnore: + return nil } + return buf.ParsedQuery() } func (tpb *tablePlanBuilder) generateWhere(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter) { @@ -469,8 +586,34 @@ func (tpb *tablePlanBuilder) generateWhere(buf *sqlparser.TrackedBuffer, bvf *bi } separator = " and " } + if tpb.lastpk != nil { + buf.WriteString(" and ") + tpb.generatePKConstraint(buf, bvf) + } +} + +func (tpb *tablePlanBuilder) generatePKConstraint(buf *sqlparser.TrackedBuffer, bvf *bindvarFormatter) { + separator := "(" + for _, pkname := range tpb.lastpk.Fields { + buf.Myprintf("%s%v", separator, &sqlparser.ColName{Name: sqlparser.NewColIdent(pkname.Name)}) + separator = "," + } + separator = ") <= (" + for _, val := range tpb.lastpk.Rows[0] { + buf.WriteString(separator) + separator = "," + val.EncodeSQL(buf) + } + buf.WriteString(")") } +// bindvarFormatter is a dual mode formatter. Its behavior +// can be changed dynamically changed to generate bind vars +// for the 'before' row or 'after' row by setting its mode +// to 'bvBefore' or 'bvAfter'. For example, inserts will always +// use bvAfter, whereas deletes will always use bvBefore. +// For updates, values being set will use bvAfter, whereas +// the where clause will use bvBefore. type bindvarFormatter struct { mode bindvarMode } @@ -478,8 +621,7 @@ type bindvarFormatter struct { type bindvarMode int const ( - bvNone = bindvarMode(iota) - bvBefore + bvBefore = bindvarMode(iota) bvAfter ) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go new file mode 100644 index 00000000000..82646741fd3 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -0,0 +1,303 @@ +/* +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 vreplication + +import ( + "bytes" + "fmt" + "io" + "strconv" + "strings" + "time" + + "github.com/golang/protobuf/proto" + "golang.org/x/net/context" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/grpcclient" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vttablet/tabletconn" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" +) + +type vcopier struct { + vr *vreplicator + tablePlan *TablePlan +} + +func newVCopier(vr *vreplicator) *vcopier { + return &vcopier{ + vr: vr, + } +} + +func (vc *vcopier) initTablesForCopy(ctx context.Context) error { + defer vc.vr.dbClient.Rollback() + + plan, err := buildReplicatorPlan(vc.vr.source.Filter, vc.vr.tableKeys, nil) + if err != nil { + return err + } + if err := vc.vr.dbClient.Begin(); err != nil { + return err + } + // Insert the table list only if at least one table matches. + if len(plan.TargetTables) != 0 { + var buf strings.Builder + buf.WriteString("insert into _vt.copy_state(vrepl_id, table_name) values ") + prefix := "" + for name := range plan.TargetTables { + fmt.Fprintf(&buf, "%s(%d, %s)", prefix, vc.vr.id, encodeString(name)) + prefix = ", " + } + if _, err := vc.vr.dbClient.ExecuteFetch(buf.String(), 1); err != nil { + return err + } + if err := vc.vr.setState(binlogplayer.VReplicationCopying, ""); err != nil { + return err + } + } else { + if err := vc.vr.setState(binlogplayer.BlpStopped, "There is nothing to replicate"); err != nil { + return err + } + } + return vc.vr.dbClient.Commit() +} + +func (vc *vcopier) copyNext(ctx context.Context, settings binlogplayer.VRSettings) error { + qr, err := vc.vr.dbClient.ExecuteFetch(fmt.Sprintf("select table_name, lastpk from _vt.copy_state where vrepl_id=%d", vc.vr.id), 10000) + if err != nil { + return err + } + var tableToCopy string + copyState := make(map[string]*sqltypes.Result) + for _, row := range qr.Rows { + tableName := row[0].ToString() + lastpk := row[1].ToString() + if tableToCopy == "" { + tableToCopy = tableName + } + copyState[tableName] = nil + if lastpk != "" { + var r querypb.QueryResult + if err := proto.UnmarshalText(lastpk, &r); err != nil { + return err + } + copyState[tableName] = sqltypes.Proto3ToResult(&r) + } + } + if len(copyState) == 0 { + return fmt.Errorf("unexpected: there are no tables to copy") + } + if err := vc.catchup(ctx, copyState); err != nil { + return err + } + return vc.copyTable(ctx, tableToCopy, copyState) +} + +func (vc *vcopier) catchup(ctx context.Context, copyState map[string]*sqltypes.Result) error { + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + settings, err := binlogplayer.ReadVRSettings(vc.vr.dbClient, vc.vr.id) + if err != nil { + return err + } + // If there's no start position, it means we're copying the + // first table. So, there's nothing to catch up to. + if settings.StartPos.IsZero() { + return nil + } + + // Start vreplication. + errch := make(chan error, 1) + go func() { + errch <- newVPlayer(vc.vr, settings, copyState, mysql.Position{}).play(ctx) + }() + + // Wait for catchup. + tmr := time.NewTimer(1 * time.Second) + seconds := int64(replicaLagTolerance / time.Second) + defer tmr.Stop() + for { + sbm := vc.vr.stats.SecondsBehindMaster.Get() + if sbm < seconds { + cancel() + // Make sure vplayer returns before returning. + <-errch + return nil + } + select { + case err := <-errch: + if err != nil { + return err + } + return io.EOF + case <-ctx.Done(): + // Make sure vplayer returns before returning. + <-errch + return io.EOF + case <-tmr.C: + } + } +} + +func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState map[string]*sqltypes.Result) error { + defer vc.vr.dbClient.Rollback() + + log.Infof("Copying table %s, lastpk: %v", tableName, copyState[tableName]) + + plan, err := buildReplicatorPlan(vc.vr.source.Filter, vc.vr.tableKeys, nil) + if err != nil { + return err + } + + initialPlan, ok := plan.TargetTables[tableName] + if !ok { + return fmt.Errorf("plan not found for table: %s, curret plans are: %#v", tableName, plan.TargetTables) + } + + vsClient, err := tabletconn.GetDialer()(vc.vr.sourceTablet, grpcclient.FailFast(false)) + if err != nil { + return fmt.Errorf("error dialing tablet: %v", err) + } + defer vsClient.Close(ctx) + + ctx, cancel := context.WithTimeout(ctx, copyTimeout) + defer cancel() + + target := &querypb.Target{ + Keyspace: vc.vr.sourceTablet.Keyspace, + Shard: vc.vr.sourceTablet.Shard, + TabletType: vc.vr.sourceTablet.Type, + } + + var lastpkpb *querypb.QueryResult + if lastpkqr := copyState[tableName]; lastpkqr != nil { + lastpkpb = sqltypes.ResultToProto3(lastpkqr) + } + + var pkfields []*querypb.Field + var updateCopyState *sqlparser.ParsedQuery + err = vsClient.VStreamRows(ctx, target, initialPlan.SendRule.Filter, lastpkpb, func(rows *binlogdatapb.VStreamRowsResponse) error { + select { + case <-ctx.Done(): + return io.EOF + default: + } + if vc.tablePlan == nil { + if len(rows.Fields) == 0 { + return fmt.Errorf("expecting field event first, got: %v", rows) + } + if err := vc.fastForward(ctx, copyState, rows.Gtid); err != nil { + return err + } + fieldEvent := &binlogdatapb.FieldEvent{ + TableName: initialPlan.SendRule.Match, + Fields: rows.Fields, + } + vc.tablePlan, err = plan.buildExecutionPlan(fieldEvent) + if err != nil { + return err + } + pkfields = rows.Pkfields + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("update _vt.copy_state set lastpk=%a where vrepl_id=%s and table_name=%s", ":lastpk", strconv.Itoa(int(vc.vr.id)), encodeString(tableName)) + updateCopyState = buf.ParsedQuery() + } + if len(rows.Rows) == 0 { + return nil + } + // The number of rows we receive depends on the packet size set + // for the row streamer. Since the packet size is roughly equivalent + // to data size, this should map to a uniform amount of pages affected + // per statement. A packet size of 30K will roughly translate to 8 + // mysql pages of 4K each. + query, err := vc.tablePlan.generateBulkInsert(rows) + if err != nil { + return err + } + var buf bytes.Buffer + err = proto.CompactText(&buf, &querypb.QueryResult{ + Fields: pkfields, + Rows: []*querypb.Row{rows.Lastpk}, + }) + if err != nil { + return err + } + bv := map[string]*querypb.BindVariable{ + "lastpk": { + Type: sqltypes.VarBinary, + Value: buf.Bytes(), + }, + } + updateState, err := updateCopyState.GenerateQuery(bv, nil) + if err != nil { + return err + } + if err := vc.vr.dbClient.Begin(); err != nil { + return err + } + if _, err := vc.vr.dbClient.ExecuteFetch(query, 0); err != nil { + return err + } + if _, err := vc.vr.dbClient.ExecuteFetch(updateState, 0); err != nil { + return err + } + if err := vc.vr.dbClient.Commit(); err != nil { + return err + } + return nil + }) + // If there was a timeout, return without an error. + select { + case <-ctx.Done(): + return nil + default: + } + if err != nil { + return err + } + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("delete from _vt.copy_state where vrepl_id=%s and table_name=%s", strconv.Itoa(int(vc.vr.id)), encodeString(tableName)) + if _, err := vc.vr.dbClient.ExecuteFetch(buf.String(), 0); err != nil { + return err + } + return nil +} + +func (vc *vcopier) fastForward(ctx context.Context, copyState map[string]*sqltypes.Result, gtid string) error { + pos, err := mysql.DecodePosition(gtid) + if err != nil { + return err + } + settings, err := binlogplayer.ReadVRSettings(vc.vr.dbClient, vc.vr.id) + if err != nil { + return err + } + if settings.StartPos.IsZero() { + update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0) + _, err := vc.vr.dbClient.ExecuteFetch(update, 0) + return err + } + return newVPlayer(vc.vr, settings, copyState, pos).play(ctx) +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go new file mode 100644 index 00000000000..ff81c6eaf45 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go @@ -0,0 +1,405 @@ +/* +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 vreplication + +import ( + "fmt" + "strings" + "testing" + "time" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +func TestPlayerCopyTables(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table src1(id int, val varbinary(128), primary key(id))", + "insert into src1 values(2, 'bbb'), (1, 'aaa')", + fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb), + "create table yes(id int, val varbinary(128), primary key(id))", + fmt.Sprintf("create table %s.yes(id int, val varbinary(128), primary key(id))", vrepldb), + "create table no(id int, val varbinary(128), primary key(id))", + }) + defer execStatements(t, []string{ + "drop table src1", + fmt.Sprintf("drop table %s.dst1", vrepldb), + "drop table yes", + fmt.Sprintf("drop table %s.yes", vrepldb), + "drop table no", + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "dst1", + Filter: "select * from src1", + }, { + Match: "/yes", + }}, + } + + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName) + qr, err := playerEngine.Exec(query) + if err != nil { + t.Fatal(err) + } + defer func() { + query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/delete", + }) + }() + + expectDBClientQueries(t, []string{ + "/insert into _vt.vreplication", + // Create the list of tables to copy and transition to Copying state. + "begin", + "/insert into _vt.copy_state", + "/update _vt.vreplication set state='Copying'", + "commit", + "rollback", + // The first fast-forward has no starting point. So, it just saves the current position. + "/update _vt.vreplication set pos=", + "begin", + "insert into dst1(id,val) values (1,'aaa'), (2,'bbb')", + `/update _vt.copy_state set lastpk='fields: rows: ' where vrepl_id=.*`, + "commit", + // copy of dst1 is done: delete from copy_state. + "/delete from _vt.copy_state.*dst1", + "rollback", + // The next FF executes and updates the position before copying. + "begin", + "/update _vt.vreplication set pos=", + "commit", + // Nothing to copy from yes. Delete from copy_state. + "/delete from _vt.copy_state.*yes", + "rollback", + // All tables copied. Final catch up followed by Running state. + "/update _vt.vreplication set state='Running'", + }) + expectData(t, "dst1", [][]string{ + {"1", "aaa"}, + {"2", "bbb"}, + }) + expectData(t, "yes", [][]string{}) +} + +// TestPlayerCopyTableContinuation tests the copy workflow where tables have been partially copied. +func TestPlayerCopyTableContinuation(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + // src1 is initialized as partially copied. + // lastpk will be initialized at (6,6) later below. + // dst1 only copies id1 and val. This will allow us to test for correctness if id2 changes in the source. + "create table src1(id1 int, id2 int, val varbinary(128), primary key(id1, id2))", + "insert into src1 values(2,2,'no change'), (3,3,'update'), (4,4,'delete'), (5,5,'move within'), (6,6,'move out'), (8,8,'no change'), (9,9,'delete'), (10,10,'update'), (11,11,'move in')", + fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb), + fmt.Sprintf("insert into %s.dst1 values(2,'no change'), (3,'update'), (4,'delete'), (5,'move within'), (6,'move out')", vrepldb), + // copied is initialized as fully copied + "create table copied(id int, val varbinary(128), primary key(id))", + "insert into copied values(1,'aaa')", + fmt.Sprintf("create table %s.copied(id int, val varbinary(128), primary key(id))", vrepldb), + fmt.Sprintf("insert into %s.copied values(1,'aaa')", vrepldb), + // not_copied yet to be copied. + "create table not_copied(id int, val varbinary(128), primary key(id))", + "insert into not_copied values(1,'aaa')", + fmt.Sprintf("create table %s.not_copied(id int, val varbinary(128), primary key(id))", vrepldb), + }) + defer execStatements(t, []string{ + "drop table src1", + fmt.Sprintf("drop table %s.dst1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "dst1", + Filter: "select id1 as id, val from src1", + }, { + Match: "copied", + Filter: "select * from copied", + }, { + Match: "not_copied", + Filter: "select * from not_copied", + }}, + } + pos := masterPosition(t) + execStatements(t, []string{ + // insert inside and outside current range. + "insert into src1 values(1,1,'insert in'), (7,7,'insert out')", + // update inside and outside current range. + "update src1 set val='updated' where id1 in (3,10)", + // delete inside and outside current range. + "delete from src1 where id1 in (4,9)", + // move row within range by changing id2. + "update src1 set id2=10 where id1=5", + // move row from within to outside range. + "update src1 set id1=12 where id1=6", + // move row from outside to witihn range. + "update src1 set id1=4 where id1=11", + // modify the copied table. + "update copied set val='bbb' where id=1", + // modify the uncopied table. + "update not_copied set val='bbb' where id=1", + }) + + // Set a hook to execute statements just before the copy begins from src1. + streamRowsHook = func(context.Context) { + execStatements(t, []string{ + "update src1 set val='updated again' where id1 = 3", + }) + // Set it back to nil. Otherwise, this will get executed again when copying not_copied. + streamRowsHook = nil + } + defer func() { streamRowsHook = nil }() + + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.BlpStopped, playerEngine.dbName) + qr, err := playerEngine.Exec(query) + if err != nil { + t.Fatal(err) + } + // As mentioned above. lastpk cut-off is set at (6,6) + lastpk := sqltypes.ResultToProto3(sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id1|id2", + "int32|int32", + ), + "6|6", + )) + lastpk.RowsAffected = 0 + execStatements(t, []string{ + fmt.Sprintf("insert into _vt.copy_state values(%d, '%s', %s)", qr.InsertID, "dst1", encodeString(fmt.Sprintf("%v", lastpk))), + fmt.Sprintf("insert into _vt.copy_state values(%d, '%s', null)", qr.InsertID, "not_copied"), + }) + id := qr.InsertID + _, err = playerEngine.Exec(fmt.Sprintf("update _vt.vreplication set state='Copying', pos=%s where id=%d", encodeString(pos), id)) + if err != nil { + t.Fatal(err) + } + defer func() { + query := fmt.Sprintf("delete from _vt.vreplication where id = %d", id) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + for q := range globalDBQueries { + if strings.HasPrefix(q, "delete from _vt.vreplication") { + break + } + } + }() + + for q := range globalDBQueries { + if strings.HasPrefix(q, "update") { + break + } + } + + expectNontxQueries(t, []string{ + // Catchup + "insert into dst1(id,val) select 1, 'insert in' where (1,1) <= (6,6)", + "insert into dst1(id,val) select 7, 'insert out' where (7,7) <= (6,6)", + "update dst1 set val='updated' where id=3 and (3,3) <= (6,6)", + "update dst1 set val='updated' where id=10 and (10,10) <= (6,6)", + "delete from dst1 where id=4 and (4,4) <= (6,6)", + "delete from dst1 where id=9 and (9,9) <= (6,6)", + "delete from dst1 where id=5 and (5,5) <= (6,6)", + "insert into dst1(id,val) select 5, 'move within' where (5,10) <= (6,6)", + "delete from dst1 where id=6 and (6,6) <= (6,6)", + "insert into dst1(id,val) select 12, 'move out' where (12,6) <= (6,6)", + "delete from dst1 where id=11 and (11,11) <= (6,6)", + "insert into dst1(id,val) select 4, 'move in' where (4,11) <= (6,6)", + "update copied set val='bbb' where id=1", + // Fast-forward + "update dst1 set val='updated again' where id=3 and (3,3) <= (6,6)", + // Copy + "insert into dst1(id,val) values (7,'insert out'), (8,'no change'), (10,'updated'), (12,'move out')", + `/update _vt.copy_state set lastpk='fields: fields: rows: ' where vrepl_id=.*`, + "/delete from _vt.copy_state.*dst1", + "rollback", + // Copy again. There should be no events for catchup. + "insert into not_copied(id,val) values (1,'bbb')", + `/update _vt.copy_state set lastpk='fields: rows: ' where vrepl_id=.*`, + "/delete from _vt.copy_state.*not_copied", + "rollback", + }) + expectData(t, "dst1", [][]string{ + {"1", "insert in"}, + {"2", "no change"}, + {"3", "updated again"}, + {"4", "move in"}, + {"5", "move within"}, + {"7", "insert out"}, + {"8", "no change"}, + {"10", "updated"}, + {"12", "move out"}, + }) + expectData(t, "copied", [][]string{ + {"1", "bbb"}, + }) + expectData(t, "not_copied", [][]string{ + {"1", "bbb"}, + }) +} + +func TestPlayerCopyTablesNone(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "dst1", + Filter: "select * from src1", + }}, + } + + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName) + qr, err := playerEngine.Exec(query) + if err != nil { + t.Fatal(err) + } + defer func() { + query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/delete", + }) + }() + + expectDBClientQueries(t, []string{ + "/insert into _vt.vreplication", + "begin", + "/update _vt.vreplication set state='Stopped'", + "commit", + "rollback", + }) +} + +func TestPlayerCopyTableCancel(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table src1(id int, val varbinary(128), primary key(id))", + "insert into src1 values(2, 'bbb'), (1, 'aaa')", + fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb), + }) + defer execStatements(t, []string{ + "drop table src1", + fmt.Sprintf("drop table %s.dst1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + saveTimeout := copyTimeout + copyTimeout = 1 * time.Millisecond + defer func() { copyTimeout = saveTimeout }() + + // Set a hook to reset the copy timeout after first call. + streamRowsHook = func(ctx context.Context) { + <-ctx.Done() + copyTimeout = saveTimeout + streamRowsHook = nil + } + defer func() { streamRowsHook = nil }() + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "dst1", + Filter: "select * from src1", + }}, + } + + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + query := binlogplayer.CreateVReplicationState("test", bls, "", binlogplayer.VReplicationInit, playerEngine.dbName) + qr, err := playerEngine.Exec(query) + if err != nil { + t.Fatal(err) + } + defer func() { + query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/delete", + }) + }() + + // Make sure rows get copied in spite of the early context cancel. + expectDBClientQueries(t, []string{ + "/insert into _vt.vreplication", + // Create the list of tables to copy and transition to Copying state. + "begin", + "/insert into _vt.copy_state", + "/update _vt.vreplication set state='Copying'", + "commit", + "rollback", + // The first copy will do nothing because we set the timeout to be too low. + // We should expect it to do an empty rollback. + "rollback", + // The next copy should proceed as planned because we've made the timeout high again. + // The first fast-forward has no starting point. So, it just saves the current position. + "/update _vt.vreplication set pos=", + "begin", + "insert into dst1(id,val) values (1,'aaa'), (2,'bbb')", + `/update _vt.copy_state set lastpk='fields: rows: ' where vrepl_id=.*`, + "commit", + // copy of dst1 is done: delete from copy_state. + "/delete from _vt.copy_state.*dst1", + "rollback", + // All tables copied. Final catch up followed by Running state. + "/update _vt.vreplication set state='Running'", + }) + expectData(t, "dst1", [][]string{ + {"1", "aaa"}, + {"2", "bbb"}, + }) +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index b1344c6ae75..8e2e630d6b4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -23,142 +23,109 @@ import ( "time" "golang.org/x/net/context" - "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/vt/mysqlctl" "vitess.io/vitess/go/vt/vttablet/tabletconn" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" -) - -var ( - // idleTimeout is set to slightly above 1s, compared to heartbeatTime - // set by VStreamer at slightly below 1s. This minimizes conflicts - // between the two timeouts. - idleTimeout = 1100 * time.Millisecond - dbLockRetryDelay = 1 * time.Second - relayLogMaxSize = 10000 - relayLogMaxItems = 1000 ) type vplayer struct { - id uint32 - source *binlogdatapb.BinlogSource - sourceTablet *topodatapb.Tablet - stats *binlogplayer.Stats - dbClient *vdbClient - // mysqld is used to fetch the local schema. - mysqld mysqlctl.MysqlDaemon + vr *vreplicator + startPos mysql.Position + stopPos mysql.Position + saveStop bool + copyState map[string]*sqltypes.Result + + replicatorPlan *ReplicatorPlan + tablePlans map[string]*TablePlan pos mysql.Position - // unsavedGTID when we receive a GTID event and reset - // if it gets saved. If Fetch returns on idleTimeout, - // we save the last unsavedGTID. - unsavedGTID *binlogdatapb.VEvent + // unsavedEvent is saved any time we skip an event without + // saving: This can be an empty commit or a skipped DDL. + unsavedEvent *binlogdatapb.VEvent // timeLastSaved is set every time a GTID is saved. timeLastSaved time.Time // lastTimestampNs is the last timestamp seen so far. lastTimestampNs int64 // timeOffsetNs keeps track of the clock difference with respect to source tablet. timeOffsetNs int64 - stopPos mysql.Position - - tableKeys map[string][]string - - // pplan is built based on the source Filter at the beginning. - pplan *PlayerPlan - // tplans[table] is built for each table based on pplan and schema info - // about the table. - tplans map[string]*TablePlan } -func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon) *vplayer { +func newVPlayer(vr *vreplicator, settings binlogplayer.VRSettings, copyState map[string]*sqltypes.Result, pausePos mysql.Position) *vplayer { + saveStop := true + if !pausePos.IsZero() { + settings.StopPos = pausePos + saveStop = false + } return &vplayer{ - id: id, - source: source, - sourceTablet: sourceTablet, - stats: stats, - dbClient: newVDBClient(dbClient, stats), - mysqld: mysqld, + vr: vr, + startPos: settings.StartPos, + pos: settings.StartPos, + stopPos: settings.StopPos, + saveStop: saveStop, + copyState: copyState, timeLastSaved: time.Now(), - tplans: make(map[string]*TablePlan), + tablePlans: make(map[string]*TablePlan), } } -func (vp *vplayer) Play(ctx context.Context) error { - if err := vp.setState(binlogplayer.BlpRunning, ""); err != nil { +// play is not resumable. If pausePos is set, play returns without updating the vreplication state. +func (vp *vplayer) play(ctx context.Context) error { + if !vp.stopPos.IsZero() && vp.startPos.AtLeast(vp.stopPos) { + if vp.saveStop { + return vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.startPos, vp.stopPos)) + } + return nil + } + + plan, err := buildReplicatorPlan(vp.vr.source.Filter, vp.vr.tableKeys, vp.copyState) + if err != nil { return err } - if err := vp.play(ctx); err != nil { + vp.replicatorPlan = plan + + if err := vp.fetchAndApply(ctx); err != nil { msg := err.Error() - vp.stats.History.Add(&binlogplayer.StatsHistoryRecord{ + vp.vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), Message: msg, }) - if err := vp.setState(binlogplayer.BlpError, msg); err != nil { - return err + if err := vp.vr.setMessage(msg); err != nil { + log.Errorf("Failed to set error state: %v", err) } return err } return nil } -func (vp *vplayer) play(ctx context.Context) error { - startPos, stopPos, _, _, err := binlogplayer.ReadVRSettings(vp.dbClient, vp.id) - if err != nil { - return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error reading VReplication settings: %v", err)) - } - vp.pos, err = mysql.DecodePosition(startPos) - if err != nil { - return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error decoding start position %v: %v", startPos, err)) - } - if stopPos != "" { - vp.stopPos, err = mysql.DecodePosition(stopPos) - if err != nil { - return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error decoding stop position %v: %v", stopPos, err)) - } - } - if !vp.stopPos.IsZero() { - if vp.pos.AtLeast(vp.stopPos) { - return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.pos, vp.stopPos)) - } - } - log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v, filter: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet, vp.source) - - tableKeys, err := vp.buildTableKeys() - if err != nil { - return err - } - vp.tableKeys = tableKeys - plan, err := buildPlayerPlan(vp.source.Filter, tableKeys) - if err != nil { - return err - } - vp.pplan = plan +func (vp *vplayer) fetchAndApply(ctx context.Context) error { + log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v, filter: %v", vp.vr.id, vp.startPos, vp.stopPos, vp.vr.sourceTablet, vp.vr.source) - vsClient, err := tabletconn.GetDialer()(vp.sourceTablet, grpcclient.FailFast(false)) + vsClient, err := tabletconn.GetDialer()(vp.vr.sourceTablet, grpcclient.FailFast(false)) if err != nil { return fmt.Errorf("error dialing tablet: %v", err) } + defer vsClient.Close(ctx) ctx, cancel := context.WithCancel(ctx) defer cancel() relay := newRelayLog(ctx, relayLogMaxItems, relayLogMaxSize) target := &querypb.Target{ - Keyspace: vp.sourceTablet.Keyspace, - Shard: vp.sourceTablet.Shard, - TabletType: vp.sourceTablet.Type, + Keyspace: vp.vr.sourceTablet.Keyspace, + Shard: vp.vr.sourceTablet.Shard, + TabletType: vp.vr.sourceTablet.Type, } - log.Infof("Sending vstream command: %v", plan.VStreamFilter) + log.Infof("Sending vstream command: %v", vp.replicatorPlan.VStreamFilter) streamErr := make(chan error, 1) go func() { - streamErr <- vsClient.VStream(ctx, target, startPos, plan.VStreamFilter, func(events []*binlogdatapb.VEvent) error { + streamErr <- vsClient.VStream(ctx, target, mysql.EncodePosition(vp.startPos), vp.replicatorPlan.VStreamFilter, func(events []*binlogdatapb.VEvent) error { return relay.Send(events) }) }() @@ -203,20 +170,67 @@ func (vp *vplayer) play(ctx context.Context) error { } } -func (vp *vplayer) buildTableKeys() (map[string][]string, error) { - schema, err := vp.mysqld.GetSchema(vp.dbClient.DBName(), []string{"/.*/"}, nil, false) - if err != nil { - return nil, err +func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.RowEvent) error { + tplan := vp.tablePlans[rowEvent.TableName] + if tplan == nil { + return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) } - tableKeys := make(map[string][]string) - for _, td := range schema.TableDefinitions { - if len(td.PrimaryKeyColumns) != 0 { - tableKeys[td.Name] = td.PrimaryKeyColumns - } else { - tableKeys[td.Name] = td.Columns + for _, change := range rowEvent.RowChanges { + queries, err := tplan.generateStatements(change) + if err != nil { + return err + } + for _, query := range queries { + if err := vp.exec(ctx, query); err != nil { + return err + } } } - return tableKeys, nil + return nil +} + +func (vp *vplayer) updatePos(ts int64) (posReached bool, err error) { + update := binlogplayer.GenerateUpdatePos(vp.vr.id, vp.pos, time.Now().Unix(), ts) + if _, err := vp.vr.dbClient.ExecuteFetch(update, 0); err != nil { + vp.vr.dbClient.Rollback() + return false, fmt.Errorf("error %v updating position", err) + } + vp.unsavedEvent = nil + vp.timeLastSaved = time.Now() + vp.vr.stats.SetLastPosition(vp.pos) + posReached = !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) + if posReached { + if vp.saveStop { + if err := vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { + return false, err + } + } + } + return posReached, nil +} + +func (vp *vplayer) exec(ctx context.Context, sql string) error { + vp.vr.stats.Timings.Record("query", time.Now()) + _, err := vp.vr.dbClient.ExecuteFetch(sql, 0) + for err != nil { + if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Number() == mysql.ERLockDeadlock || sqlErr.Number() == mysql.ERLockWaitTimeout { + log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dbLockRetryDelay) + if err := vp.vr.dbClient.Rollback(); err != nil { + return err + } + time.Sleep(dbLockRetryDelay) + // Check context here. Otherwise this can become an infinite loop. + select { + case <-ctx.Done(): + return io.EOF + default: + } + err = vp.vr.dbClient.Retry() + continue + } + return err + } + return nil } func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { @@ -229,7 +243,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { // So, we should assume we're falling behind. if len(items) == 0 { behind := time.Now().UnixNano() - vp.lastTimestampNs - vp.timeOffsetNs - vp.stats.SecondsBehindMaster.Set(behind / 1e9) + vp.vr.stats.SecondsBehindMaster.Set(behind / 1e9) } // Filtered replication often ends up receiving a large number of empty transactions. // This is required because the player needs to know the latest position of the source. @@ -244,13 +258,14 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { // 1. Fetch was idle for idleTimeout. // 2. We've been receiving empty events for longer than idleTimeout. // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved. - if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { - // Although unlikely, we should not save if a transaction is still open. - // This can happen if a large transaction is split as multiple events. - if !vp.dbClient.InTransaction { - if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { - return err - } + if time.Since(vp.timeLastSaved) >= idleTimeout && vp.unsavedEvent != nil { + posReached, err := vp.updatePos(vp.unsavedEvent.Timestamp) + if err != nil { + return err + } + if posReached { + // Unreachable. + return nil } } for i, events := range items { @@ -258,7 +273,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { if event.Timestamp != 0 { vp.lastTimestampNs = event.Timestamp * 1e9 vp.timeOffsetNs = time.Now().UnixNano() - event.CurrentTime - vp.stats.SecondsBehindMaster.Set(event.CurrentTime/1e9 - event.Timestamp) + vp.vr.stats.SecondsBehindMaster.Set(event.CurrentTime/1e9 - event.Timestamp) } mustSave := false switch event.Type { @@ -305,14 +320,17 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m return err } vp.pos = pos - vp.unsavedGTID = event + // A new position should not be saved until a commit or DDL. + vp.unsavedEvent = nil if vp.stopPos.IsZero() { return nil } if !vp.pos.Equal(vp.stopPos) && vp.pos.AtLeast(vp.stopPos) { // Code is unreachable, but bad data can cause this to happen. - if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos)); err != nil { - return err + if vp.saveStop { + if err := vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos)); err != nil { + return err + } } return io.EOF } @@ -320,61 +338,67 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m // No-op: begin is called as needed. case binlogdatapb.VEventType_COMMIT: if mustSave { - if err := vp.dbClient.Begin(); err != nil { + if err := vp.vr.dbClient.Begin(); err != nil { return err } } - if !vp.dbClient.InTransaction { + if !vp.vr.dbClient.InTransaction { + // We're skipping an empty transaction. We may have to save the position on inactivity. + vp.unsavedEvent = event return nil } - if err := vp.updatePos(event.Timestamp); err != nil { + posReached, err := vp.updatePos(event.Timestamp) + if err != nil { return err } - posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) - if posReached { - if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { - return err - } - } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.vr.dbClient.Commit(); err != nil { return err } if posReached { return io.EOF } case binlogdatapb.VEventType_FIELD: - if err := vp.dbClient.Begin(); err != nil { + if err := vp.vr.dbClient.Begin(); err != nil { return err } - if err := vp.updatePlan(event.FieldEvent); err != nil { + tplan, err := vp.replicatorPlan.buildExecutionPlan(event.FieldEvent) + if err != nil { return err } + vp.tablePlans[event.FieldEvent.TableName] = tplan case binlogdatapb.VEventType_ROW: - if err := vp.dbClient.Begin(); err != nil { + if err := vp.vr.dbClient.Begin(); err != nil { return err } if err := vp.applyRowEvent(ctx, event.RowEvent); err != nil { return err } case binlogdatapb.VEventType_DDL: - if vp.dbClient.InTransaction { + if vp.vr.dbClient.InTransaction { return fmt.Errorf("unexpected state: DDL encountered in the middle of a transaction: %v", event.Ddl) } - switch vp.source.OnDdl { + switch vp.vr.source.OnDdl { case binlogdatapb.OnDDLAction_IGNORE: - // no-op + // We still have to update the position. + posReached, err := vp.updatePos(event.Timestamp) + if err != nil { + return err + } + if posReached { + return io.EOF + } case binlogdatapb.OnDDLAction_STOP: - if err := vp.dbClient.Begin(); err != nil { + if err := vp.vr.dbClient.Begin(); err != nil { return err } - if err := vp.updatePos(event.Timestamp); err != nil { + if _, err := vp.updatePos(event.Timestamp); err != nil { return err } - if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Ddl)); err != nil { + if err := vp.vr.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Ddl)); err != nil { return err } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.vr.dbClient.Commit(); err != nil { return err } return io.EOF @@ -382,99 +406,27 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m if err := vp.exec(ctx, event.Ddl); err != nil { return err } - if err := vp.updatePos(event.Timestamp); err != nil { + posReached, err := vp.updatePos(event.Timestamp) + if err != nil { return err } + if posReached { + return io.EOF + } case binlogdatapb.OnDDLAction_EXEC_IGNORE: if err := vp.exec(ctx, event.Ddl); err != nil { log.Infof("Ignoring error: %v for DDL: %s", err, event.Ddl) } - if err := vp.updatePos(event.Timestamp); err != nil { + posReached, err := vp.updatePos(event.Timestamp) + if err != nil { return err } - } - case binlogdatapb.VEventType_HEARTBEAT: - // No-op: heartbeat timings are calculated in outer loop. - } - return nil -} - -func (vp *vplayer) setState(state, message string) error { - return binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message) -} - -func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { - prelim := vp.pplan.TablePlans[fieldEvent.TableName] - if prelim == nil { - prelim = &TablePlan{ - Name: fieldEvent.TableName, - } - } - if prelim.Insert != nil { - prelim.Fields = fieldEvent.Fields - vp.tplans[fieldEvent.TableName] = prelim - return nil - } - tplan, err := buildTablePlanFromFields(prelim.Name, fieldEvent.Fields, vp.tableKeys) - if err != nil { - return err - } - tplan.Fields = fieldEvent.Fields - vp.tplans[fieldEvent.TableName] = tplan - return nil -} - -func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.RowEvent) error { - tplan := vp.tplans[rowEvent.TableName] - if tplan == nil { - return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) - } - for _, change := range rowEvent.RowChanges { - queries, err := tplan.generateStatements(change) - if err != nil { - return err - } - for _, query := range queries { - if err := vp.exec(ctx, query); err != nil { - return err - } - } - } - return nil -} - -func (vp *vplayer) updatePos(ts int64) error { - updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) - if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { - vp.dbClient.Rollback() - return fmt.Errorf("error %v updating position", err) - } - vp.unsavedGTID = nil - vp.timeLastSaved = time.Now() - vp.stats.SetLastPosition(vp.pos) - return nil -} - -func (vp *vplayer) exec(ctx context.Context, sql string) error { - vp.stats.Timings.Record("query", time.Now()) - _, err := vp.dbClient.ExecuteFetch(sql, 0) - for err != nil { - if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Number() == mysql.ERLockDeadlock || sqlErr.Number() == mysql.ERLockWaitTimeout { - log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dbLockRetryDelay) - if err := vp.dbClient.Rollback(); err != nil { - return err - } - time.Sleep(dbLockRetryDelay) - // Check context here. Otherwise this can become an infinite loop. - select { - case <-ctx.Done(): + if posReached { return io.EOF - default: } - err = vp.dbClient.Retry() - continue } - return err + case binlogdatapb.VEventType_HEARTBEAT: + // No-op: heartbeat timings are calculated in outer loop. } return nil } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 228dba7207f..cdf5a1e1215 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -92,7 +92,7 @@ func TestPlayerFilters(t *testing.T) { input: "insert into src1 values(1, 'aaa')", output: []string{ "begin", - "insert into dst1 set id=1, val='aaa'", + "insert into dst1(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }, @@ -129,7 +129,7 @@ func TestPlayerFilters(t *testing.T) { input: "insert into src2 values(1, 2, 3)", output: []string{ "begin", - "insert into dst2 set id=1, val1=2, sval2=ifnull(3, 0), rcount=1 on duplicate key update val1=2, sval2=sval2+ifnull(3, 0), rcount=rcount+1", + "insert into dst2(id,val1,sval2,rcount) values (1,2,ifnull(3, 0),1) on duplicate key update val1=values(val1), sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1", "/update _vt.vreplication set pos=", "commit", }, @@ -168,7 +168,7 @@ func TestPlayerFilters(t *testing.T) { input: "insert into src3 values(1, 'aaa')", output: []string{ "begin", - "insert ignore into dst3 set id=1, val='aaa'", + "insert ignore into dst3(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }, @@ -181,7 +181,7 @@ func TestPlayerFilters(t *testing.T) { input: "update src3 set val='bbb'", output: []string{ "begin", - "insert ignore into dst3 set id=1, val='bbb'", + "insert ignore into dst3(id,val) values (1,'bbb')", "/update _vt.vreplication set pos=", "commit", }, @@ -206,7 +206,7 @@ func TestPlayerFilters(t *testing.T) { input: "insert into yes values(1, 'aaa')", output: []string{ "begin", - "insert into yes set id=1, val='aaa'", + "insert into yes(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }, @@ -236,7 +236,7 @@ func TestPlayerFilters(t *testing.T) { input: "insert into nopk values(1, 'aaa')", output: []string{ "begin", - "insert into nopk set id=1, val='aaa'", + "insert into nopk(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }, @@ -250,7 +250,7 @@ func TestPlayerFilters(t *testing.T) { output: []string{ "begin", "delete from nopk where id=1 and val='aaa'", - "insert into nopk set id=1, val='bbb'", + "insert into nopk(id,val) values (1,'bbb')", "/update _vt.vreplication set pos=", "commit", }, @@ -310,7 +310,7 @@ func TestPlayerUpdates(t *testing.T) { }{{ // Start with all nulls input: "insert into t1 values(1, null, null, null)", - output: "insert into t1 set id=1, grouped=null, ungrouped=null, summed=ifnull(null, 0), rcount=1 on duplicate key update ungrouped=null, summed=summed+ifnull(null, 0), rcount=rcount+1", + output: "insert into t1(id,grouped,ungrouped,summed,rcount) values (1,null,null,ifnull(null, 0),1) on duplicate key update ungrouped=values(ungrouped), summed=summed+ifnull(values(summed), 0), rcount=rcount+1", table: "t1", data: [][]string{ {"1", "", "", "0", "1"}, @@ -350,7 +350,7 @@ func TestPlayerUpdates(t *testing.T) { }, { // insert non-null values input: "insert into t1 values(2, 2, 3, 4)", - output: "insert into t1 set id=2, grouped=2, ungrouped=3, summed=ifnull(4, 0), rcount=1 on duplicate key update ungrouped=3, summed=summed+ifnull(4, 0), rcount=rcount+1", + output: "insert into t1(id,grouped,ungrouped,summed,rcount) values (2,2,3,ifnull(4, 0),1) on duplicate key update ungrouped=values(ungrouped), summed=summed+ifnull(values(summed), 0), rcount=rcount+1", table: "t1", data: [][]string{ {"1", "", "", "0", "1"}, @@ -416,9 +416,9 @@ func TestPlayerRowMove(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into dst set val1=1, sval2=ifnull(1, 0), rcount=1 on duplicate key update sval2=sval2+ifnull(1, 0), rcount=rcount+1", - "insert into dst set val1=2, sval2=ifnull(2, 0), rcount=1 on duplicate key update sval2=sval2+ifnull(2, 0), rcount=rcount+1", - "insert into dst set val1=2, sval2=ifnull(3, 0), rcount=1 on duplicate key update sval2=sval2+ifnull(3, 0), rcount=rcount+1", + "insert into dst(val1,sval2,rcount) values (1,ifnull(1, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1", + "insert into dst(val1,sval2,rcount) values (2,ifnull(2, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1", + "insert into dst(val1,sval2,rcount) values (2,ifnull(3, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1", "/update _vt.vreplication set pos=", "commit", }) @@ -433,7 +433,7 @@ func TestPlayerRowMove(t *testing.T) { expectDBClientQueries(t, []string{ "begin", "update dst set sval2=sval2-ifnull(3, 0), rcount=rcount-1 where val1=2", - "insert into dst set val1=1, sval2=ifnull(4, 0), rcount=1 on duplicate key update sval2=sval2+ifnull(4, 0), rcount=rcount+1", + "insert into dst(val1,sval2,rcount) values (1,ifnull(4, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1", "/update _vt.vreplication set pos=", "commit", }) @@ -492,42 +492,42 @@ func TestPlayerTypes(t *testing.T) { data [][]string }{{ input: "insert into vitess_ints values(-128, 255, -32768, 65535, -8388608, 16777215, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 2012)", - output: "insert into vitess_ints set tiny=-128, tinyu=255, small=-32768, smallu=65535, medium=-8388608, mediumu=16777215, normal=-2147483648, normalu=4294967295, big=-9223372036854775808, bigu=18446744073709551615, y=2012", + output: "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)", table: "vitess_ints", data: [][]string{ {"-128", "255", "-32768", "65535", "-8388608", "16777215", "-2147483648", "4294967295", "-9223372036854775808", "18446744073709551615", "2012"}, }, }, { input: "insert into vitess_fracts values(1, 1.99, 2.99, 3.99, 4.99)", - output: "insert into vitess_fracts set id=1, deci=1.99, num=2.99, f=3.99E+00, d=4.99E+00", + output: "insert into vitess_fracts(id,deci,num,f,d) values (1,1.99,2.99,3.99E+00,4.99E+00)", table: "vitess_fracts", data: [][]string{ {"1", "1.99", "2.99", "3.99", "4.99"}, }, }, { input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')", - output: "insert into vitess_strings set vb='a', c='b', vc='c', b='d\\0\\0\\0', tb='e', bl='f', ttx='g', tx='h', en='1', s='3'", + output: "insert into vitess_strings(vb,c,vc,b,tb,bl,ttx,tx,en,s) values ('a','b','c','d\\0\\0\\0','e','f','g','h','1','3')", table: "vitess_strings", data: [][]string{ {"a", "b", "c", "d\x00\x00\x00", "e", "f", "g", "h", "a", "a,b"}, }, }, { input: "insert into vitess_misc values(1, '\x01', '2012-01-01', '2012-01-01 15:45:45', '15:45:45', point(1, 2))", - output: "insert into vitess_misc set id=1, b=b'00000001', d='2012-01-01', dt='2012-01-01 15:45:45', t='15:45:45', g='\\0\\0\\0\\0\x01\x01\\0\\0\\0\\0\\0\\0\\0\\0\\0\xf0?\\0\\0\\0\\0\\0\\0\\0@'", + output: "insert into vitess_misc(id,b,d,dt,t,g) values (1,b'00000001','2012-01-01','2012-01-01 15:45:45','15:45:45','\\0\\0\\0\\0\x01\x01\\0\\0\\0\\0\\0\\0\\0\\0\\0\xf0?\\0\\0\\0\\0\\0\\0\\0@')", table: "vitess_misc", data: [][]string{ {"1", "\x01", "2012-01-01", "2012-01-01 15:45:45", "15:45:45", "\x00\x00\x00\x00\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0?\x00\x00\x00\x00\x00\x00\x00@"}, }, }, { input: "insert into vitess_null values(1, null)", - output: "insert into vitess_null set id=1, val=null", + output: "insert into vitess_null(id,val) values (1,null)", table: "vitess_null", data: [][]string{ {"1", ""}, }, }, { input: "insert into binary_pk values('a', 'aaa')", - output: "insert into binary_pk set b='a\\0\\0\\0', val='aaa'", + output: "insert into binary_pk(b,val) values ('a\\0\\0\\0','aaa')", table: "binary_pk", data: [][]string{ {"a\x00\x00\x00", "aaa"}, @@ -560,12 +560,12 @@ func TestPlayerTypes(t *testing.T) { func TestPlayerDDL(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ - "create table dummy(id int, primary key(id))", - fmt.Sprintf("create table %s.dummy(id int, primary key(id))", vrepldb), + "create table t1(id int, primary key(id))", + fmt.Sprintf("create table %s.t1(id int, primary key(id))", vrepldb), }) defer execStatements(t, []string{ - "drop table dummy", - fmt.Sprintf("drop table %s.dummy", vrepldb), + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), }) env.SchemaEngine.Reload(context.Background()) @@ -580,23 +580,26 @@ func TestPlayerDDL(t *testing.T) { // is a race between the DDLs and the schema loader of vstreamer. // Root cause seems to be with MySQL where t1 shows up in information_schema before // the actual table is created. - execStatements(t, []string{"insert into dummy values(1)"}) + execStatements(t, []string{"insert into t1 values(1)"}) expectDBClientQueries(t, []string{ "begin", - "insert into dummy set id=1", + "insert into t1(id) values (1)", "/update _vt.vreplication set pos=", "commit", }) - execStatements(t, []string{"create table t1(id int, primary key(id))"}) - execStatements(t, []string{"drop table t1"}) - expectDBClientQueries(t, []string{}) + execStatements(t, []string{"alter table t1 add column val varchar(128)"}) + execStatements(t, []string{"alter table t1 drop column val"}) + expectDBClientQueries(t, []string{ + "/update _vt.vreplication set pos=", + "/update _vt.vreplication set pos=", + }) cancel() cancel, id := startVReplication(t, filter, binlogdatapb.OnDDLAction_STOP, "") - execStatements(t, []string{"create table t1(id int, primary key(id))"}) + execStatements(t, []string{"alter table t1 add column val varchar(128)"}) pos1 := masterPosition(t) - execStatements(t, []string{"drop table t1"}) + execStatements(t, []string{"alter table t1 drop column val"}) pos2 := masterPosition(t) // The stop position must be the GTID of the first DDL expectDBClientQueries(t, []string{ @@ -612,6 +615,7 @@ func TestPlayerDDL(t *testing.T) { // It should stop at the next DDL expectDBClientQueries(t, []string{ "/update.*'Running'", + // Second update is from vreplicator. "/update.*'Running'", "begin", fmt.Sprintf("/update.*'%s'", pos2), @@ -620,49 +624,39 @@ func TestPlayerDDL(t *testing.T) { }) cancel() - execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) + execStatements(t, []string{fmt.Sprintf("alter table %s.t1 add column val2 varchar(128)", vrepldb)}) cancel, _ = startVReplication(t, filter, binlogdatapb.OnDDLAction_EXEC, "") - execStatements(t, []string{"create table t1(id int, primary key(id))"}) + execStatements(t, []string{"alter table t1 add column val1 varchar(128)"}) expectDBClientQueries(t, []string{ - "create table t1(id int, primary key(id))", + "alter table t1 add column val1 varchar(128)", "/update _vt.vreplication set pos=", }) - execStatements(t, []string{"create table t2(id int, primary key(id))"}) + execStatements(t, []string{"alter table t1 add column val2 varchar(128)"}) expectDBClientQueries(t, []string{ - "create table t2(id int, primary key(id))", - "/update _vt.vreplication set state='Error'", + "alter table t1 add column val2 varchar(128)", + "/update _vt.vreplication set message='Duplicate", }) cancel() - // Don't test drop. - // MySQL rewrites them by uppercasing, which may be version specific. execStatements(t, []string{ - "drop table t1", - fmt.Sprintf("drop table %s.t1", vrepldb), - "drop table t2", - fmt.Sprintf("drop table %s.t2", vrepldb), + "alter table t1 drop column val1", + "alter table t1 drop column val2", + fmt.Sprintf("alter table %s.t1 drop column val1", vrepldb), }) execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) cancel, _ = startVReplication(t, filter, binlogdatapb.OnDDLAction_EXEC_IGNORE, "") - execStatements(t, []string{"create table t1(id int, primary key(id))"}) + execStatements(t, []string{"alter table t1 add column val1 varchar(128)"}) expectDBClientQueries(t, []string{ - "create table t1(id int, primary key(id))", + "alter table t1 add column val1 varchar(128)", "/update _vt.vreplication set pos=", }) - execStatements(t, []string{"create table t2(id int, primary key(id))"}) + execStatements(t, []string{"alter table t1 add column val2 varchar(128)"}) expectDBClientQueries(t, []string{ - "create table t2(id int, primary key(id))", + "alter table t1 add column val2 varchar(128)", "/update _vt.vreplication set pos=", }) cancel() - - execStatements(t, []string{ - "drop table t1", - fmt.Sprintf("drop table %s.t1", vrepldb), - "drop table t2", - fmt.Sprintf("drop table %s.t2", vrepldb), - }) } func TestPlayerStopPos(t *testing.T) { @@ -692,7 +686,7 @@ func TestPlayerStopPos(t *testing.T) { OnDdl: binlogdatapb.OnDDLAction_IGNORE, } startPos := masterPosition(t) - query := binlogplayer.CreateVReplicationStopped("test", bls, startPos) + query := binlogplayer.CreateVReplicationState("test", bls, startPos, binlogplayer.BlpStopped, vrepldb) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) @@ -714,10 +708,11 @@ func TestPlayerStopPos(t *testing.T) { t.Fatal(err) } expectDBClientQueries(t, []string{ - "/update.*'Running'", // done by Engine - "/update.*'Running'", // done by vplayer on start + "/update.*'Running'", + // Second update is from vreplicator. + "/update.*'Running'", "begin", - "insert into yes set id=1, val='aaa'", + "insert into yes(id,val) values (1,'aaa')", fmt.Sprintf("/update.*'%s'", stopPos), "/update.*'Stopped'", "commit", @@ -737,8 +732,9 @@ func TestPlayerStopPos(t *testing.T) { t.Fatal(err) } expectDBClientQueries(t, []string{ - "/update.*'Running'", // done by Engine - "/update.*'Running'", // done by vplayer on start + "/update.*'Running'", + // Second update is from vreplicator. + "/update.*'Running'", "begin", // Since 'no' generates empty transactions that are skipped by // vplayer, a commit is done only for the stop position event. @@ -753,8 +749,9 @@ func TestPlayerStopPos(t *testing.T) { t.Fatal(err) } expectDBClientQueries(t, []string{ - "/update.*'Running'", // done by Engine - "/update.*'Running'", // done by vplayer on start + "/update.*'Running'", + // Second update is from vreplicator. + "/update.*'Running'", "/update.*'Stopped'.*already reached", }) } @@ -790,7 +787,7 @@ func TestPlayerIdleUpdate(t *testing.T) { start := time.Now() expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='aaa'", + "insert into t1(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }) @@ -802,7 +799,7 @@ func TestPlayerIdleUpdate(t *testing.T) { expectDBClientQueries(t, []string{ "/update _vt.vreplication set pos=", }) - if duration := time.Now().Sub(start); duration < idleTimeout { + if duration := time.Since(start); duration < idleTimeout { t.Errorf("duration: %v, must be at least %v", duration, idleTimeout) } } @@ -840,8 +837,8 @@ func TestPlayerSplitTransaction(t *testing.T) { // but still combined as one transaction. expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='123456'", - "insert into t1 set id=2, val='789012'", + "insert into t1(id,val) values (1,'123456')", + "insert into t1(id,val) values (2,'789012')", "/update _vt.vreplication set pos=", "commit", }) @@ -876,8 +873,8 @@ func TestPlayerLockErrors(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='aaa'", - "insert into t1 set id=2, val='bbb'", + "insert into t1(id,val) values (1,'aaa')", + "insert into t1(id,val) values (2,'bbb')", "/update _vt.vreplication set pos=", "commit", }) @@ -949,7 +946,7 @@ func TestPlayerCancelOnLock(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='aaa'", + "insert into t1(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }) @@ -1019,7 +1016,7 @@ func TestPlayerBatching(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='aaa'", + "insert into t1(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }) @@ -1051,8 +1048,8 @@ func TestPlayerBatching(t *testing.T) { execStatements(t, []string{ "insert into t1 values(2, 'aaa')", "insert into t1 values(3, 'aaa')", - "create table t2(id int, val varbinary(128), primary key(id))", - "drop table t2", + "alter table t1 add column val2 varbinary(128)", + "alter table t1 drop column val2", }) // Release the lock. @@ -1065,13 +1062,13 @@ func TestPlayerBatching(t *testing.T) { "/update _vt.vreplication set pos=", "commit", "begin", - "insert into t1 set id=2, val='aaa'", - "insert into t1 set id=3, val='aaa'", + "insert into t1(id,val) values (2,'aaa')", + "insert into t1(id,val) values (3,'aaa')", "/update _vt.vreplication set pos=", "commit", - "create table t2(id int, val varbinary(128), primary key(id))", + "alter table t1 add column val2 varbinary(128)", "/update _vt.vreplication set pos=", - "/", // drop table is rewritten by mysql. Don't check. + "alter table t1 drop column val2", "/update _vt.vreplication set pos=", }) } @@ -1116,7 +1113,7 @@ func TestPlayerRelayLogMaxSize(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='123456'", + "insert into t1(id,val) values (1,'123456')", "/update _vt.vreplication set pos=", "commit", }) @@ -1162,12 +1159,12 @@ func TestPlayerRelayLogMaxSize(t *testing.T) { "/update _vt.vreplication set pos=", "commit", "begin", - "insert into t1 set id=2, val='789012'", - "insert into t1 set id=3, val='345678'", + "insert into t1(id,val) values (2,'789012')", + "insert into t1(id,val) values (3,'345678')", "/update _vt.vreplication set pos=", "commit", "begin", - "insert into t1 set id=4, val='901234'", + "insert into t1(id,val) values (4,'901234')", "/update _vt.vreplication set pos=", "commit", }) @@ -1205,14 +1202,14 @@ func TestRestartOnVStreamEnd(t *testing.T) { }) expectDBClientQueries(t, []string{ "begin", - "insert into t1 set id=1, val='aaa'", + "insert into t1(id,val) values (1,'aaa')", "/update _vt.vreplication set pos=", "commit", }) streamerEngine.Close() expectDBClientQueries(t, []string{ - "/update.*'Error'.*vstream ended", + "/update _vt.vreplication set message='vstream ended'", }) if err := streamerEngine.Open(env.KeyspaceName, env.ShardName); err != nil { t.Fatal(err) @@ -1222,9 +1219,9 @@ func TestRestartOnVStreamEnd(t *testing.T) { "insert into t1 values(2, 'aaa')", }) expectDBClientQueries(t, []string{ - "/update.*'Running'", + "/update _vt.vreplication set state='Running'", "begin", - "insert into t1 set id=2, val='aaa'", + "insert into t1(id,val) values (2,'aaa')", "/update _vt.vreplication set pos=", "commit", }) @@ -1265,7 +1262,7 @@ func TestTimestamp(t *testing.T) { "begin", // The insert value for ts will be in UTC. // We'll check the row instead. - "/insert into t1 set id=", + "/insert into t1", "/update _vt.vreplication set pos=", "commit", }) @@ -1292,17 +1289,15 @@ func startVReplication(t *testing.T, filter *binlogdatapb.Filter, onddl binlogda if pos == "" { pos = masterPosition(t) } - query := binlogplayer.CreateVReplication("test", bls, pos, 9223372036854775807, 9223372036854775807, 0) + query := binlogplayer.CreateVReplication("test", bls, pos, 9223372036854775807, 9223372036854775807, 0, vrepldb) qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) } - // Eat all the initialization queries - for q := range globalDBQueries { - if strings.HasPrefix(q, "update") { - break - } - } + expectDBClientQueries(t, []string{ + "/insert into _vt.vreplication", + "/update _vt.vreplication set state='Running'", + }) return func() { t.Helper() query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go new file mode 100644 index 00000000000..108d1c8efe0 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -0,0 +1,187 @@ +/* +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 vreplication + +import ( + "fmt" + "strings" + "time" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/mysqlctl" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +var ( + // idleTimeout is set to slightly above 1s, compared to heartbeatTime + // set by VStreamer at slightly below 1s. This minimizes conflicts + // between the two timeouts. + idleTimeout = 1100 * time.Millisecond + dbLockRetryDelay = 1 * time.Second + relayLogMaxSize = 30000 + relayLogMaxItems = 1000 + copyTimeout = 1 * time.Hour + replicaLagTolerance = 10 * time.Second + + // CreateCopyState is the list of statements to execute for creating + // the _vt.copy_state table + CreateCopyState = []string{ + `create table if not exists _vt.copy_state ( + vrepl_id int, + table_name varbinary(128), + lastpk varbinary(2000), + primary key (vrepl_id, table_name))`} +) + +type vreplicator struct { + id uint32 + source *binlogdatapb.BinlogSource + sourceTablet *topodatapb.Tablet + stats *binlogplayer.Stats + dbClient *vdbClient + // mysqld is used to fetch the local schema. + mysqld mysqlctl.MysqlDaemon + + tableKeys map[string][]string +} + +func newVReplicator(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon) *vreplicator { + return &vreplicator{ + id: id, + source: source, + sourceTablet: sourceTablet, + stats: stats, + dbClient: newVDBClient(dbClient, stats), + mysqld: mysqld, + } +} + +func (vr *vreplicator) Replicate(ctx context.Context) error { + tableKeys, err := vr.buildTableKeys() + if err != nil { + return err + } + vr.tableKeys = tableKeys + + for { + settings, numTablesToCopy, err := vr.readSettings(ctx) + if err != nil { + return fmt.Errorf("error reading VReplication settings: %v", err) + } + // If any of the operations below changed state to Stopped, we should return. + if settings.State == binlogplayer.BlpStopped { + return nil + } + switch { + case numTablesToCopy != 0: + if err := newVCopier(vr).copyNext(ctx, settings); err != nil { + return err + } + case settings.StartPos.IsZero(): + if err := newVCopier(vr).initTablesForCopy(ctx); err != nil { + return err + } + default: + if err := vr.setState(binlogplayer.BlpRunning, ""); err != nil { + return err + } + return newVPlayer(vr, settings, nil, mysql.Position{}).play(ctx) + } + } +} + +func (vr *vreplicator) buildTableKeys() (map[string][]string, error) { + schema, err := vr.mysqld.GetSchema(vr.dbClient.DBName(), []string{"/.*/"}, nil, false) + if err != nil { + return nil, err + } + tableKeys := make(map[string][]string) + for _, td := range schema.TableDefinitions { + if len(td.PrimaryKeyColumns) != 0 { + tableKeys[td.Name] = td.PrimaryKeyColumns + } else { + tableKeys[td.Name] = td.Columns + } + } + return tableKeys, nil +} + +func (vr *vreplicator) readSettings(ctx context.Context) (settings binlogplayer.VRSettings, numTablesToCopy int64, err error) { + settings, err = binlogplayer.ReadVRSettings(vr.dbClient, vr.id) + if err != nil { + return settings, numTablesToCopy, fmt.Errorf("error reading VReplication settings: %v", err) + } + + query := fmt.Sprintf("select count(*) from _vt.copy_state where vrepl_id=%d", vr.id) + qr, err := vr.dbClient.ExecuteFetch(query, 10) + if err != nil { + // If it's a not found error, create it. + merr, isSQLErr := err.(*mysql.SQLError) + if !isSQLErr || !(merr.Num == mysql.ERNoSuchTable || merr.Num == mysql.ERBadDb) { + return settings, numTablesToCopy, err + } + log.Info("Looks like _vt.copy_state table may not exist. Trying to create... ") + for _, query := range CreateCopyState { + if _, merr := vr.dbClient.ExecuteFetch(query, 0); merr != nil { + log.Errorf("Failed to ensure _vt.copy_state table exists: %v", merr) + return settings, numTablesToCopy, err + } + } + // Redo the read. + qr, err = vr.dbClient.ExecuteFetch(query, 10) + if err != nil { + return settings, numTablesToCopy, err + } + } + if len(qr.Rows) == 0 || len(qr.Rows[0]) == 0 { + return settings, numTablesToCopy, fmt.Errorf("unexpected result from %s: %v", query, qr) + } + numTablesToCopy, err = sqltypes.ToInt64(qr.Rows[0][0]) + if err != nil { + return settings, numTablesToCopy, err + } + return settings, numTablesToCopy, nil +} + +func (vr *vreplicator) setMessage(message string) error { + vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ + Time: time.Now(), + Message: message, + }) + query := fmt.Sprintf("update _vt.vreplication set message=%v where id=%v", encodeString(message), vr.id) + if _, err := vr.dbClient.ExecuteFetch(query, 1); err != nil { + return fmt.Errorf("could not set message: %v: %v", query, err) + } + return nil +} + +func (vr *vreplicator) setState(state, message string) error { + return binlogplayer.SetVReplicationState(vr.dbClient, vr.id, state, message) +} + +func encodeString(in string) string { + var buf strings.Builder + sqltypes.NewVarChar(in).EncodeSQL(&buf) + return buf.String() +} diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn.go b/go/vt/vttablet/tabletserver/connpool/dbconn.go index abeef3e68b2..aee833ce3a7 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn.go @@ -371,7 +371,7 @@ func (dbc *DBConn) setDeadline(ctx context.Context) (chan bool, *sync.WaitGroup) case <-done: return } - elapsed := time.Now().Sub(startTime) + elapsed := time.Since(startTime) // Give 2x the elapsed time and some buffer as grace period // for the query to get killed. diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go index 594f2d80d3a..aad986ee70d 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go @@ -65,7 +65,9 @@ func TestDBConnExec(t *testing.T) { ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(10*time.Second)) defer cancel() dbConn, err := NewDBConn(connPool, db.ConnParams()) - defer dbConn.Close() + if dbConn != nil { + defer dbConn.Close() + } if err != nil { t.Fatalf("should not get an error, err: %v", err) } @@ -142,7 +144,9 @@ func TestDBConnDeadline(t *testing.T) { defer cancel() dbConn, err := NewDBConn(connPool, db.ConnParams()) - defer dbConn.Close() + if dbConn != nil { + defer dbConn.Close() + } if err != nil { t.Fatalf("should not get an error, err: %v", err) } @@ -200,7 +204,12 @@ func TestDBConnKill(t *testing.T) { connPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer connPool.Close() dbConn, err := NewDBConn(connPool, db.ConnParams()) - defer dbConn.Close() + if dbConn != nil { + defer dbConn.Close() + } + if err != nil { + t.Fatalf("should not get an error, err: %v", err) + } query := fmt.Sprintf("kill %d", dbConn.ID()) db.AddQuery(query, &sqltypes.Result{}) // Kill failed because we are not able to connect to the database @@ -237,9 +246,13 @@ func TestDBNoPoolConnKill(t *testing.T) { connPool := newPool() connPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer connPool.Close() - defer db.Close() dbConn, err := NewDBConnNoPool(db.ConnParams(), connPool.dbaPool) - defer dbConn.Close() + if dbConn != nil { + defer dbConn.Close() + } + if err != nil { + t.Fatalf("should not get an error, err: %v", err) + } query := fmt.Sprintf("kill %d", dbConn.ID()) db.AddQuery(query, &sqltypes.Result{}) // Kill failed because we are not able to connect to the database @@ -290,7 +303,12 @@ func TestDBConnStream(t *testing.T) { ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(10*time.Second)) defer cancel() dbConn, err := NewDBConn(connPool, db.ConnParams()) - defer dbConn.Close() + if dbConn != nil { + defer dbConn.Close() + } + if err != nil { + t.Fatalf("should not get an error, err: %v", err) + } var result sqltypes.Result err = dbConn.Stream( ctx, sql, func(r *sqltypes.Result) error { diff --git a/go/vt/vttablet/tabletserver/controller.go b/go/vt/vttablet/tabletserver/controller.go index afffe54b661..2d7a8c10fce 100644 --- a/go/vt/vttablet/tabletserver/controller.go +++ b/go/vt/vttablet/tabletserver/controller.go @@ -76,7 +76,7 @@ type Controller interface { SchemaEngine() *schema.Engine // BroadcastHealth sends the current health to all listeners - BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats) + BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats, maxCache time.Duration) // HeartbeatLag returns the current lag as calculated by the heartbeat // package, if heartbeat is enabled. Otherwise returns 0. diff --git a/go/vt/vttablet/tabletserver/messager/message_manager_test.go b/go/vt/vttablet/tabletserver/messager/message_manager_test.go index 08a58ef198b..ce7979e2e4d 100644 --- a/go/vt/vttablet/tabletserver/messager/message_manager_test.go +++ b/go/vt/vttablet/tabletserver/messager/message_manager_test.go @@ -614,7 +614,7 @@ func TestMessagesPending1(t *testing.T) { for i := 0; i < 4; i++ { <-r1.ch } - if d := time.Now().Sub(start); d > 15*time.Second { + if d := time.Since(start); d > 15*time.Second { t.Errorf("pending work trigger did not happen. Duration: %v", d) } } @@ -663,7 +663,7 @@ func TestMessagesPending2(t *testing.T) { for i := 0; i < 3; i++ { <-r1.ch } - if d := time.Now().Sub(start); d > 15*time.Second { + if d := time.Since(start); d > 15*time.Second { t.Errorf("pending work trigger did not happen. Duration: %v", d) } } diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go index 0dc8809e4fa..65aa41c71a1 100644 --- a/go/vt/vttablet/tabletserver/planbuilder/plan.go +++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go @@ -245,9 +245,8 @@ func (plan *Plan) setTable(tableName sqlparser.TableIdent, tables map[string]*sc // Build builds a plan based on the schema. func Build(statement sqlparser.Statement, tables map[string]*schema.Table) (*Plan, error) { var plan *Plan - var err error - err = checkForPoolingUnsafeConstructs(statement) + err := checkForPoolingUnsafeConstructs(statement) if err != nil { return nil, err } diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go index 69d0079d90f..d196d1df9d9 100644 --- a/go/vt/vttablet/tabletserver/query_executor.go +++ b/go/vt/vttablet/tabletserver/query_executor.go @@ -296,12 +296,14 @@ func (qre *QueryExecutor) execDmlAutoCommit() (reply *sqltypes.Result, err error } func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltypes.Result, error)) (reply *sqltypes.Result, err error) { - conn, err := qre.tsv.te.txPool.LocalBegin(qre.ctx, qre.options) + 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) - qre.logStats.AddRewrittenSQL("begin", time.Now()) + if beginSQL != "" { + qre.logStats.AddRewrittenSQL(beginSQL, time.Now()) + } reply, err = f(conn) @@ -311,8 +313,13 @@ func (qre *QueryExecutor) execAsTransaction(f func(conn *TxConnection) (*sqltype qre.logStats.AddRewrittenSQL("rollback", start) return nil, err } - err = qre.tsv.te.txPool.LocalCommit(qre.ctx, conn, qre.tsv.messager) - qre.logStats.AddRewrittenSQL("commit", start) + commitSQL, err := qre.tsv.te.txPool.LocalCommit(qre.ctx, conn, qre.tsv.messager) + + // 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 } diff --git a/go/vt/vttablet/tabletserver/rules/map_test.go b/go/vt/vttablet/tabletserver/rules/map_test.go index f6bfeac061d..1e788e228fb 100644 --- a/go/vt/vttablet/tabletserver/rules/map_test.go +++ b/go/vt/vttablet/tabletserver/rules/map_test.go @@ -29,11 +29,12 @@ var ( otherRules *Rules ) -// mimic query rules from blacklist -const blacklistQueryRules string = "BLACKLIST_QUERY_RULES" - -// mimic query rules from custom source -const customQueryRules string = "CUSTOM_QUERY_RULES" +const ( + // mimic query rules from blacklist + blacklistQueryRules string = "BLACKLIST_QUERY_RULES" + // mimic query rules from custom source + customQueryRules string = "CUSTOM_QUERY_RULES" +) func setupRules() { var qr *Rule diff --git a/go/vt/vttablet/tabletserver/rules/rules.go b/go/vt/vttablet/tabletserver/rules/rules.go index 8ce0ce7f581..d934046bbe3 100644 --- a/go/vt/vttablet/tabletserver/rules/rules.go +++ b/go/vt/vttablet/tabletserver/rules/rules.go @@ -72,9 +72,7 @@ func (qrs *Rules) Copy() (newqrs *Rules) { // Append merges the rules from another Rules into the receiver func (qrs *Rules) Append(otherqrs *Rules) { - for _, qr := range otherqrs.rules { - qrs.rules = append(qrs.rules, qr) - } + qrs.rules = append(qrs.rules, otherqrs.rules...) } // Add adds a Rule to Rules. It does not check diff --git a/go/vt/vttablet/tabletserver/rules/rules_test.go b/go/vt/vttablet/tabletserver/rules/rules_test.go index dcd79b420c2..1a389f10e30 100644 --- a/go/vt/vttablet/tabletserver/rules/rules_test.go +++ b/go/vt/vttablet/tabletserver/rules/rules_test.go @@ -276,8 +276,10 @@ func TestQueryRule(t *testing.T) { func TestBindVarStruct(t *testing.T) { qr := NewQueryRule("rule 1", "r1", QRFail) - var err error - err = qr.AddBindVarCond("b", false, true, QRNoOp, nil) + err := qr.AddBindVarCond("b", false, true, QRNoOp, nil) + if err != nil { + t.Errorf("unexpected: %v", err) + } err = qr.AddBindVarCond("a", true, false, QRNoOp, nil) if err != nil { t.Errorf("unexpected: %v", err) @@ -508,7 +510,7 @@ func TestAction(t *testing.T) { if desc != "rule 2" { t.Errorf("want rule 2, got %s", desc) } - action, desc = qrs.GetAction("1234", "user1", bv) + action, _ = qrs.GetAction("1234", "user1", bv) if action != QRContinue { t.Errorf("want continue") } diff --git a/go/vt/vttablet/tabletserver/schema/engine.go b/go/vt/vttablet/tabletserver/schema/engine.go index 630d117d940..5ca2d5573c8 100644 --- a/go/vt/vttablet/tabletserver/schema/engine.go +++ b/go/vt/vttablet/tabletserver/schema/engine.go @@ -113,7 +113,7 @@ func (se *Engine) Open() error { return nil } start := time.Now() - defer log.Infof("Time taken to load the schema: %v", time.Now().Sub(start)) + defer log.Infof("Time taken to load the schema: %v", time.Since(start)) ctx := tabletenv.LocalContext() dbaParams := se.dbconfigs.DbaWithDB() se.conns.Open(dbaParams, dbaParams, dbaParams) diff --git a/go/vt/vttablet/tabletserver/schema/load_table.go b/go/vt/vttablet/tabletserver/schema/load_table.go index 95d8a739c1c..3257af030d0 100644 --- a/go/vt/vttablet/tabletserver/schema/load_table.go +++ b/go/vt/vttablet/tabletserver/schema/load_table.go @@ -87,7 +87,7 @@ func fetchColumns(ta *Table, conn *connpool.DBConn, sqlTableName string) error { } if len(r.Rows) != 1 || len(r.Rows[0]) != 1 { // This code is unreachable. - return fmt.Errorf("Invalid rows returned from %s: %v", query, r.Rows) + return fmt.Errorf("invalid rows returned from %s: %v", query, r.Rows) } // overwrite the original value with the new one. row[4] = r.Rows[0][0] @@ -219,7 +219,7 @@ func loadMessageInfo(ta *Table, comment string) error { func getDuration(in map[string]string, key string) (time.Duration, error) { sv := in[key] if sv == "" { - return 0, fmt.Errorf("Attribute %s not specified for message table", key) + return 0, fmt.Errorf("attribute %s not specified for message table", key) } v, err := strconv.ParseFloat(sv, 64) if err != nil { @@ -231,7 +231,7 @@ func getDuration(in map[string]string, key string) (time.Duration, error) { func getNum(in map[string]string, key string) (int, error) { sv := in[key] if sv == "" { - return 0, fmt.Errorf("Attribute %s not specified for message table", key) + return 0, fmt.Errorf("attribute %s not specified for message table", key) } v, err := strconv.Atoi(sv) if err != nil { diff --git a/go/vt/vttablet/tabletserver/splitquery/split_params.go b/go/vt/vttablet/tabletserver/splitquery/split_params.go index f7e9b314435..2cc020bcf15 100644 --- a/go/vt/vttablet/tabletserver/splitquery/split_params.go +++ b/go/vt/vttablet/tabletserver/splitquery/split_params.go @@ -174,7 +174,7 @@ func newSplitParams( " (must be a simple table expression): %v", query.Sql) } tableSchema, ok := schemaMap[tableName.String()] - if tableSchema == nil { + if !ok || tableSchema == nil { return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "can't find table in schema") } diff --git a/go/vt/vttablet/tabletserver/splitquery/splitter_test.go b/go/vt/vttablet/tabletserver/splitquery/splitter_test.go index 31670d911ce..0f23abdfad9 100644 --- a/go/vt/vttablet/tabletserver/splitquery/splitter_test.go +++ b/go/vt/vttablet/tabletserver/splitquery/splitter_test.go @@ -563,7 +563,7 @@ func TestWithRealEqualSplits(t *testing.T) { }, }, nil) - equalSplits, err := NewEqualSplitsAlgorithm(splitParams, mockSQLExecuter) + equalSplits, _ := NewEqualSplitsAlgorithm(splitParams, mockSQLExecuter) splitter := NewSplitter(splitParams, equalSplits) queryParts, err := splitter.Split() if err != nil { diff --git a/go/vt/vttablet/tabletserver/tabletenv/logstats.go b/go/vt/vttablet/tabletserver/tabletenv/logstats.go index 6e8323400c3..a117668b33b 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/logstats.go +++ b/go/vt/vttablet/tabletserver/tabletenv/logstats.go @@ -103,7 +103,7 @@ func (stats *LogStats) AddRewrittenSQL(sql string, start time.Time) { stats.QuerySources |= QuerySourceMySQL stats.NumberOfQueries++ stats.rewrittenSqls = append(stats.rewrittenSqls, sql) - stats.MysqlResponseTime += time.Now().Sub(start) + stats.MysqlResponseTime += time.Since(start) } // TotalTime returns how long this query has been running diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 003b27708a0..8af8653743b 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -30,7 +30,6 @@ import ( "time" "golang.org/x/net/context" - "vitess.io/vitess/go/acl" "vitess.io/vitess/go/history" "vitess.io/vitess/go/mysql" @@ -44,6 +43,10 @@ import ( "vitess.io/vitess/go/vt/dbconnpool" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/srvtopo" "vitess.io/vitess/go/vt/tableacl" @@ -61,11 +64,6 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/txserializer" "vitess.io/vitess/go/vt/vttablet/tabletserver/txthrottler" "vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer" - - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" - topodatapb "vitess.io/vitess/go/vt/proto/topodata" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) const ( @@ -184,10 +182,11 @@ type TabletServer struct { topoServer *topo.Server // streamHealthMutex protects all the following fields - streamHealthMutex sync.Mutex - streamHealthIndex int - streamHealthMap map[int]chan<- *querypb.StreamHealthResponse - lastStreamHealthResponse *querypb.StreamHealthResponse + streamHealthMutex sync.Mutex + streamHealthIndex int + streamHealthMap map[int]chan<- *querypb.StreamHealthResponse + lastStreamHealthResponse *querypb.StreamHealthResponse + lastStreamHealthExpiration time.Time // history records changes in state for display on the status page. // It has its own internal mutex. @@ -237,12 +236,15 @@ type TxPoolController interface { // 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. + // 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, error) + Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, string, error) - // Commit commits the specified transaction. - Commit(ctx context.Context, transactionID int64, mc messageCommitter) error + // Commit commits the specified transaction, returning the statement used to execute + // the commit or "" in autocommit settings. + Commit(ctx context.Context, transactionID int64, mc messageCommitter) (string, error) // Rollback rolls back the specified transaction. Rollback(ctx context.Context, transactionID int64) error @@ -773,13 +775,25 @@ func (tsv *TabletServer) Begin(ctx context.Context, target *querypb.Target, opti "Begin", "begin", nil, target, options, true /* isBegin */, false, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { - defer tabletenv.QueryStats.Record("BEGIN", time.Now()) + startTime := time.Now() if tsv.txThrottler.Throttle() { // TODO(erez): I think this should be RESOURCE_EXHAUSTED. return vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "Transaction throttled") } - transactionID, err = tsv.teCtrl.Begin(ctx, options) + var beginSQL string + transactionID, beginSQL, err = tsv.teCtrl.Begin(ctx, options) logStats.TransactionID = transactionID + + // Record the actual statements that were executed in the logStats. + // If nothing was actually executed, don't count the operation in + // the tablet metrics, and clear out the logStats Method so that + // handlePanicAndSendLogStats doesn't log the no-op. + logStats.OriginalSQL = beginSQL + if beginSQL != "" { + tabletenv.QueryStats.Record("BEGIN", startTime) + } else { + logStats.Method = "" + } return err }, ) @@ -793,9 +807,21 @@ func (tsv *TabletServer) Commit(ctx context.Context, target *querypb.Target, tra "Commit", "commit", nil, target, nil, false /* isBegin */, true, /* allowOnShutdown */ func(ctx context.Context, logStats *tabletenv.LogStats) error { - defer tabletenv.QueryStats.Record("COMMIT", time.Now()) + startTime := time.Now() logStats.TransactionID = transactionID - return tsv.teCtrl.Commit(ctx, transactionID, tsv.messager) + + var commitSQL string + commitSQL, err = tsv.teCtrl.Commit(ctx, transactionID, tsv.messager) + + // If nothing was actually executed, don't count the operation in + // the tablet metrics, and clear out the logStats Method so that + // handlePanicAndSendLogStats doesn't log the no-op. + if commitSQL != "" { + tabletenv.QueryStats.Record("COMMIT", startTime) + } else { + logStats.Method = "" + } + return err }, ) } @@ -1325,38 +1351,28 @@ func (tsv *TabletServer) execDML(ctx context.Context, target *querypb.Target, qu // VStream streams VReplication events. func (tsv *TabletServer) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - // This code is partially duplicated from startRequest. This is because - // is allowed even if the tablet is in non-serving state. - err := func() error { - tsv.mu.Lock() - defer tsv.mu.Unlock() - - if target != nil { - // a valid target needs to be used - switch { - case target.Keyspace != tsv.target.Keyspace: - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid keyspace %v", target.Keyspace) - case target.Shard != tsv.target.Shard: - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid shard %v", target.Shard) - case target.TabletType != tsv.target.TabletType: - for _, otherType := range tsv.alsoAllow { - if target.TabletType == otherType { - return nil - } - } - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "invalid tablet type: %v, want: %v or %v", target.TabletType, tsv.target.TabletType, tsv.alsoAllow) - } - } else if !tabletenv.IsLocalContext(ctx) { - return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No target") - } - return nil - }() - if err != nil { + if err := tsv.verifyTarget(ctx, target); err != nil { return err } return tsv.vstreamer.Stream(ctx, startPos, filter, send) } +// VStreamRows streams rows from the specified starting point. +func (tsv *TabletServer) VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { + if err := tsv.verifyTarget(ctx, target); err != nil { + return err + } + var row []sqltypes.Value + if lastpk != nil { + r := sqltypes.Proto3ToResult(lastpk) + if len(r.Rows) != 1 { + return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unexpected lastpk input: %v", lastpk) + } + row = r.Rows[0] + } + return tsv.vstreamer.StreamRows(ctx, query, row, send) +} + // SplitQuery splits a query + bind variables into smaller queries that return a // subset of rows from the original query. This is the new version that supports multiple // split columns and multiple split algortihms. @@ -1450,6 +1466,32 @@ func (tsv *TabletServer) execRequest( return nil } +// verifyTarget allows requests to be executed even in non-serving state. +func (tsv *TabletServer) verifyTarget(ctx context.Context, target *querypb.Target) error { + tsv.mu.Lock() + defer tsv.mu.Unlock() + + if target != nil { + // a valid target needs to be used + switch { + case target.Keyspace != tsv.target.Keyspace: + return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid keyspace %v", target.Keyspace) + case target.Shard != tsv.target.Shard: + return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid shard %v", target.Shard) + case target.TabletType != tsv.target.TabletType: + for _, otherType := range tsv.alsoAllow { + if target.TabletType == otherType { + return nil + } + } + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "invalid tablet type: %v, want: %v or %v", target.TabletType, tsv.target.TabletType, tsv.alsoAllow) + } + } else if !tabletenv.IsLocalContext(ctx) { + return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No target") + } + return nil +} + func (tsv *TabletServer) handlePanicAndSendLogStats( sql string, bindVariables map[string]*querypb.BindVariable, @@ -1471,6 +1513,7 @@ func (tsv *TabletServer) handlePanicAndSendLogStats( // Examples where we don't send the log stats: // - ExecuteBatch() (logStats == nil) // - beginWaitForSameRangeTransactions() (Method == "") + // - Begin / Commit in autocommit mode if logStats != nil && logStats.Method != "" { logStats.Send() } @@ -1807,9 +1850,10 @@ func createSplitQueryAlgorithmObject( func (tsv *TabletServer) StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error { tsv.streamHealthMutex.Lock() shr := tsv.lastStreamHealthResponse + shrExpiration := tsv.lastStreamHealthExpiration tsv.streamHealthMutex.Unlock() // Send current state immediately. - if shr != nil { + if shr != nil && time.Now().Before(shrExpiration) { if err := callback(shr); err != nil { if err == io.EOF { return nil @@ -1855,14 +1899,14 @@ func (tsv *TabletServer) streamHealthUnregister(id int) { } // BroadcastHealth will broadcast the current health to all listeners -func (tsv *TabletServer) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats) { +func (tsv *TabletServer) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats, maxCache time.Duration) { tsv.mu.Lock() target := tsv.target tsv.mu.Unlock() shr := &querypb.StreamHealthResponse{ - Target: &target, - TabletAlias: &tsv.alias, - Serving: tsv.IsServing(), + Target: &target, + TabletAlias: &tsv.alias, + Serving: tsv.IsServing(), TabletExternallyReparentedTimestamp: terTimestamp, RealtimeStats: stats, } @@ -1877,6 +1921,7 @@ func (tsv *TabletServer) BroadcastHealth(terTimestamp int64, stats *querypb.Real } } tsv.lastStreamHealthResponse = shr + tsv.lastStreamHealthExpiration = time.Now().Add(maxCache) } // HeartbeatLag returns the current lag as calculated by the heartbeat diff --git a/go/vt/vttablet/tabletserver/tx_engine.go b/go/vt/vttablet/tabletserver/tx_engine.go index cb928f9a934..9db9a56ef5e 100644 --- a/go/vt/vttablet/tabletserver/tx_engine.go +++ b/go/vt/vttablet/tabletserver/tx_engine.go @@ -264,22 +264,24 @@ func (te *TxEngine) AcceptReadOnly() error { } } -// Begin begins a transaction, and returns the associated transaction id. +// 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. -func (te *TxEngine) Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, error) { +func (te *TxEngine) Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, string, error) { te.stateLock.Lock() canOpenTransactions := te.state == AcceptingReadOnly || te.state == AcceptingReadAndWrite if !canOpenTransactions { // We are not in a state where we can start new transactions. Abort. te.stateLock.Unlock() - return 0, vterrors.Errorf(vtrpc.Code_UNAVAILABLE, "tx engine can't accept new transactions in state %v", te.state) + return 0, "", vterrors.Errorf(vtrpc.Code_UNAVAILABLE, "tx engine can't accept new transactions in state %v", te.state) } isWriteTransaction := options == nil || options.TransactionIsolation != querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY if te.state == AcceptingReadOnly && isWriteTransaction { te.stateLock.Unlock() - return 0, vterrors.Errorf(vtrpc.Code_UNAVAILABLE, "tx engine can only accept read-only transactions in current state") + return 0, "", vterrors.Errorf(vtrpc.Code_UNAVAILABLE, "tx engine can only accept read-only transactions in current state") } // By Add() to beginRequests, we block others from initiating state @@ -292,7 +294,7 @@ func (te *TxEngine) Begin(ctx context.Context, options *querypb.ExecuteOptions) } // Commit commits the specified transaction. -func (te *TxEngine) Commit(ctx context.Context, transactionID int64, mc messageCommitter) error { +func (te *TxEngine) Commit(ctx context.Context, transactionID int64, mc messageCommitter) (string, error) { return te.txPool.Commit(ctx, transactionID, mc) } @@ -466,7 +468,7 @@ outer: if txid > maxid { maxid = txid } - conn, err := te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + conn, _, err := te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { allErr.RecordError(err) continue diff --git a/go/vt/vttablet/tabletserver/tx_engine_test.go b/go/vt/vttablet/tabletserver/tx_engine_test.go index cb7c235e41c..3c209a49bf4 100644 --- a/go/vt/vttablet/tabletserver/tx_engine_test.go +++ b/go/vt/vttablet/tabletserver/tx_engine_test.go @@ -55,10 +55,13 @@ func TestTxEngineClose(t *testing.T) { // Normal close with timeout wait. te.open() - c, err := te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + c, beginSQL, err := te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } + if beginSQL != "begin" { + t.Errorf("beginSQL: %q, want 'begin'", beginSQL) + } c.Recycle() start = time.Now() te.close(false) @@ -68,7 +71,7 @@ func TestTxEngineClose(t *testing.T) { // Immediate close. te.open() - c, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + c, beginSQL, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -82,7 +85,7 @@ func TestTxEngineClose(t *testing.T) { // Normal close with short grace period. te.shutdownGracePeriod = 250 * time.Millisecond te.open() - c, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + c, beginSQL, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -99,7 +102,7 @@ func TestTxEngineClose(t *testing.T) { // Normal close with short grace period, but pool gets empty early. te.shutdownGracePeriod = 250 * time.Millisecond te.open() - c, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + c, beginSQL, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -123,7 +126,7 @@ func TestTxEngineClose(t *testing.T) { // Immediate close, but connection is in use. te.open() - c, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + c, beginSQL, err = te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -492,6 +495,6 @@ func startTransaction(te *TxEngine, writeTransaction bool) error { } else { options.TransactionIsolation = querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY } - _, err := te.Begin(context.Background(), options) + _, _, err := te.Begin(context.Background(), options) return err } diff --git a/go/vt/vttablet/tabletserver/tx_executor.go b/go/vt/vttablet/tabletserver/tx_executor.go index f9d00174c40..e485174745f 100644 --- a/go/vt/vttablet/tabletserver/tx_executor.go +++ b/go/vt/vttablet/tabletserver/tx_executor.go @@ -68,7 +68,7 @@ func (txe *TxExecutor) Prepare(transactionID int64, dtid string) error { return vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "prepare failed for transaction %d: %v", transactionID, err) } - localConn, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) + localConn, _, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) if err != nil { return err } @@ -79,7 +79,7 @@ func (txe *TxExecutor) Prepare(transactionID int64, dtid string) error { return err } - err = txe.te.txPool.LocalCommit(txe.ctx, localConn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, localConn, txe.messager) if err != nil { return err } @@ -111,7 +111,7 @@ func (txe *TxExecutor) CommitPrepared(dtid string) error { txe.markFailed(ctx, dtid) return err } - err = txe.te.txPool.LocalCommit(ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(ctx, conn, txe.messager) if err != nil { txe.markFailed(ctx, dtid) return err @@ -130,7 +130,7 @@ func (txe *TxExecutor) CommitPrepared(dtid string) error { func (txe *TxExecutor) markFailed(ctx context.Context, dtid string) { tabletenv.InternalErrors.Add("TwopcCommit", 1) txe.te.preparedPool.SetFailed(dtid) - conn, err := txe.te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + conn, _, err := txe.te.txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { log.Errorf("markFailed: Begin failed for dtid %s: %v", dtid, err) return @@ -142,7 +142,7 @@ func (txe *TxExecutor) markFailed(ctx context.Context, dtid string) { return } - if err = txe.te.txPool.LocalCommit(ctx, conn, txe.messager); err != nil { + if _, err = txe.te.txPool.LocalCommit(ctx, conn, txe.messager); err != nil { log.Errorf("markFailed: Commit failed for dtid %s: %v", dtid, err) } } @@ -170,7 +170,7 @@ func (txe *TxExecutor) RollbackPrepared(dtid string, originalID int64) error { return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled") } defer tabletenv.QueryStats.Record("ROLLBACK_PREPARED", time.Now()) - conn, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) + conn, _, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) if err != nil { goto returnConn } @@ -181,7 +181,7 @@ func (txe *TxExecutor) RollbackPrepared(dtid string, originalID int64) error { goto returnConn } - err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) returnConn: if preparedConn := txe.te.preparedPool.FetchForRollback(dtid); preparedConn != nil { @@ -200,7 +200,7 @@ func (txe *TxExecutor) CreateTransaction(dtid string, participants []*querypb.Ta return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled") } defer tabletenv.QueryStats.Record("CREATE_TRANSACTION", time.Now()) - conn, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) + conn, _, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) if err != nil { return err } @@ -210,7 +210,8 @@ func (txe *TxExecutor) CreateTransaction(dtid string, participants []*querypb.Ta if err != nil { return err } - return txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + return err } // StartCommit atomically commits the transaction along with the @@ -232,7 +233,8 @@ func (txe *TxExecutor) StartCommit(transactionID int64, dtid string) error { if err != nil { return err } - return txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + return err } // SetRollback transitions the 2pc transaction to the Rollback state. @@ -248,7 +250,7 @@ func (txe *TxExecutor) SetRollback(dtid string, transactionID int64) error { txe.te.txPool.Rollback(txe.ctx, transactionID) } - conn, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) + conn, _, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) if err != nil { return err } @@ -259,7 +261,7 @@ func (txe *TxExecutor) SetRollback(dtid string, transactionID int64) error { return err } - err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) if err != nil { return err } @@ -275,7 +277,7 @@ func (txe *TxExecutor) ConcludeTransaction(dtid string) error { } defer tabletenv.QueryStats.Record("RESOLVE", time.Now()) - conn, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) + conn, _, err := txe.te.txPool.LocalBegin(txe.ctx, &querypb.ExecuteOptions{}) if err != nil { return err } @@ -285,7 +287,8 @@ func (txe *TxExecutor) ConcludeTransaction(dtid string) error { if err != nil { return err } - return txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + _, err = txe.te.txPool.LocalCommit(txe.ctx, conn, txe.messager) + return err } // ReadTransaction returns the metadata for the sepcified dtid. diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go index 4d766643792..5f068fba724 100644 --- a/go/vt/vttablet/tabletserver/tx_pool.go +++ b/go/vt/vttablet/tabletserver/tx_pool.go @@ -192,23 +192,26 @@ func (axp *TxPool) WaitForEmpty() { axp.activePool.WaitForEmpty() } -// Begin begins a transaction, and returns the associated transaction id. +// Begin begins a transaction, and returns the associated transaction id and +// the statements (if any) executed to initiate the transaction. In autocommit +// mode the statement will be "". +// // Subsequent statements can access the connection through the transaction id. -func (axp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, error) { +func (axp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions) (int64, string, error) { var conn *connpool.DBConn var err error immediateCaller := callerid.ImmediateCallerIDFromContext(ctx) effectiveCaller := callerid.EffectiveCallerIDFromContext(ctx) if !axp.limiter.Get(immediateCaller, effectiveCaller) { - return 0, vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "per-user transaction pool connection limit exceeded") + return 0, "", vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "per-user transaction pool connection limit exceeded") } waiterCount := axp.waiters.Add(1) defer axp.waiters.Add(-1) if waiterCount > axp.waiterCap.Get() { - return 0, vterrors.New(vtrpcpb.Code_RESOURCE_EXHAUSTED, "transaction pool waiter count exceeded") + return 0, "", vterrors.New(vtrpcpb.Code_RESOURCE_EXHAUSTED, "transaction pool waiter count exceeded") } var beginSucceeded bool @@ -231,30 +234,33 @@ func (axp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions) ( if err != nil { switch err { case connpool.ErrConnPoolClosed: - return 0, err + return 0, "", err case pools.ErrTimeout: axp.LogActive() - return 0, vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "transaction pool connection limit exceeded") + return 0, "", vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "transaction pool connection limit exceeded") } - return 0, err + return 0, "", err } autocommitTransaction := false - + beginQueries := "" if queries, ok := txIsolations[options.GetTransactionIsolation()]; ok { if queries.setIsolationLevel != "" { if _, err := conn.Exec(ctx, "set transaction isolation level "+queries.setIsolationLevel, 1, false); err != nil { - return 0, err + return 0, "", err } + + beginQueries = queries.setIsolationLevel + "; " } if _, err := conn.Exec(ctx, queries.openTransaction, 1, false); err != nil { - return 0, err + return 0, "", err } + beginQueries = beginQueries + queries.openTransaction } else if options.GetTransactionIsolation() == querypb.ExecuteOptions_AUTOCOMMIT { autocommitTransaction = true } else { - return 0, fmt.Errorf("don't know how to open a transaction of this type: %v", options.GetTransactionIsolation()) + return 0, "", fmt.Errorf("don't know how to open a transaction of this type: %v", options.GetTransactionIsolation()) } beginSucceeded = true @@ -271,14 +277,14 @@ func (axp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions) ( ), options.GetWorkload() != querypb.ExecuteOptions_DBA, ) - return transactionID, nil + return transactionID, beginQueries, nil } // Commit commits the specified transaction. -func (axp *TxPool) Commit(ctx context.Context, transactionID int64, mc messageCommitter) error { +func (axp *TxPool) Commit(ctx context.Context, transactionID int64, mc messageCommitter) (string, error) { conn, err := axp.Get(transactionID, "for commit") if err != nil { - return err + return "", err } return axp.LocalCommit(ctx, conn, mc) } @@ -305,30 +311,31 @@ func (axp *TxPool) Get(transactionID int64, reason string) (*TxConnection, error // LocalBegin is equivalent to Begin->Get. // It's used for executing transactions within a request. It's safe // to always call LocalConclude at the end. -func (axp *TxPool) LocalBegin(ctx context.Context, options *querypb.ExecuteOptions) (*TxConnection, error) { - transactionID, err := axp.Begin(ctx, options) +func (axp *TxPool) LocalBegin(ctx context.Context, options *querypb.ExecuteOptions) (*TxConnection, string, error) { + transactionID, beginSQL, err := axp.Begin(ctx, options) if err != nil { - return nil, err + return nil, "", err } - return axp.Get(transactionID, "for local query") + conn, err := axp.Get(transactionID, "for local query") + return conn, beginSQL, err } // LocalCommit is the commit function for LocalBegin. -func (axp *TxPool) LocalCommit(ctx context.Context, conn *TxConnection, mc messageCommitter) error { +func (axp *TxPool) LocalCommit(ctx context.Context, conn *TxConnection, mc messageCommitter) (string, error) { defer conn.conclude(TxCommit, "transaction committed") defer mc.LockDB(conn.NewMessages, conn.ChangedMessages)() if conn.Autocommit { mc.UpdateCaches(conn.NewMessages, conn.ChangedMessages) - return nil + return "", nil } if _, err := conn.Exec(ctx, "commit", 1, false); err != nil { conn.Close() - return err + return "", err } mc.UpdateCaches(conn.NewMessages, conn.ChangedMessages) - return nil + return "commit", nil } // LocalConclude concludes a transaction started by LocalBegin. diff --git a/go/vt/vttablet/tabletserver/tx_pool_test.go b/go/vt/vttablet/tabletserver/tx_pool_test.go index d58b0357319..b1d54686b1d 100644 --- a/go/vt/vttablet/tabletserver/tx_pool_test.go +++ b/go/vt/vttablet/tabletserver/tx_pool_test.go @@ -40,6 +40,42 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/messager" ) +func TestTxPoolExecuteCommit(t *testing.T) { + sql := "update test_column set x=1 where 1!=1" + db := fakesqldb.New(t) + defer db.Close() + db.AddQuery(sql, &sqltypes.Result{}) + db.AddQuery("begin", &sqltypes.Result{}) + db.AddQuery("commit", &sqltypes.Result{}) + + txPool := newTxPool() + txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) + defer txPool.Close() + ctx := context.Background() + transactionID, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + if err != nil { + t.Fatal(err) + } + if beginSQL != "begin" { + t.Errorf("beginSQL got %q want 'begin'", beginSQL) + } + txConn, err := txPool.Get(transactionID, "for query") + if err != nil { + t.Fatal(err) + } + txConn.RecordQuery(sql) + _, err = txConn.Exec(ctx, sql, 1, true) + txConn.Recycle() + + commitSQL, err := txPool.Commit(ctx, transactionID, &fakeMessageCommitter{}) + if err != nil { + t.Fatal(err) + } + if commitSQL != "commit" { + t.Errorf("commitSQL got %q want 'commit'", commitSQL) + } +} + func TestTxPoolExecuteRollback(t *testing.T) { sql := "alter table test_table add test_column int" db := fakesqldb.New(t) @@ -52,10 +88,13 @@ func TestTxPoolExecuteRollback(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer txPool.Close() ctx := context.Background() - transactionID, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + transactionID, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } + if beginSQL != "begin" { + t.Errorf("beginSQL got %q want 'begin'", beginSQL) + } txConn, err := txPool.Get(transactionID, "for query") if err != nil { t.Fatal(err) @@ -79,11 +118,11 @@ func TestTxPoolRollbackNonBusy(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer txPool.Close() ctx := context.Background() - txid1, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + txid1, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } - _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) + _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -173,7 +212,7 @@ func TestTxPoolTransactionKillerEnforceTimeoutEnabled(t *testing.T) { } func addQuery(ctx context.Context, sql string, txPool *TxPool, workload querypb.ExecuteOptions_Workload) (int64, error) { - transactionID, err := txPool.Begin(ctx, &querypb.ExecuteOptions{Workload: workload}) + transactionID, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{Workload: workload}) if err != nil { return 0, err } @@ -199,10 +238,13 @@ func TestTxPoolClientRowsFound(t *testing.T) { // Start a 'normal' transaction. It should take a connection // for the normal 'conns' pool. - id1, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + id1, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } + if beginSQL != "begin" { + t.Errorf("beginSQL got %q want 'begin'", beginSQL) + } if got, want := txPool.conns.Available(), startNormalSize-1; got != want { t.Errorf("Normal pool size: %d, want %d", got, want) } @@ -212,10 +254,13 @@ func TestTxPoolClientRowsFound(t *testing.T) { // Start a 'foundRows' transaction. It should take a connection // from the foundRows pool. - id2, err := txPool.Begin(ctx, &querypb.ExecuteOptions{ClientFoundRows: true}) + id2, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{ClientFoundRows: true}) if err != nil { t.Fatal(err) } + if beginSQL != "begin" { + t.Errorf("beginSQL got %q want 'begin'", beginSQL) + } if got, want := txPool.conns.Available(), startNormalSize-1; got != want { t.Errorf("Normal pool size: %d, want %d", got, want) } @@ -253,16 +298,23 @@ func TestTxPoolTransactionIsolation(t *testing.T) { ctx := context.Background() // Start a transaction with default. It should not change isolation. - _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + _, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } + if beginSQL != "begin" { + t.Errorf("beginSQL got %q want 'begin'", beginSQL) + } db.AddQuery("set transaction isolation level READ COMMITTED", &sqltypes.Result{}) - _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_READ_COMMITTED}) + _, beginSQL, err = txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_READ_COMMITTED}) if err != nil { t.Fatal(err) } + wantBeginSQL := "READ COMMITTED; begin" + if beginSQL != wantBeginSQL { + t.Errorf("beginSQL got %q want %q", beginSQL, wantBeginSQL) + } } func TestTxPoolAutocommit(t *testing.T) { @@ -276,14 +328,20 @@ func TestTxPoolAutocommit(t *testing.T) { // to mysql. // This test is meaningful because if txPool.Begin were to send a BEGIN statement to the connection, it will fatal // because is not in the list of expected queries (i.e db.AddQuery hasn't been called). - txid, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_AUTOCOMMIT}) + txid, beginSQL, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_AUTOCOMMIT}) if err != nil { t.Fatal(err) } - err = txPool.Commit(ctx, txid, &fakeMessageCommitter{}) + if beginSQL != "" { + t.Errorf("beginSQL got %q want ''", beginSQL) + } + commitSQL, err := txPool.Commit(ctx, txid, &fakeMessageCommitter{}) if err != nil { t.Fatal(err) } + if commitSQL != "" { + t.Errorf("commitSQL got %q want ''", commitSQL) + } } // TestTxPoolBeginWithPoolConnectionError_TransientErrno2006 tests the case @@ -305,7 +363,7 @@ func TestTxPoolBeginWithPoolConnectionError_Errno2006_Transient(t *testing.T) { } ctx := context.Background() - txConn, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + txConn, _, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatalf("Begin should have succeeded after the retry in DBConn.Exec(): %v", err) } @@ -336,7 +394,7 @@ func TestTxPoolBeginWithPoolConnectionError_Errno2006_Permanent(t *testing.T) { // After that, vttablet will automatically try to reconnect and this fail. // DBConn.Exec() will return the reconnect error as final error and not the // initial connection error. - _, err = txPool.LocalBegin(context.Background(), &querypb.ExecuteOptions{}) + _, _, err = txPool.LocalBegin(context.Background(), &querypb.ExecuteOptions{}) if err == nil || !strings.Contains(err.Error(), "(errno 2013)") { t.Fatalf("Begin did not return the reconnect error: %v", err) } @@ -362,7 +420,7 @@ func TestTxPoolBeginWithPoolConnectionError_Errno2013(t *testing.T) { db.EnableShouldClose() // 2013 is not retryable. DBConn.Exec() fails after the first attempt. - _, err = txPool.Begin(context.Background(), &querypb.ExecuteOptions{}) + _, _, err = txPool.Begin(context.Background(), &querypb.ExecuteOptions{}) if err == nil || !strings.Contains(err.Error(), "(errno 2013)") { t.Fatalf("Begin must return connection error with MySQL errno 2013: %v", err) } @@ -385,7 +443,7 @@ func primeTxPoolWithConnection(t *testing.T) (*fakesqldb.DB, *TxPool, error) { db.AddQuery("begin", &sqltypes.Result{}) db.AddQuery("rollback", &sqltypes.Result{}) ctx := context.Background() - txConn, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + txConn, _, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { return nil, nil, err } @@ -402,7 +460,7 @@ func TestTxPoolBeginWithError(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer txPool.Close() ctx := context.Background() - _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) want := "error: rejected" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("Begin: %v, want %s", err, want) @@ -424,7 +482,7 @@ func TestTxPoolRollbackFail(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer txPool.Close() ctx := context.Background() - transactionID, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + transactionID, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -467,7 +525,7 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) { db.AddQuery("rollback", &sqltypes.Result{}) txPool := newTxPool() txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) - id, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) + id, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}) txPool.Close() assertErrorMatch := func(id int64, reason string) { @@ -486,14 +544,14 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) { txPool = newTxPool() txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) - id, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) - if err := txPool.Commit(ctx, id, &fakeMessageCommitter{}); err != nil { + id, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) + if _, err := txPool.Commit(ctx, id, &fakeMessageCommitter{}); err != nil { t.Fatalf("got error: %v", err) } assertErrorMatch(id, "transaction committed") - id, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) + id, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) if err := txPool.Rollback(ctx, id); err != nil { t.Fatalf("got error: %v", err) } @@ -506,13 +564,13 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) defer txPool.Close() - id, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) + id, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) time.Sleep(5 * time.Millisecond) assertErrorMatch(id, "exceeded timeout: 1ms") txPool.SetTimeout(1 * time.Hour) - id, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) + id, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}) txc, err := txPool.Get(id, "for close") if err != nil { t.Fatalf("got error: %v", err) @@ -545,7 +603,7 @@ func TestTxPoolExecFailDueToConnFail_Errno2006(t *testing.T) { ctx := context.Background() // Start the transaction. - txConn, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + txConn, _, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -583,7 +641,7 @@ func TestTxPoolExecFailDueToConnFail_Errno2013(t *testing.T) { ctx := context.Background() // Start the transaction. - txConn, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) + txConn, _, err := txPool.LocalBegin(ctx, &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } @@ -615,7 +673,7 @@ func TestTxPoolCloseKillsStrayTransactions(t *testing.T) { txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams()) // Start stray transaction. - _, err := txPool.Begin(context.Background(), &querypb.ExecuteOptions{}) + _, _, err := txPool.Begin(context.Background(), &querypb.ExecuteOptions{}) if err != nil { t.Fatal(err) } diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go index c3c022ded99..5bdb0b75f90 100644 --- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go +++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go @@ -201,7 +201,7 @@ func newTxThrottler(config *txThrottlerConfig) (*TxThrottler, error) { return nil, err } if len(config.healthCheckCells) == 0 { - return nil, fmt.Errorf("Empty healthCheckCells given. %+v", config) + return nil, fmt.Errorf("empty healthCheckCells given. %+v", config) } } return &TxThrottler{ @@ -215,7 +215,7 @@ func (t *TxThrottler) Open(keyspace, shard string) error { return nil } if t.state != nil { - return fmt.Errorf("Transaction throttler already opened") + return fmt.Errorf("transaction throttler already opened") } var err error t.state, err = newTxThrottlerState(t.config, keyspace, shard) diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index 9e0e014bd95..7147289fa20 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -26,6 +26,7 @@ import ( "vitess.io/vitess/go/acl" "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/log" @@ -55,9 +56,10 @@ type Engine struct { isOpen bool // wg is incremented for every Stream, and decremented on end. // Close waits for all current streams to end by waiting on wg. - wg sync.WaitGroup - streamers map[int]*vstreamer - streamIdx int + wg sync.WaitGroup + streamers map[int]*vstreamer + rowStreamers map[int]*rowStreamer + streamIdx int // watcherOnce is used for initializing kschema // and setting up the vschema watch. It's guaranteed that @@ -78,10 +80,11 @@ type Engine struct { // Open and Close can be called multiple times and are idempotent. func NewEngine(ts srvtopo.Server, se *schema.Engine) *Engine { vse := &Engine{ - streamers: make(map[int]*vstreamer), - kschema: &vindexes.KeyspaceSchema{}, - ts: ts, - se: se, + streamers: make(map[int]*vstreamer), + rowStreamers: make(map[int]*rowStreamer), + kschema: &vindexes.KeyspaceSchema{}, + ts: ts, + se: se, } once.Do(func() { vschemaErrors = stats.NewCounter("VSchemaErrors", "Count of VSchema errors") @@ -121,6 +124,10 @@ func (vse *Engine) Close() { // cancel is non-blocking. s.Cancel() } + for _, s := range vse.rowStreamers { + // cancel is non-blocking. + s.Cancel() + } vse.isOpen = false }() @@ -174,6 +181,46 @@ func (vse *Engine) Stream(ctx context.Context, startPos string, filter *binlogda return streamer.Stream() } +// StreamRows streams rows. +func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltypes.Value, send func(*binlogdatapb.VStreamRowsResponse) error) error { + // Ensure kschema is initialized and the watcher is started. + // Starting of the watcher has to be delayed till the first call to Stream + // because this overhead should be incurred only if someone uses this feature. + vse.watcherOnce.Do(vse.setWatch) + log.Infof("Streaming rows for query %s, lastpk: %s", query, lastpk) + + // Create stream and add it to the map. + rowStreamer, idx, err := func() (*rowStreamer, int, error) { + vse.mu.Lock() + defer vse.mu.Unlock() + if !vse.isOpen { + return nil, 0, errors.New("VStreamer is not open") + } + rowStreamer := newRowStreamer(ctx, vse.cp, vse.se, query, lastpk, vse.kschema, send) + idx := vse.streamIdx + vse.rowStreamers[idx] = rowStreamer + vse.streamIdx++ + // Now that we've added the stream, increment wg. + // This must be done before releasing the lock. + vse.wg.Add(1) + return rowStreamer, idx, nil + }() + if err != nil { + return err + } + + // Remove stream from map and decrement wg when it ends. + defer func() { + vse.mu.Lock() + defer vse.mu.Unlock() + delete(vse.rowStreamers, idx) + vse.wg.Done() + }() + + // No lock is held while streaming, but wg is incremented. + return rowStreamer.Stream() +} + // ServeHTTP shows the current VSchema. func (vse *Engine) ServeHTTP(response http.ResponseWriter, request *http.Request) { if err := acl.CheckAccessHTTP(request, acl.DEBUGGING); err != nil { diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go index 296f7bb40b2..a9e593c0136 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go @@ -33,7 +33,7 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -// Plan represents the streaming plan for a table. +// Plan represents the plan for a table. type Plan struct { Table *Table ColExprs []ColExpr @@ -49,19 +49,26 @@ type ColExpr struct { Type querypb.Type } -// Table contains the metadata for a table. The -// name is dervied from mysql's Table_map_log_event. +// Table contains the metadata for a table. type Table struct { - *mysql.TableMap + Name string Columns []schema.TableColumn } -// The filter function needs the ability to perform expression evaluations. This is -// because the consumer of vstream is not just VPlayer. It can also be a dumb client -// like a mysql client that's subscribing to changes. This ability to allow users -// to directly pull events by sending a complex select query. The same reasoning -// applies to where clauses. For now, only simple functions like hour are supported, -// but this can be expanded in the future. +// fields returns the fields for the plan. +func (plan *Plan) fields() []*querypb.Field { + fields := make([]*querypb.Field, len(plan.ColExprs)) + for i, ce := range plan.ColExprs { + fields[i] = &querypb.Field{ + Name: ce.Alias.String(), + Type: ce.Type, + } + } + return fields +} + +// filter filters the row against the plan. It returns false if the row did not match. +// If the row matched, it returns the columns to be sent. func (plan *Plan) filter(values []sqltypes.Value) (bool, []sqltypes.Value, error) { result := make([]sqltypes.Value, len(plan.ColExprs)) for i, colExpr := range plan.ColExprs { @@ -211,133 +218,175 @@ func buildREPlan(ti *Table, kschema *vindexes.KeyspaceSchema, filter string) (*P } func buildTablePlan(ti *Table, kschema *vindexes.KeyspaceSchema, query string) (*Plan, error) { - statement, err := sqlparser.Parse(query) + sel, fromTable, err := analyzeSelect(query) if err != nil { return nil, err } + if fromTable.String() != ti.Name { + return nil, fmt.Errorf("unsupported: select expression table %v does not match the table entry name %s", sqlparser.String(fromTable), ti.Name) + } + plan := &Plan{ Table: ti, } + if err := plan.analyzeExprs(sel.SelectExprs); err != nil { + return nil, err + } + + if sel.Where == nil { + return plan, nil + } + + funcExpr, ok := sel.Where.Expr.(*sqlparser.FuncExpr) + if !ok { + return nil, fmt.Errorf("unsupported where clause: %v", sqlparser.String(sel.Where)) + } + if !funcExpr.Name.EqualString("in_keyrange") { + return nil, fmt.Errorf("unsupported where clause: %v", sqlparser.String(sel.Where)) + } + if err := plan.analyzeInKeyRange(kschema, funcExpr.Exprs); err != nil { + return nil, err + } + return plan, nil +} + +func analyzeSelect(query string) (sel *sqlparser.Select, fromTable sqlparser.TableIdent, err error) { + statement, err := sqlparser.Parse(query) + if err != nil { + return nil, fromTable, err + } sel, ok := statement.(*sqlparser.Select) if !ok { - return nil, fmt.Errorf("unsupported: %v", sqlparser.String(statement)) + return nil, fromTable, fmt.Errorf("unsupported: %v", sqlparser.String(statement)) } if len(sel.From) > 1 { - return nil, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) + return nil, fromTable, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) } node, ok := sel.From[0].(*sqlparser.AliasedTableExpr) if !ok { - return nil, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) + return nil, fromTable, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) } - fromTable := sqlparser.GetTableName(node.Expr) + fromTable = sqlparser.GetTableName(node.Expr) if fromTable.IsEmpty() { - return nil, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) - } - if fromTable.String() != ti.Name { - return nil, fmt.Errorf("unsupported: select expression table %v does not match the table entry name %s", sqlparser.String(fromTable), ti.Name) + return nil, fromTable, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) } + return sel, fromTable, nil +} - if _, ok := sel.SelectExprs[0].(*sqlparser.StarExpr); !ok { - for _, expr := range sel.SelectExprs { - cExpr, err := analyzeExpr(ti, expr) +func (plan *Plan) analyzeExprs(selExprs sqlparser.SelectExprs) error { + if _, ok := selExprs[0].(*sqlparser.StarExpr); !ok { + for _, expr := range selExprs { + cExpr, err := plan.analyzeExpr(expr) if err != nil { - return nil, err + return err } plan.ColExprs = append(plan.ColExprs, cExpr) } } else { - if len(sel.SelectExprs) != 1 { - return nil, fmt.Errorf("unsupported: %v", sqlparser.String(sel)) + if len(selExprs) != 1 { + return fmt.Errorf("unsupported: %v", sqlparser.String(selExprs)) } - plan.ColExprs = make([]ColExpr, len(ti.Columns)) - for i, col := range ti.Columns { + plan.ColExprs = make([]ColExpr, len(plan.Table.Columns)) + for i, col := range plan.Table.Columns { plan.ColExprs[i].ColNum = i plan.ColExprs[i].Alias = col.Name plan.ColExprs[i].Type = col.Type } } + return nil +} - if sel.Where == nil { - return plan, nil - } - - // Filter by Vindex. - funcExpr, ok := sel.Where.Expr.(*sqlparser.FuncExpr) +func (plan *Plan) analyzeExpr(selExpr sqlparser.SelectExpr) (cExpr ColExpr, err error) { + aliased, ok := selExpr.(*sqlparser.AliasedExpr) if !ok { - return nil, fmt.Errorf("unsupported where clause: %v", sqlparser.String(sel.Where)) - } - if !funcExpr.Name.EqualString("in_keyrange") { - return nil, fmt.Errorf("unsupported where clause: %v", sqlparser.String(sel.Where)) + return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(selExpr)) } - if len(funcExpr.Exprs) != 3 { - return nil, fmt.Errorf("unexpected where clause: %v", sqlparser.String(sel.Where)) + as := aliased.As + if as.IsEmpty() { + as = sqlparser.NewColIdent(sqlparser.String(aliased.Expr)) } - aexpr, ok := funcExpr.Exprs[0].(*sqlparser.AliasedExpr) + colname, ok := aliased.Expr.(*sqlparser.ColName) if !ok { - return nil, fmt.Errorf("unexpected: %v", sqlparser.String(funcExpr)) + return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(aliased.Expr)) } - colname, ok := aexpr.Expr.(*sqlparser.ColName) - if !ok { - return nil, fmt.Errorf("unexpected: %v", sqlparser.String(funcExpr)) + if !colname.Qualifier.IsEmpty() { + return ColExpr{}, fmt.Errorf("unsupported qualifier for column: %v", sqlparser.String(colname)) + } + colnum, err := findColumn(plan.Table, colname.Name) + if err != nil { + return ColExpr{}, err + } + return ColExpr{ColNum: colnum, Alias: as, Type: plan.Table.Columns[colnum].Type}, nil +} + +func (plan *Plan) analyzeInKeyRange(kschema *vindexes.KeyspaceSchema, exprs sqlparser.SelectExprs) error { + var colname sqlparser.ColIdent + var krExpr sqlparser.SelectExpr + switch len(exprs) { + case 1: + table := kschema.Tables[plan.Table.Name] + if table == nil { + return fmt.Errorf("no vschema definition for table %s", plan.Table.Name) + } + // Get Primary Vindex. + if len(table.ColumnVindexes) == 0 { + return fmt.Errorf("table %s has no primary vindex", plan.Table.Name) + } + colname = table.ColumnVindexes[0].Columns[0] + plan.Vindex = table.ColumnVindexes[0].Vindex + krExpr = exprs[0] + case 3: + aexpr, ok := exprs[0].(*sqlparser.AliasedExpr) + if !ok { + return fmt.Errorf("unexpected: %v", sqlparser.String(exprs[0])) + } + qualifiedName, ok := aexpr.Expr.(*sqlparser.ColName) + if !ok { + return fmt.Errorf("unexpected: %v", sqlparser.String(exprs[0])) + } + if !qualifiedName.Qualifier.IsEmpty() { + return fmt.Errorf("unsupported qualifier for column: %v", sqlparser.String(colname)) + } + colname = qualifiedName.Name + vtype, err := selString(exprs[1]) + if err != nil { + return err + } + plan.Vindex, err = vindexes.CreateVindex(vtype, vtype, map[string]string{}) + if err != nil { + return err + } + if !plan.Vindex.IsUnique() || !plan.Vindex.IsFunctional() { + return fmt.Errorf("vindex must be Unique and Functional to be used for VReplication: %s", vtype) + } + krExpr = exprs[2] + default: + return fmt.Errorf("unexpected in_keyrange parameters: %v", sqlparser.String(exprs)) } found := false for i, cExpr := range plan.ColExprs { - if cExpr.Alias.Equal(colname.Name) { + if cExpr.Alias.Equal(colname) { found = true plan.VindexColumn = i break } } if !found { - return nil, fmt.Errorf("keyrange expression does not reference a column in the select list: %v", sqlparser.String(colname)) - } - vtype, err := selString(funcExpr.Exprs[1]) - if err != nil { - return nil, err - } - plan.Vindex, err = vindexes.CreateVindex(vtype, vtype, map[string]string{}) - if err != nil { - return nil, err - } - if !plan.Vindex.IsUnique() || !plan.Vindex.IsFunctional() { - return nil, fmt.Errorf("vindex must be Unique and Functional to be used for VReplication: %s", vtype) + return fmt.Errorf("keyrange expression does not reference a column in the select list: %v", sqlparser.String(colname)) } - kr, err := selString(funcExpr.Exprs[2]) + kr, err := selString(krExpr) if err != nil { - return nil, err + return err } keyranges, err := key.ParseShardingSpec(kr) if err != nil { - return nil, err + return err } if len(keyranges) != 1 { - return nil, fmt.Errorf("unexpected where clause: %v", sqlparser.String(sel.Where)) + return fmt.Errorf("unexpected in_keyrange parameter: %v", sqlparser.String(krExpr)) } plan.KeyRange = keyranges[0] - return plan, nil -} - -func analyzeExpr(ti *Table, selExpr sqlparser.SelectExpr) (cExpr ColExpr, err error) { - aliased, ok := selExpr.(*sqlparser.AliasedExpr) - if !ok { - return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(selExpr)) - } - as := aliased.As - if as.IsEmpty() { - as = sqlparser.NewColIdent(sqlparser.String(aliased.Expr)) - } - colname, ok := aliased.Expr.(*sqlparser.ColName) - if !ok { - return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(aliased.Expr)) - } - if !colname.Qualifier.IsEmpty() { - return ColExpr{}, fmt.Errorf("unsupported qualifier for column: %v", sqlparser.String(colname)) - } - colnum, err := findColumn(ti, colname.Name) - if err != nil { - return ColExpr{}, err - } - return ColExpr{ColNum: colnum, Alias: as, Type: ti.Columns[colnum].Type}, nil + return nil } func selString(expr sqlparser.SelectExpr) (string, error) { diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go index 23a896eff78..5d46fb2eede 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go @@ -150,9 +150,7 @@ func TestMustSendDDL(t *testing.T) { func TestPlanbuilder(t *testing.T) { t1 := &Table{ - TableMap: &mysql.TableMap{ - Name: "t1", - }, + Name: "t1", Columns: []schema.TableColumn{{ Name: sqlparser.NewColIdent("id"), Type: sqltypes.Int64, @@ -163,18 +161,14 @@ func TestPlanbuilder(t *testing.T) { } // t1alt has no id column t1alt := &Table{ - TableMap: &mysql.TableMap{ - Name: "t1", - }, + Name: "t1", Columns: []schema.TableColumn{{ Name: sqlparser.NewColIdent("val"), Type: sqltypes.VarBinary, }}, } t2 := &Table{ - TableMap: &mysql.TableMap{ - Name: "t2", - }, + Name: "t2", Columns: []schema.TableColumn{{ Name: sqlparser.NewColIdent("id"), Type: sqltypes.Int64, @@ -275,6 +269,21 @@ func TestPlanbuilder(t *testing.T) { }}, VindexColumn: 1, }, + }, { + inTable: t1, + inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select val, id from t1 where in_keyrange('-80')"}, + outPlan: &Plan{ + ColExprs: []ColExpr{{ + ColNum: 1, + Alias: sqlparser.NewColIdent("val"), + Type: sqltypes.VarBinary, + }, { + ColNum: 0, + Alias: sqlparser.NewColIdent("id"), + Type: sqltypes.Int64, + }}, + VindexColumn: 1, + }, }, { inTable: t2, inRule: &binlogdatapb.Rule{Match: "/t1/"}, @@ -325,7 +334,7 @@ func TestPlanbuilder(t *testing.T) { }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select *, id from t1"}, - outErr: `unsupported: select *, id from t1`, + outErr: `unsupported: *, id`, }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where id=1"}, @@ -337,15 +346,15 @@ func TestPlanbuilder(t *testing.T) { }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where in_keyrange(id)"}, - outErr: `unexpected where clause: where in_keyrange(id)`, + outErr: `unsupported: id`, }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where in_keyrange(*, 'hash', '-80')"}, - outErr: `unexpected: in_keyrange(*, 'hash', '-80')`, + outErr: `unexpected: *`, }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where in_keyrange(1, 'hash', '-80')"}, - outErr: `unexpected: in_keyrange(1, 'hash', '-80')`, + outErr: `unexpected: 1`, }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where in_keyrange(none, 'hash', '-80')"}, @@ -361,7 +370,7 @@ func TestPlanbuilder(t *testing.T) { }, { inTable: t1, inRule: &binlogdatapb.Rule{Match: "t1", Filter: "select id, val from t1 where in_keyrange(id, 'hash', '-80-')"}, - outErr: `unexpected where clause: where in_keyrange(id, 'hash', '-80-')`, + outErr: `unexpected in_keyrange parameter: '-80-'`, }, { // analyzeExpr tests. inTable: t1, @@ -405,6 +414,8 @@ func TestPlanbuilder(t *testing.T) { if !reflect.DeepEqual(tcase.outPlan, plan) { t.Errorf("Plan(%v, %v):\n%v, want\n%v", tcase.inTable, tcase.inRule, plan, tcase.outPlan) } + } else if tcase.outPlan != nil { + t.Errorf("Plan(%v, %v):\nnil, want\n%v", tcase.inTable, tcase.inRule, tcase.outPlan) } gotErr := "" if err != nil { diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go new file mode 100644 index 00000000000..a0f1fc5f56e --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -0,0 +1,297 @@ +/* +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 vstreamer + +import ( + "context" + "fmt" + "sync" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/vindexes" + "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" +) + +type rowStreamer struct { + ctx context.Context + cancel func() + + cp *mysql.ConnParams + se *schema.Engine + query string + lastpk []sqltypes.Value + send func(*binlogdatapb.VStreamRowsResponse) error + kschema *vindexes.KeyspaceSchema + + plan *Plan + pkColumns []int + sendQuery string +} + +func newRowStreamer(ctx context.Context, cp *mysql.ConnParams, se *schema.Engine, query string, lastpk []sqltypes.Value, kschema *vindexes.KeyspaceSchema, send func(*binlogdatapb.VStreamRowsResponse) error) *rowStreamer { + ctx, cancel := context.WithCancel(ctx) + return &rowStreamer{ + ctx: ctx, + cancel: cancel, + cp: cp, + se: se, + query: query, + lastpk: lastpk, + send: send, + kschema: kschema, + } +} + +func (rs *rowStreamer) Cancel() { + rs.cancel() +} + +func (rs *rowStreamer) Stream() error { + // Ensure se is Open. If vttablet came up in a non_serving role, + // the schema engine may not have been initialized. + if err := rs.se.Open(); err != nil { + return err + } + + if err := rs.buildPlan(); err != nil { + return err + } + + conn, err := rs.mysqlConnect() + if err != nil { + return err + } + defer conn.Close() + return rs.streamQuery(conn, rs.send) +} + +func (rs *rowStreamer) buildPlan() error { + // This pre-parsing is required to extract the table name + // and create its metadata. + _, fromTable, err := analyzeSelect(rs.query) + if err != nil { + return err + } + st := rs.se.GetTable(fromTable) + if st == nil { + return fmt.Errorf("unknown table %v in schema", fromTable) + } + ti := &Table{ + Name: st.Name.String(), + Columns: st.Columns, + } + rs.plan, err = buildTablePlan(ti, rs.kschema, rs.query) + if err != nil { + return err + } + rs.pkColumns, err = buildPKColumns(st) + if err != nil { + return err + } + rs.sendQuery, err = rs.buildSelect() + if err != nil { + return err + } + return err +} + +func buildPKColumns(st *schema.Table) ([]int, error) { + if len(st.PKColumns) == 0 { + pkColumns := make([]int, len(st.Columns)) + for i := range st.Columns { + pkColumns[i] = i + } + return pkColumns, nil + } + for _, pk := range st.PKColumns { + if pk >= len(st.Columns) { + return nil, fmt.Errorf("primary key %d refers to non-existent column", pk) + } + } + return st.PKColumns, nil +} + +func (rs *rowStreamer) buildSelect() (string, error) { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select ") + prefix := "" + for _, col := range rs.plan.Table.Columns { + buf.Myprintf("%s%v", prefix, col.Name) + prefix = ", " + } + buf.Myprintf(" from %v", sqlparser.NewTableIdent(rs.plan.Table.Name)) + if len(rs.lastpk) != 0 { + if len(rs.lastpk) != len(rs.pkColumns) { + return "", fmt.Errorf("primary key values don't match length: %v vs %v", rs.lastpk, rs.pkColumns) + } + buf.WriteString(" where (") + prefix := "" + for _, pk := range rs.pkColumns { + buf.Myprintf("%s%v", prefix, rs.plan.Table.Columns[pk].Name) + prefix = "," + } + buf.WriteString(") > (") + prefix = "" + for _, val := range rs.lastpk { + buf.WriteString(prefix) + prefix = "," + val.EncodeSQL(buf) + } + buf.WriteString(")") + } + buf.Myprintf(" order by ", sqlparser.NewTableIdent(rs.plan.Table.Name)) + prefix = "" + for _, pk := range rs.pkColumns { + buf.Myprintf("%s%v", prefix, rs.plan.Table.Columns[pk].Name) + prefix = ", " + } + return buf.String(), nil +} + +func (rs *rowStreamer) streamQuery(conn *mysql.Conn, send func(*binlogdatapb.VStreamRowsResponse) error) error { + unlock, gtid, err := rs.lockTable() + if err != nil { + return err + } + defer unlock() + + if err := conn.ExecuteStreamFetch(rs.sendQuery); err != nil { + return err + } + + // first call the callback with the fields + flds, err := conn.Fields() + if err != nil { + return err + } + pkfields := make([]*querypb.Field, len(rs.pkColumns)) + for i, pk := range rs.pkColumns { + pkfields[i] = &querypb.Field{ + Name: flds[pk].Name, + Type: flds[pk].Type, + } + } + + err = send(&binlogdatapb.VStreamRowsResponse{ + Fields: rs.plan.fields(), + Pkfields: pkfields, + Gtid: gtid, + }) + if err != nil { + return fmt.Errorf("stream send error: %v", err) + } + if err := unlock(); err != nil { + return err + } + + response := &binlogdatapb.VStreamRowsResponse{} + lastpk := make([]sqltypes.Value, len(rs.pkColumns)) + byteCount := 0 + for { + select { + case <-rs.ctx.Done(): + return fmt.Errorf("stream ended: %v", rs.ctx.Err()) + default: + } + + row, err := conn.FetchNext() + if err != nil { + return err + } + if row == nil { + break + } + for i, pk := range rs.pkColumns { + lastpk[i] = row[pk] + } + ok, filtered, err := rs.plan.filter(row) + if err != nil { + return err + } + if ok { + response.Rows = append(response.Rows, sqltypes.RowToProto3(filtered)) + for _, s := range filtered { + byteCount += s.Len() + } + } + + if byteCount >= *packetSize { + response.Lastpk = sqltypes.RowToProto3(lastpk) + err = send(response) + if err != nil { + return err + } + // empty the rows so we start over, but we keep the + // same capacity + response.Rows = response.Rows[:0] + byteCount = 0 + } + } + + if len(response.Rows) > 0 { + response.Lastpk = sqltypes.RowToProto3(lastpk) + err = send(response) + if err != nil { + return err + } + } + + return nil +} + +func (rs *rowStreamer) lockTable() (unlock func() error, gtid string, err error) { + conn, err := rs.mysqlConnect() + if err != nil { + return nil, "", err + } + // mysql recommends this before locking tables. + if _, err := conn.ExecuteFetch("set autocommit=0", 0, false); err != nil { + return nil, "", err + } + if _, err := conn.ExecuteFetch(fmt.Sprintf("lock tables %s read", sqlparser.String(sqlparser.NewTableIdent(rs.plan.Table.Name))), 0, false); err != nil { + return nil, "", err + } + var once sync.Once + unlock = func() error { + var err error + once.Do(func() { + _, err = conn.ExecuteFetch("unlock tables", 0, false) + conn.Close() + }) + return err + } + pos, err := conn.MasterPosition() + if err != nil { + unlock() + return nil, "", err + } + return unlock, mysql.EncodePosition(pos), nil +} + +func (rs *rowStreamer) mysqlConnect() (*mysql.Conn, error) { + cp, err := dbconfigs.WithCredentials(rs.cp) + if err != nil { + return nil, err + } + return mysql.Connect(rs.ctx, cp) +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go new file mode 100644 index 00000000000..5612673dcbb --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go @@ -0,0 +1,237 @@ +/* +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 vstreamer + +import ( + "context" + "fmt" + "testing" + "time" + + "vitess.io/vitess/go/sqltypes" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +func TestStreamRowsScan(t *testing.T) { + if testing.Short() { + t.Skip() + } + + execStatements(t, []string{ + // Single PK + "create table t1(id int, val varbinary(128), primary key(id))", + "insert into t1 values (1, 'aaa'), (2, 'bbb')", + // Composite PK + "create table t2(id1 int, id2 int, val varbinary(128), primary key(id1, id2))", + "insert into t2 values (1, 2, 'aaa'), (1, 3, 'bbb')", + // No PK + "create table t3(id int, val varbinary(128))", + "insert into t3 values (1, 'aaa'), (2, 'bbb')", + }) + defer execStatements(t, []string{ + "drop table t1", + "drop table t2", + "drop table t3", + }) + engine.se.Reload(context.Background()) + + // t1: all rows + wantStream := []string{ + `fields: fields: pkfields: `, + `rows: rows: lastpk: `, + } + wantQuery := "select id, val from t1 order by id" + checkStream(t, "select * from t1", nil, wantQuery, wantStream) + + // t1: lastpk=1 + wantStream = []string{ + `fields: fields: pkfields: `, + `rows: lastpk: `, + } + wantQuery = "select id, val from t1 where (id) > (1) order by id" + checkStream(t, "select * from t1", []sqltypes.Value{sqltypes.NewInt64(1)}, wantQuery, wantStream) + + // t1: different column ordering + wantStream = []string{ + `fields: fields: pkfields: `, + `rows: rows: lastpk: `, + } + wantQuery = "select id, val from t1 order by id" + checkStream(t, "select val, id from t1", nil, wantQuery, wantStream) + + // t2: all rows + wantStream = []string{ + `fields: fields: fields: pkfields: pkfields: `, + `rows: rows: lastpk: `, + } + wantQuery = "select id1, id2, val from t2 order by id1, id2" + checkStream(t, "select * from t2", nil, wantQuery, wantStream) + + // t2: lastpk=1,2 + wantStream = []string{ + `fields: fields: fields: pkfields: pkfields: `, + `rows: lastpk: `, + } + wantQuery = "select id1, id2, val from t2 where (id1,id2) > (1,2) order by id1, id2" + checkStream(t, "select * from t2", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}, wantQuery, wantStream) + + // t3: all rows + wantStream = []string{ + `fields: fields: pkfields: pkfields: `, + `rows: rows: lastpk: `, + } + wantQuery = "select id, val from t3 order by id, val" + checkStream(t, "select * from t3", nil, wantQuery, wantStream) + + // t3: lastpk: 1,'aaa' + wantStream = []string{ + `fields: fields: pkfields: pkfields: `, + `rows: lastpk: `, + } + wantQuery = "select id, val from t3 where (id,val) > (1,'aaa') order by id, val" + checkStream(t, "select * from t3", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewVarBinary("aaa")}, wantQuery, wantStream) +} + +func TestStreamRowsKeyRange(t *testing.T) { + if testing.Short() { + t.Skip() + } + + if err := env.SetVSchema(shardedVSchema); err != nil { + t.Fatal(err) + } + defer env.SetVSchema("{}") + + execStatements(t, []string{ + "create table t1(id1 int, val varbinary(128), primary key(id1))", + "insert into t1 values (1, 'aaa'), (6, 'bbb')", + }) + defer execStatements(t, []string{ + "drop table t1", + }) + engine.se.Reload(context.Background()) + + time.Sleep(1 * time.Second) + + // Only the first row should be returned, but lastpk should be 6. + wantStream := []string{ + `fields: fields: pkfields: `, + `rows: lastpk: `, + } + wantQuery := "select id1, val from t1 order by id1" + checkStream(t, "select * from t1 where in_keyrange('-80')", nil, wantQuery, wantStream) +} + +func TestStreamRowsMultiPacket(t *testing.T) { + if testing.Short() { + t.Skip() + } + + savedSize := *packetSize + *packetSize = 10 + defer func() { *packetSize = savedSize }() + + execStatements(t, []string{ + "create table t1(id int, val varbinary(128), primary key(id))", + "insert into t1 values (1, '234'), (2, '6789'), (3, '1'), (4, '2345678901'), (5, '2')", + }) + defer execStatements(t, []string{ + "drop table t1", + }) + engine.se.Reload(context.Background()) + + wantStream := []string{ + `fields: fields: pkfields: `, + `rows: rows: rows: lastpk: `, + `rows: lastpk: `, + `rows: lastpk: `, + } + wantQuery := "select id, val from t1 order by id" + checkStream(t, "select * from t1", nil, wantQuery, wantStream) +} + +func TestStreamRowsCancel(t *testing.T) { + if testing.Short() { + t.Skip() + } + + savedSize := *packetSize + *packetSize = 10 + defer func() { *packetSize = savedSize }() + + execStatements(t, []string{ + "create table t1(id int, val varbinary(128), primary key(id))", + "insert into t1 values (1, '234567890'), (2, '234')", + }) + defer execStatements(t, []string{ + "drop table t1", + }) + engine.se.Reload(context.Background()) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err := engine.StreamRows(ctx, "select * from t1", nil, func(rows *binlogdatapb.VStreamRowsResponse) error { + cancel() + return nil + }) + if got, want := err.Error(), "stream ended: context canceled"; got != want { + t.Errorf("err: %v, want %s", err, want) + } +} + +func checkStream(t *testing.T, query string, lastpk []sqltypes.Value, wantQuery string, wantStream []string) { + t.Helper() + + i := 0 + ch := make(chan error) + // We don't want to report errors inside callback functions because + // line numbers come out wrong. + go func() { + first := true + defer close(ch) + err := engine.StreamRows(context.Background(), query, lastpk, func(rows *binlogdatapb.VStreamRowsResponse) error { + if first { + if rows.Gtid == "" { + ch <- fmt.Errorf("stream gtid is empty") + } + if got := engine.rowStreamers[engine.streamIdx-1].sendQuery; got != wantQuery { + ch <- fmt.Errorf("query sent:\n%v, want\n%v", got, wantQuery) + } + } + first = false + rows.Gtid = "" + if i >= len(wantStream) { + ch <- fmt.Errorf("unexpected stream rows: %v", rows) + return nil + } + srows := fmt.Sprintf("%v", rows) + if srows != wantStream[i] { + ch <- fmt.Errorf("stream %d:\n%s, want\n%s", i, srows, wantStream[i]) + } + i++ + return nil + }) + if err != nil { + ch <- err + } + }() + for err := range ch { + t.Error(err) + } +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index 6923e0c8b04..010fe66df69 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -32,10 +32,9 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" - querypb "vitess.io/vitess/go/vt/proto/query" ) -var packetSize = flag.Int("vstream_packet_size", 10000, "Suggested packet size for VReplication streamer. This is used only as a recommendation. The actual packet size may be more or less than this amount.") +var packetSize = flag.Int("vstream_packet_size", 30000, "Suggested packet size for VReplication streamer. This is used only as a recommendation. The actual packet size may be more or less than this amount.") // heartbeatTime is set to slightly below 1s, compared to idleTimeout // set by VPlayer at slightly above 1s. This minimizes conflicts @@ -55,13 +54,20 @@ type vstreamer struct { // A kschema is a VSchema for just one keyspace. kevents chan *vindexes.KeyspaceSchema kschema *vindexes.KeyspaceSchema - plans map[uint64]*Plan + plans map[uint64]*streamerPlan // format and pos are updated by parseEvent. format mysql.BinlogFormat pos mysql.Position } +// streamerPlan extends the original plan to also include +// the TableMap which is used to extract values from the binlog events. +type streamerPlan struct { + *Plan + TableMap *mysql.TableMap +} + func newVStreamer(ctx context.Context, cp *mysql.ConnParams, se *schema.Engine, startPos string, filter *binlogdatapb.Filter, kschema *vindexes.KeyspaceSchema, send func([]*binlogdatapb.VEvent) error) *vstreamer { ctx, cancel := context.WithCancel(ctx) return &vstreamer{ @@ -74,7 +80,7 @@ func newVStreamer(ctx context.Context, cp *mysql.ConnParams, se *schema.Engine, send: send, kevents: make(chan *vindexes.KeyspaceSchema, 1), kschema: kschema, - plans: make(map[uint64]*Plan), + plans: make(map[uint64]*streamerPlan), } } @@ -335,38 +341,35 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e vs.plans[id] = nil return nil, nil } - ti := vs.se.GetTable(sqlparser.NewTableIdent(tm.Name)) - if ti == nil { + st := vs.se.GetTable(sqlparser.NewTableIdent(tm.Name)) + if st == nil { return nil, fmt.Errorf("unknown table %v in schema", tm.Name) } - if len(ti.Columns) < len(tm.Types) { - return nil, fmt.Errorf("cannot determine table columns for %s: event has %d columns, current schema has %d: %#v", tm.Name, len(tm.Types), len(ti.Columns), ev) + if len(st.Columns) < len(tm.Types) { + return nil, fmt.Errorf("cannot determine table columns for %s: event has %d columns, current schema has %d: %#v", tm.Name, len(tm.Types), len(st.Columns), ev) } table := &Table{ - TableMap: tm, + Name: st.Name.String(), // Columns should be truncated to match those in tm. - Columns: ti.Columns[:len(tm.Types)], + Columns: st.Columns[:len(tm.Types)], } plan, err := buildPlan(table, vs.kschema, vs.filter) if err != nil { return nil, err } - vs.plans[id] = plan if plan == nil { + vs.plans[id] = nil return nil, nil } - fields := make([]*querypb.Field, len(plan.ColExprs)) - for i, ce := range plan.ColExprs { - fields[i] = &querypb.Field{ - Name: ce.Alias.String(), - Type: ce.Type, - } + vs.plans[id] = &streamerPlan{ + Plan: plan, + TableMap: tm, } vevents = append(vevents, &binlogdatapb.VEvent{ Type: binlogdatapb.VEventType_FIELD, FieldEvent: &binlogdatapb.FieldEvent{ TableName: plan.Table.Name, - Fields: fields, + Fields: plan.fields(), }, }) case ev.IsWriteRows() || ev.IsDeleteRows() || ev.IsUpdateRows(): @@ -380,7 +383,7 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e if plan == nil { return nil, nil } - rows, err := ev.Rows(vs.format, plan.Table.TableMap) + rows, err := ev.Rows(vs.format, plan.TableMap) if err != nil { return nil, err } @@ -434,12 +437,15 @@ func (vs *vstreamer) rebuildPlans() error { if err != nil { return err } - vs.plans[id] = newPlan + vs.plans[id] = &streamerPlan{ + Plan: newPlan, + TableMap: plan.TableMap, + } } return nil } -func (vs *vstreamer) extractRowAndFilter(plan *Plan, data []byte, dataColumns, nullColumns mysql.Bitmap) (bool, []sqltypes.Value, error) { +func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataColumns, nullColumns mysql.Bitmap) (bool, []sqltypes.Value, error) { if len(data) == 0 { return false, nil, nil } @@ -454,7 +460,7 @@ func (vs *vstreamer) extractRowAndFilter(plan *Plan, data []byte, dataColumns, n valueIndex++ continue } - value, l, err := mysql.CellValue(data, pos, plan.Table.Types[colNum], plan.Table.Metadata[colNum], plan.Table.Columns[colNum].Type) + value, l, err := mysql.CellValue(data, pos, plan.TableMap.Types[colNum], plan.TableMap.Metadata[colNum], plan.Table.Columns[colNum].Type) if err != nil { return false, nil, err } diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 4d8ceccb157..21dde6b6bce 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -175,7 +175,7 @@ func TestRegexp(t *testing.T) { runCases(t, filter, testcases) } -func TestREKeyrange(t *testing.T) { +func TestREKeyRange(t *testing.T) { if testing.Short() { t.Skip() } @@ -188,6 +188,11 @@ func TestREKeyrange(t *testing.T) { }) engine.se.Reload(context.Background()) + if err := env.SetVSchema(shardedVSchema); err != nil { + t.Fatal(err) + } + defer env.SetVSchema("{}") + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -199,11 +204,6 @@ func TestREKeyrange(t *testing.T) { } ch := startStream(ctx, t, filter) - if err := env.SetVSchema(shardedVSchema); err != nil { - t.Fatal(err) - } - defer env.SetVSchema("{}") - // 1, 2, 3 and 5 are in shard -80. // 4 and 6 are in shard 80-. input := []string{ @@ -357,7 +357,7 @@ func TestDDLAddColumn(t *testing.T) { go func() { defer close(ch) if err := vstream(ctx, t, pos, filter, ch); err != nil { - t.Fatal(err) + t.Error(err) } }() expectLog(ctx, t, "ddls", ch, [][]string{{ diff --git a/go/vt/vttablet/tabletservermock/controller.go b/go/vt/vttablet/tabletservermock/controller.go index d5e906240d3..f6f7ee99649 100644 --- a/go/vt/vttablet/tabletservermock/controller.go +++ b/go/vt/vttablet/tabletservermock/controller.go @@ -187,7 +187,7 @@ func (tqsc *Controller) SchemaEngine() *schema.Engine { } // BroadcastHealth is part of the tabletserver.Controller interface -func (tqsc *Controller) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats) { +func (tqsc *Controller) BroadcastHealth(terTimestamp int64, stats *querypb.RealtimeStats, maxCache time.Duration) { tqsc.mu.Lock() defer tqsc.mu.Unlock() diff --git a/go/vt/vttest/environment.go b/go/vt/vttest/environment.go index 330ca80a759..970465b9cbd 100644 --- a/go/vt/vttest/environment.go +++ b/go/vt/vttest/environment.go @@ -125,6 +125,9 @@ func GetMySQLOptions(flavor string) (string, []string, error) { mycnf = append(mycnf, "config/mycnf/default-fast.cnf") mycnf = append(mycnf, "config/mycnf/master_mariadb.cnf") + case "MySQL80": + mycnf = append(mycnf, "config/mycnf/default-fast.cnf") + mycnf = append(mycnf, "config/mycnf/master_mysql80.cnf") case "MySQL56": mycnf = append(mycnf, "config/mycnf/default-fast.cnf") mycnf = append(mycnf, "config/mycnf/master_mysql56.cnf") diff --git a/go/vt/worker/legacy_split_clone.go b/go/vt/worker/legacy_split_clone.go index c7af31b072a..aae581c0e6f 100644 --- a/go/vt/worker/legacy_split_clone.go +++ b/go/vt/worker/legacy_split_clone.go @@ -615,6 +615,9 @@ func (scw *LegacySplitCloneWorker) copy(ctx context.Context) error { } for _, si := range scw.destinationShards { + keyspaceAndShard := topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName()) + dbName := scw.destinationDbNames[keyspaceAndShard] + destinationWaitGroup.Add(1) go func(keyspace, shard string, kr *topodatapb.KeyRange) { defer destinationWaitGroup.Done() @@ -627,10 +630,12 @@ func (scw *LegacySplitCloneWorker) copy(ctx context.Context) error { Shard: src.ShardName(), KeyRange: kr, } - qr, err := exc.vreplicationExec(ctx, binlogplayer.CreateVReplication("LegacySplitClone", bls, sourcePositions[shardIndex], scw.maxTPS, throttler.ReplicationLagModuleDisabled, time.Now().Unix())) + qr, err := exc.vreplicationExec(ctx, binlogplayer.CreateVReplication("LegacySplitClone", bls, sourcePositions[shardIndex], scw.maxTPS, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), dbName)) if err != nil { processError("vreplication queries failed: %v", err) break + } else { + scw.wr.Logger().Infof("Created replication for tablet %v/%v: %v, db: %v, pos: %v, uid: %v", keyspace, shard, bls, dbName, sourcePositions[shardIndex], uint32(qr.InsertID)) } if err := scw.wr.SourceShardAdd(ctx, keyspace, shard, uint32(qr.InsertID), src.Keyspace(), src.ShardName(), src.Shard.KeyRange, nil); err != nil { processError("could not add source shard: %v", err) diff --git a/go/vt/worker/split_clone.go b/go/vt/worker/split_clone.go index c3606356306..4ef4f8157f1 100644 --- a/go/vt/worker/split_clone.go +++ b/go/vt/worker/split_clone.go @@ -1262,6 +1262,9 @@ func (scw *SplitCloneWorker) setUpVReplication(ctx context.Context) error { } for _, si := range scw.destinationShards { + keyspaceAndShard := topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName()) + dbName := scw.destinationDbNames[keyspaceAndShard] + wg.Add(1) go func(keyspace, shard string, kr *topodatapb.KeyRange) { defer wg.Done() @@ -1286,12 +1289,13 @@ func (scw *SplitCloneWorker) setUpVReplication(ctx context.Context) error { bls.Tables = scw.tables } // TODO(mberlin): Fill in scw.maxReplicationLag once the adapative throttler is enabled by default. - qr, err := exc.vreplicationExec(cancelableCtx, binlogplayer.CreateVReplication("SplitClone", bls, sourcePositions[shardIndex], scw.maxTPS, throttler.ReplicationLagModuleDisabled, time.Now().Unix())) + qr, err := exc.vreplicationExec(cancelableCtx, binlogplayer.CreateVReplication("SplitClone", bls, sourcePositions[shardIndex], scw.maxTPS, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), dbName)) if err != nil { handleError(vterrors.Wrap(err, "vreplication queries failed")) cancel() return } + scw.wr.Logger().Infof("Created replication for tablet %v/%v: %v, db: %v, pos: %v, uid: %v", keyspace, shard, bls, dbName, sourcePositions[shardIndex], uint32(qr.InsertID)) if err := scw.wr.SourceShardAdd(cancelableCtx, keyspace, shard, uint32(qr.InsertID), src.Keyspace(), src.ShardName(), src.Shard.KeyRange, scw.tables); err != nil { handleError(vterrors.Wrap(err, "could not add source shard")) break diff --git a/go/vt/worker/split_diff.go b/go/vt/worker/split_diff.go index 7fd7a40cf73..55b5c084f33 100644 --- a/go/vt/worker/split_diff.go +++ b/go/vt/worker/split_diff.go @@ -315,7 +315,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication(ctx context.Context) error { wrangler.RecordVReplicationAction(sdw.cleaner, masterInfo.Tablet, binlogplayer.StartVReplication(sdw.sourceShard.Uid)) p3qr, err := sdw.wr.TabletManagerClient().VReplicationExec(shortCtx, masterInfo.Tablet, binlogplayer.ReadVReplicationPos(sdw.sourceShard.Uid)) if err != nil { - return vterrors.Wrapf(err, "VReplicationExec(stop) for %v failed", sdw.shardInfo.MasterAlias) + return vterrors.Wrapf(err, "ReadVReplicationPos for %v failed", sdw.shardInfo.MasterAlias) } qr := sqltypes.Proto3ToResult(p3qr) if len(qr.Rows) != 1 || len(qr.Rows[0]) != 1 { diff --git a/go/vt/worker/vtworkerclienttest/client_testsuite.go b/go/vt/worker/vtworkerclienttest/client_testsuite.go index 5cf4d56ca71..b5ff551d5a9 100644 --- a/go/vt/worker/vtworkerclienttest/client_testsuite.go +++ b/go/vt/worker/vtworkerclienttest/client_testsuite.go @@ -87,7 +87,7 @@ func commandSucceeds(t *testing.T, client vtworkerclient.Client) { if logutil.EventString(got) != expected { t.Errorf("Got unexpected log line '%v' expected '%v'", got.String(), expected) } - got, err = stream.Recv() + _, err = stream.Recv() if err != io.EOF { t.Fatalf("Didn't get EOF as expected: %v", err) } diff --git a/go/vt/workflow/resharding/workflow.go b/go/vt/workflow/resharding/workflow.go index c0fe36df72b..50209eddc5f 100644 --- a/go/vt/workflow/resharding/workflow.go +++ b/go/vt/workflow/resharding/workflow.go @@ -42,12 +42,8 @@ import ( ) const ( - codeVersion = 2 - - horizontalReshardingFactoryName = "horizontal_resharding" -) - -const ( + codeVersion = 2 + horizontalReshardingFactoryName = "horizontal_resharding" phaseCopySchema workflow.PhaseType = "copy_schema" phaseClone workflow.PhaseType = "clone" phaseWaitForFilteredReplication workflow.PhaseType = "wait_for_filtered_replication" @@ -84,7 +80,7 @@ func (*Factory) Init(m *workflow.Manager, w *workflowpb.Workflow, args []string) return err } if *keyspace == "" || *vtworkersStr == "" || *minHealthyRdonlyTablets == "" || *splitCmd == "" { - return fmt.Errorf("Keyspace name, min healthy rdonly tablets, split command, and vtworkers information must be provided for horizontal resharding") + return fmt.Errorf("keyspace name, min healthy rdonly tablets, split command, and vtworkers information must be provided for horizontal resharding") } vtworkers := strings.Split(*vtworkersStr, ",") @@ -99,7 +95,7 @@ func (*Factory) Init(m *workflow.Manager, w *workflowpb.Workflow, args []string) } } if !validPhase { - return fmt.Errorf("Invalid phase in phase_enable_approvals: %v", phase) + return fmt.Errorf("invalid phase in phase_enable_approvals: %v", phase) } } @@ -429,18 +425,6 @@ func (hw *horizontalReshardingWorkflow) setUIMessage(message string) { hw.rootUINode.BroadcastChanges(false /* updateChildren */) } -func defaultPhaseDisableApprovals() map[workflow.PhaseType]bool { - return map[workflow.PhaseType]bool{ - phaseCopySchema: false, - phaseClone: false, - phaseWaitForFilteredReplication: false, - phaseDiff: false, - phaseMigrateRdonly: false, - phaseMigrateReplica: false, - phaseMigrateMaster: false, - } -} - // WorkflowPhases returns phases for resharding workflow func WorkflowPhases() []string { return []string{ diff --git a/go/vt/workflow/reshardingworkflowgen/workflow.go b/go/vt/workflow/reshardingworkflowgen/workflow.go index 50bd4de9ece..00e6c9949d0 100644 --- a/go/vt/workflow/reshardingworkflowgen/workflow.go +++ b/go/vt/workflow/reshardingworkflowgen/workflow.go @@ -69,7 +69,7 @@ func (*Factory) Init(m *workflow.Manager, w *workflowpb.Workflow, args []string) return err } if *keyspace == "" || *vtworkersStr == "" || *minHealthyRdonlyTablets == "" || *splitCmd == "" { - return fmt.Errorf("Keyspace name, min healthy rdonly tablets, split command, and vtworkers information must be provided for horizontal resharding") + return fmt.Errorf("keyspace name, min healthy rdonly tablets, split command, and vtworkers information must be provided for horizontal resharding") } vtworkers := strings.Split(*vtworkersStr, ",") diff --git a/go/vt/workflow/reshardingworkflowgen/workflow_test.go b/go/vt/workflow/reshardingworkflowgen/workflow_flaky_test.go similarity index 100% rename from go/vt/workflow/reshardingworkflowgen/workflow_test.go rename to go/vt/workflow/reshardingworkflowgen/workflow_flaky_test.go diff --git a/go/vt/wrangler/cleaner.go b/go/vt/wrangler/cleaner.go index 668e369a15f..e40b9c08d35 100644 --- a/go/vt/wrangler/cleaner.go +++ b/go/vt/wrangler/cleaner.go @@ -29,6 +29,30 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) +const ( + // + // ChangeSlaveTypeAction CleanerFunction + // + // ChangeSlaveTypeActionName is the name of the action to change a slave type + // (can be used to find such an action by name) + ChangeSlaveTypeActionName = "ChangeSlaveTypeAction" + // + // TabletTagAction CleanerFunction + // + // TabletTagActionName is the name of the Tag action + TabletTagActionName = "TabletTagAction" + // + // StartSlaveAction CleanerAction + // + // StartSlaveActionName is the name of the slave start action + StartSlaveActionName = "StartSlaveAction" + // + // VReplication CleanerAction + // + // VReplicationActionName is the name of the action to execute VReplication commands + VReplicationActionName = "VReplicationAction" +) + // Cleaner remembers a list of cleanup steps to perform. Just record // action cleanup steps, and execute them at the end in reverse // order, with various guarantees. @@ -103,14 +127,6 @@ func (cleaner *Cleaner) CleanUp(wr *Wrangler) error { return rec.Error() } -// -// ChangeSlaveTypeAction CleanerFunction -// - -// ChangeSlaveTypeActionName is the name of the action to change a slave type -// (can be used to find such an action by name) -const ChangeSlaveTypeActionName = "ChangeSlaveTypeAction" - // RecordChangeSlaveTypeAction records a new ChangeSlaveTypeAction // into the specified Cleaner func RecordChangeSlaveTypeAction(cleaner *Cleaner, tabletAlias *topodatapb.TabletAlias, from topodatapb.TabletType, to topodatapb.TabletType) { @@ -131,13 +147,6 @@ func RecordChangeSlaveTypeAction(cleaner *Cleaner, tabletAlias *topodatapb.Table }) } -// -// TabletTagAction CleanerFunction -// - -// TabletTagActionName is the name of the Tag action -const TabletTagActionName = "TabletTagAction" - // RecordTabletTagAction records a new action to set / remove a tag // into the specified Cleaner func RecordTabletTagAction(cleaner *Cleaner, tabletAlias *topodatapb.TabletAlias, name, value string) { @@ -157,13 +166,6 @@ func RecordTabletTagAction(cleaner *Cleaner, tabletAlias *topodatapb.TabletAlias }) } -// -// StartSlaveAction CleanerAction -// - -// StartSlaveActionName is the name of the slave start action -const StartSlaveActionName = "StartSlaveAction" - // RecordStartSlaveAction records a new action to restart binlog replication on a server // into the specified Cleaner func RecordStartSlaveAction(cleaner *Cleaner, tablet *topodatapb.Tablet) { @@ -172,13 +174,6 @@ func RecordStartSlaveAction(cleaner *Cleaner, tablet *topodatapb.Tablet) { }) } -// -// VReplication CleanerAction -// - -// VReplicationActionName is the name of the action to execute VReplication commands -const VReplicationActionName = "VReplicationAction" - // RecordVReplicationAction records an action to restart binlog replication on a server // into the specified Cleaner func RecordVReplicationAction(cleaner *Cleaner, tablet *topodatapb.Tablet, query string) { diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go index ac511f1ed0b..3b193f87cd7 100644 --- a/go/vt/wrangler/keyspace.go +++ b/go/vt/wrangler/keyspace.go @@ -17,6 +17,7 @@ limitations under the License. package wrangler import ( + "bytes" "flag" "fmt" "strings" @@ -37,6 +38,7 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/topotools/events" + "vitess.io/vitess/go/vt/vterrors" ) const ( @@ -89,6 +91,107 @@ func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shard return wr.ts.UpdateKeyspace(ctx, ki) } +// SplitClone initiates a SplitClone workflow. +func (wr *Wrangler) SplitClone(ctx context.Context, keyspace string, from, to []string) error { + var fromShards, toShards []*topo.ShardInfo + for _, shard := range from { + si, err := wr.ts.GetShard(ctx, keyspace, shard) + if err != nil { + return vterrors.Wrapf(err, "GetShard(%s) failed", shard) + } + fromShards = append(fromShards, si) + } + for _, shard := range to { + si, err := wr.ts.GetShard(ctx, keyspace, shard) + if err != nil { + return vterrors.Wrapf(err, "GetShard(%s) failed", shard) + } + toShards = append(toShards, si) + } + // TODO(sougou): validate from and to shards. + + for _, dest := range toShards { + master, err := wr.ts.GetTablet(ctx, dest.MasterAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", dest.MasterAlias) + } + var ids []uint64 + for _, source := range fromShards { + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + Filter: key.KeyRangeString(dest.KeyRange), + }}, + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: keyspace, + Shard: source.ShardName(), + Filter: filter, + } + cmd := binlogplayer.CreateVReplicationState("VSplitClone", bls, "", binlogplayer.BlpStopped, master.DbName()) + qr, err := wr.TabletManagerClient().VReplicationExec(ctx, master.Tablet, cmd) + if err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s) failed", dest.MasterAlias, cmd) + } + if err := wr.SourceShardAdd(ctx, keyspace, dest.ShardName(), uint32(qr.InsertId), keyspace, source.ShardName(), source.Shard.KeyRange, nil); err != nil { + return vterrors.Wrapf(err, "SourceShardAdd(%s, %s) failed", dest.ShardName(), source.ShardName()) + } + ids = append(ids, qr.InsertId) + } + // Start vreplication only if all metadata was successfully created. + for _, id := range ids { + cmd := fmt.Sprintf("update _vt.vreplication set state='%s' where id=%d", binlogplayer.VReplicationInit, id) + if _, err = wr.TabletManagerClient().VReplicationExec(ctx, master.Tablet, cmd); err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s) failed", dest.MasterAlias, cmd) + } + } + } + return wr.refreshMasters(ctx, toShards) +} + +// VerticalSplitClone initiates a VerticalSplitClone workflow. +func (wr *Wrangler) VerticalSplitClone(ctx context.Context, fromKeyspace, toKeyspace string, tables []string) error { + source, err := wr.ts.GetOnlyShard(ctx, fromKeyspace) + if err != nil { + return vterrors.Wrapf(err, "GetOnlyShard(%s) failed", fromKeyspace) + } + dest, err := wr.ts.GetOnlyShard(ctx, toKeyspace) + if err != nil { + return vterrors.Wrapf(err, "GetOnlyShard(%s) failed", toKeyspace) + } + // TODO(sougou): validate from and to shards. + + master, err := wr.ts.GetTablet(ctx, dest.MasterAlias) + if err != nil { + return vterrors.Wrapf(err, "GetTablet(%v) failed", dest.MasterAlias) + } + filter := &binlogdatapb.Filter{} + for _, table := range tables { + filter.Rules = append(filter.Rules, &binlogdatapb.Rule{ + Match: table, + }) + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: fromKeyspace, + Shard: source.ShardName(), + Filter: filter, + } + cmd := binlogplayer.CreateVReplicationState("VSplitClone", bls, "", binlogplayer.BlpStopped, master.DbName()) + qr, err := wr.TabletManagerClient().VReplicationExec(ctx, master.Tablet, cmd) + if err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s) failed", dest.MasterAlias, cmd) + } + if err := wr.SourceShardAdd(ctx, toKeyspace, dest.ShardName(), uint32(qr.InsertId), fromKeyspace, source.ShardName(), nil, tables); err != nil { + return vterrors.Wrapf(err, "SourceShardAdd(%s, %s) failed", dest.ShardName(), source.ShardName()) + } + // Start vreplication only if metadata was successfully created. + cmd = fmt.Sprintf("update _vt.vreplication set state='%s' where id=%d", binlogplayer.VReplicationInit, qr.InsertId) + if _, err = wr.TabletManagerClient().VReplicationExec(ctx, master.Tablet, cmd); err != nil { + return vterrors.Wrapf(err, "VReplicationExec(%v, %s) failed", dest.MasterAlias, cmd) + } + return wr.refreshMasters(ctx, []*topo.ShardInfo{dest}) +} + // ShowResharding shows all resharding related metadata for the keyspace/shard. func (wr *Wrangler) ShowResharding(ctx context.Context, keyspace, shard string) (err error) { ki, err := wr.ts.GetKeyspace(ctx, keyspace) @@ -135,7 +238,7 @@ func (wr *Wrangler) printShards(ctx context.Context, si []*topo.ShardInfo) error if err != nil { return err } - qr, err := wr.tmc.VReplicationExec(ctx, ti.Tablet, "select * from _vt.vreplication") + qr, err := wr.tmc.VReplicationExec(ctx, ti.Tablet, fmt.Sprintf("select * from _vt.vreplication where db_name=%v", encodeString(ti.DbName()))) if err != nil { return err } @@ -681,6 +784,11 @@ func (wr *Wrangler) setupReverseReplication(ctx context.Context, sourceShards, d // Create reverse replication for each source. for i, sourceShard := range sourceShards { + ti, err := wr.ts.GetTablet(ctx, sourceShard.MasterAlias) + if err != nil { + return err + } + dbName := ti.DbName() if len(sourceShard.SourceShards) != 0 { continue } @@ -697,16 +805,15 @@ func (wr *Wrangler) setupReverseReplication(ctx context.Context, sourceShards, d Shard: dest.ShardName(), KeyRange: kr, } - qr, err := wr.VReplicationExec(ctx, sourceShard.MasterAlias, binlogplayer.CreateVReplicationStopped("ReversedResharding", bls, masterPositions[j])) + qr, err := wr.VReplicationExec(ctx, sourceShard.MasterAlias, binlogplayer.CreateVReplicationState("ReversedResharding", bls, masterPositions[j], binlogplayer.BlpStopped, dbName)) if err != nil { return err } uids[j] = uint32(qr.InsertId) - wr.Logger().Infof("Created reverse replication for tablet %v/%v: %v, pos: %v, uid: %v", sourceShard.Keyspace(), sourceShard.ShardName(), bls, masterPositions[j], uids[j]) + wr.Logger().Infof("Created reverse replication for tablet %v/%v: %v, db: %v, pos: %v, uid: %v", sourceShard.Keyspace(), sourceShard.ShardName(), bls, dbName, masterPositions[j], uids[j]) } // Source shards have to be atomically added to ensure idempotence. // If this fails, there's no harm because the unstarted vreplication streams will just be abandoned. - var err error sourceShards[i], err = wr.ts.UpdateShardFields(ctx, sourceShard.Keyspace(), sourceShard.ShardName(), func(si *topo.ShardInfo) error { for j, dest := range destinationShards { si.SourceShards = append(si.SourceShards, &topodatapb.Shard_SourceShard{ @@ -1322,3 +1429,9 @@ func (wr *Wrangler) RemoveKeyspaceCell(ctx context.Context, keyspace, cell strin wr.Logger().Infof("Removing cell %v keyspace %v SrvKeyspace object", cell, keyspace) return wr.ts.DeleteSrvKeyspace(ctx, cell, keyspace) } + +func encodeString(in string) string { + buf := bytes.NewBuffer(nil) + sqltypes.NewVarChar(in).EncodeSQL(buf) + return buf.String() +} diff --git a/go/vt/wrangler/schema.go b/go/vt/wrangler/schema.go index 4b6bbb533d4..fd041172797 100644 --- a/go/vt/wrangler/schema.go +++ b/go/vt/wrangler/schema.go @@ -379,17 +379,21 @@ func (wr *Wrangler) copyShardMetadata(ctx context.Context, srcTabletAlias *topod return nil } - sql = "SELECT name, value FROM _vt.shard_metadata" + // TODO: 100 may be too low here for row limit + sql = "SELECT db_name, name, value FROM _vt.shard_metadata" dataProto, err := wr.ExecuteFetchAsDba(ctx, srcTabletAlias, sql, 100, false, false) if err != nil { return fmt.Errorf("ExecuteFetchAsDba(%v, %v, 100, false, false) failed: %v", srcTabletAlias, sql, err) } data := sqltypes.Proto3ToResult(dataProto) for _, row := range data.Rows { - name := row[0] - value := row[1] + dbName := row[0] + name := row[1] + value := row[2] queryBuf := bytes.Buffer{} - queryBuf.WriteString("INSERT INTO _vt.shard_metadata (name, value) VALUES (") + queryBuf.WriteString("INSERT INTO _vt.shard_metadata (db_name, name, value) VALUES (") + dbName.EncodeSQL(&queryBuf) + queryBuf.WriteByte(',') name.EncodeSQL(&queryBuf) queryBuf.WriteByte(',') value.EncodeSQL(&queryBuf) diff --git a/go/vt/wrangler/testlib/backup_test.go b/go/vt/wrangler/testlib/backup_test.go index a9abb4497d8..ddd80965452 100644 --- a/go/vt/wrangler/testlib/backup_test.go +++ b/go/vt/wrangler/testlib/backup_test.go @@ -56,6 +56,10 @@ func TestBackupRestore(t *testing.T) { db.AddQueryPattern(`SET @@session\.sql_log_bin = .*`, &sqltypes.Result{}) db.AddQueryPattern(`CREATE TABLE IF NOT EXISTS _vt\.shard_metadata .*`, &sqltypes.Result{}) db.AddQueryPattern(`CREATE TABLE IF NOT EXISTS _vt\.local_metadata .*`, &sqltypes.Result{}) + db.AddQueryPattern(`ALTER TABLE _vt\.local_metadata .*`, &sqltypes.Result{}) + db.AddQueryPattern(`ALTER TABLE _vt\.shard_metadata .*`, &sqltypes.Result{}) + db.AddQueryPattern(`UPDATE _vt\.local_metadata SET db_name=.*`, &sqltypes.Result{}) + db.AddQueryPattern(`UPDATE _vt\.shard_metadata SET db_name=.*`, &sqltypes.Result{}) db.AddQueryPattern(`INSERT INTO _vt\.local_metadata .*`, &sqltypes.Result{}) // Initialize our temp dirs diff --git a/go/vt/wrangler/testlib/copy_schema_shard_test.go b/go/vt/wrangler/testlib/copy_schema_shard_test.go index 024a44be75b..ad33097440b 100644 --- a/go/vt/wrangler/testlib/copy_schema_shard_test.go +++ b/go/vt/wrangler/testlib/copy_schema_shard_test.go @@ -115,7 +115,7 @@ func copySchema(t *testing.T, useShardAsSource bool) { " KEY `by_msg` (`msg`)\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8" selectInformationSchema := "SELECT 1 FROM information_schema.tables WHERE table_schema = '_vt' AND table_name = 'shard_metadata'" - selectShardMetadata := "SELECT name, value FROM _vt.shard_metadata" + selectShardMetadata := "SELECT db_name, name, value FROM _vt.shard_metadata" // The source table is asked about its schema. // It may be the master or the rdonly. diff --git a/go/vt/wrangler/testlib/fake_tablet.go b/go/vt/wrangler/testlib/fake_tablet.go index ca279f31f46..75766e27d61 100644 --- a/go/vt/wrangler/testlib/fake_tablet.go +++ b/go/vt/wrangler/testlib/fake_tablet.go @@ -120,7 +120,7 @@ func StartHTTPServer() TabletOption { // Use TabletOption implementations if you need to change values at creation. // 'db' can be nil if the test doesn't use a database at all. func NewFakeTablet(t *testing.T, wr *wrangler.Wrangler, cell string, uid uint32, tabletType topodatapb.TabletType, db *fakesqldb.DB, options ...TabletOption) *FakeTablet { - if uid < 0 || uid > 99 { + if uid > 99 { t.Fatalf("uid has to be between 0 and 99: %v", uid) } mysqlPort := int32(3300 + uid) diff --git a/go/vt/wrangler/testlib/migrate_served_from_test.go b/go/vt/wrangler/testlib/migrate_served_from_test.go index 7388600aead..19d8f4ef69c 100644 --- a/go/vt/wrangler/testlib/migrate_served_from_test.go +++ b/go/vt/wrangler/testlib/migrate_served_from_test.go @@ -106,8 +106,8 @@ func TestMigrateServedFrom(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } - destMaster.Agent.VREngine = vreplication.NewEngine(ts, "", destMaster.FakeMysqlDaemon, dbClientFactory) - dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + destMaster.Agent.VREngine = vreplication.NewEngine(ts, "", destMaster.FakeMysqlDaemon, dbClientFactory, dbClient.DBName()) + dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := destMaster.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } diff --git a/go/vt/wrangler/testlib/migrate_served_types_test.go b/go/vt/wrangler/testlib/migrate_served_types_test.go index 5fb5ecf4a11..5cb170c5c8c 100644 --- a/go/vt/wrangler/testlib/migrate_served_types_test.go +++ b/go/vt/wrangler/testlib/migrate_served_types_test.go @@ -153,9 +153,9 @@ func TestMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient1 := binlogplayer.NewMockDBClient(t) dbClientFactory1 := func() binlogplayer.DBClient { return dbClient1 } - dest1Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest1Master.FakeMysqlDaemon, dbClientFactory1) + dest1Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest1Master.FakeMysqlDaemon, dbClientFactory1, dbClient1.DBName()) // select * from _vt.vreplication during Open - dbClient1.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient1.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest1Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } @@ -182,9 +182,9 @@ func TestMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient2 := binlogplayer.NewMockDBClient(t) dbClientFactory2 := func() binlogplayer.DBClient { return dbClient2 } - dest2Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest2Master.FakeMysqlDaemon, dbClientFactory2) + dest2Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest2Master.FakeMysqlDaemon, dbClientFactory2, dbClient2.DBName()) // select * from _vt.vreplication during Open - dbClient2.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient2.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest2Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } @@ -419,9 +419,9 @@ func TestMultiShardMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient1 := binlogplayer.NewMockDBClient(t) dbClientFactory1 := func() binlogplayer.DBClient { return dbClient1 } - dest1Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest1Master.FakeMysqlDaemon, dbClientFactory1) + dest1Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest1Master.FakeMysqlDaemon, dbClientFactory1, "db") // select * from _vt.vreplication during Open - dbClient1.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient1.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest1Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } @@ -437,9 +437,9 @@ func TestMultiShardMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient2 := binlogplayer.NewMockDBClient(t) dbClientFactory2 := func() binlogplayer.DBClient { return dbClient2 } - dest2Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest2Master.FakeMysqlDaemon, dbClientFactory2) + dest2Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest2Master.FakeMysqlDaemon, dbClientFactory2, "db") // select * from _vt.vreplication during Open - dbClient2.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient2.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest2Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } @@ -509,9 +509,9 @@ func TestMultiShardMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient1 = binlogplayer.NewMockDBClient(t) dbClientFactory1 = func() binlogplayer.DBClient { return dbClient1 } - dest3Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest3Master.FakeMysqlDaemon, dbClientFactory1) + dest3Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest3Master.FakeMysqlDaemon, dbClientFactory1, "db") // select * from _vt.vreplication during Open - dbClient1.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient1.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest3Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } @@ -527,9 +527,9 @@ func TestMultiShardMigrateServedTypes(t *testing.T) { // Override with a fake VREngine after Agent is initialized in action loop. dbClient2 = binlogplayer.NewMockDBClient(t) dbClientFactory2 = func() binlogplayer.DBClient { return dbClient2 } - dest4Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest4Master.FakeMysqlDaemon, dbClientFactory2) + dest4Master.Agent.VREngine = vreplication.NewEngine(ts, "", dest4Master.FakeMysqlDaemon, dbClientFactory2, "db") // select * from _vt.vreplication during Open - dbClient2.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) + dbClient2.ExpectRequest("select * from _vt.vreplication where db_name='db'", &sqltypes.Result{}, nil) if err := dest4Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } diff --git a/go/vt/wrangler/testlib/reparent_external_test.go b/go/vt/wrangler/testlib/reparent_external_test.go index fff7aeae52f..153bb8d0a72 100644 --- a/go/vt/wrangler/testlib/reparent_external_test.go +++ b/go/vt/wrangler/testlib/reparent_external_test.go @@ -80,7 +80,7 @@ func TestTabletExternallyReparented(t *testing.T) { } // Make sure the master is not exported in other cells - tabletMap, err = ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell2"}) + tabletMap, _ = ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell2"}) master, err = topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) if !topo.IsErrType(err, topo.NoNode) { t.Fatalf("FindTabletByHostAndPort(master) worked in cell2: %v %v", err, master) @@ -123,7 +123,7 @@ func TestTabletExternallyReparented(t *testing.T) { // First test: reparent to the same master, make sure it works // as expected. tmc := tmclient.NewTabletManagerClient() - ti, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) + _, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) if err != nil { t.Fatalf("GetTablet failed: %v", err) } @@ -136,7 +136,7 @@ func TestTabletExternallyReparented(t *testing.T) { // This tests a bad case: the new designated master is a slave, // but we should do what we're told anyway. - ti, err = ts.GetTablet(ctx, goodSlave1.Tablet.Alias) + ti, err := ts.GetTablet(ctx, goodSlave1.Tablet.Alias) if err != nil { t.Fatalf("GetTablet failed: %v", err) } diff --git a/go/vt/zkctl/zksrv.sh b/go/vt/zkctl/zksrv.sh index 62172c5d279..aada9b9b8aa 100755 --- a/go/vt/zkctl/zksrv.sh +++ b/go/vt/zkctl/zksrv.sh @@ -20,7 +20,7 @@ logdir="$1" config="$2" pidfile="$3" - +zk_java_opts=${ZK_JAVA_OPTS:-} zk_ver=${ZK_VERSION:-3.4.14} classpath="$VTROOT/dist/vt-zookeeper-$zk_ver/lib/zookeeper-$zk_ver-fatjar.jar:/usr/local/lib/zookeeper-$zk_ver-fatjar.jar:/usr/share/java/zookeeper-$zk_ver.jar" @@ -53,7 +53,7 @@ else fi -cmd="$java -DZOO_LOG_DIR=$logdir -cp $classpath org.apache.zookeeper.server.quorum.QuorumPeerMain $config" +cmd="$java -DZOO_LOG_DIR=$logdir $zk_java_opts -cp $classpath org.apache.zookeeper.server.quorum.QuorumPeerMain $config" log "INFO starting $cmd" $cmd < /dev/null &> /dev/null & diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 4942198823a..3779a624cf9 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -231,3 +231,22 @@ message VStreamRequest { message VStreamResponse { repeated VEvent events = 1; } + +// VStreamRowsRequest is the payload for VStreamRows +message VStreamRowsRequest { + vtrpc.CallerID effective_caller_id = 1; + query.VTGateCallerID immediate_caller_id = 2; + query.Target target = 3; + + string query = 4; + query.QueryResult lastpk = 5; +} + +// VStreamRowsResponse is the response from VStreamRows +message VStreamRowsResponse { + repeated query.Field fields = 1; + repeated query.Field pkfields = 2; + string gtid = 3; + repeated query.Row rows = 4; + query.Row lastpk = 5; +} diff --git a/proto/queryservice.proto b/proto/queryservice.proto index 897cbf3f034..8b6494f5e9b 100644 --- a/proto/queryservice.proto +++ b/proto/queryservice.proto @@ -98,4 +98,7 @@ service Query { // VStream streams vreplication events. rpc VStream(binlogdata.VStreamRequest) returns (stream binlogdata.VStreamResponse) {}; + + // VStreamRows streams rows from the specified starting point. + rpc VStreamRows(binlogdata.VStreamRowsRequest) returns (stream binlogdata.VStreamRowsResponse) {}; } diff --git a/proto/vschema.proto b/proto/vschema.proto index bdf7e9834e9..12e6154f047 100644 --- a/proto/vschema.proto +++ b/proto/vschema.proto @@ -23,6 +23,20 @@ package vschema; import "query.proto"; +// RoutingRules specify the high level routing rules for the VSchema. +message RoutingRules { + // rules should ideally be a map. However protos dont't allow + // repeated fields as elements of a map. So, we use a list + // instead. + repeated RoutingRule rules = 1; +} + +// RoutingRule specifies a routing rule. +message RoutingRule { + string from_table = 1; + repeated string to_tables = 2; +} + // Keyspace is the vschema for a keyspace. message Keyspace { // If sharded is false, vindexes and tables are ignored. @@ -62,7 +76,7 @@ message Table { repeated Column columns = 4; // pinned pins an unsharded table to a specific // shard, as dictated by the keyspace id. - // The keyspace id is represened in hex form + // The keyspace id is represented in hex form // like in keyranges. string pinned = 5; // column_list_authoritative is set to true if columns is @@ -98,4 +112,5 @@ message Column { message SrvVSchema { // keyspaces is a map of keyspace name -> Keyspace object. map keyspaces = 1; + RoutingRules routing_rules = 2; } diff --git a/proto/vtgateservice.proto b/proto/vtgateservice.proto index 1749ad6527c..f884417f923 100644 --- a/proto/vtgateservice.proto +++ b/proto/vtgateservice.proto @@ -32,19 +32,19 @@ import "query.proto"; service Vitess { // Execute tries to route the query to the right shard. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 rpc Execute(vtgate.ExecuteRequest) returns (vtgate.ExecuteResponse) {}; // ExecuteBatch tries to route the list of queries on the right shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // API group: v3 rpc ExecuteBatch(vtgate.ExecuteBatchRequest) returns (vtgate.ExecuteBatchResponse) {}; // StreamExecute executes a streaming query based on shards. // It depends on the query and bind variables to provide enough - // information in conjonction with the vindexes to route the query. + // information in conjunction with the vindexes to route the query. // Use this method if the query returns a large number of rows. // API group: v3 rpc StreamExecute(vtgate.StreamExecuteRequest) returns (stream vtgate.StreamExecuteResponse) {}; diff --git a/py/vtproto/binlogdata_pb2.py b/py/vtproto/binlogdata_pb2.py index fe5c3d42e8c..2e0153062f3 100644 --- a/py/vtproto/binlogdata_pb2.py +++ b/py/vtproto/binlogdata_pb2.py @@ -23,7 +23,7 @@ package='binlogdata', syntax='proto3', serialized_options=_b('Z\'vitess.io/vitess/go/vt/proto/binlogdata'), - serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bvtrpc.proto\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xde\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\x12\'\n\x06on_ddl\x18\x07 \x01(\x0e\x32\x17.binlogdata.OnDDLAction\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\xc8\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x11\n\ttimestamp\x18\x02 \x01(\x03\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x04 \x01(\t\x12\'\n\trow_event\x18\x05 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x06 \x01(\x0b\x32\x16.binlogdata.FieldEvent\x12\x14\n\x0c\x63urrent_time\x18\x14 \x01(\x03\"\xc7\x01\n\x0eVStreamRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\x10\n\x08position\x18\x04 \x01(\t\x12\"\n\x06\x66ilter\x18\x05 \x01(\x0b\x32\x12.binlogdata.Filter\"5\n\x0fVStreamResponse\x12\"\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent*>\n\x0bOnDDLAction\x12\n\n\x06IGNORE\x10\x00\x12\x08\n\x04STOP\x10\x01\x12\x08\n\x04\x45XEC\x10\x02\x12\x0f\n\x0b\x45XEC_IGNORE\x10\x03*\xb9\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\r\x12\r\n\tHEARTBEAT\x10\x0e\x42)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') + serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bvtrpc.proto\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xde\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\x12\'\n\x06on_ddl\x18\x07 \x01(\x0e\x32\x17.binlogdata.OnDDLAction\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\xc8\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x11\n\ttimestamp\x18\x02 \x01(\x03\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x04 \x01(\t\x12\'\n\trow_event\x18\x05 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x06 \x01(\x0b\x32\x16.binlogdata.FieldEvent\x12\x14\n\x0c\x63urrent_time\x18\x14 \x01(\x03\"\xc7\x01\n\x0eVStreamRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\x10\n\x08position\x18\x04 \x01(\t\x12\"\n\x06\x66ilter\x18\x05 \x01(\x0b\x32\x12.binlogdata.Filter\"5\n\x0fVStreamResponse\x12\"\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent\"\xc8\x01\n\x12VStreamRowsRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\r\n\x05query\x18\x04 \x01(\t\x12\"\n\x06lastpk\x18\x05 \x01(\x0b\x32\x12.query.QueryResult\"\x97\x01\n\x13VStreamRowsResponse\x12\x1c\n\x06\x66ields\x18\x01 \x03(\x0b\x32\x0c.query.Field\x12\x1e\n\x08pkfields\x18\x02 \x03(\x0b\x32\x0c.query.Field\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x18\n\x04rows\x18\x04 \x03(\x0b\x32\n.query.Row\x12\x1a\n\x06lastpk\x18\x05 \x01(\x0b\x32\n.query.Row*>\n\x0bOnDDLAction\x12\n\n\x06IGNORE\x10\x00\x12\x08\n\x04STOP\x10\x01\x12\x08\n\x04\x45XEC\x10\x02\x12\x0f\n\x0b\x45XEC_IGNORE\x10\x03*\xb9\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\r\x12\r\n\tHEARTBEAT\x10\x0e\x42)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') , dependencies=[vtrpc__pb2.DESCRIPTOR,query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,]) @@ -52,8 +52,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=1929, - serialized_end=1991, + serialized_start=2286, + serialized_end=2348, ) _sym_db.RegisterEnumDescriptor(_ONDDLACTION) @@ -127,8 +127,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=1994, - serialized_end=2179, + serialized_start=2351, + serialized_end=2536, ) _sym_db.RegisterEnumDescriptor(_VEVENTTYPE) @@ -907,6 +907,124 @@ serialized_end=1927, ) + +_VSTREAMROWSREQUEST = _descriptor.Descriptor( + name='VStreamRowsRequest', + full_name='binlogdata.VStreamRowsRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='effective_caller_id', full_name='binlogdata.VStreamRowsRequest.effective_caller_id', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='immediate_caller_id', full_name='binlogdata.VStreamRowsRequest.immediate_caller_id', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='target', full_name='binlogdata.VStreamRowsRequest.target', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='query', full_name='binlogdata.VStreamRowsRequest.query', index=3, + number=4, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='lastpk', full_name='binlogdata.VStreamRowsRequest.lastpk', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1930, + serialized_end=2130, +) + + +_VSTREAMROWSRESPONSE = _descriptor.Descriptor( + name='VStreamRowsResponse', + full_name='binlogdata.VStreamRowsResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='fields', full_name='binlogdata.VStreamRowsResponse.fields', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='pkfields', full_name='binlogdata.VStreamRowsResponse.pkfields', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='gtid', full_name='binlogdata.VStreamRowsResponse.gtid', index=2, + number=3, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='rows', full_name='binlogdata.VStreamRowsResponse.rows', index=3, + number=4, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='lastpk', full_name='binlogdata.VStreamRowsResponse.lastpk', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=2133, + serialized_end=2284, +) + _BINLOGTRANSACTION_STATEMENT.fields_by_name['category'].enum_type = _BINLOGTRANSACTION_STATEMENT_CATEGORY _BINLOGTRANSACTION_STATEMENT.fields_by_name['charset'].message_type = _CHARSET _BINLOGTRANSACTION_STATEMENT.containing_type = _BINLOGTRANSACTION @@ -935,6 +1053,14 @@ _VSTREAMREQUEST.fields_by_name['target'].message_type = query__pb2._TARGET _VSTREAMREQUEST.fields_by_name['filter'].message_type = _FILTER _VSTREAMRESPONSE.fields_by_name['events'].message_type = _VEVENT +_VSTREAMROWSREQUEST.fields_by_name['effective_caller_id'].message_type = vtrpc__pb2._CALLERID +_VSTREAMROWSREQUEST.fields_by_name['immediate_caller_id'].message_type = query__pb2._VTGATECALLERID +_VSTREAMROWSREQUEST.fields_by_name['target'].message_type = query__pb2._TARGET +_VSTREAMROWSREQUEST.fields_by_name['lastpk'].message_type = query__pb2._QUERYRESULT +_VSTREAMROWSRESPONSE.fields_by_name['fields'].message_type = query__pb2._FIELD +_VSTREAMROWSRESPONSE.fields_by_name['pkfields'].message_type = query__pb2._FIELD +_VSTREAMROWSRESPONSE.fields_by_name['rows'].message_type = query__pb2._ROW +_VSTREAMROWSRESPONSE.fields_by_name['lastpk'].message_type = query__pb2._ROW DESCRIPTOR.message_types_by_name['Charset'] = _CHARSET DESCRIPTOR.message_types_by_name['BinlogTransaction'] = _BINLOGTRANSACTION DESCRIPTOR.message_types_by_name['StreamKeyRangeRequest'] = _STREAMKEYRANGEREQUEST @@ -950,6 +1076,8 @@ DESCRIPTOR.message_types_by_name['VEvent'] = _VEVENT DESCRIPTOR.message_types_by_name['VStreamRequest'] = _VSTREAMREQUEST DESCRIPTOR.message_types_by_name['VStreamResponse'] = _VSTREAMRESPONSE +DESCRIPTOR.message_types_by_name['VStreamRowsRequest'] = _VSTREAMROWSREQUEST +DESCRIPTOR.message_types_by_name['VStreamRowsResponse'] = _VSTREAMROWSRESPONSE DESCRIPTOR.enum_types_by_name['OnDDLAction'] = _ONDDLACTION DESCRIPTOR.enum_types_by_name['VEventType'] = _VEVENTTYPE _sym_db.RegisterFileDescriptor(DESCRIPTOR) @@ -1067,6 +1195,20 @@ )) _sym_db.RegisterMessage(VStreamResponse) +VStreamRowsRequest = _reflection.GeneratedProtocolMessageType('VStreamRowsRequest', (_message.Message,), dict( + DESCRIPTOR = _VSTREAMROWSREQUEST, + __module__ = 'binlogdata_pb2' + # @@protoc_insertion_point(class_scope:binlogdata.VStreamRowsRequest) + )) +_sym_db.RegisterMessage(VStreamRowsRequest) + +VStreamRowsResponse = _reflection.GeneratedProtocolMessageType('VStreamRowsResponse', (_message.Message,), dict( + DESCRIPTOR = _VSTREAMROWSRESPONSE, + __module__ = 'binlogdata_pb2' + # @@protoc_insertion_point(class_scope:binlogdata.VStreamRowsResponse) + )) +_sym_db.RegisterMessage(VStreamRowsResponse) + DESCRIPTOR._options = None # @@protoc_insertion_point(module_scope) diff --git a/py/vtproto/queryservice_pb2.py b/py/vtproto/queryservice_pb2.py index 3c443df129b..986c7701187 100644 --- a/py/vtproto/queryservice_pb2.py +++ b/py/vtproto/queryservice_pb2.py @@ -21,7 +21,7 @@ package='queryservice', syntax='proto3', serialized_options=_b('Z)vitess.io/vitess/go/vt/proto/queryservice'), - serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto\x1a\x10\x62inlogdata.proto2\xef\x0c\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12:\n\x07Prepare\x12\x15.query.PrepareRequest\x1a\x16.query.PrepareResponse\"\x00\x12O\n\x0e\x43ommitPrepared\x12\x1c.query.CommitPreparedRequest\x1a\x1d.query.CommitPreparedResponse\"\x00\x12U\n\x10RollbackPrepared\x12\x1e.query.RollbackPreparedRequest\x1a\x1f.query.RollbackPreparedResponse\"\x00\x12X\n\x11\x43reateTransaction\x12\x1f.query.CreateTransactionRequest\x1a .query.CreateTransactionResponse\"\x00\x12\x46\n\x0bStartCommit\x12\x19.query.StartCommitRequest\x1a\x1a.query.StartCommitResponse\"\x00\x12\x46\n\x0bSetRollback\x12\x19.query.SetRollbackRequest\x1a\x1a.query.SetRollbackResponse\"\x00\x12^\n\x13\x43oncludeTransaction\x12!.query.ConcludeTransactionRequest\x1a\".query.ConcludeTransactionResponse\"\x00\x12R\n\x0fReadTransaction\x12\x1d.query.ReadTransactionRequest\x1a\x1e.query.ReadTransactionResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12N\n\rMessageStream\x12\x1b.query.MessageStreamRequest\x1a\x1c.query.MessageStreamResponse\"\x00\x30\x01\x12\x43\n\nMessageAck\x12\x18.query.MessageAckRequest\x1a\x19.query.MessageAckResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x12K\n\x0cUpdateStream\x12\x1a.query.UpdateStreamRequest\x1a\x1b.query.UpdateStreamResponse\"\x00\x30\x01\x12\x46\n\x07VStream\x12\x1a.binlogdata.VStreamRequest\x1a\x1b.binlogdata.VStreamResponse\"\x00\x30\x01\x42+Z)vitess.io/vitess/go/vt/proto/queryserviceb\x06proto3') + serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto\x1a\x10\x62inlogdata.proto2\xc3\r\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12:\n\x07Prepare\x12\x15.query.PrepareRequest\x1a\x16.query.PrepareResponse\"\x00\x12O\n\x0e\x43ommitPrepared\x12\x1c.query.CommitPreparedRequest\x1a\x1d.query.CommitPreparedResponse\"\x00\x12U\n\x10RollbackPrepared\x12\x1e.query.RollbackPreparedRequest\x1a\x1f.query.RollbackPreparedResponse\"\x00\x12X\n\x11\x43reateTransaction\x12\x1f.query.CreateTransactionRequest\x1a .query.CreateTransactionResponse\"\x00\x12\x46\n\x0bStartCommit\x12\x19.query.StartCommitRequest\x1a\x1a.query.StartCommitResponse\"\x00\x12\x46\n\x0bSetRollback\x12\x19.query.SetRollbackRequest\x1a\x1a.query.SetRollbackResponse\"\x00\x12^\n\x13\x43oncludeTransaction\x12!.query.ConcludeTransactionRequest\x1a\".query.ConcludeTransactionResponse\"\x00\x12R\n\x0fReadTransaction\x12\x1d.query.ReadTransactionRequest\x1a\x1e.query.ReadTransactionResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12N\n\rMessageStream\x12\x1b.query.MessageStreamRequest\x1a\x1c.query.MessageStreamResponse\"\x00\x30\x01\x12\x43\n\nMessageAck\x12\x18.query.MessageAckRequest\x1a\x19.query.MessageAckResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x12K\n\x0cUpdateStream\x12\x1a.query.UpdateStreamRequest\x1a\x1b.query.UpdateStreamResponse\"\x00\x30\x01\x12\x46\n\x07VStream\x12\x1a.binlogdata.VStreamRequest\x1a\x1b.binlogdata.VStreamResponse\"\x00\x30\x01\x12R\n\x0bVStreamRows\x12\x1e.binlogdata.VStreamRowsRequest\x1a\x1f.binlogdata.VStreamRowsResponse\"\x00\x30\x01\x42+Z)vitess.io/vitess/go/vt/proto/queryserviceb\x06proto3') , dependencies=[query__pb2.DESCRIPTOR,binlogdata__pb2.DESCRIPTOR,]) @@ -39,7 +39,7 @@ index=0, serialized_options=None, serialized_start=68, - serialized_end=1715, + serialized_end=1799, methods=[ _descriptor.MethodDescriptor( name='Execute', @@ -239,6 +239,15 @@ output_type=binlogdata__pb2._VSTREAMRESPONSE, serialized_options=None, ), + _descriptor.MethodDescriptor( + name='VStreamRows', + full_name='queryservice.Query.VStreamRows', + index=22, + containing_service=None, + input_type=binlogdata__pb2._VSTREAMROWSREQUEST, + output_type=binlogdata__pb2._VSTREAMROWSRESPONSE, + serialized_options=None, + ), ]) _sym_db.RegisterServiceDescriptor(_QUERY) diff --git a/py/vtproto/queryservice_pb2_grpc.py b/py/vtproto/queryservice_pb2_grpc.py index 61adf9b42bd..f6d80e62adc 100644 --- a/py/vtproto/queryservice_pb2_grpc.py +++ b/py/vtproto/queryservice_pb2_grpc.py @@ -125,6 +125,11 @@ def __init__(self, channel): request_serializer=binlogdata__pb2.VStreamRequest.SerializeToString, response_deserializer=binlogdata__pb2.VStreamResponse.FromString, ) + self.VStreamRows = channel.unary_stream( + '/queryservice.Query/VStreamRows', + request_serializer=binlogdata__pb2.VStreamRowsRequest.SerializeToString, + response_deserializer=binlogdata__pb2.VStreamRowsResponse.FromString, + ) class QueryServicer(object): @@ -292,6 +297,13 @@ def VStream(self, request, context): context.set_details('Method not implemented!') raise NotImplementedError('Method not implemented!') + def VStreamRows(self, request, context): + """VStreamRows streams rows from the specified starting point. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + def add_QueryServicer_to_server(servicer, server): rpc_method_handlers = { @@ -405,6 +417,11 @@ def add_QueryServicer_to_server(servicer, server): request_deserializer=binlogdata__pb2.VStreamRequest.FromString, response_serializer=binlogdata__pb2.VStreamResponse.SerializeToString, ), + 'VStreamRows': grpc.unary_stream_rpc_method_handler( + servicer.VStreamRows, + request_deserializer=binlogdata__pb2.VStreamRowsRequest.FromString, + response_serializer=binlogdata__pb2.VStreamRowsResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( 'queryservice.Query', rpc_method_handlers) diff --git a/py/vtproto/vschema_pb2.py b/py/vtproto/vschema_pb2.py index 878d1d08dc9..94f613b8df1 100644 --- a/py/vtproto/vschema_pb2.py +++ b/py/vtproto/vschema_pb2.py @@ -7,7 +7,6 @@ from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database -from google.protobuf import descriptor_pb2 # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() @@ -20,13 +19,83 @@ name='vschema.proto', package='vschema', syntax='proto3', - serialized_pb=_b('\n\rvschema.proto\x12\x07vschema\x1a\x0bquery.proto\"\xfe\x01\n\x08Keyspace\x12\x0f\n\x07sharded\x18\x01 \x01(\x08\x12\x31\n\x08vindexes\x18\x02 \x03(\x0b\x32\x1f.vschema.Keyspace.VindexesEntry\x12-\n\x06tables\x18\x03 \x03(\x0b\x32\x1d.vschema.Keyspace.TablesEntry\x1a@\n\rVindexesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x1e\n\x05value\x18\x02 \x01(\x0b\x32\x0f.vschema.Vindex:\x02\x38\x01\x1a=\n\x0bTablesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x1d\n\x05value\x18\x02 \x01(\x0b\x32\x0e.vschema.Table:\x02\x38\x01\"\x81\x01\n\x06Vindex\x12\x0c\n\x04type\x18\x01 \x01(\t\x12+\n\x06params\x18\x02 \x03(\x0b\x32\x1b.vschema.Vindex.ParamsEntry\x12\r\n\x05owner\x18\x03 \x01(\t\x1a-\n\x0bParamsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xca\x01\n\x05Table\x12\x0c\n\x04type\x18\x01 \x01(\t\x12.\n\x0f\x63olumn_vindexes\x18\x02 \x03(\x0b\x32\x15.vschema.ColumnVindex\x12.\n\x0e\x61uto_increment\x18\x03 \x01(\x0b\x32\x16.vschema.AutoIncrement\x12 \n\x07\x63olumns\x18\x04 \x03(\x0b\x32\x0f.vschema.Column\x12\x0e\n\x06pinned\x18\x05 \x01(\t\x12!\n\x19\x63olumn_list_authoritative\x18\x06 \x01(\x08\"=\n\x0c\x43olumnVindex\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0f\n\x07\x63olumns\x18\x03 \x03(\t\"1\n\rAutoIncrement\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x10\n\x08sequence\x18\x02 \x01(\t\"1\n\x06\x43olumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x19\n\x04type\x18\x02 \x01(\x0e\x32\x0b.query.Type\"\x88\x01\n\nSrvVSchema\x12\x35\n\tkeyspaces\x18\x01 \x03(\x0b\x32\".vschema.SrvVSchema.KeyspacesEntry\x1a\x43\n\x0eKeyspacesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12 \n\x05value\x18\x02 \x01(\x0b\x32\x11.vschema.Keyspace:\x02\x38\x01\x42&Z$vitess.io/vitess/go/vt/proto/vschemab\x06proto3') + serialized_options=_b('Z$vitess.io/vitess/go/vt/proto/vschema'), + serialized_pb=_b('\n\rvschema.proto\x12\x07vschema\x1a\x0bquery.proto\"3\n\x0cRoutingRules\x12#\n\x05rules\x18\x01 \x03(\x0b\x32\x14.vschema.RoutingRule\"4\n\x0bRoutingRule\x12\x12\n\nfrom_table\x18\x01 \x01(\t\x12\x11\n\tto_tables\x18\x02 \x03(\t\"\xfe\x01\n\x08Keyspace\x12\x0f\n\x07sharded\x18\x01 \x01(\x08\x12\x31\n\x08vindexes\x18\x02 \x03(\x0b\x32\x1f.vschema.Keyspace.VindexesEntry\x12-\n\x06tables\x18\x03 \x03(\x0b\x32\x1d.vschema.Keyspace.TablesEntry\x1a@\n\rVindexesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x1e\n\x05value\x18\x02 \x01(\x0b\x32\x0f.vschema.Vindex:\x02\x38\x01\x1a=\n\x0bTablesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x1d\n\x05value\x18\x02 \x01(\x0b\x32\x0e.vschema.Table:\x02\x38\x01\"\x81\x01\n\x06Vindex\x12\x0c\n\x04type\x18\x01 \x01(\t\x12+\n\x06params\x18\x02 \x03(\x0b\x32\x1b.vschema.Vindex.ParamsEntry\x12\r\n\x05owner\x18\x03 \x01(\t\x1a-\n\x0bParamsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xca\x01\n\x05Table\x12\x0c\n\x04type\x18\x01 \x01(\t\x12.\n\x0f\x63olumn_vindexes\x18\x02 \x03(\x0b\x32\x15.vschema.ColumnVindex\x12.\n\x0e\x61uto_increment\x18\x03 \x01(\x0b\x32\x16.vschema.AutoIncrement\x12 \n\x07\x63olumns\x18\x04 \x03(\x0b\x32\x0f.vschema.Column\x12\x0e\n\x06pinned\x18\x05 \x01(\t\x12!\n\x19\x63olumn_list_authoritative\x18\x06 \x01(\x08\"=\n\x0c\x43olumnVindex\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0f\n\x07\x63olumns\x18\x03 \x03(\t\"1\n\rAutoIncrement\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x10\n\x08sequence\x18\x02 \x01(\t\"1\n\x06\x43olumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x19\n\x04type\x18\x02 \x01(\x0e\x32\x0b.query.Type\"\xb6\x01\n\nSrvVSchema\x12\x35\n\tkeyspaces\x18\x01 \x03(\x0b\x32\".vschema.SrvVSchema.KeyspacesEntry\x12,\n\rrouting_rules\x18\x02 \x01(\x0b\x32\x15.vschema.RoutingRules\x1a\x43\n\x0eKeyspacesEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12 \n\x05value\x18\x02 \x01(\x0b\x32\x11.vschema.Keyspace:\x02\x38\x01\x42&Z$vitess.io/vitess/go/vt/proto/vschemab\x06proto3') , dependencies=[query__pb2.DESCRIPTOR,]) +_ROUTINGRULES = _descriptor.Descriptor( + name='RoutingRules', + full_name='vschema.RoutingRules', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='rules', full_name='vschema.RoutingRules.rules', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=39, + serialized_end=90, +) + + +_ROUTINGRULE = _descriptor.Descriptor( + name='RoutingRule', + full_name='vschema.RoutingRule', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='from_table', full_name='vschema.RoutingRule.from_table', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='to_tables', full_name='vschema.RoutingRule.to_tables', index=1, + number=2, type=9, cpp_type=9, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=92, + serialized_end=144, +) + + _KEYSPACE_VINDEXESENTRY = _descriptor.Descriptor( name='VindexesEntry', full_name='vschema.Keyspace.VindexesEntry', @@ -40,28 +109,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='value', full_name='vschema.Keyspace.VindexesEntry.value', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + serialized_options=_b('8\001'), is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=167, - serialized_end=231, + serialized_start=274, + serialized_end=338, ) _KEYSPACE_TABLESENTRY = _descriptor.Descriptor( @@ -77,28 +146,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='value', full_name='vschema.Keyspace.TablesEntry.value', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + serialized_options=_b('8\001'), is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=233, - serialized_end=294, + serialized_start=340, + serialized_end=401, ) _KEYSPACE = _descriptor.Descriptor( @@ -114,35 +183,35 @@ has_default_value=False, default_value=False, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='vindexes', full_name='vschema.Keyspace.vindexes', index=1, number=2, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='tables', full_name='vschema.Keyspace.tables', index=2, number=3, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[_KEYSPACE_VINDEXESENTRY, _KEYSPACE_TABLESENTRY, ], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=40, - serialized_end=294, + serialized_start=147, + serialized_end=401, ) @@ -159,28 +228,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='value', full_name='vschema.Vindex.ParamsEntry.value', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + serialized_options=_b('8\001'), is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=381, - serialized_end=426, + serialized_start=488, + serialized_end=533, ) _VINDEX = _descriptor.Descriptor( @@ -196,35 +265,35 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='params', full_name='vschema.Vindex.params', index=1, number=2, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='owner', full_name='vschema.Vindex.owner', index=2, number=3, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[_VINDEX_PARAMSENTRY, ], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=297, - serialized_end=426, + serialized_start=404, + serialized_end=533, ) @@ -241,56 +310,56 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='column_vindexes', full_name='vschema.Table.column_vindexes', index=1, number=2, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='auto_increment', full_name='vschema.Table.auto_increment', index=2, number=3, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='columns', full_name='vschema.Table.columns', index=3, number=4, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='pinned', full_name='vschema.Table.pinned', index=4, number=5, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='column_list_authoritative', full_name='vschema.Table.column_list_authoritative', index=5, number=6, type=8, cpp_type=7, label=1, has_default_value=False, default_value=False, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=429, - serialized_end=631, + serialized_start=536, + serialized_end=738, ) @@ -307,35 +376,35 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='name', full_name='vschema.ColumnVindex.name', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='columns', full_name='vschema.ColumnVindex.columns', index=2, number=3, type=9, cpp_type=9, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=633, - serialized_end=694, + serialized_start=740, + serialized_end=801, ) @@ -352,28 +421,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='sequence', full_name='vschema.AutoIncrement.sequence', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=696, - serialized_end=745, + serialized_start=803, + serialized_end=852, ) @@ -390,28 +459,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='type', full_name='vschema.Column.type', index=1, number=2, type=14, cpp_type=8, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=747, - serialized_end=796, + serialized_start=854, + serialized_end=903, ) @@ -428,28 +497,28 @@ has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( name='value', full_name='vschema.SrvVSchema.KeyspacesEntry.value', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[], enum_types=[ ], - options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + serialized_options=_b('8\001'), is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=868, - serialized_end=935, + serialized_start=1021, + serialized_end=1088, ) _SRVVSCHEMA = _descriptor.Descriptor( @@ -465,23 +534,31 @@ has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, - options=None, file=DESCRIPTOR), + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='routing_rules', full_name='vschema.SrvVSchema.routing_rules', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], nested_types=[_SRVVSCHEMA_KEYSPACESENTRY, ], enum_types=[ ], - options=None, + serialized_options=None, is_extendable=False, syntax='proto3', extension_ranges=[], oneofs=[ ], - serialized_start=799, - serialized_end=935, + serialized_start=906, + serialized_end=1088, ) +_ROUTINGRULES.fields_by_name['rules'].message_type = _ROUTINGRULE _KEYSPACE_VINDEXESENTRY.fields_by_name['value'].message_type = _VINDEX _KEYSPACE_VINDEXESENTRY.containing_type = _KEYSPACE _KEYSPACE_TABLESENTRY.fields_by_name['value'].message_type = _TABLE @@ -497,6 +574,9 @@ _SRVVSCHEMA_KEYSPACESENTRY.fields_by_name['value'].message_type = _KEYSPACE _SRVVSCHEMA_KEYSPACESENTRY.containing_type = _SRVVSCHEMA _SRVVSCHEMA.fields_by_name['keyspaces'].message_type = _SRVVSCHEMA_KEYSPACESENTRY +_SRVVSCHEMA.fields_by_name['routing_rules'].message_type = _ROUTINGRULES +DESCRIPTOR.message_types_by_name['RoutingRules'] = _ROUTINGRULES +DESCRIPTOR.message_types_by_name['RoutingRule'] = _ROUTINGRULE DESCRIPTOR.message_types_by_name['Keyspace'] = _KEYSPACE DESCRIPTOR.message_types_by_name['Vindex'] = _VINDEX DESCRIPTOR.message_types_by_name['Table'] = _TABLE @@ -506,6 +586,20 @@ DESCRIPTOR.message_types_by_name['SrvVSchema'] = _SRVVSCHEMA _sym_db.RegisterFileDescriptor(DESCRIPTOR) +RoutingRules = _reflection.GeneratedProtocolMessageType('RoutingRules', (_message.Message,), dict( + DESCRIPTOR = _ROUTINGRULES, + __module__ = 'vschema_pb2' + # @@protoc_insertion_point(class_scope:vschema.RoutingRules) + )) +_sym_db.RegisterMessage(RoutingRules) + +RoutingRule = _reflection.GeneratedProtocolMessageType('RoutingRule', (_message.Message,), dict( + DESCRIPTOR = _ROUTINGRULE, + __module__ = 'vschema_pb2' + # @@protoc_insertion_point(class_scope:vschema.RoutingRule) + )) +_sym_db.RegisterMessage(RoutingRule) + Keyspace = _reflection.GeneratedProtocolMessageType('Keyspace', (_message.Message,), dict( VindexesEntry = _reflection.GeneratedProtocolMessageType('VindexesEntry', (_message.Message,), dict( @@ -588,14 +682,9 @@ _sym_db.RegisterMessage(SrvVSchema.KeyspacesEntry) -DESCRIPTOR.has_options = True -DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('Z$vitess.io/vitess/go/vt/proto/vschema')) -_KEYSPACE_VINDEXESENTRY.has_options = True -_KEYSPACE_VINDEXESENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_KEYSPACE_TABLESENTRY.has_options = True -_KEYSPACE_TABLESENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_VINDEX_PARAMSENTRY.has_options = True -_VINDEX_PARAMSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) -_SRVVSCHEMA_KEYSPACESENTRY.has_options = True -_SRVVSCHEMA_KEYSPACESENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +DESCRIPTOR._options = None +_KEYSPACE_VINDEXESENTRY._options = None +_KEYSPACE_TABLESENTRY._options = None +_VINDEX_PARAMSENTRY._options = None +_SRVVSCHEMA_KEYSPACESENTRY._options = None # @@protoc_insertion_point(module_scope) diff --git a/py/vtproto/vtgateservice_pb2_grpc.py b/py/vtproto/vtgateservice_pb2_grpc.py index 410fe2a0095..5a3d3e5be67 100644 --- a/py/vtproto/vtgateservice_pb2_grpc.py +++ b/py/vtproto/vtgateservice_pb2_grpc.py @@ -136,7 +136,7 @@ class VitessServicer(object): def Execute(self, request, context): """Execute tries to route the query to the right shard. It depends on the query and bind variables to provide enough - information in conjonction with the vindexes to route the query. + information in conjunction with the vindexes to route the query. API group: v3 """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) @@ -146,7 +146,7 @@ def Execute(self, request, context): def ExecuteBatch(self, request, context): """ExecuteBatch tries to route the list of queries on the right shards. It depends on the query and bind variables to provide enough - information in conjonction with the vindexes to route the query. + information in conjunction with the vindexes to route the query. API group: v3 """ context.set_code(grpc.StatusCode.UNIMPLEMENTED) @@ -156,7 +156,7 @@ def ExecuteBatch(self, request, context): def StreamExecute(self, request, context): """StreamExecute executes a streaming query based on shards. It depends on the query and bind variables to provide enough - information in conjonction with the vindexes to route the query. + information in conjunction with the vindexes to route the query. Use this method if the query returns a large number of rows. API group: v3 """ diff --git a/py/vttest/mysql_flavor.py b/py/vttest/mysql_flavor.py index 2e9e3f95ff3..02183d3da8f 100644 --- a/py/vttest/mysql_flavor.py +++ b/py/vttest/mysql_flavor.py @@ -73,6 +73,15 @@ def my_cnf(self): ] return ":".join(files) +class MySQL80(MysqlFlavor): + """Overrides specific to MySQL 8.0.""" + + def my_cnf(self): + files = [ + os.path.join(vttop, "config/mycnf/default-fast.cnf"), + os.path.join(vttop, "config/mycnf/master_mysql80.cnf"), + ] + return ":".join(files) __mysql_flavor = None @@ -100,6 +109,8 @@ def set_mysql_flavor(flavor): __mysql_flavor = MariaDB() elif flavor == "MariaDB103": __mysql_flavor = MariaDB103() + elif flavor == "MySQL80": + __mysql_flavor = MySQL80() elif flavor == "MySQL56": __mysql_flavor = MySQL56() else: diff --git a/test.go b/test.go index 208c073dd42..84faf767082 100755 --- a/test.go +++ b/test.go @@ -73,9 +73,6 @@ For example: go run test.go test1 test2 -- --topo-server-flavor=etcd2 ` -// List of flavors for which a bootstrap Docker image is available. -const flavors = "mariadb,mysql56,mysql57,percona,percona57" - // Flags var ( flavor = flag.String("flavor", "mysql57", "comma-separated bootstrap flavor(s) to run against (when using Docker mode). Available flavors: all,"+flavors) @@ -107,6 +104,9 @@ var ( const ( statsFileName = "test/stats.json" configFileName = "test/config.json" + + // List of flavors for which a bootstrap Docker image is available. + flavors = "mariadb,mysql56,mysql57,percona,percona57" ) // Config is the overall object serialized in test/config.json. diff --git a/test/backup.py b/test/backup.py index 99c3cc55cfa..6a5173d2f2d 100755 --- a/test/backup.py +++ b/test/backup.py @@ -27,16 +27,25 @@ import utils use_mysqlctld = False - +use_xtrabackup = False +stream_mode = 'tar' tablet_master = None tablet_replica1 = None tablet_replica2 = None - +xtrabackup_args = [] new_init_db = '' db_credentials_file = '' def setUpModule(): + xtrabackup_args = ['-backup_engine_implementation', + 'xtrabackup', + '-xtrabackup_stream_mode', + stream_mode, + '-xtrabackup_user=vt_dba', + '-xtrabackup_backup_flags', + '--password=VtDbaPass'] + global new_init_db, db_credentials_file global tablet_master, tablet_replica1, tablet_replica2 @@ -50,21 +59,6 @@ def setUpModule(): try: environment.topo_server().setup() - # Determine which column is used for user passwords in this MySQL version. - proc = tablet_master.init_mysql() - if use_mysqlctld: - tablet_master.wait_for_mysqlctl_socket() - else: - utils.wait_procs([proc]) - try: - tablet_master.mquery('mysql', 'select password from mysql.user limit 0', - user='root') - password_col = 'password' - except MySQLdb.DatabaseError: - password_col = 'authentication_string' - utils.wait_procs([tablet_master.teardown_mysql()]) - tablet_master.remove_tree(ignore_options=True) - # Create a new init_db.sql file that sets up passwords for all users. # Then we use a db-credentials-file with the passwords. new_init_db = environment.tmproot + '/init_db_with_passwords.sql' @@ -74,20 +68,14 @@ def setUpModule(): fd.write(init_db) fd.write(''' # Set real passwords for all users. -UPDATE mysql.user SET %s = PASSWORD('RootPass') - WHERE User = 'root' AND Host = 'localhost'; -UPDATE mysql.user SET %s = PASSWORD('VtDbaPass') - WHERE User = 'vt_dba' AND Host = 'localhost'; -UPDATE mysql.user SET %s = PASSWORD('VtAppPass') - WHERE User = 'vt_app' AND Host = 'localhost'; -UPDATE mysql.user SET %s = PASSWORD('VtAllprivsPass') - WHERE User = 'vt_allprivs' AND Host = 'localhost'; -UPDATE mysql.user SET %s = PASSWORD('VtReplPass') - WHERE User = 'vt_repl' AND Host = '%%'; -UPDATE mysql.user SET %s = PASSWORD('VtFilteredPass') - WHERE User = 'vt_filtered' AND Host = 'localhost'; +ALTER USER 'root'@'localhost' IDENTIFIED BY 'RootPass'; +ALTER USER 'vt_dba'@'localhost' IDENTIFIED BY 'VtDbaPass'; +ALTER USER 'vt_app'@'localhost' IDENTIFIED BY 'VtAppPass'; +ALTER USER 'vt_allprivs'@'localhost' IDENTIFIED BY 'VtAllPrivsPass'; +ALTER USER 'vt_repl'@'%' IDENTIFIED BY 'VtReplPass'; +ALTER USER 'vt_filtered'@'localhost' IDENTIFIED BY 'VtFilteredPass'; FLUSH PRIVILEGES; -''' % tuple([password_col] * 6)) +''') credentials = { 'vt_dba': ['VtDbaPass'], 'vt_app': ['VtAppPass'], @@ -152,14 +140,15 @@ def setUp(self): for t in tablet_master, tablet_replica1: t.create_db('vt_test_keyspace') + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) tablet_master.init_tablet('replica', 'test_keyspace', '0', start=True, supports_backups=True, - extra_args=['-db-credentials-file', - db_credentials_file]) + extra_args=xtra_args) tablet_replica1.init_tablet('replica', 'test_keyspace', '0', start=True, supports_backups=True, - extra_args=['-db-credentials-file', - db_credentials_file]) + extra_args=xtra_args) utils.run_vtctl(['InitShardMaster', '-force', 'test_keyspace/0', tablet_master.tablet_alias]) @@ -209,12 +198,17 @@ def _check_data(self, t, count, msg): def _restore(self, t, tablet_type='replica'): """Erase mysql/tablet dir, then start tablet with restore enabled.""" self._reset_tablet_dir(t) + + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) + t.start_vttablet(wait_for_state='SERVING', init_tablet_type=tablet_type, init_keyspace='test_keyspace', init_shard='0', supports_backups=True, - extra_args=['-db-credentials-file', db_credentials_file]) + extra_args=xtra_args) # check semi-sync is enabled for replica, disabled for rdonly. if tablet_type == 'replica': @@ -422,6 +416,24 @@ def test_master_slave_same_backup(self): self._check_data(tablet_replica1, 3, 'replica1 getting data from restored master') + # This is to test that replicationPosition is processed correctly + # while doing backup/restore after a reparent. + # It is written into the MANIFEST and read back from the MANIFEST. + + # Take another backup on the slave. + utils.run_vtctl(['Backup', tablet_replica1.tablet_alias], auto_log=True) + + # Insert more data on replica2 (current master). + self._insert_data(tablet_replica2, 4) + + # Force replica1 to restore from backup. + tablet_replica1.kill_vttablet() + self._restore(tablet_replica1) + + # Wait for replica1 to catch up. + self._check_data(tablet_replica1, 4, + 'replica1 getting data from master after reparent+backup+restore') + tablet_replica2.kill_vttablet() def _restore_old_master_test(self, restore_method): @@ -484,11 +496,20 @@ def _terminated_restore(t): logging.info('waiting for restore to finish') utils.wait_for_tablet_type(t.tablet_alias, 'replica', timeout=30) + # this test is run standalone with xtrabackup because it fails when run + # with the other master restore tests + if use_xtrabackup: + return + utils.Vtctld().start() self._restore_old_master_test(_terminated_restore) def test_backup_transform(self): """Use a transform, tests we backup and restore properly.""" + if use_xtrabackup: + # not supported + return + # Insert data on master, make sure slave gets it. tablet_master.mquery('vt_test_keyspace', self._create_vt_insert_test) self._insert_data(tablet_master, 1) @@ -496,13 +517,18 @@ def test_backup_transform(self): # Restart the replica with the transform parameter. tablet_replica1.kill_vttablet() + + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) + + hook_args = ['-backup_storage_hook', + 'test_backup_transform', + '-backup_storage_compress=false'] + xtra_args.extend(hook_args) + tablet_replica1.start_vttablet(supports_backups=True, - extra_args=[ - '-backup_storage_hook', - 'test_backup_transform', - '-backup_storage_compress=false', - '-db-credentials-file', - db_credentials_file]) + extra_args=xtra_args) # Take a backup, it should work. utils.run_vtctl(['Backup', tablet_replica1.tablet_alias], auto_log=True) @@ -537,13 +563,19 @@ def test_backup_transform(self): def test_backup_transform_error(self): """Use a transform, force an error, make sure the backup fails.""" + if use_xtrabackup: + # not supported + return + # Restart the replica with the transform parameter. tablet_replica1.kill_vttablet() + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) + hook_args = ['-backup_storage_hook','test_backup_error'] + xtra_args.extend(hook_args) tablet_replica1.start_vttablet(supports_backups=True, - extra_args=['-backup_storage_hook', - 'test_backup_error', - '-db-credentials-file', - db_credentials_file]) + extra_args=xtra_args) # This will fail, make sure we get the right error. _, err = utils.run_vtctl(['Backup', tablet_replica1.tablet_alias], diff --git a/test/base_sharding.py b/test/base_sharding.py index 64fefd28f89..5cdb5398271 100644 --- a/test/base_sharding.py +++ b/test/base_sharding.py @@ -54,7 +54,7 @@ class BaseShardingTest(object): def _insert_value(self, tablet_obj, table, mid, msg, keyspace_id): k = utils.uint64_to_hex(keyspace_id) tablet_obj.mquery( - 'vt_test_keyspace', + tablet_obj.dbname, ['begin', 'insert into %s(parent_id, id, msg, custom_ksid_col) ' 'values(%d, %d, "%s", 0x%x) /* vtgate:: keyspace_id:%s */ ' @@ -84,7 +84,7 @@ def _insert_multi_value(self, tablet_obj, table, mids, msgs, keyspace_ids): querystr += values_str tablet_obj.mquery( - 'vt_test_keyspace', + tablet_obj.dbname, ['begin', querystr, 'commit'], @@ -119,7 +119,7 @@ def _get_value(self, tablet_obj, table, mid): A tuple of results. """ return tablet_obj.mquery( - 'vt_test_keyspace', + tablet_obj.dbname, 'select parent_id, id, msg, custom_ksid_col from %s ' 'where parent_id=%d and id=%d' % (table, fixed_parent_id, mid)) diff --git a/test/config.json b/test/config.json index 87616c1491f..86c8ba830f3 100644 --- a/test/config.json +++ b/test/config.json @@ -19,6 +19,33 @@ "RetryMax": 0, "Tags": [] }, + "xtrabackup": { + "File": "xtrabackup.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 3, + "RetryMax": 0, + "Tags": [] + }, + "xtrabackup_xbstream": { + "File": "xtrabackup_xbstream.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 3, + "RetryMax": 0, + "Tags": [] + }, + "xtrabackup_xtra": { + "File": "xtrabackup_xtra.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 3, + "RetryMax": 0, + "Tags": [] + }, "binlog": { "File": "binlog.py", "Args": [], @@ -79,6 +106,15 @@ "worker_test" ] }, + "initial_sharding_multi": { + "File": "initial_sharding_multi.py", + "Args": [], + "Command": [], + "Manual": false, + "Shard": 3, + "RetryMax": 0, + "Tags": [] + }, "initial_sharding_bytes": { "File": "initial_sharding_bytes.py", "Args": [], diff --git a/test/initial_sharding_multi.py b/test/initial_sharding_multi.py new file mode 100755 index 00000000000..998864e3f67 --- /dev/null +++ b/test/initial_sharding_multi.py @@ -0,0 +1,801 @@ +#!/usr/bin/env python +# +# Copyright 2017 Google Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +"""This test simulates the first time a database has to be split +in a multi-vttablet-single-mysql environment + +We have 2 keyspaces. One keyspace is in managing mode. It's vttablets +own the MySQL instances and can reparent, start/stop server, start/stop +replication etc. Other keyspace is in non-managing mode and cannot do +any of these actions. Only TabletExternallyReparented is allowed, but +resharding should still work. + +For each keyspace: +- we start with a keyspace with a single shard and a single table +- we add and populate the sharding key +- we set the sharding key in the topology +- we clone into 2 instances +- we enable filtered replication +- we move all serving types +- we remove the source tablets +- we remove the original shard +""" + +import json +import logging +import unittest +from vtdb import keyrange_constants + +import MySQLdb + +import base_sharding +import environment +import tablet +import utils + +# initial shard, covers everything +ks1_shard_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') + +# split shards +# range '' - 80 +ks1_shard_0_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_0_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_0_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') +# range 80 - '' +ks1_shard_1_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_1_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks1_shard_1_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') + +ks1_tablets = { + '0': {'master':ks1_shard_master, 'replica':ks1_shard_replica, 'rdonly':ks1_shard_rdonly1}, + '-80': {'master':ks1_shard_0_master, 'replica':ks1_shard_0_replica, 'rdonly':ks1_shard_0_rdonly1}, + '80-': {'master':ks1_shard_1_master, 'replica':ks1_shard_1_replica, 'rdonly':ks1_shard_1_rdonly1} +} + +# initial shard, covers everything +ks2_shard_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') + +# split shards +# range '' - 80 +ks2_shard_0_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_0_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_0_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') +# range 80 - '' +ks2_shard_1_master = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_1_replica = tablet.Tablet(vt_dba_passwd='VtDbaPass') +ks2_shard_1_rdonly1 = tablet.Tablet(vt_dba_passwd='VtDbaPass') + +ks2_tablets = { + '0': {'master':ks2_shard_master, 'replica':ks2_shard_replica, 'rdonly':ks2_shard_rdonly1}, + '-80': {'master':ks2_shard_0_master, 'replica':ks2_shard_0_replica, 'rdonly':ks2_shard_0_rdonly1}, + '80-': {'master':ks2_shard_1_master, 'replica':ks2_shard_1_replica, 'rdonly':ks2_shard_1_rdonly1} +} + +all_mysql_tablets = [ks1_shard_master, ks1_shard_replica, ks1_shard_rdonly1, + ks1_shard_0_master, ks1_shard_0_replica, ks1_shard_0_rdonly1, + ks1_shard_1_master, ks1_shard_1_replica, ks1_shard_1_rdonly1] + +all_other_tablets = [ks2_shard_master, ks2_shard_replica, ks2_shard_rdonly1, + ks2_shard_0_master, ks2_shard_0_replica, ks2_shard_0_rdonly1, + ks2_shard_1_master, ks2_shard_1_replica, ks2_shard_1_rdonly1] + +def setUpModule(): + global new_init_db, db_credentials_file + + try: + # Determine which column is used for user passwords in this MySQL version. + proc = ks1_shard_master.init_mysql() + utils.wait_procs([proc]) + try: + ks1_shard_master.mquery('mysql', 'select password from mysql.user limit 0', + user='root') + password_col = 'password' + except MySQLdb.DatabaseError: + password_col = 'authentication_string' + utils.wait_procs([ks1_shard_master.teardown_mysql()]) + ks1_shard_master.remove_tree(ignore_options=True) + + # Create a new init_db.sql file that sets up passwords for all users. + # Then we use a db-credentials-file with the passwords. + new_init_db = environment.tmproot + '/init_db_with_passwords.sql' + with open(environment.vttop + '/config/init_db.sql') as fd: + init_db = fd.read() + with open(new_init_db, 'w') as fd: + fd.write(init_db) + fd.write(''' +# Set real passwords for all users. +ALTER USER 'root'@'localhost' IDENTIFIED BY 'RootPass'; +ALTER USER 'vt_dba'@'localhost' IDENTIFIED BY 'VtDbaPass'; +ALTER USER 'vt_app'@'localhost' IDENTIFIED BY 'VtAppPass'; +ALTER USER 'vt_allprivs'@'localhost' IDENTIFIED BY 'VtAllPrivsPass'; +ALTER USER 'vt_repl'@'%' IDENTIFIED BY 'VtReplPass'; +ALTER USER 'vt_filtered'@'localhost' IDENTIFIED BY 'VtFilteredPass'; + +# connecting through a port requires 127.0.0.1 +# --host=localhost will connect through socket +CREATE USER 'vt_dba'@'127.0.0.1' IDENTIFIED BY 'VtDbaPass'; +GRANT ALL ON *.* TO 'vt_dba'@'127.0.0.1'; +GRANT GRANT OPTION ON *.* TO 'vt_dba'@'127.0.0.1'; + +# User for app traffic, with global read-write access. +CREATE USER 'vt_app'@'127.0.0.1' IDENTIFIED BY 'VtAppPass'; +GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE, + REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES, + LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW, + SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER + ON *.* TO 'vt_app'@'127.0.0.1'; + +# User for administrative operations that need to be executed as non-SUPER. +# Same permissions as vt_app here. +CREATE USER 'vt_allprivs'@'127.0.0.1' IDENTIFIED BY 'VtAllPrivsPass'; +GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE, + REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES, + LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW, + SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER + ON *.* TO 'vt_allprivs'@'127.0.0.1'; + +# User for Vitess filtered replication (binlog player). +# Same permissions as vt_app. +CREATE USER 'vt_filtered'@'127.0.0.1' IDENTIFIED BY 'VtFilteredPass'; +GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE, + REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES, + LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW, + SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER + ON *.* TO 'vt_filtered'@'127.0.0.1'; + +FLUSH PRIVILEGES; +''') + credentials = { + 'vt_dba': ['VtDbaPass'], + 'vt_app': ['VtAppPass'], + 'vt_allprivs': ['VtAllprivsPass'], + 'vt_repl': ['VtReplPass'], + 'vt_filtered': ['VtFilteredPass'], + } + db_credentials_file = environment.tmproot+'/db_credentials.json' + with open(db_credentials_file, 'w') as fd: + fd.write(json.dumps(credentials)) + + setup_procs = [t.init_mysql(use_rbr=True, init_db=new_init_db, + extra_args=['-db-credentials-file', + db_credentials_file]) for t in all_mysql_tablets] + utils.wait_procs(setup_procs) + for i in range(0, len(all_other_tablets)): + all_other_tablets[i].mysql_port = all_mysql_tablets[i].mysql_port + + environment.topo_server().setup() + + except: + tearDownModule() + raise + + +def tearDownModule(): + utils.required_teardown() + if utils.options.skip_teardown: + return + + teardown_procs = [t.teardown_mysql(extra_args=['-db-credentials-file', db_credentials_file]) for t in all_mysql_tablets] + utils.wait_procs(teardown_procs, raise_on_error=False) + environment.topo_server().teardown() + utils.kill_sub_processes() + utils.remove_tmp_files() + for t in all_mysql_tablets: + t.remove_tree() + for t in all_other_tablets: + t.remove_tree() + + +class TestInitialSharding(unittest.TestCase, base_sharding.BaseShardingTest): + + # create_schema will create the same schema on the keyspace + def _create_schema(self, keyspace): + # Note that the primary key columns are not defined first on purpose to test + # that a reordered column list is correctly used everywhere in vtworker. + create_table_template = '''create table %s( +msg varchar(64), +id bigint not null, +parent_id bigint not null, +primary key (parent_id, id), +index by_msg (msg) +) Engine=InnoDB''' + + utils.run_vtctl(['ApplySchema', + '-sql=' + create_table_template % ('resharding1'), + keyspace], + auto_log=True) + + def _add_sharding_key_to_schema(self, keyspace): + if base_sharding.keyspace_id_type == keyrange_constants.KIT_BYTES: + t = 'varbinary(64)' + else: + t = 'bigint(20) unsigned' + sql = 'alter table %s add custom_ksid_col ' + t + utils.run_vtctl(['ApplySchema', + '-sql=' + sql % ('resharding1'), + keyspace], + auto_log=True) + + def _mark_sharding_key_not_null(self, keyspace): + if base_sharding.keyspace_id_type == keyrange_constants.KIT_BYTES: + t = 'varbinary(64)' + else: + t = 'bigint(20) unsigned' + sql = 'alter table %s modify custom_ksid_col ' + t + ' not null' + utils.run_vtctl(['ApplySchema', + '-sql=' + sql % ('resharding1'), + keyspace], + auto_log=True) + + # _insert_startup_value inserts a value in the MySQL database before it + # is sharded + def _insert_startup_value(self, keyspace, tablet_obj, table, mid, msg): + tablet_obj.mquery('vt_' + keyspace, [ + 'begin', + 'insert into %s(parent_id, id, msg) values(%d, %d, "%s")' % + (table, base_sharding.fixed_parent_id, mid, msg), + 'commit' + ], write=True) + + def _insert_startup_values(self, keyspace, master_tablet): + self._insert_startup_value(keyspace, master_tablet, 'resharding1', 1, 'msg1') + self._insert_startup_value(keyspace, master_tablet, 'resharding1', 2, 'msg2') + self._insert_startup_value(keyspace, master_tablet, 'resharding1', 3, 'msg3') + + def _backfill_keyspace_id(self, keyspace, tablet_obj): + tablet_obj.mquery('vt_' + keyspace, [ + 'begin', + 'update resharding1 set custom_ksid_col=0x1000000000000000 where id=1', + 'update resharding1 set custom_ksid_col=0x9000000000000000 where id=2', + 'update resharding1 set custom_ksid_col=0xD000000000000000 where id=3', + 'commit' + ], write=True) + + def _check_startup_values(self, keyspace, tablets): + # check first value is in the left shard + for t in tablets['-80'].values(): + self._check_value(t, 'resharding1', 1, 'msg1', 0x1000000000000000) + for t in tablets['80-'].values(): + self._check_value(t, 'resharding1', 1, 'msg1', + 0x1000000000000000, should_be_here=False) + + # check second value is in the right shard + for t in tablets['-80'].values(): + self._check_value(t, 'resharding1', 2, 'msg2', 0x9000000000000000, + should_be_here=False) + for t in tablets['80-'].values(): + self._check_value(t, 'resharding1', 2, 'msg2', 0x9000000000000000) + + # check third value is in the right shard too + for t in tablets['-80'].values(): + self._check_value(t, 'resharding1', 3, 'msg3', 0xD000000000000000, + should_be_here=False) + for t in tablets['80-'].values(): + self._check_value(t, 'resharding1', 3, 'msg3', 0xD000000000000000) + + def _insert_lots(self, keyspace, master_tablet, count, base=0): + for i in xrange(count): + self._insert_value(master_tablet, 'resharding1', 10000 + base + i, + 'msg-range1-%d' % i, 0xA000000000000000 + base + i) + self._insert_value(master_tablet, 'resharding1', 20000 + base + i, + 'msg-range2-%d' % i, 0xE000000000000000 + base + i) + + # _check_lots returns how many of the values we have, in percents. + def _check_lots(self, replica_tablet, count, base=0): + found = 0 + for i in xrange(count): + if self._is_value_present_and_correct(replica_tablet, 'resharding1', + 10000 + base + i, 'msg-range1-%d' % + i, 0xA000000000000000 + base + i): + found += 1 + if self._is_value_present_and_correct(replica_tablet, 'resharding1', + 20000 + base + i, 'msg-range2-%d' % + i, 0xE000000000000000 + base + i): + found += 1 + percent = found * 100 / count / 2 + logging.debug('I have %d%% of the data', percent) + return percent + + def _check_lots_timeout(self, replica_tablet, count, threshold, timeout, base=0): + while True: + value = self._check_lots(replica_tablet, count, base=base) + if value >= threshold: + return value + timeout = utils.wait_step('enough data went through', timeout) + + # _check_lots_not_present makes sure no data is in the wrong shard + def _check_lots_not_present(self, replica_tablet, count, base=0): + for i in xrange(count): + self._check_value(replica_tablet, 'resharding1', 10000 + base + i, + 'msg-range1-%d' % i, 0xA000000000000000 + base + i, + should_be_here=False) + self._check_value(replica_tablet, 'resharding1', 20000 + base + i, + 'msg-range2-%d' % i, 0xE000000000000000 + base + i, + should_be_here=False) + + def _test_resharding(self, keyspace, tablet_map, external_mysql=False): + # create the keyspace with just one shard + shard_master = tablet_map['0']['master'] + shard_replica = tablet_map['0']['replica'] + shard_rdonly = tablet_map['0']['rdonly'] + shard_0_master = tablet_map['-80']['master'] + shard_0_replica = tablet_map['-80']['replica'] + shard_0_rdonly = tablet_map['-80']['rdonly'] + shard_1_master = tablet_map['80-']['master'] + shard_1_replica = tablet_map['80-']['replica'] + shard_1_rdonly = tablet_map['80-']['rdonly'] + shard_master.init_tablet( + 'replica', + keyspace=keyspace, + shard='0', + tablet_index=0, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_replica.init_tablet( + 'replica', + keyspace=keyspace, + shard='0', + tablet_index=1, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_rdonly.init_tablet( + 'rdonly', + keyspace=keyspace, + shard='0', + tablet_index=2, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + + for t in [shard_master, shard_replica, shard_rdonly]: + t.create_db('vt_' + keyspace) + + shard_master.start_vttablet(wait_for_state=None, + binlog_use_v3_resharding_mode=False, + supports_backups=False, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_rdonly.start_vttablet(wait_for_state=None, + binlog_use_v3_resharding_mode=False, + supports_backups=False, + extra_args=['-db-credentials-file', db_credentials_file]) + + if not external_mysql: + for t in [shard_master, shard_rdonly]: + t.wait_for_vttablet_state('NOT_SERVING') + + # start replica + shard_replica.start_vttablet(wait_for_state=None, + binlog_use_v3_resharding_mode=False, + supports_backups=False, + extra_args=['-db-credentials-file', db_credentials_file]) + + if not external_mysql: + shard_replica.wait_for_vttablet_state('NOT_SERVING') + + # reparent to make the tablets work + utils.run_vtctl(['InitShardMaster', '-force', keyspace+'/0', + shard_master.tablet_alias], auto_log=True) + + utils.wait_for_tablet_type(shard_replica.tablet_alias, 'replica') + utils.wait_for_tablet_type(shard_rdonly.tablet_alias, 'rdonly') + else: + shard_replica.wait_for_vttablet_state('SERVING') + # default mode is VTCTL_AUTO which makes this command hang + _, stderr = utils.run_vtctl(['TabletExternallyReparented', shard_master.tablet_alias], mode=utils.VTCTL_VTCTL, auto_log=True) + + for t in [shard_master, shard_replica, shard_rdonly]: + t.wait_for_vttablet_state('SERVING') + + # create the tables and add startup values + self._create_schema(keyspace) + self._insert_startup_values(keyspace, shard_master) + + # reload schema on all tablets so we can query them + for t in [shard_master, shard_replica, shard_rdonly]: + utils.run_vtctl(['ReloadSchema', t.tablet_alias], auto_log=True) + + # We must start vtgate after tablets are up, or else wait until 1min refresh + # (that is the tablet_refresh_interval parameter for discovery gateway) + # we want cache_ttl at zero so we re-read the topology for every test query. + + utils.VtGate().start(cache_ttl='0', tablets=[ + shard_master, shard_replica, shard_rdonly]) + utils.vtgate.wait_for_endpoints(keyspace + '.0.master', 1) + utils.vtgate.wait_for_endpoints(keyspace + '.0.replica', 1) + utils.vtgate.wait_for_endpoints(keyspace + '.0.rdonly', 1) + + # check the Map Reduce API works correctly, should use ExecuteShards, + # as we're not sharded yet. + # we have 3 values in the database, asking for 4 splits will get us + # a single query. + sql = 'select id, msg from resharding1' + s = utils.vtgate.split_query(sql, keyspace, 4) + self.assertEqual(len(s), 1) + self.assertEqual(s[0]['shard_part']['shards'][0], '0') + + # change the schema, backfill keyspace_id, and change schema again + self._add_sharding_key_to_schema(keyspace) + self._backfill_keyspace_id(keyspace, shard_master) + self._mark_sharding_key_not_null(keyspace) + + # now we can be a sharded keyspace (and propagate to SrvKeyspace) + utils.run_vtctl(['SetKeyspaceShardingInfo', keyspace, + 'custom_ksid_col', base_sharding.keyspace_id_type]) + utils.run_vtctl(['RebuildKeyspaceGraph', keyspace], + auto_log=True) + + # run a health check on source replica so it responds to discovery + utils.run_vtctl(['RunHealthCheck', shard_replica.tablet_alias]) + + # create the split shards + shard_0_master.init_tablet( + 'replica', + keyspace=keyspace, + shard='-80', + tablet_index=0, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_0_replica.init_tablet( + 'replica', + keyspace=keyspace, + shard='-80', + tablet_index=1, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_0_rdonly.init_tablet( + 'rdonly', + keyspace=keyspace, + shard='-80', + tablet_index=2, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_1_master.init_tablet( + 'replica', + keyspace=keyspace, + shard='80-', + tablet_index=0, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_1_replica.init_tablet( + 'replica', + keyspace=keyspace, + shard='80-', + tablet_index=1, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + shard_1_rdonly.init_tablet( + 'rdonly', + keyspace=keyspace, + shard='80-', + tablet_index=2, + external_mysql=external_mysql, + extra_args=['-db-credentials-file', db_credentials_file]) + + for t in [shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly]: + t.create_db('vt_' + keyspace) + t.start_vttablet(wait_for_state=None, + binlog_use_v3_resharding_mode=False, + supports_backups=False, + extra_args=['-db-credentials-file', db_credentials_file]) + + sharded_tablets = [shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly] + if not external_mysql: + for t in sharded_tablets: + t.wait_for_vttablet_state('NOT_SERVING') + + utils.run_vtctl(['InitShardMaster', '-force', keyspace + '/-80', + shard_0_master.tablet_alias], auto_log=True) + utils.run_vtctl(['InitShardMaster', '-force', keyspace + '/80-', + shard_1_master.tablet_alias], auto_log=True) + + for t in [shard_0_replica, shard_1_replica]: + utils.wait_for_tablet_type(t.tablet_alias, 'replica') + for t in [shard_0_rdonly, shard_1_rdonly]: + utils.wait_for_tablet_type(t.tablet_alias, 'rdonly') + + for t in sharded_tablets: + t.wait_for_vttablet_state('SERVING') + else: + # default mode is VTCTL_AUTO which makes this command hang + _, stderr = utils.run_vtctl(['TabletExternallyReparented', shard_0_master.tablet_alias], mode=utils.VTCTL_VTCTL, auto_log=True) + _, stderr = utils.run_vtctl(['TabletExternallyReparented', shard_1_master.tablet_alias], mode=utils.VTCTL_VTCTL, auto_log=True) + + # must restart vtgate after tablets are up, or else wait until 1min refresh + # we want cache_ttl at zero so we re-read the topology for every test query. + utils.vtgate.kill() + + utils.vtgate = None + utils.VtGate().start(cache_ttl='0', tablets=[ + shard_master, shard_replica, shard_rdonly, + shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly]) + var = None + + # Wait for the endpoints, either local or remote. + utils.vtgate.wait_for_endpoints(keyspace + '.0.master', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.0.replica', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.0.rdonly', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.-80.master', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.-80.replica', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.-80.rdonly', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.80-.master', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.80-.replica', 1, var=var) + utils.vtgate.wait_for_endpoints(keyspace + '.80-.rdonly', 1, var=var) + + # check the Map Reduce API works correctly, should use ExecuteKeyRanges now, + # as we are sharded (with just one shard). + # again, we have 3 values in the database, asking for 4 splits will get us + # a single query. + sql = 'select id, msg from resharding1' + s = utils.vtgate.split_query(sql, keyspace, 4) + self.assertEqual(len(s), 1) + self.assertEqual(s[0]['key_range_part']['keyspace'], keyspace) + # There must be one empty KeyRange which represents the full keyspace. + self.assertEqual(len(s[0]['key_range_part']['key_ranges']), 1) + self.assertEqual(s[0]['key_range_part']['key_ranges'][0], {}) + + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -\n' + 'Partitions(rdonly): -\n' + 'Partitions(replica): -\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # we need to create the schema, and the worker will do data copying + for keyspace_shard in (keyspace + '/-80', keyspace + '/80-'): + utils.run_vtctl(['CopySchemaShard', + '--exclude_tables', 'unrelated', + shard_rdonly.tablet_alias, + keyspace_shard], + auto_log=True) + utils.run_vtctl(['RunHealthCheck', shard_rdonly.tablet_alias]) + + # Run vtworker as daemon for the following SplitClone commands. + worker_proc, worker_port, worker_rpc_port = utils.run_vtworker_bg( + ['--cell', 'test_nj', '--command_display_interval', '10ms', + '--use_v3_resharding_mode=false'], + auto_log=True) + + # Initial clone (online). + workerclient_proc = utils.run_vtworker_client_bg( + ['SplitClone', + '--offline=false', + '--exclude_tables', 'unrelated', + '--chunk_count', '10', + '--min_rows_per_chunk', '1', + '--min_healthy_rdonly_tablets', '1', + keyspace + '/0'], + worker_rpc_port) + utils.wait_procs([workerclient_proc]) + self.verify_reconciliation_counters(worker_port, 'Online', 'resharding1', + 3, 0, 0, 0) + + # Reset vtworker such that we can run the next command. + workerclient_proc = utils.run_vtworker_client_bg(['Reset'], worker_rpc_port) + utils.wait_procs([workerclient_proc]) + + # Modify the destination shard. SplitClone will revert the changes. + # Delete row 1 (provokes an insert). + shard_0_master.mquery('vt_' + keyspace, + 'delete from resharding1 where id=1', write=True) + # Delete row 2 (provokes an insert). + shard_1_master.mquery('vt_' + keyspace, + 'delete from resharding1 where id=2', write=True) + # Update row 3 (provokes an update). + shard_1_master.mquery('vt_' + keyspace, + "update resharding1 set msg='msg-not-3' where id=3", + write=True) + # Insert row 4 (provokes a delete). + self._insert_value(shard_1_master, 'resharding1', 4, 'msg4', + 0xD000000000000000) + + workerclient_proc = utils.run_vtworker_client_bg( + ['SplitClone', + '--exclude_tables', 'unrelated', + '--chunk_count', '10', + '--min_rows_per_chunk', '1', + '--min_healthy_rdonly_tablets', '1', + keyspace + '/0'], + worker_rpc_port) + utils.wait_procs([workerclient_proc]) + self.verify_reconciliation_counters(worker_port, 'Online', 'resharding1', + 2, 1, 1, 0) + self.verify_reconciliation_counters(worker_port, 'Offline', 'resharding1', + 0, 0, 0, 3) + # Terminate worker daemon because it is no longer needed. + utils.kill_sub_process(worker_proc, soft=True) + + # check the startup values are in the right place + self._check_startup_values(keyspace, tablet_map) + + # check the schema too + utils.run_vtctl(['ValidateSchemaKeyspace', keyspace], auto_log=True) + + # check the binlog players are running + logging.debug('Waiting for binlog players to start on new masters...') + self.check_destination_master(shard_0_master, [keyspace + '/0']) + self.check_destination_master(shard_1_master, [keyspace + '/0']) + + # check that binlog server exported the stats vars + self.check_binlog_server_vars(shard_replica, horizontal=True) + + # testing filtered replication: insert a bunch of data on shard 1, + # check we get most of it after a few seconds, wait for binlog server + # timeout, check we get all of it. + logging.debug('Inserting lots of data on source shard') + self._insert_lots(keyspace, shard_master, 1000) + logging.debug('Checking 80 percent of data is sent quickly') + v = self._check_lots_timeout(shard_1_replica, 1000, 80, 5) + if v != 100: + logging.debug('Checking all data goes through eventually') + self._check_lots_timeout(shard_1_replica, 1000, 100, 20) + logging.debug('Checking no data was sent the wrong way') + self._check_lots_not_present(shard_0_replica, 1000) + self.check_binlog_player_vars(shard_0_master, [keyspace + '/0'], + seconds_behind_master_max=30) + self.check_binlog_player_vars(shard_1_master, [keyspace + '/0'], + seconds_behind_master_max=30) + self.check_binlog_server_vars(shard_replica, horizontal=True, + min_statements=1000, min_transactions=1000) + + # use vtworker to compare the data + for t in [shard_0_rdonly, shard_1_rdonly]: + utils.run_vtctl(['RunHealthCheck', t.tablet_alias]) + + # get status for the destination master tablet, make sure we have it all + if not external_mysql: + self.check_running_binlog_player(shard_0_master, 2000, 2000) + self.check_running_binlog_player(shard_1_master, 6000, 2000) + else: + self.check_running_binlog_player(shard_0_master, 2002, 2002) + self.check_running_binlog_player(shard_1_master, 6002, 2002) + + + # check we can't migrate the master just yet + utils.run_vtctl(['MigrateServedTypes', keyspace + '/0', 'master'], + expect_fail=True) + + # now serve rdonly from the split shards + utils.run_vtctl(['MigrateServedTypes', keyspace + '/0', 'rdonly'], + auto_log=True) + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # make sure rdonly tablets are back to serving before hitting vtgate. + for t in [shard_0_rdonly, shard_1_rdonly]: + t.wait_for_vttablet_state('SERVING') + + utils.vtgate.wait_for_endpoints(keyspace + '.-80.rdonly', 1) + utils.vtgate.wait_for_endpoints(keyspace + '.80-.rdonly', 1) + + # check the Map Reduce API works correctly, should use ExecuteKeyRanges + # on both destination shards now. + # we ask for 2 splits to only have one per shard + sql = 'select id, msg from resharding1' + timeout = 10.0 + while True: + try: + s = utils.vtgate.split_query(sql, keyspace, 2) + break + except Exception: # pylint: disable=broad-except + timeout = utils.wait_step( + 'vtgate executes split_query properly', timeout) + self.assertEqual(len(s), 2) + self.assertEqual(s[0]['key_range_part']['keyspace'], keyspace) + self.assertEqual(s[1]['key_range_part']['keyspace'], keyspace) + self.assertEqual(len(s[0]['key_range_part']['key_ranges']), 1) + self.assertEqual(len(s[1]['key_range_part']['key_ranges']), 1) + + # then serve replica from the split shards + source_tablet = shard_replica + destination_tablets = [shard_0_replica, shard_1_replica] + + utils.run_vtctl( + ['MigrateServedTypes', keyspace + '/0', 'replica'], auto_log=True) + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -80 80-\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # move replica back and forth + utils.run_vtctl( + ['MigrateServedTypes', '-reverse', keyspace + '/0', 'replica'], + auto_log=True) + # After a backwards migration, queryservice should be enabled on + # source and disabled on destinations + utils.check_tablet_query_service(self, source_tablet, True, False) + utils.check_tablet_query_services(self, destination_tablets, False, True) + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + utils.run_vtctl(['MigrateServedTypes', keyspace + '/0', 'replica'], + auto_log=True) + # After a forwards migration, queryservice should be disabled on + # source and enabled on destinations + utils.check_tablet_query_service(self, source_tablet, False, True) + utils.check_tablet_query_services(self, destination_tablets, True, False) + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -80 80-\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # then serve master from the split shards + utils.run_vtctl(['MigrateServedTypes', keyspace + '/0', 'master'], + auto_log=True) + utils.check_srv_keyspace('test_nj', keyspace, + 'Partitions(master): -80 80-\n' + 'Partitions(rdonly): -80 80-\n' + 'Partitions(replica): -80 80-\n', + keyspace_id_type=base_sharding.keyspace_id_type, + sharding_column_name='custom_ksid_col') + + # check the binlog players are gone now + self.check_no_binlog_player(shard_0_master) + self.check_no_binlog_player(shard_1_master) + + def kill_all_tablets(self, keyspace, tablet_map): + shard_master = tablet_map['0']['master'] + shard_replica = tablet_map['0']['replica'] + shard_rdonly = tablet_map['0']['rdonly'] + shard_0_master = tablet_map['-80']['master'] + shard_0_replica = tablet_map['-80']['replica'] + shard_0_rdonly = tablet_map['-80']['rdonly'] + shard_1_master = tablet_map['80-']['master'] + shard_1_replica = tablet_map['80-']['replica'] + shard_1_rdonly = tablet_map['80-']['rdonly'] + + # remove the original tablets in the original shard + tablet.kill_tablets([shard_master, shard_replica, shard_rdonly]) + for t in [shard_replica, shard_rdonly]: + utils.run_vtctl(['DeleteTablet', t.tablet_alias], auto_log=True) + utils.run_vtctl(['DeleteTablet', '-allow_master', + shard_master.tablet_alias], auto_log=True) + + # rebuild the serving graph, all mentions of the old shards should be gone + utils.run_vtctl(['RebuildKeyspaceGraph', keyspace], auto_log=True) + + # delete the original shard + utils.run_vtctl(['DeleteShard', keyspace + '/0'], auto_log=True) + + # kill everything else + tablet.kill_tablets([shard_0_master, shard_0_replica, shard_0_rdonly, + shard_1_master, shard_1_replica, shard_1_rdonly]) + + def test_resharding(self): + self._test_resharding('test_keyspace1', ks1_tablets) + self._test_resharding('test_keyspace2', ks2_tablets, True) + self.kill_all_tablets('test_keyspace1', ks1_tablets) + self.kill_all_tablets('test_keyspace2', ks2_tablets) + +if __name__ == '__main__': + utils.main() diff --git a/test/mysql_flavor.py b/test/mysql_flavor.py index 56fb9495055..bc6d484395c 100644 --- a/test/mysql_flavor.py +++ b/test/mysql_flavor.py @@ -137,7 +137,7 @@ def extra_my_cnf(self): return environment.vttop + "/config/mycnf/master_mariadb103.cnf" class MySQL56(MysqlFlavor): - """Overrides specific to MySQL 5.6.""" + """Overrides specific to MySQL 5.6/5.7""" def master_position(self, tablet): gtid = tablet.mquery("", "SELECT @@GLOBAL.gtid_executed")[0][0] @@ -165,6 +165,11 @@ def change_master_commands(self, host, port, pos): "MASTER_USER='vt_repl', MASTER_AUTO_POSITION = 1" % (host, port)] +class MySQL80(MySQL56): + """Overrides specific to MySQL 8.0.""" + def extra_my_cnf(self): + return environment.vttop + "/config/mycnf/master_mysql80.cnf" + # Map of registered MysqlFlavor classes (keyed by an identifier). flavor_map = {} @@ -238,3 +243,4 @@ def register_flavor(flavor, cls, env): register_flavor("MariaDB", MariaDB, "MariaDB") register_flavor("MariaDB103", MariaDB103, "MariaDB103") register_flavor("MySQL56", MySQL56, "MySQL56") +register_flavor("MySQL80", MySQL80, "MySQL80") diff --git a/test/resharding.py b/test/resharding.py index e0d68c5517c..4bc50021cff 100755 --- a/test/resharding.py +++ b/test/resharding.py @@ -1083,6 +1083,8 @@ def test_resharding(self): # update our test variables to point at the new master shard_2_master, shard_2_replica1 = shard_2_replica1, shard_2_master + utils.pause('check state of _vt.vreplication') + logging.debug('Inserting lots of data on source shard after reparenting') self._insert_lots(3000, base=2000) logging.debug('Checking 80 percent of data was sent fairly quickly') diff --git a/test/tablet.py b/test/tablet.py index 1d64bfbe752..06e80492469 100644 --- a/test/tablet.py +++ b/test/tablet.py @@ -98,7 +98,8 @@ def __init__(self, tablet_uid=None, port=None, mysql_port=None, cell=None, self.shard = None self.index = None self.tablet_index = None - + # default to false + self.external_mysql = False # utility variables self.tablet_alias = 'test_%s-%010d' % (self.cell, self.tablet_uid) self.zk_tablet_path = ( @@ -229,7 +230,6 @@ def remove_tree(self, ignore_options=False): def mysql_connection_parameters(self, dbname, user='vt_dba'): result = dict(user=user, - unix_socket=self.tablet_dir + '/mysql.sock', db=dbname) if user == 'vt_dba' and self.vt_dba_passwd: result['passwd'] = self.vt_dba_passwd @@ -237,6 +237,10 @@ def mysql_connection_parameters(self, dbname, user='vt_dba'): def connect(self, dbname='', user='vt_dba', **params): params.update(self.mysql_connection_parameters(dbname, user)) + if 'port' not in params.keys(): + params['unix_socket']=self.tablet_dir + '/mysql.sock' + else: + params['host']='127.0.0.1' conn = MySQLdb.Connect(**params) return conn, conn.cursor() @@ -257,6 +261,8 @@ def mquery(self, dbname, query, write=False, user='vt_dba', conn_params=None, """ if conn_params is None: conn_params = {} + if self.external_mysql: + conn_params['port']=self.mysql_port conn, cursor = self.connect(dbname, user=user, **conn_params) if write: conn.begin() @@ -381,13 +387,14 @@ def update_addrs(self): def init_tablet(self, tablet_type, keyspace, shard, tablet_index=None, start=False, dbname=None, parent=True, wait_for_start=True, - include_mysql_port=True, **kwargs): + include_mysql_port=True, external_mysql=False, **kwargs): """Initialize a tablet's record in topology.""" self.tablet_type = tablet_type self.keyspace = keyspace self.shard = shard self.tablet_index = tablet_index + self.external_mysql = external_mysql self.dbname = dbname or ('vt_' + self.keyspace) @@ -487,6 +494,10 @@ def start_vttablet( args.extend( ['-mysqlctl_socket', os.path.join(self.tablet_dir, 'mysqlctl.sock')]) + if self.external_mysql: + args.extend(['-db_host', '127.0.0.1']) + args.extend(['-db_port', str(self.mysql_port)]) + args.append('-disable_active_reparents') if full_mycnf_args: # this flag is used to specify all the mycnf_ flags, to make # sure that code works. diff --git a/test/xtrabackup.py b/test/xtrabackup.py new file mode 100755 index 00000000000..135bdf60ce8 --- /dev/null +++ b/test/xtrabackup.py @@ -0,0 +1,25 @@ +#!/usr/bin/env python + +# 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. + + +"""Re-runs backup.py with use_xtrabackup=True.""" + +import backup +import utils + +if __name__ == '__main__': + backup.use_xtrabackup = True + utils.main(backup) diff --git a/test/xtrabackup_xbstream.py b/test/xtrabackup_xbstream.py new file mode 100755 index 00000000000..751d2f6af62 --- /dev/null +++ b/test/xtrabackup_xbstream.py @@ -0,0 +1,26 @@ +#!/usr/bin/env python + +# 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. + + +"""Re-runs backup.py with use_xtrabackup=True.""" + +import backup +import utils + +if __name__ == '__main__': + backup.use_xtrabackup = True + backup.stream_mode = 'xbstream' + utils.main(backup) diff --git a/test/xtrabackup_xtra.py b/test/xtrabackup_xtra.py new file mode 100755 index 00000000000..f33dce89c60 --- /dev/null +++ b/test/xtrabackup_xtra.py @@ -0,0 +1,299 @@ +#!/usr/bin/env python + +# 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. + + +import json +import logging +import os +import unittest + +import MySQLdb + +import environment +import tablet +import utils + +use_mysqlctld = False +use_xtrabackup = True +stream_mode = 'xbstream' +xtrabackup_args = [] +tablet_master = None +tablet_replica1 = None +tablet_replica2 = None + +new_init_db = '' +db_credentials_file = '' + + +def setUpModule(): + xtrabackup_args = ['-backup_engine_implementation', + 'xtrabackup', + '-xtrabackup_stream_mode', + stream_mode, + '-xtrabackup_user=vt_dba', + '-xtrabackup_backup_flags', + '--password=VtDbaPass'] + global new_init_db, db_credentials_file + global tablet_master, tablet_replica1, tablet_replica2 + + tablet_master = tablet.Tablet(use_mysqlctld=use_mysqlctld, + vt_dba_passwd='VtDbaPass') + tablet_replica1 = tablet.Tablet(use_mysqlctld=use_mysqlctld, + vt_dba_passwd='VtDbaPass') + tablet_replica2 = tablet.Tablet(use_mysqlctld=use_mysqlctld, + vt_dba_passwd='VtDbaPass') + + try: + environment.topo_server().setup() + + # Create a new init_db.sql file that sets up passwords for all users. + # Then we use a db-credentials-file with the passwords. + new_init_db = environment.tmproot + '/init_db_with_passwords.sql' + with open(environment.vttop + '/config/init_db.sql') as fd: + init_db = fd.read() + with open(new_init_db, 'w') as fd: + fd.write(init_db) + fd.write(''' +# Set real passwords for all users. +ALTER USER 'root'@'localhost' IDENTIFIED BY 'RootPass'; +ALTER USER 'vt_dba'@'localhost' IDENTIFIED BY 'VtDbaPass'; +ALTER USER 'vt_app'@'localhost' IDENTIFIED BY 'VtAppPass'; +ALTER USER 'vt_allprivs'@'localhost' IDENTIFIED BY 'VtAllPrivsPass'; +ALTER USER 'vt_repl'@'%' IDENTIFIED BY 'VtReplPass'; +ALTER USER 'vt_filtered'@'localhost' IDENTIFIED BY 'VtFilteredPass'; +FLUSH PRIVILEGES; +''') + credentials = { + 'vt_dba': ['VtDbaPass'], + 'vt_app': ['VtAppPass'], + 'vt_allprivs': ['VtAllprivsPass'], + 'vt_repl': ['VtReplPass'], + 'vt_filtered': ['VtFilteredPass'], + } + db_credentials_file = environment.tmproot+'/db_credentials.json' + with open(db_credentials_file, 'w') as fd: + fd.write(json.dumps(credentials)) + + # start mysql instance external to the test + setup_procs = [ + tablet_master.init_mysql(init_db=new_init_db, + extra_args=['-db-credentials-file', + db_credentials_file]), + tablet_replica1.init_mysql(init_db=new_init_db, + extra_args=['-db-credentials-file', + db_credentials_file]), + tablet_replica2.init_mysql(init_db=new_init_db, + extra_args=['-db-credentials-file', + db_credentials_file]), + ] + if use_mysqlctld: + tablet_master.wait_for_mysqlctl_socket() + tablet_replica1.wait_for_mysqlctl_socket() + tablet_replica2.wait_for_mysqlctl_socket() + else: + utils.wait_procs(setup_procs) + except: + tearDownModule() + raise + + +def tearDownModule(): + utils.required_teardown() + if utils.options.skip_teardown: + return + + teardown_procs = [ + tablet_master.teardown_mysql(extra_args=['-db-credentials-file', + db_credentials_file]), + tablet_replica1.teardown_mysql(extra_args=['-db-credentials-file', + db_credentials_file]), + tablet_replica2.teardown_mysql(extra_args=['-db-credentials-file', + db_credentials_file]), + ] + utils.wait_procs(teardown_procs, raise_on_error=False) + + environment.topo_server().teardown() + utils.kill_sub_processes() + utils.remove_tmp_files() + + tablet_master.remove_tree() + tablet_replica1.remove_tree() + tablet_replica2.remove_tree() + + +class TestXtraBackup(unittest.TestCase): + + def setUp(self): + for t in tablet_master, tablet_replica1: + t.create_db('vt_test_keyspace') + + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) + tablet_master.init_tablet('replica', 'test_keyspace', '0', start=True, + supports_backups=True, + extra_args=xtra_args) + tablet_replica1.init_tablet('replica', 'test_keyspace', '0', start=True, + supports_backups=True, + extra_args=xtra_args) + utils.run_vtctl(['InitShardMaster', '-force', 'test_keyspace/0', + tablet_master.tablet_alias]) + + def tearDown(self): + for t in tablet_master, tablet_replica1, tablet_replica2: + t.kill_vttablet() + + tablet.Tablet.check_vttablet_count() + environment.topo_server().wipe() + for t in [tablet_master, tablet_replica1, tablet_replica2]: + t.reset_replication() + t.set_semi_sync_enabled(master=False, slave=False) + t.clean_dbs() + + for backup in self._list_backups(): + self._remove_backup(backup) + + _create_vt_insert_test = '''create table vt_insert_test ( + id bigint auto_increment, + msg varchar(64), + primary key (id) + ) Engine=InnoDB''' + + def _insert_data(self, t, index): + """Add a single row with value 'index' to the given tablet.""" + t.mquery( + 'vt_test_keyspace', + "insert into vt_insert_test (msg) values ('test %s')" % + index, write=True) + + def _check_data(self, t, count, msg): + """Check that the specified tablet has the expected number of rows.""" + timeout = 10 + while True: + try: + result = t.mquery( + 'vt_test_keyspace', 'select count(*) from vt_insert_test') + if result[0][0] == count: + break + except MySQLdb.DatabaseError: + # ignore exceptions, we'll just timeout (the tablet creation + # can take some time to replicate, and we get a 'table vt_insert_test + # does not exist exception in some rare cases) + logging.exception('exception waiting for data to replicate') + timeout = utils.wait_step(msg, timeout) + + def _restore(self, t, tablet_type='replica'): + """Erase mysql/tablet dir, then start tablet with restore enabled.""" + self._reset_tablet_dir(t) + + xtra_args = ['-db-credentials-file', db_credentials_file] + if use_xtrabackup: + xtra_args.extend(xtrabackup_args) + + t.start_vttablet(wait_for_state='SERVING', + init_tablet_type=tablet_type, + init_keyspace='test_keyspace', + init_shard='0', + supports_backups=True, + extra_args=xtra_args) + + # check semi-sync is enabled for replica, disabled for rdonly. + if tablet_type == 'replica': + t.check_db_var('rpl_semi_sync_slave_enabled', 'ON') + t.check_db_status('rpl_semi_sync_slave_status', 'ON') + else: + t.check_db_var('rpl_semi_sync_slave_enabled', 'OFF') + t.check_db_status('rpl_semi_sync_slave_status', 'OFF') + + def _reset_tablet_dir(self, t): + """Stop mysql, delete everything including tablet dir, restart mysql.""" + extra_args = ['-db-credentials-file', db_credentials_file] + utils.wait_procs([t.teardown_mysql(extra_args=extra_args)]) + # Specify ignore_options because we want to delete the tree even + # if the test's -k / --keep-logs was specified on the command line. + t.remove_tree(ignore_options=True) + proc = t.init_mysql(init_db=new_init_db, extra_args=extra_args) + if use_mysqlctld: + t.wait_for_mysqlctl_socket() + else: + utils.wait_procs([proc]) + + def _list_backups(self): + """Get a list of backup names for the test shard.""" + backups, _ = utils.run_vtctl(tablet.get_backup_storage_flags() + + ['ListBackups', 'test_keyspace/0'], + mode=utils.VTCTL_VTCTL, trap_output=True) + return backups.splitlines() + + def _remove_backup(self, backup): + """Remove a named backup from the test shard.""" + utils.run_vtctl( + tablet.get_backup_storage_flags() + + ['RemoveBackup', 'test_keyspace/0', backup], + auto_log=True, mode=utils.VTCTL_VTCTL) + + def _restore_old_master_test(self, restore_method): + """Test that a former master replicates correctly after being restored. + + - Take a backup. + - Reparent from old master to new master. + - Force old master to restore from a previous backup using restore_method. + + Args: + restore_method: function accepting one parameter of type tablet.Tablet, + this function is called to force a restore on the provided tablet + """ + + # insert data on master, wait for slave to get it + tablet_master.mquery('vt_test_keyspace', self._create_vt_insert_test) + self._insert_data(tablet_master, 1) + self._check_data(tablet_replica1, 1, 'replica1 tablet getting data') + + # backup the slave + utils.run_vtctl(['Backup', tablet_replica1.tablet_alias], auto_log=True) + + # insert more data on the master + self._insert_data(tablet_master, 2) + + # reparent to replica1 + utils.run_vtctl(['PlannedReparentShard', + '-keyspace_shard', 'test_keyspace/0', + '-new_master', tablet_replica1.tablet_alias]) + + # insert more data on new master + self._insert_data(tablet_replica1, 3) + + # force the old master to restore at the latest backup. + restore_method(tablet_master) + + # wait for it to catch up. + self._check_data(tablet_master, 3, 'former master catches up after restore') + + def test_terminated_restore(self): + def _terminated_restore(t): + for e in utils.vtctld_connection.execute_vtctl_command( + ['RestoreFromBackup', t.tablet_alias]): + logging.info('%s', e.value) + if 'shutdown mysqld' in e.value: + break + logging.info('waiting for restore to finish') + utils.wait_for_tablet_type(t.tablet_alias, 'replica', timeout=30) + + utils.Vtctld().start() + self._restore_old_master_test(_terminated_restore) + +if __name__ == '__main__': + utils.main() diff --git a/vagrant-scripts/bootstrap_vm.sh b/vagrant-scripts/bootstrap_vm.sh index 5515e80dc16..bc1d0150207 100755 --- a/vagrant-scripts/bootstrap_vm.sh +++ b/vagrant-scripts/bootstrap_vm.sh @@ -30,7 +30,8 @@ apt-get install -y make \ pkg-config \ bison \ curl \ - openjdk-7-jre \ + openjdk-7-jdk \ + ant \ zip \ unzip diff --git a/vagrant-scripts/vitess/build.sh b/vagrant-scripts/vitess/build.sh index 893b1061397..100b2f202c3 100755 --- a/vagrant-scripts/vitess/build.sh +++ b/vagrant-scripts/vitess/build.sh @@ -14,6 +14,10 @@ printf "\nBuilding Vitess...\n" sudo chown "$(whoami)":"$(whoami)" /vagrant sudo chown "$(whoami)":"$(whoami)" /vagrant/src cd "$VITESS_WORKSPACE" + +# open-jdk version that we are using in the VM needs this flag, otherwise we will fail to build ZK +export JAVA_TOOL_OPTIONS="-Dhttps.protocols=TLSv1.2" + ./bootstrap.sh # shellcheck disable=SC1091 source dev.env From 1d7a89c90cf8b2107c1b75792ca7fefb9565b533 Mon Sep 17 00:00:00 2001 From: Brian Ramos Date: Thu, 23 May 2019 13:38:10 -0700 Subject: [PATCH 66/72] [dbconfigs] revert vitessio/vitess#4814 as it breaks tablet utf8mb4 settings (#131) --- go/vt/dbconfigs/dbconfigs.go | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/go/vt/dbconfigs/dbconfigs.go b/go/vt/dbconfigs/dbconfigs.go index 24764d37bbb..c8a86106b16 100644 --- a/go/vt/dbconfigs/dbconfigs.go +++ b/go/vt/dbconfigs/dbconfigs.go @@ -237,23 +237,27 @@ func HasConnectionParams() bool { // is used to initialize the per-user conn params. func Init(defaultSocketFile string) (*DBConfigs, error) { // The new base configs, if set, supersede legacy settings. - for _, uc := range dbConfigs.userConfigs { - if HasConnectionParams() { + if HasConnectionParams() { + for _, uc := range dbConfigs.userConfigs { uc.param.Host = baseConfig.Host uc.param.Port = baseConfig.Port uc.param.UnixSocket = baseConfig.UnixSocket - } else if uc.param.UnixSocket == "" && uc.param.Host == "" { - uc.param.UnixSocket = defaultSocketFile + uc.param.Charset = baseConfig.Charset + uc.param.Flags = baseConfig.Flags + if uc.useSSL { + uc.param.SslCa = baseConfig.SslCa + uc.param.SslCaPath = baseConfig.SslCaPath + uc.param.SslCert = baseConfig.SslCert + uc.param.SslKey = baseConfig.SslKey + uc.param.ServerName = baseConfig.ServerName + } } - - uc.param.Charset = baseConfig.Charset - uc.param.Flags = baseConfig.Flags - if uc.useSSL { - uc.param.SslCa = baseConfig.SslCa - uc.param.SslCaPath = baseConfig.SslCaPath - uc.param.SslCert = baseConfig.SslCert - uc.param.SslKey = baseConfig.SslKey - uc.param.ServerName = baseConfig.ServerName + } else { + // Use supplied socket value if conn parameters are not specified. + for _, uc := range dbConfigs.userConfigs { + if uc.param.UnixSocket == "" && uc.param.Host == "" { + uc.param.UnixSocket = defaultSocketFile + } } } From 64ce96f5d533664ec77157f917a0be8dac15140b Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 22 May 2018 20:53:18 -0700 Subject: [PATCH 67/72] add skeleton plugins for writing query logs to murron Signed-off-by: Michael Demmer --- go/cmd/vtgate/plugin_murron_logger.go | 59 +++++++++++++++++++++++++ go/cmd/vttablet/plugin_murron_logger.go | 59 +++++++++++++++++++++++++ 2 files changed, 118 insertions(+) create mode 100644 go/cmd/vtgate/plugin_murron_logger.go create mode 100644 go/cmd/vttablet/plugin_murron_logger.go diff --git a/go/cmd/vtgate/plugin_murron_logger.go b/go/cmd/vtgate/plugin_murron_logger.go new file mode 100644 index 00000000000..573a9d0e9e4 --- /dev/null +++ b/go/cmd/vtgate/plugin_murron_logger.go @@ -0,0 +1,59 @@ +/* +Copyright 2018 Slack Inc. + +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 main + +import ( + "flag" + "fmt" + + "vitess.io/vitess/go/streamlog" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/vtgate" +) + +var ( + murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") +) + +func init() { + servenv.OnRun(func() { + if *murronServerName != "" { + initMurron(*murronServerName) + } + }) +} + +func initMurron(server string) { + log.Infof("enabling query logging to murron server %s", server) + + // TODO: initialize murron connection here + + logChan := vtgate.QueryLogger.Subscribe("Murron") + formatParams := map[string][]string{"full": {}} + formatter := streamlog.GetFormatter(vtgate.QueryLogger) + + go func() { + for { + record := <-logChan + message := formatter(formatParams, record) + + // TODO: actually send logs to murron here + fmt.Printf("SENDING TO MURRON: %s\n", message) + } + }() +} diff --git a/go/cmd/vttablet/plugin_murron_logger.go b/go/cmd/vttablet/plugin_murron_logger.go new file mode 100644 index 00000000000..4f68ae5b8e5 --- /dev/null +++ b/go/cmd/vttablet/plugin_murron_logger.go @@ -0,0 +1,59 @@ +/* +Copyright 2018 Slack Inc. + +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 main + +import ( + "flag" + "fmt" + + "vitess.io/vitess/go/streamlog" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" +) + +var ( + murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") +) + +func init() { + servenv.OnRun(func() { + if *murronServerName != "" { + initMurron(*murronServerName) + } + }) +} + +func initMurron(server string) { + log.Infof("enabling query logging to murron server %s", server) + + // TODO: initialize murron connection here + + logChan := tabletenv.StatsLogger.Subscribe("Murron") + formatParams := map[string][]string{"full": {}} + formatter := streamlog.GetFormatter(tabletenv.StatsLogger) + + go func() { + for { + record := <-logChan + message := formatter(formatParams, record) + + // TODO: actually send logs to murron here + fmt.Printf("SENDING TO MURRON: %s\n", message) + } + }() +} From 7864c0e4665954107702e71aa8f0cdb357850428 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Thu, 24 May 2018 07:06:44 -0700 Subject: [PATCH 68/72] consolidate the flags and add stubs for an actual murron logger Signed-off-by: Michael Demmer --- go/cmd/vtgate/plugin_murron_logger.go | 24 +++-------- go/cmd/vttablet/plugin_murron_logger.go | 24 +++-------- go/slack/murronlogger.go | 56 +++++++++++++++++++++++++ 3 files changed, 68 insertions(+), 36 deletions(-) create mode 100644 go/slack/murronlogger.go diff --git a/go/cmd/vtgate/plugin_murron_logger.go b/go/cmd/vtgate/plugin_murron_logger.go index 573a9d0e9e4..3a991e6925d 100644 --- a/go/cmd/vtgate/plugin_murron_logger.go +++ b/go/cmd/vtgate/plugin_murron_logger.go @@ -17,32 +17,22 @@ limitations under the License. package main import ( - "flag" - "fmt" - + "vitess.io/vitess/go/slack" "vitess.io/vitess/go/streamlog" - "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/vtgate" ) -var ( - murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") -) - func init() { servenv.OnRun(func() { - if *murronServerName != "" { - initMurron(*murronServerName) + if slack.EnableMurronLogging() { + initMurronLogger() } }) } -func initMurron(server string) { - log.Infof("enabling query logging to murron server %s", server) - - // TODO: initialize murron connection here - +func initMurronLogger() { + logger := slack.InitMurronLogger() logChan := vtgate.QueryLogger.Subscribe("Murron") formatParams := map[string][]string{"full": {}} formatter := streamlog.GetFormatter(vtgate.QueryLogger) @@ -51,9 +41,7 @@ func initMurron(server string) { for { record := <-logChan message := formatter(formatParams, record) - - // TODO: actually send logs to murron here - fmt.Printf("SENDING TO MURRON: %s\n", message) + logger.SendMessage(message) } }() } diff --git a/go/cmd/vttablet/plugin_murron_logger.go b/go/cmd/vttablet/plugin_murron_logger.go index 4f68ae5b8e5..557e6f51624 100644 --- a/go/cmd/vttablet/plugin_murron_logger.go +++ b/go/cmd/vttablet/plugin_murron_logger.go @@ -17,32 +17,22 @@ limitations under the License. package main import ( - "flag" - "fmt" - + "vitess.io/vitess/go/slack" "vitess.io/vitess/go/streamlog" - "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" ) -var ( - murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") -) - func init() { servenv.OnRun(func() { - if *murronServerName != "" { - initMurron(*murronServerName) + if slack.EnableMurronLogging() { + initMurronLogger() } }) } -func initMurron(server string) { - log.Infof("enabling query logging to murron server %s", server) - - // TODO: initialize murron connection here - +func initMurronLogger() { + logger := slack.InitMurronLogger() logChan := tabletenv.StatsLogger.Subscribe("Murron") formatParams := map[string][]string{"full": {}} formatter := streamlog.GetFormatter(tabletenv.StatsLogger) @@ -51,9 +41,7 @@ func initMurron(server string) { for { record := <-logChan message := formatter(formatParams, record) - - // TODO: actually send logs to murron here - fmt.Printf("SENDING TO MURRON: %s\n", message) + logger.SendMessage(message) } }() } diff --git a/go/slack/murronlogger.go b/go/slack/murronlogger.go new file mode 100644 index 00000000000..d362c121c8f --- /dev/null +++ b/go/slack/murronlogger.go @@ -0,0 +1,56 @@ +/* +Copyright 2018 Slack Inc. + +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 slack + +import ( + "flag" + "fmt" + + "vitess.io/vitess/go/vt/log" +) + +var ( + // TODO: add actual murron config here + murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") +) + +// EnableMurronLogging returns true if murron logging should be enabled +func EnableMurronLogging() bool { + return *murronServerName != "" +} + +// MurronLogger is a logger abstraction to send to murron +type MurronLogger struct { + server string +} + +// InitMurronLogger creates a new logger to send to murron +func InitMurronLogger() *MurronLogger { + log.Infof("enabling query logging to murron server %s", *murronServerName) + + // TODO: actually initialize murron connection here + return &MurronLogger{ + server: *murronServerName, + } +} + +// SendMessage sends the given message to murron +func (*MurronLogger) SendMessage(message string) { + // TODO: actually send logs to murron here + fmt.Printf("SENDING TO MURRON: %s\n", message) + +} From 4cee5c611334b6fe5b7bd716854906e389afa7cb Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 28 Aug 2018 10:05:37 -0700 Subject: [PATCH 69/72] add murron vendor dependencies Signed-off-by: Michael Demmer --- vendor/vendor.json | 114 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/vendor/vendor.json b/vendor/vendor.json index 7c35cd37d1c..d35287d873a 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -32,6 +32,12 @@ "revision": "686f0e89858ea78eae54d4b2021e6bfc7d3a30ca", "revisionTime": "2016-12-16T21:27:53Z" }, + { + "checksumSHA1": "gzdeZh1Pnc0wnEu1em+MhOT+kvM=", + "path": "github.com/Shopify/sarama", + "revision": "647feef69a1a7e0e23cccceeb786176bc4fcf5ea", + "revisionTime": "2018-08-20T17:20:58Z" + }, { "checksumSHA1": "t5pzf8AGtuCmECrPlJM9oAky+dk=", "path": "github.com/aws/aws-sdk-go/aws", @@ -364,6 +370,24 @@ "revision": "5215b55f46b2b919f50a1df0eaa5886afe4e3b3d", "revisionTime": "2015-11-05T21:09:06Z" }, + { + "checksumSHA1": "y2Kh4iPlgCPXSGTCcFpzePYdzzg=", + "path": "github.com/eapache/go-resiliency/breaker", + "revision": "ea41b0fad31007accc7f806884dcdf3da98b79ce", + "revisionTime": "2018-03-26T13:24:23Z" + }, + { + "checksumSHA1": "w5itvm+eKlJJg3hGILnceM3sono=", + "path": "github.com/eapache/go-xerial-snappy", + "revision": "776d5712da21bc4762676d614db1d8a64f4238b0", + "revisionTime": "2018-08-14T17:44:37Z" + }, + { + "checksumSHA1": "SJhV59StVXLFvLU0l9s0d00nV10=", + "path": "github.com/eapache/queue", + "revision": "093482f3f8ce946c05bcba64badd2c82369e084d", + "revisionTime": "2018-02-27T14:14:24Z" + }, { "checksumSHA1": "muGVyM8mY3/gcap6kr4Ib3F5Xn4=", "path": "github.com/ghodss/yaml", @@ -376,6 +400,12 @@ "revision": "72ba3e6b9e6b87e0c74c9a7a4dc86e8dd8ba4355", "revisionTime": "2016-06-01T19:11:21Z" }, + { + "checksumSHA1": "2nEUYzHhmnw2o4tFPGDb3/Giz44=", + "path": "github.com/gogo/protobuf/proto", + "revision": "98f6aa8b3bcf03e45b1a8b48257c754766d5c3c8", + "revisionTime": "2018-08-21T10:22:07Z" + }, { "checksumSHA1": "HmbftipkadrLlCfzzVQ+iFHbl6g=", "path": "github.com/golang/glog", @@ -574,6 +604,24 @@ "revision": "f849b5445de4819127e123ca96ba0eeb62b5e479", "revisionTime": "2019-01-18T09:38:23Z" }, + { + "checksumSHA1": "Rf3QgJeAX2809t/DZvMjZbGHe9U=", + "path": "github.com/grpc-ecosystem/go-grpc-middleware/tags", + "revision": "15ea7401d63d2d8c2f17472d22359564615a9f7a", + "revisionTime": "2018-08-24T10:49:23Z" + }, + { + "checksumSHA1": "40Idbv0LvYI4eMdoV3UCFH0zvjA=", + "path": "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing", + "revision": "15ea7401d63d2d8c2f17472d22359564615a9f7a", + "revisionTime": "2018-08-24T10:49:23Z" + }, + { + "checksumSHA1": "L5z1C445GhhQmDKSisTFv754LdU=", + "path": "github.com/grpc-ecosystem/go-grpc-middleware/util/metautils", + "revision": "15ea7401d63d2d8c2f17472d22359564615a9f7a", + "revisionTime": "2018-08-24T10:49:23Z" + }, { "checksumSHA1": "9dP53doJ/haDqTJyD0iuv8g0XFs=", "path": "github.com/grpc-ecosystem/go-grpc-prometheus", @@ -744,12 +792,42 @@ "revision": "cca8bbc0798408af109aaaa239cbd2634846b340", "revisionTime": "2016-01-15T11:10:02Z" }, + { + "checksumSHA1": "wIcN7tZiF441h08RHAm4NV8cYO4=", + "path": "github.com/opentracing/opentracing-go", + "revision": "bd9c3193394760d98b2fa6ebb2291f0cd1d06a7d", + "revisionTime": "2018-06-06T20:41:48Z" + }, + { + "checksumSHA1": "uhDxBvLEqRAMZKgpTZ8MFuLIIM8=", + "path": "github.com/opentracing/opentracing-go/ext", + "revision": "bd9c3193394760d98b2fa6ebb2291f0cd1d06a7d", + "revisionTime": "2018-06-06T20:41:48Z" + }, + { + "checksumSHA1": "tnkdNJbJxNKuPZMWapP1xhKIIGw=", + "path": "github.com/opentracing/opentracing-go/log", + "revision": "bd9c3193394760d98b2fa6ebb2291f0cd1d06a7d", + "revisionTime": "2018-06-06T20:41:48Z" + }, { "checksumSHA1": "mhvIMH8oAtOiEyg37zWKmgb+6v4=", "path": "github.com/pborman/uuid", "revision": "b984ec7fa9ff9e428bd0cf0abf429384dfbe3e37", "revisionTime": "2016-08-24T21:06:00Z" }, + { + "checksumSHA1": "dyKMDNL6H34LGyuRvZhLEx0a/Z4=", + "path": "github.com/pierrec/lz4", + "revision": "dbe9298ce0992bc74bd23afef466c795d0d8afa0", + "revisionTime": "2018-08-26T16:56:52Z" + }, + { + "checksumSHA1": "YzBjaYp2pbrwPhT6XHY0CBSh71A=", + "path": "github.com/pierrec/lz4/internal/xxh32", + "revision": "dbe9298ce0992bc74bd23afef466c795d0d8afa0", + "revisionTime": "2018-08-26T16:56:52Z" + }, { "checksumSHA1": "LuFv4/jlrmFNnDb/5SCSEPAM9vU=", "path": "github.com/pmezard/go-difflib/difflib", @@ -798,6 +876,12 @@ "revision": "abf152e5f3e97f2fafac028d2cc06c1feb87ffa5", "revisionTime": "2016-04-11T19:08:41Z" }, + { + "checksumSHA1": "an5RM8wjgPPloolUUYkvEncbHu4=", + "path": "github.com/rcrowley/go-metrics", + "revision": "e2704e165165ec55d062f5919b4b29494e9fa790", + "revisionTime": "2018-05-03T17:46:38Z" + }, { "checksumSHA1": "dF3fORwN1HTgrlrdmll9K2cOjOg=", "path": "github.com/samuel/go-zookeeper/zk", @@ -1457,6 +1541,36 @@ "path": "gopkg.in/yaml.v2", "revision": "a5b47d31c556af34a302ce5d659e6fea44d90de0", "revisionTime": "2016-09-28T15:37:09Z" + }, + { + "checksumSHA1": "rE4z8/fpbG6pYFal6rdEz09Fw3I=", + "path": "slack-github.com/slack/murron/pkg/lib", + "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", + "revisionTime": "2018-08-22T21:54:10Z" + }, + { + "checksumSHA1": "DghGol8loNN4nq1p8j5d4bvNMSA=", + "path": "slack-github.com/slack/murron/pkg/log", + "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", + "revisionTime": "2018-08-22T21:54:10Z" + }, + { + "checksumSHA1": "yDr26+cKvGtatERB6wgzigbCueQ=", + "path": "slack-github.com/slack/murron/pkg/outputs", + "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", + "revisionTime": "2018-08-22T21:54:10Z" + }, + { + "checksumSHA1": "ITTEcZ9tX4I85CjJ589kMUQR/Bc=", + "path": "slack-github.com/slack/murron/pkg/routes", + "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", + "revisionTime": "2018-08-22T21:54:10Z" + }, + { + "checksumSHA1": "O22RLMWp6l6zujEHIq91SrdS5eg=", + "path": "slack-github.com/slack/murron/proto", + "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", + "revisionTime": "2018-08-22T21:54:10Z" } ], "rootPath": "vitess.io/vitess" From 546dc4ee5c6557a5ee79de7de7b44e46ce3f7abc Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 28 Aug 2018 10:05:51 -0700 Subject: [PATCH 70/72] fill in the implementation of an actual murron logger Signed-off-by: Michael Demmer --- go/cmd/vtgate/plugin_murron_logger.go | 16 +++-- go/cmd/vttablet/plugin_murron_logger.go | 16 +++-- go/slack/murronlogger.go | 85 ++++++++++++++++++++----- 3 files changed, 90 insertions(+), 27 deletions(-) diff --git a/go/cmd/vtgate/plugin_murron_logger.go b/go/cmd/vtgate/plugin_murron_logger.go index 3a991e6925d..0023d92733d 100644 --- a/go/cmd/vtgate/plugin_murron_logger.go +++ b/go/cmd/vtgate/plugin_murron_logger.go @@ -19,29 +19,33 @@ package main import ( "vitess.io/vitess/go/slack" "vitess.io/vitess/go/streamlog" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/vtgate" ) func init() { servenv.OnRun(func() { - if slack.EnableMurronLogging() { + if slack.MurronLoggerEnabled() { initMurronLogger() } }) } func initMurronLogger() { - logger := slack.InitMurronLogger() + logger, err := slack.InitMurronLogger("vtgate_querylog") + if err != nil { + log.Errorf("error initializing murron logger: %v", err) + return + } + logChan := vtgate.QueryLogger.Subscribe("Murron") - formatParams := map[string][]string{"full": {}} formatter := streamlog.GetFormatter(vtgate.QueryLogger) go func() { for { - record := <-logChan - message := formatter(formatParams, record) - logger.SendMessage(message) + log := <-logChan + logger.SendMessage(formatter, log) } }() } diff --git a/go/cmd/vttablet/plugin_murron_logger.go b/go/cmd/vttablet/plugin_murron_logger.go index 557e6f51624..b9cb0f48e10 100644 --- a/go/cmd/vttablet/plugin_murron_logger.go +++ b/go/cmd/vttablet/plugin_murron_logger.go @@ -19,29 +19,33 @@ package main import ( "vitess.io/vitess/go/slack" "vitess.io/vitess/go/streamlog" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" ) func init() { servenv.OnRun(func() { - if slack.EnableMurronLogging() { + if slack.MurronLoggerEnabled() { initMurronLogger() } }) } func initMurronLogger() { - logger := slack.InitMurronLogger() + logger, err := slack.InitMurronLogger("vtgate_querylog") + if err != nil { + log.Errorf("error initializing murron logger: %v", err) + return + } + logChan := tabletenv.StatsLogger.Subscribe("Murron") - formatParams := map[string][]string{"full": {}} formatter := streamlog.GetFormatter(tabletenv.StatsLogger) go func() { for { - record := <-logChan - message := formatter(formatParams, record) - logger.SendMessage(message) + log := <-logChan + logger.SendMessage(formatter, log) } }() } diff --git a/go/slack/murronlogger.go b/go/slack/murronlogger.go index d362c121c8f..bea6b737236 100644 --- a/go/slack/murronlogger.go +++ b/go/slack/murronlogger.go @@ -17,40 +17,95 @@ limitations under the License. package slack import ( + "bytes" "flag" - "fmt" + "os" + "time" + "vitess.io/vitess/go/stats" + "vitess.io/vitess/go/streamlog" "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/servenv" + + murronlib "slack-github.com/slack/murron/pkg/lib" + murronoutputs "slack-github.com/slack/murron/pkg/outputs" + murronpb "slack-github.com/slack/murron/proto" ) var ( - // TODO: add actual murron config here - murronServerName = flag.String("murron_server", "", "Enable query logging to the specified murron server") + murronConfigFile = flag.String("murron_config_file", "", "If specified, enables murron logging using the agent configuration in the given file.") + + murronLogs = stats.NewCounter("MurronLogs", "count of logs dispatched to the murron queue") + murronErrors = stats.NewCountersWithSingleLabel("MurronErrors", "count of errors sending to murron by error type", "Type") + + hostname, _ = os.Hostname() ) -// EnableMurronLogging returns true if murron logging should be enabled -func EnableMurronLogging() bool { - return *murronServerName != "" +// MurronLoggerEnabled returns true if murron logging should be enabled +func MurronLoggerEnabled() bool { + return *murronConfigFile != "" } // MurronLogger is a logger abstraction to send to murron type MurronLogger struct { - server string + logType string + queue chan *murronpb.MurronMessage + client murronoutputs.OutputService } // InitMurronLogger creates a new logger to send to murron -func InitMurronLogger() *MurronLogger { - log.Infof("enabling query logging to murron server %s", *murronServerName) +// Should only be called with a valid config file +func InitMurronLogger(logType string) (*MurronLogger, error) { + log.Infof("enabling %s query logging to murron", logType) - // TODO: actually initialize murron connection here - return &MurronLogger{ - server: *murronServerName, + /* Load murron configuration */ + config, err := murronlib.ReadConfig(*murronConfigFile) + if err != nil { + return nil, err } + + ml := &MurronLogger{ + logType: logType, + queue: make(chan *murronpb.MurronMessage, config.QueueSize), + } + + servenv.OnClose(func() { + log.Infof("closing murron client") + ml.client.Close() + log.Infof("closed murron client") + }) + + ml.client = murronoutputs.NewMurronBatchClient(config.ServerAddress, config.OutboundConnections, config.ClientBatchCount, config.ClientBatchSize, config.ClientBatchTimeout, ml.queue) + + return ml, nil } // SendMessage sends the given message to murron -func (*MurronLogger) SendMessage(message string) { - // TODO: actually send logs to murron here - fmt.Printf("SENDING TO MURRON: %s\n", message) +func (ml *MurronLogger) SendMessage(formatter streamlog.LogFormatter, message interface{}) { + + // TODO: this would be a great use case for a sync.Pool to avoid + // thrashing the GC with these allocations / deallocations + // but there's no indication from the murron client when it's done + // processing a given message. + + var buf bytes.Buffer + formatParams := map[string][]string{"full": {}} + if err := formatter(&buf, formatParams, message); err != nil { + murronErrors.Add("Format", 1) + return + } + msg := &murronpb.MurronMessage{ + OriginHost: hostname, + Timestamp: time.Now().UnixNano(), + Host: hostname, + Type: ml.logType, + Message: buf.Bytes(), + } + select { + case ml.queue <- msg: + murronLogs.Add(1) + default: + murronErrors.Add("QueueFull", 1) + } } From 24f61a371772b923450412494bee502091d0b1d2 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Tue, 4 Sep 2018 05:50:56 -0700 Subject: [PATCH 71/72] move murron log type configuration into a command line flag Signed-off-by: Michael Demmer --- go/cmd/vtgate/plugin_murron_logger.go | 2 +- go/cmd/vttablet/plugin_murron_logger.go | 2 +- go/slack/murronlogger.go | 23 ++++++++++++++--------- 3 files changed, 16 insertions(+), 11 deletions(-) diff --git a/go/cmd/vtgate/plugin_murron_logger.go b/go/cmd/vtgate/plugin_murron_logger.go index 0023d92733d..b8448c1f04b 100644 --- a/go/cmd/vtgate/plugin_murron_logger.go +++ b/go/cmd/vtgate/plugin_murron_logger.go @@ -33,7 +33,7 @@ func init() { } func initMurronLogger() { - logger, err := slack.InitMurronLogger("vtgate_querylog") + logger, err := slack.InitMurronLogger() if err != nil { log.Errorf("error initializing murron logger: %v", err) return diff --git a/go/cmd/vttablet/plugin_murron_logger.go b/go/cmd/vttablet/plugin_murron_logger.go index b9cb0f48e10..cc637ad1de2 100644 --- a/go/cmd/vttablet/plugin_murron_logger.go +++ b/go/cmd/vttablet/plugin_murron_logger.go @@ -33,7 +33,7 @@ func init() { } func initMurronLogger() { - logger, err := slack.InitMurronLogger("vtgate_querylog") + logger, err := slack.InitMurronLogger() if err != nil { log.Errorf("error initializing murron logger: %v", err) return diff --git a/go/slack/murronlogger.go b/go/slack/murronlogger.go index bea6b737236..7eac119e34f 100644 --- a/go/slack/murronlogger.go +++ b/go/slack/murronlogger.go @@ -19,6 +19,7 @@ package slack import ( "bytes" "flag" + "fmt" "os" "time" @@ -34,6 +35,7 @@ import ( var ( murronConfigFile = flag.String("murron_config_file", "", "If specified, enables murron logging using the agent configuration in the given file.") + murronLogType = flag.String("murron_querylog_type", "", "Type string for query logs sent to murron. Must be set if murron logging is enabled.") murronLogs = stats.NewCounter("MurronLogs", "count of logs dispatched to the murron queue") murronErrors = stats.NewCountersWithSingleLabel("MurronErrors", "count of errors sending to murron by error type", "Type") @@ -48,15 +50,19 @@ func MurronLoggerEnabled() bool { // MurronLogger is a logger abstraction to send to murron type MurronLogger struct { - logType string - queue chan *murronpb.MurronMessage - client murronoutputs.OutputService + queue chan *murronpb.MurronMessage + client murronoutputs.OutputService } // InitMurronLogger creates a new logger to send to murron -// Should only be called with a valid config file -func InitMurronLogger(logType string) (*MurronLogger, error) { - log.Infof("enabling %s query logging to murron", logType) +// Should only be called with a valid config file and if log type is set +func InitMurronLogger() (*MurronLogger, error) { + + if *murronConfigFile == "" || *murronLogType == "" { + return nil, fmt.Errorf("murron logging requires -murron_config_file and -murron_querylog_type to be set") + } + + log.Infof("enabling %s query logging to murron", *murronLogType) /* Load murron configuration */ config, err := murronlib.ReadConfig(*murronConfigFile) @@ -65,8 +71,7 @@ func InitMurronLogger(logType string) (*MurronLogger, error) { } ml := &MurronLogger{ - logType: logType, - queue: make(chan *murronpb.MurronMessage, config.QueueSize), + queue: make(chan *murronpb.MurronMessage, config.QueueSize), } servenv.OnClose(func() { @@ -99,7 +104,7 @@ func (ml *MurronLogger) SendMessage(formatter streamlog.LogFormatter, message in OriginHost: hostname, Timestamp: time.Now().UnixNano(), Host: hostname, - Type: ml.logType, + Type: *murronLogType, Message: buf.Bytes(), } select { From b1ee36436660289b9f273a345ec21c542fe37d85 Mon Sep 17 00:00:00 2001 From: Michael Demmer Date: Fri, 7 Jun 2019 14:07:22 -0700 Subject: [PATCH 72/72] update murron to the latest using vendor stuff Signed-off-by: Michael Demmer --- vendor/vendor.json | 334 ++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 299 insertions(+), 35 deletions(-) diff --git a/vendor/vendor.json b/vendor/vendor.json index d35287d873a..6e4a255cbf9 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -33,10 +33,16 @@ "revisionTime": "2016-12-16T21:27:53Z" }, { - "checksumSHA1": "gzdeZh1Pnc0wnEu1em+MhOT+kvM=", + "checksumSHA1": "eruIVA8JnsB23rVKjETHvqJ0sj8=", + "path": "github.com/DataDog/zstd", + "revision": "2bf71ec4836011b92dc78df3b9ace6b40e65f7df", + "revisionTime": "2016-07-06T22:07:25Z" + }, + { + "checksumSHA1": "pWpvwUwcVfUjl5PgMr1s2RDmoK4=", "path": "github.com/Shopify/sarama", - "revision": "647feef69a1a7e0e23cccceeb786176bc4fcf5ea", - "revisionTime": "2018-08-20T17:20:58Z" + "revision": "cd910a683f9faa57222e5120d17b60d2e65f7fa9", + "revisionTime": "2019-06-04T11:44:37Z" }, { "checksumSHA1": "t5pzf8AGtuCmECrPlJM9oAky+dk=", @@ -359,10 +365,10 @@ "versionExact": "v2.0.0" }, { - "checksumSHA1": "5rPfda8jFccr3A6heL+JAmi9K9g=", + "checksumSHA1": "CSPbwbyzqA6sfORicn4HFtIhF/c=", "path": "github.com/davecgh/go-spew/spew", - "revision": "5215b55f46b2b919f50a1df0eaa5886afe4e3b3d", - "revisionTime": "2015-11-05T21:09:06Z" + "revision": "d8f796af33cc11cb798c1aaeb27a4ebc5099927d", + "revisionTime": "2018-08-30T19:11:22Z" }, { "checksumSHA1": "a2yC46a1qsJomgY6rb+FkTFiqmE=", @@ -373,8 +379,8 @@ { "checksumSHA1": "y2Kh4iPlgCPXSGTCcFpzePYdzzg=", "path": "github.com/eapache/go-resiliency/breaker", - "revision": "ea41b0fad31007accc7f806884dcdf3da98b79ce", - "revisionTime": "2018-03-26T13:24:23Z" + "revision": "842e16ec2c98ef0c59eebfe60d2d3500a793ba19", + "revisionTime": "2019-02-08T16:25:53Z" }, { "checksumSHA1": "w5itvm+eKlJJg3hGILnceM3sono=", @@ -401,10 +407,10 @@ "revisionTime": "2016-06-01T19:11:21Z" }, { - "checksumSHA1": "2nEUYzHhmnw2o4tFPGDb3/Giz44=", + "checksumSHA1": "jyCYGbN0KvpV6PasLXLOmNsaA3s=", "path": "github.com/gogo/protobuf/proto", - "revision": "98f6aa8b3bcf03e45b1a8b48257c754766d5c3c8", - "revisionTime": "2018-08-21T10:22:07Z" + "revision": "21df5aa0e680850681b8643f0024f92d3b09930c", + "revisionTime": "2019-06-01T10:31:08Z" }, { "checksumSHA1": "HmbftipkadrLlCfzzVQ+iFHbl6g=", @@ -666,12 +672,30 @@ "revision": "6bb64b370b90e7ef1fa532be9e591a81c3493e00", "revisionTime": "2016-05-03T14:34:40Z" }, + { + "checksumSHA1": "5AxXPtBqAKyFGcttFzxT5hp/3Tk=", + "path": "github.com/hashicorp/go-uuid", + "revision": "4f571afc59f3043a65f8fe6bf46d887b10a01d43", + "revisionTime": "2018-11-28T13:14:45Z" + }, { "checksumSHA1": "E3Xcanc9ouQwL+CZGOUyA/+giLg=", "path": "github.com/hashicorp/serf/coordinate", "revision": "d3a67ab21bc8a4643fa53a3633f2d951dd50c6ca", "revisionTime": "2016-12-07T01:17:43Z" }, + { + "checksumSHA1": "fPE6hs5I61ZEXc54kkSoFaafqOk=", + "path": "github.com/jcmturner/gofork/encoding/asn1", + "revision": "dc7c13fece037a4a36e2b3c69db4991498d30692", + "revisionTime": "2019-03-28T16:16:33Z" + }, + { + "checksumSHA1": "jdBMz1QxC+2C2oeI8clgMKuWHt4=", + "path": "github.com/jcmturner/gofork/x/crypto/pbkdf2", + "revision": "dc7c13fece037a4a36e2b3c69db4991498d30692", + "revisionTime": "2019-03-28T16:16:33Z" + }, { "checksumSHA1": "0ZrwvB6KoGPj2PoDNSEJwxQ6Mog=", "path": "github.com/jmespath/go-jmespath", @@ -817,16 +841,16 @@ "revisionTime": "2016-08-24T21:06:00Z" }, { - "checksumSHA1": "dyKMDNL6H34LGyuRvZhLEx0a/Z4=", + "checksumSHA1": "ShSR6wnfWp4AaYx4C68hr8IpUaU=", "path": "github.com/pierrec/lz4", - "revision": "dbe9298ce0992bc74bd23afef466c795d0d8afa0", - "revisionTime": "2018-08-26T16:56:52Z" + "revision": "dbab9a2a023889b984e97d23bae9a3196d6f4865", + "revisionTime": "2019-06-06T19:12:28Z" }, { - "checksumSHA1": "YzBjaYp2pbrwPhT6XHY0CBSh71A=", + "checksumSHA1": "XxQwMkOgl4p0iuaVClAr0KIDtLg=", "path": "github.com/pierrec/lz4/internal/xxh32", - "revision": "dbe9298ce0992bc74bd23afef466c795d0d8afa0", - "revisionTime": "2018-08-26T16:56:52Z" + "revision": "dbab9a2a023889b984e97d23bae9a3196d6f4865", + "revisionTime": "2019-06-06T19:12:28Z" }, { "checksumSHA1": "LuFv4/jlrmFNnDb/5SCSEPAM9vU=", @@ -877,10 +901,10 @@ "revisionTime": "2016-04-11T19:08:41Z" }, { - "checksumSHA1": "an5RM8wjgPPloolUUYkvEncbHu4=", + "checksumSHA1": "LmajbO3+qtbE7JA0MQ29PXbmKNM=", "path": "github.com/rcrowley/go-metrics", - "revision": "e2704e165165ec55d062f5919b4b29494e9fa790", - "revisionTime": "2018-05-03T17:46:38Z" + "revision": "3113b8401b8a98917cde58f8bbd42a1b1c03b1fd", + "revisionTime": "2018-10-16T18:43:25Z" }, { "checksumSHA1": "dF3fORwN1HTgrlrdmll9K2cOjOg=", @@ -1014,6 +1038,18 @@ "revision": "b8fe1690c61389d7d2a8074a507d1d40c5d30448", "revisionTime": "2019-01-30T22:18:58Z" }, + { + "checksumSHA1": "UDvj5huw3BaGehfVRCB1UGQAtP4=", + "path": "golang.org/x/crypto/md4", + "revision": "f99c8df09eb5bff426315721bfa5f16a99cad32c", + "revisionTime": "2019-04-11T19:51:25Z" + }, + { + "checksumSHA1": "1MGpGDQqnUoRpv7VEcQrXOBydXE=", + "path": "golang.org/x/crypto/pbkdf2", + "revision": "f99c8df09eb5bff426315721bfa5f16a99cad32c", + "revisionTime": "2019-04-11T19:51:25Z" + }, { "checksumSHA1": "N5fb5y92DFIP+wUhi1rSwPp9vyk=", "path": "golang.org/x/crypto/ssh/terminal", @@ -1056,6 +1092,12 @@ "revision": "adae6a3d119ae4890b46832a2e88a95adc62b8e7", "revisionTime": "2018-11-14T21:44:15Z" }, + { + "checksumSHA1": "f3Y7JIZH61oMmp8nphqe8Mg+XoU=", + "path": "golang.org/x/net/internal/socks", + "revision": "461777fb6f67e8cb9d70cda16573678d085a74cf", + "revisionTime": "2019-06-07T17:48:03Z" + }, { "checksumSHA1": "UxahDzW2v4mf/+aFxruuupaoIwo=", "path": "golang.org/x/net/internal/timeseries", @@ -1068,6 +1110,12 @@ "revision": "5f8847ae0d0e90b6a9dc8148e7ad616874625171", "revisionTime": "2017-06-23T17:10:45Z" }, + { + "checksumSHA1": "28Sn0XihdqNv3MysxyRalibC3Tg=", + "path": "golang.org/x/net/proxy", + "revision": "461777fb6f67e8cb9d70cda16573678d085a74cf", + "revisionTime": "2019-06-07T17:48:03Z" + }, { "checksumSHA1": "j6leSoJatxWHJGLjRxIjZ8GbaDQ=", "path": "golang.org/x/net/publicsuffix", @@ -1530,6 +1578,210 @@ "revision": "6ed8d5f64cd79a498d1f3fab5880cc376ce41bbe", "revisionTime": "2019-01-03T01:53:35Z" }, + { + "checksumSHA1": "F+Irnk0yiBmKAsGWR2J2yvBFOZ8=", + "path": "gopkg.in/jcmturner/aescts.v1", + "revision": "f6abebb3171c4c1b1fea279cb7c7325020a26290", + "revisionTime": "2017-09-29T18:09:25Z" + }, + { + "checksumSHA1": "kpLq6IZ79NmMyWXernPOy4+fGHE=", + "path": "gopkg.in/jcmturner/dnsutils.v1", + "revision": "13eeb8d49ffb74d7a75784c35e4d900607a3943c", + "revisionTime": "2017-12-07T21:26:23Z" + }, + { + "checksumSHA1": "Uuwr2cH01D0aq0Gl5giJeAtWpnA=", + "path": "gopkg.in/jcmturner/gokrb5.v7/asn1tools", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "FVmUCSePixUAfx6jhzDW7EllQVc=", + "path": "gopkg.in/jcmturner/gokrb5.v7/client", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "gUDxRmkZO5pzlBHhZpuqvTfDnr4=", + "path": "gopkg.in/jcmturner/gokrb5.v7/config", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "LR9FEwfag+3acELa8ZNZcq1DfXg=", + "path": "gopkg.in/jcmturner/gokrb5.v7/credentials", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "ZuOkj9s02YBLtes1AvkOpDVGs/U=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "pSFrSD7w/jpi4+Gws+lc693ZOPA=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/common", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "MkKryd01aVXIdIFkWmWNeJaj4a0=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/etype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "8gbnVCpAOIOGzvtG5MHKyhV44w8=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3961", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "wgngoC64auRynoUWrXrU8Xl0/PQ=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/rfc3962", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "E8JwK4/IVqMtHPOmbdMggi2mXr4=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/rfc4757", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "2LZK7rQlMsCRqr9aDR928qPZxf8=", + "path": "gopkg.in/jcmturner/gokrb5.v7/crypto/rfc8009", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "UdNU0Nbxp91z2B/OPMzxovzHk4g=", + "path": "gopkg.in/jcmturner/gokrb5.v7/gssapi", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "FPN5n1+8jSEKuYAja+8pdXBvOY0=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "VGvdIIUbQnbjD34n0iIvK9IqR/c=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/addrtype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "eSCgwe8KcJ+Qc8l5/iPj3d+AZMo=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/adtype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "DSOjFrJRw8vWOq7yrWkJwjXYVuY=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/asnAppTag", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "Huf6Wp1LerUE5uThtf0NpNB4Nmo=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/chksumtype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "ClzQM3VsBqq9GZHcEyoKoTTbVVM=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/errorcode", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "WRY3wrbI2eVnza55P7zkO7EerNw=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/etypeID", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "zCDf0s+ln8SYxmadl+sWaxAQnso=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/flags", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "+S0w9xx42wKoyBygBcEOUrNG/jE=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/keyusage", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "OPi/ZTOtb/9TEI1iwB0sFKe0o+0=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/msgtype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "14gaT1595+oJb6qDi9Y+Iqny+lw=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/nametype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "2CmQeNxb3m70Ot+n9E2OV8YhCqE=", + "path": "gopkg.in/jcmturner/gokrb5.v7/iana/patype", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "32g/oJpR4H+GrJ0ZaCJZWMi2ovs=", + "path": "gopkg.in/jcmturner/gokrb5.v7/kadmin", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "6PEjlx97yL9wrjgJ6nfF8QxK+5A=", + "path": "gopkg.in/jcmturner/gokrb5.v7/keytab", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "Qio9pGLPgRZUTQuxIlpF2Lgy6ms=", + "path": "gopkg.in/jcmturner/gokrb5.v7/krberror", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "+KW7iEa6vOpFTCEqA+iWBxq4iB8=", + "path": "gopkg.in/jcmturner/gokrb5.v7/messages", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "CSBvso2BfxKarWPB6n1R+Ooixus=", + "path": "gopkg.in/jcmturner/gokrb5.v7/pac", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "Rb6mLWorZojin6cBRP2B3nUUT14=", + "path": "gopkg.in/jcmturner/gokrb5.v7/types", + "revision": "bae8ea1f6fab91f6bcb830efe54eb697c8350050", + "revisionTime": "2019-05-10T06:12:22Z" + }, + { + "checksumSHA1": "yFddxhhyhrcwdaXQ46OsoyNBx3A=", + "path": "gopkg.in/jcmturner/rpc.v1/mstypes", + "revision": "99a8ce2fbf8b8087b6ed12a37c61b10f04070043", + "revisionTime": "2018-08-26T21:10:00Z" + }, + { + "checksumSHA1": "WdV2JpSQC+oO/boLNH0E91x0hvY=", + "path": "gopkg.in/jcmturner/rpc.v1/ndr", + "revision": "99a8ce2fbf8b8087b6ed12a37c61b10f04070043", + "revisionTime": "2018-08-26T21:10:00Z" + }, { "checksumSHA1": "itYnRitfdzJjy2mZlvJ+hCJZvtY=", "path": "gopkg.in/ldap.v2", @@ -1543,34 +1795,46 @@ "revisionTime": "2016-09-28T15:37:09Z" }, { - "checksumSHA1": "rE4z8/fpbG6pYFal6rdEz09Fw3I=", + "checksumSHA1": "U9NZxwVFlXunU+p/HdvsUOMYgE4=", "path": "slack-github.com/slack/murron/pkg/lib", - "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", - "revisionTime": "2018-08-22T21:54:10Z" + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" }, { - "checksumSHA1": "DghGol8loNN4nq1p8j5d4bvNMSA=", + "checksumSHA1": "+I5C3O9nnWE9B8ZEim6uft0LGFs=", "path": "slack-github.com/slack/murron/pkg/log", - "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", - "revisionTime": "2018-08-22T21:54:10Z" + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" + }, + { + "checksumSHA1": "WN1kZFmoB1HGJu0laq5UVn7BQdA=", + "path": "slack-github.com/slack/murron/pkg/message_types", + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" }, { - "checksumSHA1": "yDr26+cKvGtatERB6wgzigbCueQ=", + "checksumSHA1": "RyuYHsgagnIEw47ryKtqK1PB73U=", "path": "slack-github.com/slack/murron/pkg/outputs", - "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", - "revisionTime": "2018-08-22T21:54:10Z" + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" + }, + { + "checksumSHA1": "60w5sGQ2gaoImGpetu8ifntsYJc=", + "path": "slack-github.com/slack/murron/pkg/queue", + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" }, { - "checksumSHA1": "ITTEcZ9tX4I85CjJ589kMUQR/Bc=", + "checksumSHA1": "ycFkK9PukDDlYzcl4OYDrG/kUTE=", "path": "slack-github.com/slack/murron/pkg/routes", - "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", - "revisionTime": "2018-08-22T21:54:10Z" + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" }, { - "checksumSHA1": "O22RLMWp6l6zujEHIq91SrdS5eg=", + "checksumSHA1": "Ajn9+e4jkzifoZ2bnhaV7Irh9xs=", "path": "slack-github.com/slack/murron/proto", - "revision": "14f1d3f7a7a3d5914b4ae7975720e6661484e2ba", - "revisionTime": "2018-08-22T21:54:10Z" + "revision": "99c10901b80a9d8d498604954da280809256415b", + "revisionTime": "2019-05-30T21:20:53Z" } ], "rootPath": "vitess.io/vitess"