mirror of https://github.com/minio/minio.git
allow bootstrap to capture time-spent for each initializers (#17900)
This commit is contained in:
parent
adb8be069e
commit
af564b8ba0
|
@ -19,98 +19,51 @@ package cmd
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/minio/madmin-go/v3"
|
"github.com/minio/madmin-go/v3"
|
||||||
"github.com/minio/minio/internal/pubsub"
|
"github.com/minio/minio/internal/pubsub"
|
||||||
)
|
)
|
||||||
|
|
||||||
const bootstrapMsgsLimit = 4 << 10
|
const bootstrapTraceLimit = 4 << 10
|
||||||
|
|
||||||
type bootstrapInfo struct {
|
|
||||||
msg string
|
|
||||||
ts time.Time
|
|
||||||
source string
|
|
||||||
}
|
|
||||||
type bootstrapTracer struct {
|
type bootstrapTracer struct {
|
||||||
mu sync.RWMutex
|
mu sync.RWMutex
|
||||||
idx int
|
info []madmin.TraceInfo
|
||||||
info [bootstrapMsgsLimit]bootstrapInfo
|
|
||||||
lastUpdate time.Time
|
|
||||||
}
|
}
|
||||||
|
|
||||||
var globalBootstrapTracer = &bootstrapTracer{}
|
var globalBootstrapTracer = &bootstrapTracer{}
|
||||||
|
|
||||||
func (bs *bootstrapTracer) DropEvents() {
|
func (bs *bootstrapTracer) Record(info madmin.TraceInfo) {
|
||||||
bs.mu.Lock()
|
bs.mu.Lock()
|
||||||
defer bs.mu.Unlock()
|
defer bs.mu.Unlock()
|
||||||
|
|
||||||
if time.Now().UTC().Sub(bs.lastUpdate) > 24*time.Hour {
|
if len(bs.info) > bootstrapTraceLimit {
|
||||||
bs.info = [4096]bootstrapInfo{}
|
return
|
||||||
bs.idx = 0
|
|
||||||
}
|
}
|
||||||
}
|
bs.info = append(bs.info, info)
|
||||||
|
|
||||||
func (bs *bootstrapTracer) Empty() bool {
|
|
||||||
var empty bool
|
|
||||||
bs.mu.RLock()
|
|
||||||
empty = bs.info[0].msg == ""
|
|
||||||
bs.mu.RUnlock()
|
|
||||||
|
|
||||||
return empty
|
|
||||||
}
|
|
||||||
|
|
||||||
func (bs *bootstrapTracer) Record(msg string, skip int) {
|
|
||||||
source := getSource(skip + 1)
|
|
||||||
bs.mu.Lock()
|
|
||||||
now := time.Now().UTC()
|
|
||||||
bs.info[bs.idx] = bootstrapInfo{
|
|
||||||
msg: msg,
|
|
||||||
ts: now,
|
|
||||||
source: source,
|
|
||||||
}
|
|
||||||
bs.lastUpdate = now
|
|
||||||
bs.idx = (bs.idx + 1) % bootstrapMsgsLimit
|
|
||||||
bs.mu.Unlock()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bs *bootstrapTracer) Events() []madmin.TraceInfo {
|
func (bs *bootstrapTracer) Events() []madmin.TraceInfo {
|
||||||
traceInfo := make([]madmin.TraceInfo, 0, bootstrapMsgsLimit)
|
traceInfo := make([]madmin.TraceInfo, 0, bootstrapTraceLimit)
|
||||||
|
|
||||||
// Add all messages in order
|
|
||||||
addAll := func(info []bootstrapInfo) {
|
|
||||||
for _, msg := range info {
|
|
||||||
if msg.ts.IsZero() {
|
|
||||||
continue // skip empty events
|
|
||||||
}
|
|
||||||
traceInfo = append(traceInfo, madmin.TraceInfo{
|
|
||||||
TraceType: madmin.TraceBootstrap,
|
|
||||||
Time: msg.ts,
|
|
||||||
NodeName: globalLocalNodeName,
|
|
||||||
FuncName: "BOOTSTRAP",
|
|
||||||
Message: fmt.Sprintf("%s %s", msg.source, msg.msg),
|
|
||||||
})
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bs.mu.RLock()
|
bs.mu.RLock()
|
||||||
addAll(bs.info[bs.idx:])
|
for _, i := range bs.info {
|
||||||
addAll(bs.info[:bs.idx])
|
traceInfo = append(traceInfo, i)
|
||||||
|
}
|
||||||
bs.mu.RUnlock()
|
bs.mu.RUnlock()
|
||||||
|
|
||||||
return traceInfo
|
return traceInfo
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bs *bootstrapTracer) Publish(ctx context.Context, trace *pubsub.PubSub[madmin.TraceInfo, madmin.TraceType]) {
|
func (bs *bootstrapTracer) Publish(ctx context.Context, trace *pubsub.PubSub[madmin.TraceInfo, madmin.TraceType]) {
|
||||||
if bs.Empty() {
|
|
||||||
return
|
|
||||||
}
|
|
||||||
for _, bsEvent := range bs.Events() {
|
for _, bsEvent := range bs.Events() {
|
||||||
select {
|
if bsEvent.Message != "" {
|
||||||
case <-ctx.Done():
|
select {
|
||||||
default:
|
case <-ctx.Done():
|
||||||
trace.Publish(bsEvent)
|
default:
|
||||||
|
trace.Publish(bsEvent)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,60 +0,0 @@
|
||||||
// Copyright (c) 2015-2023 MinIO, Inc.
|
|
||||||
//
|
|
||||||
// This file is part of MinIO Object Storage stack
|
|
||||||
//
|
|
||||||
// This program is free software: you can redistribute it and/or modify
|
|
||||||
// it under the terms of the GNU Affero General Public License as published by
|
|
||||||
// the Free Software Foundation, either version 3 of the License, or
|
|
||||||
// (at your option) any later version.
|
|
||||||
//
|
|
||||||
// This program is distributed in the hope that it will be useful
|
|
||||||
// but WITHOUT ANY WARRANTY; without even the implied warranty of
|
|
||||||
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
|
||||||
// GNU Affero General Public License for more details.
|
|
||||||
//
|
|
||||||
// You should have received a copy of the GNU Affero General Public License
|
|
||||||
// along with this program. If not, see <http://www.gnu.org/licenses/>.
|
|
||||||
|
|
||||||
package cmd
|
|
||||||
|
|
||||||
import (
|
|
||||||
"fmt"
|
|
||||||
"strings"
|
|
||||||
"testing"
|
|
||||||
"time"
|
|
||||||
)
|
|
||||||
|
|
||||||
func TestBootstrap(t *testing.T) {
|
|
||||||
// Bootstrap events exceed bootstrap messages limit
|
|
||||||
bsTracer := &bootstrapTracer{}
|
|
||||||
for i := 0; i < bootstrapMsgsLimit+10; i++ {
|
|
||||||
bsTracer.Record(fmt.Sprintf("msg-%d", i), 1)
|
|
||||||
}
|
|
||||||
|
|
||||||
traceInfos := bsTracer.Events()
|
|
||||||
if len(traceInfos) != bootstrapMsgsLimit {
|
|
||||||
t.Fatalf("Expected length of events %d but got %d", bootstrapMsgsLimit, len(traceInfos))
|
|
||||||
}
|
|
||||||
|
|
||||||
// Simulate the case where bootstrap events were updated a day ago
|
|
||||||
bsTracer.lastUpdate = time.Now().UTC().Add(-25 * time.Hour)
|
|
||||||
bsTracer.DropEvents()
|
|
||||||
if !bsTracer.Empty() {
|
|
||||||
t.Fatalf("Expected all bootstrap events to have been dropped, but found %d events", len(bsTracer.Events()))
|
|
||||||
}
|
|
||||||
|
|
||||||
// Fewer than 4K bootstrap events
|
|
||||||
for i := 0; i < 10; i++ {
|
|
||||||
bsTracer.Record(fmt.Sprintf("msg-%d", i), 1)
|
|
||||||
}
|
|
||||||
events := bsTracer.Events()
|
|
||||||
if len(events) != 10 {
|
|
||||||
t.Fatalf("Expected length of events %d but got %d", 10, len(events))
|
|
||||||
}
|
|
||||||
for i, traceInfo := range bsTracer.Events() {
|
|
||||||
msg := fmt.Sprintf("msg-%d", i)
|
|
||||||
if !strings.HasSuffix(traceInfo.Message, msg) {
|
|
||||||
t.Fatalf("Expected %s but got %s", msg, traceInfo.Message)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -221,7 +221,7 @@ func verifyServerSystemConfig(ctx context.Context, endpointServerPools EndpointS
|
||||||
for onlineServers < len(clnts)/2 {
|
for onlineServers < len(clnts)/2 {
|
||||||
for _, clnt := range clnts {
|
for _, clnt := range clnts {
|
||||||
if err := clnt.Verify(ctx, srcCfg); err != nil {
|
if err := clnt.Verify(ctx, srcCfg); err != nil {
|
||||||
bootstrapTrace(fmt.Sprintf("clnt.Verify: %v, endpoint: %v", err, clnt.endpoint))
|
bootstrapTraceMsg(fmt.Sprintf("clnt.Verify: %v, endpoint: %v", err, clnt.endpoint))
|
||||||
if !isNetworkError(err) {
|
if !isNetworkError(err) {
|
||||||
logger.LogOnceIf(ctx, fmt.Errorf("%s has incorrect configuration: %w", clnt.String(), err), clnt.String())
|
logger.LogOnceIf(ctx, fmt.Errorf("%s has incorrect configuration: %w", clnt.String(), err), clnt.String())
|
||||||
incorrectConfigs = append(incorrectConfigs, fmt.Errorf("%s has incorrect configuration: %w", clnt.String(), err))
|
incorrectConfigs = append(incorrectConfigs, fmt.Errorf("%s has incorrect configuration: %w", clnt.String(), err))
|
||||||
|
|
|
@ -440,7 +440,7 @@ func lookupConfigs(s config.Config, objAPI ObjectLayer) {
|
||||||
logger.LogIf(ctx, fmt.Errorf("Unable to initialize remote webhook DNS config %w", err))
|
logger.LogIf(ctx, fmt.Errorf("Unable to initialize remote webhook DNS config %w", err))
|
||||||
}
|
}
|
||||||
if err == nil && dnsURL != "" {
|
if err == nil && dnsURL != "" {
|
||||||
bootstrapTrace("initialize remote bucket DNS store")
|
bootstrapTraceMsg("initialize remote bucket DNS store")
|
||||||
globalDNSConfig, err = dns.NewOperatorDNS(dnsURL,
|
globalDNSConfig, err = dns.NewOperatorDNS(dnsURL,
|
||||||
dns.Authentication(dnsUser, dnsPass),
|
dns.Authentication(dnsUser, dnsPass),
|
||||||
dns.RootCAs(globalRootCAs))
|
dns.RootCAs(globalRootCAs))
|
||||||
|
@ -455,7 +455,7 @@ func lookupConfigs(s config.Config, objAPI ObjectLayer) {
|
||||||
}
|
}
|
||||||
|
|
||||||
if etcdCfg.Enabled {
|
if etcdCfg.Enabled {
|
||||||
bootstrapTrace("initialize etcd store")
|
bootstrapTraceMsg("initialize etcd store")
|
||||||
globalEtcdClient, err = etcd.New(etcdCfg)
|
globalEtcdClient, err = etcd.New(etcdCfg)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.LogIf(ctx, fmt.Errorf("Unable to initialize etcd config: %w", err))
|
logger.LogIf(ctx, fmt.Errorf("Unable to initialize etcd config: %w", err))
|
||||||
|
@ -514,19 +514,19 @@ func lookupConfigs(s config.Config, objAPI ObjectLayer) {
|
||||||
|
|
||||||
transport := NewHTTPTransport()
|
transport := NewHTTPTransport()
|
||||||
|
|
||||||
bootstrapTrace("initialize the event notification targets")
|
bootstrapTraceMsg("initialize the event notification targets")
|
||||||
globalNotifyTargetList, err = notify.FetchEnabledTargets(GlobalContext, s, transport)
|
globalNotifyTargetList, err = notify.FetchEnabledTargets(GlobalContext, s, transport)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.LogIf(ctx, fmt.Errorf("Unable to initialize notification target(s): %w", err))
|
logger.LogIf(ctx, fmt.Errorf("Unable to initialize notification target(s): %w", err))
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("initialize the lambda targets")
|
bootstrapTraceMsg("initialize the lambda targets")
|
||||||
globalLambdaTargetList, err = lambda.FetchEnabledTargets(GlobalContext, s, transport)
|
globalLambdaTargetList, err = lambda.FetchEnabledTargets(GlobalContext, s, transport)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.LogIf(ctx, fmt.Errorf("Unable to initialize lambda target(s): %w", err))
|
logger.LogIf(ctx, fmt.Errorf("Unable to initialize lambda target(s): %w", err))
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("applying the dynamic configuration")
|
bootstrapTraceMsg("applying the dynamic configuration")
|
||||||
// Apply dynamic config values
|
// Apply dynamic config values
|
||||||
if err := applyDynamicConfig(ctx, objAPI, s); err != nil {
|
if err := applyDynamicConfig(ctx, objAPI, s); err != nil {
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
|
@ -786,13 +786,13 @@ func getValidConfig(objAPI ObjectLayer) (config.Config, error) {
|
||||||
// from env if found and valid
|
// from env if found and valid
|
||||||
// data is optional. If nil it will be loaded from backend.
|
// data is optional. If nil it will be loaded from backend.
|
||||||
func loadConfig(objAPI ObjectLayer, data []byte) error {
|
func loadConfig(objAPI ObjectLayer, data []byte) error {
|
||||||
bootstrapTrace("load the configuration")
|
bootstrapTraceMsg("load the configuration")
|
||||||
srvCfg, err := readServerConfig(GlobalContext, objAPI, data)
|
srvCfg, err := readServerConfig(GlobalContext, objAPI, data)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("lookup the configuration")
|
bootstrapTraceMsg("lookup the configuration")
|
||||||
// Override any values from ENVs.
|
// Override any values from ENVs.
|
||||||
lookupConfigs(srvCfg, objAPI)
|
lookupConfigs(srvCfg, objAPI)
|
||||||
|
|
||||||
|
|
|
@ -2423,7 +2423,7 @@ func migrateV27ToV28() error {
|
||||||
// Migrates ${HOME}/.minio/config.json to '<export_path>/.minio.sys/config/config.json'
|
// Migrates ${HOME}/.minio/config.json to '<export_path>/.minio.sys/config/config.json'
|
||||||
// if etcd is configured then migrates /config/config.json to '<export_path>/.minio.sys/config/config.json'
|
// if etcd is configured then migrates /config/config.json to '<export_path>/.minio.sys/config/config.json'
|
||||||
func migrateConfigToMinioSys(objAPI ObjectLayer) (err error) {
|
func migrateConfigToMinioSys(objAPI ObjectLayer) (err error) {
|
||||||
bootstrapTrace("migrate config to .minio.sys/config/config.json")
|
bootstrapTraceMsg("migrate config to .minio.sys/config/config.json")
|
||||||
// Construct path to config.json for the given bucket.
|
// Construct path to config.json for the given bucket.
|
||||||
configFile := path.Join(minioConfigPrefix, minioConfigFile)
|
configFile := path.Join(minioConfigPrefix, minioConfigFile)
|
||||||
|
|
||||||
|
|
|
@ -197,10 +197,10 @@ func NewConfigSys() *ConfigSys {
|
||||||
|
|
||||||
// Initialize and load config from remote etcd or local config directory
|
// Initialize and load config from remote etcd or local config directory
|
||||||
func initConfig(objAPI ObjectLayer) (err error) {
|
func initConfig(objAPI ObjectLayer) (err error) {
|
||||||
bootstrapTrace("load the configuration")
|
bootstrapTraceMsg("load the configuration")
|
||||||
defer func() {
|
defer func() {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
bootstrapTrace(fmt.Sprintf("loading configuration failed: %v", err))
|
bootstrapTraceMsg(fmt.Sprintf("loading configuration failed: %v", err))
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
|
@ -213,7 +213,7 @@ func initConfig(objAPI ObjectLayer) (err error) {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("lookup the configuration")
|
bootstrapTraceMsg("lookup the configuration")
|
||||||
|
|
||||||
// Override any values from ENVs.
|
// Override any values from ENVs.
|
||||||
lookupConfigs(srvCfg, objAPI)
|
lookupConfigs(srvCfg, objAPI)
|
||||||
|
|
|
@ -384,10 +384,12 @@ func (iamOS *IAMObjectStore) listAllIAMConfigItems(ctx context.Context) (map[str
|
||||||
|
|
||||||
// Assumes cache is locked by caller.
|
// Assumes cache is locked by caller.
|
||||||
func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iamCache) error {
|
func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iamCache) error {
|
||||||
bootstrapTrace("loading all IAM items")
|
|
||||||
if iamOS.objAPI == nil {
|
if iamOS.objAPI == nil {
|
||||||
return errServerNotInitialized
|
return errServerNotInitialized
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bootstrapTraceMsg("loading all IAM items")
|
||||||
|
|
||||||
listedConfigItems, err := iamOS.listAllIAMConfigItems(ctx)
|
listedConfigItems, err := iamOS.listAllIAMConfigItems(ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return fmt.Errorf("unable to list IAM data: %w", err)
|
return fmt.Errorf("unable to list IAM data: %w", err)
|
||||||
|
@ -395,7 +397,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
|
|
||||||
// Loads things in the same order as `LoadIAMCache()`
|
// Loads things in the same order as `LoadIAMCache()`
|
||||||
|
|
||||||
bootstrapTrace("loading policy documents")
|
bootstrapTraceMsg("loading policy documents")
|
||||||
|
|
||||||
policiesList := listedConfigItems[policiesListKey]
|
policiesList := listedConfigItems[policiesListKey]
|
||||||
for _, item := range policiesList {
|
for _, item := range policiesList {
|
||||||
|
@ -407,7 +409,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
setDefaultCannedPolicies(cache.iamPolicyDocsMap)
|
setDefaultCannedPolicies(cache.iamPolicyDocsMap)
|
||||||
|
|
||||||
if iamOS.usersSysType == MinIOUsersSysType {
|
if iamOS.usersSysType == MinIOUsersSysType {
|
||||||
bootstrapTrace("loading regular IAM users")
|
bootstrapTraceMsg("loading regular IAM users")
|
||||||
regUsersList := listedConfigItems[usersListKey]
|
regUsersList := listedConfigItems[usersListKey]
|
||||||
for _, item := range regUsersList {
|
for _, item := range regUsersList {
|
||||||
userName := path.Dir(item)
|
userName := path.Dir(item)
|
||||||
|
@ -416,7 +418,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading regular IAM groups")
|
bootstrapTraceMsg("loading regular IAM groups")
|
||||||
groupsList := listedConfigItems[groupsListKey]
|
groupsList := listedConfigItems[groupsListKey]
|
||||||
for _, item := range groupsList {
|
for _, item := range groupsList {
|
||||||
group := path.Dir(item)
|
group := path.Dir(item)
|
||||||
|
@ -426,7 +428,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading user policy mapping")
|
bootstrapTraceMsg("loading user policy mapping")
|
||||||
userPolicyMappingsList := listedConfigItems[policyDBUsersListKey]
|
userPolicyMappingsList := listedConfigItems[policyDBUsersListKey]
|
||||||
for _, item := range userPolicyMappingsList {
|
for _, item := range userPolicyMappingsList {
|
||||||
userName := strings.TrimSuffix(item, ".json")
|
userName := strings.TrimSuffix(item, ".json")
|
||||||
|
@ -435,7 +437,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading group policy mapping")
|
bootstrapTraceMsg("loading group policy mapping")
|
||||||
groupPolicyMappingsList := listedConfigItems[policyDBGroupsListKey]
|
groupPolicyMappingsList := listedConfigItems[policyDBGroupsListKey]
|
||||||
for _, item := range groupPolicyMappingsList {
|
for _, item := range groupPolicyMappingsList {
|
||||||
groupName := strings.TrimSuffix(item, ".json")
|
groupName := strings.TrimSuffix(item, ".json")
|
||||||
|
@ -444,7 +446,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading service accounts")
|
bootstrapTraceMsg("loading service accounts")
|
||||||
svcAccList := listedConfigItems[svcAccListKey]
|
svcAccList := listedConfigItems[svcAccListKey]
|
||||||
for _, item := range svcAccList {
|
for _, item := range svcAccList {
|
||||||
userName := path.Dir(item)
|
userName := path.Dir(item)
|
||||||
|
@ -453,7 +455,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading STS users")
|
bootstrapTraceMsg("loading STS users")
|
||||||
stsUsersList := listedConfigItems[stsListKey]
|
stsUsersList := listedConfigItems[stsListKey]
|
||||||
for _, item := range stsUsersList {
|
for _, item := range stsUsersList {
|
||||||
userName := path.Dir(item)
|
userName := path.Dir(item)
|
||||||
|
@ -462,7 +464,7 @@ func (iamOS *IAMObjectStore) loadAllFromObjStore(ctx context.Context, cache *iam
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading STS policy mapping")
|
bootstrapTraceMsg("loading STS policy mapping")
|
||||||
stsPolicyMappingsList := listedConfigItems[policyDBSTSUsersListKey]
|
stsPolicyMappingsList := listedConfigItems[policyDBSTSUsersListKey]
|
||||||
for _, item := range stsPolicyMappingsList {
|
for _, item := range stsPolicyMappingsList {
|
||||||
stsName := strings.TrimSuffix(item, ".json")
|
stsName := strings.TrimSuffix(item, ".json")
|
||||||
|
|
|
@ -109,7 +109,7 @@ func getUserIdentityPath(user string, userType IAMUserType) string {
|
||||||
}
|
}
|
||||||
|
|
||||||
func saveIAMFormat(ctx context.Context, store IAMStorageAPI) error {
|
func saveIAMFormat(ctx context.Context, store IAMStorageAPI) error {
|
||||||
bootstrapTrace("Load IAM format file")
|
bootstrapTraceMsg("Load IAM format file")
|
||||||
var iamFmt iamFormat
|
var iamFmt iamFormat
|
||||||
path := getIAMFormatFilePath()
|
path := getIAMFormatFilePath()
|
||||||
if err := store.loadIAMConfig(ctx, &iamFmt, path); err != nil {
|
if err := store.loadIAMConfig(ctx, &iamFmt, path); err != nil {
|
||||||
|
@ -127,7 +127,7 @@ func saveIAMFormat(ctx context.Context, store IAMStorageAPI) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("Write IAM format file")
|
bootstrapTraceMsg("Write IAM format file")
|
||||||
// Save iam format to version 1.
|
// Save iam format to version 1.
|
||||||
if err := store.saveIAMConfig(ctx, newIAMFormatVersion1(), path); err != nil {
|
if err := store.saveIAMConfig(ctx, newIAMFormatVersion1(), path); err != nil {
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
|
@ -462,7 +462,7 @@ func setDefaultCannedPolicies(policies map[string]PolicyDoc) {
|
||||||
// LoadIAMCache reads all IAM items and populates a new iamCache object and
|
// LoadIAMCache reads all IAM items and populates a new iamCache object and
|
||||||
// replaces the in-memory cache object.
|
// replaces the in-memory cache object.
|
||||||
func (store *IAMStoreSys) LoadIAMCache(ctx context.Context) error {
|
func (store *IAMStoreSys) LoadIAMCache(ctx context.Context) error {
|
||||||
bootstrapTrace("loading IAM data")
|
bootstrapTraceMsg("loading IAM data")
|
||||||
|
|
||||||
newCache := newIamCache()
|
newCache := newIamCache()
|
||||||
|
|
||||||
|
@ -475,7 +475,7 @@ func (store *IAMStoreSys) LoadIAMCache(ctx context.Context) error {
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
bootstrapTrace("loading policy documents")
|
bootstrapTraceMsg("loading policy documents")
|
||||||
if err := store.loadPolicyDocs(ctx, newCache.iamPolicyDocsMap); err != nil {
|
if err := store.loadPolicyDocs(ctx, newCache.iamPolicyDocsMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -484,41 +484,41 @@ func (store *IAMStoreSys) LoadIAMCache(ctx context.Context) error {
|
||||||
setDefaultCannedPolicies(newCache.iamPolicyDocsMap)
|
setDefaultCannedPolicies(newCache.iamPolicyDocsMap)
|
||||||
|
|
||||||
if store.getUsersSysType() == MinIOUsersSysType {
|
if store.getUsersSysType() == MinIOUsersSysType {
|
||||||
bootstrapTrace("loading regular users")
|
bootstrapTraceMsg("loading regular users")
|
||||||
if err := store.loadUsers(ctx, regUser, newCache.iamUsersMap); err != nil {
|
if err := store.loadUsers(ctx, regUser, newCache.iamUsersMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
bootstrapTrace("loading regular groups")
|
bootstrapTraceMsg("loading regular groups")
|
||||||
if err := store.loadGroups(ctx, newCache.iamGroupsMap); err != nil {
|
if err := store.loadGroups(ctx, newCache.iamGroupsMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading user policy mapping")
|
bootstrapTraceMsg("loading user policy mapping")
|
||||||
// load polices mapped to users
|
// load polices mapped to users
|
||||||
if err := store.loadMappedPolicies(ctx, regUser, false, newCache.iamUserPolicyMap); err != nil {
|
if err := store.loadMappedPolicies(ctx, regUser, false, newCache.iamUserPolicyMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading group policy mapping")
|
bootstrapTraceMsg("loading group policy mapping")
|
||||||
// load policies mapped to groups
|
// load policies mapped to groups
|
||||||
if err := store.loadMappedPolicies(ctx, regUser, true, newCache.iamGroupPolicyMap); err != nil {
|
if err := store.loadMappedPolicies(ctx, regUser, true, newCache.iamGroupPolicyMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading service accounts")
|
bootstrapTraceMsg("loading service accounts")
|
||||||
// load service accounts
|
// load service accounts
|
||||||
if err := store.loadUsers(ctx, svcUser, newCache.iamUsersMap); err != nil {
|
if err := store.loadUsers(ctx, svcUser, newCache.iamUsersMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading STS users")
|
bootstrapTraceMsg("loading STS users")
|
||||||
// load STS temp users
|
// load STS temp users
|
||||||
if err := store.loadUsers(ctx, stsUser, newCache.iamUsersMap); err != nil {
|
if err := store.loadUsers(ctx, stsUser, newCache.iamUsersMap); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("loading STS policy mapping")
|
bootstrapTraceMsg("loading STS policy mapping")
|
||||||
// load STS policy mappings
|
// load STS policy mappings
|
||||||
if err := store.loadMappedPolicies(ctx, stsUser, false, newCache.iamUserPolicyMap); err != nil {
|
if err := store.loadMappedPolicies(ctx, stsUser, false, newCache.iamUserPolicyMap); err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
30
cmd/iam.go
30
cmd/iam.go
|
@ -187,7 +187,7 @@ func (sys *IAMSys) Initialized() bool {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Load - loads all credentials, policies and policy mappings.
|
// Load - loads all credentials, policies and policy mappings.
|
||||||
func (sys *IAMSys) Load(ctx context.Context) error {
|
func (sys *IAMSys) Load(ctx context.Context, firstTime bool) error {
|
||||||
loadStartTime := time.Now()
|
loadStartTime := time.Now()
|
||||||
err := sys.store.LoadIAMCache(ctx)
|
err := sys.store.LoadIAMCache(ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -200,6 +200,10 @@ func (sys *IAMSys) Load(ctx context.Context) error {
|
||||||
atomic.StoreUint64(&sys.LastRefreshTimeUnixNano, uint64(loadStartTime.Add(loadDuration).UnixNano()))
|
atomic.StoreUint64(&sys.LastRefreshTimeUnixNano, uint64(loadStartTime.Add(loadDuration).UnixNano()))
|
||||||
atomic.AddUint64(&sys.TotalRefreshSuccesses, 1)
|
atomic.AddUint64(&sys.TotalRefreshSuccesses, 1)
|
||||||
|
|
||||||
|
if firstTime {
|
||||||
|
bootstrapTraceMsg(fmt.Sprintf("globalIAMSys.Load(): (duration: %s)", loadDuration))
|
||||||
|
}
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case <-sys.configLoaded:
|
case <-sys.configLoaded:
|
||||||
default:
|
default:
|
||||||
|
@ -210,7 +214,7 @@ func (sys *IAMSys) Load(ctx context.Context) error {
|
||||||
|
|
||||||
// Init - initializes config system by reading entries from config/iam
|
// Init - initializes config system by reading entries from config/iam
|
||||||
func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer, etcdClient *etcd.Client, iamRefreshInterval time.Duration) {
|
func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer, etcdClient *etcd.Client, iamRefreshInterval time.Duration) {
|
||||||
bootstrapTrace("IAM initialization started")
|
bootstrapTraceMsg("IAM initialization started")
|
||||||
globalServerConfigMu.RLock()
|
globalServerConfigMu.RLock()
|
||||||
s := globalServerConfig
|
s := globalServerConfig
|
||||||
globalServerConfigMu.RUnlock()
|
globalServerConfigMu.RUnlock()
|
||||||
|
@ -300,7 +304,7 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer, etcdClient *etc
|
||||||
|
|
||||||
// Load IAM data from storage.
|
// Load IAM data from storage.
|
||||||
for {
|
for {
|
||||||
if err := sys.Load(retryCtx); err != nil {
|
if err := sys.Load(retryCtx, true); err != nil {
|
||||||
if configRetriableErrors(err) {
|
if configRetriableErrors(err) {
|
||||||
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
logger.Info("Waiting for all MinIO IAM sub-system to be initialized.. possible cause (%v)", err)
|
||||||
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
time.Sleep(time.Duration(r.Float64() * float64(5*time.Second)))
|
||||||
|
@ -313,8 +317,6 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer, etcdClient *etc
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("finishing IAM loading")
|
|
||||||
|
|
||||||
refreshInterval := sys.iamRefreshInterval
|
refreshInterval := sys.iamRefreshInterval
|
||||||
|
|
||||||
// Set up polling for expired accounts and credentials purging.
|
// Set up polling for expired accounts and credentials purging.
|
||||||
|
@ -371,6 +373,8 @@ func (sys *IAMSys) Init(ctx context.Context, objAPI ObjectLayer, etcdClient *etc
|
||||||
}
|
}
|
||||||
|
|
||||||
sys.printIAMRoles()
|
sys.printIAMRoles()
|
||||||
|
|
||||||
|
bootstrapTraceMsg("finishing IAM loading")
|
||||||
}
|
}
|
||||||
|
|
||||||
func (sys *IAMSys) validateAndAddRolePolicyMappings(ctx context.Context, m map[arn.ARN]string) {
|
func (sys *IAMSys) validateAndAddRolePolicyMappings(ctx context.Context, m map[arn.ARN]string) {
|
||||||
|
@ -446,7 +450,7 @@ func (sys *IAMSys) watch(ctx context.Context) {
|
||||||
select {
|
select {
|
||||||
case <-timer.C:
|
case <-timer.C:
|
||||||
refreshStart := time.Now()
|
refreshStart := time.Now()
|
||||||
if err := sys.Load(ctx); err != nil {
|
if err := sys.Load(ctx, false); err != nil {
|
||||||
logger.LogIf(ctx, fmt.Errorf("Failure in periodic refresh for IAM (took %.2fs): %v", time.Since(refreshStart).Seconds(), err))
|
logger.LogIf(ctx, fmt.Errorf("Failure in periodic refresh for IAM (took %.2fs): %v", time.Since(refreshStart).Seconds(), err))
|
||||||
} else {
|
} else {
|
||||||
took := time.Since(refreshStart).Seconds()
|
took := time.Since(refreshStart).Seconds()
|
||||||
|
@ -590,12 +594,7 @@ func (sys *IAMSys) ListPolicies(ctx context.Context, bucketName string) (map[str
|
||||||
return nil, errServerNotInitialized
|
return nil, errServerNotInitialized
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
return sys.store.ListPolicies(ctx, bucketName)
|
||||||
case <-sys.configLoaded:
|
|
||||||
return sys.store.ListPolicies(ctx, bucketName)
|
|
||||||
case <-ctx.Done():
|
|
||||||
return nil, ctx.Err()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListPolicyDocs - lists all canned policy docs.
|
// ListPolicyDocs - lists all canned policy docs.
|
||||||
|
@ -604,12 +603,7 @@ func (sys *IAMSys) ListPolicyDocs(ctx context.Context, bucketName string) (map[s
|
||||||
return nil, errServerNotInitialized
|
return nil, errServerNotInitialized
|
||||||
}
|
}
|
||||||
|
|
||||||
select {
|
return sys.store.ListPolicyDocs(ctx, bucketName)
|
||||||
case <-sys.configLoaded:
|
|
||||||
return sys.store.ListPolicyDocs(ctx, bucketName)
|
|
||||||
case <-ctx.Done():
|
|
||||||
return nil, ctx.Err()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// SetPolicy - sets a new named policy.
|
// SetPolicy - sets a new named policy.
|
||||||
|
|
|
@ -392,23 +392,51 @@ func configRetriableErrors(err error) bool {
|
||||||
errors.Is(err, os.ErrDeadlineExceeded)
|
errors.Is(err, os.ErrDeadlineExceeded)
|
||||||
}
|
}
|
||||||
|
|
||||||
func bootstrapTrace(msg string) {
|
func bootstrapTraceMsg(msg string) {
|
||||||
globalBootstrapTracer.Record(msg, 2)
|
info := madmin.TraceInfo{
|
||||||
|
TraceType: madmin.TraceBootstrap,
|
||||||
|
Time: UTCNow(),
|
||||||
|
NodeName: globalLocalNodeName,
|
||||||
|
FuncName: "BOOTSTRAP",
|
||||||
|
Message: fmt.Sprintf("%s %s", getSource(2), msg),
|
||||||
|
}
|
||||||
|
globalBootstrapTracer.Record(info)
|
||||||
|
|
||||||
if serverDebugLog {
|
if serverDebugLog {
|
||||||
logger.Info(fmt.Sprint(time.Now().Round(time.Millisecond).Format(time.RFC3339), " bootstrap: ", msg))
|
logger.Info(fmt.Sprint(time.Now().Round(time.Millisecond).Format(time.RFC3339), " bootstrap: ", msg))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
noSubs := globalTrace.NumSubscribers(madmin.TraceBootstrap) == 0
|
||||||
|
if noSubs {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
globalTrace.Publish(info)
|
||||||
|
}
|
||||||
|
|
||||||
|
func bootstrapTrace(msg string, worker func()) {
|
||||||
|
if serverDebugLog {
|
||||||
|
logger.Info(fmt.Sprint(time.Now().Round(time.Millisecond).Format(time.RFC3339), " bootstrap: ", msg))
|
||||||
|
}
|
||||||
|
|
||||||
|
now := time.Now()
|
||||||
|
worker()
|
||||||
|
dur := time.Since(now)
|
||||||
|
|
||||||
|
info := madmin.TraceInfo{
|
||||||
|
TraceType: madmin.TraceBootstrap,
|
||||||
|
Time: UTCNow(),
|
||||||
|
NodeName: globalLocalNodeName,
|
||||||
|
FuncName: "BOOTSTRAP",
|
||||||
|
Message: fmt.Sprintf("%s %s (duration: %s)", getSource(2), msg, dur),
|
||||||
|
}
|
||||||
|
globalBootstrapTracer.Record(info)
|
||||||
|
|
||||||
if globalTrace.NumSubscribers(madmin.TraceBootstrap) == 0 {
|
if globalTrace.NumSubscribers(madmin.TraceBootstrap) == 0 {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
globalTrace.Publish(madmin.TraceInfo{
|
globalTrace.Publish(info)
|
||||||
TraceType: madmin.TraceBootstrap,
|
|
||||||
Time: time.Now().UTC(),
|
|
||||||
NodeName: globalLocalNodeName,
|
|
||||||
FuncName: "BOOTSTRAP",
|
|
||||||
Message: fmt.Sprintf("%s %s", getSource(2), msg),
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func initServerConfig(ctx context.Context, newObject ObjectLayer) error {
|
func initServerConfig(ctx context.Context, newObject ObjectLayer) error {
|
||||||
|
@ -548,40 +576,40 @@ func serverMain(ctx *cli.Context) {
|
||||||
|
|
||||||
setDefaultProfilerRates()
|
setDefaultProfilerRates()
|
||||||
|
|
||||||
// Initialize globalConsoleSys system
|
|
||||||
bootstrapTrace("newConsoleLogger")
|
|
||||||
globalConsoleSys = NewConsoleLogger(GlobalContext)
|
|
||||||
logger.AddSystemTarget(GlobalContext, globalConsoleSys)
|
|
||||||
|
|
||||||
// Perform any self-tests
|
|
||||||
bootstrapTrace("selftests")
|
|
||||||
bitrotSelfTest()
|
|
||||||
erasureSelfTest()
|
|
||||||
compressSelfTest()
|
|
||||||
|
|
||||||
// Handle all server environment vars.
|
// Handle all server environment vars.
|
||||||
bootstrapTrace("serverHandleEnvVars")
|
|
||||||
serverHandleEnvVars()
|
serverHandleEnvVars()
|
||||||
|
|
||||||
// Handle all server command args.
|
// Handle all server command args.
|
||||||
bootstrapTrace("serverHandleCmdArgs")
|
bootstrapTrace("serverHandleCmdArgs", func() {
|
||||||
serverHandleCmdArgs(ctx)
|
serverHandleCmdArgs(ctx)
|
||||||
|
})
|
||||||
|
|
||||||
|
// Initialize globalConsoleSys system
|
||||||
|
bootstrapTrace("newConsoleLogger", func() {
|
||||||
|
globalConsoleSys = NewConsoleLogger(GlobalContext)
|
||||||
|
logger.AddSystemTarget(GlobalContext, globalConsoleSys)
|
||||||
|
|
||||||
|
// Set node name, only set for distributed setup.
|
||||||
|
globalConsoleSys.SetNodeName(globalLocalNodeName)
|
||||||
|
})
|
||||||
|
|
||||||
|
// Perform any self-tests
|
||||||
|
bootstrapTrace("selftests", func() {
|
||||||
|
bitrotSelfTest()
|
||||||
|
erasureSelfTest()
|
||||||
|
compressSelfTest()
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize KMS configuration
|
// Initialize KMS configuration
|
||||||
bootstrapTrace("handleKMSConfig")
|
bootstrapTrace("handleKMSConfig", handleKMSConfig)
|
||||||
handleKMSConfig()
|
|
||||||
|
|
||||||
// Set node name, only set for distributed setup.
|
|
||||||
bootstrapTrace("setNodeName")
|
|
||||||
globalConsoleSys.SetNodeName(globalLocalNodeName)
|
|
||||||
|
|
||||||
// Initialize all help
|
// Initialize all help
|
||||||
bootstrapTrace("initHelp")
|
bootstrapTrace("initHelp", initHelp)
|
||||||
initHelp()
|
|
||||||
|
|
||||||
// Initialize all sub-systems
|
// Initialize all sub-systems
|
||||||
bootstrapTrace("initAllSubsystems")
|
bootstrapTrace("initAllSubsystems", func() {
|
||||||
initAllSubsystems(GlobalContext)
|
initAllSubsystems(GlobalContext)
|
||||||
|
})
|
||||||
|
|
||||||
// Is distributed setup, error out if no certificates are found for HTTPS endpoints.
|
// Is distributed setup, error out if no certificates are found for HTTPS endpoints.
|
||||||
if globalIsDistErasure {
|
if globalIsDistErasure {
|
||||||
|
@ -597,14 +625,16 @@ func serverMain(ctx *cli.Context) {
|
||||||
go func() {
|
go func() {
|
||||||
if !globalCLIContext.Quiet && !globalInplaceUpdateDisabled {
|
if !globalCLIContext.Quiet && !globalInplaceUpdateDisabled {
|
||||||
// Check for new updates from dl.min.io.
|
// Check for new updates from dl.min.io.
|
||||||
bootstrapTrace("checkUpdate")
|
bootstrapTrace("checkUpdate", func() {
|
||||||
checkUpdate(getMinioMode())
|
checkUpdate(getMinioMode())
|
||||||
|
})
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
// Set system resources to maximum.
|
// Set system resources to maximum.
|
||||||
bootstrapTrace("setMaxResources")
|
bootstrapTrace("setMaxResources", func() {
|
||||||
setMaxResources()
|
_ = setMaxResources()
|
||||||
|
})
|
||||||
|
|
||||||
// Verify kernel release and version.
|
// Verify kernel release and version.
|
||||||
if oldLinux() {
|
if oldLinux() {
|
||||||
|
@ -617,64 +647,65 @@ func serverMain(ctx *cli.Context) {
|
||||||
logger.Info(color.RedBoldf("WARNING: Detected GOMAXPROCS(%d) < NumCPU(%d), please make sure to provide all PROCS to MinIO for optimal performance", maxProcs, cpuProcs))
|
logger.Info(color.RedBoldf("WARNING: Detected GOMAXPROCS(%d) < NumCPU(%d), please make sure to provide all PROCS to MinIO for optimal performance", maxProcs, cpuProcs))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Configure server.
|
|
||||||
bootstrapTrace("configureServerHandler")
|
|
||||||
handler, err := configureServerHandler(globalEndpoints)
|
|
||||||
if err != nil {
|
|
||||||
logger.Fatal(config.ErrUnexpectedError(err), "Unable to configure one of server's RPC services")
|
|
||||||
}
|
|
||||||
|
|
||||||
var getCert certs.GetCertificateFunc
|
var getCert certs.GetCertificateFunc
|
||||||
if globalTLSCerts != nil {
|
if globalTLSCerts != nil {
|
||||||
getCert = globalTLSCerts.GetCertificate
|
getCert = globalTLSCerts.GetCertificate
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("xhttp.NewServer")
|
// Configure server.
|
||||||
httpServer := xhttp.NewServer(getServerListenAddrs()).
|
bootstrapTrace("configureServer", func() {
|
||||||
UseHandler(setCriticalErrorHandler(corsHandler(handler))).
|
handler, err := configureServerHandler(globalEndpoints)
|
||||||
UseTLSConfig(newTLSConfig(getCert)).
|
|
||||||
UseShutdownTimeout(ctx.Duration("shutdown-timeout")).
|
|
||||||
UseIdleTimeout(ctx.Duration("idle-timeout")).
|
|
||||||
UseReadHeaderTimeout(ctx.Duration("read-header-timeout")).
|
|
||||||
UseBaseContext(GlobalContext).
|
|
||||||
UseCustomLogger(log.New(io.Discard, "", 0)). // Turn-off random logging by Go stdlib
|
|
||||||
UseTCPOptions(globalTCPOptions)
|
|
||||||
|
|
||||||
httpServer.TCPOptions.Trace = bootstrapTrace
|
|
||||||
go func() {
|
|
||||||
serveFn, err := httpServer.Init(GlobalContext, func(listenAddr string, err error) {
|
|
||||||
logger.LogIf(GlobalContext, fmt.Errorf("Unable to listen on `%s`: %v", listenAddr, err))
|
|
||||||
})
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
globalHTTPServerErrorCh <- err
|
logger.Fatal(config.ErrUnexpectedError(err), "Unable to configure one of server's RPC services")
|
||||||
return
|
|
||||||
}
|
}
|
||||||
globalHTTPServerErrorCh <- serveFn()
|
|
||||||
}()
|
|
||||||
|
|
||||||
bootstrapTrace("setHTTPServer")
|
httpServer := xhttp.NewServer(getServerListenAddrs()).
|
||||||
setHTTPServer(httpServer)
|
UseHandler(setCriticalErrorHandler(corsHandler(handler))).
|
||||||
|
UseTLSConfig(newTLSConfig(getCert)).
|
||||||
|
UseShutdownTimeout(ctx.Duration("shutdown-timeout")).
|
||||||
|
UseIdleTimeout(ctx.Duration("idle-timeout")).
|
||||||
|
UseReadHeaderTimeout(ctx.Duration("read-header-timeout")).
|
||||||
|
UseBaseContext(GlobalContext).
|
||||||
|
UseCustomLogger(log.New(io.Discard, "", 0)). // Turn-off random logging by Go stdlib
|
||||||
|
UseTCPOptions(globalTCPOptions)
|
||||||
|
|
||||||
|
httpServer.TCPOptions.Trace = bootstrapTraceMsg
|
||||||
|
go func() {
|
||||||
|
serveFn, err := httpServer.Init(GlobalContext, func(listenAddr string, err error) {
|
||||||
|
logger.LogIf(GlobalContext, fmt.Errorf("Unable to listen on `%s`: %v", listenAddr, err))
|
||||||
|
})
|
||||||
|
if err != nil {
|
||||||
|
globalHTTPServerErrorCh <- err
|
||||||
|
return
|
||||||
|
}
|
||||||
|
globalHTTPServerErrorCh <- serveFn()
|
||||||
|
}()
|
||||||
|
|
||||||
|
setHTTPServer(httpServer)
|
||||||
|
})
|
||||||
|
|
||||||
if globalIsDistErasure {
|
if globalIsDistErasure {
|
||||||
bootstrapTrace("verifying system configuration")
|
bootstrapTrace("verifying system configuration", func() {
|
||||||
// Additionally in distributed setup, validate the setup and configuration.
|
// Additionally in distributed setup, validate the setup and configuration.
|
||||||
if err := verifyServerSystemConfig(GlobalContext, globalEndpoints); err != nil {
|
if err := verifyServerSystemConfig(GlobalContext, globalEndpoints); err != nil {
|
||||||
logger.Fatal(err, "Unable to start the server")
|
logger.Fatal(err, "Unable to start the server")
|
||||||
}
|
}
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
if !globalDisableFreezeOnBoot {
|
if !globalDisableFreezeOnBoot {
|
||||||
// Freeze the services until the bucket notification subsystem gets initialized.
|
// Freeze the services until the bucket notification subsystem gets initialized.
|
||||||
bootstrapTrace("freezeServices")
|
bootstrapTrace("freezeServices", freezeServices)
|
||||||
freezeServices()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("newObjectLayer")
|
var newObject ObjectLayer
|
||||||
newObject, err := newObjectLayer(GlobalContext, globalEndpoints)
|
bootstrapTrace("newObjectLayer", func() {
|
||||||
if err != nil {
|
var err error
|
||||||
logFatalErrs(err, Endpoint{}, true)
|
newObject, err = newObjectLayer(GlobalContext, globalEndpoints)
|
||||||
}
|
if err != nil {
|
||||||
bootstrapTrace("newObjectLayer (initialized)")
|
logFatalErrs(err, Endpoint{}, true)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
xhttp.SetDeploymentID(globalDeploymentID)
|
xhttp.SetDeploymentID(globalDeploymentID)
|
||||||
xhttp.SetMinIOVersion(Version)
|
xhttp.SetMinIOVersion(Version)
|
||||||
|
@ -688,43 +719,49 @@ func serverMain(ctx *cli.Context) {
|
||||||
globalNodeNamesHex[hex.EncodeToString(nodeNameSum[:])] = struct{}{}
|
globalNodeNamesHex[hex.EncodeToString(nodeNameSum[:])] = struct{}{}
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("newSharedLock")
|
bootstrapTrace("newSharedLock", func() {
|
||||||
globalLeaderLock = newSharedLock(GlobalContext, newObject, "leader.lock")
|
globalLeaderLock = newSharedLock(GlobalContext, newObject, "leader.lock")
|
||||||
|
})
|
||||||
|
|
||||||
// Enable background operations on
|
// Enable background operations on
|
||||||
//
|
//
|
||||||
// - Disk auto healing
|
// - Disk auto healing
|
||||||
// - MRF (most recently failed) healing
|
// - MRF (most recently failed) healing
|
||||||
// - Background expiration routine for lifecycle policies
|
// - Background expiration routine for lifecycle policies
|
||||||
bootstrapTrace("initAutoHeal")
|
bootstrapTrace("initAutoHeal", func() {
|
||||||
initAutoHeal(GlobalContext, newObject)
|
initAutoHeal(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
bootstrapTrace("initHealMRF")
|
bootstrapTrace("initHealMRF", func() {
|
||||||
initHealMRF(GlobalContext, newObject)
|
initHealMRF(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
bootstrapTrace("initBackgroundExpiry")
|
bootstrapTrace("initBackgroundExpiry", func() {
|
||||||
initBackgroundExpiry(GlobalContext, newObject)
|
initBackgroundExpiry(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
bootstrapTrace("initServerConfig")
|
var err error
|
||||||
if err = initServerConfig(GlobalContext, newObject); err != nil {
|
bootstrapTrace("initServerConfig", func() {
|
||||||
var cerr config.Err
|
if err = initServerConfig(GlobalContext, newObject); err != nil {
|
||||||
// For any config error, we don't need to drop into safe-mode
|
var cerr config.Err
|
||||||
// instead its a user error and should be fixed by user.
|
// For any config error, we don't need to drop into safe-mode
|
||||||
if errors.As(err, &cerr) {
|
// instead its a user error and should be fixed by user.
|
||||||
logger.FatalIf(err, "Unable to initialize the server")
|
if errors.As(err, &cerr) {
|
||||||
|
logger.FatalIf(err, "Unable to initialize the server")
|
||||||
|
}
|
||||||
|
|
||||||
|
// If context was canceled
|
||||||
|
if errors.Is(err, context.Canceled) {
|
||||||
|
logger.FatalIf(err, "Server startup canceled upon user request")
|
||||||
|
}
|
||||||
|
|
||||||
|
logger.LogIf(GlobalContext, err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// If context was canceled
|
if !globalCLIContext.StrictS3Compat {
|
||||||
if errors.Is(err, context.Canceled) {
|
logger.Info(color.RedBold("WARNING: Strict AWS S3 compatible incoming PUT, POST content payload validation is turned off, caution is advised do not use in production"))
|
||||||
logger.FatalIf(err, "Server startup canceled upon user request")
|
|
||||||
}
|
}
|
||||||
|
})
|
||||||
logger.LogIf(GlobalContext, err)
|
|
||||||
}
|
|
||||||
|
|
||||||
if !globalCLIContext.StrictS3Compat {
|
|
||||||
logger.Info(color.RedBold("WARNING: Strict AWS S3 compatible incoming PUT, POST content payload validation is turned off, caution is advised do not use in production"))
|
|
||||||
}
|
|
||||||
|
|
||||||
if globalActiveCred.Equal(auth.DefaultCredentials) {
|
if globalActiveCred.Equal(auth.DefaultCredentials) {
|
||||||
msg := fmt.Sprintf("WARNING: Detected default credentials '%s', we recommend that you change these values with 'MINIO_ROOT_USER' and 'MINIO_ROOT_PASSWORD' environment variables",
|
msg := fmt.Sprintf("WARNING: Detected default credentials '%s', we recommend that you change these values with 'MINIO_ROOT_USER' and 'MINIO_ROOT_PASSWORD' environment variables",
|
||||||
|
@ -734,43 +771,44 @@ func serverMain(ctx *cli.Context) {
|
||||||
|
|
||||||
// Initialize users credentials and policies in background right after config has initialized.
|
// Initialize users credentials and policies in background right after config has initialized.
|
||||||
go func() {
|
go func() {
|
||||||
bootstrapTrace("globalIAMSys.Init")
|
bootstrapTrace("globalIAMSys.Init", func() {
|
||||||
globalIAMSys.Init(GlobalContext, newObject, globalEtcdClient, globalRefreshIAMInterval)
|
globalIAMSys.Init(GlobalContext, newObject, globalEtcdClient, globalRefreshIAMInterval)
|
||||||
bootstrapTrace("globalIAMSys.Initialized")
|
})
|
||||||
|
|
||||||
// Initialize Console UI
|
// Initialize Console UI
|
||||||
if globalBrowserEnabled {
|
if globalBrowserEnabled {
|
||||||
bootstrapTrace("initConsoleServer")
|
bootstrapTrace("initConsoleServer", func() {
|
||||||
srv, err := initConsoleServer()
|
srv, err := initConsoleServer()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.FatalIf(err, "Unable to initialize console service")
|
logger.FatalIf(err, "Unable to initialize console service")
|
||||||
}
|
}
|
||||||
|
|
||||||
bootstrapTrace("setConsoleSrv")
|
setConsoleSrv(srv)
|
||||||
setConsoleSrv(srv)
|
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
logger.FatalIf(newConsoleServerFn().Serve(), "Unable to initialize console server")
|
logger.FatalIf(newConsoleServerFn().Serve(), "Unable to initialize console server")
|
||||||
}()
|
}()
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// if we see FTP args, start FTP if possible
|
// if we see FTP args, start FTP if possible
|
||||||
if len(ctx.StringSlice("ftp")) > 0 {
|
if len(ctx.StringSlice("ftp")) > 0 {
|
||||||
bootstrapTrace("startFTPServer")
|
bootstrapTrace("go startFTPServer", func() {
|
||||||
go startFTPServer(ctx)
|
go startFTPServer(ctx)
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// If we see SFTP args, start SFTP if possible
|
// If we see SFTP args, start SFTP if possible
|
||||||
if len(ctx.StringSlice("sftp")) > 0 {
|
if len(ctx.StringSlice("sftp")) > 0 {
|
||||||
bootstrapTrace("startFTPServer")
|
bootstrapTrace("go startFTPServer", func() {
|
||||||
go startSFTPServer(ctx)
|
go startSFTPServer(ctx)
|
||||||
|
})
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
if !globalDisableFreezeOnBoot {
|
if !globalDisableFreezeOnBoot {
|
||||||
defer unfreezeServices()
|
defer bootstrapTrace("unfreezeServices", unfreezeServices)
|
||||||
defer bootstrapTrace("unfreezeServices")
|
|
||||||
t := time.AfterFunc(5*time.Minute, func() {
|
t := time.AfterFunc(5*time.Minute, func() {
|
||||||
logger.Info(color.Yellow("WARNING: Taking more time to initialize the config subsystem. Please set '_MINIO_DISABLE_API_FREEZE_ON_BOOT=true' to not freeze the APIs"))
|
logger.Info(color.Yellow("WARNING: Taking more time to initialize the config subsystem. Please set '_MINIO_DISABLE_API_FREEZE_ON_BOOT=true' to not freeze the APIs"))
|
||||||
})
|
})
|
||||||
|
@ -778,32 +816,38 @@ func serverMain(ctx *cli.Context) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize data scanner.
|
// Initialize data scanner.
|
||||||
bootstrapTrace("initDataScanner")
|
bootstrapTrace("initDataScanner", func() {
|
||||||
initDataScanner(GlobalContext, newObject)
|
initDataScanner(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize background replication
|
// Initialize background replication
|
||||||
bootstrapTrace("initBackgroundReplication")
|
bootstrapTrace("initBackgroundReplication", func() {
|
||||||
initBackgroundReplication(GlobalContext, newObject)
|
initBackgroundReplication(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
bootstrapTrace("globalTransitionState.Init")
|
bootstrapTrace("globalTransitionState.Init", func() {
|
||||||
globalTransitionState.Init(newObject)
|
globalTransitionState.Init(newObject)
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize batch job pool.
|
// Initialize batch job pool.
|
||||||
bootstrapTrace("newBatchJobPool")
|
bootstrapTrace("newBatchJobPool", func() {
|
||||||
globalBatchJobPool = newBatchJobPool(GlobalContext, newObject, 100)
|
globalBatchJobPool = newBatchJobPool(GlobalContext, newObject, 100)
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize the license update job
|
// Initialize the license update job
|
||||||
bootstrapTrace("initLicenseUpdateJob")
|
bootstrapTrace("initLicenseUpdateJob", func() {
|
||||||
initLicenseUpdateJob(GlobalContext, newObject)
|
initLicenseUpdateJob(GlobalContext, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
// Initialize transition tier configuration manager
|
// Initialize transition tier configuration manager
|
||||||
bootstrapTrace("globalTierConfigMgr.Init")
|
bootstrapTrace("globalTierConfigMgr.Init", func() {
|
||||||
if err := globalTierConfigMgr.Init(GlobalContext, newObject); err != nil {
|
if err := globalTierConfigMgr.Init(GlobalContext, newObject); err != nil {
|
||||||
logger.LogIf(GlobalContext, err)
|
logger.LogIf(GlobalContext, err)
|
||||||
} else {
|
} else {
|
||||||
logger.FatalIf(globalTierJournal.Init(GlobalContext), "Unable to initialize remote tier pending deletes journal")
|
logger.FatalIf(globalTierJournal.Init(GlobalContext), "Unable to initialize remote tier pending deletes journal")
|
||||||
}
|
}
|
||||||
|
})
|
||||||
}()
|
}()
|
||||||
|
|
||||||
// initialize the new disk cache objects.
|
// initialize the new disk cache objects.
|
||||||
|
@ -817,40 +861,45 @@ func serverMain(ctx *cli.Context) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize bucket notification system.
|
// Initialize bucket notification system.
|
||||||
bootstrapTrace("initBucketTargets")
|
bootstrapTrace("initBucketTargets", func() {
|
||||||
logger.LogIf(GlobalContext, globalEventNotifier.InitBucketTargets(GlobalContext, newObject))
|
logger.LogIf(GlobalContext, globalEventNotifier.InitBucketTargets(GlobalContext, newObject))
|
||||||
|
})
|
||||||
|
|
||||||
|
var buckets []BucketInfo
|
||||||
// List buckets to initialize bucket metadata sub-sys.
|
// List buckets to initialize bucket metadata sub-sys.
|
||||||
bootstrapTrace("listBuckets")
|
bootstrapTrace("listBuckets", func() {
|
||||||
buckets, err := newObject.ListBuckets(GlobalContext, BucketOptions{})
|
buckets, err = newObject.ListBuckets(GlobalContext, BucketOptions{})
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logger.LogIf(GlobalContext, fmt.Errorf("Unable to list buckets to initialize bucket metadata sub-system: %w", err))
|
logger.LogIf(GlobalContext, fmt.Errorf("Unable to list buckets to initialize bucket metadata sub-system: %w", err))
|
||||||
}
|
}
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize bucket metadata sub-system.
|
// Initialize bucket metadata sub-system.
|
||||||
bootstrapTrace("globalBucketMetadataSys.Init")
|
bootstrapTrace("globalBucketMetadataSys.Init", func() {
|
||||||
globalBucketMetadataSys.Init(GlobalContext, buckets, newObject)
|
globalBucketMetadataSys.Init(GlobalContext, buckets, newObject)
|
||||||
bootstrapTrace("globalBucketMetadataSys.Initialized")
|
})
|
||||||
|
|
||||||
// initialize replication resync state.
|
// initialize replication resync state.
|
||||||
bootstrapTrace("initResync")
|
bootstrapTrace("initResync", func() {
|
||||||
globalReplicationPool.initResync(GlobalContext, buckets, newObject)
|
globalReplicationPool.initResync(GlobalContext, buckets, newObject)
|
||||||
|
})
|
||||||
|
|
||||||
// Initialize site replication manager after bucket metadata
|
// Initialize site replication manager after bucket metadata
|
||||||
bootstrapTrace("globalSiteReplicationSys.Init")
|
bootstrapTrace("globalSiteReplicationSys.Init", func() {
|
||||||
globalSiteReplicationSys.Init(GlobalContext, newObject)
|
globalSiteReplicationSys.Init(GlobalContext, newObject)
|
||||||
bootstrapTrace("globalSiteReplicationSys.Initialized")
|
})
|
||||||
|
|
||||||
// Initialize quota manager.
|
// Initialize quota manager.
|
||||||
bootstrapTrace("globalBucketQuotaSys.Init")
|
bootstrapTrace("globalBucketQuotaSys.Init", func() {
|
||||||
globalBucketQuotaSys.Init(newObject)
|
globalBucketQuotaSys.Init(newObject)
|
||||||
bootstrapTrace("globalBucketQuotaSys.Initialized")
|
})
|
||||||
|
|
||||||
// Populate existing buckets to the etcd backend
|
// Populate existing buckets to the etcd backend
|
||||||
if globalDNSConfig != nil {
|
if globalDNSConfig != nil {
|
||||||
// Background this operation.
|
// Background this operation.
|
||||||
bootstrapTrace("go initFederatorBackend")
|
bootstrapTrace("go initFederatorBackend", func() {
|
||||||
go initFederatorBackend(buckets, newObject)
|
go initFederatorBackend(buckets, newObject)
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
// Prints the formatted startup message, if err is not nil then it prints additional information as well.
|
// Prints the formatted startup message, if err is not nil then it prints additional information as well.
|
||||||
|
@ -866,14 +915,15 @@ func serverMain(ctx *cli.Context) {
|
||||||
if region == "" {
|
if region == "" {
|
||||||
region = "us-east-1"
|
region = "us-east-1"
|
||||||
}
|
}
|
||||||
bootstrapTrace("globalMinioClient")
|
bootstrapTrace("globalMinioClient", func() {
|
||||||
globalMinioClient, err = minio.New(globalLocalNodeName, &minio.Options{
|
globalMinioClient, err = minio.New(globalLocalNodeName, &minio.Options{
|
||||||
Creds: credentials.NewStaticV4(globalActiveCred.AccessKey, globalActiveCred.SecretKey, ""),
|
Creds: credentials.NewStaticV4(globalActiveCred.AccessKey, globalActiveCred.SecretKey, ""),
|
||||||
Secure: globalIsTLS,
|
Secure: globalIsTLS,
|
||||||
Transport: globalProxyTransport,
|
Transport: globalProxyTransport,
|
||||||
Region: region,
|
Region: region,
|
||||||
|
})
|
||||||
|
logger.FatalIf(err, "Unable to initialize MinIO client")
|
||||||
})
|
})
|
||||||
logger.FatalIf(err, "Unable to initialize MinIO client")
|
|
||||||
|
|
||||||
// Add User-Agent to differentiate the requests.
|
// Add User-Agent to differentiate the requests.
|
||||||
globalMinioClient.SetAppInfo("minio-perf-test", ReleaseTag)
|
globalMinioClient.SetAppInfo("minio-perf-test", ReleaseTag)
|
||||||
|
|
Loading…
Reference in New Issue