mirror of
https://github.com/minio/minio.git
synced 2025-11-07 04:42:56 -05:00
Refactor HTTP server to address bugs (#4636)
* Refactor HTTP server to address bugs * Remove unnecessary goroutine to start multiple TCP listeners. * HTTP server waits for shutdown to maximum of Server.ShutdownTimeout than per serverShutdownPoll. * Handles new connection errors properly. * Handles read and write timeout properly. * Handles error on start of HTTP server properly by exiting minio process. Fixes #4494 #4476 & fixed review comments
This commit is contained in:
55
cmd/certs.go
55
cmd/certs.go
@@ -17,6 +17,7 @@
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"encoding/pem"
|
||||
"fmt"
|
||||
@@ -24,35 +25,34 @@ import (
|
||||
"path/filepath"
|
||||
)
|
||||
|
||||
func parsePublicCertFile(certFile string) (certs []*x509.Certificate, err error) {
|
||||
var bytes []byte
|
||||
|
||||
if bytes, err = ioutil.ReadFile(certFile); err != nil {
|
||||
return certs, err
|
||||
func parsePublicCertFile(certFile string) (x509Certs []*x509.Certificate, err error) {
|
||||
// Read certificate file.
|
||||
var data []byte
|
||||
if data, err = ioutil.ReadFile(certFile); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Parse all certs in the chain.
|
||||
var block *pem.Block
|
||||
var cert *x509.Certificate
|
||||
current := bytes
|
||||
current := data
|
||||
for len(current) > 0 {
|
||||
if block, current = pem.Decode(current); block == nil {
|
||||
err = fmt.Errorf("Could not read PEM block from file %s", certFile)
|
||||
return certs, err
|
||||
var pemBlock *pem.Block
|
||||
if pemBlock, current = pem.Decode(current); pemBlock == nil {
|
||||
return nil, fmt.Errorf("Could not read PEM block from file %s", certFile)
|
||||
}
|
||||
|
||||
if cert, err = x509.ParseCertificate(block.Bytes); err != nil {
|
||||
return certs, err
|
||||
var x509Cert *x509.Certificate
|
||||
if x509Cert, err = x509.ParseCertificate(pemBlock.Bytes); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
certs = append(certs, cert)
|
||||
x509Certs = append(x509Certs, x509Cert)
|
||||
}
|
||||
|
||||
if len(certs) == 0 {
|
||||
err = fmt.Errorf("Empty public certificate file %s", certFile)
|
||||
if len(x509Certs) == 0 {
|
||||
return nil, fmt.Errorf("Empty public certificate file %s", certFile)
|
||||
}
|
||||
|
||||
return certs, err
|
||||
return x509Certs, nil
|
||||
}
|
||||
|
||||
func getRootCAs(certsCAsDir string) (*x509.CertPool, error) {
|
||||
@@ -81,7 +81,7 @@ func getRootCAs(certsCAsDir string) (*x509.CertPool, error) {
|
||||
for _, caFile := range caFiles {
|
||||
caCert, err := ioutil.ReadFile(caFile)
|
||||
if err != nil {
|
||||
return rootCAs, err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rootCAs.AppendCertsFromPEM(caCert)
|
||||
@@ -90,19 +90,26 @@ func getRootCAs(certsCAsDir string) (*x509.CertPool, error) {
|
||||
return rootCAs, nil
|
||||
}
|
||||
|
||||
func getSSLConfig() (publicCerts []*x509.Certificate, rootCAs *x509.CertPool, secureConn bool, err error) {
|
||||
func getSSLConfig() (x509Certs []*x509.Certificate, rootCAs *x509.CertPool, tlsCert *tls.Certificate, secureConn bool, err error) {
|
||||
if !(isFile(getPublicCertFile()) && isFile(getPrivateKeyFile())) {
|
||||
return publicCerts, rootCAs, secureConn, err
|
||||
return nil, nil, nil, false, nil
|
||||
}
|
||||
|
||||
if publicCerts, err = parsePublicCertFile(getPublicCertFile()); err != nil {
|
||||
return publicCerts, rootCAs, secureConn, err
|
||||
if x509Certs, err = parsePublicCertFile(getPublicCertFile()); err != nil {
|
||||
return nil, nil, nil, false, err
|
||||
}
|
||||
|
||||
var cert tls.Certificate
|
||||
if cert, err = tls.LoadX509KeyPair(getPublicCertFile(), getPrivateKeyFile()); err != nil {
|
||||
return nil, nil, nil, false, err
|
||||
}
|
||||
|
||||
tlsCert = &cert
|
||||
|
||||
if rootCAs, err = getRootCAs(getCADir()); err != nil {
|
||||
return publicCerts, rootCAs, secureConn, err
|
||||
return nil, nil, nil, false, err
|
||||
}
|
||||
|
||||
secureConn = true
|
||||
return publicCerts, rootCAs, secureConn, err
|
||||
return x509Certs, rootCAs, tlsCert, secureConn, nil
|
||||
}
|
||||
|
||||
@@ -20,11 +20,15 @@ import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/url"
|
||||
"os"
|
||||
"os/signal"
|
||||
"runtime"
|
||||
"strings"
|
||||
"syscall"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/cli"
|
||||
miniohttp "github.com/minio/minio/pkg/http"
|
||||
)
|
||||
|
||||
const azureGatewayTemplate = `NAME:
|
||||
@@ -314,8 +318,8 @@ func gatewayMain(ctx *cli.Context, backendType gatewayBackend) {
|
||||
|
||||
// Check and load SSL certificates.
|
||||
var err error
|
||||
globalPublicCerts, globalRootCAs, globalIsSSL, err = getSSLConfig()
|
||||
fatalIf(err, "Invalid SSL key file")
|
||||
globalPublicCerts, globalRootCAs, globalTLSCertificate, globalIsSSL, err = getSSLConfig()
|
||||
fatalIf(err, "Invalid SSL certificate file")
|
||||
|
||||
initNSLock(false) // Enable local namespace lock.
|
||||
|
||||
@@ -359,17 +363,15 @@ func gatewayMain(ctx *cli.Context, backendType gatewayBackend) {
|
||||
|
||||
}
|
||||
|
||||
apiServer := NewServerMux(ctx.GlobalString("address"), registerHandlers(router, handlerFns...))
|
||||
globalHTTPServer = miniohttp.NewServer([]string{ctx.GlobalString("address")}, registerHandlers(router, handlerFns...), globalTLSCertificate)
|
||||
|
||||
// Start server, automatically configures TLS if certs are available.
|
||||
go func() {
|
||||
cert, key := "", ""
|
||||
if globalIsSSL {
|
||||
cert, key = getPublicCertFile(), getPrivateKeyFile()
|
||||
}
|
||||
fatalIf(apiServer.ListenAndServe(cert, key), "Failed to start minio server")
|
||||
globalHTTPServerErrorCh <- globalHTTPServer.Start()
|
||||
}()
|
||||
|
||||
signal.Notify(globalOSSignalCh, os.Interrupt, syscall.SIGTERM)
|
||||
|
||||
// Once endpoints are finalized, initialize the new object api.
|
||||
globalObjLayerMutex.Lock()
|
||||
globalObjectAPI = newObject
|
||||
@@ -391,8 +393,8 @@ func gatewayMain(ctx *cli.Context, backendType gatewayBackend) {
|
||||
checkUpdate(mode)
|
||||
|
||||
// Print gateway startup message.
|
||||
printGatewayStartupMessage(getAPIEndpoints(apiServer.Addr), backendType)
|
||||
printGatewayStartupMessage(getAPIEndpoints(ctx.String("address")), backendType)
|
||||
}
|
||||
|
||||
<-globalServiceDoneCh
|
||||
handleSignals()
|
||||
}
|
||||
|
||||
@@ -17,12 +17,15 @@
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"os"
|
||||
"runtime"
|
||||
"time"
|
||||
|
||||
humanize "github.com/dustin/go-humanize"
|
||||
"github.com/fatih/color"
|
||||
miniohttp "github.com/minio/minio/pkg/http"
|
||||
)
|
||||
|
||||
// minio configuration related constants.
|
||||
@@ -106,6 +109,12 @@ var (
|
||||
// IsSSL indicates if the server is configured with SSL.
|
||||
globalIsSSL bool
|
||||
|
||||
globalTLSCertificate *tls.Certificate
|
||||
|
||||
globalHTTPServer *miniohttp.Server
|
||||
globalHTTPServerErrorCh = make(chan error)
|
||||
globalOSSignalCh = make(chan os.Signal, 1)
|
||||
|
||||
// List of admin peers.
|
||||
globalAdminPeers = adminPeers{}
|
||||
|
||||
|
||||
@@ -17,11 +17,15 @@
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
"os"
|
||||
"os/signal"
|
||||
"runtime"
|
||||
"syscall"
|
||||
|
||||
"github.com/minio/cli"
|
||||
"github.com/minio/dsync"
|
||||
miniohttp "github.com/minio/minio/pkg/http"
|
||||
)
|
||||
|
||||
var serverFlags = []cli.Flag{
|
||||
@@ -149,8 +153,8 @@ func serverMain(ctx *cli.Context) {
|
||||
|
||||
// Check and load SSL certificates.
|
||||
var err error
|
||||
globalPublicCerts, globalRootCAs, globalIsSSL, err = getSSLConfig()
|
||||
fatalIf(err, "Invalid SSL key file")
|
||||
globalPublicCerts, globalRootCAs, globalTLSCertificate, globalIsSSL, err = getSSLConfig()
|
||||
fatalIf(err, "Invalid SSL certificate file")
|
||||
|
||||
if !quietFlag {
|
||||
// Check for new updates from dl.minio.io.
|
||||
@@ -176,43 +180,47 @@ func serverMain(ctx *cli.Context) {
|
||||
initNSLock(globalIsDistXL)
|
||||
|
||||
// Configure server.
|
||||
handler, err := configureServerHandler(globalEndpoints)
|
||||
// Declare handler to avoid lint errors.
|
||||
var handler http.Handler
|
||||
handler, err = configureServerHandler(globalEndpoints)
|
||||
fatalIf(err, "Unable to configure one of server's RPC services.")
|
||||
|
||||
// Initialize a new HTTP server.
|
||||
apiServer := NewServerMux(globalMinioAddr, handler)
|
||||
|
||||
// Initialize S3 Peers inter-node communication only in distributed setup.
|
||||
initGlobalS3Peers(globalEndpoints)
|
||||
|
||||
// Initialize Admin Peers inter-node communication only in distributed setup.
|
||||
initGlobalAdminPeers(globalEndpoints)
|
||||
|
||||
// Start server, automatically configures TLS if certs are available.
|
||||
globalHTTPServer = miniohttp.NewServer([]string{globalMinioAddr}, handler, globalTLSCertificate)
|
||||
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
||||
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
||||
globalHTTPServer.ErrorLogFunc = errorIf
|
||||
go func() {
|
||||
cert, key := "", ""
|
||||
if globalIsSSL {
|
||||
cert, key = getPublicCertFile(), getPrivateKeyFile()
|
||||
}
|
||||
fatalIf(apiServer.ListenAndServe(cert, key), "Failed to start minio server.")
|
||||
globalHTTPServerErrorCh <- globalHTTPServer.Start()
|
||||
}()
|
||||
|
||||
signal.Notify(globalOSSignalCh, os.Interrupt, syscall.SIGTERM)
|
||||
|
||||
newObject, err := newObjectLayer(globalEndpoints)
|
||||
fatalIf(err, "Initializing object layer failed")
|
||||
if err != nil {
|
||||
errorIf(err, "Initializing object layer failed")
|
||||
err = globalHTTPServer.Shutdown()
|
||||
errorIf(err, "Unable to shutdown http server")
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
globalObjLayerMutex.Lock()
|
||||
globalObjectAPI = newObject
|
||||
globalObjLayerMutex.Unlock()
|
||||
|
||||
// Prints the formatted startup message once object layer is initialized.
|
||||
apiEndpoints := getAPIEndpoints(apiServer.Addr)
|
||||
apiEndpoints := getAPIEndpoints(globalMinioAddr)
|
||||
printStartupMessage(apiEndpoints)
|
||||
|
||||
// Set uptime time after object layer has initialized.
|
||||
globalBootTime = UTCNow()
|
||||
|
||||
// Waits on the server.
|
||||
<-globalServiceDoneCh
|
||||
handleSignals()
|
||||
}
|
||||
|
||||
// Initialize object layer with the supplied disks, objectLayer is nil upon any error.
|
||||
|
||||
@@ -1,526 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"crypto/tls"
|
||||
"errors"
|
||||
"io"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/url"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
serverShutdownPoll = 500 * time.Millisecond
|
||||
)
|
||||
|
||||
// The value chosen below is longest word chosen
|
||||
// from all the http verbs comprising of
|
||||
// "PRI", "OPTIONS", "GET", "HEAD", "POST",
|
||||
// "PUT", "DELETE", "TRACE", "CONNECT".
|
||||
const (
|
||||
maxHTTPVerbLen = 7
|
||||
)
|
||||
|
||||
// HTTP2 PRI method.
|
||||
var httpMethodPRI = "PRI"
|
||||
|
||||
var defaultHTTP2Methods = []string{
|
||||
httpMethodPRI,
|
||||
}
|
||||
|
||||
var defaultHTTP1Methods = []string{
|
||||
http.MethodOptions,
|
||||
http.MethodGet,
|
||||
http.MethodHead,
|
||||
http.MethodPost,
|
||||
http.MethodPut,
|
||||
http.MethodDelete,
|
||||
http.MethodTrace,
|
||||
http.MethodConnect,
|
||||
}
|
||||
|
||||
// ConnMux - Peeks into the incoming connection for relevant
|
||||
// protocol without advancing the underlying net.Conn (io.Reader).
|
||||
// ConnMux - allows us to multiplex between TLS and Regular HTTP
|
||||
// connections on the same listeners.
|
||||
type ConnMux struct {
|
||||
net.Conn
|
||||
// To peek net.Conn incoming data
|
||||
peeker *bufio.Reader
|
||||
}
|
||||
|
||||
// NewConnMux - creates a new ConnMux instance
|
||||
func NewConnMux(c net.Conn) *ConnMux {
|
||||
br := bufio.NewReader(c)
|
||||
return &ConnMux{
|
||||
Conn: c,
|
||||
peeker: bufio.NewReader(br),
|
||||
}
|
||||
}
|
||||
|
||||
// List of protocols to be detected by PeekProtocol function.
|
||||
const (
|
||||
protocolTLS = "tls"
|
||||
protocolHTTP1 = "http"
|
||||
protocolHTTP2 = "http2"
|
||||
)
|
||||
|
||||
// PeekProtocol - reads the first bytes, then checks if it is similar
|
||||
// to one of the default http methods. Returns error if there are any
|
||||
// errors in peeking over the connection.
|
||||
func (c *ConnMux) PeekProtocol() (string, error) {
|
||||
// Peek for HTTP verbs.
|
||||
buf, err := c.peeker.Peek(maxHTTPVerbLen)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
// Check for HTTP2 methods first.
|
||||
for _, m := range defaultHTTP2Methods {
|
||||
if strings.HasPrefix(string(buf), m) {
|
||||
return protocolHTTP2, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Check for HTTP1 methods.
|
||||
for _, m := range defaultHTTP1Methods {
|
||||
if strings.HasPrefix(string(buf), m) {
|
||||
return protocolHTTP1, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Default to TLS, this is not a real indication
|
||||
// that the connection is TLS but that will be
|
||||
// validated later by doing a handshake.
|
||||
return protocolTLS, nil
|
||||
}
|
||||
|
||||
// Read reads from the tcp session for data sent by
|
||||
// the client, additionally sets deadline for 15 secs
|
||||
// after each successful read. Deadline cancels and
|
||||
// returns error if the client does not send any
|
||||
// data in 15 secs. Also keeps track of the total
|
||||
// bytes received from the client.
|
||||
func (c *ConnMux) Read(b []byte) (n int, err error) {
|
||||
// Update total incoming number of bytes.
|
||||
defer func() {
|
||||
globalConnStats.incInputBytes(n)
|
||||
}()
|
||||
|
||||
n, err = c.peeker.Read(b)
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
|
||||
// Read deadline was already set previously, set again
|
||||
// after a successful read operation for future read
|
||||
// operations.
|
||||
c.Conn.SetReadDeadline(UTCNow().Add(defaultTCPReadTimeout))
|
||||
|
||||
// Success.
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// Write to the client over a tcp session, additionally
|
||||
// keeps track of the total bytes written by the server.
|
||||
func (c *ConnMux) Write(b []byte) (n int, err error) {
|
||||
// Update total outgoing number of bytes.
|
||||
defer func() {
|
||||
globalConnStats.incOutputBytes(n)
|
||||
}()
|
||||
|
||||
// Call the conn write wrapper.
|
||||
return c.Conn.Write(b)
|
||||
}
|
||||
|
||||
// Close closes the underlying tcp connection.
|
||||
func (c *ConnMux) Close() (err error) {
|
||||
// Make sure that we always close a connection,
|
||||
return c.Conn.Close()
|
||||
}
|
||||
|
||||
// ListenerMux wraps the standard net.Listener to inspect
|
||||
// the communication protocol upon network connection
|
||||
// ListenerMux also wraps net.Listener to ensure that once
|
||||
// Listener.Close returns, the underlying socket has been closed.
|
||||
//
|
||||
// - https://github.com/golang/go/issues/10527
|
||||
//
|
||||
// The default Listener returns from Close before the underlying
|
||||
// socket has been closed if another goroutine has an active
|
||||
// reference (e.g. is in Accept).
|
||||
//
|
||||
// The following sequence of events can happen:
|
||||
//
|
||||
// Goroutine 1 is running Accept, and is blocked, waiting for epoll
|
||||
//
|
||||
// Goroutine 2 calls Close. It sees an extra reference, and so cannot
|
||||
// destroy the socket, but instead decrements a reference, marks the
|
||||
// connection as closed and unblocks epoll.
|
||||
//
|
||||
// Goroutine 2 returns to the caller, makes a new connection.
|
||||
// The new connection is sent to the socket (since it hasn't been destroyed)
|
||||
//
|
||||
// Goroutine 1 returns from epoll, and accepts the new connection.
|
||||
//
|
||||
// To avoid accepting connections after Close, we block Goroutine 2
|
||||
// from returning from Close till Accept returns an error to the user.
|
||||
type ListenerMux struct {
|
||||
net.Listener
|
||||
config *tls.Config
|
||||
// acceptResCh is a channel for transporting wrapped net.Conn (regular or tls)
|
||||
// after peeking the content of the latter
|
||||
acceptResCh chan ListenerMuxAcceptRes
|
||||
// Cond is used to signal Close when there are no references to the listener.
|
||||
cond *sync.Cond
|
||||
refs int
|
||||
}
|
||||
|
||||
// ListenerMuxAcceptRes contains then final net.Conn data (wrapper by tls or not) to be sent to the http handler
|
||||
type ListenerMuxAcceptRes struct {
|
||||
conn net.Conn
|
||||
err error
|
||||
}
|
||||
|
||||
// Default keep alive interval timeout, on your Linux system to figure out
|
||||
// maximum probes sent
|
||||
//
|
||||
// > cat /proc/sys/net/ipv4/tcp_keepalive_probes
|
||||
// ! 9
|
||||
//
|
||||
// Final value of total keep-alive comes upto 9 x 10 * seconds = 1.5 minutes.
|
||||
const defaultKeepAliveTimeout = 10 * time.Second // 10 seconds.
|
||||
|
||||
// Timeout to close and return error to the client when not sending any data.
|
||||
const defaultTCPReadTimeout = 15 * time.Second // 15 seconds.
|
||||
|
||||
// newListenerMux listens and wraps accepted connections with tls after protocol peeking
|
||||
func newListenerMux(listener net.Listener, config *tls.Config) *ListenerMux {
|
||||
l := ListenerMux{
|
||||
Listener: listener,
|
||||
config: config,
|
||||
cond: sync.NewCond(&sync.Mutex{}),
|
||||
acceptResCh: make(chan ListenerMuxAcceptRes),
|
||||
}
|
||||
// Start listening, wrap connections with tls when needed
|
||||
go func() {
|
||||
// Extract tcp listener.
|
||||
tcpListener, ok := l.Listener.(*net.TCPListener)
|
||||
if !ok {
|
||||
l.acceptResCh <- ListenerMuxAcceptRes{err: errInvalidArgument}
|
||||
return
|
||||
}
|
||||
|
||||
// Loop for accepting new connections
|
||||
for {
|
||||
// Use accept TCP method to receive the connection.
|
||||
conn, err := tcpListener.AcceptTCP()
|
||||
if err != nil {
|
||||
l.acceptResCh <- ListenerMuxAcceptRes{err: err}
|
||||
continue
|
||||
}
|
||||
|
||||
// Enable Read timeout
|
||||
conn.SetReadDeadline(UTCNow().Add(defaultTCPReadTimeout))
|
||||
|
||||
// Enable keep alive for each connection.
|
||||
conn.SetKeepAlive(true)
|
||||
conn.SetKeepAlivePeriod(defaultKeepAliveTimeout)
|
||||
|
||||
// Allocate new conn muxer.
|
||||
connMux := NewConnMux(conn)
|
||||
|
||||
// Wrap the connection with ConnMux to be able to peek the data in the incoming connection
|
||||
// and decide if we need to wrap the connection itself with a TLS or not
|
||||
go func(connMux *ConnMux) {
|
||||
protocol, cerr := connMux.PeekProtocol()
|
||||
if cerr != nil {
|
||||
// io.EOF is usually returned by non-http clients,
|
||||
// just close the connection to avoid any leak.
|
||||
if cerr != io.EOF {
|
||||
errorIf(cerr, "Unable to peek into incoming protocol")
|
||||
}
|
||||
connMux.Close()
|
||||
return
|
||||
}
|
||||
switch protocol {
|
||||
case protocolTLS:
|
||||
tlsConn := tls.Server(connMux, l.config)
|
||||
// Make sure to handshake so that we know that this
|
||||
// is a TLS connection, if not we should close and reject
|
||||
// such a connection.
|
||||
if cerr = tlsConn.Handshake(); cerr != nil {
|
||||
// Close for junk message.
|
||||
tlsConn.Close()
|
||||
return
|
||||
}
|
||||
l.acceptResCh <- ListenerMuxAcceptRes{
|
||||
conn: tlsConn,
|
||||
}
|
||||
default:
|
||||
l.acceptResCh <- ListenerMuxAcceptRes{
|
||||
conn: connMux,
|
||||
}
|
||||
}
|
||||
}(connMux)
|
||||
}
|
||||
}()
|
||||
return &l
|
||||
}
|
||||
|
||||
// IsClosed - Returns if the underlying listener is closed fully.
|
||||
func (l *ListenerMux) IsClosed() bool {
|
||||
l.cond.L.Lock()
|
||||
defer l.cond.L.Unlock()
|
||||
return l.refs == 0
|
||||
}
|
||||
|
||||
func (l *ListenerMux) incRef() {
|
||||
l.cond.L.Lock()
|
||||
l.refs++
|
||||
l.cond.L.Unlock()
|
||||
}
|
||||
|
||||
func (l *ListenerMux) decRef() {
|
||||
l.cond.L.Lock()
|
||||
l.refs--
|
||||
newRefs := l.refs
|
||||
l.cond.L.Unlock()
|
||||
if newRefs == 0 {
|
||||
l.cond.Broadcast()
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes the listener.
|
||||
// Any blocked Accept operations will be unblocked and return errors.
|
||||
func (l *ListenerMux) Close() error {
|
||||
if l == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := l.Listener.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
l.cond.L.Lock()
|
||||
for l.refs > 0 {
|
||||
l.cond.Wait()
|
||||
}
|
||||
l.cond.L.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Accept - peek the protocol to decide if we should wrap the
|
||||
// network stream with the TLS server
|
||||
func (l *ListenerMux) Accept() (net.Conn, error) {
|
||||
l.incRef()
|
||||
defer l.decRef()
|
||||
|
||||
res := <-l.acceptResCh
|
||||
return res.conn, res.err
|
||||
}
|
||||
|
||||
// ServerMux - the main mux server
|
||||
type ServerMux struct {
|
||||
Addr string
|
||||
handler http.Handler
|
||||
listeners []*ListenerMux
|
||||
|
||||
// Current number of concurrent http requests
|
||||
currentReqs int32
|
||||
// Time to wait before forcing server shutdown
|
||||
gracefulTimeout time.Duration
|
||||
|
||||
mu sync.RWMutex // guards closing, and listeners
|
||||
closing bool
|
||||
}
|
||||
|
||||
// NewServerMux constructor to create a ServerMux
|
||||
func NewServerMux(addr string, handler http.Handler) *ServerMux {
|
||||
m := &ServerMux{
|
||||
Addr: addr,
|
||||
handler: handler,
|
||||
// Wait for 5 seconds for new incoming connnections, otherwise
|
||||
// forcibly close them during graceful stop or restart.
|
||||
gracefulTimeout: 5 * time.Second,
|
||||
}
|
||||
|
||||
// Returns configured HTTP server.
|
||||
return m
|
||||
}
|
||||
|
||||
// Initialize listeners on all ports.
|
||||
func initListeners(serverAddr string, tls *tls.Config) ([]*ListenerMux, error) {
|
||||
host, port, err := net.SplitHostPort(serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var listeners []*ListenerMux
|
||||
if host == "" {
|
||||
var listener net.Listener
|
||||
listener, err = net.Listen("tcp", serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
listeners = append(listeners, newListenerMux(listener, tls))
|
||||
return listeners, nil
|
||||
}
|
||||
var addrs []string
|
||||
if net.ParseIP(host) != nil {
|
||||
addrs = append(addrs, host)
|
||||
} else {
|
||||
addrs, err = net.LookupHost(host)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(addrs) == 0 {
|
||||
return nil, errUnexpected
|
||||
}
|
||||
}
|
||||
for _, addr := range addrs {
|
||||
var listener net.Listener
|
||||
listener, err = net.Listen("tcp", net.JoinHostPort(addr, port))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
listeners = append(listeners, newListenerMux(listener, tls))
|
||||
}
|
||||
return listeners, nil
|
||||
}
|
||||
|
||||
// ListenAndServe - serve HTTP requests with protocol multiplexing support
|
||||
// TLS is actived when certFile and keyFile parameters are not empty.
|
||||
func (m *ServerMux) ListenAndServe(certFile, keyFile string) (err error) {
|
||||
|
||||
tlsEnabled := certFile != "" && keyFile != ""
|
||||
|
||||
config := &tls.Config{
|
||||
// Causes servers to use Go's default ciphersuite preferences,
|
||||
// which are tuned to avoid attacks. Does nothing on clients.
|
||||
PreferServerCipherSuites: true,
|
||||
// Set minimum version to TLS 1.2
|
||||
MinVersion: tls.VersionTLS12,
|
||||
} // Always instantiate.
|
||||
|
||||
if tlsEnabled {
|
||||
// Configure TLS in the server
|
||||
if config.NextProtos == nil {
|
||||
config.NextProtos = []string{"http/1.1", "h2"}
|
||||
}
|
||||
config.Certificates = make([]tls.Certificate, 1)
|
||||
config.Certificates[0], err = tls.LoadX509KeyPair(certFile, keyFile)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
go m.handleServiceSignals()
|
||||
|
||||
listeners, err := initListeners(m.Addr, config)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
m.mu.Lock()
|
||||
m.listeners = listeners
|
||||
m.mu.Unlock()
|
||||
|
||||
// All http requests start to be processed by httpHandler
|
||||
httpHandler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
if tlsEnabled && r.TLS == nil {
|
||||
// TLS is enabled but request is not TLS
|
||||
// configured - return error to client.
|
||||
writeErrorResponse(w, ErrInsecureClientRequest, &url.URL{})
|
||||
} else {
|
||||
|
||||
// Return ServiceUnavailable for clients which are sending requests
|
||||
// in shutdown phase
|
||||
m.mu.RLock()
|
||||
closing := m.closing
|
||||
m.mu.RUnlock()
|
||||
if closing {
|
||||
w.WriteHeader(http.StatusServiceUnavailable)
|
||||
return
|
||||
}
|
||||
|
||||
// Execute registered handlers, update currentReqs to keep
|
||||
// track of concurrent requests processing on the server
|
||||
atomic.AddInt32(&m.currentReqs, 1)
|
||||
m.handler.ServeHTTP(w, r)
|
||||
atomic.AddInt32(&m.currentReqs, -1)
|
||||
}
|
||||
})
|
||||
|
||||
var wg = &sync.WaitGroup{}
|
||||
for _, listener := range listeners {
|
||||
wg.Add(1)
|
||||
go func(listener *ListenerMux) {
|
||||
defer wg.Done()
|
||||
serr := http.Serve(listener, httpHandler)
|
||||
// Do not print the error if the listener is closed.
|
||||
if !listener.IsClosed() {
|
||||
errorIf(serr, "Unable to serve incoming requests.")
|
||||
}
|
||||
}(listener)
|
||||
}
|
||||
// Wait for all http.Serve's to return.
|
||||
wg.Wait()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Close initiates the graceful shutdown
|
||||
func (m *ServerMux) Close() error {
|
||||
m.mu.Lock()
|
||||
|
||||
if m.closing {
|
||||
m.mu.Unlock()
|
||||
return errors.New("Server has been closed")
|
||||
}
|
||||
// Closed completely.
|
||||
m.closing = true
|
||||
|
||||
// Close the listeners.
|
||||
for _, listener := range m.listeners {
|
||||
if err := listener.Close(); err != nil {
|
||||
m.mu.Unlock()
|
||||
return err
|
||||
}
|
||||
}
|
||||
m.mu.Unlock()
|
||||
|
||||
// Starting graceful shutdown. Check if all requests are finished
|
||||
// in regular interval or force the shutdown
|
||||
ticker := time.NewTicker(serverShutdownPoll)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-time.After(m.gracefulTimeout):
|
||||
return nil
|
||||
case <-ticker.C:
|
||||
if atomic.LoadInt32(&m.currentReqs) <= 0 {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,506 +0,0 @@
|
||||
/*
|
||||
* Minio Cloud Storage, (C) 2015, 2016, 2017 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"crypto/rand"
|
||||
"crypto/rsa"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"crypto/x509/pkix"
|
||||
"encoding/pem"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"math/big"
|
||||
"net"
|
||||
"net/http"
|
||||
"os"
|
||||
"runtime"
|
||||
"strings"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestListenerAcceptAfterClose(t *testing.T) {
|
||||
var wg sync.WaitGroup
|
||||
for i := 0; i < 16; i++ {
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
runTest(t)
|
||||
}
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
func runTest(t *testing.T) {
|
||||
const connectionsBeforeClose = 1
|
||||
|
||||
ln, err := net.Listen("tcp", "127.0.0.1:0")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
ln = newListenerMux(ln, &tls.Config{})
|
||||
|
||||
addr := ln.Addr().String()
|
||||
waitForListener := make(chan error)
|
||||
go func() {
|
||||
defer close(waitForListener)
|
||||
|
||||
var connCount int
|
||||
for {
|
||||
conn, aerr := ln.Accept()
|
||||
if aerr != nil {
|
||||
return
|
||||
}
|
||||
|
||||
connCount++
|
||||
if connCount > connectionsBeforeClose {
|
||||
waitForListener <- errUnexpected
|
||||
return
|
||||
}
|
||||
conn.Close()
|
||||
}
|
||||
}()
|
||||
|
||||
for i := 0; i < connectionsBeforeClose; i++ {
|
||||
err = dial(addr)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
ln.Close()
|
||||
dial(addr)
|
||||
|
||||
err = <-waitForListener
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func dial(addr string) error {
|
||||
conn, err := net.Dial("tcp", addr)
|
||||
if err == nil {
|
||||
conn.Close()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// Tests initializing listeners.
|
||||
func TestInitListeners(t *testing.T) {
|
||||
testCases := []struct {
|
||||
serverAddr string
|
||||
shouldPass bool
|
||||
}{
|
||||
// Test 1 with ip and port.
|
||||
{
|
||||
serverAddr: net.JoinHostPort("127.0.0.1", "0"),
|
||||
shouldPass: true,
|
||||
},
|
||||
// Test 2 only port.
|
||||
{
|
||||
serverAddr: net.JoinHostPort("", "0"),
|
||||
shouldPass: true,
|
||||
},
|
||||
// Test 3 with no port error.
|
||||
{
|
||||
serverAddr: "127.0.0.1",
|
||||
shouldPass: false,
|
||||
},
|
||||
// Test 4 with 'foobar' host not resolvable.
|
||||
{
|
||||
serverAddr: "foobar:9000",
|
||||
shouldPass: false,
|
||||
},
|
||||
}
|
||||
for i, testCase := range testCases {
|
||||
listeners, err := initListeners(testCase.serverAddr, &tls.Config{})
|
||||
if testCase.shouldPass {
|
||||
if err != nil {
|
||||
t.Fatalf("Test %d: Unable to initialize listeners %s", i+1, err)
|
||||
}
|
||||
for _, listener := range listeners {
|
||||
if err = listener.Close(); err != nil {
|
||||
t.Fatalf("Test %d: Unable to close listeners %s", i+1, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
if err == nil && !testCase.shouldPass {
|
||||
t.Fatalf("Test %d: Should fail but is successful", i+1)
|
||||
}
|
||||
}
|
||||
// Windows doesn't have 'localhost' hostname.
|
||||
if runtime.GOOS != globalWindowsOSName {
|
||||
listeners, err := initListeners("localhost:"+getFreePort(), &tls.Config{})
|
||||
if err != nil {
|
||||
t.Fatalf("Test 3: Unable to initialize listeners %s", err)
|
||||
}
|
||||
for _, listener := range listeners {
|
||||
if err = listener.Close(); err != nil {
|
||||
t.Fatalf("Test 3: Unable to close listeners %s", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestClose(t *testing.T) {
|
||||
// Create ServerMux
|
||||
m := NewServerMux("", nil)
|
||||
|
||||
if err := m.Close(); err != nil {
|
||||
t.Error("Server errored while trying to Close", err)
|
||||
}
|
||||
|
||||
// Closing again should return an error.
|
||||
if err := m.Close(); err.Error() != "Server has been closed" {
|
||||
t.Error("Unexepcted error expected \"Server has been closed\", got", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerMux(t *testing.T) {
|
||||
var err error
|
||||
var got []byte
|
||||
var res *http.Response
|
||||
|
||||
// Create ServerMux
|
||||
m := NewServerMux("127.0.0.1:0", http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
fmt.Fprint(w, "hello")
|
||||
}))
|
||||
// Start serving requests
|
||||
go m.ListenAndServe("", "")
|
||||
|
||||
// Issue a GET request. Since we started server in a goroutine, it could be not ready
|
||||
// at this point. So we allow until 5 failed retries before declare there is an error
|
||||
for i := 0; i < 5; i++ {
|
||||
// Sleep one second
|
||||
time.Sleep(1 * time.Second)
|
||||
// Check if one listener is ready
|
||||
m.mu.Lock()
|
||||
listenersCount := len(m.listeners)
|
||||
m.mu.Unlock()
|
||||
if listenersCount == 0 {
|
||||
continue
|
||||
}
|
||||
m.mu.Lock()
|
||||
listenerAddr := m.listeners[0].Addr().String()
|
||||
m.mu.Unlock()
|
||||
// Issue the GET request
|
||||
client := http.Client{}
|
||||
res, err = client.Get("http://" + listenerAddr)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// Read the request response
|
||||
got, err = ioutil.ReadAll(res.Body)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
// We've got a response, quit the loop
|
||||
break
|
||||
}
|
||||
|
||||
// Check for error persisted after 5 times
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Check the web service response
|
||||
if string(got) != "hello" {
|
||||
t.Errorf("got %q, want hello", string(got))
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerCloseBlocking(t *testing.T) {
|
||||
// Create ServerMux
|
||||
m := NewServerMux("127.0.0.1:0", http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
fmt.Fprint(w, "hello")
|
||||
}))
|
||||
|
||||
// Start serving requests in a goroutine
|
||||
go m.ListenAndServe("", "")
|
||||
|
||||
// Dial, try until 5 times before declaring a failure
|
||||
dial := func() (net.Conn, error) {
|
||||
var c net.Conn
|
||||
var err error
|
||||
for i := 0; i < 5; i++ {
|
||||
// Sleep one second in case of the server is not ready yet
|
||||
time.Sleep(1 * time.Second)
|
||||
// Check if there is at least one listener configured
|
||||
m.mu.Lock()
|
||||
if len(m.listeners) == 0 {
|
||||
m.mu.Unlock()
|
||||
continue
|
||||
}
|
||||
m.mu.Unlock()
|
||||
// Run the actual Dial
|
||||
m.mu.Lock()
|
||||
c, err = net.Dial("tcp", m.listeners[0].Addr().String())
|
||||
m.mu.Unlock()
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return c, err
|
||||
}
|
||||
|
||||
// Dial to open a StateNew but don't send anything
|
||||
cnew, err := dial()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer cnew.Close()
|
||||
|
||||
// Dial another connection but idle after a request to have StateIdle
|
||||
cidle, err := dial()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer cidle.Close()
|
||||
|
||||
cidle.Write([]byte("HEAD / HTTP/1.1\r\nHost: foo\r\n\r\n"))
|
||||
_, err = http.ReadResponse(bufio.NewReader(cidle), nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Make sure we don't block forever.
|
||||
m.Close()
|
||||
}
|
||||
|
||||
func TestServerListenAndServePlain(t *testing.T) {
|
||||
wait := make(chan struct{})
|
||||
addr := net.JoinHostPort("127.0.0.1", getFreePort())
|
||||
errc := make(chan error)
|
||||
once := &sync.Once{}
|
||||
|
||||
// Initialize done channel specifically for each tests.
|
||||
globalServiceDoneCh = make(chan struct{}, 1)
|
||||
|
||||
// Create ServerMux and when we receive a request we stop waiting
|
||||
m := NewServerMux(addr, http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
fmt.Fprint(w, "hello")
|
||||
once.Do(func() { close(wait) })
|
||||
}))
|
||||
|
||||
// ListenAndServe in a goroutine, but we don't know when it's ready
|
||||
go func() { errc <- m.ListenAndServe("", "") }()
|
||||
|
||||
// Keep trying the server until it's accepting connections
|
||||
go func() {
|
||||
client := http.Client{Timeout: time.Millisecond * 10}
|
||||
for {
|
||||
res, _ := client.Get("http://" + addr)
|
||||
if res != nil && res.StatusCode == http.StatusOK {
|
||||
break
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
wg := &sync.WaitGroup{}
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
select {
|
||||
case err := <-errc:
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
case <-wait:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
// Wait until we get an error or wait closed
|
||||
wg.Wait()
|
||||
|
||||
// Shutdown the ServerMux
|
||||
m.Close()
|
||||
}
|
||||
|
||||
func TestServerListenAndServeTLS(t *testing.T) {
|
||||
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("Init Test config failed")
|
||||
}
|
||||
defer removeAll(rootPath)
|
||||
|
||||
wait := make(chan struct{})
|
||||
addr := net.JoinHostPort("127.0.0.1", getFreePort())
|
||||
errc := make(chan error)
|
||||
once := &sync.Once{}
|
||||
|
||||
// Initialize done channel specifically for each tests.
|
||||
globalServiceDoneCh = make(chan struct{}, 1)
|
||||
|
||||
// Create ServerMux and when we receive a request we stop waiting
|
||||
m := NewServerMux(addr, http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
fmt.Fprint(w, "hello")
|
||||
once.Do(func() { close(wait) })
|
||||
}))
|
||||
|
||||
// Create a cert
|
||||
err = createConfigDir()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
certFile := getPublicCertFile()
|
||||
keyFile := getPrivateKeyFile()
|
||||
defer os.RemoveAll(certFile)
|
||||
defer os.RemoveAll(keyFile)
|
||||
|
||||
err = generateTestCert(addr)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
|
||||
// ListenAndServe in a goroutine, but we don't know when it's ready
|
||||
go func() { errc <- m.ListenAndServe(certFile, keyFile) }()
|
||||
|
||||
wg := &sync.WaitGroup{}
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
tr := &http.Transport{
|
||||
TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
|
||||
}
|
||||
client := http.Client{
|
||||
Timeout: time.Millisecond * 10,
|
||||
Transport: tr,
|
||||
}
|
||||
// Keep trying the server until it's accepting connections
|
||||
start := UTCNow()
|
||||
for {
|
||||
res, _ := client.Get("https://" + addr)
|
||||
if res != nil && res.StatusCode == http.StatusOK {
|
||||
break
|
||||
}
|
||||
// Explicit check to terminate loop after 5 minutes
|
||||
// (for investigational purpose of issue #4461)
|
||||
if UTCNow().Sub(start) >= 5*time.Minute {
|
||||
t.Fatalf("Failed to establish connection after 5 minutes")
|
||||
}
|
||||
}
|
||||
|
||||
// Once a request succeeds, subsequent requests should
|
||||
// work fine.
|
||||
res, err := client.Get("http://" + addr)
|
||||
if err != nil {
|
||||
t.Errorf("Got unexpected error: %v", err)
|
||||
}
|
||||
// Without TLS we expect a Bad-Request response from the server.
|
||||
if !(res != nil && res.StatusCode == http.StatusBadRequest && res.Request.URL.Scheme == httpScheme) {
|
||||
t.Fatalf("Plaintext request to TLS server did not have expected response!")
|
||||
}
|
||||
|
||||
body, err := ioutil.ReadAll(res.Body)
|
||||
if err != nil {
|
||||
t.Errorf("Error reading body")
|
||||
}
|
||||
|
||||
// Check that the expected error is received.
|
||||
bodyStr := string(body)
|
||||
apiErr := getAPIError(ErrInsecureClientRequest)
|
||||
if !(strings.Contains(bodyStr, apiErr.Code) && strings.Contains(bodyStr, apiErr.Description)) {
|
||||
t.Fatalf("Plaintext request to TLS server did not have expected response body!")
|
||||
}
|
||||
wg.Done()
|
||||
}()
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
select {
|
||||
case err := <-errc:
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
return
|
||||
}
|
||||
case <-wait:
|
||||
return
|
||||
}
|
||||
}()
|
||||
|
||||
// Wait until we get an error or wait closed
|
||||
wg.Wait()
|
||||
|
||||
// Shutdown the ServerMux
|
||||
m.Close()
|
||||
}
|
||||
|
||||
// generateTestCert creates a cert and a key used for testing only
|
||||
func generateTestCert(host string) error {
|
||||
certPath := getPublicCertFile()
|
||||
keyPath := getPrivateKeyFile()
|
||||
priv, err := rsa.GenerateKey(rand.Reader, 2048)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
serialNumberLimit := new(big.Int).Lsh(big.NewInt(1), 128)
|
||||
serialNumber, err := rand.Int(rand.Reader, serialNumberLimit)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
template := x509.Certificate{
|
||||
SerialNumber: serialNumber,
|
||||
Subject: pkix.Name{
|
||||
Organization: []string{"Minio Test Cert"},
|
||||
},
|
||||
NotBefore: UTCNow(),
|
||||
NotAfter: UTCNow().Add(time.Minute * 1),
|
||||
|
||||
KeyUsage: x509.KeyUsageKeyEncipherment | x509.KeyUsageDigitalSignature,
|
||||
ExtKeyUsage: []x509.ExtKeyUsage{x509.ExtKeyUsageServerAuth},
|
||||
BasicConstraintsValid: true,
|
||||
}
|
||||
|
||||
if ip := net.ParseIP(host); ip != nil {
|
||||
template.IPAddresses = append(template.IPAddresses, ip)
|
||||
}
|
||||
|
||||
template.IsCA = true
|
||||
template.KeyUsage |= x509.KeyUsageCertSign
|
||||
|
||||
derBytes, err := x509.CreateCertificate(rand.Reader, &template, &template, &priv.PublicKey, priv)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
certOut, err := os.Create(certPath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pem.Encode(certOut, &pem.Block{Type: "CERTIFICATE", Bytes: derBytes})
|
||||
certOut.Close()
|
||||
|
||||
keyOut, err := os.OpenFile(keyPath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
pem.Encode(keyOut, &pem.Block{Type: "RSA PRIVATE KEY", Bytes: x509.MarshalPKCS1PrivateKey(priv)})
|
||||
keyOut.Close()
|
||||
return nil
|
||||
}
|
||||
@@ -19,8 +19,6 @@ package cmd
|
||||
import (
|
||||
"os"
|
||||
"os/exec"
|
||||
"syscall"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Type of service signals currently supported.
|
||||
@@ -64,59 +62,3 @@ func restartProcess() error {
|
||||
cmd.Stderr = os.Stderr
|
||||
return cmd.Start()
|
||||
}
|
||||
|
||||
// Handles all serviceSignal and execute service functions.
|
||||
func (m *ServerMux) handleServiceSignals() error {
|
||||
// Custom exit function
|
||||
runExitFn := func(err error) {
|
||||
// If global profiler is set stop before we exit.
|
||||
if globalProfiler != nil {
|
||||
globalProfiler.Stop()
|
||||
}
|
||||
|
||||
// Call user supplied user exit function
|
||||
fatalIf(err, "Unable to gracefully complete service operation.")
|
||||
|
||||
// We are usually done here, close global service done channel.
|
||||
globalServiceDoneCh <- struct{}{}
|
||||
}
|
||||
// Wait for SIGTERM in a go-routine.
|
||||
trapCh := signalTrap(os.Interrupt, syscall.SIGTERM)
|
||||
go func(trapCh <-chan bool) {
|
||||
<-trapCh
|
||||
globalServiceSignalCh <- serviceStop
|
||||
}(trapCh)
|
||||
|
||||
// Start listening on service signal. Monitor signals.
|
||||
for {
|
||||
signal := <-globalServiceSignalCh
|
||||
switch signal {
|
||||
case serviceStatus:
|
||||
/// We don't do anything for this.
|
||||
case serviceRestart:
|
||||
if err := m.Close(); err != nil {
|
||||
errorIf(err, "Unable to close server gracefully")
|
||||
}
|
||||
if err := restartProcess(); err != nil {
|
||||
errorIf(err, "Unable to restart the server.")
|
||||
}
|
||||
runExitFn(nil)
|
||||
case serviceStop:
|
||||
log.Println("Received signal to exit.")
|
||||
go func() {
|
||||
time.Sleep(serverShutdownPoll + time.Millisecond*100)
|
||||
log.Println("Waiting for active connections to terminate - press Ctrl+C to quit immediately.")
|
||||
}()
|
||||
if err := m.Close(); err != nil {
|
||||
errorIf(err, "Unable to close server gracefully")
|
||||
}
|
||||
objAPI := newObjectLayerFn()
|
||||
if objAPI == nil {
|
||||
// Server not initialized yet, exit happily.
|
||||
runExitFn(nil)
|
||||
} else {
|
||||
runExitFn(objAPI.Shutdown())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
/*
|
||||
* Minio Client, (C) 2015 Minio, Inc.
|
||||
* Minio Cloud Storage, (C) 2015, 2016, 2017 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
@@ -18,32 +18,67 @@ package cmd
|
||||
|
||||
import (
|
||||
"os"
|
||||
"os/signal"
|
||||
)
|
||||
|
||||
// signalTrap traps the registered signals and notifies the caller.
|
||||
func signalTrap(sig ...os.Signal) <-chan bool {
|
||||
// channel to notify the caller.
|
||||
trapCh := make(chan bool, 1)
|
||||
func handleSignals() {
|
||||
// Custom exit function
|
||||
exit := func(state bool) {
|
||||
// If global profiler is set stop before we exit.
|
||||
if globalProfiler != nil {
|
||||
globalProfiler.Stop()
|
||||
}
|
||||
|
||||
go func(chan<- bool) {
|
||||
// channel to receive signals.
|
||||
sigCh := make(chan os.Signal, 1)
|
||||
defer close(sigCh)
|
||||
if state {
|
||||
os.Exit(0)
|
||||
}
|
||||
|
||||
// `signal.Notify` registers the given channel to
|
||||
// receive notifications of the specified signals.
|
||||
signal.Notify(sigCh, sig...)
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
// Wait for the signal.
|
||||
<-sigCh
|
||||
stopProcess := func() bool {
|
||||
var err, oerr error
|
||||
|
||||
// Once signal has been received stop signal Notify handler.
|
||||
signal.Stop(sigCh)
|
||||
err = globalHTTPServer.Shutdown()
|
||||
errorIf(err, "Unable to shutdown http server")
|
||||
|
||||
// Notify the caller.
|
||||
trapCh <- true
|
||||
}(trapCh)
|
||||
if objAPI := newObjectLayerFn(); objAPI != nil {
|
||||
oerr = objAPI.Shutdown()
|
||||
errorIf(oerr, "Unable to shutdown object layer")
|
||||
}
|
||||
|
||||
return trapCh
|
||||
return (err == nil && oerr == nil)
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case err := <-globalHTTPServerErrorCh:
|
||||
errorIf(err, "http server exited abnormally")
|
||||
var oerr error
|
||||
if objAPI := newObjectLayerFn(); objAPI != nil {
|
||||
oerr = objAPI.Shutdown()
|
||||
errorIf(oerr, "Unable to shutdown object layer")
|
||||
}
|
||||
|
||||
exit(err == nil && oerr == nil)
|
||||
case osSignal := <-globalOSSignalCh:
|
||||
log.Printf("Exiting on signal %v\n", osSignal)
|
||||
exit(stopProcess())
|
||||
case signal := <-globalServiceSignalCh:
|
||||
switch signal {
|
||||
case serviceStatus:
|
||||
// Ignore this at the moment.
|
||||
case serviceRestart:
|
||||
log.Println("Restarting on service signal")
|
||||
err := globalHTTPServer.Shutdown()
|
||||
errorIf(err, "Unable to shutdown http server")
|
||||
rerr := restartProcess()
|
||||
errorIf(rerr, "Unable to restart the server")
|
||||
|
||||
exit(err == nil && rerr == nil)
|
||||
case serviceStop:
|
||||
log.Println("Stopping on service signal")
|
||||
exit(stopProcess())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user