Skip to content

Commit

Permalink
*: Enable tiflash_fastscan variable and remove fast mode alter statme…
Browse files Browse the repository at this point in the history
  • Loading branch information
hongyunyan authored Aug 29, 2022
1 parent 7425d2f commit ccd4f34
Show file tree
Hide file tree
Showing 31 changed files with 9,816 additions and 10,036 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -2810,8 +2810,8 @@ def go_deps():
name = "com_github_pingcap_tipb",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/tipb",
sum = "h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4=",
version = "v0.0.0-20220718022156-3e2483c20a9e",
sum = "h1:kWYridgsn8xSKYJ2EkXp7uj5HwJnG5snpY3XP8oYmPU=",
version = "v0.0.0-20220824081009-0714a57aff1d",
)
go_repository(
name = "com_github_pkg_browser",
Expand Down
38 changes: 0 additions & 38 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -3237,8 +3237,6 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
}
case ast.AlterTableSetTiFlashReplica:
err = d.AlterTableSetTiFlashReplica(sctx, ident, spec.TiFlashReplica)
case ast.AlterTableSetTiFlashMode:
err = d.AlterTableSetTiFlashMode(sctx, ident, spec.TiFlashMode)
case ast.AlterTableOrderByColumns:
err = d.OrderByColumns(sctx, ident)
case ast.AlterTableIndexInvisible:
Expand Down Expand Up @@ -5051,42 +5049,6 @@ func isTableTiFlashSupported(schema *model.DBInfo, tb table.Table) error {
return nil
}

func (d *ddl) AlterTableSetTiFlashMode(ctx sessionctx.Context, ident ast.Ident, mode model.TiFlashMode) error {
schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident)
if err != nil {
return errors.Trace(err)
}

if mode != model.TiFlashModeNormal && mode != model.TiFlashModeFast {
return fmt.Errorf("unsupported TiFlash mode %s", mode)
}

err = isTableTiFlashSupported(schema, tb)
if err != nil {
return errors.Trace(err)
}

// Prompt warning when there is no TiFlash replica, as TiFlash mode will
// only take effect when executing in TiFlash.
tbReplicaInfo := tb.Meta().TiFlashReplica
if tbReplicaInfo == nil || tbReplicaInfo.Count == 0 {
ctx.GetSessionVars().StmtCtx.AppendNote(dbterror.ErrAlterTiFlashModeForTableWithoutTiFlashReplica)
}

job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
SchemaName: schema.Name.L,
TableName: tb.Meta().Name.L,
Type: model.ActionSetTiFlashMode,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{mode},
}
err = d.DoDDLJob(ctx, job)
err = d.callHookOnChanged(job, err)
return errors.Trace(err)
}

func checkTiFlashReplicaCount(ctx sessionctx.Context, replicaCount uint64) error {
// Check the tiflash replica count should be less than the total tiflash stores.
tiflashStoreCnt, err := infoschema.GetTiFlashStoreCount(ctx)
Expand Down
50 changes: 0 additions & 50 deletions ddl/ddl_tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -383,56 +383,6 @@ func TestTiFlashReplicaAvailable(t *testing.T) {
require.False(t, ok)
}

// set TiFlash mode shall be eventually available.
func TestSetTiFlashModeAvailable(t *testing.T) {
s, teardown := createTiFlashContext(t)
defer teardown()
tk := testkit.NewTestKit(t, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists ddltiflash")
tk.MustExec("create table ddltiflash(z int)")
tk.MustExec("alter table ddltiflash set tiflash replica 1")
tk.MustExec("alter table ddltiflash set tiflash mode fast")
time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3)
tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash"))
require.NoError(t, err)
tiflashmode := tb.Meta().TiFlashMode
require.NotNil(t, tiflashmode)
require.Equal(t, tiflashmode, model.TiFlashModeFast)

tk.MustExec("alter table ddltiflash set tiflash mode normal")
time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3)
tb, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash"))
require.NoError(t, err)
tiflashmode = tb.Meta().TiFlashMode
require.NotNil(t, tiflashmode)
require.Equal(t, tiflashmode, model.TiFlashModeNormal)

// check the warning when set tiflash mode on the table whose tiflash replica is nil
tk.MustExec("use test")
tk.MustExec("drop table if exists ddltiflash")
tk.MustExec("create table ddltiflash(z int)")
tk.MustExec("alter table ddltiflash set tiflash mode fast")
tk.MustQuery("show warnings").Check(
testkit.Rows("Note 0 TiFlash mode will take effect after at least one TiFlash replica is set for the table"))
}

// check for the condition that unsupport set tiflash mode
func TestSetTiFlashModeUnsupported(t *testing.T) {
s, teardown := createTiFlashContext(t)
defer teardown()
tk := testkit.NewTestKit(t, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists ddltiflash")
tk.MustExec("create temporary table ddltiflash(z int)")
// unsupport for temporary table
tk.MustGetErrMsg("alter table ddltiflash set tiflash mode fast", "[ddl:8200]TiDB doesn't support ALTER TABLE for local temporary table")
// unsupport for system table
tk.MustGetErrMsg("alter table information_schema.tiflash_replica set tiflash mode fast", "[ddl:8200]Unsupported ALTER TiFlash settings for system table and memory table")
}

// Truncate partition shall not block.
func TestTiFlashTruncatePartition(t *testing.T) {
s, teardown := createTiFlashContext(t)
Expand Down
2 changes: 0 additions & 2 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -1191,8 +1191,6 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onUnlockTables(d, t, job)
case model.ActionSetTiFlashReplica:
ver, err = w.onSetTableFlashReplica(d, t, job)
case model.ActionSetTiFlashMode:
ver, err = w.onSetTiFlashMode(d, t, job)
case model.ActionUpdateTiFlashReplicaStatus:
ver, err = onUpdateFlashReplicaStatus(d, t, job)
case model.ActionCreateSequence:
Expand Down
22 changes: 0 additions & 22 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1155,28 +1155,6 @@ func (w *worker) onSetTableFlashReplica(d *ddlCtx, t *meta.Meta, job *model.Job)
return ver, nil
}

func (w *worker) onSetTiFlashMode(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
var mode model.TiFlashMode
if err := job.DecodeArgs(&mode); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}

tblInfo.TiFlashMode = mode

ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err)
}
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
return ver, nil
}

func (w *worker) checkTiFlashReplicaCount(replicaCount uint64) error {
ctx, err := w.sessPool.get()
if err != nil {
Expand Down
1 change: 0 additions & 1 deletion ddl/tiflash_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,6 @@ func TestSetTableFlashReplica(t *testing.T) {
require.NotNil(t, tbl.Meta().TiFlashReplica)
require.Equal(t, uint64(2), tbl.Meta().TiFlashReplica.Count)
require.Equal(t, "a,b", strings.Join(tbl.Meta().TiFlashReplica.LocationLabels, ","))
require.Equal(t, model.TiFlashModeNormal, tbl.Meta().TiFlashMode) // check the default tiflash mode

tk.MustExec("alter table t_flash set tiflash replica 0")
tbl = external.GetTableByName(t, tk, "test", "t_flash")
Expand Down
2 changes: 1 addition & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -333,7 +333,7 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64

func canSkipSchemaCheckerDDL(tp model.ActionType) bool {
switch tp {
case model.ActionUpdateTiFlashReplicaStatus, model.ActionSetTiFlashReplica, model.ActionSetTiFlashMode:
case model.ActionUpdateTiFlashReplicaStatus, model.ActionSetTiFlashReplica:
return true
}
return false
Expand Down
1 change: 0 additions & 1 deletion executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -2216,7 +2216,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s
strings.Join(tbl.TiFlashReplica.LocationLabels, ","), // LOCATION_LABELS
tbl.TiFlashReplica.Available, // AVAILABLE
progress, // PROGRESS
tbl.TiFlashMode.String(), // TABLE_MPDE
)
rows = append(rows, record)
}
Expand Down
6 changes: 2 additions & 4 deletions executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -629,13 +629,11 @@ func TestForTableTiFlashReplica(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int, index idx(a))")
tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';")
tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS,TABLE_MODE from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 0 0 NORMAL"))
tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 0 0"))
tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tbl.Meta().TiFlashReplica.Available = true
tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS,TABLE_MODE from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 1 1 NORMAL"))
tbl.Meta().TiFlashMode = model.TiFlashModeFast
tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS,TABLE_MODE from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 1 1 FAST"))
tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 1 1"))
}

func TestSequences(t *testing.T) {
Expand Down
19 changes: 19 additions & 0 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1922,3 +1922,22 @@ func TestTiFlashFineGrainedShuffle(t *testing.T) {
tk.MustExec("set global tiflash_fine_grained_shuffle_stream_count = -1")
tk.MustExec("set global tiflash_fine_grained_shuffle_batch_size = 8192")
}

func TestSetTiFlashFastScanVariable(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int);")
tk.MustExec("insert into t values(1);")

// check the default tiflash read mode
tk.MustQuery("select @@session.tiflash_fastscan").Check(testkit.Rows("0"))
tk.MustQuery("select @@global.tiflash_fastscan").Check(testkit.Rows("0"))

tk.MustExec("set @@tiflash_fastscan=ON;")
tk.MustQuery("select @@session.tiflash_fastscan").Check(testkit.Rows("1"))

tk.MustExec("set GLOBAL tiflash_fastscan=OFF;")
tk.MustQuery("select @@global.tiflash_fastscan").Check(testkit.Rows("0"))
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ require (
github.com/pingcap/log v1.1.0
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d
github.com/prometheus/client_golang v1.12.2
github.com/prometheus/client_model v0.2.0
github.com/prometheus/common v0.32.1
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -767,8 +767,8 @@ github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8=
github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM=
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops=
github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4=
github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d h1:kWYridgsn8xSKYJ2EkXp7uj5HwJnG5snpY3XP8oYmPU=
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
Expand Down
1 change: 0 additions & 1 deletion infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1126,7 +1126,6 @@ var tableTableTiFlashReplicaCols = []columnInfo{
{name: "LOCATION_LABELS", tp: mysql.TypeVarchar, size: 64},
{name: "AVAILABLE", tp: mysql.TypeTiny, size: 1},
{name: "PROGRESS", tp: mysql.TypeDouble, size: 22},
{name: "TABLE_MODE", tp: mysql.TypeVarchar, size: 64},
}

var tableInspectionResultCols = []columnInfo{
Expand Down
6 changes: 0 additions & 6 deletions parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -2587,8 +2587,6 @@ const (
AlterTableCache
AlterTableNoCache
AlterTableStatsOptions
// AlterTableSetTiFlashMode uses to alter the table mode of TiFlash.
AlterTableSetTiFlashMode
AlterTableDropFirstPartition
AlterTableAddLastPartition
AlterTableReorganizeLastPartition
Expand Down Expand Up @@ -2689,7 +2687,6 @@ type AlterTableSpec struct {
Num uint64
Visibility IndexVisibility
TiFlashReplica *TiFlashReplicaSpec
TiFlashMode model.TiFlashMode
Writeable bool
Statistics *StatisticsSpec
AttributesSpec *AttributesSpec
Expand Down Expand Up @@ -2752,9 +2749,6 @@ func (n *AlterTableSpec) Restore(ctx *format.RestoreCtx) error {
}
ctx.WriteString(v)
}
case AlterTableSetTiFlashMode:
ctx.WriteKeyWord("SET TIFLASH MODE ")
ctx.WriteKeyWord(n.TiFlashMode.String())
case AlterTableAddStatistics:
ctx.WriteKeyWord("ADD STATS_EXTENDED ")
if n.IfNotExists {
Expand Down
2 changes: 0 additions & 2 deletions parser/ast/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -536,8 +536,6 @@ func TestAlterTableSpecRestore(t *testing.T) {
{"placement policy p1", "PLACEMENT POLICY = `p1`"},
{"placement policy p1 comment='aaa'", "PLACEMENT POLICY = `p1` COMMENT = 'aaa'"},
{"partition p0 placement policy p1", "PARTITION `p0` PLACEMENT POLICY = `p1`"},
{"set tiflash mode normal", "SET TIFLASH MODE NORMAL"},
{"set tiflash mode fast", "SET TIFLASH MODE FAST"},
}
extractNodeFunc := func(node Node) Node {
return node.(*AlterTableStmt).Specs[0]
Expand Down
2 changes: 0 additions & 2 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -786,8 +786,6 @@ var tokenMap = map[string]int{
"VERBOSE": verboseType,
"VOTER": voter,
"VOTER_CONSTRAINTS": voterConstraints,
"NORMAL": normal,
"FAST": fast,
"VOTERS": voters,
"VIEW": view,
"VIRTUAL": virtual,
Expand Down
2 changes: 0 additions & 2 deletions parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,6 @@ const (
ActionAlterNoCacheTable ActionType = 59
ActionCreateTables ActionType = 60
ActionMultiSchemaChange ActionType = 61
ActionSetTiFlashMode ActionType = 62
)

var actionMap = map[ActionType]string{
Expand Down Expand Up @@ -157,7 +156,6 @@ var actionMap = map[ActionType]string{
ActionAlterNoCacheTable: "alter table nocache",
ActionAlterTableStatsOptions: "alter table statistics options",
ActionMultiSchemaChange: "alter table multi-schema change",
ActionSetTiFlashMode: "set tiflash mode",

// `ActionAlterTableAlterPartition` is removed and will never be used.
// Just left a tombstone here for compatibility.
Expand Down
27 changes: 0 additions & 27 deletions parser/model/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -454,10 +454,6 @@ type TableInfo struct {
// TiFlashReplica means the TiFlash replica info.
TiFlashReplica *TiFlashReplicaInfo `json:"tiflash_replica"`

// TiFlashMode means the table's mode in TiFlash.
// Table's default mode is TiFlashModeNormal
TiFlashMode TiFlashMode `json:"tiflash_mode"`

// IsColumnar means the table is column-oriented.
// It's true when the engine of the table is TiFlash only.
IsColumnar bool `json:"is_columnar"`
Expand Down Expand Up @@ -611,29 +607,6 @@ func (t TableLockType) String() string {
return ""
}

// TiFlashMode is the type of the table's mode in TiFlash.
type TiFlashMode string

//revive:disable:exported
const (
// In order to be compatible with the old version without tiflash mode
// we set the normal mode(default mode) as empty
TiFlashModeNormal TiFlashMode = ""
TiFlashModeFast TiFlashMode = "fast"
)

//revive:enable:exported

func (t TiFlashMode) String() string {
switch t {
case TiFlashModeNormal:
return "NORMAL"
case TiFlashModeFast:
return "FAST"
}
return ""
}

// TiFlashReplicaInfo means the flash replica info.
type TiFlashReplicaInfo struct {
Count uint64
Expand Down
Loading

0 comments on commit ccd4f34

Please sign in to comment.