diff --git a/modules/graceful/manager_common.go b/modules/graceful/manager_common.go
new file mode 100644
index 0000000000..aaf008670c
--- /dev/null
+++ b/modules/graceful/manager_common.go
@@ -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)
+}
diff --git a/modules/graceful/manager_unix.go b/modules/graceful/manager_unix.go
index bdf23a4fde..f4af4993d9 100644
--- a/modules/graceful/manager_unix.go
+++ b/modules/graceful/manager_unix.go
@@ -12,7 +12,6 @@ import (
 	"os/signal"
 	"runtime/pprof"
 	"strconv"
-	"sync"
 	"syscall"
 	"time"
 
@@ -22,51 +21,6 @@ import (
 	"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 {
 	return systemdNotifyMsg("MAINPID=" + strconv.Itoa(os.Getpid()))
 }
@@ -89,27 +43,13 @@ func (g *Manager) notify(msg systemdNotifyMsg) {
 	}
 }
 
-func (g *Manager) start(ctx context.Context) {
-	// 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"))
-
+func (g *Manager) start() {
 	// Now label this and all goroutines created by this goroutine with the graceful-lifecycle manager
 	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(pidMsg())
 	go g.handleSignals(g.managerCtx)
@@ -118,11 +58,9 @@ func (g *Manager) start(ctx context.Context) {
 	startupDone := make(chan struct{})
 	go func() {
 		defer close(startupDone)
-		// Wait till we're done getting all of the listeners and then close
-		// the unused ones
+		// Wait till we're done getting all the listeners and then close the unused ones
 		g.createServerWaitGroup.Wait()
-		// Ignore the error here there's not much we can do with it
-		// They're logged in the CloseProvidedListeners function
+		// Ignore the error here there's not much we can do with it, they're logged in the CloseProvidedListeners function
 		_ = CloseProvidedListeners()
 		g.notify(readyMsg)
 	}()
@@ -133,7 +71,7 @@ func (g *Manager) start(ctx context.Context) {
 				return
 			case <-g.IsShutdown():
 				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() {
 						_ = recover()
 					}()
@@ -255,29 +193,3 @@ func (g *Manager) DoGracefulRestart() {
 		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)
-}
diff --git a/modules/graceful/manager_windows.go b/modules/graceful/manager_windows.go
index c2ea5383cc..0248dcb24d 100644
--- a/modules/graceful/manager_windows.go
+++ b/modules/graceful/manager_windows.go
@@ -7,11 +7,9 @@
 package graceful
 
 import (
-	"context"
 	"os"
 	"runtime/pprof"
 	"strconv"
-	"sync"
 	"time"
 
 	"code.gitea.io/gitea/modules/log"
@@ -30,64 +28,11 @@ const (
 	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() {
-	// 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
 	pprof.SetGoroutineLabels(g.managerCtx)
 	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 {
 		log.Trace("Skipping SVC check as SKIP_MINWINSVC is set")
 		return
@@ -201,30 +146,6 @@ hammerLoop:
 	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 {
 	c := make(chan struct{})
 	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
+}