2016-01-22 23:59:21 +03:00
|
|
|
// Copyright 2016, Google Inc. All rights reserved.
|
2015-08-25 23:58:42 +03:00
|
|
|
// Use of this source code is governed by a BSD-style
|
|
|
|
// license that can be found in the LICENSE file.
|
|
|
|
|
|
|
|
package vtgate
|
|
|
|
|
|
|
|
import (
|
|
|
|
"flag"
|
2015-12-11 02:07:38 +03:00
|
|
|
"strings"
|
|
|
|
"sync"
|
2015-08-25 23:58:42 +03:00
|
|
|
"time"
|
|
|
|
|
|
|
|
log "github.com/golang/glog"
|
|
|
|
"golang.org/x/net/context"
|
|
|
|
|
2015-11-10 11:50:46 +03:00
|
|
|
"github.com/youtube/vitess/go/sqltypes"
|
2015-08-25 23:58:42 +03:00
|
|
|
"github.com/youtube/vitess/go/stats"
|
2015-09-19 00:25:42 +03:00
|
|
|
"github.com/youtube/vitess/go/vt/discovery"
|
2015-12-02 18:38:26 +03:00
|
|
|
"github.com/youtube/vitess/go/vt/tabletserver/querytypes"
|
2015-08-25 23:58:42 +03:00
|
|
|
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
|
2015-09-17 06:50:53 +03:00
|
|
|
"github.com/youtube/vitess/go/vt/topo"
|
2015-12-02 18:38:26 +03:00
|
|
|
|
|
|
|
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
2015-08-25 23:58:42 +03:00
|
|
|
)
|
|
|
|
|
|
|
|
var (
|
2015-08-26 00:01:18 +03:00
|
|
|
// GatewayImplementation controls the implementation of Gateway.
|
2015-08-31 20:45:08 +03:00
|
|
|
GatewayImplementation = flag.String("gateway_implementation", "shardgateway", "The implementation of gateway")
|
2015-08-25 23:58:42 +03:00
|
|
|
)
|
|
|
|
|
|
|
|
// A Gateway is the query processing module for each shard,
|
|
|
|
// which is used by ScatterConn.
|
|
|
|
type Gateway interface {
|
2015-08-31 17:26:39 +03:00
|
|
|
// InitializeConnections creates connections to VTTablets.
|
|
|
|
InitializeConnections(ctx context.Context) error
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// Execute executes the non-streaming query for the specified keyspace, shard, and tablet type.
|
2015-11-12 12:13:51 +03:00
|
|
|
Execute(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}, transactionID int64) (*sqltypes.Result, error)
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// ExecuteBatch executes a group of queries for the specified keyspace, shard, and tablet type.
|
2015-12-02 18:38:26 +03:00
|
|
|
ExecuteBatch(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, queries []querytypes.BoundQuery, asTransaction bool, transactionID int64) ([]sqltypes.Result, error)
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// StreamExecute executes a streaming query for the specified keyspace, shard, and tablet type.
|
2015-11-12 12:13:51 +03:00
|
|
|
StreamExecute(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, query string, bindVars map[string]interface{}, transactionID int64) (<-chan *sqltypes.Result, tabletconn.ErrFunc)
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// Begin starts a transaction for the specified keyspace, shard, and tablet type.
|
|
|
|
// It returns the transaction ID.
|
2015-11-12 12:13:51 +03:00
|
|
|
Begin(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType) (int64, error)
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// Commit commits the current transaction for the specified keyspace, shard, and tablet type.
|
2015-11-12 12:13:51 +03:00
|
|
|
Commit(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, transactionID int64) error
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// Rollback rolls back the current transaction for the specified keyspace, shard, and tablet type.
|
2015-11-12 12:13:51 +03:00
|
|
|
Rollback(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, transactionID int64) error
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
// SplitQuery splits a query into sub-queries for the specified keyspace, shard, and tablet type.
|
2015-12-02 18:38:26 +03:00
|
|
|
SplitQuery(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, sql string, bindVariables map[string]interface{}, splitColumn string, splitCount int64) ([]querytypes.QuerySplit, error)
|
2015-08-25 23:58:42 +03:00
|
|
|
|
2015-08-26 00:01:18 +03:00
|
|
|
// Close shuts down underlying connections.
|
2015-10-02 01:00:19 +03:00
|
|
|
Close(ctx context.Context) error
|
2015-12-11 02:07:38 +03:00
|
|
|
|
|
|
|
// CacheStatus returns a list of GatewayEndPointCacheStatus per endpoint.
|
|
|
|
CacheStatus() GatewayEndPointCacheStatusList
|
2015-08-25 23:58:42 +03:00
|
|
|
}
|
|
|
|
|
2015-08-26 00:01:18 +03:00
|
|
|
// GatewayCreator is the func which can create the actual gateway object.
|
2015-12-05 01:24:32 +03:00
|
|
|
type GatewayCreator func(hc discovery.HealthCheck, topoServer topo.Server, serv topo.SrvTopoServer, cell string, retryDelay time.Duration, retryCount int, connTimeoutTotal, connTimeoutPerConn, connLife time.Duration, connTimings *stats.MultiTimings, tabletTypesToWait []topodatapb.TabletType) Gateway
|
2015-08-25 23:58:42 +03:00
|
|
|
|
|
|
|
var gatewayCreators = make(map[string]GatewayCreator)
|
|
|
|
|
2015-08-26 00:01:18 +03:00
|
|
|
// RegisterGatewayCreator registers a GatewayCreator with given name.
|
2015-08-25 23:58:42 +03:00
|
|
|
func RegisterGatewayCreator(name string, gc GatewayCreator) {
|
|
|
|
if _, ok := gatewayCreators[name]; ok {
|
|
|
|
log.Fatalf("Gateway %s already exists", name)
|
|
|
|
}
|
|
|
|
gatewayCreators[name] = gc
|
|
|
|
}
|
|
|
|
|
2015-08-26 00:01:18 +03:00
|
|
|
// GetGatewayCreator returns the GatewayCreator specified by GatewayImplementation flag.
|
2015-08-25 23:58:42 +03:00
|
|
|
func GetGatewayCreator() GatewayCreator {
|
|
|
|
gc, ok := gatewayCreators[*GatewayImplementation]
|
|
|
|
if !ok {
|
|
|
|
log.Fatalf("No gateway registered as %s", *GatewayImplementation)
|
2015-09-18 01:33:37 +03:00
|
|
|
}
|
|
|
|
return gc
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetGatewayCreatorByName returns the GatewayCreator specified by the given name.
|
|
|
|
func GetGatewayCreatorByName(name string) GatewayCreator {
|
|
|
|
gc, ok := gatewayCreators[name]
|
|
|
|
if !ok {
|
|
|
|
log.Errorf("No gateway registered as %s", name)
|
|
|
|
return nil
|
2015-08-25 23:58:42 +03:00
|
|
|
}
|
|
|
|
return gc
|
|
|
|
}
|
2015-12-11 02:07:38 +03:00
|
|
|
|
|
|
|
// GatewayEndPointCacheStatusList is a slice of GatewayEndPointCacheStatus.
|
|
|
|
type GatewayEndPointCacheStatusList []*GatewayEndPointCacheStatus
|
|
|
|
|
|
|
|
// Len is part of sort.Interface.
|
|
|
|
func (gepcsl GatewayEndPointCacheStatusList) Len() int {
|
|
|
|
return len(gepcsl)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Less is part of sort.Interface.
|
|
|
|
func (gepcsl GatewayEndPointCacheStatusList) Less(i, j int) bool {
|
|
|
|
iKey := strings.Join([]string{gepcsl[i].Keyspace, gepcsl[i].Shard, string(gepcsl[i].TabletType), gepcsl[i].Name}, ".")
|
|
|
|
jKey := strings.Join([]string{gepcsl[j].Keyspace, gepcsl[j].Shard, string(gepcsl[j].TabletType), gepcsl[j].Name}, ".")
|
|
|
|
return iKey < jKey
|
|
|
|
}
|
|
|
|
|
|
|
|
// Swap is part of sort.Interface.
|
|
|
|
func (gepcsl GatewayEndPointCacheStatusList) Swap(i, j int) {
|
|
|
|
gepcsl[i], gepcsl[j] = gepcsl[j], gepcsl[i]
|
|
|
|
}
|
|
|
|
|
|
|
|
// GatewayEndPointCacheStatus contains the status per endpoint for a gateway.
|
|
|
|
type GatewayEndPointCacheStatus struct {
|
|
|
|
Keyspace string
|
|
|
|
Shard string
|
|
|
|
TabletType topodatapb.TabletType
|
|
|
|
Name string
|
|
|
|
Addr string
|
|
|
|
|
|
|
|
QueryCount uint64
|
|
|
|
QueryError uint64
|
|
|
|
QPS uint64
|
2016-01-22 23:59:21 +03:00
|
|
|
AvgLatency float64 // in milliseconds
|
2015-12-11 02:07:38 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewGatewayEndPointStatusAggregator creates a GatewayEndPointStatusAggregator.
|
|
|
|
func NewGatewayEndPointStatusAggregator() *GatewayEndPointStatusAggregator {
|
|
|
|
gepsa := &GatewayEndPointStatusAggregator{}
|
|
|
|
go func() {
|
|
|
|
ticker := time.NewTicker(time.Second)
|
2015-12-16 02:38:45 +03:00
|
|
|
for range ticker.C {
|
2015-12-11 02:07:38 +03:00
|
|
|
gepsa.resetNextSlot()
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
return gepsa
|
|
|
|
}
|
|
|
|
|
|
|
|
// GatewayEndPointStatusAggregator tracks endpoint status for a gateway.
|
|
|
|
type GatewayEndPointStatusAggregator struct {
|
|
|
|
Keyspace string
|
|
|
|
Shard string
|
|
|
|
TabletType topodatapb.TabletType
|
|
|
|
Name string // the alternative name of an endpoint
|
|
|
|
Addr string // the host:port of an endpoint
|
|
|
|
|
|
|
|
// mu protects below fields.
|
|
|
|
mu sync.RWMutex
|
|
|
|
QueryCount uint64
|
|
|
|
QueryError uint64
|
|
|
|
// for QPS and latency (avg value over a minute)
|
2016-01-22 23:59:21 +03:00
|
|
|
tick uint32
|
2015-12-11 02:07:38 +03:00
|
|
|
queryCountInMinute [60]uint64
|
|
|
|
latencyInMinute [60]time.Duration
|
|
|
|
}
|
|
|
|
|
|
|
|
// UpdateQueryInfo updates the aggregator with the given information about a query.
|
2016-01-22 23:59:21 +03:00
|
|
|
func (gepsa *GatewayEndPointStatusAggregator) UpdateQueryInfo(addr string, tabletType topodatapb.TabletType, elapsed time.Duration, hasError bool) {
|
2015-12-11 02:07:38 +03:00
|
|
|
gepsa.mu.Lock()
|
|
|
|
defer gepsa.mu.Unlock()
|
2016-01-22 23:59:21 +03:00
|
|
|
if addr != "" {
|
|
|
|
gepsa.Addr = addr
|
|
|
|
}
|
2015-12-12 00:46:02 +03:00
|
|
|
gepsa.TabletType = tabletType
|
2015-12-11 02:07:38 +03:00
|
|
|
gepsa.QueryCount++
|
2016-01-22 23:59:21 +03:00
|
|
|
gepsa.queryCountInMinute[gepsa.tick]++
|
|
|
|
gepsa.latencyInMinute[gepsa.tick] += elapsed
|
2015-12-11 02:07:38 +03:00
|
|
|
if hasError {
|
|
|
|
gepsa.QueryError++
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// GetCacheStatus returns a GatewayEndPointCacheStatus representing the current gateway status.
|
|
|
|
func (gepsa *GatewayEndPointStatusAggregator) GetCacheStatus() *GatewayEndPointCacheStatus {
|
|
|
|
status := &GatewayEndPointCacheStatus{
|
2016-01-22 23:59:21 +03:00
|
|
|
Keyspace: gepsa.Keyspace,
|
|
|
|
Shard: gepsa.Shard,
|
|
|
|
Name: gepsa.Name,
|
2015-12-11 02:07:38 +03:00
|
|
|
}
|
|
|
|
gepsa.mu.RLock()
|
|
|
|
defer gepsa.mu.RUnlock()
|
2016-01-22 23:59:21 +03:00
|
|
|
status.TabletType = gepsa.TabletType
|
|
|
|
status.Addr = gepsa.Addr
|
2015-12-11 02:07:38 +03:00
|
|
|
status.QueryCount = gepsa.QueryCount
|
|
|
|
status.QueryError = gepsa.QueryError
|
|
|
|
var totalQuery uint64
|
|
|
|
for _, c := range gepsa.queryCountInMinute {
|
|
|
|
totalQuery += c
|
|
|
|
}
|
|
|
|
var totalLatency time.Duration
|
|
|
|
for _, d := range gepsa.latencyInMinute {
|
|
|
|
totalLatency += d
|
|
|
|
}
|
|
|
|
status.QPS = totalQuery / 60
|
|
|
|
if totalQuery > 0 {
|
2016-01-22 23:59:21 +03:00
|
|
|
status.AvgLatency = float64(totalLatency.Nanoseconds()) / float64(totalQuery) / 1000000
|
2015-12-11 02:07:38 +03:00
|
|
|
}
|
|
|
|
return status
|
|
|
|
}
|
|
|
|
|
|
|
|
// resetNextSlot resets the next tracking slot.
|
|
|
|
func (gepsa *GatewayEndPointStatusAggregator) resetNextSlot() {
|
|
|
|
gepsa.mu.Lock()
|
|
|
|
defer gepsa.mu.Unlock()
|
2016-01-22 23:59:21 +03:00
|
|
|
gepsa.tick = (gepsa.tick + 1) % 60
|
|
|
|
gepsa.queryCountInMinute[gepsa.tick] = 0
|
|
|
|
gepsa.latencyInMinute[gepsa.tick] = time.Duration(0)
|
2015-12-11 02:07:38 +03:00
|
|
|
}
|