make QueryRuleSources authority of query rules

This commit is contained in:
Jeff (Zhefu) Jiang 2014-12-18 16:07:44 -08:00
Родитель 65189e199a
Коммит f8c86ddcbb
9 изменённых файлов: 150 добавлений и 101 удалений

Просмотреть файл

@ -63,7 +63,7 @@ func main() {
}
tabletserver.InitQueryService()
err = tabletserver.SqlQueryRpcService.SetQueryRules(tabletserver.CustomQueryRules, tabletserver.LoadCustomRules())
err = tabletserver.SetQueryRules(tabletserver.CustomQueryRules, tabletserver.LoadCustomRules())
if err != nil {
log.Warningf("Fail to load query rule set %s, Error message: %s", tabletserver.CustomQueryRules, err)
}

Просмотреть файл

@ -91,7 +91,7 @@ func main() {
binlog.RegisterUpdateStreamService(mycnf)
// Load custom query rules
err = tabletserver.SqlQueryRpcService.SetQueryRules(tabletserver.CustomQueryRules, tabletserver.LoadCustomRules())
err = tabletserver.SetQueryRules(tabletserver.CustomQueryRules, tabletserver.LoadCustomRules())
if err != nil {
log.Warningf("Fail to load query rule set %s, Error message: %s", tabletserver.CustomQueryRules, err)
}

Просмотреть файл

@ -34,6 +34,12 @@ var (
historyLength = 16
)
// Query rules from keyrange
const KeyrangeQueryRules string = "KEYRANGE_QUERY_RULES"
// Query rules from blacklist
const BlacklistQueryRules string = "BLACKLIST_QUERY_RULES"
func (agent *ActionAgent) allowQueries(tablet *topo.Tablet, blacklistedTables []string) error {
if agent.DBConfigs == nil {
// test instance, do nothing
@ -114,14 +120,14 @@ func (agent *ActionAgent) loadKeyspaceAndBlacklistRules(tablet *topo.Tablet, bla
blacklistRules.Add(qr)
}
// Push all three sets of QueryRules to SqlQueryRpcService
loadRuleErr := tabletserver.SqlQueryRpcService.SetQueryRules(tabletserver.KeyrangeQueryRules, keyrangeRules)
loadRuleErr := tabletserver.SetQueryRules(KeyrangeQueryRules, keyrangeRules)
if loadRuleErr != nil {
log.Warningf("Fail to load query rule set %s, Error message: %s", tabletserver.KeyrangeQueryRules, loadRuleErr)
log.Warningf("Fail to load query rule set %s: %s", KeyrangeQueryRules, loadRuleErr)
}
loadRuleErr = tabletserver.SqlQueryRpcService.SetQueryRules(tabletserver.BlacklistQueryRules, blacklistRules)
loadRuleErr = tabletserver.SetQueryRules(BlacklistQueryRules, blacklistRules)
if loadRuleErr != nil {
log.Warningf("Fail to load query rule set %s, Error message: %s", tabletserver.BlacklistQueryRules, loadRuleErr)
log.Warningf("Fail to load query rule set %s: %s", BlacklistQueryRules, loadRuleErr)
}
return nil
}
@ -233,3 +239,16 @@ func (agent *ActionAgent) changeCallback(ctx context.Context, oldTablet, newTabl
}
return nil
}
func init() {
// Register query rule sources under control of agent
err := tabletserver.QueryRuleSources.RegisterQueryRuleSource(KeyrangeQueryRules)
if err != nil {
log.Errorf("Cannot register %s with tabletserver.QueryRuleSources: %v", KeyrangeQueryRules, err)
}
err = tabletserver.QueryRuleSources.RegisterQueryRuleSource(BlacklistQueryRules)
if err != nil {
log.Errorf("Cannot register %s with tabletserver.QueryRuleSources: %v", BlacklistQueryRules, err)
}
}

Просмотреть файл

@ -15,11 +15,20 @@ import (
)
var (
keyrangeRules *QueryRules
blacklistRules *QueryRules
customQueryRules *QueryRules
keyrangeRules *QueryRules
blacklistRules *QueryRules
otherRules *QueryRules
)
// mimic query rules from keyrange
const keyrangeQueryRules string = "KEYRANGE_QUERY_RULES"
// mimic query rules from blacklist
const blacklistQueryRules string = "BLACKLIST_QUERY_RULES"
// mimic query rules from custom source
const customQueryRules string = "CUSTOM_QUERY_RULES"
func setupQueryRules() {
var qr *QueryRule
// mock keyrange rules
@ -55,19 +64,33 @@ func setupQueryRules() {
blacklistRules.Add(qr)
// mock custom rules
customQueryRules = NewQueryRules()
otherRules = NewQueryRules()
qr = NewQueryRule("sample custom rule", "customrule_ban_bindvar", QR_FAIL)
qr.AddTableCond("t_customer")
qr.AddBindVarCond("bindvar1", true, false, QR_NOOP, nil)
customQueryRules.Add(qr)
otherRules.Add(qr)
}
func TestQueryRuleInfoGetSetQueryRules(t *testing.T) {
var qrs *QueryRules
setupQueryRules()
qri := NewQueryRuleInfo()
err := qri.RegisterQueryRuleSource(keyrangeQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", keyrangeQueryRules, err)
}
err = qri.RegisterQueryRuleSource(blacklistQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", blacklistQueryRules, err)
}
err = qri.RegisterQueryRuleSource(customQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", customQueryRules, err)
}
// Test if we can get a QueryRules without a predefined rule set name
err, qrs := qri.GetRules("Foo")
err, qrs = qri.GetRules("Foo")
if err == nil {
t.Errorf("GetRules shouldn't succeed with 'Foo' as the rule set name")
}
@ -85,54 +108,69 @@ func TestQueryRuleInfoGetSetQueryRules(t *testing.T) {
}
// Test if we can successfully set QueryRules previously mocked into QueryRuleInfo
err = qri.SetRules(KeyrangeQueryRules, keyrangeRules)
err = qri.SetRules(keyrangeQueryRules, keyrangeRules)
if err != nil {
t.Errorf("Failed to set keyrange QueryRules, errmsg: %s", err)
t.Errorf("Failed to set keyrange QueryRules : %s", err)
}
err = qri.SetRules(BlacklistQueryRules, blacklistRules)
err = qri.SetRules(blacklistQueryRules, blacklistRules)
if err != nil {
t.Errorf("Failed to set blacklist QueryRules, errmsg: %s", err)
t.Errorf("Failed to set blacklist QueryRules: %s", err)
}
err = qri.SetRules(CustomQueryRules, customQueryRules)
err = qri.SetRules(customQueryRules, otherRules)
if err != nil {
t.Errorf("Failed to set custom QueryRules, errmsg: %s", err)
t.Errorf("Failed to set custom QueryRules: %s", err)
}
// Test if we can successfully retrive rules that've been set
err, qrs = qri.GetRules(KeyrangeQueryRules)
err, qrs = qri.GetRules(keyrangeQueryRules)
if err != nil {
t.Errorf("GetRules failed to retrieve KeyrangeQueryRules that has been set, errmsg: %s", err)
t.Errorf("GetRules failed to retrieve keyrangeQueryRules that has been set: %s", err)
}
if !reflect.DeepEqual(qrs, keyrangeRules) {
t.Errorf("KeyrangeQueryRules retrived is %v, but the expected value should be %v", qrs, keyrangeRules)
t.Errorf("keyrangeQueryRules retrived is %v, but the expected value should be %v", qrs, keyrangeRules)
}
err, qrs = qri.GetRules(BlacklistQueryRules)
err, qrs = qri.GetRules(blacklistQueryRules)
if err != nil {
t.Errorf("GetRules failed to retrieve BlacklistQueryRules that has been set, errmsg: %s", err)
t.Errorf("GetRules failed to retrieve blacklistQueryRules that has been set: %s", err)
}
if !reflect.DeepEqual(qrs, blacklistRules) {
t.Errorf("BlacklistQueryRules retrived is %v, but the expected value should be %v", qrs, blacklistRules)
t.Errorf("blacklistQueryRules retrived is %v, but the expected value should be %v", qrs, blacklistRules)
}
err, qrs = qri.GetRules(CustomQueryRules)
err, qrs = qri.GetRules(customQueryRules)
if err != nil {
t.Errorf("GetRules failed to retrieve CustomQueryRules that has been set, errmsg: %s", err)
t.Errorf("GetRules failed to retrieve customQueryRules that has been set: %s", err)
}
if !reflect.DeepEqual(qrs, customQueryRules) {
t.Errorf("CustomQueryRules retrived is %v, but the expected value should be %v", qrs, customQueryRules)
if !reflect.DeepEqual(qrs, otherRules) {
t.Errorf("customQueryRules retrived is %v, but the expected value should be %v", qrs, customQueryRules)
}
}
func TestQueryRuleInfoFilterByPlan(t *testing.T) {
var qrs *QueryRules
setupQueryRules()
qri := NewQueryRuleInfo()
qri.SetRules(KeyrangeQueryRules, keyrangeRules)
qri.SetRules(BlacklistQueryRules, blacklistRules)
qri.SetRules(CustomQueryRules, customQueryRules)
err := qri.RegisterQueryRuleSource(keyrangeQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", keyrangeQueryRules, err)
}
err = qri.RegisterQueryRuleSource(blacklistQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", blacklistQueryRules, err)
}
err = qri.RegisterQueryRuleSource(customQueryRules)
if err != nil {
t.Errorf("Fail to register query rule source %s: %s", customQueryRules, err)
}
qri.SetRules(keyrangeQueryRules, keyrangeRules)
qri.SetRules(blacklistQueryRules, blacklistRules)
qri.SetRules(customQueryRules, otherRules)
// Test filter by keyrange rule
qrs := qri.filterByPlan("insert into t_test values(123, 456, 'abc')", planbuilder.PLAN_INSERT_PK, "t_test")
qrs = qri.filterByPlan("insert into t_test values(123, 456, 'abc')", planbuilder.PLAN_INSERT_PK, "t_test")
if l := len(qrs.rules); l != 1 {
t.Errorf("Insert PK query matches %d rules, but we expect %d", l, 1)
}
@ -159,11 +197,11 @@ func TestQueryRuleInfoFilterByPlan(t *testing.T) {
}
// Test match two rules: both keyrange rule and custom rule will be matched
customQueryRules = NewQueryRules()
otherRules = NewQueryRules()
qr := NewQueryRule("sample custom rule", "customrule_ban_bindvar", QR_FAIL)
qr.AddBindVarCond("bindvar1", true, false, QR_NOOP, nil)
customQueryRules.Add(qr)
qri.SetRules(CustomQueryRules, customQueryRules)
otherRules.Add(qr)
qri.SetRules(customQueryRules, otherRules)
qrs = qri.filterByPlan("insert into t_test values (:bindvar1, 123, 'test')", planbuilder.PLAN_INSERT_PK, "t_test")
if l := len(qrs.rules); l != 2 {
t.Errorf("Insert into t_test matches %d rules: %v, but we expect %d rules to be matched", l, qrs.rules, 2)

Просмотреть файл

@ -38,9 +38,8 @@ const spotCheckMultiplier = 1e6
// panic with NewTabletError as the error type.
// TODO(sougou): Switch to error return scheme.
type QueryEngine struct {
schemaInfo *SchemaInfo
queryRuleInfo *QueryRuleInfo
dbconfigs *dbconfigs.DBConfigs
schemaInfo *SchemaInfo
dbconfigs *dbconfigs.DBConfigs
// Pools
cachePool *CachePool
@ -121,7 +120,6 @@ func NewQueryEngine(config Config) *QueryEngine {
time.Duration(config.SchemaReloadTime*1e9),
time.Duration(config.IdleTimeout*1e9),
)
qe.queryRuleInfo = NewQueryRuleInfo()
mysqlStats = stats.NewTimings("Mysql")
@ -279,5 +277,4 @@ func (qe *QueryEngine) Close() {
qe.schemaInfo.Close()
qe.cachePool.Close()
qe.dbconfigs = nil
qe.queryRuleInfo = NewQueryRuleInfo()
}

Просмотреть файл

@ -183,12 +183,19 @@ func GetSessionId() int64 {
return SqlQueryRpcService.sessionId
}
func SetQueryRules(queryRuleSet string, qrs *QueryRules) error {
return SqlQueryRpcService.qe.queryRuleInfo.SetRules(queryRuleSet, qrs)
// GetQueryRules is the tabletserver level API to get current query rules
func GetQueryRules(ruleSource string) (error, *QueryRules) {
return QueryRuleSources.GetRules(ruleSource)
}
func GetQueryRules(queryRuleSet string) (error, *QueryRules) {
return SqlQueryRpcService.qe.queryRuleInfo.GetRules(queryRuleSet)
// SetQueryRules is the tabletserver level API to write current query rules
func SetQueryRules(ruleSource string, qrs *QueryRules) error {
err := QueryRuleSources.SetRules(ruleSource, qrs)
if err != nil {
return err
}
SqlQueryRpcService.qe.schemaInfo.ClearQueryPlanCache()
return nil
}
// IsHealthy returns nil if the query service is healthy (able to
@ -237,6 +244,13 @@ func InitQueryService() {
RegisterQueryService()
}
// QueryRules from custom rules
const CustomQueryRules string = "CUSTOM_QUERY_RULES"
func init() {
QueryRuleSources.RegisterQueryRuleSource(CustomQueryRules)
}
// LoadCustomRules returns custom rules as specified by the command
// line flags.
func LoadCustomRules() (qrs *QueryRules) {

Просмотреть файл

@ -11,64 +11,67 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/planbuilder"
)
var QueryRuleSources *QueryRuleInfo = NewQueryRuleInfo()
// QueryRuleInfo is the maintainer of QueryRules from multiple sources
type QueryRuleInfo struct {
mu sync.Mutex
// mutex to protect following queryRulesMap
mu sync.Mutex
// queryRulesMap maps the names of different query rule sources to the actual QueryRules structure
queryRulesMap map[string]*QueryRules
}
// Names for QueryRules coming from different sources
// QueryRules from keyrange
const KeyrangeQueryRules string = "KEYRANGE_QUERY_RULES"
// QueryRules from blacklist
const BlacklistQueryRules string = "BLACKLIST_QUERY_RULES"
// QueryRules from custom rules
const CustomQueryRules string = "CUSTOM_QUERY_RULES"
func NewQueryRuleInfo() *QueryRuleInfo {
qri := &QueryRuleInfo{
queryRulesMap: map[string]*QueryRules{
KeyrangeQueryRules: NewQueryRules(),
BlacklistQueryRules: NewQueryRules(),
CustomQueryRules: NewQueryRules(),
},
queryRulesMap: map[string]*QueryRules{},
}
return qri
}
// SetRules takes a external QueryRules structure and overwrite one of the
// internal QueryRules as designated by queryRuleSet parameter
func (qri *QueryRuleInfo) SetRules(queryRuleSet string, newRules *QueryRules) error {
// RegisterQueryRuleSource registers a query rule source name with QueryRuleInfo
func (qri *QueryRuleInfo) RegisterQueryRuleSource(ruleSource string) error {
qri.mu.Lock()
defer qri.mu.Unlock()
if _, existed := qri.queryRulesMap[ruleSource]; existed {
return errors.New("Query rule source " + ruleSource + " has been registered")
}
qri.queryRulesMap[ruleSource] = NewQueryRules()
return nil
}
// SetRules takes an external QueryRules structure and overwrite one of the
// internal QueryRules as designated by ruleSource parameter
func (qri *QueryRuleInfo) SetRules(ruleSource string, newRules *QueryRules) error {
if newRules == nil {
newRules = NewQueryRules()
}
qri.mu.Lock()
defer qri.mu.Unlock()
if _, ok := qri.queryRulesMap[queryRuleSet]; ok {
qri.queryRulesMap[queryRuleSet] = newRules.Copy()
if _, ok := qri.queryRulesMap[ruleSource]; ok {
qri.queryRulesMap[ruleSource] = newRules.Copy()
return nil
}
return errors.New("QueryRules identifier " + queryRuleSet + " is not valid")
return errors.New("Rule source identifier " + ruleSource + " is not valid")
}
// GetRules returns the corresponding QueryRules as designated by queryRuleSet parameter
func (qri *QueryRuleInfo) GetRules(queryRuleSet string) (error, *QueryRules) {
// GetRules returns the corresponding QueryRules as designated by ruleSource parameter
func (qri *QueryRuleInfo) GetRules(ruleSource string) (error, *QueryRules) {
qri.mu.Lock()
defer qri.mu.Unlock()
if ruleset, ok := qri.queryRulesMap[queryRuleSet]; ok {
if ruleset, ok := qri.queryRulesMap[ruleSource]; ok {
return nil, ruleset.Copy()
}
return errors.New("QueryRules identifier " + queryRuleSet + " is not valid"), NewQueryRules()
return errors.New("Rule source identifier " + ruleSource + " is not valid"), NewQueryRules()
}
// filterByPlan creates a new QueryRules by prefiltering on all query rules that are contained in internal
// QueryRules structures, in other words, query rules from all predefined sources will be applied
func (qri *QueryRuleInfo) filterByPlan(query string, planid planbuilder.PlanType, tableName string) (newqrs *QueryRules) {
qri.mu.Lock()
defer qri.mu.Unlock()
newqrs = NewQueryRules()
newqrs.Append(qri.queryRulesMap[KeyrangeQueryRules].filterByPlan(query, planid, tableName))
newqrs.Append(qri.queryRulesMap[BlacklistQueryRules].filterByPlan(query, planid, tableName))
newqrs.Append(qri.queryRulesMap[CustomQueryRules].filterByPlan(query, planid, tableName))
for _, rules := range qri.queryRulesMap {
newqrs.Append(rules.filterByPlan(query, planid, tableName))
}
return newqrs
}

Просмотреть файл

@ -324,7 +324,7 @@ func (si *SchemaInfo) DropTable(tableName string) {
log.Infof("Table %s forgotten", tableName)
}
func (si *SchemaInfo) GetPlan(logStats *SQLQueryStats, sql string, queryRuleInfo *QueryRuleInfo) *ExecPlan {
func (si *SchemaInfo) GetPlan(logStats *SQLQueryStats, sql string) *ExecPlan {
// Fastpath if plan already exists.
if plan := si.getQuery(sql); plan != nil {
return plan
@ -350,7 +350,7 @@ func (si *SchemaInfo) GetPlan(logStats *SQLQueryStats, sql string, queryRuleInfo
panic(NewTabletError(FAIL, "%s", err))
}
plan := &ExecPlan{ExecPlan: splan, TableInfo: tableInfo}
plan.Rules = queryRuleInfo.filterByPlan(sql, plan.PlanId, plan.TableName)
plan.Rules = QueryRuleSources.filterByPlan(sql, plan.PlanId, plan.TableName)
plan.Authorized = tableacl.Authorized(plan.TableName, plan.PlanId.MinRole())
if plan.PlanId.IsSelect() {
if plan.FieldQuery == nil {
@ -376,7 +376,7 @@ func (si *SchemaInfo) GetPlan(logStats *SQLQueryStats, sql string, queryRuleInfo
// GetStreamPlan is similar to GetPlan, but doesn't use the cache
// and doesn't enforce a limit. It just returns the parsed query.
func (si *SchemaInfo) GetStreamPlan(sql string, queryRuleInfo *QueryRuleInfo) *ExecPlan {
func (si *SchemaInfo) GetStreamPlan(sql string) *ExecPlan {
var tableInfo *TableInfo
GetTable := func(tableName string) (table *schema.Table, ok bool) {
si.mu.Lock()
@ -392,7 +392,7 @@ func (si *SchemaInfo) GetStreamPlan(sql string, queryRuleInfo *QueryRuleInfo) *E
panic(NewTabletError(FAIL, "%s", err))
}
plan := &ExecPlan{ExecPlan: splan, TableInfo: tableInfo}
plan.Rules = queryRuleInfo.filterByPlan(sql, plan.PlanId, plan.TableName)
plan.Rules = QueryRuleSources.filterByPlan(sql, plan.PlanId, plan.TableName)
plan.Authorized = tableacl.Authorized(plan.TableName, plan.PlanId.MinRole())
return plan
}

Просмотреть файл

@ -100,28 +100,6 @@ func (sq *SqlQuery) setState(state int64) {
sq.state.Set(state)
}
// SetQueryRules sets one or more of the following QueryRule sets:
// 1. Custom rule
// 2. Tablet key range rule
// 3. Table blacklist rule
func (sq *SqlQuery) SetQueryRules(queryRuleSet string, newRules *QueryRules) error {
sq.mu.Lock()
defer sq.mu.Unlock()
err := sq.qe.queryRuleInfo.SetRules(queryRuleSet, newRules)
sq.qe.schemaInfo.ClearQueryPlanCache()
return err
}
// GetQueryRules returns one or more of the following QueryRule sets that is currently in use:
// 1. Custom rule
// 2. Tablet key range rule
// 3. Table blacklist rule
func (sq *SqlQuery) GetQueryRules(queryRuleSet string, newRules *QueryRules) (error, *QueryRules) {
sq.mu.Lock()
defer sq.mu.Unlock()
return sq.qe.queryRuleInfo.GetRules(queryRuleSet)
}
// allowQueries starts the query service.
// If the state is anything other than NOT_SERVING, it fails.
// If allowQuery succeeds, the resulting state is SERVING.
@ -333,7 +311,7 @@ func (sq *SqlQuery) Execute(context context.Context, query *proto.Query, reply *
query: query.Sql,
bindVars: query.BindVariables,
transactionID: query.TransactionId,
plan: sq.qe.schemaInfo.GetPlan(logStats, query.Sql, sq.qe.queryRuleInfo),
plan: sq.qe.schemaInfo.GetPlan(logStats, query.Sql),
RequestContext: RequestContext{
ctx: context,
logStats: logStats,
@ -370,7 +348,7 @@ func (sq *SqlQuery) StreamExecute(context context.Context, query *proto.Query, s
query: query.Sql,
bindVars: query.BindVariables,
transactionID: query.TransactionId,
plan: sq.qe.schemaInfo.GetStreamPlan(query.Sql, sq.qe.queryRuleInfo),
plan: sq.qe.schemaInfo.GetStreamPlan(query.Sql),
RequestContext: RequestContext{
ctx: context,
logStats: logStats,