Skip to content

Commit

Permalink
Merge branch 'master' of github.com:pingcap/tidb into collate_to_expl…
Browse files Browse the repository at this point in the history
…ain_1
  • Loading branch information
wjhuang2016 committed Jan 13, 2022
2 parents d69877b + 794e838 commit 65e1ce0
Show file tree
Hide file tree
Showing 37 changed files with 500 additions and 162 deletions.
30 changes: 0 additions & 30 deletions br/pkg/lightning/backend/local/duplicate.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"context"
"io"
"math"
"sort"
"sync"

"github.com/cockroachdb/pebble"
Expand Down Expand Up @@ -90,27 +89,6 @@ func (indexHandles *pendingIndexHandles) append(
indexHandles.rawHandles = append(indexHandles.rawHandles, rawHandle)
}

// appendAt pushes `other[i]` to the end of indexHandles.
func (indexHandles *pendingIndexHandles) appendAt(
other *pendingIndexHandles,
i int,
) {
indexHandles.append(
other.dataConflictInfos[i],
other.indexNames[i],
other.handles[i],
other.rawHandles[i],
)
}

// extends concatenates `other` to the end of indexHandles.
func (indexHandles *pendingIndexHandles) extend(other *pendingIndexHandles) {
indexHandles.dataConflictInfos = append(indexHandles.dataConflictInfos, other.dataConflictInfos...)
indexHandles.indexNames = append(indexHandles.indexNames, other.indexNames...)
indexHandles.handles = append(indexHandles.handles, other.handles...)
indexHandles.rawHandles = append(indexHandles.rawHandles, other.rawHandles...)
}

// truncate resets all arrays in indexHandles to length zero, but keeping the allocated capacity.
func (indexHandles *pendingIndexHandles) truncate() {
indexHandles.dataConflictInfos = indexHandles.dataConflictInfos[:0]
Expand All @@ -137,14 +115,6 @@ func (indexHandles *pendingIndexHandles) Swap(i, j int) {
indexHandles.rawHandles[i], indexHandles.rawHandles[j] = indexHandles.rawHandles[j], indexHandles.rawHandles[i]
}

// searchSortedRawHandle looks up for the index i such that `rawHandles[i] == rawHandle`.
// This function assumes indexHandles is already sorted, and rawHandle does exist in it.
func (indexHandles *pendingIndexHandles) searchSortedRawHandle(rawHandle []byte) int {
return sort.Search(indexHandles.Len(), func(i int) bool {
return bytes.Compare(indexHandles.rawHandles[i], rawHandle) >= 0
})
}

type pendingKeyRange tidbkv.KeyRange

func (kr pendingKeyRange) Less(other btree.Item) bool {
Expand Down
7 changes: 0 additions & 7 deletions br/pkg/lightning/backend/local/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,13 +274,6 @@ func (r rangeProperties) Encode() []byte {
return b
}

func (r rangeProperties) get(key []byte) rangeOffsets {
idx := sort.Search(len(r), func(i int) bool {
return bytes.Compare(r[i].Key, key) >= 0
})
return r[idx].rangeOffsets
}

type RangePropertiesCollector struct {
props rangeProperties
lastOffsets rangeOffsets
Expand Down
15 changes: 0 additions & 15 deletions br/pkg/lightning/backend/local/local_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,21 +174,6 @@ func TestRangeProperties(t *testing.T) {
require.Equal(t, cases[len(cases)-1].key, props[len(props)-1].Key)
require.Len(t, props, 7)

a := props.get([]byte("a"))
require.Equal(t, uint64(1), a.Size)
e := props.get([]byte("e"))
require.Equal(t, uint64(defaultPropSizeIndexDistance+5), e.Size)
i := props.get([]byte("i"))
require.Equal(t, uint64(defaultPropSizeIndexDistance/8*17+9), i.Size)
k := props.get([]byte("k"))
require.Equal(t, uint64(defaultPropSizeIndexDistance/8*25+11), k.Size)
m := props.get([]byte("m"))
require.Equal(t, uint64(defaultPropKeysIndexDistance+11), m.Keys)
n := props.get([]byte("n"))
require.Equal(t, uint64(defaultPropKeysIndexDistance*2+11), n.Keys)
o := props.get([]byte("o"))
require.Equal(t, uint64(defaultPropKeysIndexDistance*2+12), o.Keys)

props2 := rangeProperties([]rangeProperty{
{[]byte("b"), rangeOffsets{defaultPropSizeIndexDistance + 10, defaultPropKeysIndexDistance / 2}},
{[]byte("h"), rangeOffsets{defaultPropSizeIndexDistance * 3 / 2, defaultPropKeysIndexDistance * 3 / 2}},
Expand Down
11 changes: 3 additions & 8 deletions br/pkg/lightning/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@ const (
ErrorOnDup = "error"

defaultDistSQLScanConcurrency = 15
distSQLScanConcurrencyPerStore = 4
defaultBuildStatsConcurrency = 20
defaultIndexSerialScanConcurrency = 20
defaultChecksumTableConcurrency = 2
Expand All @@ -88,9 +87,9 @@ const (
//
// With cron.check-disk-quota = 1m, region-concurrency = 40, this should
// contribute 2.3 GiB to the reserved size.
autoDiskQuotaLocalReservedSpeed uint64 = 1 * units.KiB
defaultEngineMemCacheSize = 512 * units.MiB
defaultLocalWriterMemCacheSize = 128 * units.MiB
// autoDiskQuotaLocalReservedSpeed uint64 = 1 * units.KiB
defaultEngineMemCacheSize = 512 * units.MiB
defaultLocalWriterMemCacheSize = 128 * units.MiB

defaultCSVDataCharacterSet = "binary"
defaultCSVDataInvalidCharReplace = utf8.RuneError
Expand Down Expand Up @@ -950,10 +949,6 @@ func (cfg *Config) CheckAndAdjustForLocalBackend() error {
sortedKVDirInfo, err := os.Stat(storageSizeDir)

switch {
case os.IsNotExist(err):
// the sorted-kv-dir does not exist, meaning we will create it automatically.
// so we extract the storage size from its parent directory.
storageSizeDir = filepath.Dir(storageSizeDir)
case err == nil:
if !sortedKVDirInfo.IsDir() {
return errors.Errorf("tikv-importer.sorted-kv-dir ('%s') is not a directory", storageSizeDir)
Expand Down
2 changes: 0 additions & 2 deletions br/pkg/lightning/mydump/charset_convertor.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,8 +87,6 @@ func (cc *CharsetConvertor) initEncoder() error {
return errors.Errorf("not support %s as the conversion source yet", cc.sourceCharacterSet)
}

var utf8RuneErrorStr = string(utf8.RuneError)

// Decode does the charset conversion work from sourceCharacterSet to utf8mb4.
// It will return a string as the conversion result whose length may be less or greater
// than the original string `src`.
Expand Down
2 changes: 0 additions & 2 deletions br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,8 +806,6 @@ outer:

if tableCSVCount >= 2 && hasUniqueIdx {
tableMeta = tblMeta
csvCount = tableCSVCount
hasUniqueIdx = tableHasUniqueIdx
// if a perfect table source is found, we can stop check more tables
break outer
}
Expand Down
6 changes: 4 additions & 2 deletions br/pkg/lightning/restore/table_restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -928,12 +928,14 @@ func (tr *TableRestore) importKV(
regionSplitSize := int64(rc.cfg.TikvImporter.RegionSplitSize)
if regionSplitSize == 0 && rc.taskMgr != nil {
regionSplitSize = int64(config.SplitRegionSize)
rc.taskMgr.CheckTasksExclusively(ctx, func(tasks []taskMeta) ([]taskMeta, error) {
if err := rc.taskMgr.CheckTasksExclusively(ctx, func(tasks []taskMeta) ([]taskMeta, error) {
if len(tasks) > 0 {
regionSplitSize = int64(config.SplitRegionSize) * int64(utils.MinInt(len(tasks), config.MaxSplitRegionSizeRatio))
}
return nil, nil
})
}); err != nil {
return errors.Trace(err)
}
}
err := closedEngine.Import(ctx, regionSplitSize)
saveCpErr := rc.saveStatusCheckpoint(ctx, tr.tableName, engineID, err, checkpoints.CheckpointStatusImported)
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/restore/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ func (rs *RegionSplitter) ScatterRegionsWithBackoffer(ctx context.Context, newRe
for _, region := range newRegionSet {
err := rs.client.ScatterRegion(ctx, region)
if err == nil {
// it is safe accroding to the Go language spec.
// it is safe according to the Go language spec.
delete(newRegionSet, region.Region.Id)
} else if !pdErrorCanRetry(err) {
log.Warn("scatter meet error cannot be retried, skipping",
Expand All @@ -322,7 +322,7 @@ func (rs *RegionSplitter) ScatterRegionsWithBackoffer(ctx context.Context, newRe
logutil.ShortError(err),
logutil.AbbreviatedArray("failed-regions", newRegionSet, func(i interface{}) []string {
m := i.(map[uint64]*RegionInfo)
result := make([]string, len(m))
result := make([]string, 0, len(m))
for id := range m {
result = append(result, strconv.Itoa(int(id)))
}
Expand Down
6 changes: 6 additions & 0 deletions cmd/explaintest/r/new_character_set.result
Original file line number Diff line number Diff line change
Expand Up @@ -105,3 +105,9 @@ insert into t values('{"点赞": "你好"}', '{"点赞": "你好"}', '{"点赞":
select * from t;
j b s1 s2 st en
{"点赞": "你好"} {"鐐硅禐": "浣犲ソ"} {"鐐硅禐": "浣犲ソ"} {"点赞": "你好"} {"点赞": "你好"} {"点赞": "你好"}
set names utf8mb4;
set @@character_set_client=gbk;
set @@character_set_connection=gbk;
select hex('一a'), '一a';
hex('涓?') 涓?
E4B83F 涓?
5 changes: 5 additions & 0 deletions cmd/explaintest/t/new_character_set.test
Original file line number Diff line number Diff line change
Expand Up @@ -63,3 +63,8 @@ set character_set_connection = utf8mb4;
create table t(j json, b blob, s1 varchar(255) collate binary, s2 varchar(255), st set('{"点赞": "你好"}'), en enum('{"点赞": "你好"}'));
insert into t values('{"点赞": "你好"}', '{"点赞": "你好"}', '{"点赞": "你好"}', '{"点赞": "你好"}', '{"点赞": "你好"}', '{"点赞": "你好"}');
select * from t;

set names utf8mb4;
set @@character_set_client=gbk;
set @@character_set_connection=gbk;
select hex('一a'), '一a';
4 changes: 4 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,6 +264,10 @@ func checkAndNormalizePlacement(ctx sessionctx.Context, placementPolicyRef *mode
}

if directPlacementOpts != nil {
if !ctx.GetSessionVars().EnableDirectPlacement {
return nil, nil, errors.New("Direct placement is disabled")
}

// check the direct placement option compatibility.
if err := checkPolicyValidation(directPlacementOpts); err != nil {
return nil, nil, errors.Trace(err)
Expand Down
Loading

0 comments on commit 65e1ce0

Please sign in to comment.