mirror of
				https://github.com/go-gitea/gitea
				synced 2025-11-03 21:08:25 +00:00 
			
		
		
		
	Refactor graceful manager to use shared code (#28073)
Make "windows" and "unix" share as much code as possible. No logic change.
This commit is contained in:
		
							
								
								
									
										104
									
								
								modules/graceful/manager_common.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										104
									
								
								modules/graceful/manager_common.go
									
									
									
									
									
										Normal file
									
								
							@@ -0,0 +1,104 @@
 | 
				
			|||||||
 | 
					// Copyright 2023 The Gitea Authors. All rights reserved.
 | 
				
			||||||
 | 
					// SPDX-License-Identifier: MIT
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					package graceful
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					import (
 | 
				
			||||||
 | 
						"context"
 | 
				
			||||||
 | 
						"runtime/pprof"
 | 
				
			||||||
 | 
						"sync"
 | 
				
			||||||
 | 
						"time"
 | 
				
			||||||
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					type systemdNotifyMsg string
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					const (
 | 
				
			||||||
 | 
						readyMsg     systemdNotifyMsg = "READY=1"
 | 
				
			||||||
 | 
						stoppingMsg  systemdNotifyMsg = "STOPPING=1"
 | 
				
			||||||
 | 
						reloadingMsg systemdNotifyMsg = "RELOADING=1"
 | 
				
			||||||
 | 
						watchdogMsg  systemdNotifyMsg = "WATCHDOG=1"
 | 
				
			||||||
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func statusMsg(msg string) systemdNotifyMsg {
 | 
				
			||||||
 | 
						return systemdNotifyMsg("STATUS=" + msg)
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// Manager manages the graceful shutdown process
 | 
				
			||||||
 | 
					type Manager struct {
 | 
				
			||||||
 | 
						ctx                    context.Context
 | 
				
			||||||
 | 
						isChild                bool
 | 
				
			||||||
 | 
						forked                 bool
 | 
				
			||||||
 | 
						lock                   sync.RWMutex
 | 
				
			||||||
 | 
						state                  state
 | 
				
			||||||
 | 
						shutdownCtx            context.Context
 | 
				
			||||||
 | 
						hammerCtx              context.Context
 | 
				
			||||||
 | 
						terminateCtx           context.Context
 | 
				
			||||||
 | 
						managerCtx             context.Context
 | 
				
			||||||
 | 
						shutdownCtxCancel      context.CancelFunc
 | 
				
			||||||
 | 
						hammerCtxCancel        context.CancelFunc
 | 
				
			||||||
 | 
						terminateCtxCancel     context.CancelFunc
 | 
				
			||||||
 | 
						managerCtxCancel       context.CancelFunc
 | 
				
			||||||
 | 
						runningServerWaitGroup sync.WaitGroup
 | 
				
			||||||
 | 
						createServerWaitGroup  sync.WaitGroup
 | 
				
			||||||
 | 
						terminateWaitGroup     sync.WaitGroup
 | 
				
			||||||
 | 
						shutdownRequested      chan struct{}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						toRunAtShutdown  []func()
 | 
				
			||||||
 | 
						toRunAtTerminate []func()
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func newGracefulManager(ctx context.Context) *Manager {
 | 
				
			||||||
 | 
						manager := &Manager{ctx: ctx, shutdownRequested: make(chan struct{})}
 | 
				
			||||||
 | 
						manager.createServerWaitGroup.Add(numberOfServersToCreate)
 | 
				
			||||||
 | 
						manager.prepare(ctx)
 | 
				
			||||||
 | 
						manager.start()
 | 
				
			||||||
 | 
						return manager
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func (g *Manager) prepare(ctx context.Context) {
 | 
				
			||||||
 | 
						g.terminateCtx, g.terminateCtxCancel = context.WithCancel(ctx)
 | 
				
			||||||
 | 
						g.shutdownCtx, g.shutdownCtxCancel = context.WithCancel(ctx)
 | 
				
			||||||
 | 
						g.hammerCtx, g.hammerCtxCancel = context.WithCancel(ctx)
 | 
				
			||||||
 | 
						g.managerCtx, g.managerCtxCancel = context.WithCancel(ctx)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						g.terminateCtx = pprof.WithLabels(g.terminateCtx, pprof.Labels("graceful-lifecycle", "with-terminate"))
 | 
				
			||||||
 | 
						g.shutdownCtx = pprof.WithLabels(g.shutdownCtx, pprof.Labels("graceful-lifecycle", "with-shutdown"))
 | 
				
			||||||
 | 
						g.hammerCtx = pprof.WithLabels(g.hammerCtx, pprof.Labels("graceful-lifecycle", "with-hammer"))
 | 
				
			||||||
 | 
						g.managerCtx = pprof.WithLabels(g.managerCtx, pprof.Labels("graceful-lifecycle", "with-manager"))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						if !g.setStateTransition(stateInit, stateRunning) {
 | 
				
			||||||
 | 
							panic("invalid graceful manager state: transition from init to running failed")
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// DoImmediateHammer causes an immediate hammer
 | 
				
			||||||
 | 
					func (g *Manager) DoImmediateHammer() {
 | 
				
			||||||
 | 
						g.notify(statusMsg("Sending immediate hammer"))
 | 
				
			||||||
 | 
						g.doHammerTime(0 * time.Second)
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// DoGracefulShutdown causes a graceful shutdown
 | 
				
			||||||
 | 
					func (g *Manager) DoGracefulShutdown() {
 | 
				
			||||||
 | 
						g.lock.Lock()
 | 
				
			||||||
 | 
						select {
 | 
				
			||||||
 | 
						case <-g.shutdownRequested:
 | 
				
			||||||
 | 
						default:
 | 
				
			||||||
 | 
							close(g.shutdownRequested)
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
						forked := g.forked
 | 
				
			||||||
 | 
						g.lock.Unlock()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						if !forked {
 | 
				
			||||||
 | 
							g.notify(stoppingMsg)
 | 
				
			||||||
 | 
						} else {
 | 
				
			||||||
 | 
							g.notify(statusMsg("Shutting down after fork"))
 | 
				
			||||||
 | 
						}
 | 
				
			||||||
 | 
						g.doShutdown()
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					// RegisterServer registers the running of a listening server, in the case of unix this means that the parent process can now die.
 | 
				
			||||||
 | 
					// Any call to RegisterServer must be matched by a call to ServerDone
 | 
				
			||||||
 | 
					func (g *Manager) RegisterServer() {
 | 
				
			||||||
 | 
						KillParent()
 | 
				
			||||||
 | 
						g.runningServerWaitGroup.Add(1)
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
@@ -12,7 +12,6 @@ import (
 | 
				
			|||||||
	"os/signal"
 | 
						"os/signal"
 | 
				
			||||||
	"runtime/pprof"
 | 
						"runtime/pprof"
 | 
				
			||||||
	"strconv"
 | 
						"strconv"
 | 
				
			||||||
	"sync"
 | 
					 | 
				
			||||||
	"syscall"
 | 
						"syscall"
 | 
				
			||||||
	"time"
 | 
						"time"
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@@ -22,51 +21,6 @@ import (
 | 
				
			|||||||
	"code.gitea.io/gitea/modules/setting"
 | 
						"code.gitea.io/gitea/modules/setting"
 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Manager manages the graceful shutdown process
 | 
					 | 
				
			||||||
type Manager struct {
 | 
					 | 
				
			||||||
	isChild                bool
 | 
					 | 
				
			||||||
	forked                 bool
 | 
					 | 
				
			||||||
	lock                   *sync.RWMutex
 | 
					 | 
				
			||||||
	state                  state
 | 
					 | 
				
			||||||
	shutdownCtx            context.Context
 | 
					 | 
				
			||||||
	hammerCtx              context.Context
 | 
					 | 
				
			||||||
	terminateCtx           context.Context
 | 
					 | 
				
			||||||
	managerCtx             context.Context
 | 
					 | 
				
			||||||
	shutdownCtxCancel      context.CancelFunc
 | 
					 | 
				
			||||||
	hammerCtxCancel        context.CancelFunc
 | 
					 | 
				
			||||||
	terminateCtxCancel     context.CancelFunc
 | 
					 | 
				
			||||||
	managerCtxCancel       context.CancelFunc
 | 
					 | 
				
			||||||
	runningServerWaitGroup sync.WaitGroup
 | 
					 | 
				
			||||||
	createServerWaitGroup  sync.WaitGroup
 | 
					 | 
				
			||||||
	terminateWaitGroup     sync.WaitGroup
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	toRunAtShutdown  []func()
 | 
					 | 
				
			||||||
	toRunAtTerminate []func()
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func newGracefulManager(ctx context.Context) *Manager {
 | 
					 | 
				
			||||||
	manager := &Manager{
 | 
					 | 
				
			||||||
		isChild: len(os.Getenv(listenFDsEnv)) > 0 && os.Getppid() > 1,
 | 
					 | 
				
			||||||
		lock:    &sync.RWMutex{},
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	manager.createServerWaitGroup.Add(numberOfServersToCreate)
 | 
					 | 
				
			||||||
	manager.start(ctx)
 | 
					 | 
				
			||||||
	return manager
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
type systemdNotifyMsg string
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
const (
 | 
					 | 
				
			||||||
	readyMsg     systemdNotifyMsg = "READY=1"
 | 
					 | 
				
			||||||
	stoppingMsg  systemdNotifyMsg = "STOPPING=1"
 | 
					 | 
				
			||||||
	reloadingMsg systemdNotifyMsg = "RELOADING=1"
 | 
					 | 
				
			||||||
	watchdogMsg  systemdNotifyMsg = "WATCHDOG=1"
 | 
					 | 
				
			||||||
)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func statusMsg(msg string) systemdNotifyMsg {
 | 
					 | 
				
			||||||
	return systemdNotifyMsg("STATUS=" + msg)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func pidMsg() systemdNotifyMsg {
 | 
					func pidMsg() systemdNotifyMsg {
 | 
				
			||||||
	return systemdNotifyMsg("MAINPID=" + strconv.Itoa(os.Getpid()))
 | 
						return systemdNotifyMsg("MAINPID=" + strconv.Itoa(os.Getpid()))
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
@@ -89,27 +43,13 @@ func (g *Manager) notify(msg systemdNotifyMsg) {
 | 
				
			|||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
func (g *Manager) start(ctx context.Context) {
 | 
					func (g *Manager) start() {
 | 
				
			||||||
	// Make contexts
 | 
					 | 
				
			||||||
	g.terminateCtx, g.terminateCtxCancel = context.WithCancel(ctx)
 | 
					 | 
				
			||||||
	g.shutdownCtx, g.shutdownCtxCancel = context.WithCancel(ctx)
 | 
					 | 
				
			||||||
	g.hammerCtx, g.hammerCtxCancel = context.WithCancel(ctx)
 | 
					 | 
				
			||||||
	g.managerCtx, g.managerCtxCancel = context.WithCancel(ctx)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	// Next add pprof labels to these contexts
 | 
					 | 
				
			||||||
	g.terminateCtx = pprof.WithLabels(g.terminateCtx, pprof.Labels("graceful-lifecycle", "with-terminate"))
 | 
					 | 
				
			||||||
	g.shutdownCtx = pprof.WithLabels(g.shutdownCtx, pprof.Labels("graceful-lifecycle", "with-shutdown"))
 | 
					 | 
				
			||||||
	g.hammerCtx = pprof.WithLabels(g.hammerCtx, pprof.Labels("graceful-lifecycle", "with-hammer"))
 | 
					 | 
				
			||||||
	g.managerCtx = pprof.WithLabels(g.managerCtx, pprof.Labels("graceful-lifecycle", "with-manager"))
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	// Now label this and all goroutines created by this goroutine with the graceful-lifecycle manager
 | 
						// Now label this and all goroutines created by this goroutine with the graceful-lifecycle manager
 | 
				
			||||||
	pprof.SetGoroutineLabels(g.managerCtx)
 | 
						pprof.SetGoroutineLabels(g.managerCtx)
 | 
				
			||||||
	defer pprof.SetGoroutineLabels(ctx)
 | 
						defer pprof.SetGoroutineLabels(g.ctx)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
						g.isChild = len(os.Getenv(listenFDsEnv)) > 0 && os.Getppid() > 1
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Set the running state & handle signals
 | 
					 | 
				
			||||||
	if !g.setStateTransition(stateInit, stateRunning) {
 | 
					 | 
				
			||||||
		panic("invalid graceful manager state: transition from init to running failed")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	g.notify(statusMsg("Starting Gitea"))
 | 
						g.notify(statusMsg("Starting Gitea"))
 | 
				
			||||||
	g.notify(pidMsg())
 | 
						g.notify(pidMsg())
 | 
				
			||||||
	go g.handleSignals(g.managerCtx)
 | 
						go g.handleSignals(g.managerCtx)
 | 
				
			||||||
@@ -118,11 +58,9 @@ func (g *Manager) start(ctx context.Context) {
 | 
				
			|||||||
	startupDone := make(chan struct{})
 | 
						startupDone := make(chan struct{})
 | 
				
			||||||
	go func() {
 | 
						go func() {
 | 
				
			||||||
		defer close(startupDone)
 | 
							defer close(startupDone)
 | 
				
			||||||
		// Wait till we're done getting all of the listeners and then close
 | 
							// Wait till we're done getting all the listeners and then close the unused ones
 | 
				
			||||||
		// the unused ones
 | 
					 | 
				
			||||||
		g.createServerWaitGroup.Wait()
 | 
							g.createServerWaitGroup.Wait()
 | 
				
			||||||
		// Ignore the error here there's not much we can do with it
 | 
							// Ignore the error here there's not much we can do with it, they're logged in the CloseProvidedListeners function
 | 
				
			||||||
		// They're logged in the CloseProvidedListeners function
 | 
					 | 
				
			||||||
		_ = CloseProvidedListeners()
 | 
							_ = CloseProvidedListeners()
 | 
				
			||||||
		g.notify(readyMsg)
 | 
							g.notify(readyMsg)
 | 
				
			||||||
	}()
 | 
						}()
 | 
				
			||||||
@@ -133,7 +71,7 @@ func (g *Manager) start(ctx context.Context) {
 | 
				
			|||||||
				return
 | 
									return
 | 
				
			||||||
			case <-g.IsShutdown():
 | 
								case <-g.IsShutdown():
 | 
				
			||||||
				func() {
 | 
									func() {
 | 
				
			||||||
					// When waitgroup counter goes negative it will panic - we don't care about this so we can just ignore it.
 | 
										// When WaitGroup counter goes negative it will panic - we don't care about this so we can just ignore it.
 | 
				
			||||||
					defer func() {
 | 
										defer func() {
 | 
				
			||||||
						_ = recover()
 | 
											_ = recover()
 | 
				
			||||||
					}()
 | 
										}()
 | 
				
			||||||
@@ -255,29 +193,3 @@ func (g *Manager) DoGracefulRestart() {
 | 
				
			|||||||
		g.doShutdown()
 | 
							g.doShutdown()
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					 | 
				
			||||||
// DoImmediateHammer causes an immediate hammer
 | 
					 | 
				
			||||||
func (g *Manager) DoImmediateHammer() {
 | 
					 | 
				
			||||||
	g.notify(statusMsg("Sending immediate hammer"))
 | 
					 | 
				
			||||||
	g.doHammerTime(0 * time.Second)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
// DoGracefulShutdown causes a graceful shutdown
 | 
					 | 
				
			||||||
func (g *Manager) DoGracefulShutdown() {
 | 
					 | 
				
			||||||
	g.lock.Lock()
 | 
					 | 
				
			||||||
	if !g.forked {
 | 
					 | 
				
			||||||
		g.lock.Unlock()
 | 
					 | 
				
			||||||
		g.notify(stoppingMsg)
 | 
					 | 
				
			||||||
	} else {
 | 
					 | 
				
			||||||
		g.lock.Unlock()
 | 
					 | 
				
			||||||
		g.notify(statusMsg("Shutting down after fork"))
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	g.doShutdown()
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
// RegisterServer registers the running of a listening server, in the case of unix this means that the parent process can now die.
 | 
					 | 
				
			||||||
// Any call to RegisterServer must be matched by a call to ServerDone
 | 
					 | 
				
			||||||
func (g *Manager) RegisterServer() {
 | 
					 | 
				
			||||||
	KillParent()
 | 
					 | 
				
			||||||
	g.runningServerWaitGroup.Add(1)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 
 | 
				
			|||||||
@@ -7,11 +7,9 @@
 | 
				
			|||||||
package graceful
 | 
					package graceful
 | 
				
			||||||
 | 
					
 | 
				
			||||||
import (
 | 
					import (
 | 
				
			||||||
	"context"
 | 
					 | 
				
			||||||
	"os"
 | 
						"os"
 | 
				
			||||||
	"runtime/pprof"
 | 
						"runtime/pprof"
 | 
				
			||||||
	"strconv"
 | 
						"strconv"
 | 
				
			||||||
	"sync"
 | 
					 | 
				
			||||||
	"time"
 | 
						"time"
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	"code.gitea.io/gitea/modules/log"
 | 
						"code.gitea.io/gitea/modules/log"
 | 
				
			||||||
@@ -30,64 +28,11 @@ const (
 | 
				
			|||||||
	acceptHammerCode = svc.Accepted(hammerCode)
 | 
						acceptHammerCode = svc.Accepted(hammerCode)
 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// Manager manages the graceful shutdown process
 | 
					 | 
				
			||||||
type Manager struct {
 | 
					 | 
				
			||||||
	ctx                    context.Context
 | 
					 | 
				
			||||||
	isChild                bool
 | 
					 | 
				
			||||||
	lock                   *sync.RWMutex
 | 
					 | 
				
			||||||
	state                  state
 | 
					 | 
				
			||||||
	shutdownCtx            context.Context
 | 
					 | 
				
			||||||
	hammerCtx              context.Context
 | 
					 | 
				
			||||||
	terminateCtx           context.Context
 | 
					 | 
				
			||||||
	managerCtx             context.Context
 | 
					 | 
				
			||||||
	shutdownCtxCancel      context.CancelFunc
 | 
					 | 
				
			||||||
	hammerCtxCancel        context.CancelFunc
 | 
					 | 
				
			||||||
	terminateCtxCancel     context.CancelFunc
 | 
					 | 
				
			||||||
	managerCtxCancel       context.CancelFunc
 | 
					 | 
				
			||||||
	runningServerWaitGroup sync.WaitGroup
 | 
					 | 
				
			||||||
	createServerWaitGroup  sync.WaitGroup
 | 
					 | 
				
			||||||
	terminateWaitGroup     sync.WaitGroup
 | 
					 | 
				
			||||||
	shutdownRequested      chan struct{}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	toRunAtShutdown  []func()
 | 
					 | 
				
			||||||
	toRunAtTerminate []func()
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func newGracefulManager(ctx context.Context) *Manager {
 | 
					 | 
				
			||||||
	manager := &Manager{
 | 
					 | 
				
			||||||
		isChild: false,
 | 
					 | 
				
			||||||
		lock:    &sync.RWMutex{},
 | 
					 | 
				
			||||||
		ctx:     ctx,
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	manager.createServerWaitGroup.Add(numberOfServersToCreate)
 | 
					 | 
				
			||||||
	manager.start()
 | 
					 | 
				
			||||||
	return manager
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func (g *Manager) start() {
 | 
					func (g *Manager) start() {
 | 
				
			||||||
	// Make contexts
 | 
					 | 
				
			||||||
	g.terminateCtx, g.terminateCtxCancel = context.WithCancel(g.ctx)
 | 
					 | 
				
			||||||
	g.shutdownCtx, g.shutdownCtxCancel = context.WithCancel(g.ctx)
 | 
					 | 
				
			||||||
	g.hammerCtx, g.hammerCtxCancel = context.WithCancel(g.ctx)
 | 
					 | 
				
			||||||
	g.managerCtx, g.managerCtxCancel = context.WithCancel(g.ctx)
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	// Next add pprof labels to these contexts
 | 
					 | 
				
			||||||
	g.terminateCtx = pprof.WithLabels(g.terminateCtx, pprof.Labels("graceful-lifecycle", "with-terminate"))
 | 
					 | 
				
			||||||
	g.shutdownCtx = pprof.WithLabels(g.shutdownCtx, pprof.Labels("graceful-lifecycle", "with-shutdown"))
 | 
					 | 
				
			||||||
	g.hammerCtx = pprof.WithLabels(g.hammerCtx, pprof.Labels("graceful-lifecycle", "with-hammer"))
 | 
					 | 
				
			||||||
	g.managerCtx = pprof.WithLabels(g.managerCtx, pprof.Labels("graceful-lifecycle", "with-manager"))
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	// Now label this and all goroutines created by this goroutine with the graceful-lifecycle manager
 | 
						// Now label this and all goroutines created by this goroutine with the graceful-lifecycle manager
 | 
				
			||||||
	pprof.SetGoroutineLabels(g.managerCtx)
 | 
						pprof.SetGoroutineLabels(g.managerCtx)
 | 
				
			||||||
	defer pprof.SetGoroutineLabels(g.ctx)
 | 
						defer pprof.SetGoroutineLabels(g.ctx)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
	// Make channels
 | 
					 | 
				
			||||||
	g.shutdownRequested = make(chan struct{})
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
	// Set the running state
 | 
					 | 
				
			||||||
	if !g.setStateTransition(stateInit, stateRunning) {
 | 
					 | 
				
			||||||
		panic("invalid graceful manager state: transition from init to running failed")
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
	if skip, _ := strconv.ParseBool(os.Getenv("SKIP_MINWINSVC")); skip {
 | 
						if skip, _ := strconv.ParseBool(os.Getenv("SKIP_MINWINSVC")); skip {
 | 
				
			||||||
		log.Trace("Skipping SVC check as SKIP_MINWINSVC is set")
 | 
							log.Trace("Skipping SVC check as SKIP_MINWINSVC is set")
 | 
				
			||||||
		return
 | 
							return
 | 
				
			||||||
@@ -201,30 +146,6 @@ hammerLoop:
 | 
				
			|||||||
	return false, 0
 | 
						return false, 0
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
// DoImmediateHammer causes an immediate hammer
 | 
					 | 
				
			||||||
func (g *Manager) DoImmediateHammer() {
 | 
					 | 
				
			||||||
	g.doHammerTime(0 * time.Second)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
// DoGracefulShutdown causes a graceful shutdown
 | 
					 | 
				
			||||||
func (g *Manager) DoGracefulShutdown() {
 | 
					 | 
				
			||||||
	g.lock.Lock()
 | 
					 | 
				
			||||||
	select {
 | 
					 | 
				
			||||||
	case <-g.shutdownRequested:
 | 
					 | 
				
			||||||
		g.lock.Unlock()
 | 
					 | 
				
			||||||
	default:
 | 
					 | 
				
			||||||
		close(g.shutdownRequested)
 | 
					 | 
				
			||||||
		g.lock.Unlock()
 | 
					 | 
				
			||||||
		g.doShutdown()
 | 
					 | 
				
			||||||
	}
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
// RegisterServer registers the running of a listening server.
 | 
					 | 
				
			||||||
// Any call to RegisterServer must be matched by a call to ServerDone
 | 
					 | 
				
			||||||
func (g *Manager) RegisterServer() {
 | 
					 | 
				
			||||||
	g.runningServerWaitGroup.Add(1)
 | 
					 | 
				
			||||||
}
 | 
					 | 
				
			||||||
 | 
					 | 
				
			||||||
func (g *Manager) awaitServer(limit time.Duration) bool {
 | 
					func (g *Manager) awaitServer(limit time.Duration) bool {
 | 
				
			||||||
	c := make(chan struct{})
 | 
						c := make(chan struct{})
 | 
				
			||||||
	go func() {
 | 
						go func() {
 | 
				
			||||||
@@ -249,3 +170,11 @@ func (g *Manager) awaitServer(limit time.Duration) bool {
 | 
				
			|||||||
		}
 | 
							}
 | 
				
			||||||
	}
 | 
						}
 | 
				
			||||||
}
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func (g *Manager) notify(msg systemdNotifyMsg) {
 | 
				
			||||||
 | 
						// Windows doesn't use systemd to notify
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					func KillParent() {
 | 
				
			||||||
 | 
						// Windows doesn't need to "kill parent" because there is no graceful restart
 | 
				
			||||||
 | 
					}
 | 
				
			||||||
 
 | 
				
			|||||||
		Reference in New Issue
	
	Block a user