*: improve the user experience for using Ctrl+C to make tidb exit (#58537)
close pingcap/tidb#58418
This commit is contained in:
@ -325,9 +325,9 @@ func main() {
|
||||
exited := make(chan struct{})
|
||||
signal.SetupSignalHandler(func() {
|
||||
svr.Close()
|
||||
resourcemanager.InstanceResourceManager.Stop()
|
||||
cleanup(svr, storage, dom)
|
||||
cpuprofile.StopCPUProfiler()
|
||||
resourcemanager.InstanceResourceManager.Stop()
|
||||
executor.Stop()
|
||||
close(exited)
|
||||
})
|
||||
|
||||
@ -1004,7 +1004,7 @@ func (do *Domain) CheckAutoAnalyzeWindows() {
|
||||
}
|
||||
}
|
||||
|
||||
func (do *Domain) refreshMDLCheckTableInfo() {
|
||||
func (do *Domain) refreshMDLCheckTableInfo(ctx context.Context) {
|
||||
se, err := do.sysSessionPool.Get()
|
||||
|
||||
if err != nil {
|
||||
@ -1013,7 +1013,7 @@ func (do *Domain) refreshMDLCheckTableInfo() {
|
||||
}
|
||||
// Make sure the session is new.
|
||||
sctx := se.(sessionctx.Context)
|
||||
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta)
|
||||
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
|
||||
if _, err := sctx.GetSQLExecutor().ExecuteInternal(ctx, "rollback"); err != nil {
|
||||
se.Close()
|
||||
return
|
||||
@ -1182,7 +1182,7 @@ func (do *Domain) loadSchemaInLoop(ctx context.Context) {
|
||||
case <-do.exit:
|
||||
return
|
||||
}
|
||||
do.refreshMDLCheckTableInfo()
|
||||
do.refreshMDLCheckTableInfo(ctx)
|
||||
select {
|
||||
case do.mdlCheckCh <- struct{}{}:
|
||||
default:
|
||||
|
||||
@ -32,7 +32,7 @@ func SetupSignalHandler(shutdownFunc func()) {
|
||||
|
||||
signal.Notify(usrDefSignalChan, syscall.SIGUSR1)
|
||||
go func() {
|
||||
buf := make([]byte, 1<<16)
|
||||
buf := make([]byte, 1<<17)
|
||||
for {
|
||||
sig := <-usrDefSignalChan
|
||||
if sig == syscall.SIGUSR1 {
|
||||
|
||||
@ -29,17 +29,20 @@ import (
|
||||
// if the `exited` signal is true by print them on log.
|
||||
type WaitGroupEnhancedWrapper struct {
|
||||
sync.WaitGroup
|
||||
source string
|
||||
registerProcess sync.Map
|
||||
source string
|
||||
mu struct {
|
||||
sync.Mutex
|
||||
registerProcess map[string]struct{}
|
||||
}
|
||||
}
|
||||
|
||||
// NewWaitGroupEnhancedWrapper returns WaitGroupEnhancedWrapper, the empty source indicates the unit test, then
|
||||
// the `checkUnExitedProcess` won't be executed.
|
||||
func NewWaitGroupEnhancedWrapper(source string, exit chan struct{}, exitedCheck bool) *WaitGroupEnhancedWrapper {
|
||||
wgew := &WaitGroupEnhancedWrapper{
|
||||
source: source,
|
||||
registerProcess: sync.Map{},
|
||||
source: source,
|
||||
}
|
||||
wgew.mu.registerProcess = make(map[string]struct{})
|
||||
if exitedCheck {
|
||||
wgew.Add(1)
|
||||
go wgew.checkUnExitedProcess(exit)
|
||||
@ -56,7 +59,7 @@ func (w *WaitGroupEnhancedWrapper) checkUnExitedProcess(exit chan struct{}) {
|
||||
<-exit
|
||||
logutil.BgLogger().Info("waitGroupWrapper start exit-checking", zap.String("source", w.source))
|
||||
if w.check() {
|
||||
ticker := time.NewTimer(2 * time.Second)
|
||||
ticker := time.NewTicker(2 * time.Second)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
<-ticker.C
|
||||
@ -70,10 +73,11 @@ func (w *WaitGroupEnhancedWrapper) checkUnExitedProcess(exit chan struct{}) {
|
||||
|
||||
func (w *WaitGroupEnhancedWrapper) check() bool {
|
||||
unexitedProcess := make([]string, 0)
|
||||
w.registerProcess.Range(func(key, _ any) bool {
|
||||
unexitedProcess = append(unexitedProcess, key.(string))
|
||||
return true
|
||||
})
|
||||
w.mu.Lock()
|
||||
for key := range w.mu.registerProcess {
|
||||
unexitedProcess = append(unexitedProcess, key)
|
||||
}
|
||||
w.mu.Unlock()
|
||||
if len(unexitedProcess) > 0 {
|
||||
logutil.BgLogger().Warn("background process unexited while received exited signal",
|
||||
zap.Strings("process", unexitedProcess),
|
||||
@ -123,20 +127,26 @@ func (w *WaitGroupEnhancedWrapper) RunWithRecover(exec func(), recoverFn func(r
|
||||
}
|
||||
|
||||
func (w *WaitGroupEnhancedWrapper) onStart(label string) {
|
||||
_, ok := w.registerProcess.Load(label)
|
||||
w.mu.Lock()
|
||||
_, ok := w.mu.registerProcess[label]
|
||||
w.mu.Unlock()
|
||||
if ok {
|
||||
logutil.BgLogger().Panic("WaitGroupEnhancedWrapper received duplicated source process",
|
||||
zap.String("source", w.source),
|
||||
zap.String("process", label))
|
||||
}
|
||||
w.registerProcess.Store(label, struct{}{})
|
||||
w.mu.Lock()
|
||||
w.mu.registerProcess[label] = struct{}{}
|
||||
w.mu.Unlock()
|
||||
logutil.BgLogger().Info("background process started",
|
||||
zap.String("source", w.source),
|
||||
zap.String("process", label))
|
||||
}
|
||||
|
||||
func (w *WaitGroupEnhancedWrapper) onExit(label string) {
|
||||
w.registerProcess.Delete(label)
|
||||
w.mu.Lock()
|
||||
delete(w.mu.registerProcess, label)
|
||||
w.mu.Unlock()
|
||||
logutil.BgLogger().Info("background process exited",
|
||||
zap.String("source", w.source),
|
||||
zap.String("process", label))
|
||||
|
||||
Reference in New Issue
Block a user