2013-01-26 05:09:32 +04:00
|
|
|
// Copyright 2012, Google Inc. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style
|
|
|
|
// license that can be found in the LICENSE file.
|
|
|
|
|
|
|
|
package tabletserver
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
2014-02-15 03:14:33 +04:00
|
|
|
"sync"
|
2014-02-06 05:00:16 +04:00
|
|
|
"time"
|
2013-01-26 05:09:32 +04:00
|
|
|
|
Automatic rewrite of relog import paths and calls to use glog.
Commands run:
find go -name "*.go" | xargs sed --in-place -r 's,"github.com/youtube/vitess/go/relog",log "github.com/golang/glog",g; s,relog.Info,log.Infof,g; s,relog.Warning,log.Warningf,g; s,relog.Error,log.Errorf,g; s,relog.Fatal,log.Fatalf,g; s,relog.Debug,log.V(6).Infof,g'
find . -name '*.go' -exec gofmt -w {} \;
2013-08-07 01:56:00 +04:00
|
|
|
log "github.com/golang/glog"
|
2013-07-19 05:18:20 +04:00
|
|
|
"github.com/youtube/vitess/go/sqltypes"
|
2013-11-18 12:41:05 +04:00
|
|
|
"github.com/youtube/vitess/go/stats"
|
2013-07-19 05:18:20 +04:00
|
|
|
"github.com/youtube/vitess/go/sync2"
|
2014-02-06 05:00:16 +04:00
|
|
|
"github.com/youtube/vitess/go/tb"
|
2014-01-27 21:49:54 +04:00
|
|
|
"github.com/youtube/vitess/go/vt/binlog"
|
|
|
|
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
|
2014-02-15 03:14:33 +04:00
|
|
|
"github.com/youtube/vitess/go/vt/mysqlctl"
|
2014-07-25 22:39:42 +04:00
|
|
|
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
|
2014-10-06 10:22:20 +04:00
|
|
|
"github.com/youtube/vitess/go/vt/schema"
|
|
|
|
"github.com/youtube/vitess/go/vt/sqlparser"
|
|
|
|
"github.com/youtube/vitess/go/vt/tabletserver/planbuilder"
|
2013-01-26 05:09:32 +04:00
|
|
|
)
|
|
|
|
|
2014-05-05 08:45:13 +04:00
|
|
|
// RowcacheInvalidator runs the service to invalidate
|
|
|
|
// the rowcache based on binlog events.
|
2014-02-15 03:14:33 +04:00
|
|
|
type RowcacheInvalidator struct {
|
2014-08-28 11:07:14 +04:00
|
|
|
qe *QueryEngine
|
|
|
|
dbname string
|
|
|
|
mysqld *mysqlctl.Mysqld
|
|
|
|
|
2014-02-15 03:14:33 +04:00
|
|
|
svm sync2.ServiceManager
|
2013-11-23 05:06:27 +04:00
|
|
|
|
2014-09-04 04:43:40 +04:00
|
|
|
posMutex sync.Mutex
|
|
|
|
pos myproto.ReplicationPosition
|
2014-08-28 11:07:14 +04:00
|
|
|
lagSeconds sync2.AtomicInt64
|
|
|
|
}
|
|
|
|
|
2014-09-04 04:43:40 +04:00
|
|
|
// AppendGTID updates the current replication position by appending a GTID to
|
|
|
|
// the set of transactions that have been processed.
|
|
|
|
func (rci *RowcacheInvalidator) AppendGTID(gtid myproto.GTID) {
|
|
|
|
rci.posMutex.Lock()
|
|
|
|
defer rci.posMutex.Unlock()
|
|
|
|
rci.pos = myproto.AppendGTID(rci.pos, gtid)
|
2014-07-25 22:39:42 +04:00
|
|
|
}
|
|
|
|
|
2014-09-04 04:43:40 +04:00
|
|
|
// SetPosition sets the current ReplicationPosition.
|
|
|
|
func (rci *RowcacheInvalidator) SetPosition(rp myproto.ReplicationPosition) {
|
|
|
|
rci.posMutex.Lock()
|
|
|
|
defer rci.posMutex.Unlock()
|
|
|
|
rci.pos = rp
|
2014-07-25 22:39:42 +04:00
|
|
|
}
|
|
|
|
|
2014-09-04 04:43:40 +04:00
|
|
|
// Position returns the current ReplicationPosition.
|
|
|
|
func (rci *RowcacheInvalidator) Position() myproto.ReplicationPosition {
|
|
|
|
rci.posMutex.Lock()
|
|
|
|
defer rci.posMutex.Unlock()
|
|
|
|
return rci.pos
|
|
|
|
}
|
|
|
|
|
|
|
|
// PositionString returns the current ReplicationPosition as a string.
|
|
|
|
func (rci *RowcacheInvalidator) PositionString() string {
|
|
|
|
return rci.Position().String()
|
2014-07-25 22:39:42 +04:00
|
|
|
}
|
|
|
|
|
2014-02-15 03:14:33 +04:00
|
|
|
// NewRowcacheInvalidator creates a new RowcacheInvalidator.
|
|
|
|
// Just like QueryEngine, this is a singleton class.
|
|
|
|
// You must call this only once.
|
|
|
|
func NewRowcacheInvalidator(qe *QueryEngine) *RowcacheInvalidator {
|
|
|
|
rci := &RowcacheInvalidator{qe: qe}
|
|
|
|
stats.Publish("RowcacheInvalidatorState", stats.StringFunc(rci.svm.StateName))
|
2014-09-04 04:43:40 +04:00
|
|
|
stats.Publish("RowcacheInvalidatorPosition", stats.StringFunc(rci.PositionString))
|
2014-08-18 08:05:53 +04:00
|
|
|
stats.Publish("RowcacheInvalidatorLagSeconds", stats.IntFunc(rci.lagSeconds.Get))
|
2014-02-15 03:14:33 +04:00
|
|
|
return rci
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
|
|
|
|
2014-02-15 03:14:33 +04:00
|
|
|
// Open runs the invalidation loop.
|
|
|
|
func (rci *RowcacheInvalidator) Open(dbname string, mysqld *mysqlctl.Mysqld) {
|
2014-09-04 04:43:40 +04:00
|
|
|
rp, err := mysqld.MasterPosition()
|
2014-02-15 03:14:33 +04:00
|
|
|
if err != nil {
|
2015-01-06 20:37:34 +03:00
|
|
|
panic(NewTabletError(ErrFatal, "Rowcache invalidator aborting: cannot determine replication position: %v", err))
|
2013-07-12 02:09:27 +04:00
|
|
|
}
|
2014-02-15 03:14:33 +04:00
|
|
|
if mysqld.Cnf().BinLogPath == "" {
|
2015-01-06 20:37:34 +03:00
|
|
|
panic(NewTabletError(ErrFatal, "Rowcache invalidator aborting: binlog path not specified"))
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
2014-08-28 11:07:14 +04:00
|
|
|
rci.dbname = dbname
|
|
|
|
rci.mysqld = mysqld
|
2014-09-04 04:43:40 +04:00
|
|
|
rci.SetPosition(rp)
|
2014-02-15 03:14:33 +04:00
|
|
|
|
2014-08-28 11:07:14 +04:00
|
|
|
ok := rci.svm.Go(rci.run)
|
2014-02-15 03:14:33 +04:00
|
|
|
if ok {
|
2014-09-04 04:43:40 +04:00
|
|
|
log.Infof("Rowcache invalidator starting, dbname: %s, path: %s, position: %v", dbname, mysqld.Cnf().BinLogPath, rp)
|
2014-02-15 03:14:33 +04:00
|
|
|
} else {
|
2013-11-18 12:41:05 +04:00
|
|
|
log.Infof("Rowcache invalidator already running")
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
2014-02-15 03:14:33 +04:00
|
|
|
}
|
2013-11-23 05:06:27 +04:00
|
|
|
|
2014-02-15 03:14:33 +04:00
|
|
|
// Close terminates the invalidation loop. It returns only of the
|
|
|
|
// loop has terminated.
|
|
|
|
func (rci *RowcacheInvalidator) Close() {
|
2014-06-15 05:23:45 +04:00
|
|
|
rci.svm.Stop()
|
2014-02-15 03:14:33 +04:00
|
|
|
}
|
2013-07-12 02:09:27 +04:00
|
|
|
|
2014-08-28 11:07:14 +04:00
|
|
|
func (rci *RowcacheInvalidator) run(ctx *sync2.ServiceContext) error {
|
2014-02-06 05:00:16 +04:00
|
|
|
for {
|
2014-09-04 04:43:40 +04:00
|
|
|
evs := binlog.NewEventStreamer(rci.dbname, rci.mysqld, rci.Position(), rci.processEvent)
|
2014-02-06 05:00:16 +04:00
|
|
|
// We wrap this code in a func so we can catch all panics.
|
|
|
|
// If an error is returned, we log it, wait 1 second, and retry.
|
2014-02-15 03:14:33 +04:00
|
|
|
// This loop can only be stopped by calling Close.
|
2014-02-06 05:00:16 +04:00
|
|
|
err := func() (inner error) {
|
|
|
|
defer func() {
|
|
|
|
if x := recover(); x != nil {
|
|
|
|
inner = fmt.Errorf("%v: uncaught panic:\n%s", x, tb.Stack(4))
|
|
|
|
}
|
|
|
|
}()
|
2014-08-28 11:07:14 +04:00
|
|
|
return evs.Stream(ctx)
|
2014-02-06 05:00:16 +04:00
|
|
|
}()
|
|
|
|
if err == nil {
|
|
|
|
break
|
|
|
|
}
|
2015-01-23 10:22:09 +03:00
|
|
|
if IsConnErr(err) {
|
|
|
|
go CheckMySQL()
|
|
|
|
}
|
2014-02-15 03:14:33 +04:00
|
|
|
log.Errorf("binlog.ServeUpdateStream returned err '%v', retrying in 1 second.", err.Error())
|
2014-02-06 05:00:16 +04:00
|
|
|
internalErrors.Add("Invalidation", 1)
|
|
|
|
time.Sleep(1 * time.Second)
|
2013-07-12 02:09:27 +04:00
|
|
|
}
|
2013-11-18 12:41:05 +04:00
|
|
|
log.Infof("Rowcache invalidator stopped")
|
2014-08-28 11:07:14 +04:00
|
|
|
return nil
|
2013-07-12 02:09:27 +04:00
|
|
|
}
|
|
|
|
|
2014-05-05 08:45:13 +04:00
|
|
|
func handleInvalidationError(event *blproto.StreamEvent) {
|
|
|
|
if x := recover(); x != nil {
|
|
|
|
terr, ok := x.(*TabletError)
|
|
|
|
if !ok {
|
|
|
|
log.Errorf("Uncaught panic for %+v:\n%v\n%s", event, x, tb.Stack(4))
|
|
|
|
internalErrors.Add("Panic", 1)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Errorf("%v: %+v", terr, event)
|
|
|
|
internalErrors.Add("Invalidation", 1)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-08-28 11:07:14 +04:00
|
|
|
func (rci *RowcacheInvalidator) processEvent(event *blproto.StreamEvent) error {
|
2014-05-05 08:45:13 +04:00
|
|
|
defer handleInvalidationError(event)
|
2013-11-18 12:41:05 +04:00
|
|
|
switch event.Category {
|
|
|
|
case "DDL":
|
2014-08-08 01:20:36 +04:00
|
|
|
log.Infof("DDL invalidation: %s", event.Sql)
|
2014-10-06 10:22:20 +04:00
|
|
|
rci.handleDDLEvent(event.Sql)
|
2013-11-18 12:41:05 +04:00
|
|
|
case "DML":
|
2014-10-06 10:22:20 +04:00
|
|
|
rci.handleDMLEvent(event)
|
2013-11-18 12:41:05 +04:00
|
|
|
case "ERR":
|
2014-10-06 10:22:20 +04:00
|
|
|
rci.handleUnrecognizedEvent(event.Sql)
|
2013-11-18 12:41:05 +04:00
|
|
|
case "POS":
|
2014-09-04 04:43:40 +04:00
|
|
|
rci.AppendGTID(event.GTIDField.Value)
|
2013-01-26 05:09:32 +04:00
|
|
|
default:
|
2014-02-06 05:00:16 +04:00
|
|
|
log.Errorf("unknown event: %#v", event)
|
|
|
|
internalErrors.Add("Invalidation", 1)
|
2014-08-28 11:07:14 +04:00
|
|
|
return nil
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
2014-08-18 08:05:53 +04:00
|
|
|
rci.lagSeconds.Set(time.Now().Unix() - event.Timestamp)
|
2014-08-28 11:07:14 +04:00
|
|
|
return nil
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
|
|
|
|
2014-10-06 10:22:20 +04:00
|
|
|
func (rci *RowcacheInvalidator) handleDMLEvent(event *blproto.StreamEvent) {
|
|
|
|
invalidations := int64(0)
|
|
|
|
tableInfo := rci.qe.schemaInfo.GetTable(event.TableName)
|
|
|
|
if tableInfo == nil {
|
2015-01-06 20:37:34 +03:00
|
|
|
panic(NewTabletError(ErrFail, "Table %s not found", event.TableName))
|
2014-10-06 10:22:20 +04:00
|
|
|
}
|
|
|
|
if tableInfo.CacheType == schema.CACHE_NONE {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2013-12-03 09:31:51 +04:00
|
|
|
sqlTypeKeys := make([]sqltypes.Value, 0, len(event.PKColNames))
|
|
|
|
for _, pkTuple := range event.PKValues {
|
2013-01-26 05:09:32 +04:00
|
|
|
sqlTypeKeys = sqlTypeKeys[:0]
|
|
|
|
for _, pkVal := range pkTuple {
|
|
|
|
key, err := sqltypes.BuildValue(pkVal)
|
|
|
|
if err != nil {
|
2013-11-18 12:41:05 +04:00
|
|
|
log.Errorf("Error building invalidation key for %#v: '%v'", event, err)
|
2013-12-08 08:16:52 +04:00
|
|
|
internalErrors.Add("Invalidation", 1)
|
2013-11-18 12:41:05 +04:00
|
|
|
return
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
|
|
|
sqlTypeKeys = append(sqlTypeKeys, key)
|
|
|
|
}
|
2014-10-06 10:22:20 +04:00
|
|
|
newKey := validateKey(tableInfo, buildKey(sqlTypeKeys))
|
|
|
|
if newKey == "" {
|
|
|
|
continue
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
2014-10-06 10:22:20 +04:00
|
|
|
tableInfo.Cache.Delete(newKey)
|
|
|
|
invalidations++
|
|
|
|
}
|
|
|
|
tableInfo.invalidations.Add(invalidations)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (rci *RowcacheInvalidator) handleDDLEvent(ddl string) {
|
|
|
|
ddlPlan := planbuilder.DDLParse(ddl)
|
|
|
|
if ddlPlan.Action == "" {
|
2015-01-06 20:37:34 +03:00
|
|
|
panic(NewTabletError(ErrFail, "DDL is not understood"))
|
2013-01-26 05:09:32 +04:00
|
|
|
}
|
2014-10-06 10:22:20 +04:00
|
|
|
if ddlPlan.TableName != "" && ddlPlan.TableName != ddlPlan.NewName {
|
|
|
|
// It's a drop or rename.
|
|
|
|
rci.qe.schemaInfo.DropTable(ddlPlan.TableName)
|
|
|
|
}
|
|
|
|
if ddlPlan.NewName != "" {
|
|
|
|
rci.qe.schemaInfo.CreateOrUpdateTable(ddlPlan.NewName)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (rci *RowcacheInvalidator) handleUnrecognizedEvent(sql string) {
|
|
|
|
statement, err := sqlparser.Parse(sql)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("Error: %v: %s", err, sql)
|
|
|
|
internalErrors.Add("Invalidation", 1)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
var table *sqlparser.TableName
|
|
|
|
switch stmt := statement.(type) {
|
|
|
|
case *sqlparser.Insert:
|
|
|
|
// Inserts don't affect rowcache.
|
|
|
|
return
|
|
|
|
case *sqlparser.Update:
|
|
|
|
table = stmt.Table
|
|
|
|
case *sqlparser.Delete:
|
|
|
|
table = stmt.Table
|
|
|
|
default:
|
|
|
|
log.Errorf("Unrecognized: %s", sql)
|
|
|
|
internalErrors.Add("Invalidation", 1)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ignore cross-db statements.
|
2014-11-26 10:45:56 +03:00
|
|
|
if table.Qualifier != nil && string(table.Qualifier) != rci.qe.dbconfigs.App.DbName {
|
2014-10-06 10:22:20 +04:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ignore if it's an uncached table.
|
|
|
|
tableName := string(table.Name)
|
|
|
|
tableInfo := rci.qe.schemaInfo.GetTable(tableName)
|
|
|
|
if tableInfo == nil {
|
|
|
|
log.Errorf("Table %s not found: %s", tableName, sql)
|
|
|
|
internalErrors.Add("Invalidation", 1)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if tableInfo.CacheType == schema.CACHE_NONE {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Treat the statement as a DDL.
|
|
|
|
// It will conservatively invalidate all rows of the table.
|
|
|
|
log.Warningf("Treating '%s' as DDL for table %s", sql, tableName)
|
|
|
|
rci.qe.schemaInfo.CreateOrUpdateTable(tableName)
|
2014-08-08 01:20:36 +04:00
|
|
|
}
|