Skip to content

Commit

Permalink
lightning: make pre-check output message clearer (#30439) (#30888)
Browse files Browse the repository at this point in the history
close #30395
  • Loading branch information
ti-srebot authored Feb 18, 2022
1 parent 0063773 commit 37671db
Show file tree
Hide file tree
Showing 6 changed files with 120 additions and 32 deletions.
7 changes: 6 additions & 1 deletion br/pkg/lightning/common/storage_unix.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,18 @@ import (
"syscall"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"golang.org/x/sys/unix"
)

// GetStorageSize gets storage's capacity and available size
func GetStorageSize(dir string) (size StorageSize, err error) {
var stat unix.Statfs_t
failpoint.Inject("GetStorageSize", func(val failpoint.Value) {
injectedSize := val.(int)
failpoint.Return(StorageSize{Capacity: uint64(injectedSize), Available: uint64(injectedSize)}, nil)
})

var stat unix.Statfs_t
err = unix.Statfs(dir, &stat)
if err != nil {
return size, errors.Annotatef(err, "cannot get disk capacity at %s", dir)
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/lightning/common/storage_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"unsafe"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
)

var (
Expand All @@ -32,6 +33,10 @@ var (

// GetStorageSize gets storage's capacity and available size
func GetStorageSize(dir string) (size StorageSize, err error) {
failpoint.Inject("GetStorageSize", func(val failpoint.Value) {
injectedSize := val.(int)
failpoint.Return(StorageSize{Capacity: uint64(injectedSize), Available: uint64(injectedSize)}, nil)
})
r, _, e := getDiskFreeSpaceExW.Call(
uintptr(unsafe.Pointer(syscall.StringToUTF16Ptr(dir))),
uintptr(unsafe.Pointer(&size.Available)),
Expand Down
36 changes: 17 additions & 19 deletions br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -454,33 +454,31 @@ func (rc *Controller) localResource(sourceSize int64) error {
if err != nil {
return errors.Trace(err)
}
localAvailable := storageSize.Available
localAvailable := int64(storageSize.Available)

var message string
var passed bool
switch {
case localAvailable > uint64(sourceSize):
case localAvailable > sourceSize:
message = fmt.Sprintf("local disk resources are rich, estimate sorted data size %s, local available is %s",
units.BytesSize(float64(sourceSize)), units.BytesSize(float64(localAvailable)))
passed = true
case int64(rc.cfg.TikvImporter.DiskQuota) > localAvailable:
message = fmt.Sprintf("local disk space may not enough to finish import, estimate sorted data size is %s,"+
" but local available is %s, please set `tikv-importer.disk-quota` to a smaller value than %s"+
" or change `mydumper.sorted-kv-dir` to another disk with enough space to finish imports",
units.BytesSize(float64(sourceSize)),
units.BytesSize(float64(localAvailable)), units.BytesSize(float64(localAvailable)))
passed = false
log.L().Error(message)
default:
if int64(rc.cfg.TikvImporter.DiskQuota) > int64(localAvailable) {
message = fmt.Sprintf("local disk space may not enough to finish import"+
"estimate sorted data size is %s, but local available is %s,"+
"you need a smaller number for tikv-importer.disk-quota (%s) to finish imports",
units.BytesSize(float64(sourceSize)),
units.BytesSize(float64(localAvailable)), units.BytesSize(float64(rc.cfg.TikvImporter.DiskQuota)))
passed = false
log.L().Error(message)
} else {
message = fmt.Sprintf("local disk space may not enough to finish import, "+
"estimate sorted data size is %s, but local available is %s,"+
"we will use disk-quota (size: %s) to finish imports, which may slow down import",
units.BytesSize(float64(sourceSize)),
units.BytesSize(float64(localAvailable)), units.BytesSize(float64(rc.cfg.TikvImporter.DiskQuota)))
passed = true
log.L().Warn(message)
}
message = fmt.Sprintf("local disk space may not enough to finish import, "+
"estimate sorted data size is %s, but local available is %s,"+
"we will use disk-quota (size: %s) to finish imports, which may slow down import",
units.BytesSize(float64(sourceSize)),
units.BytesSize(float64(localAvailable)), units.BytesSize(float64(rc.cfg.TikvImporter.DiskQuota)))
passed = true
log.L().Warn(message)
}
rc.checkTemplate.Collect(Critical, passed, message)
return nil
Expand Down
80 changes: 80 additions & 0 deletions br/pkg/lightning/restore/check_info_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package restore

import (
"context"

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"

"github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/br/pkg/lightning/worker"
"github.com/pingcap/tidb/br/pkg/storage"
)

var _ = Suite(&checkInfoSuite{})

type checkInfoSuite struct{}

func (s *checkInfoSuite) TestLocalResource(c *C) {
dir := c.MkDir()
mockStore, err := storage.NewLocalStorage(dir)
c.Assert(err, IsNil)

err = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/common/GetStorageSize", "return(2048)")
c.Assert(err, IsNil)
defer func() {
_ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/common/GetStorageSize")
}()

cfg := config.NewConfig()
cfg.Mydumper.SourceDir = dir
cfg.TikvImporter.SortedKVDir = dir
cfg.TikvImporter.Backend = "local"
rc := &Controller{
cfg: cfg,
store: mockStore,
ioWorkers: worker.NewPool(context.Background(), 1, "io"),
}

// 1. source-size is smaller than disk-size, won't trigger error information
rc.checkTemplate = NewSimpleTemplate()
err = rc.localResource(1000)
c.Assert(err, IsNil)
tmpl := rc.checkTemplate.(*SimpleTemplate)
c.Assert(tmpl.warnFailedCount, Equals, 1)
c.Assert(tmpl.criticalFailedCount, Equals, 0)
c.Assert(tmpl.normalMsgs[1], Matches, "local disk resources are rich, estimate sorted data size 1000B, local available is 2KiB")

// 2. source-size is bigger than disk-size, with default disk-quota will trigger a critical error
rc.checkTemplate = NewSimpleTemplate()
err = rc.localResource(4096)
c.Assert(err, IsNil)
tmpl = rc.checkTemplate.(*SimpleTemplate)
c.Assert(tmpl.warnFailedCount, Equals, 1)
c.Assert(tmpl.criticalFailedCount, Equals, 1)
c.Assert(tmpl.criticalMsgs[0], Matches, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB, please set `tikv-importer.disk-quota` to a smaller value than 2KiB or change `mydumper.sorted-kv-dir` to another disk with enough space to finish imports")

// 3. source-size is bigger than disk-size, with a vaild disk-quota will trigger a warning
rc.checkTemplate = NewSimpleTemplate()
rc.cfg.TikvImporter.DiskQuota = config.ByteSize(1024)
err = rc.localResource(4096)
c.Assert(err, IsNil)
tmpl = rc.checkTemplate.(*SimpleTemplate)
c.Assert(tmpl.warnFailedCount, Equals, 1)
c.Assert(tmpl.criticalFailedCount, Equals, 0)
c.Assert(tmpl.normalMsgs[1], Matches, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB,we will use disk-quota \\(size: 1KiB\\) to finish imports, which may slow down import")
}
21 changes: 11 additions & 10 deletions br/pkg/lightning/restore/check_template.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,8 @@ type SimpleTemplate struct {
count int
warnFailedCount int
criticalFailedCount int
failedMsg []string
normalMsgs []string // only used in unit test now
criticalMsgs []string
t table.Writer
}

Expand All @@ -65,16 +66,12 @@ func NewSimpleTemplate() Template {
{Name: "Passed", WidthMax: 6},
})
return &SimpleTemplate{
0,
0,
0,
make([]string, 0),
t,
t: t,
}
}

func (c *SimpleTemplate) FailedMsg() string {
return strings.Join(c.failedMsg, ";\n")
return strings.Join(c.criticalMsgs, ";\n")
}

func (c *SimpleTemplate) Collect(t CheckType, passed bool, msg string) {
Expand All @@ -87,7 +84,11 @@ func (c *SimpleTemplate) Collect(t CheckType, passed bool, msg string) {
c.warnFailedCount++
}
}
c.failedMsg = append(c.failedMsg, msg)
if !passed && t == Critical {
c.criticalMsgs = append(c.criticalMsgs, msg)
} else {
c.normalMsgs = append(c.normalMsgs, msg)
}
c.t.AppendRow(table.Row{c.count, msg, t, passed})
c.t.AppendSeparator()
}
Expand All @@ -108,7 +109,7 @@ func (c *SimpleTemplate) FailedCount(t CheckType) int {

func (c *SimpleTemplate) Output() string {
c.t.SetAllowedRowLength(170)
c.t.SetRowPainter(table.RowPainter(func(row table.Row) text.Colors {
c.t.SetRowPainter(func(row table.Row) text.Colors {
if passed, ok := row[3].(bool); ok {
if !passed {
if typ, ok := row[2].(CheckType); ok {
Expand All @@ -122,7 +123,7 @@ func (c *SimpleTemplate) Output() string {
}
}
return nil
}))
})
res := c.t.Render()
summary := "\n"
if c.criticalFailedCount > 0 {
Expand Down
3 changes: 1 addition & 2 deletions br/pkg/lightning/restore/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -1913,8 +1913,7 @@ func (rc *Controller) preCheckRequirements(ctx context.Context) error {
if !taskExist && rc.taskMgr != nil {
rc.taskMgr.CleanupTask(ctx)
}
return errors.Errorf("tidb-lightning check failed."+
" Please fix the failed check(s):\n %s", rc.checkTemplate.FailedMsg())
return errors.Errorf("tidb-lightning pre-check failed: %s", rc.checkTemplate.FailedMsg())
}
return nil
}
Expand Down

0 comments on commit 37671db

Please sign in to comment.