Skip to content

Commit

Permalink
add interface
Browse files Browse the repository at this point in the history
  • Loading branch information
Defined2014 committed Dec 20, 2021
1 parent d208b62 commit a5b47cf
Show file tree
Hide file tree
Showing 31 changed files with 102 additions and 73 deletions.
6 changes: 3 additions & 3 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) {
}

exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source
stmt, err := exec.ParseWithParamsInternal(context.TODO(), `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source
FROM mysql.bind_info WHERE update_time > %? ORDER BY update_time, create_time`, updateTime)
if err != nil {
return err
Expand Down Expand Up @@ -697,7 +697,7 @@ func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) {
tables: make(map[stmtctx.TableEntry]struct{}),
}
exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`)
stmt, err := exec.ParseWithParamsInternal(context.TODO(), `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`)
if err != nil {
logutil.BgLogger().Warn("[sql-bind] failed to parse query for mysql.capture_plan_baselines_blacklist load", zap.Error(err))
return
Expand Down Expand Up @@ -923,7 +923,7 @@ func (h *BindHandle) SaveEvolveTasksToStore() {
}

func getEvolveParameters(ctx sessionctx.Context) (time.Duration, time.Time, time.Time, error) {
stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(
stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParamsInternal(
context.TODO(),
"SELECT variable_name, variable_value FROM mysql.global_variables WHERE variable_name IN (%?, %?, %?)",
variable.TiDBEvolvePlanTaskMaxTime,
Expand Down
4 changes: 2 additions & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -996,7 +996,7 @@ func (w *worker) doModifyColumnTypeWithData(
}
defer w.sessPool.put(ctx)

stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(context.Background(), valStr)
stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParamsInternal(context.Background(), valStr)
if err != nil {
job.State = model.JobStateCancelled
failpoint.Return(ver, err)
Expand Down Expand Up @@ -1703,7 +1703,7 @@ func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTrunc
}
}
buf.WriteString(" limit 1")
stmt, err := sctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(ctx, buf.String(), paramsList...)
stmt, err := sctx.(sqlexec.RestrictedSQLExecutor).ParseWithParamsInternal(ctx, buf.String(), paramsList...)
if err != nil {
return errors.Trace(err)
}
Expand Down
4 changes: 2 additions & 2 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -1551,7 +1551,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, inde
}
defer w.sessPool.put(ctx)

stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(w.ddlJobCtx, sql, paramList...)
stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParamsInternal(w.ddlJobCtx, sql, paramList...)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -1569,7 +1569,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, inde
func buildCheckSQLForRangeExprPartition(pi *model.PartitionInfo, index int, schemaName, tableName model.CIStr) (string, []interface{}) {
var buf strings.Builder
paramList := make([]interface{}, 0, 4)
// Since the pi.Expr string may contain the identifier, which couldn't be escaped in our ParseWithParams(...)
// Since the pi.Expr string may contain the identifier, which couldn't be escaped in our ParseWithParamsInternal(...)
// So we write it to the origin sql string here.
if index == 0 {
buf.WriteString("select 1 from %n.%n where ")
Expand Down
2 changes: 1 addition & 1 deletion ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -341,7 +341,7 @@ func getTableTotalCount(w *worker, tblInfo *model.TableInfo) int64 {
return statistics.PseudoRowCount
}
sql := "select table_rows from information_schema.tables where tidb_table_id=%?;"
stmt, err := executor.ParseWithParams(w.ddlJobCtx, sql, tblInfo.ID)
stmt, err := executor.ParseWithParamsInternal(w.ddlJobCtx, sql, tblInfo.ID)
if err != nil {
return statistics.PseudoRowCount
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ func LoadGlobalVars(ctx context.Context, sctx sessionctx.Context, varNames []str
paramNames = append(paramNames, name)
}
buf.WriteString(")")
stmt, err := e.ParseWithParams(ctx, buf.String(), paramNames...)
stmt, err := e.ParseWithParamsInternal(ctx, buf.String(), paramNames...)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion domain/sysvar_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ func (do *Domain) fetchTableValues(ctx sessionctx.Context) (map[string]string, e
tableContents := make(map[string]string)
// Copy all variables from the table to tableContents
exec := ctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.Background(), `SELECT variable_name, variable_value FROM mysql.global_variables`)
stmt, err := exec.ParseWithParamsInternal(context.Background(), `SELECT variable_name, variable_value FROM mysql.global_variables`)
if err != nil {
return tableContents, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -1570,7 +1570,7 @@ type AnalyzeFastExec struct {
func (e *AnalyzeFastExec) calculateEstimateSampleStep() (err error) {
exec := e.ctx.(sqlexec.RestrictedSQLExecutor)
var stmt ast.StmtNode
stmt, err = exec.ParseWithParams(context.TODO(), "select flag from mysql.stats_histograms where table_id = %?", e.tableID.GetStatisticsID())
stmt, err = exec.ParseWithParamsInternal(context.TODO(), "select flag from mysql.stats_histograms where table_id = %?", e.tableID.GetStatisticsID())
if err != nil {
return
}
Expand Down
2 changes: 1 addition & 1 deletion executor/brie.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ func (gs *tidbGlueSession) CreateSession(store kv.Storage) (glue.Session, error)
// These queries execute without privilege checking, since the calling statements
// such as BACKUP and RESTORE have already been privilege checked.
func (gs *tidbGlueSession) Execute(ctx context.Context, sql string) error {
stmt, err := gs.se.(sqlexec.RestrictedSQLExecutor).ParseWithParams(ctx, sql)
stmt, err := gs.se.(sqlexec.RestrictedSQLExecutor).ParseWithParamsInternal(ctx, sql)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -506,7 +506,7 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx
zap.String("table", fullti.Name.O),
)
exec := e.ctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), "admin check table %n.%n", fullti.Schema.O, fullti.Name.O)
stmt, err := exec.ParseWithParamsInternal(context.TODO(), "admin check table %n.%n", fullti.Schema.O, fullti.Name.O)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex

func getRowCountAllTable(ctx context.Context, sctx sessionctx.Context) (map[int64]uint64, error) {
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "select table_id, count from mysql.stats_meta")
stmt, err := exec.ParseWithParamsInternal(ctx, "select table_id, count from mysql.stats_meta")
if err != nil {
return nil, err
}
Expand All @@ -215,7 +215,7 @@ type tableHistID struct {

func getColLengthAllTables(ctx context.Context, sctx sessionctx.Context) (map[tableHistID]uint64, error) {
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0")
stmt, err := exec.ParseWithParamsInternal(ctx, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0")
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/inspection_profile.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ func (n *metricNode) getLabelValue(label string) *metricValue {

func (n *metricNode) queryRowsByLabel(pb *profileBuilder, query string, handleRowFn func(label string, v float64)) error {
exec := pb.sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), query)
stmt, err := exec.ParseWithParamsInternal(context.TODO(), query)
if err != nil {
return err
}
Expand Down
30 changes: 15 additions & 15 deletions executor/inspection_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ func (e *inspectionResultRetriever) retrieve(ctx context.Context, sctx sessionct
e.statusToInstanceAddress = make(map[string]string)
var rows []chunk.Row
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "select instance,status_address from information_schema.cluster_info;")
stmt, err := exec.ParseWithParamsInternal(ctx, "select instance,status_address from information_schema.cluster_info;")
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -251,7 +251,7 @@ func (configInspection) inspectDiffConfig(ctx context.Context, sctx sessionctx.C
}
var rows []chunk.Row
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "select type, `key`, count(distinct value) as c from information_schema.cluster_config where `key` not in (%?) group by type, `key` having c > 1", ignoreConfigKey)
stmt, err := exec.ParseWithParamsInternal(ctx, "select type, `key`, count(distinct value) as c from information_schema.cluster_config where `key` not in (%?) group by type, `key` having c > 1", ignoreConfigKey)
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand All @@ -261,7 +261,7 @@ func (configInspection) inspectDiffConfig(ctx context.Context, sctx sessionctx.C

generateDetail := func(tp, item string) string {
var rows []chunk.Row
stmt, err := exec.ParseWithParams(ctx, "select value, instance from information_schema.cluster_config where type=%? and `key`=%?;", tp, item)
stmt, err := exec.ParseWithParamsInternal(ctx, "select value, instance from information_schema.cluster_config where type=%? and `key`=%?;", tp, item)
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -347,7 +347,7 @@ func (c configInspection) inspectCheckConfig(ctx context.Context, sctx sessionct
}
sql.Reset()
fmt.Fprintf(sql, "select type,instance,value from information_schema.%s where %s", cas.table, cas.cond)
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -378,7 +378,7 @@ func (c configInspection) checkTiKVBlockCacheSizeConfig(ctx context.Context, sct
}
var rows []chunk.Row
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "select instance,value from information_schema.cluster_config where type='tikv' and `key` = 'storage.block-cache.capacity'")
stmt, err := exec.ParseWithParamsInternal(ctx, "select instance,value from information_schema.cluster_config where type='tikv' and `key` = 'storage.block-cache.capacity'")
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand All @@ -405,7 +405,7 @@ func (c configInspection) checkTiKVBlockCacheSizeConfig(ctx context.Context, sct
ipToCount[ip]++
}

stmt, err = exec.ParseWithParams(ctx, "select instance, value from metrics_schema.node_total_memory where time=now()")
stmt, err = exec.ParseWithParamsInternal(ctx, "select instance, value from metrics_schema.node_total_memory where time=now()")
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -473,7 +473,7 @@ func (versionInspection) inspect(ctx context.Context, sctx sessionctx.Context, f
exec := sctx.(sqlexec.RestrictedSQLExecutor)
var rows []chunk.Row
// check the configuration consistent
stmt, err := exec.ParseWithParams(ctx, "select type, count(distinct git_hash) as c from information_schema.cluster_info group by type having c > 1;")
stmt, err := exec.ParseWithParamsInternal(ctx, "select type, count(distinct git_hash) as c from information_schema.cluster_info group by type having c > 1;")
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -643,7 +643,7 @@ func (criticalErrorInspection) inspectError(ctx context.Context, sctx sessionctx
sql.Reset()
fmt.Fprintf(sql, "select `%[1]s`,sum(value) as total from `%[2]s`.`%[3]s` %[4]s group by `%[1]s` having total>=1.0",
strings.Join(def.Labels, "`,`"), util.MetricSchemaName.L, rule.tbl, condition)
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -698,7 +698,7 @@ func (criticalErrorInspection) inspectForServerDown(ctx context.Context, sctx se
(select instance,job from metrics_schema.up %[1]s group by instance,job having max(value)-min(value)>0) as t1 join
(select instance,min(time) as min_time from metrics_schema.up %[1]s and value=0 group by instance,job) as t2 on t1.instance=t2.instance order by job`, condition)
var rows []chunk.Row
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -726,7 +726,7 @@ func (criticalErrorInspection) inspectForServerDown(ctx context.Context, sctx se
// Check from log.
sql.Reset()
fmt.Fprintf(sql, "select type,instance,time from information_schema.cluster_log %s and level = 'info' and message like '%%Welcome to'", condition)
stmt, err = exec.ParseWithParams(ctx, sql.String())
stmt, err = exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -863,7 +863,7 @@ func (thresholdCheckInspection) inspectThreshold1(ctx context.Context, sctx sess
(select instance, max(value) as cpu from metrics_schema.tikv_thread_cpu %[3]s and name like '%[1]s' group by instance) as t1
where t1.cpu > %[2]f;`, rule.component, rule.threshold, condition)
}
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -1036,7 +1036,7 @@ func (thresholdCheckInspection) inspectThreshold2(ctx context.Context, sctx sess
} else {
fmt.Fprintf(sql, "select instance, max(value)/%.0f as max_value from metrics_schema.%s %s group by instance having max_value > %f;", rule.factor, rule.tbl, cond, rule.threshold)
}
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down Expand Up @@ -1222,7 +1222,7 @@ func checkRules(ctx context.Context, sctx sessionctx.Context, filter inspectionF
continue
}
sql := rule.genSQL(filter.timeRange)
stmt, err := exec.ParseWithParams(ctx, sql)
stmt, err := exec.ParseWithParamsInternal(ctx, sql)
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand All @@ -1245,7 +1245,7 @@ func (c thresholdCheckInspection) inspectForLeaderDrop(ctx context.Context, sctx
exec := sctx.(sqlexec.RestrictedSQLExecutor)

var rows []chunk.Row
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
rows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand All @@ -1259,7 +1259,7 @@ func (c thresholdCheckInspection) inspectForLeaderDrop(ctx context.Context, sctx
sql.Reset()
fmt.Fprintf(sql, `select time, value from metrics_schema.pd_scheduler_store_status %s and type='leader_count' and address = '%s' order by time`, condition, address)
var subRows []chunk.Row
stmt, err := exec.ParseWithParams(ctx, sql.String())
stmt, err := exec.ParseWithParamsInternal(ctx, sql.String())
if err == nil {
subRows, _, err = exec.ExecRestrictedStmt(ctx, stmt)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/inspection_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -460,7 +460,7 @@ func (e *inspectionSummaryRetriever) retrieve(ctx context.Context, sctx sessionc
util.MetricSchemaName.L, name, cond)
}
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, sql)
stmt, err := exec.ParseWithParamsInternal(ctx, sql)
if err != nil {
return nil, errors.Errorf("execute '%s' failed: %v", sql, err)
}
Expand Down
4 changes: 2 additions & 2 deletions executor/metrics_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ func (e *MetricsSummaryRetriever) retrieve(ctx context.Context, sctx sessionctx.
}

exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, sql)
stmt, err := exec.ParseWithParamsInternal(ctx, sql)
if err != nil {
return nil, errors.Errorf("execute '%s' failed: %v", sql, err)
}
Expand Down Expand Up @@ -318,7 +318,7 @@ func (e *MetricsSummaryByLabelRetriever) retrieve(ctx context.Context, sctx sess
util.MetricSchemaName.L, name, cond)
}
exec := sctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, sql)
stmt, err := exec.ParseWithParamsInternal(ctx, sql)
if err != nil {
return nil, errors.Errorf("execute '%s' failed: %v", sql, err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/opt_rule_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func (e *ReloadOptRuleBlacklistExec) Next(ctx context.Context, _ *chunk.Chunk) e
// LoadOptRuleBlacklist loads the latest data from table mysql.opt_rule_blacklist.
func LoadOptRuleBlacklist(ctx sessionctx.Context) (err error) {
exec := ctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), "select HIGH_PRIORITY name from mysql.opt_rule_blacklist")
stmt, err := exec.ParseWithParamsInternal(context.TODO(), "select HIGH_PRIORITY name from mysql.opt_rule_blacklist")
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/reload_expr_pushdown_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func (e *ReloadExprPushdownBlacklistExec) Next(ctx context.Context, _ *chunk.Chu
// LoadExprPushdownBlacklist loads the latest data from table mysql.expr_pushdown_blacklist.
func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) {
exec := ctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(context.TODO(), "select HIGH_PRIORITY name, store_type from mysql.expr_pushdown_blacklist")
stmt, err := exec.ParseWithParamsInternal(context.TODO(), "select HIGH_PRIORITY name, store_type from mysql.expr_pushdown_blacklist")
if err != nil {
return err
}
Expand Down
8 changes: 4 additions & 4 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ func (e *ShowExec) fetchShowBind() error {
func (e *ShowExec) fetchShowEngines(ctx context.Context) error {
exec := e.ctx.(sqlexec.RestrictedSQLExecutor)

stmt, err := exec.ParseWithParams(ctx, `SELECT * FROM information_schema.engines`)
stmt, err := exec.ParseWithParamsInternal(ctx, `SELECT * FROM information_schema.engines`)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -473,7 +473,7 @@ func (e *ShowExec) fetchShowTableStatus(ctx context.Context) error {

exec := e.ctx.(sqlexec.RestrictedSQLExecutor)

stmt, err := exec.ParseWithParams(ctx, `SELECT
stmt, err := exec.ParseWithParamsInternal(ctx, `SELECT
table_name, engine, version, row_format, table_rows,
avg_row_length, data_length, max_data_length, index_length,
data_free, auto_increment, create_time, update_time, check_time,
Expand Down Expand Up @@ -1433,7 +1433,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error {

exec := e.ctx.(sqlexec.RestrictedSQLExecutor)

stmt, err := exec.ParseWithParams(ctx, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, strings.ToLower(hostName))
stmt, err := exec.ParseWithParamsInternal(ctx, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, strings.ToLower(hostName))
if err != nil {
return errors.Trace(err)
}
Expand All @@ -1453,7 +1453,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error {
authplugin = rows[0].GetString(0)
}

stmt, err = exec.ParseWithParams(ctx, `SELECT Priv FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.GlobalPrivTable, userName, hostName)
stmt, err = exec.ParseWithParamsInternal(ctx, `SELECT Priv FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.GlobalPrivTable, userName, hostName)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/show_placement.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func (b *showPlacementLabelsResultBuilder) sortMapKeys(m map[string]interface{})

func (e *ShowExec) fetchShowPlacementLabels(ctx context.Context) error {
exec := e.ctx.(sqlexec.RestrictedSQLExecutor)
stmt, err := exec.ParseWithParams(ctx, "SELECT DISTINCT LABEL FROM %n.%n", "INFORMATION_SCHEMA", infoschema.TableTiKVStoreStatus)
stmt, err := exec.ParseWithParamsInternal(ctx, "SELECT DISTINCT LABEL FROM %n.%n", "INFORMATION_SCHEMA", infoschema.TableTiKVStoreStatus)
if err != nil {
return errors.Trace(err)
}
Expand Down
Loading

0 comments on commit a5b47cf

Please sign in to comment.