Skip to content

Commit

Permalink
Merge remote-tracking branch 'pingcap/master' into is-part-expr-40931
Browse files Browse the repository at this point in the history
  • Loading branch information
mjonss committed Feb 10, 2023
2 parents e04176e + 278a9fe commit 584cb7a
Show file tree
Hide file tree
Showing 8 changed files with 56 additions and 9 deletions.
8 changes: 7 additions & 1 deletion ddl/dist_owner.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,10 @@ import (
)

// CheckBackfillJobFinishInterval is export for test.
var CheckBackfillJobFinishInterval = 300 * time.Millisecond
var (
CheckBackfillJobFinishInterval = 300 * time.Millisecond
telemetryDistReorgUsage = metrics.TelemetryDistReorgCnt
)

const (
distPhysicalTableConcurrency = 16
Expand All @@ -48,6 +51,9 @@ const (
func initDistReorg(reorgMeta *model.DDLReorgMeta) {
isDistReorg := variable.DDLEnableDistributeReorg.Load()
reorgMeta.IsDistReorg = isDistReorg
if isDistReorg {
metrics.TelemetryDistReorgCnt.Inc()
}
}

// BackfillJobRangeMeta is export for test.
Expand Down
8 changes: 4 additions & 4 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func buildIndexColumns(ctx sessionctx.Context, columns []*model.ColumnInfo, inde
// The multiple column index and the unique index in which the length sum exceeds the maximum size
// will return an error instead produce a warning.
if ctx == nil || ctx.GetSessionVars().StrictSQLMode || mysql.HasUniKeyFlag(col.GetFlag()) || len(indexPartSpecifications) > 1 {
return nil, false, dbterror.ErrTooLongKey.GenWithStackByArgs(maxIndexLength)
return nil, false, dbterror.ErrTooLongKey.GenWithStackByArgs(sumLength, maxIndexLength)
}
// truncate index length and produce warning message in non-restrict sql mode.
colLenPerUint, err := getIndexColumnLength(col, 1)
Expand All @@ -110,7 +110,7 @@ func buildIndexColumns(ctx sessionctx.Context, columns []*model.ColumnInfo, inde
}
indexColLen = maxIndexLength / colLenPerUint
// produce warning message
ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTooLongKey.FastGenByArgs(maxIndexLength))
ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTooLongKey.FastGenByArgs(sumLength, maxIndexLength))
}

idxParts = append(idxParts, &model.IndexColumn{
Expand Down Expand Up @@ -149,7 +149,7 @@ func checkIndexPrefixLength(columns []*model.ColumnInfo, idxColumns []*model.Ind
return err
}
if idxLen > config.GetGlobalConfig().MaxIndexLength {
return dbterror.ErrTooLongKey.GenWithStackByArgs(config.GetGlobalConfig().MaxIndexLength)
return dbterror.ErrTooLongKey.GenWithStackByArgs(idxLen, config.GetGlobalConfig().MaxIndexLength)
}
return nil
}
Expand Down Expand Up @@ -211,7 +211,7 @@ func checkIndexColumn(ctx sessionctx.Context, col *model.ColumnInfo, indexColumn
maxIndexLength := config.GetGlobalConfig().MaxIndexLength
if indexColumnLen > maxIndexLength && (ctx == nil || ctx.GetSessionVars().StrictSQLMode) {
// return error in strict sql mode
return dbterror.ErrTooLongKey.GenWithStackByArgs(maxIndexLength)
return dbterror.ErrTooLongKey.GenWithStackByArgs(indexColumnLen, maxIndexLength)
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ func TestChangeMaxIndexLength(t *testing.T) {
tk.MustExec("create table t (c1 varchar(3073), index(c1)) charset = ascii")
tk.MustExec(fmt.Sprintf("create table t1 (c1 varchar(%d), index(c1)) charset = ascii;", config.DefMaxOfMaxIndexLength))
err := tk.ExecToErr(fmt.Sprintf("create table t2 (c1 varchar(%d), index(c1)) charset = ascii;", config.DefMaxOfMaxIndexLength+1))
require.EqualError(t, err, "[ddl:1071]Specified key was too long; max key length is 12288 bytes")
require.EqualError(t, err, "[ddl:1071]Specified key was too long (12289 bytes); max key length is 12288 bytes")
}

func TestCreateTableWithLike(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrMultiplePriKey: mysql.Message("Multiple primary key defined", nil),
ErrTooManyKeys: mysql.Message("Too many keys specified; max %d keys allowed", nil),
ErrTooManyKeyParts: mysql.Message("Too many key parts specified; max %d parts allowed", nil),
ErrTooLongKey: mysql.Message("Specified key was too long; max key length is %d bytes", nil),
ErrTooLongKey: mysql.Message("Specified key was too long (%d bytes); max key length is %d bytes", nil),
ErrKeyColumnDoesNotExits: mysql.Message("Key column '%-.192s' doesn't exist in table", nil),
ErrBlobUsedAsKey: mysql.Message("BLOB column '%-.192s' can't be used in key specification with the used table type", nil),
ErrTooBigFieldlength: mysql.Message("Column length too big for column '%-.192s' (max = %d); use BLOB or TEXT instead", nil),
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -668,7 +668,7 @@ Too many keys specified; max %d keys allowed

["ddl:1071"]
error = '''
Specified key was too long; max key length is %d bytes
Specified key was too long (%d bytes); max key length is %d bytes
'''

["ddl:1072"]
Expand Down
10 changes: 10 additions & 0 deletions metrics/telemetry.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,13 @@ var (
Name: "compact_partition_usage",
Help: "Counter of compact table partition",
})
TelemetryDistReorgCnt = prometheus.NewCounter(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "telemetry",
Name: "distributed_reorg_count",
Help: "Counter of usage of distributed reorg DDL tasks count",
})
TelemetryStoreBatchedQueryCnt = prometheus.NewCounter(
prometheus.CounterOpts{
Namespace: "tidb",
Expand Down Expand Up @@ -414,13 +421,15 @@ type DDLUsageCounter struct {
AddIndexIngestUsed int64 `json:"add_index_ingest_used"`
MetadataLockUsed bool `json:"metadata_lock_used"`
FlashbackClusterUsed int64 `json:"flashback_cluster_used"`
DistReorgUsed int64 `json:"dist_reorg_used"`
}

// Sub returns the difference of two counters.
func (a DDLUsageCounter) Sub(rhs DDLUsageCounter) DDLUsageCounter {
return DDLUsageCounter{
AddIndexIngestUsed: a.AddIndexIngestUsed - rhs.AddIndexIngestUsed,
FlashbackClusterUsed: a.FlashbackClusterUsed - rhs.FlashbackClusterUsed,
DistReorgUsed: a.DistReorgUsed - rhs.DistReorgUsed,
}
}

Expand All @@ -429,6 +438,7 @@ func GetDDLUsageCounter() DDLUsageCounter {
return DDLUsageCounter{
AddIndexIngestUsed: readCounter(TelemetryAddIndexIngestCnt),
FlashbackClusterUsed: readCounter(TelemetryFlashbackClusterCnt),
DistReorgUsed: readCounter(TelemetryDistReorgCnt),
}
}

Expand Down
2 changes: 1 addition & 1 deletion parser/mysql/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ var MySQLErrName = map[uint16]*ErrMessage{
ErrMultiplePriKey: Message("Multiple primary key defined", nil),
ErrTooManyKeys: Message("Too many keys specified; max %d keys allowed", nil),
ErrTooManyKeyParts: Message("Too many key parts specified; max %d parts allowed", nil),
ErrTooLongKey: Message("Specified key was too long; max key length is %d bytes", nil),
ErrTooLongKey: Message("Specified key was too long (%d bytes); max key length is %d bytes", nil),
ErrKeyColumnDoesNotExits: Message("Key column '%-.192s' doesn't exist in table", nil),
ErrBlobUsedAsKey: Message("BLOB column '%-.192s' can't be used in key specification with the used table type", nil),
ErrTooBigFieldlength: Message("Column length too big for column '%-.192s' (max = %d); use BLOB or TEXT instead", nil),
Expand Down
31 changes: 31 additions & 0 deletions telemetry/data_feature_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -585,6 +585,37 @@ func TestAddIndexAccelerationAndMDL(t *testing.T) {
require.Equal(t, true, usage.DDLUsageCounter.MetadataLockUsed)
}

func TestDistReorgUsage(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
usage, err := telemetry.GetFeatureUsage(tk.Session())
require.NoError(t, err)
initCount := usage.DDLUsageCounter.DistReorgUsed

tk.MustExec("set @@global.tidb_ddl_distribute_reorg = off")
allow := variable.DDLEnableDistributeReorg.Load()
require.Equal(t, false, allow)
tk.MustExec("use test")
tk.MustExec("drop table if exists tele_t")
tk.MustExec("create table tele_t(id int, b int)")
tk.MustExec("insert into tele_t values(1,1),(2,2);")
tk.MustExec("alter table tele_t add index idx_org(b)")
usage, err = telemetry.GetFeatureUsage(tk.Session())
require.NoError(t, err)
require.Equal(t, initCount, usage.DDLUsageCounter.DistReorgUsed)

tk.MustExec("set @@global.tidb_ddl_distribute_reorg = on")
allow = variable.DDLEnableDistributeReorg.Load()
require.Equal(t, true, allow)
usage, err = telemetry.GetFeatureUsage(tk.Session())
require.NoError(t, err)
require.Equal(t, initCount, usage.DDLUsageCounter.DistReorgUsed)
tk.MustExec("alter table tele_t add index idx_new(b)")
usage, err = telemetry.GetFeatureUsage(tk.Session())
require.NoError(t, err)
require.Equal(t, initCount+1, usage.DDLUsageCounter.DistReorgUsed)
}

func TestGlobalMemoryControl(t *testing.T) {
store := testkit.CreateMockStore(t)

Expand Down

0 comments on commit 584cb7a

Please sign in to comment.