Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/release-4.0' into release-4.0-…
Browse files Browse the repository at this point in the history
…57cd6947311c
  • Loading branch information
wshwsh12 committed Jan 26, 2021
2 parents b2e2381 + c7beb87 commit 5cb8470
Show file tree
Hide file tree
Showing 128 changed files with 2,624 additions and 600 deletions.
5 changes: 3 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -306,8 +306,9 @@ tools/bin/ineffassign:tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/ineffassign github.com/gordonklaus/ineffassign

tools/bin/errdoc-gen: go.mod
$(GO) build -o $@ github.com/pingcap/tiup/components/errdoc/errdoc-gen
tools/bin/errdoc-gen: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/errdoc-gen github.com/pingcap/errors/errdoc-gen

tools/bin/golangci-lint:
curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s -- -b ./tools/bin v1.21.0
Expand Down
360 changes: 265 additions & 95 deletions bindinfo/bind_test.go

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ func (br *BindRecord) shallowCopy() *BindRecord {
}

func (br *BindRecord) isSame(other *BindRecord) bool {
return br.OriginalSQL == other.OriginalSQL && br.Db == other.Db
return br.OriginalSQL == other.OriginalSQL
}

var statusIndex = map[string]int{
Expand Down
27 changes: 13 additions & 14 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,7 +373,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e
func (h *BindHandle) lockBindInfoTable() error {
// h.sctx already locked.
exec, _ := h.sctx.Context.(sqlexec.SQLExecutor)
_, err := exec.ExecuteInternal(context.TODO(), h.lockBindInfoSQL())
_, err := exec.ExecuteInternal(context.TODO(), h.LockBindInfoSQL())
return err
}

Expand Down Expand Up @@ -539,7 +539,7 @@ func (c cache) removeDeletedBindRecord(hash string, meta *BindRecord) {
func (c cache) setBindRecord(hash string, meta *BindRecord) {
metas := c[hash]
for i := range metas {
if metas[i].Db == meta.Db && metas[i].OriginalSQL == meta.OriginalSQL {
if metas[i].OriginalSQL == meta.OriginalSQL {
metas[i] = meta
return
}
Expand Down Expand Up @@ -568,7 +568,7 @@ func copyBindRecordUpdateMap(oldMap map[string]*bindRecordUpdate) map[string]*bi
func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindRecord {
bindRecords := c[hash]
for _, bindRecord := range bindRecords {
if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db {
if bindRecord.OriginalSQL == normdOrigSQL {
return bindRecord
}
}
Expand All @@ -577,9 +577,8 @@ func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindRecord {

func (h *BindHandle) deleteBindInfoSQL(normdOrigSQL, db, bindSQL string) string {
sql := fmt.Sprintf(
`DELETE FROM mysql.bind_info WHERE original_sql=%s AND LOWER(default_db)=%s`,
`DELETE FROM mysql.bind_info WHERE original_sql=%s`,
expression.Quote(normdOrigSQL),
expression.Quote(db),
)
if bindSQL == "" {
return sql
Expand All @@ -601,20 +600,19 @@ func (h *BindHandle) insertBindInfoSQL(orignalSQL string, db string, info Bindin
)
}

// lockBindInfoSQL simulates LOCK TABLE by updating a same row in each pessimistic transaction.
func (h *BindHandle) lockBindInfoSQL() string {
// LockBindInfoSQL simulates LOCK TABLE by updating a same row in each pessimistic transaction.
func (h *BindHandle) LockBindInfoSQL() string {
return fmt.Sprintf("UPDATE mysql.bind_info SET source=%s WHERE original_sql=%s",
expression.Quote(Builtin),
expression.Quote(BuiltinPseudoSQL4BindLock))
}

func (h *BindHandle) logicalDeleteBindInfoSQL(originalSQL, db string, updateTs types.Time, bindingSQL string) string {
updateTsStr := updateTs.String()
sql := fmt.Sprintf(`UPDATE mysql.bind_info SET status=%s,update_time=%s WHERE original_sql=%s and LOWER(default_db)=%s and update_time<%s`,
sql := fmt.Sprintf(`UPDATE mysql.bind_info SET status=%s,update_time=%s WHERE original_sql=%s and update_time<%s`,
expression.Quote(deleted),
expression.Quote(updateTsStr),
expression.Quote(originalSQL),
expression.Quote(db),
expression.Quote(updateTsStr))
if bindingSQL == "" {
return sql
Expand All @@ -635,12 +633,12 @@ func (h *BindHandle) CaptureBaselines() {
if insertStmt, ok := stmt.(*ast.InsertStmt); ok && insertStmt.Select == nil {
continue
}
normalizedSQL, digest := parser.NormalizeDigest(bindableStmt.Query)
dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema)
normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName))
if r := h.GetBindRecord(digest, normalizedSQL, dbName); r != nil && r.HasUsingBinding() {
continue
}
bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true)
bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName)
if bindSQL == "" {
continue
}
Expand All @@ -653,9 +651,9 @@ func (h *BindHandle) CaptureBaselines() {
Source: Capture,
}
// We don't need to pass the `sctx` because the BindSQL has been validated already.
err = h.AddBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}})
err = h.CreateBindRecord(nil, &BindRecord{OriginalSQL: normalizedSQL, Db: dbName, Bindings: []Binding{binding}})
if err != nil {
logutil.BgLogger().Debug("[sql-bind] add bind record failed in baseline capture", zap.String("SQL", bindableStmt.Query), zap.Error(err))
logutil.BgLogger().Debug("[sql-bind] create bind record failed in baseline capture", zap.String("SQL", bindableStmt.Query), zap.Error(err))
}
}
}
Expand All @@ -680,7 +678,7 @@ func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
}

// GenerateBindSQL generates binding sqls from stmt node and plan hints.
func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string, captured bool) string {
func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string, captured bool, defaultDB string) string {
// If would be nil for very simple cases such as point get, we do not need to evolve for them.
if planHint == "" {
return ""
Expand All @@ -699,6 +697,7 @@ func GenerateBindSQL(ctx context.Context, stmtNode ast.StmtNode, planHint string
hint.BindHint(stmtNode, &hint.HintsSet{})
var sb strings.Builder
restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb)
restoreCtx.DefaultDB = defaultDB
err := stmtNode.Restore(restoreCtx)
if err != nil {
logutil.Logger(ctx).Debug("[sql-bind] restore SQL failed when generating bind SQL", zap.Error(err))
Expand Down
2 changes: 1 addition & 1 deletion bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindRecord {
hash := parser.DigestNormalized(normdOrigSQL)
bindRecords := h.ch[hash]
for _, bindRecord := range bindRecords {
if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db {
if bindRecord.OriginalSQL == normdOrigSQL {
return bindRecord
}
}
Expand Down
15 changes: 15 additions & 0 deletions cmd/explaintest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -480,3 +480,18 @@ Projection_7 10000.00 root minus(Column#5, test.t.x)->Column#7
└─Sort_11 10000.00 root test.t.i:asc
└─TableReader_10 10000.00 root data:TableRangeScan_9
└─TableRangeScan_9 10000.00 cop[tikv] table:t range:[0,+inf], keep order:false, stats:pseudo
create table precise_types (
a BIGINT UNSIGNED NOT NULL,
b BIGINT NOT NULL,
c DECIMAL(21,1) NOT NULL,
d DOUBLE(21,1) NOT NULL
);
insert into precise_types values (
18446744073709551614,
-9223372036854775806,
99999999999999999999,
18446744073709551614
);
SELECT a, b, c, d FROM precise_types;
a b c d
18446744073709551614 -9223372036854775806 99999999999999999999.0 1.8446744073709552e19
15 changes: 15 additions & 0 deletions cmd/explaintest/t/select.test
Original file line number Diff line number Diff line change
Expand Up @@ -229,3 +229,18 @@ CREATE TABLE t (id int(10) unsigned NOT NULL AUTO_INCREMENT,
PRIMARY KEY (`id`)
);
explain select row_number() over( partition by i ) - x as rnk from t;

# for issue 21692
create table precise_types (
a BIGINT UNSIGNED NOT NULL,
b BIGINT NOT NULL,
c DECIMAL(21,1) NOT NULL,
d DOUBLE(21,1) NOT NULL
);
insert into precise_types values (
18446744073709551614,
-9223372036854775806,
99999999999999999999,
18446744073709551614
);
SELECT a, b, c, d FROM precise_types;
2 changes: 1 addition & 1 deletion ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -510,7 +510,7 @@ func (w *worker) doModifyColumn(
func checkAndApplyNewAutoRandomBits(job *model.Job, t *meta.Meta, tblInfo *model.TableInfo,
newCol *model.ColumnInfo, oldName *model.CIStr, newAutoRandBits uint64) error {
schemaID := job.SchemaID
newLayout := autoid.NewAutoRandomIDLayout(&newCol.FieldType, newAutoRandBits)
newLayout := autoid.NewShardIDLayout(&newCol.FieldType, newAutoRandBits)

// GenAutoRandomID first to prevent concurrent update.
_, err := t.GenAutoRandomID(schemaID, tblInfo.ID, 1)
Expand Down
46 changes: 46 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/israce"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -2302,3 +2303,48 @@ func (s *testIntegrationSuite3) TestIssue20741WithSetField(c *C) {
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[table:1364]Field 'cc' doesn't have a default value")
}

func (s *testIntegrationSuite7) TestDuplicateErrorMessage(c *C) {
defer collate.SetNewCollationEnabledForTest(false)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

type testdata struct {
types []string
values []string
}
tests := []testdata{
{[]string{"int"}, []string{"1"}},
{[]string{"datetime"}, []string{"'2020-01-01 00:00:00'"}},
{[]string{"varchar(10)"}, []string{"'qwe'"}},
{[]string{"enum('r', 'g', 'b')"}, []string{"'r'"}},
{[]string{"int", "datetime", "varchar(10)", "enum('r', 'g', 'b')"}, []string{"1", "'2020-01-01 00:00:00'", "'qwe'", "'r'"}},
}

for _, newCollate := range []bool{false, true} {
collate.SetNewCollationEnabledForTest(newCollate)
for _, t := range tests {
tk.MustExec("drop table if exists t;")
fields := make([]string, len(t.types))

for i, tp := range t.types {
fields[i] = fmt.Sprintf("a%d %s", i, tp)
}
tk.MustExec("create table t (id1 int, id2 varchar(10), " + strings.Join(fields, ",") + ",primary key(id1, id2)) " +
"collate utf8mb4_general_ci " +
"partition by range (id1) (partition p1 values less than (2), partition p2 values less than (maxvalue))")

vals := strings.Join(t.values, ",")
tk.MustExec(fmt.Sprintf("insert into t values (1, 'asd', %s), (1, 'dsa', %s)", vals, vals))
for i := range t.types {
fields[i] = fmt.Sprintf("a%d", i)
}
index := strings.Join(fields, ",")
for i, val := range t.values {
fields[i] = strings.Replace(val, "'", "", -1)
}
tk.MustGetErrMsg("alter table t add unique index t_idx(id1,"+index+")",
fmt.Sprintf("[kv:1062]Duplicate entry '1-%s' for key 't_idx'", strings.Join(fields, "-")))
}
}
}
Loading

0 comments on commit 5cb8470

Please sign in to comment.