Skip to content

Commit

Permalink
Merge branch 'master' into ran-huang-patch-1
Browse files Browse the repository at this point in the history
  • Loading branch information
ran-huang authored Dec 30, 2022
2 parents 76ecda9 + 6131e3f commit 8a4b939
Show file tree
Hide file tree
Showing 6 changed files with 84 additions and 13 deletions.
8 changes: 5 additions & 3 deletions br/pkg/conn/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,7 +281,8 @@ func (mgr *Mgr) GetTS(ctx context.Context) (uint64, error) {
}

// GetMergeRegionSizeAndCount returns the tikv config `coprocessor.region-split-size` and `coprocessor.region-split-key`.
func (mgr *Mgr) GetMergeRegionSizeAndCount(ctx context.Context, client *http.Client) (uint64, uint64, error) {
// returns the default config when failed.
func (mgr *Mgr) GetMergeRegionSizeAndCount(ctx context.Context, client *http.Client) (uint64, uint64) {
regionSplitSize := DefaultMergeRegionSizeBytes
regionSplitKeys := DefaultMergeRegionKeyCount
type coprocessor struct {
Expand Down Expand Up @@ -310,9 +311,10 @@ func (mgr *Mgr) GetMergeRegionSizeAndCount(ctx context.Context, client *http.Cli
return nil
})
if err != nil {
return 0, 0, errors.Trace(err)
log.Warn("meet error when getting config from TiKV; using default", logutil.ShortError(err))
return DefaultMergeRegionSizeBytes, DefaultMergeRegionKeyCount
}
return regionSplitSize, regionSplitKeys, nil
return regionSplitSize, regionSplitKeys
}

// GetConfigFromTiKV get configs from all alive tikv stores.
Expand Down
35 changes: 33 additions & 2 deletions br/pkg/conn/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -292,6 +292,38 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
regionSplitSize: DefaultMergeRegionSizeBytes,
regionSplitKeys: DefaultMergeRegionKeyCount,
},
{
stores: []*metapb.Store{
{
Id: 1,
State: metapb.StoreState_Up,
Labels: []*metapb.StoreLabel{
{
Key: "engine",
Value: "tiflash",
},
},
},
{
Id: 2,
State: metapb.StoreState_Up,
Labels: []*metapb.StoreLabel{
{
Key: "engine",
Value: "tikv",
},
},
},
},
content: []string{
"",
// Assuming the TiKV has failed due to some reason.
"",
},
// no tikv detected in this case
regionSplitSize: DefaultMergeRegionSizeBytes,
regionSplitKeys: DefaultMergeRegionKeyCount,
},
{
stores: []*metapb.Store{
{
Expand Down Expand Up @@ -388,8 +420,7 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
httpCli := mockServer.Client()
mgr := &Mgr{PdController: &pdutil.PdController{}}
mgr.PdController.SetPDClient(pdCli)
rs, rk, err := mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
require.NoError(t, err)
rs, rk := mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
require.Equal(t, ca.regionSplitSize, rs)
require.Equal(t, ca.regionSplitKeys, rk)
mockServer.Close()
Expand Down
5 changes: 1 addition & 4 deletions br/pkg/task/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,10 +494,7 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf
// according to https://github.com/pingcap/tidb/issues/34167.
// we should get the real config from tikv to adapt the dynamic region.
httpCli := httputil.NewClient(mgr.GetTLSConfig())
mergeRegionSize, mergeRegionCount, err = mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
if err != nil {
return errors.Trace(err)
}
mergeRegionSize, mergeRegionCount = mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
}

keepaliveCfg.PermitWithoutStream = true
Expand Down
5 changes: 1 addition & 4 deletions br/pkg/task/restore_raw.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,10 +80,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR
// according to https://github.com/pingcap/tidb/issues/34167.
// we should get the real config from tikv to adapt the dynamic region.
httpCli := httputil.NewClient(mgr.GetTLSConfig())
mergeRegionSize, mergeRegionCount, err = mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
if err != nil {
return errors.Trace(err)
}
mergeRegionSize, mergeRegionCount = mgr.GetMergeRegionSizeAndCount(ctx, httpCli)
}

keepaliveCfg := GetKeepalive(&cfg.Config)
Expand Down
9 changes: 9 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -1654,6 +1654,15 @@ func updateColumnDefaultValue(d *ddlCtx, t *meta.Meta, job *model.Job, newCol *m
job.State = model.JobStateCancelled
return ver, infoschema.ErrColumnNotExists.GenWithStackByArgs(newCol.Name, tblInfo.Name)
}

if hasDefaultValue, _, err := checkColumnDefaultValue(newContext(d.store), table.ToColumn(oldCol.Clone()), newCol.DefaultValue); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
} else if !hasDefaultValue {
job.State = model.JobStateCancelled
return ver, dbterror.ErrInvalidDefaultValue.GenWithStackByArgs(newCol.Name)
}

// The newCol's offset may be the value of the old schema version, so we can't use newCol directly.
oldCol.DefaultValue = newCol.DefaultValue
oldCol.DefaultValueBit = newCol.DefaultValueBit
Expand Down
35 changes: 35 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"math"
"strconv"
"strings"
"sync"
"testing"
"time"

Expand Down Expand Up @@ -1789,3 +1790,37 @@ func TestHashPartitionAddColumn(t *testing.T) {
dom.DDL().SetHook(hook)
tk.MustExec("alter table t add column c int")
}

func TestSetInvalidDefaultValueAfterModifyColumn(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("create table t(a int, b int)")

var wg sync.WaitGroup
var checkErr error
one := false
hook := &ddl.TestDDLCallback{Do: dom}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.SchemaState != model.StateDeleteOnly {
return
}
if !one {
one = true
} else {
return
}
wg.Add(1)
go func() {
tk2 := testkit.NewTestKit(t, store)
tk2.MustExec("use test")
_, checkErr = tk2.Exec("alter table t alter column a set default 1")
wg.Done()
}()
}
dom.DDL().SetHook(hook)
tk.MustExec("alter table t modify column a text(100)")
wg.Wait()
require.EqualError(t, checkErr, "[ddl:1101]BLOB/TEXT/JSON column 'a' can't have a default value")
}

0 comments on commit 8a4b939

Please sign in to comment.