]> source.dussan.org Git - gitea.git/commitdiff
Add Graceful shutdown for Windows and hooks for shutdown of goroutines (#8964)
authorzeripath <art27@cantab.net>
Thu, 21 Nov 2019 18:32:02 +0000 (18:32 +0000)
committertechknowlogick <techknowlogick@gitea.io>
Thu, 21 Nov 2019 18:32:02 +0000 (13:32 -0500)
* Graceful Shutdown for windows and others

Restructures modules/graceful, adding shutdown for windows, removing and
replacing the old minwinsvc code.

Creates a new waitGroup - terminate which allows for goroutines to
finish up after the shutdown of the servers.

Shutdown and terminate hooks are added for goroutines.

* Remove unused functions - these can be added in a different PR

* Add startup timeout functionality

* Document STARTUP_TIMEOUT

32 files changed:
cmd/web.go
cmd/web_graceful.go
cmd/web_windows.go [deleted file]
custom/conf/app.ini.sample
docs/content/doc/advanced/config-cheat-sheet.en-us.md
models/repo_indexer.go
modules/graceful/cleanup.go [deleted file]
modules/graceful/graceful_windows.go [deleted file]
modules/graceful/manager.go [new file with mode: 0644]
modules/graceful/manager_unix.go [new file with mode: 0644]
modules/graceful/manager_windows.go [new file with mode: 0644]
modules/graceful/net.go [deleted file]
modules/graceful/net_unix.go [new file with mode: 0644]
modules/graceful/net_windows.go [new file with mode: 0644]
modules/graceful/restart.go [deleted file]
modules/graceful/restart_unix.go [new file with mode: 0644]
modules/graceful/server.go
modules/graceful/server_hooks.go
modules/graceful/server_http.go
modules/graceful/server_signals.go [deleted file]
modules/indexer/issues/indexer.go
modules/minwinsvc/LICENSE [deleted file]
modules/minwinsvc/README.md [deleted file]
modules/minwinsvc/minwinsvc.go [deleted file]
modules/minwinsvc/svc_other.go [deleted file]
modules/minwinsvc/svc_windows.go [deleted file]
modules/setting/setting.go
modules/ssh/ssh_graceful.go
modules/ssh/ssh_windows.go [deleted file]
vendor/golang.org/x/sys/windows/svc/debug/log.go [new file with mode: 0644]
vendor/golang.org/x/sys/windows/svc/debug/service.go [new file with mode: 0644]
vendor/modules.txt

index 3ca4041a7de340abe0367996fda292b90c3a129b..22a7f9082db127663e76fe2bf883006492a6ce89 100644 (file)
@@ -227,7 +227,8 @@ func runWeb(ctx *cli.Context) error {
                log.Critical("Failed to start server: %v", err)
        }
        log.Info("HTTP Listener: %s Closed", listenAddr)
-       graceful.WaitForServers()
+       graceful.Manager.WaitForServers()
+       graceful.Manager.WaitForTerminate()
        log.Close()
        return nil
 }
index 07b5a964c5461c90a769c1ad8e65a41f9d09fbfb..a37f669d0909333ceb2d806bade2a25f9a5a1b3a 100644 (file)
@@ -1,5 +1,3 @@
-// +build !windows
-
 // Copyright 2016 The Gitea Authors. All rights reserved.
 // Use of this source code is governed by a MIT-style
 // license that can be found in the LICENSE file.
@@ -27,11 +25,11 @@ func runHTTPSWithTLSConfig(listenAddr string, tlsConfig *tls.Config, m http.Hand
 
 // NoHTTPRedirector tells our cleanup routine that we will not be using a fallback http redirector
 func NoHTTPRedirector() {
-       graceful.InformCleanup()
+       graceful.Manager.InformCleanup()
 }
 
 // NoMainListener tells our cleanup routine that we will not be using a possibly provided listener
 // for our main HTTP/HTTPS service
 func NoMainListener() {
-       graceful.InformCleanup()
+       graceful.Manager.InformCleanup()
 }
diff --git a/cmd/web_windows.go b/cmd/web_windows.go
deleted file mode 100644 (file)
index cdd2cc5..0000000
+++ /dev/null
@@ -1,37 +0,0 @@
-// +build windows
-
-// Copyright 2016 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-
-package cmd
-
-import (
-       "crypto/tls"
-       "net/http"
-)
-
-func runHTTP(listenAddr string, m http.Handler) error {
-       return http.ListenAndServe(listenAddr, m)
-}
-
-func runHTTPS(listenAddr, certFile, keyFile string, m http.Handler) error {
-       return http.ListenAndServeTLS(listenAddr, certFile, keyFile, m)
-}
-
-func runHTTPSWithTLSConfig(listenAddr string, tlsConfig *tls.Config, m http.Handler) error {
-       server := &http.Server{
-               Addr:      listenAddr,
-               Handler:   m,
-               TLSConfig: tlsConfig,
-       }
-       return server.ListenAndServeTLS("", "")
-}
-
-// NoHTTPRedirector is a no-op on Windows
-func NoHTTPRedirector() {
-}
-
-// NoMainListener is a no-op on Windows
-func NoMainListener() {
-}
index 599498a4b62c5afa359ba6f2654fb1f404e6b8ab..53488dfd477eee6bf4a3bdf6858cb41e4edb2bae 100644 (file)
@@ -287,6 +287,9 @@ ALLOW_GRACEFUL_RESTARTS = true
 ; shutting down. Force shutdown if this process takes longer than this delay.
 ; set to a negative value to disable
 GRACEFUL_HAMMER_TIME = 60s
+; Allows the setting of a startup timeout and waithint for Windows as SVC service
+; 0 disables this.
+STARTUP_TIMEOUT = 0
 ; Static resources, includes resources on custom/, public/ and all uploaded avatars web browser cache time, default is 6h
 STATIC_CACHE_TIME = 6h
 
@@ -897,4 +900,4 @@ QUEUE_CONN_STR = "addrs=127.0.0.1:6379 db=0"
 ; Max attempts per http/https request on migrations.
 MAX_ATTEMPTS = 3
 ; Backoff time per http/https request retry (seconds)
-RETRY_BACKOFF = 3
\ No newline at end of file
+RETRY_BACKOFF = 3
index 327efb34bd0974fbe942f3b259548d8e6a9a519d..11c0686c512e01f47d8dee2543195706f8ffaabf 100644 (file)
@@ -189,6 +189,7 @@ Values containing `#` or `;` must be quoted using `` ` `` or `"""`.
 - `LETSENCRYPT_EMAIL`: **email@example.com**: Email used by Letsencrypt to notify about problems with issued certificates. (No default)
 - `ALLOW_GRACEFUL_RESTARTS`: **true**: Perform a graceful restart on SIGHUP
 - `GRACEFUL_HAMMER_TIME`: **60s**: After a restart the parent process will stop accepting new connections and will allow requests to finish before stopping. Shutdown will be forced if it takes longer than this time.
+- `STARTUP_TIMEOUT`: **0**: Shutsdown the server if startup takes longer than the provided time. On Windows setting this sends a waithint to the SVC host to tell the SVC host startup may take some time. Please note startup is determined by the opening of the listeners - HTTP/HTTPS/SSH. Indexers may take longer to startup and can have their own timeouts.
 
 ## Database (`database`)
 
index 9cc002a8ab69df2c3a29d0591b4ae637e1e33798..7bfcde8db0bcec0fe9b39126bfab5de742ff0f52 100644 (file)
@@ -84,7 +84,7 @@ func InitRepoIndexer() {
        if setting.Indexer.StartupTimeout > 0 {
                go func() {
                        timeout := setting.Indexer.StartupTimeout
-                       if graceful.IsChild && setting.GracefulHammerTime > 0 {
+                       if graceful.Manager.IsChild() && setting.GracefulHammerTime > 0 {
                                timeout += setting.GracefulHammerTime
                        }
                        select {
diff --git a/modules/graceful/cleanup.go b/modules/graceful/cleanup.go
deleted file mode 100644 (file)
index 84355a9..0000000
+++ /dev/null
@@ -1,40 +0,0 @@
-// +build !windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-
-package graceful
-
-import "sync"
-
-var cleanupWaitGroup sync.WaitGroup
-
-func init() {
-       cleanupWaitGroup = sync.WaitGroup{}
-
-       // There are three places that could inherit sockets:
-       //
-       // * HTTP or HTTPS main listener
-       // * HTTP redirection fallback
-       // * SSH
-       //
-       // If you add an additional place you must increment this number
-       // and add a function to call InformCleanup if it's not going to be used
-       cleanupWaitGroup.Add(3)
-
-       // Wait till we're done getting all of the listeners and then close
-       // the unused ones
-       go func() {
-               cleanupWaitGroup.Wait()
-               // Ignore the error here there's not much we can do with it
-               // They're logged in the CloseProvidedListeners function
-               _ = CloseProvidedListeners()
-       }()
-}
-
-// InformCleanup tells the cleanup wait group that we have either taken a listener
-// or will not be taking a listener
-func InformCleanup() {
-       cleanupWaitGroup.Done()
-}
diff --git a/modules/graceful/graceful_windows.go b/modules/graceful/graceful_windows.go
deleted file mode 100644 (file)
index 281b255..0000000
+++ /dev/null
@@ -1,16 +0,0 @@
-// +build windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
-
-package graceful
-
-// This file contains shims for windows builds
-const IsChild = false
-
-// WaitForServers waits for all running servers to finish
-func WaitForServers() {
-
-}
diff --git a/modules/graceful/manager.go b/modules/graceful/manager.go
new file mode 100644 (file)
index 0000000..48f7663
--- /dev/null
@@ -0,0 +1,187 @@
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+
+package graceful
+
+import (
+       "time"
+
+       "code.gitea.io/gitea/modules/log"
+       "code.gitea.io/gitea/modules/setting"
+)
+
+type state uint8
+
+const (
+       stateInit state = iota
+       stateRunning
+       stateShuttingDown
+       stateTerminate
+)
+
+// There are three places that could inherit sockets:
+//
+// * HTTP or HTTPS main listener
+// * HTTP redirection fallback
+// * SSH
+//
+// If you add an additional place you must increment this number
+// and add a function to call manager.InformCleanup if it's not going to be used
+const numberOfServersToCreate = 3
+
+// Manager represents the graceful server manager interface
+var Manager *gracefulManager
+
+func init() {
+       Manager = newGracefulManager()
+}
+
+func (g *gracefulManager) doShutdown() {
+       if !g.setStateTransition(stateRunning, stateShuttingDown) {
+               return
+       }
+       g.lock.Lock()
+       close(g.shutdown)
+       g.lock.Unlock()
+
+       if setting.GracefulHammerTime >= 0 {
+               go g.doHammerTime(setting.GracefulHammerTime)
+       }
+       go func() {
+               g.WaitForServers()
+               <-time.After(1 * time.Second)
+               g.doTerminate()
+       }()
+}
+
+func (g *gracefulManager) doHammerTime(d time.Duration) {
+       time.Sleep(d)
+       select {
+       case <-g.hammer:
+       default:
+               log.Warn("Setting Hammer condition")
+               close(g.hammer)
+       }
+
+}
+
+func (g *gracefulManager) doTerminate() {
+       if !g.setStateTransition(stateShuttingDown, stateTerminate) {
+               return
+       }
+       g.lock.Lock()
+       close(g.terminate)
+       g.lock.Unlock()
+}
+
+// IsChild returns if the current process is a child of previous Gitea process
+func (g *gracefulManager) IsChild() bool {
+       return g.isChild
+}
+
+// IsShutdown returns a channel which will be closed at shutdown.
+// The order of closure is IsShutdown, IsHammer (potentially), IsTerminate
+func (g *gracefulManager) IsShutdown() <-chan struct{} {
+       g.lock.RLock()
+       if g.shutdown == nil {
+               g.lock.RUnlock()
+               g.lock.Lock()
+               if g.shutdown == nil {
+                       g.shutdown = make(chan struct{})
+               }
+               defer g.lock.Unlock()
+               return g.shutdown
+       }
+       defer g.lock.RUnlock()
+       return g.shutdown
+}
+
+// IsHammer returns a channel which will be closed at hammer
+// The order of closure is IsShutdown, IsHammer (potentially), IsTerminate
+// Servers running within the running server wait group should respond to IsHammer
+// if not shutdown already
+func (g *gracefulManager) IsHammer() <-chan struct{} {
+       g.lock.RLock()
+       if g.hammer == nil {
+               g.lock.RUnlock()
+               g.lock.Lock()
+               if g.hammer == nil {
+                       g.hammer = make(chan struct{})
+               }
+               defer g.lock.Unlock()
+               return g.hammer
+       }
+       defer g.lock.RUnlock()
+       return g.hammer
+}
+
+// IsTerminate returns a channel which will be closed at terminate
+// The order of closure is IsShutdown, IsHammer (potentially), IsTerminate
+// IsTerminate will only close once all running servers have stopped
+func (g *gracefulManager) IsTerminate() <-chan struct{} {
+       g.lock.RLock()
+       if g.terminate == nil {
+               g.lock.RUnlock()
+               g.lock.Lock()
+               if g.terminate == nil {
+                       g.terminate = make(chan struct{})
+               }
+               defer g.lock.Unlock()
+               return g.terminate
+       }
+       defer g.lock.RUnlock()
+       return g.terminate
+}
+
+// ServerDone declares a running server done and subtracts one from the
+// running server wait group. Users probably do not want to call this
+// and should use one of the RunWithShutdown* functions
+func (g *gracefulManager) ServerDone() {
+       g.runningServerWaitGroup.Done()
+}
+
+// WaitForServers waits for all running servers to finish. Users should probably
+// instead use AtTerminate or IsTerminate
+func (g *gracefulManager) WaitForServers() {
+       g.runningServerWaitGroup.Wait()
+}
+
+// WaitForTerminate waits for all terminating actions to finish.
+// Only the main go-routine should use this
+func (g *gracefulManager) WaitForTerminate() {
+       g.terminateWaitGroup.Wait()
+}
+
+func (g *gracefulManager) getState() state {
+       g.lock.RLock()
+       defer g.lock.RUnlock()
+       return g.state
+}
+
+func (g *gracefulManager) setStateTransition(old, new state) bool {
+       if old != g.getState() {
+               return false
+       }
+       g.lock.Lock()
+       if g.state != old {
+               g.lock.Unlock()
+               return false
+       }
+       g.state = new
+       g.lock.Unlock()
+       return true
+}
+
+func (g *gracefulManager) setState(st state) {
+       g.lock.Lock()
+       defer g.lock.Unlock()
+
+       g.state = st
+}
+
+// InformCleanup tells the cleanup wait group that we have either taken a listener
+// or will not be taking a listener
+func (g *gracefulManager) InformCleanup() {
+       g.createServerWaitGroup.Done()
+}
diff --git a/modules/graceful/manager_unix.go b/modules/graceful/manager_unix.go
new file mode 100644 (file)
index 0000000..15b0ff4
--- /dev/null
@@ -0,0 +1,141 @@
+// +build !windows
+
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+
+package graceful
+
+import (
+       "errors"
+       "os"
+       "os/signal"
+       "sync"
+       "syscall"
+       "time"
+
+       "code.gitea.io/gitea/modules/log"
+       "code.gitea.io/gitea/modules/setting"
+)
+
+type gracefulManager struct {
+       isChild                bool
+       forked                 bool
+       lock                   *sync.RWMutex
+       state                  state
+       shutdown               chan struct{}
+       hammer                 chan struct{}
+       terminate              chan struct{}
+       runningServerWaitGroup sync.WaitGroup
+       createServerWaitGroup  sync.WaitGroup
+       terminateWaitGroup     sync.WaitGroup
+}
+
+func newGracefulManager() *gracefulManager {
+       manager := &gracefulManager{
+               isChild: len(os.Getenv(listenFDs)) > 0 && os.Getppid() > 1,
+               lock:    &sync.RWMutex{},
+       }
+       manager.createServerWaitGroup.Add(numberOfServersToCreate)
+       manager.Run()
+       return manager
+}
+
+func (g *gracefulManager) Run() {
+       g.setState(stateRunning)
+       go g.handleSignals()
+       c := make(chan struct{})
+       go func() {
+               defer close(c)
+               // Wait till we're done getting all of 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
+               _ = CloseProvidedListeners()
+       }()
+       if setting.StartupTimeout > 0 {
+               go func() {
+                       select {
+                       case <-c:
+                               return
+                       case <-g.IsShutdown():
+                               return
+                       case <-time.After(setting.StartupTimeout):
+                               log.Error("Startup took too long! Shutting down")
+                               g.doShutdown()
+                       }
+               }()
+       }
+}
+
+func (g *gracefulManager) handleSignals() {
+       var sig os.Signal
+
+       signalChannel := make(chan os.Signal, 1)
+
+       signal.Notify(
+               signalChannel,
+               syscall.SIGHUP,
+               syscall.SIGUSR1,
+               syscall.SIGUSR2,
+               syscall.SIGINT,
+               syscall.SIGTERM,
+               syscall.SIGTSTP,
+       )
+
+       pid := syscall.Getpid()
+       for {
+               sig = <-signalChannel
+               switch sig {
+               case syscall.SIGHUP:
+                       if setting.GracefulRestartable {
+                               log.Info("PID: %d. Received SIGHUP. Forking...", pid)
+                               err := g.doFork()
+                               if err != nil && err.Error() != "another process already forked. Ignoring this one" {
+                                       log.Error("Error whilst forking from PID: %d : %v", pid, err)
+                               }
+                       } else {
+                               log.Info("PID: %d. Received SIGHUP. Not set restartable. Shutting down...", pid)
+
+                               g.doShutdown()
+                       }
+               case syscall.SIGUSR1:
+                       log.Info("PID %d. Received SIGUSR1.", pid)
+               case syscall.SIGUSR2:
+                       log.Warn("PID %d. Received SIGUSR2. Hammering...", pid)
+                       g.doHammerTime(0 * time.Second)
+               case syscall.SIGINT:
+                       log.Warn("PID %d. Received SIGINT. Shutting down...", pid)
+                       g.doShutdown()
+               case syscall.SIGTERM:
+                       log.Warn("PID %d. Received SIGTERM. Shutting down...", pid)
+                       g.doShutdown()
+               case syscall.SIGTSTP:
+                       log.Info("PID %d. Received SIGTSTP.", pid)
+               default:
+                       log.Info("PID %d. Received %v.", pid, sig)
+               }
+       }
+}
+
+func (g *gracefulManager) doFork() error {
+       g.lock.Lock()
+       if g.forked {
+               g.lock.Unlock()
+               return errors.New("another process already forked. Ignoring this one")
+       }
+       g.forked = true
+       g.lock.Unlock()
+       // We need to move the file logs to append pids
+       setting.RestartLogsWithPIDSuffix()
+
+       _, err := RestartProcess()
+
+       return err
+}
+
+func (g *gracefulManager) RegisterServer() {
+       KillParent()
+       g.runningServerWaitGroup.Add(1)
+}
diff --git a/modules/graceful/manager_windows.go b/modules/graceful/manager_windows.go
new file mode 100644 (file)
index 0000000..ab21314
--- /dev/null
@@ -0,0 +1,162 @@
+// +build windows
+
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
+
+package graceful
+
+import (
+       "os"
+       "strconv"
+       "sync"
+       "time"
+
+       "code.gitea.io/gitea/modules/log"
+       "code.gitea.io/gitea/modules/setting"
+
+       "golang.org/x/sys/windows/svc"
+       "golang.org/x/sys/windows/svc/debug"
+)
+
+var WindowsServiceName = "gitea"
+
+const (
+       hammerCode       = 128
+       hammerCmd        = svc.Cmd(hammerCode)
+       acceptHammerCode = svc.Accepted(hammerCode)
+)
+
+
+type gracefulManager struct {
+       isChild                bool
+       lock                   *sync.RWMutex
+       state                  state
+       shutdown               chan struct{}
+       hammer                 chan struct{}
+       terminate              chan struct{}
+       runningServerWaitGroup sync.WaitGroup
+       createServerWaitGroup  sync.WaitGroup
+       terminateWaitGroup     sync.WaitGroup
+}
+
+func newGracefulManager() *gracefulManager {
+       manager := &gracefulManager{
+               isChild: false,
+               lock:    &sync.RWMutex{},
+       }
+       manager.createServerWaitGroup.Add(numberOfServersToCreate)
+       manager.Run()
+       return manager
+}
+
+func (g *gracefulManager) Run() {
+       g.setState(stateRunning)
+       if skip, _ := strconv.ParseBool(os.Getenv("SKIP_MINWINSVC")); skip {
+               return
+       }
+       run := svc.Run
+       isInteractive, err := svc.IsAnInteractiveSession()
+       if err != nil {
+               log.Error("Unable to ascertain if running as an Interactive Session: %v", err)
+               return
+       }
+       if isInteractive {
+               run = debug.Run
+       }
+       go run(WindowsServiceName, g)
+}
+
+// Execute makes gracefulManager implement svc.Handler
+func (g *gracefulManager) Execute(args []string, changes <-chan svc.ChangeRequest, status chan<- svc.Status) (svcSpecificEC bool, exitCode uint32) {
+       if setting.StartupTimeout > 0 {
+               status <- svc.Status{State: svc.StartPending}
+       } else {
+               status <- svc.Status{State: svc.StartPending, WaitHint: uint32(setting.StartupTimeout/time.Millisecond)}
+       }
+
+       // Now need to wait for everything to start...
+       if !g.awaitServer(setting.StartupTimeout) {
+               return false, 1
+       }
+
+       // We need to implement some way of svc.AcceptParamChange/svc.ParamChange
+       status <- svc.Status{
+               State:   svc.Running,
+               Accepts: svc.AcceptStop | svc.AcceptShutdown | acceptHammerCode,
+       }
+
+       waitTime := 30 * time.Second
+
+loop:
+       for change := range changes {
+               switch change.Cmd {
+               case svc.Interrogate:
+                       status <- change.CurrentStatus
+               case svc.Stop, svc.Shutdown:
+                       g.doShutdown()
+                       waitTime += setting.GracefulHammerTime
+                       break loop
+               case hammerCode:
+                       g.doShutdown()
+                       g.doHammerTime(0 *time.Second)
+                       break loop
+               default:
+                       log.Debug("Unexpected control request: %v", change.Cmd)
+               }
+       }
+
+       status <- svc.Status{
+               State: svc.StopPending,
+               WaitHint: uint32(waitTime/time.Millisecond),
+       }
+
+hammerLoop:
+       for {
+               select {
+               case change := <-changes:
+                       switch change.Cmd {
+                       case svc.Interrogate:
+                               status <- change.CurrentStatus
+                       case svc.Stop, svc.Shutdown, hammerCmd:
+                               g.doHammerTime(0 * time.Second)
+                               break hammerLoop
+                       default:
+                               log.Debug("Unexpected control request: %v", change.Cmd)
+                       }
+               case <-g.hammer:
+                       break hammerLoop
+               }
+       }
+       return false, 0
+}
+
+func (g *gracefulManager) RegisterServer() {
+       g.runningServerWaitGroup.Add(1)
+}
+
+func (g *gracefulManager) awaitServer(limit time.Duration) bool {
+       c := make(chan struct{})
+       go func() {
+               defer close(c)
+               g.createServerWaitGroup.Wait()
+       }()
+       if limit > 0 {
+               select {
+               case <-c:
+                       return true // completed normally
+               case <-time.After(limit):
+                       return false // timed out
+               case <-g.IsShutdown():
+                       return false
+               }
+       } else {
+               select {
+               case <-c:
+                       return true // completed normally
+               case <-g.IsShutdown():
+                       return false
+               }
+       }
+}
diff --git a/modules/graceful/net.go b/modules/graceful/net.go
deleted file mode 100644 (file)
index af48464..0000000
+++ /dev/null
@@ -1,211 +0,0 @@
-// +build !windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
-
-package graceful
-
-import (
-       "fmt"
-       "net"
-       "os"
-       "strconv"
-       "strings"
-       "sync"
-
-       "code.gitea.io/gitea/modules/log"
-)
-
-const (
-       listenFDs = "LISTEN_FDS"
-       startFD   = 3
-)
-
-// In order to keep the working directory the same as when we started we record
-// it at startup.
-var originalWD, _ = os.Getwd()
-
-var (
-       once  = sync.Once{}
-       mutex = sync.Mutex{}
-
-       providedListeners = []net.Listener{}
-       activeListeners   = []net.Listener{}
-)
-
-func getProvidedFDs() (savedErr error) {
-       // Only inherit the provided FDS once but we will save the error so that repeated calls to this function will return the same error
-       once.Do(func() {
-               mutex.Lock()
-               defer mutex.Unlock()
-
-               numFDs := os.Getenv(listenFDs)
-               if numFDs == "" {
-                       return
-               }
-               n, err := strconv.Atoi(numFDs)
-               if err != nil {
-                       savedErr = fmt.Errorf("%s is not a number: %s. Err: %v", listenFDs, numFDs, err)
-                       return
-               }
-
-               for i := startFD; i < n+startFD; i++ {
-                       file := os.NewFile(uintptr(i), fmt.Sprintf("listener_FD%d", i))
-
-                       l, err := net.FileListener(file)
-                       if err == nil {
-                               // Close the inherited file if it's a listener
-                               if err = file.Close(); err != nil {
-                                       savedErr = fmt.Errorf("error closing provided socket fd %d: %s", i, err)
-                                       return
-                               }
-                               providedListeners = append(providedListeners, l)
-                               continue
-                       }
-
-                       // If needed we can handle packetconns here.
-                       savedErr = fmt.Errorf("Error getting provided socket fd %d: %v", i, err)
-                       return
-               }
-       })
-       return savedErr
-}
-
-// CloseProvidedListeners closes all unused provided listeners.
-func CloseProvidedListeners() error {
-       mutex.Lock()
-       defer mutex.Unlock()
-       var returnableError error
-       for _, l := range providedListeners {
-               err := l.Close()
-               if err != nil {
-                       log.Error("Error in closing unused provided listener: %v", err)
-                       if returnableError != nil {
-                               returnableError = fmt.Errorf("%v & %v", returnableError, err)
-                       } else {
-                               returnableError = err
-                       }
-               }
-       }
-       providedListeners = []net.Listener{}
-
-       return returnableError
-}
-
-// GetListener obtains a listener for the local network address. The network must be
-// a stream-oriented network: "tcp", "tcp4", "tcp6", "unix" or "unixpacket". It
-// returns an provided net.Listener for the matching network and address, or
-// creates a new one using net.Listen.
-func GetListener(network, address string) (net.Listener, error) {
-       // Add a deferral to say that we've tried to grab a listener
-       defer InformCleanup()
-       switch network {
-       case "tcp", "tcp4", "tcp6":
-               tcpAddr, err := net.ResolveTCPAddr(network, address)
-               if err != nil {
-                       return nil, err
-               }
-               return GetListenerTCP(network, tcpAddr)
-       case "unix", "unixpacket":
-               unixAddr, err := net.ResolveUnixAddr(network, address)
-               if err != nil {
-                       return nil, err
-               }
-               return GetListenerUnix(network, unixAddr)
-       default:
-               return nil, net.UnknownNetworkError(network)
-       }
-}
-
-// GetListenerTCP announces on the local network address. The network must be:
-// "tcp", "tcp4" or "tcp6". It returns a provided net.Listener for the
-// matching network and address, or creates a new one using net.ListenTCP.
-func GetListenerTCP(network string, address *net.TCPAddr) (*net.TCPListener, error) {
-       if err := getProvidedFDs(); err != nil {
-               return nil, err
-       }
-
-       mutex.Lock()
-       defer mutex.Unlock()
-
-       // look for a provided listener
-       for i, l := range providedListeners {
-               if isSameAddr(l.Addr(), address) {
-                       providedListeners = append(providedListeners[:i], providedListeners[i+1:]...)
-
-                       activeListeners = append(activeListeners, l)
-                       return l.(*net.TCPListener), nil
-               }
-       }
-
-       // no provided listener for this address -> make a fresh listener
-       l, err := net.ListenTCP(network, address)
-       if err != nil {
-               return nil, err
-       }
-       activeListeners = append(activeListeners, l)
-       return l, nil
-}
-
-// GetListenerUnix announces on the local network address. The network must be:
-// "unix" or "unixpacket". It returns a provided net.Listener for the
-// matching network and address, or creates a new one using net.ListenUnix.
-func GetListenerUnix(network string, address *net.UnixAddr) (*net.UnixListener, error) {
-       if err := getProvidedFDs(); err != nil {
-               return nil, err
-       }
-
-       mutex.Lock()
-       defer mutex.Unlock()
-
-       // look for a provided listener
-       for i, l := range providedListeners {
-               if isSameAddr(l.Addr(), address) {
-                       providedListeners = append(providedListeners[:i], providedListeners[i+1:]...)
-                       activeListeners = append(activeListeners, l)
-                       return l.(*net.UnixListener), nil
-               }
-       }
-
-       // make a fresh listener
-       l, err := net.ListenUnix(network, address)
-       if err != nil {
-               return nil, err
-       }
-       activeListeners = append(activeListeners, l)
-       return l, nil
-}
-
-func isSameAddr(a1, a2 net.Addr) bool {
-       // If the addresses are not on the same network fail.
-       if a1.Network() != a2.Network() {
-               return false
-       }
-
-       // If the two addresses have the same string representation they're equal
-       a1s := a1.String()
-       a2s := a2.String()
-       if a1s == a2s {
-               return true
-       }
-
-       // This allows for ipv6 vs ipv4 local addresses to compare as equal. This
-       // scenario is common when listening on localhost.
-       const ipv6prefix = "[::]"
-       a1s = strings.TrimPrefix(a1s, ipv6prefix)
-       a2s = strings.TrimPrefix(a2s, ipv6prefix)
-       const ipv4prefix = "0.0.0.0"
-       a1s = strings.TrimPrefix(a1s, ipv4prefix)
-       a2s = strings.TrimPrefix(a2s, ipv4prefix)
-       return a1s == a2s
-}
-
-func getActiveListeners() []net.Listener {
-       mutex.Lock()
-       defer mutex.Unlock()
-       listeners := make([]net.Listener, len(activeListeners))
-       copy(listeners, activeListeners)
-       return listeners
-}
diff --git a/modules/graceful/net_unix.go b/modules/graceful/net_unix.go
new file mode 100644 (file)
index 0000000..2b8efe0
--- /dev/null
@@ -0,0 +1,211 @@
+// +build !windows
+
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
+
+package graceful
+
+import (
+       "fmt"
+       "net"
+       "os"
+       "strconv"
+       "strings"
+       "sync"
+
+       "code.gitea.io/gitea/modules/log"
+)
+
+const (
+       listenFDs = "LISTEN_FDS"
+       startFD   = 3
+)
+
+// In order to keep the working directory the same as when we started we record
+// it at startup.
+var originalWD, _ = os.Getwd()
+
+var (
+       once  = sync.Once{}
+       mutex = sync.Mutex{}
+
+       providedListeners = []net.Listener{}
+       activeListeners   = []net.Listener{}
+)
+
+func getProvidedFDs() (savedErr error) {
+       // Only inherit the provided FDS once but we will save the error so that repeated calls to this function will return the same error
+       once.Do(func() {
+               mutex.Lock()
+               defer mutex.Unlock()
+
+               numFDs := os.Getenv(listenFDs)
+               if numFDs == "" {
+                       return
+               }
+               n, err := strconv.Atoi(numFDs)
+               if err != nil {
+                       savedErr = fmt.Errorf("%s is not a number: %s. Err: %v", listenFDs, numFDs, err)
+                       return
+               }
+
+               for i := startFD; i < n+startFD; i++ {
+                       file := os.NewFile(uintptr(i), fmt.Sprintf("listener_FD%d", i))
+
+                       l, err := net.FileListener(file)
+                       if err == nil {
+                               // Close the inherited file if it's a listener
+                               if err = file.Close(); err != nil {
+                                       savedErr = fmt.Errorf("error closing provided socket fd %d: %s", i, err)
+                                       return
+                               }
+                               providedListeners = append(providedListeners, l)
+                               continue
+                       }
+
+                       // If needed we can handle packetconns here.
+                       savedErr = fmt.Errorf("Error getting provided socket fd %d: %v", i, err)
+                       return
+               }
+       })
+       return savedErr
+}
+
+// CloseProvidedListeners closes all unused provided listeners.
+func CloseProvidedListeners() error {
+       mutex.Lock()
+       defer mutex.Unlock()
+       var returnableError error
+       for _, l := range providedListeners {
+               err := l.Close()
+               if err != nil {
+                       log.Error("Error in closing unused provided listener: %v", err)
+                       if returnableError != nil {
+                               returnableError = fmt.Errorf("%v & %v", returnableError, err)
+                       } else {
+                               returnableError = err
+                       }
+               }
+       }
+       providedListeners = []net.Listener{}
+
+       return returnableError
+}
+
+// GetListener obtains a listener for the local network address. The network must be
+// a stream-oriented network: "tcp", "tcp4", "tcp6", "unix" or "unixpacket". It
+// returns an provided net.Listener for the matching network and address, or
+// creates a new one using net.Listen.
+func GetListener(network, address string) (net.Listener, error) {
+       // Add a deferral to say that we've tried to grab a listener
+       defer Manager.InformCleanup()
+       switch network {
+       case "tcp", "tcp4", "tcp6":
+               tcpAddr, err := net.ResolveTCPAddr(network, address)
+               if err != nil {
+                       return nil, err
+               }
+               return GetListenerTCP(network, tcpAddr)
+       case "unix", "unixpacket":
+               unixAddr, err := net.ResolveUnixAddr(network, address)
+               if err != nil {
+                       return nil, err
+               }
+               return GetListenerUnix(network, unixAddr)
+       default:
+               return nil, net.UnknownNetworkError(network)
+       }
+}
+
+// GetListenerTCP announces on the local network address. The network must be:
+// "tcp", "tcp4" or "tcp6". It returns a provided net.Listener for the
+// matching network and address, or creates a new one using net.ListenTCP.
+func GetListenerTCP(network string, address *net.TCPAddr) (*net.TCPListener, error) {
+       if err := getProvidedFDs(); err != nil {
+               return nil, err
+       }
+
+       mutex.Lock()
+       defer mutex.Unlock()
+
+       // look for a provided listener
+       for i, l := range providedListeners {
+               if isSameAddr(l.Addr(), address) {
+                       providedListeners = append(providedListeners[:i], providedListeners[i+1:]...)
+
+                       activeListeners = append(activeListeners, l)
+                       return l.(*net.TCPListener), nil
+               }
+       }
+
+       // no provided listener for this address -> make a fresh listener
+       l, err := net.ListenTCP(network, address)
+       if err != nil {
+               return nil, err
+       }
+       activeListeners = append(activeListeners, l)
+       return l, nil
+}
+
+// GetListenerUnix announces on the local network address. The network must be:
+// "unix" or "unixpacket". It returns a provided net.Listener for the
+// matching network and address, or creates a new one using net.ListenUnix.
+func GetListenerUnix(network string, address *net.UnixAddr) (*net.UnixListener, error) {
+       if err := getProvidedFDs(); err != nil {
+               return nil, err
+       }
+
+       mutex.Lock()
+       defer mutex.Unlock()
+
+       // look for a provided listener
+       for i, l := range providedListeners {
+               if isSameAddr(l.Addr(), address) {
+                       providedListeners = append(providedListeners[:i], providedListeners[i+1:]...)
+                       activeListeners = append(activeListeners, l)
+                       return l.(*net.UnixListener), nil
+               }
+       }
+
+       // make a fresh listener
+       l, err := net.ListenUnix(network, address)
+       if err != nil {
+               return nil, err
+       }
+       activeListeners = append(activeListeners, l)
+       return l, nil
+}
+
+func isSameAddr(a1, a2 net.Addr) bool {
+       // If the addresses are not on the same network fail.
+       if a1.Network() != a2.Network() {
+               return false
+       }
+
+       // If the two addresses have the same string representation they're equal
+       a1s := a1.String()
+       a2s := a2.String()
+       if a1s == a2s {
+               return true
+       }
+
+       // This allows for ipv6 vs ipv4 local addresses to compare as equal. This
+       // scenario is common when listening on localhost.
+       const ipv6prefix = "[::]"
+       a1s = strings.TrimPrefix(a1s, ipv6prefix)
+       a2s = strings.TrimPrefix(a2s, ipv6prefix)
+       const ipv4prefix = "0.0.0.0"
+       a1s = strings.TrimPrefix(a1s, ipv4prefix)
+       a2s = strings.TrimPrefix(a2s, ipv4prefix)
+       return a1s == a2s
+}
+
+func getActiveListeners() []net.Listener {
+       mutex.Lock()
+       defer mutex.Unlock()
+       listeners := make([]net.Listener, len(activeListeners))
+       copy(listeners, activeListeners)
+       return listeners
+}
diff --git a/modules/graceful/net_windows.go b/modules/graceful/net_windows.go
new file mode 100644 (file)
index 0000000..e191eca
--- /dev/null
@@ -0,0 +1,19 @@
+// +build windows
+
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
+
+package graceful
+
+import "net"
+
+// GetListener obtains a listener for the local network address.
+// On windows this is basically just a shim around net.Listen.
+func GetListener(network, address string) (net.Listener, error) {
+       // Add a deferral to say that we've tried to grab a listener
+       defer Manager.InformCleanup()
+
+       return net.Listen(network, address)
+}
diff --git a/modules/graceful/restart.go b/modules/graceful/restart.go
deleted file mode 100644 (file)
index 04ee072..0000000
+++ /dev/null
@@ -1,85 +0,0 @@
-// +build !windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
-
-package graceful
-
-import (
-       "fmt"
-       "os"
-       "os/exec"
-       "strings"
-       "sync"
-       "syscall"
-)
-
-var killParent sync.Once
-
-// KillParent sends the kill signal to the parent process if we are a child
-func KillParent() {
-       killParent.Do(func() {
-               if IsChild {
-                       ppid := syscall.Getppid()
-                       if ppid > 1 {
-                               _ = syscall.Kill(ppid, syscall.SIGTERM)
-                       }
-               }
-       })
-}
-
-// RestartProcess starts a new process passing it the active listeners. It
-// doesn't fork, but starts a new process using the same environment and
-// arguments as when it was originally started. This allows for a newly
-// deployed binary to be started. It returns the pid of the newly started
-// process when successful.
-func RestartProcess() (int, error) {
-       listeners := getActiveListeners()
-
-       // Extract the fds from the listeners.
-       files := make([]*os.File, len(listeners))
-       for i, l := range listeners {
-               var err error
-               // Now, all our listeners actually have File() functions so instead of
-               // individually casting we just use a hacky interface
-               files[i], err = l.(filer).File()
-               if err != nil {
-                       return 0, err
-               }
-               // Remember to close these at the end.
-               defer files[i].Close()
-       }
-
-       // Use the original binary location. This works with symlinks such that if
-       // the file it points to has been changed we will use the updated symlink.
-       argv0, err := exec.LookPath(os.Args[0])
-       if err != nil {
-               return 0, err
-       }
-
-       // Pass on the environment and replace the old count key with the new one.
-       var env []string
-       for _, v := range os.Environ() {
-               if !strings.HasPrefix(v, listenFDs+"=") {
-                       env = append(env, v)
-               }
-       }
-       env = append(env, fmt.Sprintf("%s=%d", listenFDs, len(listeners)))
-
-       allFiles := append([]*os.File{os.Stdin, os.Stdout, os.Stderr}, files...)
-       process, err := os.StartProcess(argv0, os.Args, &os.ProcAttr{
-               Dir:   originalWD,
-               Env:   env,
-               Files: allFiles,
-       })
-       if err != nil {
-               return 0, err
-       }
-       return process.Pid, nil
-}
-
-type filer interface {
-       File() (*os.File, error)
-}
diff --git a/modules/graceful/restart_unix.go b/modules/graceful/restart_unix.go
new file mode 100644 (file)
index 0000000..8c68965
--- /dev/null
@@ -0,0 +1,81 @@
+// +build !windows
+
+// Copyright 2019 The Gitea Authors. All rights reserved.
+// Use of this source code is governed by a MIT-style
+// license that can be found in the LICENSE file.
+// This code is heavily inspired by the archived gofacebook/gracenet/net.go handler
+
+package graceful
+
+import (
+       "fmt"
+       "os"
+       "os/exec"
+       "strings"
+       "sync"
+       "syscall"
+)
+
+var killParent sync.Once
+
+// KillParent sends the kill signal to the parent process if we are a child
+func KillParent() {
+       killParent.Do(func() {
+               if Manager.IsChild() {
+                       ppid := syscall.Getppid()
+                       if ppid > 1 {
+                               _ = syscall.Kill(ppid, syscall.SIGTERM)
+                       }
+               }
+       })
+}
+
+// RestartProcess starts a new process passing it the active listeners. It
+// doesn't fork, but starts a new process using the same environment and
+// arguments as when it was originally started. This allows for a newly
+// deployed binary to be started. It returns the pid of the newly started
+// process when successful.
+func RestartProcess() (int, error) {
+       listeners := getActiveListeners()
+
+       // Extract the fds from the listeners.
+       files := make([]*os.File, len(listeners))
+       for i, l := range listeners {
+               var err error
+               // Now, all our listeners actually have File() functions so instead of
+               // individually casting we just use a hacky interface
+               files[i], err = l.(filer).File()
+               if err != nil {
+                       return 0, err
+               }
+               // Remember to close these at the end.
+               defer files[i].Close()
+       }
+
+       // Use the original binary location. This works with symlinks such that if
+       // the file it points to has been changed we will use the updated symlink.
+       argv0, err := exec.LookPath(os.Args[0])
+       if err != nil {
+               return 0, err
+       }
+
+       // Pass on the environment and replace the old count key with the new one.
+       var env []string
+       for _, v := range os.Environ() {
+               if !strings.HasPrefix(v, listenFDs+"=") {
+                       env = append(env, v)
+               }
+       }
+       env = append(env, fmt.Sprintf("%s=%d", listenFDs, len(listeners)))
+
+       allFiles := append([]*os.File{os.Stdin, os.Stdout, os.Stderr}, files...)
+       process, err := os.StartProcess(argv0, os.Args, &os.ProcAttr{
+               Dir:   originalWD,
+               Env:   env,
+               Files: allFiles,
+       })
+       if err != nil {
+               return 0, err
+       }
+       return process.Pid, nil
+}
index 896d547b46ca838e5a9849107c567ac4fc392502..c6692cbb7511b058c169d910a3753faff943418e 100644 (file)
@@ -1,5 +1,3 @@
-// +build !windows
-
 // Copyright 2019 The Gitea Authors. All rights reserved.
 // Use of this source code is governed by a MIT-style
 // license that can be found in the LICENSE file.
@@ -19,37 +17,16 @@ import (
        "code.gitea.io/gitea/modules/log"
 )
 
-type state uint8
-
-const (
-       stateInit state = iota
-       stateRunning
-       stateShuttingDown
-       stateTerminate
-)
-
 var (
-       // RWMutex for when adding servers or shutting down
-       runningServerReg sync.RWMutex
-       runningServerWG  sync.WaitGroup
-       // ensure we only fork once
-       runningServersForked bool
-
        // DefaultReadTimeOut default read timeout
        DefaultReadTimeOut time.Duration
        // DefaultWriteTimeOut default write timeout
        DefaultWriteTimeOut time.Duration
        // DefaultMaxHeaderBytes default max header bytes
        DefaultMaxHeaderBytes int
-
-       // IsChild reports if we are a fork iff LISTEN_FDS is set and our parent PID is not 1
-       IsChild = len(os.Getenv(listenFDs)) > 0 && os.Getppid() > 1
 )
 
 func init() {
-       runningServerReg = sync.RWMutex{}
-       runningServerWG = sync.WaitGroup{}
-
        DefaultMaxHeaderBytes = 0 // use http.DefaultMaxHeaderBytes - which currently is 1 << 20 (1MB)
 }
 
@@ -58,43 +35,29 @@ type ServeFunction = func(net.Listener) error
 
 // Server represents our graceful server
 type Server struct {
-       network         string
-       address         string
-       listener        net.Listener
-       PreSignalHooks  map[os.Signal][]func()
-       PostSignalHooks map[os.Signal][]func()
-       wg              sync.WaitGroup
-       sigChan         chan os.Signal
-       state           state
-       lock            *sync.RWMutex
-       BeforeBegin     func(network, address string)
-       OnShutdown      func()
-}
-
-// WaitForServers waits for all running servers to finish
-func WaitForServers() {
-       runningServerWG.Wait()
+       network     string
+       address     string
+       listener    net.Listener
+       wg          sync.WaitGroup
+       state       state
+       lock        *sync.RWMutex
+       BeforeBegin func(network, address string)
+       OnShutdown  func()
 }
 
 // NewServer creates a server on network at provided address
 func NewServer(network, address string) *Server {
-       runningServerReg.Lock()
-       defer runningServerReg.Unlock()
-
-       if IsChild {
+       if Manager.IsChild() {
                log.Info("Restarting new server: %s:%s on PID: %d", network, address, os.Getpid())
        } else {
                log.Info("Starting new server: %s:%s on PID: %d", network, address, os.Getpid())
        }
        srv := &Server{
-               wg:              sync.WaitGroup{},
-               sigChan:         make(chan os.Signal),
-               PreSignalHooks:  map[os.Signal][]func(){},
-               PostSignalHooks: map[os.Signal][]func(){},
-               state:           stateInit,
-               lock:            &sync.RWMutex{},
-               network:         network,
-               address:         address,
+               wg:      sync.WaitGroup{},
+               state:   stateInit,
+               lock:    &sync.RWMutex{},
+               network: network,
+               address: address,
        }
 
        srv.BeforeBegin = func(network, addr string) {
@@ -107,7 +70,7 @@ func NewServer(network, address string) *Server {
 // ListenAndServe listens on the provided network address and then calls Serve
 // to handle requests on incoming connections.
 func (srv *Server) ListenAndServe(serve ServeFunction) error {
-       go srv.handleSignals()
+       go srv.awaitShutdown()
 
        l, err := GetListener(srv.network, srv.address)
        if err != nil {
@@ -117,8 +80,6 @@ func (srv *Server) ListenAndServe(serve ServeFunction) error {
 
        srv.listener = newWrappedListener(l, srv)
 
-       KillParent()
-
        srv.BeforeBegin(srv.network, srv.address)
 
        return srv.Serve(serve)
@@ -150,7 +111,7 @@ func (srv *Server) ListenAndServeTLS(certFile, keyFile string, serve ServeFuncti
 // ListenAndServeTLSConfig listens on the provided network address and then calls
 // Serve to handle requests on incoming TLS connections.
 func (srv *Server) ListenAndServeTLSConfig(tlsConfig *tls.Config, serve ServeFunction) error {
-       go srv.handleSignals()
+       go srv.awaitShutdown()
 
        l, err := GetListener(srv.network, srv.address)
        if err != nil {
@@ -161,7 +122,6 @@ func (srv *Server) ListenAndServeTLSConfig(tlsConfig *tls.Config, serve ServeFun
        wl := newWrappedListener(l, srv)
        srv.listener = tls.NewListener(wl, tlsConfig)
 
-       KillParent()
        srv.BeforeBegin(srv.network, srv.address)
 
        return srv.Serve(serve)
@@ -178,12 +138,12 @@ func (srv *Server) ListenAndServeTLSConfig(tlsConfig *tls.Config, serve ServeFun
 func (srv *Server) Serve(serve ServeFunction) error {
        defer log.Debug("Serve() returning... (PID: %d)", syscall.Getpid())
        srv.setState(stateRunning)
-       runningServerWG.Add(1)
+       Manager.RegisterServer()
        err := serve(srv.listener)
        log.Debug("Waiting for connections to finish... (PID: %d)", syscall.Getpid())
        srv.wg.Wait()
        srv.setState(stateTerminate)
-       runningServerWG.Done()
+       Manager.ServerDone()
        // use of closed means that the listeners are closed - i.e. we should be shutting down - return nil
        if err != nil && strings.Contains(err.Error(), "use of closed") {
                return nil
@@ -205,6 +165,10 @@ func (srv *Server) setState(st state) {
        srv.state = st
 }
 
+type filer interface {
+       File() (*os.File, error)
+}
+
 type wrappedListener struct {
        net.Listener
        stopped bool
index b8ca20ddf509fbd463918fd24f7d92839a064859..74b0fcb885814bdc0ce6edadb6458d76c27b6090 100644 (file)
@@ -1,5 +1,3 @@
-// +build !windows
-
 // Copyright 2019 The Gitea Authors. All rights reserved.
 // Use of this source code is governed by a MIT-style
 // license that can be found in the LICENSE file.
@@ -7,29 +5,37 @@
 package graceful
 
 import (
-       "errors"
-       "fmt"
        "os"
        "runtime"
-       "time"
 
        "code.gitea.io/gitea/modules/log"
-       "code.gitea.io/gitea/modules/setting"
 )
 
+// awaitShutdown waits for the shutdown signal from the Manager
+func (srv *Server) awaitShutdown() {
+       select {
+       case <-Manager.IsShutdown():
+               // Shutdown
+               srv.doShutdown()
+       case <-Manager.IsHammer():
+               // Hammer
+               srv.doShutdown()
+               srv.doHammer()
+       }
+       <-Manager.IsHammer()
+       srv.doHammer()
+}
+
 // shutdown closes the listener so that no new connections are accepted
 // and starts a goroutine that will hammer (stop all running requests) the server
 // after setting.GracefulHammerTime.
-func (srv *Server) shutdown() {
+func (srv *Server) doShutdown() {
        // only shutdown if we're running.
        if srv.getState() != stateRunning {
                return
        }
 
        srv.setState(stateShuttingDown)
-       if setting.GracefulHammerTime >= 0 {
-               go srv.hammerTime(setting.GracefulHammerTime)
-       }
 
        if srv.OnShutdown != nil {
                srv.OnShutdown()
@@ -42,14 +48,7 @@ func (srv *Server) shutdown() {
        }
 }
 
-// hammerTime forces the server to shutdown in a given timeout - whether it
-// finished outstanding requests or not. if Read/WriteTimeout are not set or the
-// max header size is very big a connection could hang...
-//
-// srv.Serve() will not return until all connections are served. this will
-// unblock the srv.wg.Wait() in Serve() thus causing ListenAndServe* functions to
-// return.
-func (srv *Server) hammerTime(d time.Duration) {
+func (srv *Server) doHammer() {
        defer func() {
                // We call srv.wg.Done() until it panics.
                // This happens if we call Done() when the WaitGroup counter is already at 0
@@ -62,7 +61,6 @@ func (srv *Server) hammerTime(d time.Duration) {
        if srv.getState() != stateShuttingDown {
                return
        }
-       time.Sleep(d)
        log.Warn("Forcefully shutting down parent")
        for {
                if srv.getState() == stateTerminate {
@@ -74,48 +72,3 @@ func (srv *Server) hammerTime(d time.Duration) {
                runtime.Gosched()
        }
 }
-
-func (srv *Server) fork() error {
-       runningServerReg.Lock()
-       defer runningServerReg.Unlock()
-
-       // only one server instance should fork!
-       if runningServersForked {
-               return errors.New("another process already forked. Ignoring this one")
-       }
-
-       runningServersForked = true
-
-       // We need to move the file logs to append pids
-       setting.RestartLogsWithPIDSuffix()
-
-       _, err := RestartProcess()
-
-       return err
-}
-
-// RegisterPreSignalHook registers a function to be run before the signal handler for
-// a given signal. These are not mutex locked and should therefore be only called before Serve.
-func (srv *Server) RegisterPreSignalHook(sig os.Signal, f func()) (err error) {
-       for _, s := range hookableSignals {
-               if s == sig {
-                       srv.PreSignalHooks[sig] = append(srv.PreSignalHooks[sig], f)
-                       return
-               }
-       }
-       err = fmt.Errorf("Signal %v is not supported", sig)
-       return
-}
-
-// RegisterPostSignalHook registers a function to be run after the signal handler for
-// a given signal. These are not mutex locked and should therefore be only called before Serve.
-func (srv *Server) RegisterPostSignalHook(sig os.Signal, f func()) (err error) {
-       for _, s := range hookableSignals {
-               if s == sig {
-                       srv.PostSignalHooks[sig] = append(srv.PostSignalHooks[sig], f)
-                       return
-               }
-       }
-       err = fmt.Errorf("Signal %v is not supported", sig)
-       return
-}
index 446f0f55519f8f815cab35155c0c56fc380e9302..1052637d5e74a218a713f33da8adffe8d48f7a57 100644 (file)
@@ -1,5 +1,3 @@
-// +build !windows
-
 // Copyright 2019 The Gitea Authors. All rights reserved.
 // Use of this source code is governed by a MIT-style
 // license that can be found in the LICENSE file.
diff --git a/modules/graceful/server_signals.go b/modules/graceful/server_signals.go
deleted file mode 100644 (file)
index a4bcd00..0000000
+++ /dev/null
@@ -1,95 +0,0 @@
-// +build !windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-
-package graceful
-
-import (
-       "os"
-       "os/signal"
-       "syscall"
-       "time"
-
-       "code.gitea.io/gitea/modules/log"
-       "code.gitea.io/gitea/modules/setting"
-)
-
-var hookableSignals []os.Signal
-
-func init() {
-       hookableSignals = []os.Signal{
-               syscall.SIGHUP,
-               syscall.SIGUSR1,
-               syscall.SIGUSR2,
-               syscall.SIGINT,
-               syscall.SIGTERM,
-               syscall.SIGTSTP,
-       }
-}
-
-// handleSignals listens for os Signals and calls any hooked in function that the
-// user had registered with the signal.
-func (srv *Server) handleSignals() {
-       var sig os.Signal
-
-       signal.Notify(
-               srv.sigChan,
-               hookableSignals...,
-       )
-
-       pid := syscall.Getpid()
-       for {
-               sig = <-srv.sigChan
-               srv.preSignalHooks(sig)
-               switch sig {
-               case syscall.SIGHUP:
-                       if setting.GracefulRestartable {
-                               log.Info("PID: %d. Received SIGHUP. Forking...", pid)
-                               err := srv.fork()
-                               if err != nil && err.Error() != "another process already forked. Ignoring this one" {
-                                       log.Error("Error whilst forking from PID: %d : %v", pid, err)
-                               }
-                       } else {
-                               log.Info("PID: %d. Received SIGHUP. Not set restartable. Shutting down...", pid)
-
-                               srv.shutdown()
-                       }
-               case syscall.SIGUSR1:
-                       log.Info("PID %d. Received SIGUSR1.", pid)
-               case syscall.SIGUSR2:
-                       log.Warn("PID %d. Received SIGUSR2. Hammering...", pid)
-                       srv.hammerTime(0 * time.Second)
-               case syscall.SIGINT:
-                       log.Warn("PID %d. Received SIGINT. Shutting down...", pid)
-                       srv.shutdown()
-               case syscall.SIGTERM:
-                       log.Warn("PID %d. Received SIGTERM. Shutting down...", pid)
-                       srv.shutdown()
-               case syscall.SIGTSTP:
-                       log.Info("PID %d. Received SIGTSTP.")
-               default:
-                       log.Info("PID %d. Received %v.", sig)
-               }
-               srv.postSignalHooks(sig)
-       }
-}
-
-func (srv *Server) preSignalHooks(sig os.Signal) {
-       if _, notSet := srv.PreSignalHooks[sig]; !notSet {
-               return
-       }
-       for _, f := range srv.PreSignalHooks[sig] {
-               f()
-       }
-}
-
-func (srv *Server) postSignalHooks(sig os.Signal) {
-       if _, notSet := srv.PostSignalHooks[sig]; !notSet {
-               return
-       }
-       for _, f := range srv.PostSignalHooks[sig] {
-               f()
-       }
-}
index 76da46d7591b0ae8fd7c1a1cead375ea7381ecae..f4771136be5f817706161134eb3060a35ef30ea3 100644 (file)
@@ -172,7 +172,7 @@ func InitIssueIndexer(syncReindex bool) {
        } else if setting.Indexer.StartupTimeout > 0 {
                go func() {
                        timeout := setting.Indexer.StartupTimeout
-                       if graceful.IsChild && setting.GracefulHammerTime > 0 {
+                       if graceful.Manager.IsChild() && setting.GracefulHammerTime > 0 {
                                timeout += setting.GracefulHammerTime
                        }
                        select {
diff --git a/modules/minwinsvc/LICENSE b/modules/minwinsvc/LICENSE
deleted file mode 100644 (file)
index fce91b4..0000000
+++ /dev/null
@@ -1,20 +0,0 @@
-Copyright (c) 2015 Daniel Theophanes
-
-This software is provided 'as-is', without any express or implied
-warranty. In no event will the authors be held liable for any damages
-arising from the use of this software.
-
-Permission is granted to anyone to use this software for any purpose,
-including commercial applications, and to alter it and redistribute it
-freely, subject to the following restrictions:
-
-   1. The origin of this software must not be misrepresented; you must not
-   claim that you wrote the original software. If you use this software
-   in a product, an acknowledgment in the product documentation would be
-   appreciated but is not required.
-
-   2. Altered source versions must be plainly marked as such, and must not be
-   misrepresented as being the original software.
-
-   3. This notice may not be removed or altered from any source
-   distribution.
diff --git a/modules/minwinsvc/README.md b/modules/minwinsvc/README.md
deleted file mode 100644 (file)
index 260dcee..0000000
+++ /dev/null
@@ -1,18 +0,0 @@
-### Minimal windows service stub
-
-Programs designed to run from most *nix style operating systems
-can import this package to enable running programs as services without modifying
-them.
-
-```
-import _ "github.com/kardianos/minwinsvc"
-```
-
-If you need more control over the exit behavior, set
-```
-minwinsvc.SetOnExit(func() {
-       // Do something.
-       // Within 10 seconds call:
-       os.Exit(0)
-})
-```
diff --git a/modules/minwinsvc/minwinsvc.go b/modules/minwinsvc/minwinsvc.go
deleted file mode 100644 (file)
index 4cd89ff..0000000
+++ /dev/null
@@ -1,18 +0,0 @@
-// Copyright 2015 Daniel Theophanes.
-// Use of this source code is governed by a zlib-style
-// license that can be found in the LICENSE file.package service
-
-// Package minwinsvc is a minimal non-invasive windows only service stub.
-//
-// Import to allow running as a windows service.
-//   import _ "github.com/kardianos/minwinsvc"
-// This will detect if running as a windows service
-// and install required callbacks for windows.
-package minwinsvc
-
-// SetOnExit sets the function to be called when the windows service
-// requests an exit. If this is not called, or if it is called where
-// f == nil, then it defaults to calling "os.Exit(0)".
-func SetOnExit(f func()) {
-       setOnExit(f)
-}
diff --git a/modules/minwinsvc/svc_other.go b/modules/minwinsvc/svc_other.go
deleted file mode 100644 (file)
index 197d300..0000000
+++ /dev/null
@@ -1,11 +0,0 @@
-// Copyright 2015 Daniel Theophanes.
-// Use of this source code is governed by a zlib-style
-// license that can be found in the LICENSE file.package service
-
-//+build !windows
-
-package minwinsvc
-
-func setOnExit(f func()) {
-       // Nothing.
-}
diff --git a/modules/minwinsvc/svc_windows.go b/modules/minwinsvc/svc_windows.go
deleted file mode 100644 (file)
index 609bf06..0000000
+++ /dev/null
@@ -1,76 +0,0 @@
-// Copyright 2015 Daniel Theophanes.
-// Use of this source code is governed by a zlib-style
-// license that can be found in the LICENSE file.package service
-
-//+build windows
-
-package minwinsvc
-
-import (
-       "os"
-       "strconv"
-       "sync"
-
-       "golang.org/x/sys/windows/svc"
-)
-
-var (
-       onExit  func()
-       guard   sync.Mutex
-       skip, _ = strconv.ParseBool(os.Getenv("SKIP_MINWINSVC"))
-       isSSH   = os.Getenv("SSH_ORIGINAL_COMMAND") != ""
-)
-
-func init() {
-       if skip || isSSH {
-               return
-       }
-       interactive, err := svc.IsAnInteractiveSession()
-       if err != nil {
-               panic(err)
-       }
-       if interactive {
-               return
-       }
-       go func() {
-               _ = svc.Run("", runner{})
-
-               guard.Lock()
-               f := onExit
-               guard.Unlock()
-
-               // Don't hold this lock in user code.
-               if f != nil {
-                       f()
-               }
-               // Make sure we exit.
-               os.Exit(0)
-       }()
-}
-
-func setOnExit(f func()) {
-       guard.Lock()
-       onExit = f
-       guard.Unlock()
-}
-
-type runner struct{}
-
-func (runner) Execute(args []string, r <-chan svc.ChangeRequest, changes chan<- svc.Status) (bool, uint32) {
-       const cmdsAccepted = svc.AcceptStop | svc.AcceptShutdown
-       changes <- svc.Status{State: svc.StartPending}
-
-       changes <- svc.Status{State: svc.Running, Accepts: cmdsAccepted}
-       for {
-               c := <-r
-               switch c.Cmd {
-               case svc.Interrogate:
-                       changes <- c.CurrentStatus
-               case svc.Stop, svc.Shutdown:
-                       changes <- svc.Status{State: svc.StopPending}
-                       return false, 0
-               }
-       }
-
-       return false, 0
-}
index 6a45797cf2ebde07a92021e3f995c20dcea80676..dbe64fa3fd8f95de5de741a32c2d4d11e618ae2f 100644 (file)
@@ -24,7 +24,6 @@ import (
        "code.gitea.io/gitea/modules/generate"
        "code.gitea.io/gitea/modules/git"
        "code.gitea.io/gitea/modules/log"
-       _ "code.gitea.io/gitea/modules/minwinsvc" // import minwinsvc for windows services
        "code.gitea.io/gitea/modules/user"
 
        shellquote "github.com/kballard/go-shellquote"
@@ -99,6 +98,7 @@ var (
        LetsEncryptEmail     string
        GracefulRestartable  bool
        GracefulHammerTime   time.Duration
+       StartupTimeout       time.Duration
        StaticURLPrefix      string
 
        SSH = struct {
@@ -569,6 +569,7 @@ func NewContext() {
        HTTPPort = sec.Key("HTTP_PORT").MustString("3000")
        GracefulRestartable = sec.Key("ALLOW_GRACEFUL_RESTARTS").MustBool(true)
        GracefulHammerTime = sec.Key("GRACEFUL_HAMMER_TIME").MustDuration(60 * time.Second)
+       StartupTimeout = sec.Key("STARTUP_TIMEOUT").MustDuration(0 * time.Second)
 
        defaultAppURL := string(Protocol) + "://" + Domain
        if (Protocol == HTTP && HTTPPort != "80") || (Protocol == HTTPS && HTTPPort != "443") {
index d66c7d6540c52c8bfdbd5e0b68cbc70cd0c6948a..4d7557e2ee4d050138455f200c6eb3ec7e7dce1e 100644 (file)
@@ -1,5 +1,3 @@
-// +build !windows
-
 // Copyright 2019 The Gitea Authors. All rights reserved.
 // Use of this source code is governed by a MIT-style
 // license that can be found in the LICENSE file.
@@ -26,5 +24,5 @@ func listen(server *ssh.Server) {
 
 // Unused informs our cleanup routine that we will not be using a ssh port
 func Unused() {
-       graceful.InformCleanup()
+       graceful.Manager.InformCleanup()
 }
diff --git a/modules/ssh/ssh_windows.go b/modules/ssh/ssh_windows.go
deleted file mode 100644 (file)
index 55032e1..0000000
+++ /dev/null
@@ -1,24 +0,0 @@
-// +build windows
-
-// Copyright 2019 The Gitea Authors. All rights reserved.
-// Use of this source code is governed by a MIT-style
-// license that can be found in the LICENSE file.
-
-package ssh
-
-import (
-       "code.gitea.io/gitea/modules/log"
-       "github.com/gliderlabs/ssh"
-)
-
-func listen(server *ssh.Server) {
-       err := server.ListenAndServe()
-       if err != nil {
-               log.Critical("Failed to serve with builtin SSH server. %s", err)
-       }
-}
-
-// Unused does nothing on windows
-func Unused() {
-       // Do nothing
-}
diff --git a/vendor/golang.org/x/sys/windows/svc/debug/log.go b/vendor/golang.org/x/sys/windows/svc/debug/log.go
new file mode 100644 (file)
index 0000000..e51ab42
--- /dev/null
@@ -0,0 +1,56 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+package debug
+
+import (
+       "os"
+       "strconv"
+)
+
+// Log interface allows different log implementations to be used.
+type Log interface {
+       Close() error
+       Info(eid uint32, msg string) error
+       Warning(eid uint32, msg string) error
+       Error(eid uint32, msg string) error
+}
+
+// ConsoleLog provides access to the console.
+type ConsoleLog struct {
+       Name string
+}
+
+// New creates new ConsoleLog.
+func New(source string) *ConsoleLog {
+       return &ConsoleLog{Name: source}
+}
+
+// Close closes console log l.
+func (l *ConsoleLog) Close() error {
+       return nil
+}
+
+func (l *ConsoleLog) report(kind string, eid uint32, msg string) error {
+       s := l.Name + "." + kind + "(" + strconv.Itoa(int(eid)) + "): " + msg + "\n"
+       _, err := os.Stdout.Write([]byte(s))
+       return err
+}
+
+// Info writes an information event msg with event id eid to the console l.
+func (l *ConsoleLog) Info(eid uint32, msg string) error {
+       return l.report("info", eid, msg)
+}
+
+// Warning writes an warning event msg with event id eid to the console l.
+func (l *ConsoleLog) Warning(eid uint32, msg string) error {
+       return l.report("warn", eid, msg)
+}
+
+// Error writes an error event msg with event id eid to the console l.
+func (l *ConsoleLog) Error(eid uint32, msg string) error {
+       return l.report("error", eid, msg)
+}
diff --git a/vendor/golang.org/x/sys/windows/svc/debug/service.go b/vendor/golang.org/x/sys/windows/svc/debug/service.go
new file mode 100644 (file)
index 0000000..e621b87
--- /dev/null
@@ -0,0 +1,45 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+// Package debug provides facilities to execute svc.Handler on console.
+//
+package debug
+
+import (
+       "os"
+       "os/signal"
+       "syscall"
+
+       "golang.org/x/sys/windows/svc"
+)
+
+// Run executes service name by calling appropriate handler function.
+// The process is running on console, unlike real service. Use Ctrl+C to
+// send "Stop" command to your service.
+func Run(name string, handler svc.Handler) error {
+       cmds := make(chan svc.ChangeRequest)
+       changes := make(chan svc.Status)
+
+       sig := make(chan os.Signal)
+       signal.Notify(sig)
+
+       go func() {
+               status := svc.Status{State: svc.Stopped}
+               for {
+                       select {
+                       case <-sig:
+                               cmds <- svc.ChangeRequest{Cmd: svc.Stop, CurrentStatus: status}
+                       case status = <-changes:
+                       }
+               }
+       }()
+
+       _, errno := handler.Execute([]string{name}, cmds, changes)
+       if errno != 0 {
+               return syscall.Errno(errno)
+       }
+       return nil
+}
index 414c802772b68750921721316333e338198c9c40..bd049267ff50c5b2e5469feaf72912f95b66d6fa 100644 (file)
@@ -482,6 +482,7 @@ golang.org/x/sys/cpu
 golang.org/x/sys/unix
 golang.org/x/sys/windows
 golang.org/x/sys/windows/svc
+golang.org/x/sys/windows/svc/debug
 # golang.org/x/text v0.3.2
 golang.org/x/text/encoding
 golang.org/x/text/encoding/charmap