mirror of https://github.com/minio/minio.git
Implement HTTP POST based RPC (#5840)
Added support for new RPC support using HTTP POST. RPC's arguments and reply are Gob encoded and sent as HTTP request/response body. This patch also removes Go RPC based implementation.
This commit is contained in:
parent
9d41051e91
commit
6a53dd1701
|
@ -245,7 +245,7 @@ func (a adminAPIHandlers) ServerInfoHandler(w http.ResponseWriter, r *http.Reque
|
|||
// Initialize server info at index
|
||||
reply[idx] = ServerInfo{Addr: peer.addr}
|
||||
|
||||
serverInfoData, err := peer.cmdRunner.ServerInfoData()
|
||||
serverInfoData, err := peer.cmdRunner.ServerInfo()
|
||||
if err != nil {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", peer.addr)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
|
@ -808,14 +808,6 @@ func (a adminAPIHandlers) UpdateCredentialsHandler(w http.ResponseWriter,
|
|||
globalServerConfigMu.Lock()
|
||||
defer globalServerConfigMu.Unlock()
|
||||
|
||||
// Notify all other Minio peers to update credentials
|
||||
updateErrs := updateCredsOnPeers(creds)
|
||||
for peer, err := range updateErrs {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", peer)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
// Update local credentials in memory.
|
||||
globalServerConfig.SetCredential(creds)
|
||||
if err = globalServerConfig.Save(); err != nil {
|
||||
|
@ -823,6 +815,13 @@ func (a adminAPIHandlers) UpdateCredentialsHandler(w http.ResponseWriter,
|
|||
return
|
||||
}
|
||||
|
||||
// Notify all other Minio peers to update credentials
|
||||
for host, err := range globalNotificationSys.SetCredentials(creds) {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", host.String())
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
// At this stage, the operation is successful, return 200 OK
|
||||
w.WriteHeader(http.StatusOK)
|
||||
}
|
||||
|
|
|
@ -18,39 +18,128 @@ package cmd
|
|||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"net"
|
||||
"os"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio-go/pkg/set"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
const (
|
||||
// Admin service names
|
||||
signalServiceRPC = "Admin.SignalService"
|
||||
reInitFormatRPC = "Admin.ReInitFormat"
|
||||
listLocksRPC = "Admin.ListLocks"
|
||||
serverInfoDataRPC = "Admin.ServerInfoData"
|
||||
getConfigRPC = "Admin.GetConfig"
|
||||
writeTmpConfigRPC = "Admin.WriteTmpConfig"
|
||||
commitConfigRPC = "Admin.CommitConfig"
|
||||
)
|
||||
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
|
||||
|
||||
// localAdminClient - represents admin operation to be executed locally.
|
||||
type localAdminClient struct {
|
||||
// AdminRPCClient - admin RPC client talks to admin RPC server.
|
||||
type AdminRPCClient struct {
|
||||
*RPCClient
|
||||
}
|
||||
|
||||
// remoteAdminClient - represents admin operation to be executed
|
||||
// remotely, via RPC.
|
||||
type remoteAdminClient struct {
|
||||
*AuthRPCClient
|
||||
// SignalService - calls SignalService RPC.
|
||||
func (rpcClient *AdminRPCClient) SignalService(signal serviceSignal) (err error) {
|
||||
args := SignalServiceArgs{Sig: signal}
|
||||
reply := VoidReply{}
|
||||
|
||||
return rpcClient.Call(adminServiceName+".SignalService", &args, &reply)
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format, remotely.
|
||||
func (rpcClient *AdminRPCClient) ReInitFormat(dryRun bool) error {
|
||||
args := ReInitFormatArgs{DryRun: dryRun}
|
||||
reply := VoidReply{}
|
||||
|
||||
return rpcClient.Call(adminServiceName+".ReInitFormat", &args, &reply)
|
||||
}
|
||||
|
||||
// ListLocks - Sends list locks command to remote server via RPC.
|
||||
func (rpcClient *AdminRPCClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
args := ListLocksQuery{
|
||||
Bucket: bucket,
|
||||
Prefix: prefix,
|
||||
Duration: duration,
|
||||
}
|
||||
var reply []VolumeLockInfo
|
||||
|
||||
err := rpcClient.Call(adminServiceName+".ListLocks", &args, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// ServerInfo - returns the server info of the server to which the RPC call is made.
|
||||
func (rpcClient *AdminRPCClient) ServerInfo() (sid ServerInfoData, err error) {
|
||||
err = rpcClient.Call(adminServiceName+".ServerInfo", &AuthArgs{}, &sid)
|
||||
return sid, err
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the remote server.
|
||||
func (rpcClient *AdminRPCClient) GetConfig() ([]byte, error) {
|
||||
args := AuthArgs{}
|
||||
var reply []byte
|
||||
|
||||
err := rpcClient.Call(adminServiceName+".GetConfig", &args, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on a remote node.
|
||||
func (rpcClient *AdminRPCClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
args := WriteConfigArgs{
|
||||
TmpFileName: tmpFileName,
|
||||
Buf: configBytes,
|
||||
}
|
||||
reply := VoidReply{}
|
||||
|
||||
err := rpcClient.Call(adminServiceName+".WriteTmpConfig", &args, &reply)
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json on a remote node.
|
||||
func (rpcClient *AdminRPCClient) CommitConfig(tmpFileName string) error {
|
||||
args := CommitConfigArgs{FileName: tmpFileName}
|
||||
reply := VoidReply{}
|
||||
|
||||
err := rpcClient.Call(adminServiceName+".CommitConfig", &args, &reply)
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
// NewAdminRPCClient - returns new admin RPC client.
|
||||
func NewAdminRPCClient(host *xnet.Host) (*AdminRPCClient, error) {
|
||||
scheme := "http"
|
||||
if globalIsSSL {
|
||||
scheme = "https"
|
||||
}
|
||||
|
||||
serviceURL := &xnet.URL{
|
||||
Scheme: scheme,
|
||||
Host: host.String(),
|
||||
Path: adminServicePath,
|
||||
}
|
||||
|
||||
var tlsConfig *tls.Config
|
||||
if globalIsSSL {
|
||||
tlsConfig = &tls.Config{
|
||||
ServerName: host.Name,
|
||||
RootCAs: globalRootCAs,
|
||||
}
|
||||
}
|
||||
|
||||
rpcClient, err := NewRPCClient(
|
||||
RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: adminServiceName,
|
||||
ServiceURL: serviceURL,
|
||||
TLSConfig: tlsConfig,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &AdminRPCClient{rpcClient}, nil
|
||||
}
|
||||
|
||||
// adminCmdRunner - abstracts local and remote execution of admin
|
||||
|
@ -59,189 +148,12 @@ type adminCmdRunner interface {
|
|||
SignalService(s serviceSignal) error
|
||||
ReInitFormat(dryRun bool) error
|
||||
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
||||
ServerInfoData() (ServerInfoData, error)
|
||||
ServerInfo() (ServerInfoData, error)
|
||||
GetConfig() ([]byte, error)
|
||||
WriteTmpConfig(tmpFileName string, configBytes []byte) error
|
||||
CommitConfig(tmpFileName string) error
|
||||
}
|
||||
|
||||
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
|
||||
|
||||
// SignalService - sends a restart or stop signal to the local server
|
||||
func (lc localAdminClient) SignalService(s serviceSignal) error {
|
||||
switch s {
|
||||
case serviceRestart, serviceStop:
|
||||
globalServiceSignalCh <- s
|
||||
default:
|
||||
return errUnsupportedSignal
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format.
|
||||
func (lc localAdminClient) ReInitFormat(dryRun bool) error {
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ReloadFormat(context.Background(), dryRun)
|
||||
}
|
||||
|
||||
// ListLocks - Fetches lock information from local lock instrumentation.
|
||||
func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
// check if objectLayer is initialized, if not return.
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return nil, errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ListLocks(context.Background(), bucket, prefix, duration)
|
||||
}
|
||||
|
||||
func (rc remoteAdminClient) SignalService(s serviceSignal) (err error) {
|
||||
switch s {
|
||||
case serviceRestart, serviceStop:
|
||||
reply := AuthRPCReply{}
|
||||
err = rc.Call(signalServiceRPC, &SignalServiceArgs{Sig: s},
|
||||
&reply)
|
||||
default:
|
||||
err = errUnsupportedSignal
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format, remotely.
|
||||
func (rc remoteAdminClient) ReInitFormat(dryRun bool) error {
|
||||
reply := AuthRPCReply{}
|
||||
return rc.Call(reInitFormatRPC, &ReInitFormatArgs{
|
||||
DryRun: dryRun,
|
||||
}, &reply)
|
||||
}
|
||||
|
||||
// ListLocks - Sends list locks command to remote server via RPC.
|
||||
func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
listArgs := ListLocksQuery{
|
||||
Bucket: bucket,
|
||||
Prefix: prefix,
|
||||
Duration: duration,
|
||||
}
|
||||
var reply ListLocksReply
|
||||
if err := rc.Call(listLocksRPC, &listArgs, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return reply.VolLocks, nil
|
||||
}
|
||||
|
||||
// ServerInfoData - Returns the server info of this server.
|
||||
func (lc localAdminClient) ServerInfoData() (sid ServerInfoData, e error) {
|
||||
if globalBootTime.IsZero() {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
|
||||
// Build storage info
|
||||
objLayer := newObjectLayerFn()
|
||||
if objLayer == nil {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
storage := objLayer.StorageInfo(context.Background())
|
||||
|
||||
return ServerInfoData{
|
||||
StorageInfo: storage,
|
||||
ConnStats: globalConnStats.toServerConnStats(),
|
||||
HTTPStats: globalHTTPStats.toServerHTTPStats(),
|
||||
Properties: ServerProperties{
|
||||
Uptime: UTCNow().Sub(globalBootTime),
|
||||
Version: Version,
|
||||
CommitID: CommitID,
|
||||
SQSARN: globalNotificationSys.GetARNList(),
|
||||
Region: globalServerConfig.GetRegion(),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// ServerInfo - returns the server info of the server to which the RPC call is made.
|
||||
func (rc remoteAdminClient) ServerInfoData() (sid ServerInfoData, e error) {
|
||||
args := AuthRPCArgs{}
|
||||
reply := ServerInfoDataReply{}
|
||||
err := rc.Call(serverInfoDataRPC, &args, &reply)
|
||||
if err != nil {
|
||||
return sid, err
|
||||
}
|
||||
|
||||
return reply.ServerInfoData, nil
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the local server.
|
||||
func (lc localAdminClient) GetConfig() ([]byte, error) {
|
||||
if globalServerConfig == nil {
|
||||
return nil, fmt.Errorf("config not present")
|
||||
}
|
||||
|
||||
return json.Marshal(globalServerConfig)
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the remote server.
|
||||
func (rc remoteAdminClient) GetConfig() ([]byte, error) {
|
||||
args := AuthRPCArgs{}
|
||||
reply := ConfigReply{}
|
||||
if err := rc.Call(getConfigRPC, &args, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return reply.Config, nil
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on
|
||||
// the local server.
|
||||
func (lc localAdminClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
return writeTmpConfigCommon(tmpFileName, configBytes)
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on
|
||||
// a remote node.
|
||||
func (rc remoteAdminClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
wArgs := WriteConfigArgs{
|
||||
TmpFileName: tmpFileName,
|
||||
Buf: configBytes,
|
||||
}
|
||||
|
||||
err := rc.Call(writeTmpConfigRPC, &wArgs, &WriteConfigReply{})
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json
|
||||
// on a local node.
|
||||
func (lc localAdminClient) CommitConfig(tmpFileName string) error {
|
||||
configFile := getConfigFile()
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), tmpFileName)
|
||||
|
||||
err := os.Rename(tmpConfigFile, configFile)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
reqInfo.AppendTags("configFile", configFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json
|
||||
// on a remote node.
|
||||
func (rc remoteAdminClient) CommitConfig(tmpFileName string) error {
|
||||
cArgs := CommitConfigArgs{
|
||||
FileName: tmpFileName,
|
||||
}
|
||||
cReply := CommitConfigReply{}
|
||||
err := rc.Call(commitConfigRPC, &cArgs, &cReply)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// adminPeer - represents an entity that implements admin API RPCs.
|
||||
type adminPeer struct {
|
||||
addr string
|
||||
|
@ -254,36 +166,25 @@ type adminPeers []adminPeer
|
|||
|
||||
// makeAdminPeers - helper function to construct a collection of adminPeer.
|
||||
func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
|
||||
thisPeer := globalMinioAddr
|
||||
if globalMinioHost == "" {
|
||||
// When host is not explicitly provided simply
|
||||
// use the first IPv4.
|
||||
thisPeer = net.JoinHostPort(sortIPs(localIP4.ToSlice())[0], globalMinioPort)
|
||||
localAddr := GetLocalPeer(endpoints)
|
||||
if strings.HasPrefix(localAddr, "127.0.0.1:") {
|
||||
// Use first IPv4 instead of loopback address.
|
||||
localAddr = net.JoinHostPort(sortIPs(localIP4.ToSlice())[0], globalMinioPort)
|
||||
}
|
||||
adminPeerList = append(adminPeerList, adminPeer{
|
||||
thisPeer,
|
||||
localAdminClient{},
|
||||
true,
|
||||
addr: localAddr,
|
||||
cmdRunner: localAdminClient{},
|
||||
isLocal: true,
|
||||
})
|
||||
|
||||
hostSet := set.CreateStringSet(globalMinioAddr)
|
||||
cred := globalServerConfig.GetCredential()
|
||||
serviceEndpoint := path.Join(minioReservedBucketPath, adminPath)
|
||||
for _, host := range GetRemotePeers(endpoints) {
|
||||
if hostSet.Contains(host) {
|
||||
continue
|
||||
}
|
||||
hostSet.Add(host)
|
||||
for _, hostStr := range GetRemotePeers(endpoints) {
|
||||
host, err := xnet.ParseHost(hostStr)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
rpcClient, err := NewAdminRPCClient(host)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
adminPeerList = append(adminPeerList, adminPeer{
|
||||
addr: host,
|
||||
cmdRunner: &remoteAdminClient{newAuthRPCClient(authConfig{
|
||||
accessKey: cred.AccessKey,
|
||||
secretKey: cred.SecretKey,
|
||||
serverAddr: host,
|
||||
serviceEndpoint: serviceEndpoint,
|
||||
secureConn: globalIsSSL,
|
||||
serviceName: "Admin",
|
||||
})},
|
||||
addr: hostStr,
|
||||
cmdRunner: rpcClient,
|
||||
})
|
||||
}
|
||||
|
||||
|
@ -429,7 +330,7 @@ func getPeerUptimes(peers adminPeers) (time.Duration, error) {
|
|||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
serverInfoData, rpcErr := peer.cmdRunner.ServerInfoData()
|
||||
serverInfoData, rpcErr := peer.cmdRunner.ServerInfo()
|
||||
uptimes[idx].uptime, uptimes[idx].err = serverInfoData.Properties.Uptime, rpcErr
|
||||
}(i, peer)
|
||||
}
|
||||
|
|
|
@ -0,0 +1,650 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2014, 2015, 2016, 2017, 2018 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 (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"net"
|
||||
"os"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio-go/pkg/set"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
const (
|
||||
// Admin service names
|
||||
signalServiceRPC = "Admin.SignalService"
|
||||
reInitFormatRPC = "Admin.ReInitFormat"
|
||||
listLocksRPC = "Admin.ListLocks"
|
||||
serverInfoDataRPC = "Admin.ServerInfoData"
|
||||
getConfigRPC = "Admin.GetConfig"
|
||||
writeTmpConfigRPC = "Admin.WriteTmpConfig"
|
||||
commitConfigRPC = "Admin.CommitConfig"
|
||||
)
|
||||
|
||||
// localAdminClient - represents admin operation to be executed locally.
|
||||
type localAdminClient struct {
|
||||
}
|
||||
|
||||
// remoteAdminClient - represents admin operation to be executed
|
||||
// remotely, via RPC.
|
||||
type remoteAdminClient struct {
|
||||
*AuthRPCClient
|
||||
}
|
||||
|
||||
// adminCmdRunner - abstracts local and remote execution of admin
|
||||
// commands like service stop and service restart.
|
||||
type adminCmdRunner interface {
|
||||
SignalService(s serviceSignal) error
|
||||
ReInitFormat(dryRun bool) error
|
||||
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
||||
ServerInfoData() (ServerInfoData, error)
|
||||
GetConfig() ([]byte, error)
|
||||
WriteTmpConfig(tmpFileName string, configBytes []byte) error
|
||||
CommitConfig(tmpFileName string) error
|
||||
}
|
||||
|
||||
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
|
||||
|
||||
// SignalService - sends a restart or stop signal to the local server
|
||||
func (lc localAdminClient) SignalService(s serviceSignal) error {
|
||||
switch s {
|
||||
case serviceRestart, serviceStop:
|
||||
globalServiceSignalCh <- s
|
||||
default:
|
||||
return errUnsupportedSignal
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format.
|
||||
func (lc localAdminClient) ReInitFormat(dryRun bool) error {
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ReloadFormat(context.Background(), dryRun)
|
||||
}
|
||||
|
||||
// ListLocks - Fetches lock information from local lock instrumentation.
|
||||
func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
// check if objectLayer is initialized, if not return.
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return nil, errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ListLocks(context.Background(), bucket, prefix, duration)
|
||||
}
|
||||
|
||||
func (rc remoteAdminClient) SignalService(s serviceSignal) (err error) {
|
||||
switch s {
|
||||
case serviceRestart, serviceStop:
|
||||
reply := AuthRPCReply{}
|
||||
err = rc.Call(signalServiceRPC, &SignalServiceArgs{Sig: s},
|
||||
&reply)
|
||||
default:
|
||||
err = errUnsupportedSignal
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format, remotely.
|
||||
func (rc remoteAdminClient) ReInitFormat(dryRun bool) error {
|
||||
reply := AuthRPCReply{}
|
||||
return rc.Call(reInitFormatRPC, &ReInitFormatArgs{
|
||||
DryRun: dryRun,
|
||||
}, &reply)
|
||||
}
|
||||
|
||||
// ListLocks - Sends list locks command to remote server via RPC.
|
||||
func (rc remoteAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
listArgs := ListLocksQuery{
|
||||
Bucket: bucket,
|
||||
Prefix: prefix,
|
||||
Duration: duration,
|
||||
}
|
||||
var reply ListLocksReply
|
||||
if err := rc.Call(listLocksRPC, &listArgs, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return reply.VolLocks, nil
|
||||
}
|
||||
|
||||
// ServerInfoData - Returns the server info of this server.
|
||||
func (lc localAdminClient) ServerInfoData() (sid ServerInfoData, e error) {
|
||||
if globalBootTime.IsZero() {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
|
||||
// Build storage info
|
||||
objLayer := newObjectLayerFn()
|
||||
if objLayer == nil {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
storage := objLayer.StorageInfo(context.Background())
|
||||
|
||||
return ServerInfoData{
|
||||
StorageInfo: storage,
|
||||
ConnStats: globalConnStats.toServerConnStats(),
|
||||
HTTPStats: globalHTTPStats.toServerHTTPStats(),
|
||||
Properties: ServerProperties{
|
||||
Uptime: UTCNow().Sub(globalBootTime),
|
||||
Version: Version,
|
||||
CommitID: CommitID,
|
||||
SQSARN: globalNotificationSys.GetARNList(),
|
||||
Region: globalServerConfig.GetRegion(),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// ServerInfo - returns the server info of the server to which the RPC call is made.
|
||||
func (rc remoteAdminClient) ServerInfoData() (sid ServerInfoData, e error) {
|
||||
args := AuthRPCArgs{}
|
||||
reply := ServerInfoDataReply{}
|
||||
err := rc.Call(serverInfoDataRPC, &args, &reply)
|
||||
if err != nil {
|
||||
return sid, err
|
||||
}
|
||||
|
||||
return reply.ServerInfoData, nil
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the local server.
|
||||
func (lc localAdminClient) GetConfig() ([]byte, error) {
|
||||
if globalServerConfig == nil {
|
||||
return nil, fmt.Errorf("config not present")
|
||||
}
|
||||
|
||||
return json.Marshal(globalServerConfig)
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the remote server.
|
||||
func (rc remoteAdminClient) GetConfig() ([]byte, error) {
|
||||
args := AuthRPCArgs{}
|
||||
reply := ConfigReply{}
|
||||
if err := rc.Call(getConfigRPC, &args, &reply); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return reply.Config, nil
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on
|
||||
// the local server.
|
||||
func (lc localAdminClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
return writeTmpConfigCommon(tmpFileName, configBytes)
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on
|
||||
// a remote node.
|
||||
func (rc remoteAdminClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
wArgs := WriteConfigArgs{
|
||||
TmpFileName: tmpFileName,
|
||||
Buf: configBytes,
|
||||
}
|
||||
|
||||
err := rc.Call(writeTmpConfigRPC, &wArgs, &WriteConfigReply{})
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json
|
||||
// on a local node.
|
||||
func (lc localAdminClient) CommitConfig(tmpFileName string) error {
|
||||
configFile := getConfigFile()
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), tmpFileName)
|
||||
|
||||
err := os.Rename(tmpConfigFile, configFile)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
reqInfo.AppendTags("configFile", configFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json
|
||||
// on a remote node.
|
||||
func (rc remoteAdminClient) CommitConfig(tmpFileName string) error {
|
||||
cArgs := CommitConfigArgs{
|
||||
FileName: tmpFileName,
|
||||
}
|
||||
cReply := CommitConfigReply{}
|
||||
err := rc.Call(commitConfigRPC, &cArgs, &cReply)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// adminPeer - represents an entity that implements admin API RPCs.
|
||||
type adminPeer struct {
|
||||
addr string
|
||||
cmdRunner adminCmdRunner
|
||||
isLocal bool
|
||||
}
|
||||
|
||||
// type alias for a collection of adminPeer.
|
||||
type adminPeers []adminPeer
|
||||
|
||||
// makeAdminPeers - helper function to construct a collection of adminPeer.
|
||||
func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
|
||||
thisPeer := globalMinioAddr
|
||||
if globalMinioHost == "" {
|
||||
// When host is not explicitly provided simply
|
||||
// use the first IPv4.
|
||||
thisPeer = net.JoinHostPort(sortIPs(localIP4.ToSlice())[0], globalMinioPort)
|
||||
}
|
||||
adminPeerList = append(adminPeerList, adminPeer{
|
||||
thisPeer,
|
||||
localAdminClient{},
|
||||
true,
|
||||
})
|
||||
|
||||
hostSet := set.CreateStringSet(globalMinioAddr)
|
||||
cred := globalServerConfig.GetCredential()
|
||||
serviceEndpoint := path.Join(minioReservedBucketPath, adminPath)
|
||||
for _, host := range GetRemotePeers(endpoints) {
|
||||
if hostSet.Contains(host) {
|
||||
continue
|
||||
}
|
||||
hostSet.Add(host)
|
||||
adminPeerList = append(adminPeerList, adminPeer{
|
||||
addr: host,
|
||||
cmdRunner: &remoteAdminClient{newAuthRPCClient(authConfig{
|
||||
accessKey: cred.AccessKey,
|
||||
secretKey: cred.SecretKey,
|
||||
serverAddr: host,
|
||||
serviceEndpoint: serviceEndpoint,
|
||||
secureConn: globalIsSSL,
|
||||
serviceName: "Admin",
|
||||
})},
|
||||
})
|
||||
}
|
||||
|
||||
return adminPeerList
|
||||
}
|
||||
|
||||
// peersReInitFormat - reinitialize remote object layers to new format.
|
||||
func peersReInitFormat(peers adminPeers, dryRun bool) error {
|
||||
errs := make([]error, len(peers))
|
||||
|
||||
// Send ReInitFormat RPC call to all nodes.
|
||||
// for local adminPeer this is a no-op.
|
||||
wg := sync.WaitGroup{}
|
||||
for i, peer := range peers {
|
||||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
if !peer.isLocal {
|
||||
errs[idx] = peer.cmdRunner.ReInitFormat(dryRun)
|
||||
}
|
||||
}(i, peer)
|
||||
}
|
||||
wg.Wait()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Initialize global adminPeer collection.
|
||||
func initGlobalAdminPeers(endpoints EndpointList) {
|
||||
globalAdminPeers = makeAdminPeers(endpoints)
|
||||
}
|
||||
|
||||
// invokeServiceCmd - Invoke Restart/Stop command.
|
||||
func invokeServiceCmd(cp adminPeer, cmd serviceSignal) (err error) {
|
||||
switch cmd {
|
||||
case serviceRestart, serviceStop:
|
||||
err = cp.cmdRunner.SignalService(cmd)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// sendServiceCmd - Invoke Restart command on remote peers
|
||||
// adminPeer followed by on the local peer.
|
||||
func sendServiceCmd(cps adminPeers, cmd serviceSignal) {
|
||||
// Send service command like stop or restart to all remote nodes and finally run on local node.
|
||||
errs := make([]error, len(cps))
|
||||
var wg sync.WaitGroup
|
||||
remotePeers := cps[1:]
|
||||
for i := range remotePeers {
|
||||
wg.Add(1)
|
||||
go func(idx int) {
|
||||
defer wg.Done()
|
||||
// we use idx+1 because remotePeers slice is 1 position shifted w.r.t cps
|
||||
errs[idx+1] = invokeServiceCmd(remotePeers[idx], cmd)
|
||||
}(i)
|
||||
}
|
||||
wg.Wait()
|
||||
errs[0] = invokeServiceCmd(cps[0], cmd)
|
||||
}
|
||||
|
||||
// listPeerLocksInfo - fetch list of locks held on the given bucket,
|
||||
// matching prefix held longer than duration from all peer servers.
|
||||
func listPeerLocksInfo(peers adminPeers, bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
// Used to aggregate volume lock information from all nodes.
|
||||
allLocks := make([][]VolumeLockInfo, len(peers))
|
||||
errs := make([]error, len(peers))
|
||||
var wg sync.WaitGroup
|
||||
localPeer := peers[0]
|
||||
remotePeers := peers[1:]
|
||||
for i, remotePeer := range remotePeers {
|
||||
wg.Add(1)
|
||||
go func(idx int, remotePeer adminPeer) {
|
||||
defer wg.Done()
|
||||
// `remotePeers` is right-shifted by one position relative to `peers`
|
||||
allLocks[idx], errs[idx] = remotePeer.cmdRunner.ListLocks(bucket, prefix, duration)
|
||||
}(i+1, remotePeer)
|
||||
}
|
||||
wg.Wait()
|
||||
allLocks[0], errs[0] = localPeer.cmdRunner.ListLocks(bucket, prefix, duration)
|
||||
|
||||
// Summarizing errors received for ListLocks RPC across all
|
||||
// nodes. N B the possible unavailability of quorum in errors
|
||||
// applies only to distributed setup.
|
||||
errCount, err := reduceErrs(errs, []error{})
|
||||
if err != nil {
|
||||
if errCount >= (len(peers)/2 + 1) {
|
||||
return nil, err
|
||||
}
|
||||
return nil, InsufficientReadQuorum{}
|
||||
}
|
||||
|
||||
// Group lock information across nodes by (bucket, object)
|
||||
// pair. For readability only.
|
||||
paramLockMap := make(map[nsParam][]VolumeLockInfo)
|
||||
for _, nodeLocks := range allLocks {
|
||||
for _, lockInfo := range nodeLocks {
|
||||
param := nsParam{
|
||||
volume: lockInfo.Bucket,
|
||||
path: lockInfo.Object,
|
||||
}
|
||||
paramLockMap[param] = append(paramLockMap[param], lockInfo)
|
||||
}
|
||||
}
|
||||
groupedLockInfos := []VolumeLockInfo{}
|
||||
for _, volLocks := range paramLockMap {
|
||||
groupedLockInfos = append(groupedLockInfos, volLocks...)
|
||||
}
|
||||
return groupedLockInfos, nil
|
||||
}
|
||||
|
||||
// uptimeSlice - used to sort uptimes in chronological order.
|
||||
type uptimeSlice []struct {
|
||||
err error
|
||||
uptime time.Duration
|
||||
}
|
||||
|
||||
func (ts uptimeSlice) Len() int {
|
||||
return len(ts)
|
||||
}
|
||||
|
||||
func (ts uptimeSlice) Less(i, j int) bool {
|
||||
return ts[i].uptime < ts[j].uptime
|
||||
}
|
||||
|
||||
func (ts uptimeSlice) Swap(i, j int) {
|
||||
ts[i], ts[j] = ts[j], ts[i]
|
||||
}
|
||||
|
||||
// getPeerUptimes - returns the uptime since the last time read quorum
|
||||
// was established on success. Otherwise returns errXLReadQuorum.
|
||||
func getPeerUptimes(peers adminPeers) (time.Duration, error) {
|
||||
// In a single node Erasure or FS backend setup the uptime of
|
||||
// the setup is the uptime of the single minio server
|
||||
// instance.
|
||||
if !globalIsDistXL {
|
||||
return UTCNow().Sub(globalBootTime), nil
|
||||
}
|
||||
|
||||
uptimes := make(uptimeSlice, len(peers))
|
||||
|
||||
// Get up time of all servers.
|
||||
wg := sync.WaitGroup{}
|
||||
for i, peer := range peers {
|
||||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
serverInfoData, rpcErr := peer.cmdRunner.ServerInfoData()
|
||||
uptimes[idx].uptime, uptimes[idx].err = serverInfoData.Properties.Uptime, rpcErr
|
||||
}(i, peer)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
// Sort uptimes in chronological order.
|
||||
sort.Sort(uptimes)
|
||||
|
||||
// Pick the readQuorum'th uptime in chronological order. i.e,
|
||||
// the time at which read quorum was (re-)established.
|
||||
readQuorum := len(uptimes) / 2
|
||||
validCount := 0
|
||||
latestUptime := time.Duration(0)
|
||||
for _, uptime := range uptimes {
|
||||
if uptime.err != nil {
|
||||
logger.LogIf(context.Background(), uptime.err)
|
||||
continue
|
||||
}
|
||||
|
||||
validCount++
|
||||
if validCount >= readQuorum {
|
||||
latestUptime = uptime.uptime
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// Less than readQuorum "Admin.Uptime" RPC call returned
|
||||
// successfully, so read-quorum unavailable.
|
||||
if validCount < readQuorum {
|
||||
return time.Duration(0), InsufficientReadQuorum{}
|
||||
}
|
||||
|
||||
return latestUptime, nil
|
||||
}
|
||||
|
||||
// getPeerConfig - Fetches config.json from all nodes in the setup and
|
||||
// returns the one that occurs in a majority of them.
|
||||
func getPeerConfig(peers adminPeers) ([]byte, error) {
|
||||
if !globalIsDistXL {
|
||||
return peers[0].cmdRunner.GetConfig()
|
||||
}
|
||||
|
||||
errs := make([]error, len(peers))
|
||||
configs := make([][]byte, len(peers))
|
||||
|
||||
// Get config from all servers.
|
||||
wg := sync.WaitGroup{}
|
||||
for i, peer := range peers {
|
||||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
configs[idx], errs[idx] = peer.cmdRunner.GetConfig()
|
||||
}(i, peer)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
// Find the maximally occurring config among peers in a
|
||||
// distributed setup.
|
||||
|
||||
serverConfigs := make([]serverConfig, len(peers))
|
||||
for i, configBytes := range configs {
|
||||
if errs[i] != nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Unmarshal the received config files.
|
||||
err := json.Unmarshal(configBytes, &serverConfigs[i])
|
||||
if err != nil {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", peers[i].addr)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
configJSON, err := getValidServerConfig(serverConfigs, errs)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Return the config.json that was present quorum or more
|
||||
// number of disks.
|
||||
return json.Marshal(configJSON)
|
||||
}
|
||||
|
||||
// getValidServerConfig - finds the server config that is present in
|
||||
// quorum or more number of servers.
|
||||
func getValidServerConfig(serverConfigs []serverConfig, errs []error) (scv serverConfig, e error) {
|
||||
// majority-based quorum
|
||||
quorum := len(serverConfigs)/2 + 1
|
||||
|
||||
// Count the number of disks a config.json was found in.
|
||||
configCounter := make([]int, len(serverConfigs))
|
||||
|
||||
// We group equal serverConfigs by the lowest index of the
|
||||
// same value; e.g, let us take the following serverConfigs
|
||||
// in a 4-node setup,
|
||||
// serverConfigs == [c1, c2, c1, c1]
|
||||
// configCounter == [3, 1, 0, 0]
|
||||
// c1, c2 are the only distinct values that appear. c1 is
|
||||
// identified by 0, the lowest index it appears in and c2 is
|
||||
// identified by 1. So, we need to find the number of times
|
||||
// each of these distinct values occur.
|
||||
|
||||
// Invariants:
|
||||
|
||||
// 1. At the beginning of the i-th iteration, the number of
|
||||
// unique configurations seen so far is equal to the number of
|
||||
// non-zero counter values in config[:i].
|
||||
|
||||
// 2. At the beginning of the i-th iteration, the sum of
|
||||
// elements of configCounter[:i] is equal to the number of
|
||||
// non-error configurations seen so far.
|
||||
|
||||
// For each of the serverConfig ...
|
||||
for i := range serverConfigs {
|
||||
// Skip nodes where getConfig failed.
|
||||
if errs[i] != nil {
|
||||
continue
|
||||
}
|
||||
// Check if it is equal to any of the configurations
|
||||
// seen so far. If j == i is reached then we have an
|
||||
// unseen configuration.
|
||||
for j := 0; j <= i; j++ {
|
||||
if j < i && configCounter[j] == 0 {
|
||||
// serverConfigs[j] is known to be
|
||||
// equal to a value that was already
|
||||
// seen. See example above for
|
||||
// clarity.
|
||||
continue
|
||||
} else if j < i && serverConfigs[i].ConfigDiff(&serverConfigs[j]) == "" {
|
||||
// serverConfigs[i] is equal to
|
||||
// serverConfigs[j], update
|
||||
// serverConfigs[j]'s counter since it
|
||||
// is the lower index.
|
||||
configCounter[j]++
|
||||
break
|
||||
} else if j == i {
|
||||
// serverConfigs[i] is equal to no
|
||||
// other value seen before. It is
|
||||
// unique so far.
|
||||
configCounter[i] = 1
|
||||
break
|
||||
} // else invariants specified above are violated.
|
||||
}
|
||||
}
|
||||
|
||||
// We find the maximally occurring server config and check if
|
||||
// there is quorum.
|
||||
var configJSON serverConfig
|
||||
maxOccurrence := 0
|
||||
for i, count := range configCounter {
|
||||
if maxOccurrence < count {
|
||||
maxOccurrence = count
|
||||
configJSON = serverConfigs[i]
|
||||
}
|
||||
}
|
||||
|
||||
// If quorum nodes don't agree.
|
||||
if maxOccurrence < quorum {
|
||||
return scv, errXLWriteQuorum
|
||||
}
|
||||
|
||||
return configJSON, nil
|
||||
}
|
||||
|
||||
// Write config contents into a temporary file on all nodes.
|
||||
func writeTmpConfigPeers(peers adminPeers, tmpFileName string, configBytes []byte) []error {
|
||||
// For a single-node minio server setup.
|
||||
if !globalIsDistXL {
|
||||
err := peers[0].cmdRunner.WriteTmpConfig(tmpFileName, configBytes)
|
||||
return []error{err}
|
||||
}
|
||||
|
||||
errs := make([]error, len(peers))
|
||||
|
||||
// Write config into temporary file on all nodes.
|
||||
wg := sync.WaitGroup{}
|
||||
for i, peer := range peers {
|
||||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
errs[idx] = peer.cmdRunner.WriteTmpConfig(tmpFileName, configBytes)
|
||||
}(i, peer)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
// Return bytes written and errors (if any) during writing
|
||||
// temporary config file.
|
||||
return errs
|
||||
}
|
||||
|
||||
// Move config contents from the given temporary file onto config.json
|
||||
// on all nodes.
|
||||
func commitConfigPeers(peers adminPeers, tmpFileName string) []error {
|
||||
// For a single-node minio server setup.
|
||||
if !globalIsDistXL {
|
||||
return []error{peers[0].cmdRunner.CommitConfig(tmpFileName)}
|
||||
}
|
||||
|
||||
errs := make([]error, len(peers))
|
||||
|
||||
// Rename temporary config file into configDir/config.json on
|
||||
// all nodes.
|
||||
wg := sync.WaitGroup{}
|
||||
for i, peer := range peers {
|
||||
wg.Add(1)
|
||||
go func(idx int, peer adminPeer) {
|
||||
defer wg.Done()
|
||||
errs[idx] = peer.cmdRunner.CommitConfig(tmpFileName)
|
||||
}(i, peer)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
// Return errors (if any) received during rename.
|
||||
return errs
|
||||
}
|
|
@ -1,260 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"encoding/json"
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
var (
|
||||
config1 = []byte(`{
|
||||
"version": "13",
|
||||
"credential": {
|
||||
"accessKey": "minio",
|
||||
"secretKey": "minio123"
|
||||
},
|
||||
"region": "us-east-1",
|
||||
"logger": {
|
||||
"console": {
|
||||
"enable": true,
|
||||
"level": "debug"
|
||||
},
|
||||
"file": {
|
||||
"enable": false,
|
||||
"fileName": "",
|
||||
"level": ""
|
||||
}
|
||||
},
|
||||
"notify": {
|
||||
"amqp": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"exchange": "",
|
||||
"routingKey": "",
|
||||
"exchangeType": "",
|
||||
"mandatory": false,
|
||||
"immediate": false,
|
||||
"durable": false,
|
||||
"internal": false,
|
||||
"noWait": false,
|
||||
"autoDeleted": false
|
||||
}
|
||||
},
|
||||
"nats": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"subject": "",
|
||||
"username": "",
|
||||
"password": "",
|
||||
"token": "",
|
||||
"secure": false,
|
||||
"pingInterval": 0,
|
||||
"streaming": {
|
||||
"enable": false,
|
||||
"clusterID": "",
|
||||
"clientID": "",
|
||||
"async": false,
|
||||
"maxPubAcksInflight": 0
|
||||
}
|
||||
}
|
||||
},
|
||||
"elasticsearch": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"index": ""
|
||||
}
|
||||
},
|
||||
"redis": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"password": "",
|
||||
"key": ""
|
||||
}
|
||||
},
|
||||
"postgresql": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"connectionString": "",
|
||||
"table": "",
|
||||
"host": "",
|
||||
"port": "",
|
||||
"user": "",
|
||||
"password": "",
|
||||
"database": ""
|
||||
}
|
||||
},
|
||||
"kafka": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"brokers": null,
|
||||
"topic": ""
|
||||
}
|
||||
},
|
||||
"webhook": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"endpoint": ""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
`)
|
||||
// diff from config1 - amqp.Enable is True
|
||||
config2 = []byte(`{
|
||||
"version": "13",
|
||||
"credential": {
|
||||
"accessKey": "minio",
|
||||
"secretKey": "minio123"
|
||||
},
|
||||
"region": "us-east-1",
|
||||
"logger": {
|
||||
"console": {
|
||||
"enable": true,
|
||||
"level": "debug"
|
||||
},
|
||||
"file": {
|
||||
"enable": false,
|
||||
"fileName": "",
|
||||
"level": ""
|
||||
}
|
||||
},
|
||||
"notify": {
|
||||
"amqp": {
|
||||
"1": {
|
||||
"enable": true,
|
||||
"url": "",
|
||||
"exchange": "",
|
||||
"routingKey": "",
|
||||
"exchangeType": "",
|
||||
"mandatory": false,
|
||||
"immediate": false,
|
||||
"durable": false,
|
||||
"internal": false,
|
||||
"noWait": false,
|
||||
"autoDeleted": false
|
||||
}
|
||||
},
|
||||
"nats": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"subject": "",
|
||||
"username": "",
|
||||
"password": "",
|
||||
"token": "",
|
||||
"secure": false,
|
||||
"pingInterval": 0,
|
||||
"streaming": {
|
||||
"enable": false,
|
||||
"clusterID": "",
|
||||
"clientID": "",
|
||||
"async": false,
|
||||
"maxPubAcksInflight": 0
|
||||
}
|
||||
}
|
||||
},
|
||||
"elasticsearch": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"index": ""
|
||||
}
|
||||
},
|
||||
"redis": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"password": "",
|
||||
"key": ""
|
||||
}
|
||||
},
|
||||
"postgresql": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"connectionString": "",
|
||||
"table": "",
|
||||
"host": "",
|
||||
"port": "",
|
||||
"user": "",
|
||||
"password": "",
|
||||
"database": ""
|
||||
}
|
||||
},
|
||||
"kafka": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"brokers": null,
|
||||
"topic": ""
|
||||
}
|
||||
},
|
||||
"webhook": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"endpoint": ""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
`)
|
||||
)
|
||||
|
||||
// TestGetValidServerConfig - test for getValidServerConfig.
|
||||
func TestGetValidServerConfig(t *testing.T) {
|
||||
var c1, c2 serverConfig
|
||||
err := json.Unmarshal(config1, &c1)
|
||||
if err != nil {
|
||||
t.Fatalf("json unmarshal of %s failed: %v", string(config1), err)
|
||||
}
|
||||
|
||||
err = json.Unmarshal(config2, &c2)
|
||||
if err != nil {
|
||||
t.Fatalf("json unmarshal of %s failed: %v", string(config2), err)
|
||||
}
|
||||
|
||||
// Valid config.
|
||||
noErrs := []error{nil, nil, nil, nil}
|
||||
serverConfigs := []serverConfig{c1, c2, c1, c1}
|
||||
validConfig, err := getValidServerConfig(serverConfigs, noErrs)
|
||||
if err != nil {
|
||||
t.Errorf("Expected a valid config but received %v instead", err)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(validConfig, c1) {
|
||||
t.Errorf("Expected valid config to be %v but received %v", config1, validConfig)
|
||||
}
|
||||
|
||||
// Invalid config - no quorum.
|
||||
serverConfigs = []serverConfig{c1, c2, c2, c1}
|
||||
_, err = getValidServerConfig(serverConfigs, noErrs)
|
||||
if err != errXLWriteQuorum {
|
||||
t.Errorf("Expected to fail due to lack of quorum but received %v", err)
|
||||
}
|
||||
|
||||
// All errors
|
||||
allErrs := []error{errDiskNotFound, errDiskNotFound, errDiskNotFound, errDiskNotFound}
|
||||
serverConfigs = []serverConfig{{}, {}, {}, {}}
|
||||
_, err = getValidServerConfig(serverConfigs, allErrs)
|
||||
if err != errXLWriteQuorum {
|
||||
t.Errorf("Expected to fail due to lack of quorum but received %v", err)
|
||||
}
|
||||
}
|
|
@ -18,223 +18,110 @@ package cmd
|
|||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"path"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
)
|
||||
|
||||
const adminPath = "/admin"
|
||||
const adminServiceName = "Admin"
|
||||
const adminServiceSubPath = "/admin"
|
||||
|
||||
// adminCmd - exports RPC methods for service status, stop and
|
||||
// restart commands.
|
||||
type adminCmd struct {
|
||||
AuthRPCServer
|
||||
var adminServicePath = path.Join(minioReservedBucketPath, adminServiceSubPath)
|
||||
|
||||
// adminRPCReceiver - Admin RPC receiver for admin RPC server.
|
||||
type adminRPCReceiver struct {
|
||||
local *localAdminClient
|
||||
}
|
||||
|
||||
// SignalServiceArgs - provides the signal argument to SignalService RPC
|
||||
type SignalServiceArgs struct {
|
||||
AuthRPCArgs
|
||||
AuthArgs
|
||||
Sig serviceSignal
|
||||
}
|
||||
|
||||
// SignalService - Send a restart or stop signal to the service
|
||||
func (receiver *adminRPCReceiver) SignalService(args *SignalServiceArgs, reply *VoidReply) error {
|
||||
return receiver.local.SignalService(args.Sig)
|
||||
}
|
||||
|
||||
// ListLocksQuery - wraps ListLocks API's query values to send over RPC.
|
||||
type ListLocksQuery struct {
|
||||
AuthRPCArgs
|
||||
AuthArgs
|
||||
Bucket string
|
||||
Prefix string
|
||||
Duration time.Duration
|
||||
}
|
||||
|
||||
// ListLocksReply - wraps ListLocks response over RPC.
|
||||
type ListLocksReply struct {
|
||||
AuthRPCReply
|
||||
VolLocks []VolumeLockInfo
|
||||
// ListLocks - lists locks held by requests handled by this server instance.
|
||||
func (receiver *adminRPCReceiver) ListLocks(args *ListLocksQuery, reply *[]VolumeLockInfo) (err error) {
|
||||
*reply, err = receiver.local.ListLocks(args.Bucket, args.Prefix, args.Duration)
|
||||
return err
|
||||
}
|
||||
|
||||
// ServerInfoDataReply - wraps the server info response over RPC.
|
||||
type ServerInfoDataReply struct {
|
||||
AuthRPCReply
|
||||
ServerInfoData ServerInfoData
|
||||
// ServerInfo - returns the server info when object layer was initialized on this server.
|
||||
func (receiver *adminRPCReceiver) ServerInfo(args *AuthArgs, reply *ServerInfoData) (err error) {
|
||||
*reply, err = receiver.local.ServerInfo()
|
||||
return err
|
||||
}
|
||||
|
||||
// ConfigReply - wraps the server config response over RPC.
|
||||
type ConfigReply struct {
|
||||
AuthRPCReply
|
||||
Config []byte // json-marshalled bytes of serverConfigV13
|
||||
}
|
||||
|
||||
// SignalService - Send a restart or stop signal to the service
|
||||
func (s *adminCmd) SignalService(args *SignalServiceArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
globalServiceSignalCh <- args.Sig
|
||||
return nil
|
||||
// GetConfig - returns the config.json of this server.
|
||||
func (receiver *adminRPCReceiver) GetConfig(args *AuthArgs, reply *[]byte) (err error) {
|
||||
*reply, err = receiver.local.GetConfig()
|
||||
return err
|
||||
}
|
||||
|
||||
// ReInitFormatArgs - provides dry-run information to re-initialize format.json
|
||||
type ReInitFormatArgs struct {
|
||||
AuthRPCArgs
|
||||
AuthArgs
|
||||
DryRun bool
|
||||
}
|
||||
|
||||
// ReInitFormat - re-init 'format.json'
|
||||
func (s *adminCmd) ReInitFormat(args *ReInitFormatArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ReloadFormat(context.Background(), args.DryRun)
|
||||
}
|
||||
|
||||
// ListLocks - lists locks held by requests handled by this server instance.
|
||||
func (s *adminCmd) ListLocks(query *ListLocksQuery, reply *ListLocksReply) error {
|
||||
if err := query.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
volLocks, err := objectAPI.ListLocks(context.Background(), query.Bucket, query.Prefix, query.Duration)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*reply = ListLocksReply{VolLocks: volLocks}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ServerInfo - returns the server info when object layer was initialized on this server.
|
||||
func (s *adminCmd) ServerInfoData(args *AuthRPCArgs, reply *ServerInfoDataReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if globalBootTime.IsZero() {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
|
||||
// Build storage info
|
||||
objLayer := newObjectLayerFn()
|
||||
if objLayer == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
storageInfo := objLayer.StorageInfo(context.Background())
|
||||
|
||||
reply.ServerInfoData = ServerInfoData{
|
||||
Properties: ServerProperties{
|
||||
Uptime: UTCNow().Sub(globalBootTime),
|
||||
Version: Version,
|
||||
CommitID: CommitID,
|
||||
Region: globalServerConfig.GetRegion(),
|
||||
SQSARN: globalNotificationSys.GetARNList(),
|
||||
},
|
||||
StorageInfo: storageInfo,
|
||||
ConnStats: globalConnStats.toServerConnStats(),
|
||||
HTTPStats: globalHTTPStats.toServerHTTPStats(),
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetConfig - returns the config.json of this server.
|
||||
func (s *adminCmd) GetConfig(args *AuthRPCArgs, reply *ConfigReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if globalServerConfig == nil {
|
||||
return fmt.Errorf("config not present")
|
||||
}
|
||||
|
||||
jsonBytes, err := json.Marshal(globalServerConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
reply.Config = jsonBytes
|
||||
return nil
|
||||
func (receiver *adminRPCReceiver) ReInitFormat(args *ReInitFormatArgs, reply *VoidReply) error {
|
||||
return receiver.local.ReInitFormat(args.DryRun)
|
||||
}
|
||||
|
||||
// WriteConfigArgs - wraps the bytes to be written and temporary file name.
|
||||
type WriteConfigArgs struct {
|
||||
AuthRPCArgs
|
||||
AuthArgs
|
||||
TmpFileName string
|
||||
Buf []byte
|
||||
}
|
||||
|
||||
// WriteConfigReply - wraps the result of a writing config into a temporary file.
|
||||
// the remote node.
|
||||
type WriteConfigReply struct {
|
||||
AuthRPCReply
|
||||
}
|
||||
|
||||
func writeTmpConfigCommon(tmpFileName string, configBytes []byte) error {
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), tmpFileName)
|
||||
err := ioutil.WriteFile(tmpConfigFile, configBytes, 0666)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes the supplied config contents onto the
|
||||
// supplied temporary file.
|
||||
func (s *adminCmd) WriteTmpConfig(wArgs *WriteConfigArgs, wReply *WriteConfigReply) error {
|
||||
if err := wArgs.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return writeTmpConfigCommon(wArgs.TmpFileName, wArgs.Buf)
|
||||
func (receiver *adminRPCReceiver) WriteTmpConfig(args *WriteConfigArgs, reply *VoidReply) error {
|
||||
return receiver.local.WriteTmpConfig(args.TmpFileName, args.Buf)
|
||||
}
|
||||
|
||||
// CommitConfigArgs - wraps the config file name that needs to be
|
||||
// committed into config.json on this node.
|
||||
type CommitConfigArgs struct {
|
||||
AuthRPCArgs
|
||||
AuthArgs
|
||||
FileName string
|
||||
}
|
||||
|
||||
// CommitConfigReply - represents response to commit of config file on
|
||||
// this node.
|
||||
type CommitConfigReply struct {
|
||||
AuthRPCReply
|
||||
}
|
||||
|
||||
// CommitConfig - Renames the temporary file into config.json on this node.
|
||||
func (s *adminCmd) CommitConfig(cArgs *CommitConfigArgs, cReply *CommitConfigReply) error {
|
||||
configFile := getConfigFile()
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), cArgs.FileName)
|
||||
|
||||
err := os.Rename(tmpConfigFile, configFile)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
reqInfo.AppendTags("configFile", configFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
func (receiver *adminRPCReceiver) CommitConfig(args *CommitConfigArgs, reply *VoidReply) error {
|
||||
return receiver.local.CommitConfig(args.FileName)
|
||||
}
|
||||
|
||||
// registerAdminRPCRouter - registers RPC methods for service status,
|
||||
// stop and restart commands.
|
||||
func registerAdminRPCRouter(router *mux.Router) error {
|
||||
adminRPCHandler := &adminCmd{}
|
||||
adminRPCServer := newRPCServer()
|
||||
err := adminRPCServer.RegisterName("Admin", adminRPCHandler)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
// NewAdminRPCServer - returns new admin RPC server.
|
||||
func NewAdminRPCServer() (*xrpc.Server, error) {
|
||||
rpcServer := xrpc.NewServer()
|
||||
if err := rpcServer.RegisterName(adminServiceName, &adminRPCReceiver{&localAdminClient{}}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
adminRouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
adminRouter.Path(adminPath).Handler(adminRPCServer)
|
||||
return nil
|
||||
return rpcServer, nil
|
||||
}
|
||||
|
||||
// registerAdminRPCRouter - creates and registers Admin RPC server and its router.
|
||||
func registerAdminRPCRouter(router *mux.Router) {
|
||||
rpcServer, err := NewAdminRPCServer()
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
subrouter.Path(adminServiceSubPath).Handler(rpcServer)
|
||||
}
|
||||
|
|
|
@ -1,262 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"encoding/json"
|
||||
"os"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func testAdminCmd(cmd cmdType, t *testing.T) {
|
||||
// reset globals. this is to make sure that the tests are not
|
||||
// affected by modified globals.
|
||||
resetTestGlobals()
|
||||
|
||||
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create test config - %v", err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
creds := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
adminServer := adminCmd{}
|
||||
args := LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
err = adminServer.Login(&args, &LoginRPCReply{})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to login to admin server - %v", err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
// A test signal receiver
|
||||
<-globalServiceSignalCh
|
||||
}()
|
||||
|
||||
sa := SignalServiceArgs{
|
||||
AuthRPCArgs: AuthRPCArgs{AuthToken: token, Version: globalRPCAPIVersion},
|
||||
Sig: cmd.toServiceSignal(),
|
||||
}
|
||||
|
||||
genReply := AuthRPCReply{}
|
||||
switch cmd {
|
||||
case restartCmd, stopCmd:
|
||||
if err = adminServer.SignalService(&sa, &genReply); err != nil {
|
||||
t.Errorf("restartCmd/stopCmd: Expected: <nil>, got: %v",
|
||||
err)
|
||||
}
|
||||
default:
|
||||
err = adminServer.SignalService(&sa, &genReply)
|
||||
if err != nil && err.Error() != errUnsupportedSignal.Error() {
|
||||
t.Errorf("invalidSignal %s: unexpected error got: %v",
|
||||
cmd, err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestAdminRestart - test for Admin.Restart RPC service.
|
||||
func TestAdminRestart(t *testing.T) {
|
||||
testAdminCmd(restartCmd, t)
|
||||
}
|
||||
|
||||
// TestAdminStop - test for Admin.Stop RPC service.
|
||||
func TestAdminStop(t *testing.T) {
|
||||
testAdminCmd(stopCmd, t)
|
||||
}
|
||||
|
||||
// TestAdminStatus - test for Admin.Status RPC service (error case)
|
||||
func TestAdminStatus(t *testing.T) {
|
||||
testAdminCmd(statusCmd, t)
|
||||
}
|
||||
|
||||
// TestReInitFormat - test for Admin.ReInitFormat RPC service.
|
||||
func TestReInitFormat(t *testing.T) {
|
||||
// Reset global variables to start afresh.
|
||||
resetTestGlobals()
|
||||
|
||||
rootPath, err := newTestConfig("us-east-1")
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to initialize server config. %s", err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
// Initializing objectLayer for HealFormatHandler.
|
||||
_, xlDirs, xlErr := initTestXLObjLayer()
|
||||
if xlErr != nil {
|
||||
t.Fatalf("failed to initialize XL based object layer - %v.", xlErr)
|
||||
}
|
||||
defer removeRoots(xlDirs)
|
||||
|
||||
// Set globalEndpoints for a single node XL setup.
|
||||
globalEndpoints = mustGetNewEndpointList(xlDirs...)
|
||||
|
||||
// Setup admin rpc server for an XL backend.
|
||||
globalIsXL = true
|
||||
adminServer := adminCmd{}
|
||||
|
||||
creds := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
args := LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
err = adminServer.Login(&args, &LoginRPCReply{})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to login to admin server - %v", err)
|
||||
}
|
||||
|
||||
authArgs := AuthRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
}
|
||||
authReply := AuthRPCReply{}
|
||||
|
||||
err = adminServer.ReInitFormat(&ReInitFormatArgs{
|
||||
AuthRPCArgs: authArgs,
|
||||
DryRun: false,
|
||||
}, &authReply)
|
||||
if err != nil {
|
||||
t.Errorf("Expected to pass, but failed with %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestGetConfig - Test for GetConfig admin RPC.
|
||||
func TestGetConfig(t *testing.T) {
|
||||
// Reset global variables to start afresh.
|
||||
resetTestGlobals()
|
||||
|
||||
rootPath, err := newTestConfig("us-east-1")
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to initialize server config. %s", err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
adminServer := adminCmd{}
|
||||
creds := globalServerConfig.GetCredential()
|
||||
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
args := LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
reply := LoginRPCReply{}
|
||||
err = adminServer.Login(&args, &reply)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to login to admin server - %v", err)
|
||||
}
|
||||
|
||||
authArgs := AuthRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
}
|
||||
|
||||
configReply := ConfigReply{}
|
||||
|
||||
err = adminServer.GetConfig(&authArgs, &configReply)
|
||||
if err != nil {
|
||||
t.Errorf("Expected GetConfig to pass but failed with %v", err)
|
||||
}
|
||||
|
||||
var config serverConfigV13
|
||||
err = json.Unmarshal(configReply.Config, &config)
|
||||
if err != nil {
|
||||
t.Errorf("Expected json unmarshal to pass but failed with %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestWriteAndCommitConfig - test for WriteTmpConfig and CommitConfig
|
||||
// RPC handler.
|
||||
func TestWriteAndCommitConfig(t *testing.T) {
|
||||
// Reset global variables to start afresh.
|
||||
resetTestGlobals()
|
||||
|
||||
rootPath, err := newTestConfig("us-east-1")
|
||||
if err != nil {
|
||||
t.Fatalf("Unable to initialize server config. %s", err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
adminServer := adminCmd{}
|
||||
creds := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
args := LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
reply := LoginRPCReply{}
|
||||
err = adminServer.Login(&args, &reply)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to login to admin server - %v", err)
|
||||
}
|
||||
|
||||
// Write temporary config.
|
||||
buf := []byte("hello")
|
||||
tmpFileName := mustGetUUID()
|
||||
wArgs := WriteConfigArgs{
|
||||
AuthRPCArgs: AuthRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
},
|
||||
TmpFileName: tmpFileName,
|
||||
Buf: buf,
|
||||
}
|
||||
|
||||
err = adminServer.WriteTmpConfig(&wArgs, &WriteConfigReply{})
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to write temporary config %v", err)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
t.Errorf("Expected to succeed but failed %v", err)
|
||||
}
|
||||
|
||||
cArgs := CommitConfigArgs{
|
||||
AuthRPCArgs: AuthRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
},
|
||||
FileName: tmpFileName,
|
||||
}
|
||||
cReply := CommitConfigReply{}
|
||||
|
||||
err = adminServer.CommitConfig(&cArgs, &cReply)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to commit config file %v", err)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,613 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
//
|
||||
// localAdminClient and AdminRPCClient are adminCmdRunner interface compatible,
|
||||
// hence below test functions are available for both clients.
|
||||
//
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
//
|
||||
// Admin RPC server, adminRPCReceiver and AdminRPCClient are
|
||||
// inter-dependent, below test functions are sufficient to test all of them.
|
||||
//
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
func testAdminCmdRunnerSignalService(t *testing.T, client adminCmdRunner) {
|
||||
tmpGlobalServiceSignalCh := globalServiceSignalCh
|
||||
globalServiceSignalCh = make(chan serviceSignal, 10)
|
||||
defer func() {
|
||||
globalServiceSignalCh = tmpGlobalServiceSignalCh
|
||||
}()
|
||||
|
||||
testCases := []struct {
|
||||
signal serviceSignal
|
||||
expectErr bool
|
||||
}{
|
||||
{serviceRestart, false},
|
||||
{serviceStop, false},
|
||||
{serviceStatus, true},
|
||||
{serviceSignal(100), true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := client.SignalService(testCase.signal)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerReInitFormat(t *testing.T, client adminCmdRunner) {
|
||||
tmpGlobalObjectAPI := globalObjectAPI
|
||||
defer func() {
|
||||
globalObjectAPI = tmpGlobalObjectAPI
|
||||
}()
|
||||
|
||||
testCases := []struct {
|
||||
objectAPI ObjectLayer
|
||||
dryRun bool
|
||||
expectErr bool
|
||||
}{
|
||||
{&DummyObjectLayer{}, true, false},
|
||||
{&DummyObjectLayer{}, false, false},
|
||||
{nil, true, true},
|
||||
{nil, false, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
globalObjectAPI = testCase.objectAPI
|
||||
err := client.ReInitFormat(testCase.dryRun)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerListLocks(t *testing.T, client adminCmdRunner) {
|
||||
tmpGlobalObjectAPI := globalObjectAPI
|
||||
defer func() {
|
||||
globalObjectAPI = tmpGlobalObjectAPI
|
||||
}()
|
||||
|
||||
testCases := []struct {
|
||||
objectAPI ObjectLayer
|
||||
expectErr bool
|
||||
}{
|
||||
{&DummyObjectLayer{}, false},
|
||||
{nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
globalObjectAPI = testCase.objectAPI
|
||||
_, err := client.ListLocks("", "", time.Duration(0))
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerServerInfo(t *testing.T, client adminCmdRunner) {
|
||||
tmpGlobalBootTime := globalBootTime
|
||||
tmpGlobalObjectAPI := globalObjectAPI
|
||||
tmpGlobalConnStats := globalConnStats
|
||||
tmpGlobalHTTPStats := globalHTTPStats
|
||||
tmpGlobalNotificationSys := globalNotificationSys
|
||||
defer func() {
|
||||
globalBootTime = tmpGlobalBootTime
|
||||
globalObjectAPI = tmpGlobalObjectAPI
|
||||
globalConnStats = tmpGlobalConnStats
|
||||
globalHTTPStats = tmpGlobalHTTPStats
|
||||
globalNotificationSys = tmpGlobalNotificationSys
|
||||
}()
|
||||
|
||||
endpoints := new(EndpointList)
|
||||
notificationSys, err := NewNotificationSys(globalServerConfig, *endpoints)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
bootTime time.Time
|
||||
objectAPI ObjectLayer
|
||||
connStats *ConnStats
|
||||
httpStats *HTTPStats
|
||||
notificationSys *NotificationSys
|
||||
expectErr bool
|
||||
}{
|
||||
{UTCNow(), &DummyObjectLayer{}, newConnStats(), newHTTPStats(), notificationSys, false},
|
||||
{time.Time{}, nil, nil, nil, nil, true},
|
||||
{UTCNow(), nil, nil, nil, nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
globalBootTime = testCase.bootTime
|
||||
globalObjectAPI = testCase.objectAPI
|
||||
globalConnStats = testCase.connStats
|
||||
globalHTTPStats = testCase.httpStats
|
||||
globalNotificationSys = testCase.notificationSys
|
||||
_, err := client.ServerInfo()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerGetConfig(t *testing.T, client adminCmdRunner) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
|
||||
config := newServerConfig()
|
||||
|
||||
testCases := []struct {
|
||||
config *serverConfig
|
||||
expectErr bool
|
||||
}{
|
||||
{globalServerConfig, false},
|
||||
{config, false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
globalServerConfig = testCase.config
|
||||
_, err := client.GetConfig()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerWriteTmpConfig(t *testing.T, client adminCmdRunner) {
|
||||
tmpConfigDir := configDir
|
||||
defer func() {
|
||||
configDir = tmpConfigDir
|
||||
}()
|
||||
|
||||
tempDir, err := ioutil.TempDir("", ".AdminCmdRunnerWriteTmpConfig.")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
defer os.RemoveAll(tempDir)
|
||||
configDir = &ConfigDir{dir: tempDir}
|
||||
|
||||
testCases := []struct {
|
||||
tmpFilename string
|
||||
configBytes []byte
|
||||
expectErr bool
|
||||
}{
|
||||
{"config1.json", []byte(`{"version":"23","region":"us-west-1a"}`), false},
|
||||
// Overwrite test.
|
||||
{"config1.json", []byte(`{"version":"23","region":"us-west-1a","browser":"on"}`), false},
|
||||
{"config2.json", []byte{}, false},
|
||||
{"config3.json", nil, false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := client.WriteTmpConfig(testCase.tmpFilename, testCase.configBytes)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testAdminCmdRunnerCommitConfig(t *testing.T, client adminCmdRunner) {
|
||||
tmpConfigDir := configDir
|
||||
defer func() {
|
||||
configDir = tmpConfigDir
|
||||
}()
|
||||
|
||||
tempDir, err := ioutil.TempDir("", ".AdminCmdRunnerCommitConfig.")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
defer os.RemoveAll(tempDir)
|
||||
configDir = &ConfigDir{dir: tempDir}
|
||||
err = ioutil.WriteFile(filepath.Join(tempDir, "config.json"), []byte{}, os.ModePerm)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
err = client.WriteTmpConfig("config1.json", []byte(`{"version":"23","region":"us-west-1a"}`))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
tmpFilename string
|
||||
expectErr bool
|
||||
}{
|
||||
{"config1.json", false},
|
||||
{"config2.json", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := client.CommitConfig(testCase.tmpFilename)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func newAdminRPCHTTPServerClient(t *testing.T) (*httptest.Server, *AdminRPCClient, *serverConfig) {
|
||||
rpcServer, err := NewAdminRPCServer()
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
httpServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
rpcServer.ServeHTTP(w, r)
|
||||
}))
|
||||
|
||||
url, err := xnet.ParseURL(httpServer.URL)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
host, err := xnet.ParseHost(url.Host)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
prevGlobalServerConfig := globalServerConfig
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
rpcClient, err := NewAdminRPCClient(host)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
return httpServer, rpcClient, prevGlobalServerConfig
|
||||
}
|
||||
|
||||
func TestAdminRPCClientSignalService(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerSignalService(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientReInitFormat(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerReInitFormat(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientListLocks(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerListLocks(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientServerInfo(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerServerInfo(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientGetConfig(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerGetConfig(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientWriteTmpConfig(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerWriteTmpConfig(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestAdminRPCClientCommitConfig(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig := newAdminRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
|
||||
testAdminCmdRunnerCommitConfig(t, rpcClient)
|
||||
}
|
||||
|
||||
var (
|
||||
config1 = []byte(`{
|
||||
"version": "13",
|
||||
"credential": {
|
||||
"accessKey": "minio",
|
||||
"secretKey": "minio123"
|
||||
},
|
||||
"region": "us-east-1",
|
||||
"logger": {
|
||||
"console": {
|
||||
"enable": true,
|
||||
"level": "debug"
|
||||
},
|
||||
"file": {
|
||||
"enable": false,
|
||||
"fileName": "",
|
||||
"level": ""
|
||||
}
|
||||
},
|
||||
"notify": {
|
||||
"amqp": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"exchange": "",
|
||||
"routingKey": "",
|
||||
"exchangeType": "",
|
||||
"mandatory": false,
|
||||
"immediate": false,
|
||||
"durable": false,
|
||||
"internal": false,
|
||||
"noWait": false,
|
||||
"autoDeleted": false
|
||||
}
|
||||
},
|
||||
"nats": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"subject": "",
|
||||
"username": "",
|
||||
"password": "",
|
||||
"token": "",
|
||||
"secure": false,
|
||||
"pingInterval": 0,
|
||||
"streaming": {
|
||||
"enable": false,
|
||||
"clusterID": "",
|
||||
"clientID": "",
|
||||
"async": false,
|
||||
"maxPubAcksInflight": 0
|
||||
}
|
||||
}
|
||||
},
|
||||
"elasticsearch": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"index": ""
|
||||
}
|
||||
},
|
||||
"redis": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"password": "",
|
||||
"key": ""
|
||||
}
|
||||
},
|
||||
"postgresql": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"connectionString": "",
|
||||
"table": "",
|
||||
"host": "",
|
||||
"port": "",
|
||||
"user": "",
|
||||
"password": "",
|
||||
"database": ""
|
||||
}
|
||||
},
|
||||
"kafka": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"brokers": null,
|
||||
"topic": ""
|
||||
}
|
||||
},
|
||||
"webhook": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"endpoint": ""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
`)
|
||||
// diff from config1 - amqp.Enable is True
|
||||
config2 = []byte(`{
|
||||
"version": "13",
|
||||
"credential": {
|
||||
"accessKey": "minio",
|
||||
"secretKey": "minio123"
|
||||
},
|
||||
"region": "us-east-1",
|
||||
"logger": {
|
||||
"console": {
|
||||
"enable": true,
|
||||
"level": "debug"
|
||||
},
|
||||
"file": {
|
||||
"enable": false,
|
||||
"fileName": "",
|
||||
"level": ""
|
||||
}
|
||||
},
|
||||
"notify": {
|
||||
"amqp": {
|
||||
"1": {
|
||||
"enable": true,
|
||||
"url": "",
|
||||
"exchange": "",
|
||||
"routingKey": "",
|
||||
"exchangeType": "",
|
||||
"mandatory": false,
|
||||
"immediate": false,
|
||||
"durable": false,
|
||||
"internal": false,
|
||||
"noWait": false,
|
||||
"autoDeleted": false
|
||||
}
|
||||
},
|
||||
"nats": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"subject": "",
|
||||
"username": "",
|
||||
"password": "",
|
||||
"token": "",
|
||||
"secure": false,
|
||||
"pingInterval": 0,
|
||||
"streaming": {
|
||||
"enable": false,
|
||||
"clusterID": "",
|
||||
"clientID": "",
|
||||
"async": false,
|
||||
"maxPubAcksInflight": 0
|
||||
}
|
||||
}
|
||||
},
|
||||
"elasticsearch": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"url": "",
|
||||
"index": ""
|
||||
}
|
||||
},
|
||||
"redis": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"address": "",
|
||||
"password": "",
|
||||
"key": ""
|
||||
}
|
||||
},
|
||||
"postgresql": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"connectionString": "",
|
||||
"table": "",
|
||||
"host": "",
|
||||
"port": "",
|
||||
"user": "",
|
||||
"password": "",
|
||||
"database": ""
|
||||
}
|
||||
},
|
||||
"kafka": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"brokers": null,
|
||||
"topic": ""
|
||||
}
|
||||
},
|
||||
"webhook": {
|
||||
"1": {
|
||||
"enable": false,
|
||||
"endpoint": ""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
`)
|
||||
)
|
||||
|
||||
// TestGetValidServerConfig - test for getValidServerConfig.
|
||||
func TestGetValidServerConfig(t *testing.T) {
|
||||
var c1, c2 serverConfig
|
||||
err := json.Unmarshal(config1, &c1)
|
||||
if err != nil {
|
||||
t.Fatalf("json unmarshal of %s failed: %v", string(config1), err)
|
||||
}
|
||||
|
||||
err = json.Unmarshal(config2, &c2)
|
||||
if err != nil {
|
||||
t.Fatalf("json unmarshal of %s failed: %v", string(config2), err)
|
||||
}
|
||||
|
||||
// Valid config.
|
||||
noErrs := []error{nil, nil, nil, nil}
|
||||
serverConfigs := []serverConfig{c1, c2, c1, c1}
|
||||
validConfig, err := getValidServerConfig(serverConfigs, noErrs)
|
||||
if err != nil {
|
||||
t.Errorf("Expected a valid config but received %v instead", err)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(validConfig, c1) {
|
||||
t.Errorf("Expected valid config to be %v but received %v", config1, validConfig)
|
||||
}
|
||||
|
||||
// Invalid config - no quorum.
|
||||
serverConfigs = []serverConfig{c1, c2, c2, c1}
|
||||
_, err = getValidServerConfig(serverConfigs, noErrs)
|
||||
if err != errXLWriteQuorum {
|
||||
t.Errorf("Expected to fail due to lack of quorum but received %v", err)
|
||||
}
|
||||
|
||||
// All errors
|
||||
allErrs := []error{errDiskNotFound, errDiskNotFound, errDiskNotFound, errDiskNotFound}
|
||||
serverConfigs = []serverConfig{{}, {}, {}, {}}
|
||||
_, err = getValidServerConfig(serverConfigs, allErrs)
|
||||
if err != errXLWriteQuorum {
|
||||
t.Errorf("Expected to fail due to lack of quorum but received %v", err)
|
||||
}
|
||||
}
|
|
@ -1,322 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2016, 2017, 2018 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"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/rpc"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
// Attempt to retry only this many number of times before
|
||||
// giving up on the remote RPC entirely.
|
||||
const globalAuthRPCRetryThreshold = 1
|
||||
|
||||
// authConfig requires to make new AuthRPCClient.
|
||||
type authConfig struct {
|
||||
accessKey string // Access key (like username) for authentication.
|
||||
secretKey string // Secret key (like Password) for authentication.
|
||||
serverAddr string // RPC server address.
|
||||
serviceEndpoint string // Endpoint on the server to make any RPC call.
|
||||
secureConn bool // Make TLS connection to RPC server or not.
|
||||
serviceName string // Service name of auth server.
|
||||
disableReconnect bool // Disable reconnect on failure or not.
|
||||
|
||||
/// Retry configurable values.
|
||||
|
||||
// Each retry unit multiplicative, measured in time.Duration.
|
||||
// This is the basic unit used for calculating backoffs.
|
||||
retryUnit time.Duration
|
||||
// Maximum retry duration i.e A caller would wait no more than this
|
||||
// duration to continue their loop.
|
||||
retryCap time.Duration
|
||||
|
||||
// Maximum retries an call authRPC client would do for a failed
|
||||
// RPC call.
|
||||
retryAttemptThreshold int
|
||||
}
|
||||
|
||||
// AuthRPCClient is a authenticated RPC client which does authentication before doing Call().
|
||||
type AuthRPCClient struct {
|
||||
sync.RWMutex // Mutex to lock this object.
|
||||
rpcClient *rpc.Client // RPC client to make any RPC call.
|
||||
config authConfig // Authentication configuration information.
|
||||
authToken string // Authentication token.
|
||||
version semVersion // RPC version.
|
||||
}
|
||||
|
||||
// newAuthRPCClient - returns a JWT based authenticated (go) rpc client, which does automatic reconnect.
|
||||
func newAuthRPCClient(config authConfig) *AuthRPCClient {
|
||||
// Check if retry params are set properly if not default them.
|
||||
emptyDuration := time.Duration(int64(0))
|
||||
if config.retryUnit == emptyDuration {
|
||||
config.retryUnit = defaultRetryUnit
|
||||
}
|
||||
if config.retryCap == emptyDuration {
|
||||
config.retryCap = defaultRetryCap
|
||||
}
|
||||
if config.retryAttemptThreshold == 0 {
|
||||
config.retryAttemptThreshold = globalAuthRPCRetryThreshold
|
||||
}
|
||||
|
||||
return &AuthRPCClient{
|
||||
config: config,
|
||||
version: globalRPCAPIVersion,
|
||||
}
|
||||
}
|
||||
|
||||
// Login a JWT based authentication is performed with rpc server.
|
||||
func (authClient *AuthRPCClient) Login() (err error) {
|
||||
// Login should be attempted one at a time.
|
||||
//
|
||||
// The reason for large region lock here is
|
||||
// to avoid two simultaneous login attempts
|
||||
// racing over each other.
|
||||
//
|
||||
// #1 Login() gets the lock proceeds to login.
|
||||
// #2 Login() waits for the unlock to happen
|
||||
// after login in #1.
|
||||
// #1 Successfully completes login saves the
|
||||
// newly acquired token.
|
||||
// #2 Successfully gets the lock and proceeds,
|
||||
// but since we have acquired the token
|
||||
// already the call quickly returns.
|
||||
authClient.Lock()
|
||||
defer authClient.Unlock()
|
||||
|
||||
// Attempt to login if not logged in already.
|
||||
if authClient.authToken == "" {
|
||||
var authToken string
|
||||
authToken, err = authenticateNode(authClient.config.accessKey, authClient.config.secretKey)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Login to authenticate your token.
|
||||
var (
|
||||
loginMethod = authClient.config.serviceName + loginMethodName
|
||||
loginArgs = LoginRPCArgs{
|
||||
AuthToken: authToken,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
)
|
||||
|
||||
// Re-dial after we have disconnected or if its a fresh run.
|
||||
var rpcClient *rpc.Client
|
||||
rpcClient, err = rpcDial(authClient.config.serverAddr, authClient.config.serviceEndpoint, authClient.config.secureConn)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err = rpcClient.Call(loginMethod, &loginArgs, &LoginRPCReply{}); err != nil {
|
||||
// Closing the connection here.
|
||||
rpcClient.Close()
|
||||
|
||||
// gob doesn't provide any typed errors for us to reflect
|
||||
// upon, this is the only way to return proper error.
|
||||
if strings.Contains(err.Error(), "gob: wrong type") {
|
||||
return errRPCAPIVersionUnsupported
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// Initialize rpc client and auth token after a successful login.
|
||||
authClient.authToken = authToken
|
||||
authClient.rpcClient = rpcClient
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// call makes a RPC call after logs into the server.
|
||||
func (authClient *AuthRPCClient) call(serviceMethod string, args interface {
|
||||
SetAuthToken(authToken string)
|
||||
SetRPCAPIVersion(version semVersion)
|
||||
}, reply interface{}) (err error) {
|
||||
if err = authClient.Login(); err != nil {
|
||||
return err
|
||||
} // On successful login, execute RPC call.
|
||||
|
||||
// Set token before the rpc call.
|
||||
authClient.RLock()
|
||||
defer authClient.RUnlock()
|
||||
args.SetAuthToken(authClient.authToken)
|
||||
args.SetRPCAPIVersion(authClient.version)
|
||||
|
||||
// Do an RPC call.
|
||||
return authClient.rpcClient.Call(serviceMethod, args, reply)
|
||||
}
|
||||
|
||||
// Call executes RPC call till success or globalAuthRPCRetryThreshold on ErrShutdown.
|
||||
func (authClient *AuthRPCClient) Call(serviceMethod string, args interface {
|
||||
SetAuthToken(authToken string)
|
||||
SetRPCAPIVersion(version semVersion)
|
||||
}, reply interface{}) (err error) {
|
||||
|
||||
// Done channel is used to close any lingering retry routine, as soon
|
||||
// as this function returns.
|
||||
doneCh := make(chan struct{})
|
||||
defer close(doneCh)
|
||||
|
||||
for i := range newRetryTimer(authClient.config.retryUnit, authClient.config.retryCap, doneCh) {
|
||||
if err = authClient.call(serviceMethod, args, reply); err == rpc.ErrShutdown {
|
||||
// As connection at server side is closed, close the rpc client.
|
||||
authClient.Close()
|
||||
|
||||
// Retry if reconnect is not disabled.
|
||||
if !authClient.config.disableReconnect {
|
||||
// Retry until threshold reaches.
|
||||
if i < authClient.config.retryAttemptThreshold {
|
||||
continue
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// gob doesn't provide any typed errors for us to reflect
|
||||
// upon, this is the only way to return proper error.
|
||||
if err != nil && strings.Contains(err.Error(), "gob: wrong type") {
|
||||
// Close the rpc client also when the servers have mismatching rpc versions.
|
||||
authClient.Close()
|
||||
|
||||
err = errRPCAPIVersionUnsupported
|
||||
}
|
||||
|
||||
break
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// Close closes underlying RPC Client.
|
||||
func (authClient *AuthRPCClient) Close() error {
|
||||
authClient.Lock()
|
||||
defer authClient.Unlock()
|
||||
|
||||
if authClient.rpcClient == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
authClient.authToken = ""
|
||||
return authClient.rpcClient.Close()
|
||||
}
|
||||
|
||||
// ServerAddr returns the serverAddr (network address) of the connection.
|
||||
func (authClient *AuthRPCClient) ServerAddr() string {
|
||||
return authClient.config.serverAddr
|
||||
}
|
||||
|
||||
// ServiceEndpoint returns the RPC service endpoint of the connection.
|
||||
func (authClient *AuthRPCClient) ServiceEndpoint() string {
|
||||
return authClient.config.serviceEndpoint
|
||||
}
|
||||
|
||||
// default Dial timeout for RPC connections.
|
||||
const defaultDialTimeout = 3 * time.Second
|
||||
|
||||
// Connect success message required from rpc server.
|
||||
const connectSuccessMessage = "200 Connected to Go RPC"
|
||||
|
||||
// dial tries to establish a connection to serverAddr in a safe manner.
|
||||
// If there is a valid rpc.Cliemt, it returns that else creates a new one.
|
||||
func rpcDial(serverAddr, serviceEndpoint string, secureConn bool) (netRPCClient *rpc.Client, err error) {
|
||||
if serverAddr == "" || serviceEndpoint == "" {
|
||||
return nil, errInvalidArgument
|
||||
}
|
||||
d := &net.Dialer{
|
||||
Timeout: defaultDialTimeout,
|
||||
}
|
||||
var conn net.Conn
|
||||
if secureConn {
|
||||
var hostname string
|
||||
if hostname, _, err = net.SplitHostPort(serverAddr); err != nil {
|
||||
return nil, &net.OpError{
|
||||
Op: "dial-http",
|
||||
Net: serverAddr + serviceEndpoint,
|
||||
Addr: nil,
|
||||
Err: fmt.Errorf("Unable to parse server address <%s>/<%s>: %s", serverAddr, serviceEndpoint, err),
|
||||
}
|
||||
}
|
||||
// ServerName in tls.Config needs to be specified to support SNI certificates.
|
||||
conn, err = tls.DialWithDialer(d, "tcp", serverAddr, &tls.Config{
|
||||
ServerName: hostname,
|
||||
RootCAs: globalRootCAs,
|
||||
})
|
||||
} else {
|
||||
conn, err = d.Dial("tcp", serverAddr)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
// Print RPC connection errors that are worthy to display in log.
|
||||
switch err.(type) {
|
||||
case x509.HostnameError:
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("serverAddr", serverAddr)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
return nil, &net.OpError{
|
||||
Op: "dial-http",
|
||||
Net: serverAddr + serviceEndpoint,
|
||||
Addr: nil,
|
||||
Err: err,
|
||||
}
|
||||
}
|
||||
|
||||
// Check for network errors writing over the dialed conn.
|
||||
if _, err = io.WriteString(conn, "CONNECT "+serviceEndpoint+" HTTP/1.0\n\n"); err != nil {
|
||||
conn.Close()
|
||||
return nil, &net.OpError{
|
||||
Op: "dial-http",
|
||||
Net: serverAddr + serviceEndpoint,
|
||||
Addr: nil,
|
||||
Err: err,
|
||||
}
|
||||
}
|
||||
|
||||
// Attempt to read the HTTP response for the HTTP method CONNECT, upon
|
||||
// success return the RPC connection instance.
|
||||
resp, err := http.ReadResponse(bufio.NewReader(conn), &http.Request{
|
||||
Method: http.MethodConnect,
|
||||
})
|
||||
if err != nil {
|
||||
conn.Close()
|
||||
return nil, &net.OpError{
|
||||
Op: "dial-http",
|
||||
Net: serverAddr + serviceEndpoint,
|
||||
Addr: nil,
|
||||
Err: err,
|
||||
}
|
||||
}
|
||||
if resp.Status != connectSuccessMessage {
|
||||
conn.Close()
|
||||
return nil, fmt.Errorf("Unexpected HTTP response: %s from %s/%s", resp.Status, serverAddr, serviceEndpoint)
|
||||
}
|
||||
|
||||
// Initialize rpc client.
|
||||
return rpc.NewClient(conn), nil
|
||||
}
|
|
@ -1,138 +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 (
|
||||
"crypto/x509"
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
)
|
||||
|
||||
// Tests authorized RPC client.
|
||||
func TestAuthRPCClient(t *testing.T) {
|
||||
// reset globals.
|
||||
// this is to make sure that the tests are not affected by modified globals.
|
||||
resetTestGlobals()
|
||||
|
||||
authCfg := authConfig{
|
||||
accessKey: "123",
|
||||
secretKey: "123",
|
||||
serverAddr: "localhost:9000",
|
||||
serviceEndpoint: "/rpc/disk",
|
||||
secureConn: false,
|
||||
serviceName: "MyPackage",
|
||||
}
|
||||
authRPC := newAuthRPCClient(authCfg)
|
||||
if authRPC.ServerAddr() != authCfg.serverAddr {
|
||||
t.Fatalf("Unexpected node value %s, but expected %s", authRPC.ServerAddr(), authCfg.serverAddr)
|
||||
}
|
||||
if authRPC.ServiceEndpoint() != authCfg.serviceEndpoint {
|
||||
t.Fatalf("Unexpected node value %s, but expected %s", authRPC.ServiceEndpoint(), authCfg.serviceEndpoint)
|
||||
}
|
||||
authCfg = authConfig{
|
||||
accessKey: "123",
|
||||
secretKey: "123",
|
||||
secureConn: false,
|
||||
serviceName: "MyPackage",
|
||||
}
|
||||
authRPC = newAuthRPCClient(authCfg)
|
||||
if authRPC.ServerAddr() != authCfg.serverAddr {
|
||||
t.Fatalf("Unexpected node value %s, but expected %s", authRPC.ServerAddr(), authCfg.serverAddr)
|
||||
}
|
||||
if authRPC.ServiceEndpoint() != authCfg.serviceEndpoint {
|
||||
t.Fatalf("Unexpected node value %s, but expected %s", authRPC.ServiceEndpoint(), authCfg.serviceEndpoint)
|
||||
}
|
||||
}
|
||||
|
||||
// Test rpc dial test.
|
||||
func TestRPCDial(t *testing.T) {
|
||||
prevRootCAs := globalRootCAs
|
||||
defer func() {
|
||||
globalRootCAs = prevRootCAs
|
||||
}()
|
||||
|
||||
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
testServer := StartTestServer(t, "")
|
||||
defer testServer.Stop()
|
||||
|
||||
cert, key, err := generateTLSCertKey("127.0.0.1")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Set global root CAs.
|
||||
globalRootCAs = x509.NewCertPool()
|
||||
globalRootCAs.AppendCertsFromPEM(cert)
|
||||
|
||||
testServerTLS := StartTestTLSServer(t, "", cert, key)
|
||||
defer testServerTLS.Stop()
|
||||
|
||||
adminEndpoint := path.Join(minioReservedBucketPath, adminPath)
|
||||
testCases := []struct {
|
||||
serverAddr string
|
||||
serverEndpoint string
|
||||
success bool
|
||||
secure bool
|
||||
}{
|
||||
// Empty server addr should fail.
|
||||
{
|
||||
serverAddr: "",
|
||||
serverEndpoint: adminEndpoint,
|
||||
success: false,
|
||||
},
|
||||
// Unexpected server addr should fail.
|
||||
{
|
||||
serverAddr: "example.com",
|
||||
serverEndpoint: adminEndpoint,
|
||||
success: false,
|
||||
},
|
||||
// Server addr connects but fails for CONNECT call.
|
||||
{
|
||||
serverAddr: "example.com:80",
|
||||
serverEndpoint: "/",
|
||||
success: false,
|
||||
},
|
||||
// Successful connecting to insecure RPC server.
|
||||
{
|
||||
serverAddr: testServer.Server.Listener.Addr().String(),
|
||||
serverEndpoint: adminEndpoint,
|
||||
success: true,
|
||||
},
|
||||
// Successful connecting to secure RPC server.
|
||||
{
|
||||
serverAddr: testServerTLS.Server.Listener.Addr().String(),
|
||||
serverEndpoint: adminEndpoint,
|
||||
success: true,
|
||||
secure: true,
|
||||
},
|
||||
}
|
||||
for i, testCase := range testCases {
|
||||
_, err = rpcDial(testCase.serverAddr, testCase.serverEndpoint, testCase.secure)
|
||||
if err != nil && testCase.success {
|
||||
t.Errorf("Test %d: Expected success but found failure instead %s", i+1, err)
|
||||
}
|
||||
if err == nil && !testCase.success {
|
||||
t.Errorf("Test %d: Expected failure but found success instead", i+1)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,38 +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
|
||||
|
||||
// Base login method name. It should be used along with service name.
|
||||
const loginMethodName = ".Login"
|
||||
|
||||
// AuthRPCServer RPC server authenticates using JWT.
|
||||
type AuthRPCServer struct{}
|
||||
|
||||
// Login - Handles JWT based RPC login.
|
||||
func (b AuthRPCServer) Login(args *LoginRPCArgs, reply *LoginRPCReply) error {
|
||||
// Validate LoginRPCArgs
|
||||
if err := args.IsValid(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Return an error if token is not valid.
|
||||
if !isAuthTokenValid(args.AuthToken) {
|
||||
return errAuthentication
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,88 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
func TestLogin(t *testing.T) {
|
||||
rootPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to create test config - %v", err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
creds := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
ls := AuthRPCServer{}
|
||||
testCases := []struct {
|
||||
args LoginRPCArgs
|
||||
skewTime time.Duration
|
||||
expectedErr error
|
||||
}{
|
||||
// Valid case.
|
||||
{
|
||||
args: LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
},
|
||||
skewTime: 0,
|
||||
expectedErr: nil,
|
||||
},
|
||||
// Valid username, password and request time, not version.
|
||||
{
|
||||
args: LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: semVersion{1, 0, 0},
|
||||
},
|
||||
skewTime: 0,
|
||||
expectedErr: errRPCAPIVersionUnsupported,
|
||||
},
|
||||
// Valid username, password and version, not request time
|
||||
{
|
||||
args: LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
},
|
||||
skewTime: 20 * time.Minute,
|
||||
expectedErr: errServerTimeMismatch,
|
||||
},
|
||||
// Invalid token, fails with authentication error
|
||||
{
|
||||
args: LoginRPCArgs{
|
||||
AuthToken: "",
|
||||
Version: globalRPCAPIVersion,
|
||||
},
|
||||
skewTime: 0,
|
||||
expectedErr: errAuthentication,
|
||||
},
|
||||
}
|
||||
for i, test := range testCases {
|
||||
reply := LoginRPCReply{}
|
||||
test.args.RequestTime = UTCNow().Add(test.skewTime)
|
||||
err := ls.Login(&test.args, &reply)
|
||||
if err != test.expectedErr {
|
||||
t.Errorf("Test %d: Expected error %v but received %v",
|
||||
i+1, test.expectedErr, err)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,142 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2016, 2017, 2018 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 (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/auth"
|
||||
)
|
||||
|
||||
// SetAuthPeerArgs - Arguments collection for SetAuth RPC call
|
||||
type SetAuthPeerArgs struct {
|
||||
// For Auth
|
||||
AuthRPCArgs
|
||||
|
||||
// New credentials that receiving peer should update to.
|
||||
Creds auth.Credentials
|
||||
}
|
||||
|
||||
// SetAuthPeer - Update to new credentials sent from a peer Minio
|
||||
// server. Since credentials are already validated on the sending
|
||||
// peer, here we just persist to file and update in-memory config. All
|
||||
// subsequently running isAuthTokenValid() calls will fail, and clients
|
||||
// will be forced to re-establish connections. Connections will be
|
||||
// re-established only when the sending client has also updated its
|
||||
// credentials.
|
||||
func (br *browserPeerAPIHandlers) SetAuthPeer(args SetAuthPeerArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !args.Creds.IsValid() {
|
||||
return fmt.Errorf("Invalid credential passed")
|
||||
}
|
||||
|
||||
// Acquire lock before updating global configuration.
|
||||
globalServerConfigMu.Lock()
|
||||
defer globalServerConfigMu.Unlock()
|
||||
|
||||
// Update credentials in memory
|
||||
prevCred := globalServerConfig.SetCredential(args.Creds)
|
||||
|
||||
// Save credentials to config file
|
||||
if err := globalServerConfig.Save(); err != nil {
|
||||
// Save the current creds when failed to update.
|
||||
globalServerConfig.SetCredential(prevCred)
|
||||
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Sends SetAuthPeer RPCs to all peers in the Minio cluster
|
||||
func updateCredsOnPeers(creds auth.Credentials) map[string]error {
|
||||
peers := GetRemotePeers(globalEndpoints)
|
||||
|
||||
// Array of errors for each peer
|
||||
errs := make([]error, len(peers))
|
||||
var wg sync.WaitGroup
|
||||
|
||||
serverCred := globalServerConfig.GetCredential()
|
||||
// Launch go routines to send request to each peer in parallel.
|
||||
for ix := range peers {
|
||||
wg.Add(1)
|
||||
go func(ix int) {
|
||||
defer wg.Done()
|
||||
|
||||
// Exclude self to avoid race with
|
||||
// invalidating the RPC token.
|
||||
if peers[ix] == globalMinioAddr {
|
||||
errs[ix] = nil
|
||||
return
|
||||
}
|
||||
|
||||
// Initialize client
|
||||
client := newAuthRPCClient(authConfig{
|
||||
accessKey: serverCred.AccessKey,
|
||||
secretKey: serverCred.SecretKey,
|
||||
serverAddr: peers[ix],
|
||||
secureConn: globalIsSSL,
|
||||
serviceEndpoint: path.Join(minioReservedBucketPath, browserPeerPath),
|
||||
serviceName: "BrowserPeer",
|
||||
})
|
||||
|
||||
// Construct RPC call arguments.
|
||||
args := SetAuthPeerArgs{Creds: creds}
|
||||
|
||||
// Make RPC call - we only care about error
|
||||
// response and not the reply.
|
||||
err := client.Call("BrowserPeer.SetAuthPeer", &args, &AuthRPCReply{})
|
||||
|
||||
// We try a bit hard (3 attempts with 1 second delay)
|
||||
// to set creds on peers in case of failure.
|
||||
if err != nil {
|
||||
for i := 0; i < 2; i++ {
|
||||
time.Sleep(1 * time.Second) // 1 second delay.
|
||||
err = client.Call("BrowserPeer.SetAuthPeer", &args, &AuthRPCReply{})
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Send result down the channel
|
||||
errs[ix] = err
|
||||
}(ix)
|
||||
}
|
||||
|
||||
// Wait for requests to complete.
|
||||
wg.Wait()
|
||||
|
||||
// Put errors into map.
|
||||
errsMap := make(map[string]error)
|
||||
for i, err := range errs {
|
||||
if err != nil {
|
||||
errsMap[peers[i]] = err
|
||||
}
|
||||
}
|
||||
|
||||
return errsMap
|
||||
}
|
|
@ -1,116 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/minio/minio/pkg/auth"
|
||||
)
|
||||
|
||||
// API suite container common to both FS and XL.
|
||||
type TestRPCBrowserPeerSuite struct {
|
||||
serverType string
|
||||
testServer TestServer
|
||||
testAuthConf authConfig
|
||||
}
|
||||
|
||||
// Setting up the test suite and starting the Test server.
|
||||
func (s *TestRPCBrowserPeerSuite) SetUpSuite(t *testing.T) {
|
||||
s.testServer = StartTestBrowserPeerRPCServer(t, s.serverType)
|
||||
s.testAuthConf = authConfig{
|
||||
serverAddr: s.testServer.Server.Listener.Addr().String(),
|
||||
accessKey: s.testServer.AccessKey,
|
||||
secretKey: s.testServer.SecretKey,
|
||||
serviceEndpoint: path.Join(minioReservedBucketPath, browserPeerPath),
|
||||
serviceName: "BrowserPeer",
|
||||
}
|
||||
}
|
||||
|
||||
// TeatDownSuite - called implicitly by after all tests are run in
|
||||
// browser peer rpc suite.
|
||||
func (s *TestRPCBrowserPeerSuite) TearDownSuite(t *testing.T) {
|
||||
s.testServer.Stop()
|
||||
}
|
||||
|
||||
func TestBrowserPeerRPC(t *testing.T) {
|
||||
// setup code
|
||||
s := &TestRPCBrowserPeerSuite{serverType: "XL"}
|
||||
s.SetUpSuite(t)
|
||||
|
||||
// run test
|
||||
s.testBrowserPeerRPC(t)
|
||||
|
||||
// teardown code
|
||||
s.TearDownSuite(t)
|
||||
}
|
||||
|
||||
// Tests for browser peer rpc.
|
||||
func (s *TestRPCBrowserPeerSuite) testBrowserPeerRPC(t *testing.T) {
|
||||
// Construct RPC call arguments.
|
||||
creds, err := auth.CreateCredentials("abcd1", "abcd1234")
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create credential. %v", err)
|
||||
}
|
||||
|
||||
// Validate for invalid token.
|
||||
args := SetAuthPeerArgs{Creds: creds}
|
||||
rclient := newAuthRPCClient(s.testAuthConf)
|
||||
defer rclient.Close()
|
||||
if err = rclient.Login(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rclient.authToken = "garbage"
|
||||
if err = rclient.Call("BrowserPeer.SetAuthPeer", &args, &AuthRPCReply{}); err != nil {
|
||||
if err.Error() != errInvalidToken.Error() {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
// Validate for successful Peer update.
|
||||
args = SetAuthPeerArgs{Creds: creds}
|
||||
client := newAuthRPCClient(s.testAuthConf)
|
||||
defer client.Close()
|
||||
err = client.Call("BrowserPeer.SetAuthPeer", &args, &AuthRPCReply{})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Validate for failure in login handler with previous credentials.
|
||||
rclient = newAuthRPCClient(s.testAuthConf)
|
||||
defer rclient.Close()
|
||||
token, err := authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rclient.authToken = token
|
||||
if err = rclient.Login(); err != nil {
|
||||
if err.Error() != errInvalidAccessKeyID.Error() {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
token, err = authenticateNode(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
rclient.authToken = token
|
||||
if err = rclient.Login(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
|
@ -1,53 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2014-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 (
|
||||
"context"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
// Set up an RPC endpoint that receives browser related calls. The
|
||||
// original motivation is for propagating credentials change
|
||||
// throughout Minio cluster, initiated from a Minio browser session.
|
||||
|
||||
const (
|
||||
browserPeerPath = "/browser/setauth"
|
||||
)
|
||||
|
||||
// The Type exporting methods exposed for RPC calls.
|
||||
type browserPeerAPIHandlers struct {
|
||||
AuthRPCServer
|
||||
}
|
||||
|
||||
// Register RPC router
|
||||
func registerBrowserPeerRPCRouter(router *mux.Router) error {
|
||||
bpHandlers := &browserPeerAPIHandlers{AuthRPCServer{}}
|
||||
|
||||
bpRPCServer := newRPCServer()
|
||||
err := bpRPCServer.RegisterName("BrowserPeer", bpHandlers)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
bpRouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
bpRouter.Path(browserPeerPath).Handler(bpRPCServer)
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,165 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"context"
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio/pkg/hash"
|
||||
"github.com/minio/minio/pkg/madmin"
|
||||
"github.com/minio/minio/pkg/policy"
|
||||
)
|
||||
|
||||
type DummyObjectLayer struct{}
|
||||
|
||||
func (api *DummyObjectLayer) Shutdown(context.Context) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) StorageInfo(context.Context) (si StorageInfo) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) MakeBucketWithLocation(ctx context.Context, bucket string, location string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) GetBucketInfo(ctx context.Context, bucket string) (bucketInfo BucketInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListBuckets(ctx context.Context) (buckets []BucketInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) DeleteBucket(ctx context.Context, bucket string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListObjects(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (result ListObjectsInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListObjectsV2(ctx context.Context, bucket, prefix, continuationToken, delimiter string, maxKeys int, fetchOwner bool, startAfter string) (result ListObjectsV2Info, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) GetObject(ctx context.Context, bucket, object string, startOffset int64, length int64, writer io.Writer, etag string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) GetObjectInfo(ctx context.Context, bucket, object string) (objInfo ObjectInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) PutObject(ctx context.Context, bucket, object string, data *hash.Reader, metadata map[string]string) (objInfo ObjectInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) CopyObject(ctx context.Context, srcBucket, srcObject, destBucket, destObject string, srcInfo ObjectInfo) (objInfo ObjectInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) DeleteObject(ctx context.Context, bucket, object string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListMultipartUploads(ctx context.Context, bucket, prefix, keyMarker, uploadIDMarker, delimiter string, maxUploads int) (result ListMultipartsInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) NewMultipartUpload(ctx context.Context, bucket, object string, metadata map[string]string) (uploadID string, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) CopyObjectPart(ctx context.Context, srcBucket, srcObject, destBucket, destObject string, uploadID string, partID int, startOffset int64, length int64, srcInfo ObjectInfo) (info PartInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) PutObjectPart(ctx context.Context, bucket, object, uploadID string, partID int, data *hash.Reader) (info PartInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListObjectParts(ctx context.Context, bucket, object, uploadID string, partNumberMarker int, maxParts int) (result ListPartsInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) AbortMultipartUpload(ctx context.Context, bucket, object, uploadID string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) CompleteMultipartUpload(ctx context.Context, bucket, object, uploadID string, uploadedParts []CompletePart) (objInfo ObjectInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ReloadFormat(ctx context.Context, dryRun bool) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) HealFormat(ctx context.Context, dryRun bool) (item madmin.HealResultItem, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) HealBucket(ctx context.Context, bucket string, dryRun bool) (items []madmin.HealResultItem, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) HealObject(ctx context.Context, bucket, object string, dryRun bool) (item madmin.HealResultItem, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListBucketsHeal(ctx context.Context) (buckets []BucketInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListObjectsHeal(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (info ListObjectsInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ListLocks(ctx context.Context, bucket, prefix string, duration time.Duration) (info []VolumeLockInfo, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) ClearLocks(context.Context, []VolumeLockInfo) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) SetBucketPolicy(context.Context, string, *policy.Policy) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) GetBucketPolicy(context.Context, string) (bucketPolicy *policy.Policy, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) RefreshBucketPolicy(context.Context, string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) DeleteBucketPolicy(context.Context, string) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) IsNotificationSupported() (b bool) {
|
||||
return
|
||||
}
|
||||
|
||||
func (api *DummyObjectLayer) IsEncryptionSupported() (b bool) {
|
||||
return
|
||||
}
|
|
@ -198,8 +198,6 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
|||
}
|
||||
|
||||
globalHTTPServer = xhttp.NewServer([]string{gatewayAddr}, registerHandlers(router, globalHandlers...), getCert)
|
||||
globalHTTPServer.ReadTimeout = globalConnReadTimeout
|
||||
globalHTTPServer.WriteTimeout = globalConnWriteTimeout
|
||||
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
||||
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
||||
go func() {
|
||||
|
|
|
@ -222,7 +222,7 @@ func guessIsRPCReq(req *http.Request) bool {
|
|||
if req == nil {
|
||||
return false
|
||||
}
|
||||
return req.Method == http.MethodConnect && req.Proto == "HTTP/1.0"
|
||||
return req.Method == http.MethodPost
|
||||
}
|
||||
|
||||
func (h redirectHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
|
|
|
@ -76,7 +76,7 @@ func TestGuessIsRPC(t *testing.T) {
|
|||
}
|
||||
r := &http.Request{
|
||||
Proto: "HTTP/1.0",
|
||||
Method: http.MethodConnect,
|
||||
Method: http.MethodPost,
|
||||
}
|
||||
if !guessIsRPCReq(r) {
|
||||
t.Fatal("Test shouldn't fail for a possible net/rpc request.")
|
||||
|
|
|
@ -68,10 +68,6 @@ const (
|
|||
// date and server date during signature verification.
|
||||
globalMaxSkewTime = 15 * time.Minute // 15 minutes skew allowed.
|
||||
|
||||
// Default Read/Write timeouts for each connection.
|
||||
globalConnReadTimeout = 15 * time.Minute // Timeout after 15 minutes of no data sent by the client.
|
||||
globalConnWriteTimeout = 15 * time.Minute // Timeout after 15 minutes if no data received by the client.
|
||||
|
||||
// Expiry duration after which the multipart uploads are deemed stale.
|
||||
globalMultipartExpiry = time.Hour * 24 * 14 // 2 weeks.
|
||||
// Cleanup interval when the stale multipart cleanup is initiated.
|
||||
|
@ -186,6 +182,13 @@ var (
|
|||
globalCacheExcludes []string
|
||||
// Disk cache expiry
|
||||
globalCacheExpiry = 90
|
||||
|
||||
// RPC V1 - Initial version
|
||||
// RPC V2 - format.json XL version changed to 2
|
||||
// RPC V3 - format.json XL version changed to 3
|
||||
// Current RPC version
|
||||
globalRPCAPIVersion = RPCVersion{3, 0, 0}
|
||||
|
||||
// Add new variable global values here.
|
||||
|
||||
// Default usage check interval value.
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 http
|
||||
|
||||
import (
|
||||
"net"
|
||||
"time"
|
||||
)
|
||||
|
||||
// TimeoutConn - is wrapped net.Conn with read/write timeouts.
|
||||
type TimeoutConn struct {
|
||||
QuirkConn
|
||||
readTimeout time.Duration
|
||||
writeTimeout time.Duration
|
||||
}
|
||||
|
||||
func (c *TimeoutConn) setReadTimeout() {
|
||||
if c.readTimeout != 0 && c.canSetReadDeadline() {
|
||||
c.SetReadDeadline(time.Now().UTC().Add(c.readTimeout))
|
||||
}
|
||||
}
|
||||
|
||||
func (c *TimeoutConn) setWriteTimeout() {
|
||||
if c.writeTimeout != 0 {
|
||||
c.SetWriteDeadline(time.Now().UTC().Add(c.writeTimeout))
|
||||
}
|
||||
}
|
||||
|
||||
// Read - reads data from the connection with timeout.
|
||||
func (c *TimeoutConn) Read(b []byte) (n int, err error) {
|
||||
c.setReadTimeout()
|
||||
return c.Conn.Read(b)
|
||||
}
|
||||
|
||||
// Write - writes data to the connection with timeout.
|
||||
func (c *TimeoutConn) Write(b []byte) (n int, err error) {
|
||||
c.setWriteTimeout()
|
||||
return c.Conn.Write(b)
|
||||
}
|
||||
|
||||
// NewTimeoutConn - creates a new timeout connection.
|
||||
func NewTimeoutConn(c net.Conn, readTimeout, writeTimeout time.Duration) *TimeoutConn {
|
||||
return &TimeoutConn{
|
||||
QuirkConn: QuirkConn{Conn: c},
|
||||
readTimeout: readTimeout,
|
||||
writeTimeout: writeTimeout,
|
||||
}
|
||||
}
|
|
@ -136,3 +136,10 @@ func webRequestAuthenticate(req *http.Request) error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func newAuthToken() string {
|
||||
cred := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(cred.AccessKey, cred.SecretKey)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
return token
|
||||
}
|
||||
|
|
|
@ -0,0 +1,123 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
// localAdminClient - represents admin operation to be executed locally.
|
||||
type localAdminClient struct{}
|
||||
|
||||
// SignalService - sends a restart or stop signal to the local server
|
||||
func (lc localAdminClient) SignalService(s serviceSignal) error {
|
||||
switch s {
|
||||
case serviceRestart, serviceStop:
|
||||
globalServiceSignalCh <- s
|
||||
default:
|
||||
return errUnsupportedSignal
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ReInitFormat - re-initialize disk format.
|
||||
func (lc localAdminClient) ReInitFormat(dryRun bool) error {
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return errServerNotInitialized
|
||||
}
|
||||
return objectAPI.ReloadFormat(context.Background(), dryRun)
|
||||
}
|
||||
|
||||
// ListLocks - Fetches lock information from local lock instrumentation.
|
||||
func (lc localAdminClient) ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error) {
|
||||
objectAPI := newObjectLayerFn()
|
||||
if objectAPI == nil {
|
||||
return nil, errServerNotInitialized
|
||||
}
|
||||
|
||||
return objectAPI.ListLocks(context.Background(), bucket, prefix, duration)
|
||||
}
|
||||
|
||||
// ServerInfo - Returns the server info of this server.
|
||||
func (lc localAdminClient) ServerInfo() (sid ServerInfoData, e error) {
|
||||
if globalBootTime.IsZero() {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
|
||||
// Build storage info
|
||||
objLayer := newObjectLayerFn()
|
||||
if objLayer == nil {
|
||||
return sid, errServerNotInitialized
|
||||
}
|
||||
storage := objLayer.StorageInfo(context.Background())
|
||||
|
||||
return ServerInfoData{
|
||||
StorageInfo: storage,
|
||||
ConnStats: globalConnStats.toServerConnStats(),
|
||||
HTTPStats: globalHTTPStats.toServerHTTPStats(),
|
||||
Properties: ServerProperties{
|
||||
Uptime: UTCNow().Sub(globalBootTime),
|
||||
Version: Version,
|
||||
CommitID: CommitID,
|
||||
SQSARN: globalNotificationSys.GetARNList(),
|
||||
Region: globalServerConfig.GetRegion(),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// GetConfig - returns config.json of the local server.
|
||||
func (lc localAdminClient) GetConfig() ([]byte, error) {
|
||||
if globalServerConfig == nil {
|
||||
return nil, fmt.Errorf("config not present")
|
||||
}
|
||||
|
||||
return json.Marshal(globalServerConfig)
|
||||
}
|
||||
|
||||
// WriteTmpConfig - writes config file content to a temporary file on
|
||||
// the local server.
|
||||
func (lc localAdminClient) WriteTmpConfig(tmpFileName string, configBytes []byte) error {
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), tmpFileName)
|
||||
err := ioutil.WriteFile(tmpConfigFile, configBytes, 0666)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
|
||||
// CommitConfig - Move the new config in tmpFileName onto config.json
|
||||
// on a local node.
|
||||
func (lc localAdminClient) CommitConfig(tmpFileName string) error {
|
||||
configFile := getConfigFile()
|
||||
tmpConfigFile := filepath.Join(getConfigDir(), tmpFileName)
|
||||
|
||||
err := os.Rename(tmpConfigFile, configFile)
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("tmpConfigFile", tmpConfigFile)
|
||||
reqInfo.AppendTags("configFile", configFile)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2016 Minio, Inc.
|
||||
* Minio Cloud Storage, (C) 2018 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
|
@ -17,14 +17,40 @@
|
|||
package cmd
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
// TestListLocksInfo - Test for listLocksInfo.
|
||||
func TestLocalAdminClientSignalService(t *testing.T) {
|
||||
testAdminCmdRunnerSignalService(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientReInitFormat(t *testing.T) {
|
||||
testAdminCmdRunnerReInitFormat(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientListLocks(t *testing.T) {
|
||||
testAdminCmdRunnerListLocks(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientServerInfo(t *testing.T) {
|
||||
testAdminCmdRunnerServerInfo(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientGetConfig(t *testing.T) {
|
||||
testAdminCmdRunnerGetConfig(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientWriteTmpConfig(t *testing.T) {
|
||||
testAdminCmdRunnerWriteTmpConfig(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestLocalAdminClientCommitConfig(t *testing.T) {
|
||||
testAdminCmdRunnerCommitConfig(t, &localAdminClient{})
|
||||
}
|
||||
|
||||
func TestListLocksInfo(t *testing.T) {
|
||||
// reset global variables to start afresh.
|
||||
resetTestGlobals()
|
||||
|
@ -70,6 +96,8 @@ func TestListLocksInfo(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
client := &localAdminClient{}
|
||||
|
||||
testCases := []struct {
|
||||
bucket string
|
||||
prefix string
|
||||
|
@ -100,9 +128,9 @@ func TestListLocksInfo(t *testing.T) {
|
|||
}
|
||||
|
||||
for i, test := range testCases {
|
||||
actual, err := objAPI.ListLocks(context.Background(), test.bucket, test.prefix, test.duration)
|
||||
actual, err := client.ListLocks(test.bucket, test.prefix, test.duration)
|
||||
if err != nil {
|
||||
t.Errorf("Test %d - Expected success, got %s", i+1, err)
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
if len(actual) != test.numLocks {
|
||||
t.Errorf("Test %d - Expected %d locks but observed %d locks",
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/dsync"
|
||||
)
|
||||
|
||||
// lockRequesterInfo stores various info from the client for each lock that is requested.
|
||||
type lockRequesterInfo struct {
|
||||
writer bool // Bool whether write or read lock.
|
||||
node string // Network address of client claiming lock.
|
||||
serviceEndpoint string // RPC path of client claiming lock.
|
||||
uid string // UID to uniquely identify request of client.
|
||||
timestamp time.Time // Timestamp set at the time of initialization.
|
||||
timeLastCheck time.Time // Timestamp for last check of validity of lock.
|
||||
}
|
||||
|
||||
// isWriteLock returns whether the lock is a write or read lock.
|
||||
func isWriteLock(lri []lockRequesterInfo) bool {
|
||||
return len(lri) == 1 && lri[0].writer
|
||||
}
|
||||
|
||||
// localLocker implements Dsync.NetLocker
|
||||
type localLocker struct {
|
||||
mutex sync.Mutex
|
||||
serviceEndpoint string
|
||||
serverAddr string
|
||||
lockMap map[string][]lockRequesterInfo
|
||||
}
|
||||
|
||||
func (l *localLocker) ServerAddr() string {
|
||||
return l.serverAddr
|
||||
}
|
||||
|
||||
func (l *localLocker) ServiceEndpoint() string {
|
||||
return l.serviceEndpoint
|
||||
}
|
||||
|
||||
func (l *localLocker) Lock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
_, isLockTaken := l.lockMap[args.Resource]
|
||||
if !isLockTaken { // No locks held on the given name, so claim write lock
|
||||
l.lockMap[args.Resource] = []lockRequesterInfo{
|
||||
{
|
||||
writer: true,
|
||||
node: args.ServerAddr,
|
||||
serviceEndpoint: args.ServiceEndpoint,
|
||||
uid: args.UID,
|
||||
timestamp: UTCNow(),
|
||||
timeLastCheck: UTCNow(),
|
||||
},
|
||||
}
|
||||
}
|
||||
// return reply=true if lock was granted.
|
||||
return !isLockTaken, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
var lri []lockRequesterInfo
|
||||
if lri, reply = l.lockMap[args.Resource]; !reply {
|
||||
// No lock is held on the given name
|
||||
return reply, fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Resource)
|
||||
}
|
||||
if reply = isWriteLock(lri); !reply {
|
||||
// Unless it is a write lock
|
||||
return reply, fmt.Errorf("Unlock attempted on a read locked entity: %s (%d read locks active)", args.Resource, len(lri))
|
||||
}
|
||||
if !l.removeEntry(args.Resource, args.UID, &lri) {
|
||||
return false, fmt.Errorf("Unlock unable to find corresponding lock for uid: %s", args.UID)
|
||||
}
|
||||
return true, nil
|
||||
|
||||
}
|
||||
|
||||
func (l *localLocker) RLock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
lrInfo := lockRequesterInfo{
|
||||
writer: false,
|
||||
node: args.ServerAddr,
|
||||
serviceEndpoint: args.ServiceEndpoint,
|
||||
uid: args.UID,
|
||||
timestamp: UTCNow(),
|
||||
timeLastCheck: UTCNow(),
|
||||
}
|
||||
if lri, ok := l.lockMap[args.Resource]; ok {
|
||||
if reply = !isWriteLock(lri); reply {
|
||||
// Unless there is a write lock
|
||||
l.lockMap[args.Resource] = append(l.lockMap[args.Resource], lrInfo)
|
||||
}
|
||||
} else {
|
||||
// No locks held on the given name, so claim (first) read lock
|
||||
l.lockMap[args.Resource] = []lockRequesterInfo{lrInfo}
|
||||
reply = true
|
||||
}
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
var lri []lockRequesterInfo
|
||||
if lri, reply = l.lockMap[args.Resource]; !reply {
|
||||
// No lock is held on the given name
|
||||
return reply, fmt.Errorf("RUnlock attempted on an unlocked entity: %s", args.Resource)
|
||||
}
|
||||
if reply = !isWriteLock(lri); !reply {
|
||||
// A write-lock is held, cannot release a read lock
|
||||
return reply, fmt.Errorf("RUnlock attempted on a write locked entity: %s", args.Resource)
|
||||
}
|
||||
if !l.removeEntry(args.Resource, args.UID, &lri) {
|
||||
return false, fmt.Errorf("RUnlock unable to find corresponding read lock for uid: %s", args.UID)
|
||||
}
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
if len(args.UID) != 0 {
|
||||
return false, fmt.Errorf("ForceUnlock called with non-empty UID: %s", args.UID)
|
||||
}
|
||||
if _, ok := l.lockMap[args.Resource]; ok {
|
||||
// Only clear lock when it is taken
|
||||
// Remove the lock (irrespective of write or read lock)
|
||||
delete(l.lockMap, args.Resource)
|
||||
}
|
||||
return true, nil
|
||||
}
|
|
@ -16,56 +16,99 @@
|
|||
|
||||
package cmd
|
||||
|
||||
import "github.com/minio/dsync"
|
||||
import (
|
||||
"crypto/tls"
|
||||
|
||||
"github.com/minio/dsync"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
// LockRPCClient is authenticable lock RPC client compatible to dsync.NetLocker
|
||||
type LockRPCClient struct {
|
||||
*AuthRPCClient
|
||||
*RPCClient
|
||||
}
|
||||
|
||||
// newLockRPCClient returns new lock RPC client object.
|
||||
func newLockRPCClient(config authConfig) *LockRPCClient {
|
||||
return &LockRPCClient{newAuthRPCClient(config)}
|
||||
// ServerAddr - dsync.NetLocker interface compatible method.
|
||||
func (lockRPC *LockRPCClient) ServerAddr() string {
|
||||
url := lockRPC.ServiceURL()
|
||||
return url.Host
|
||||
}
|
||||
|
||||
// ServiceEndpoint - dsync.NetLocker interface compatible method.
|
||||
func (lockRPC *LockRPCClient) ServiceEndpoint() string {
|
||||
url := lockRPC.ServiceURL()
|
||||
return url.Path
|
||||
}
|
||||
|
||||
// RLock calls read lock RPC.
|
||||
func (lockRPCClient *LockRPCClient) RLock(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.RLock", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) RLock(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".RLock", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// Lock calls write lock RPC.
|
||||
func (lockRPCClient *LockRPCClient) Lock(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Lock", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) Lock(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".Lock", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// RUnlock calls read unlock RPC.
|
||||
func (lockRPCClient *LockRPCClient) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.RUnlock", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".RUnlock", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// Unlock calls write unlock RPC.
|
||||
func (lockRPCClient *LockRPCClient) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Unlock", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".Unlock", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// ForceUnlock calls force unlock RPC.
|
||||
func (lockRPCClient *LockRPCClient) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.ForceUnlock", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".ForceUnlock", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// Expired calls expired RPC.
|
||||
func (lockRPCClient *LockRPCClient) Expired(args dsync.LockArgs) (reply bool, err error) {
|
||||
lockArgs := newLockArgs(args)
|
||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Expired", &lockArgs, &reply)
|
||||
func (lockRPC *LockRPCClient) Expired(args dsync.LockArgs) (reply bool, err error) {
|
||||
err = lockRPC.Call(lockServiceName+".Expired", &LockArgs{LockArgs: args}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// NewLockRPCClient - returns new lock RPC client.
|
||||
func NewLockRPCClient(host *xnet.Host) (*LockRPCClient, error) {
|
||||
scheme := "http"
|
||||
if globalIsSSL {
|
||||
scheme = "https"
|
||||
}
|
||||
|
||||
serviceURL := &xnet.URL{
|
||||
Scheme: scheme,
|
||||
Host: host.String(),
|
||||
Path: lockServicePath,
|
||||
}
|
||||
|
||||
var tlsConfig *tls.Config
|
||||
if globalIsSSL {
|
||||
tlsConfig = &tls.Config{
|
||||
ServerName: host.Name,
|
||||
RootCAs: globalRootCAs,
|
||||
}
|
||||
}
|
||||
|
||||
rpcClient, err := NewRPCClient(
|
||||
RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: lockServiceName,
|
||||
ServiceURL: serviceURL,
|
||||
TLSConfig: tlsConfig,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &LockRPCClient{rpcClient}, nil
|
||||
}
|
||||
|
|
|
@ -17,25 +17,25 @@
|
|||
package cmd
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/minio/dsync"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
// Tests lock rpc client.
|
||||
func TestLockRPCClient(t *testing.T) {
|
||||
lkClient := newLockRPCClient(authConfig{
|
||||
accessKey: "abcd",
|
||||
secretKey: "abcd123",
|
||||
serverAddr: fmt.Sprintf("%X", UTCNow().UnixNano()),
|
||||
serviceEndpoint: pathJoin(lockServicePath, "/test/1"),
|
||||
secureConn: false,
|
||||
serviceName: lockServiceName,
|
||||
})
|
||||
host, err := xnet.ParseHost("localhost:9000")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
lkClient, err := NewLockRPCClient(host)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
// Attempt all calls.
|
||||
_, err := lkClient.RLock(dsync.LockArgs{})
|
||||
_, err = lkClient.RLock(dsync.LockArgs{})
|
||||
if err == nil {
|
||||
t.Fatal("Expected for Rlock to fail")
|
||||
}
|
||||
|
|
|
@ -24,6 +24,12 @@ import (
|
|||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
// nameLockRequesterInfoPair is a helper type for lock maintenance
|
||||
type nameLockRequesterInfoPair struct {
|
||||
name string
|
||||
lri lockRequesterInfo
|
||||
}
|
||||
|
||||
// Similar to removeEntry but only removes an entry only if the lock entry exists in map.
|
||||
func (l *localLocker) removeEntryIfExists(nlrip nameLockRequesterInfoPair) {
|
||||
// Check if entry is still in map (could have been removed altogether by 'concurrent' (R)Unlock of last entry)
|
||||
|
|
|
@ -18,258 +18,76 @@ package cmd
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"path"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/dsync"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
const (
|
||||
// Lock rpc server endpoint.
|
||||
lockServicePath = "/lock"
|
||||
lockServiceSubPath = "/lock"
|
||||
|
||||
// Lock rpc service name.
|
||||
lockServiceName = "Dsync"
|
||||
|
||||
// Lock maintenance interval.
|
||||
lockMaintenanceInterval = 1 * time.Minute // 1 minute.
|
||||
lockMaintenanceInterval = 1 * time.Minute
|
||||
|
||||
// Lock validity check interval.
|
||||
lockValidityCheckInterval = 2 * time.Minute // 2 minutes.
|
||||
lockValidityCheckInterval = 2 * time.Minute
|
||||
)
|
||||
|
||||
// lockRequesterInfo stores various info from the client for each lock that is requested.
|
||||
type lockRequesterInfo struct {
|
||||
writer bool // Bool whether write or read lock.
|
||||
node string // Network address of client claiming lock.
|
||||
serviceEndpoint string // RPC path of client claiming lock.
|
||||
uid string // UID to uniquely identify request of client.
|
||||
timestamp time.Time // Timestamp set at the time of initialization.
|
||||
timeLastCheck time.Time // Timestamp for last check of validity of lock.
|
||||
var lockServicePath = path.Join(minioReservedBucketPath, lockServiceSubPath)
|
||||
|
||||
// LockArgs represents arguments for any authenticated lock RPC call.
|
||||
type LockArgs struct {
|
||||
AuthArgs
|
||||
LockArgs dsync.LockArgs
|
||||
}
|
||||
|
||||
// isWriteLock returns whether the lock is a write or read lock.
|
||||
func isWriteLock(lri []lockRequesterInfo) bool {
|
||||
return len(lri) == 1 && lri[0].writer
|
||||
}
|
||||
|
||||
// lockServer is type for RPC handlers
|
||||
type lockServer struct {
|
||||
AuthRPCServer
|
||||
// lockRPCReceiver is type for RPC handlers
|
||||
type lockRPCReceiver struct {
|
||||
ll localLocker
|
||||
}
|
||||
|
||||
// Start lock maintenance from all lock servers.
|
||||
func startLockMaintenance(lkSrv *lockServer) {
|
||||
// Start loop for stale lock maintenance
|
||||
go func(lk *lockServer) {
|
||||
// Initialize a new ticker with a minute between each ticks.
|
||||
ticker := time.NewTicker(lockMaintenanceInterval)
|
||||
// Stop the timer upon service closure and cleanup the go-routine.
|
||||
defer ticker.Stop()
|
||||
|
||||
// Start with random sleep time, so as to avoid "synchronous checks" between servers
|
||||
time.Sleep(time.Duration(rand.Float64() * float64(lockMaintenanceInterval)))
|
||||
for {
|
||||
// Verifies every minute for locks held more than 2minutes.
|
||||
select {
|
||||
case <-globalServiceDoneCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
lk.lockMaintenance(lockValidityCheckInterval)
|
||||
}
|
||||
}
|
||||
}(lkSrv)
|
||||
}
|
||||
|
||||
// Register distributed NS lock handlers.
|
||||
func registerDistNSLockRouter(router *mux.Router, endpoints EndpointList) error {
|
||||
// Start lock maintenance from all lock servers.
|
||||
startLockMaintenance(globalLockServer)
|
||||
|
||||
// Register initialized lock servers to their respective rpc endpoints.
|
||||
return registerStorageLockers(router, globalLockServer)
|
||||
}
|
||||
|
||||
// registerStorageLockers - register locker rpc handlers for net/rpc library clients
|
||||
func registerStorageLockers(router *mux.Router, lkSrv *lockServer) error {
|
||||
lockRPCServer := newRPCServer()
|
||||
if err := lockRPCServer.RegisterName(lockServiceName, lkSrv); err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
lockRouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
lockRouter.Path(lockServicePath).Handler(lockRPCServer)
|
||||
return nil
|
||||
}
|
||||
|
||||
// localLocker implements Dsync.NetLocker
|
||||
type localLocker struct {
|
||||
mutex sync.Mutex
|
||||
serviceEndpoint string
|
||||
serverAddr string
|
||||
lockMap map[string][]lockRequesterInfo
|
||||
}
|
||||
|
||||
func (l *localLocker) ServerAddr() string {
|
||||
return l.serverAddr
|
||||
}
|
||||
|
||||
func (l *localLocker) ServiceEndpoint() string {
|
||||
return l.serviceEndpoint
|
||||
}
|
||||
|
||||
func (l *localLocker) Lock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
_, isLockTaken := l.lockMap[args.Resource]
|
||||
if !isLockTaken { // No locks held on the given name, so claim write lock
|
||||
l.lockMap[args.Resource] = []lockRequesterInfo{
|
||||
{
|
||||
writer: true,
|
||||
node: args.ServerAddr,
|
||||
serviceEndpoint: args.ServiceEndpoint,
|
||||
uid: args.UID,
|
||||
timestamp: UTCNow(),
|
||||
timeLastCheck: UTCNow(),
|
||||
},
|
||||
}
|
||||
}
|
||||
// return reply=true if lock was granted.
|
||||
return !isLockTaken, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
var lri []lockRequesterInfo
|
||||
if lri, reply = l.lockMap[args.Resource]; !reply {
|
||||
// No lock is held on the given name
|
||||
return reply, fmt.Errorf("Unlock attempted on an unlocked entity: %s", args.Resource)
|
||||
}
|
||||
if reply = isWriteLock(lri); !reply {
|
||||
// Unless it is a write lock
|
||||
return reply, fmt.Errorf("Unlock attempted on a read locked entity: %s (%d read locks active)", args.Resource, len(lri))
|
||||
}
|
||||
if !l.removeEntry(args.Resource, args.UID, &lri) {
|
||||
return false, fmt.Errorf("Unlock unable to find corresponding lock for uid: %s", args.UID)
|
||||
}
|
||||
return true, nil
|
||||
|
||||
}
|
||||
|
||||
func (l *localLocker) RLock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
lrInfo := lockRequesterInfo{
|
||||
writer: false,
|
||||
node: args.ServerAddr,
|
||||
serviceEndpoint: args.ServiceEndpoint,
|
||||
uid: args.UID,
|
||||
timestamp: UTCNow(),
|
||||
timeLastCheck: UTCNow(),
|
||||
}
|
||||
if lri, ok := l.lockMap[args.Resource]; ok {
|
||||
if reply = !isWriteLock(lri); reply {
|
||||
// Unless there is a write lock
|
||||
l.lockMap[args.Resource] = append(l.lockMap[args.Resource], lrInfo)
|
||||
}
|
||||
} else {
|
||||
// No locks held on the given name, so claim (first) read lock
|
||||
l.lockMap[args.Resource] = []lockRequesterInfo{lrInfo}
|
||||
reply = true
|
||||
}
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
var lri []lockRequesterInfo
|
||||
if lri, reply = l.lockMap[args.Resource]; !reply {
|
||||
// No lock is held on the given name
|
||||
return reply, fmt.Errorf("RUnlock attempted on an unlocked entity: %s", args.Resource)
|
||||
}
|
||||
if reply = !isWriteLock(lri); !reply {
|
||||
// A write-lock is held, cannot release a read lock
|
||||
return reply, fmt.Errorf("RUnlock attempted on a write locked entity: %s", args.Resource)
|
||||
}
|
||||
if !l.removeEntry(args.Resource, args.UID, &lri) {
|
||||
return false, fmt.Errorf("RUnlock unable to find corresponding read lock for uid: %s", args.UID)
|
||||
}
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (l *localLocker) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||
l.mutex.Lock()
|
||||
defer l.mutex.Unlock()
|
||||
if len(args.UID) != 0 {
|
||||
return false, fmt.Errorf("ForceUnlock called with non-empty UID: %s", args.UID)
|
||||
}
|
||||
if _, ok := l.lockMap[args.Resource]; ok {
|
||||
// Only clear lock when it is taken
|
||||
// Remove the lock (irrespective of write or read lock)
|
||||
delete(l.lockMap, args.Resource)
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
/// Distributed lock handlers
|
||||
|
||||
// Lock - rpc handler for (single) write lock operation.
|
||||
func (l *lockServer) Lock(args *LockArgs, reply *bool) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) Lock(args *LockArgs, reply *bool) (err error) {
|
||||
*reply, err = l.ll.Lock(args.LockArgs)
|
||||
return err
|
||||
}
|
||||
|
||||
// Unlock - rpc handler for (single) write unlock operation.
|
||||
func (l *lockServer) Unlock(args *LockArgs, reply *bool) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) Unlock(args *LockArgs, reply *bool) (err error) {
|
||||
*reply, err = l.ll.Unlock(args.LockArgs)
|
||||
return err
|
||||
}
|
||||
|
||||
// RLock - rpc handler for read lock operation.
|
||||
func (l *lockServer) RLock(args *LockArgs, reply *bool) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) RLock(args *LockArgs, reply *bool) (err error) {
|
||||
*reply, err = l.ll.RLock(args.LockArgs)
|
||||
return err
|
||||
}
|
||||
|
||||
// RUnlock - rpc handler for read unlock operation.
|
||||
func (l *lockServer) RUnlock(args *LockArgs, reply *bool) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) RUnlock(args *LockArgs, reply *bool) (err error) {
|
||||
*reply, err = l.ll.RUnlock(args.LockArgs)
|
||||
return err
|
||||
}
|
||||
|
||||
// ForceUnlock - rpc handler for force unlock operation.
|
||||
func (l *lockServer) ForceUnlock(args *LockArgs, reply *bool) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) ForceUnlock(args *LockArgs, reply *bool) (err error) {
|
||||
*reply, err = l.ll.ForceUnlock(args.LockArgs)
|
||||
return err
|
||||
}
|
||||
|
||||
// Expired - rpc handler for expired lock status.
|
||||
func (l *lockServer) Expired(args *LockArgs, reply *bool) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
func (l *lockRPCReceiver) Expired(args *LockArgs, reply *bool) error {
|
||||
l.ll.mutex.Lock()
|
||||
defer l.ll.mutex.Unlock()
|
||||
// Lock found, proceed to verify if belongs to given uid.
|
||||
|
@ -288,12 +106,6 @@ func (l *lockServer) Expired(args *LockArgs, reply *bool) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// nameLockRequesterInfoPair is a helper type for lock maintenance
|
||||
type nameLockRequesterInfoPair struct {
|
||||
name string
|
||||
lri lockRequesterInfo
|
||||
}
|
||||
|
||||
// lockMaintenance loops over locks that have been active for some time and checks back
|
||||
// with the original server whether it is still alive or not
|
||||
//
|
||||
|
@ -302,24 +114,22 @@ type nameLockRequesterInfoPair struct {
|
|||
// - some network error (and server is up normally)
|
||||
//
|
||||
// We will ignore the error, and we will retry later to get a resolve on this lock
|
||||
func (l *lockServer) lockMaintenance(interval time.Duration) {
|
||||
func (l *lockRPCReceiver) lockMaintenance(interval time.Duration) {
|
||||
l.ll.mutex.Lock()
|
||||
// Get list of long lived locks to check for staleness.
|
||||
nlripLongLived := getLongLivedLocks(l.ll.lockMap, interval)
|
||||
l.ll.mutex.Unlock()
|
||||
|
||||
serverCred := globalServerConfig.GetCredential()
|
||||
// Validate if long lived locks are indeed clean.
|
||||
for _, nlrip := range nlripLongLived {
|
||||
// Initialize client based on the long live locks.
|
||||
c := newLockRPCClient(authConfig{
|
||||
accessKey: serverCred.AccessKey,
|
||||
secretKey: serverCred.SecretKey,
|
||||
serverAddr: nlrip.lri.node,
|
||||
secureConn: globalIsSSL,
|
||||
serviceEndpoint: nlrip.lri.serviceEndpoint,
|
||||
serviceName: lockServiceName,
|
||||
})
|
||||
host, err := xnet.ParseHost(nlrip.lri.node)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
c, err := NewLockRPCClient(host)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
continue
|
||||
}
|
||||
|
||||
// Call back to original server verify whether the lock is still active (based on name & uid)
|
||||
expired, _ := c.Expired(dsync.LockArgs{
|
||||
|
@ -328,7 +138,7 @@ func (l *lockServer) lockMaintenance(interval time.Duration) {
|
|||
})
|
||||
|
||||
// Close the connection regardless of the call response.
|
||||
c.AuthRPCClient.Close()
|
||||
c.Close()
|
||||
|
||||
// For successful response, verify if lock is indeed active or stale.
|
||||
if expired {
|
||||
|
@ -340,3 +150,44 @@ func (l *lockServer) lockMaintenance(interval time.Duration) {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Start lock maintenance from all lock servers.
|
||||
func startLockMaintenance(lkSrv *lockRPCReceiver) {
|
||||
// Initialize a new ticker with a minute between each ticks.
|
||||
ticker := time.NewTicker(lockMaintenanceInterval)
|
||||
// Stop the timer upon service closure and cleanup the go-routine.
|
||||
defer ticker.Stop()
|
||||
|
||||
// Start with random sleep time, so as to avoid "synchronous checks" between servers
|
||||
time.Sleep(time.Duration(rand.Float64() * float64(lockMaintenanceInterval)))
|
||||
for {
|
||||
// Verifies every minute for locks held more than 2minutes.
|
||||
select {
|
||||
case <-globalServiceDoneCh:
|
||||
return
|
||||
case <-ticker.C:
|
||||
lkSrv.lockMaintenance(lockValidityCheckInterval)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// NewLockRPCServer - returns new lock RPC server.
|
||||
func NewLockRPCServer() (*xrpc.Server, error) {
|
||||
rpcServer := xrpc.NewServer()
|
||||
if err := rpcServer.RegisterName(lockServiceName, globalLockServer); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return rpcServer, nil
|
||||
}
|
||||
|
||||
// Register distributed NS lock handlers.
|
||||
func registerDistNSLockRouter(router *mux.Router) {
|
||||
rpcServer, err := NewLockRPCServer()
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
|
||||
// Start lock maintenance from all lock servers.
|
||||
go startLockMaintenance(globalLockServer)
|
||||
|
||||
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
subrouter.Path(lockServiceSubPath).Handler(rpcServer)
|
||||
}
|
||||
|
|
|
@ -43,14 +43,13 @@ func testLockEquality(lriLeft, lriRight []lockRequesterInfo) bool {
|
|||
}
|
||||
|
||||
// Helper function to create a lock server for testing
|
||||
func createLockTestServer(t *testing.T) (string, *lockServer, string) {
|
||||
func createLockTestServer(t *testing.T) (string, *lockRPCReceiver, string) {
|
||||
testPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("unable initialize config file, %s", err)
|
||||
}
|
||||
|
||||
locker := &lockServer{
|
||||
AuthRPCServer: AuthRPCServer{},
|
||||
locker := &lockRPCReceiver{
|
||||
ll: localLocker{
|
||||
mutex: sync.Mutex{},
|
||||
serviceEndpoint: "rpc-path",
|
||||
|
@ -62,16 +61,6 @@ func createLockTestServer(t *testing.T) (string, *lockServer, string) {
|
|||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
loginArgs := LoginRPCArgs{
|
||||
AuthToken: token,
|
||||
Version: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
loginReply := LoginRPCReply{}
|
||||
err = locker.Login(&loginArgs, &loginReply)
|
||||
if err != nil {
|
||||
t.Fatalf("Failed to login to lock server - %v", err)
|
||||
}
|
||||
return testPath, locker, token
|
||||
}
|
||||
|
||||
|
@ -80,14 +69,18 @@ func TestLockRpcServerLock(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// Claim a lock
|
||||
var result bool
|
||||
|
@ -114,14 +107,18 @@ func TestLockRpcServerLock(t *testing.T) {
|
|||
}
|
||||
|
||||
// Try to claim same lock again (will fail)
|
||||
la2 := newLockArgs(dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la2.SetAuthToken(token)
|
||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la2 := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
err = locker.Lock(&la2, &result)
|
||||
if err != nil {
|
||||
|
@ -138,14 +135,18 @@ func TestLockRpcServerUnlock(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// First test return of error when attempting to unlock a lock that does not exist
|
||||
var result bool
|
||||
|
@ -184,14 +185,18 @@ func TestLockRpcServerRLock(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// Claim a lock
|
||||
var result bool
|
||||
|
@ -218,14 +223,18 @@ func TestLockRpcServerRLock(t *testing.T) {
|
|||
}
|
||||
|
||||
// Try to claim same again (will succeed)
|
||||
la2 := newLockArgs(dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la2.SetAuthToken(token)
|
||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la2 := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
err = locker.RLock(&la2, &result)
|
||||
if err != nil {
|
||||
|
@ -242,14 +251,18 @@ func TestLockRpcServerRUnlock(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// First test return of error when attempting to unlock a read-lock that does not exist
|
||||
var result bool
|
||||
|
@ -267,14 +280,18 @@ func TestLockRpcServerRUnlock(t *testing.T) {
|
|||
}
|
||||
|
||||
// Try to claim same again (will succeed)
|
||||
la2 := newLockArgs(dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la2.SetAuthToken(token)
|
||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la2 := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "89ab-cdef",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// ... and create a second lock on same resource
|
||||
err = locker.RLock(&la2, &result)
|
||||
|
@ -330,14 +347,18 @@ func TestLockRpcServerForceUnlock(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
laForce := newLockArgs(dsync.LockArgs{
|
||||
UID: "1234-5678",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
laForce.SetAuthToken(token)
|
||||
laForce.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
laForce := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "1234-5678",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// First test that UID should be empty
|
||||
var result bool
|
||||
|
@ -353,14 +374,18 @@ func TestLockRpcServerForceUnlock(t *testing.T) {
|
|||
t.Errorf("Expected no error, got %#v", err)
|
||||
}
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// Create lock ... (so that we can force unlock)
|
||||
err = locker.Lock(&la, &result)
|
||||
|
@ -396,14 +421,18 @@ func TestLockRpcServerExpired(t *testing.T) {
|
|||
testPath, locker, token := createLockTestServer(t)
|
||||
defer os.RemoveAll(testPath)
|
||||
|
||||
la := newLockArgs(dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
})
|
||||
la.SetAuthToken(token)
|
||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
||||
la := LockArgs{
|
||||
AuthArgs: AuthArgs{
|
||||
Token: token,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
},
|
||||
LockArgs: dsync.LockArgs{
|
||||
UID: "0123-4567",
|
||||
Resource: "name",
|
||||
ServerAddr: "node",
|
||||
ServiceEndpoint: "rpc-path",
|
||||
}}
|
||||
|
||||
// Unknown lock at server will return expired = true
|
||||
var expired bool
|
||||
|
@ -500,7 +529,7 @@ func TestLockServerInit(t *testing.T) {
|
|||
t.Fatalf("Got unexpected error initializing lock servers: %v", err)
|
||||
}
|
||||
if globalLockServer == nil && testCase.isDistXL {
|
||||
t.Errorf("Test %d: Expected initialized lockServer, but got uninitialized", i+1)
|
||||
t.Errorf("Test %d: Expected initialized lock RPC receiver, but got uninitialized", i+1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,13 +31,14 @@ import (
|
|||
"github.com/minio/lsync"
|
||||
"github.com/minio/minio-go/pkg/set"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
// Global name space lock.
|
||||
var globalNSMutex *nsLockMap
|
||||
|
||||
// Global lock server one per server.
|
||||
var globalLockServer *lockServer
|
||||
var globalLockServer *lockRPCReceiver
|
||||
|
||||
// Instance of dsync for distributed clients.
|
||||
var globalDsync *dsync.Dsync
|
||||
|
@ -61,7 +62,6 @@ type RWLockerSync interface {
|
|||
// Initialize distributed locking only in case of distributed setup.
|
||||
// Returns lock clients and the node index for the current server.
|
||||
func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int) {
|
||||
cred := globalServerConfig.GetCredential()
|
||||
myNode = -1
|
||||
seenHosts := set.NewStringSet()
|
||||
for _, endpoint := range endpoints {
|
||||
|
@ -69,35 +69,29 @@ func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int)
|
|||
continue
|
||||
}
|
||||
seenHosts.Add(endpoint.Host)
|
||||
if !endpoint.IsLocal {
|
||||
// For a remote endpoints setup a lock RPC client.
|
||||
clnts = append(clnts, newLockRPCClient(authConfig{
|
||||
accessKey: cred.AccessKey,
|
||||
secretKey: cred.SecretKey,
|
||||
serverAddr: endpoint.Host,
|
||||
secureConn: globalIsSSL,
|
||||
serviceEndpoint: pathutil.Join(minioReservedBucketPath, lockServicePath),
|
||||
serviceName: lockServiceName,
|
||||
}))
|
||||
continue
|
||||
|
||||
var locker dsync.NetLocker
|
||||
if endpoint.IsLocal {
|
||||
myNode = len(clnts)
|
||||
|
||||
receiver := &lockRPCReceiver{
|
||||
ll: localLocker{
|
||||
serverAddr: endpoint.Host,
|
||||
serviceEndpoint: lockServicePath,
|
||||
lockMap: make(map[string][]lockRequesterInfo),
|
||||
},
|
||||
}
|
||||
|
||||
globalLockServer = receiver
|
||||
locker = &(receiver.ll)
|
||||
} else {
|
||||
host, err := xnet.ParseHost(endpoint.Host)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
locker, err = NewLockRPCClient(host)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
}
|
||||
|
||||
// Local endpoint
|
||||
myNode = len(clnts)
|
||||
|
||||
// For a local endpoint, setup a local lock server to
|
||||
// avoid network requests.
|
||||
localLockServer := lockServer{
|
||||
AuthRPCServer: AuthRPCServer{},
|
||||
ll: localLocker{
|
||||
serverAddr: endpoint.Host,
|
||||
serviceEndpoint: pathutil.Join(minioReservedBucketPath, lockServicePath),
|
||||
lockMap: make(map[string][]lockRequesterInfo),
|
||||
},
|
||||
}
|
||||
|
||||
globalLockServer = &localLockServer
|
||||
clnts = append(clnts, &(localLockServer.ll))
|
||||
clnts = append(clnts, locker)
|
||||
}
|
||||
|
||||
return clnts, myNode
|
||||
|
|
|
@ -26,8 +26,10 @@ import (
|
|||
"net/url"
|
||||
"path"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/auth"
|
||||
"github.com/minio/minio/pkg/event"
|
||||
"github.com/minio/minio/pkg/hash"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
|
@ -90,6 +92,33 @@ func (sys *NotificationSys) DeleteBucket(bucketName string) <-chan NotificationP
|
|||
return errCh
|
||||
}
|
||||
|
||||
// SetCredentials - calls SetCredentials RPC call on all peers.
|
||||
func (sys *NotificationSys) SetCredentials(credentials auth.Credentials) map[xnet.Host]error {
|
||||
errors := make(map[xnet.Host]error)
|
||||
var wg sync.WaitGroup
|
||||
for addr, client := range sys.peerRPCClientMap {
|
||||
wg.Add(1)
|
||||
go func(addr xnet.Host, client *PeerRPCClient) {
|
||||
defer wg.Done()
|
||||
// Try to set credentials in three attempts.
|
||||
for i := 0; i < 3; i++ {
|
||||
err := client.SetCredentials(credentials)
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
errors[addr] = err
|
||||
// Wait for one second and no need wait after last attempt.
|
||||
if i < 2 {
|
||||
time.Sleep(1 * time.Second)
|
||||
}
|
||||
}
|
||||
}(addr, client)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
return errors
|
||||
}
|
||||
|
||||
// SetBucketPolicy - calls SetBucketPolicy RPC call on all peers.
|
||||
func (sys *NotificationSys) SetBucketPolicy(bucketName string, bucketPolicy *policy.Policy) <-chan NotificationPeerErr {
|
||||
errCh := make(chan NotificationPeerErr)
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 "github.com/minio/minio/pkg/event"
|
||||
|
||||
// PeerRPCClientTarget - RPCClient is an event.Target which sends event to target of remote peer.
|
||||
type PeerRPCClientTarget struct {
|
||||
id event.TargetID
|
||||
remoteTargetID event.TargetID
|
||||
rpcClient *PeerRPCClient
|
||||
bucketName string
|
||||
}
|
||||
|
||||
// ID - returns target ID.
|
||||
func (target *PeerRPCClientTarget) ID() event.TargetID {
|
||||
return target.id
|
||||
}
|
||||
|
||||
// Send - sends event to remote peer by making RPC call.
|
||||
func (target *PeerRPCClientTarget) Send(eventData event.Event) error {
|
||||
return target.rpcClient.SendEvent(target.bucketName, target.id, target.remoteTargetID, eventData)
|
||||
}
|
||||
|
||||
// Close - does nothing and available for interface compatibility.
|
||||
func (target *PeerRPCClientTarget) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewPeerRPCClientTarget - creates RPCClient target with given target ID available in remote peer.
|
||||
func NewPeerRPCClientTarget(bucketName string, targetID event.TargetID, rpcClient *PeerRPCClient) *PeerRPCClientTarget {
|
||||
return &PeerRPCClientTarget{
|
||||
id: event.TargetID{targetID.ID, targetID.Name + "+" + mustGetUUID()},
|
||||
remoteTargetID: targetID,
|
||||
bucketName: bucketName,
|
||||
rpcClient: rpcClient,
|
||||
}
|
||||
}
|
|
@ -0,0 +1,176 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/auth"
|
||||
"github.com/minio/minio/pkg/event"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
"github.com/minio/minio/pkg/policy"
|
||||
)
|
||||
|
||||
// PeerRPCClient - peer RPC client talks to peer RPC server.
|
||||
type PeerRPCClient struct {
|
||||
*RPCClient
|
||||
}
|
||||
|
||||
// DeleteBucket - calls delete bucket RPC.
|
||||
func (rpcClient *PeerRPCClient) DeleteBucket(bucketName string) error {
|
||||
args := DeleteBucketArgs{BucketName: bucketName}
|
||||
reply := VoidReply{}
|
||||
return rpcClient.Call(peerServiceName+".DeleteBucket", &args, &reply)
|
||||
}
|
||||
|
||||
// SetBucketPolicy - calls set bucket policy RPC.
|
||||
func (rpcClient *PeerRPCClient) SetBucketPolicy(bucketName string, bucketPolicy *policy.Policy) error {
|
||||
args := SetBucketPolicyArgs{
|
||||
BucketName: bucketName,
|
||||
Policy: *bucketPolicy,
|
||||
}
|
||||
reply := VoidReply{}
|
||||
return rpcClient.Call(peerServiceName+".SetBucketPolicy", &args, &reply)
|
||||
}
|
||||
|
||||
// RemoveBucketPolicy - calls remove bucket policy RPC.
|
||||
func (rpcClient *PeerRPCClient) RemoveBucketPolicy(bucketName string) error {
|
||||
args := RemoveBucketPolicyArgs{
|
||||
BucketName: bucketName,
|
||||
}
|
||||
reply := VoidReply{}
|
||||
return rpcClient.Call(peerServiceName+".RemoveBucketPolicy", &args, &reply)
|
||||
}
|
||||
|
||||
// PutBucketNotification - calls put bukcet notification RPC.
|
||||
func (rpcClient *PeerRPCClient) PutBucketNotification(bucketName string, rulesMap event.RulesMap) error {
|
||||
args := PutBucketNotificationArgs{
|
||||
BucketName: bucketName,
|
||||
RulesMap: rulesMap,
|
||||
}
|
||||
reply := VoidReply{}
|
||||
return rpcClient.Call(peerServiceName+".PutBucketNotification", &args, &reply)
|
||||
}
|
||||
|
||||
// ListenBucketNotification - calls listen bucket notification RPC.
|
||||
func (rpcClient *PeerRPCClient) ListenBucketNotification(bucketName string, eventNames []event.Name,
|
||||
pattern string, targetID event.TargetID, addr xnet.Host) error {
|
||||
args := ListenBucketNotificationArgs{
|
||||
BucketName: bucketName,
|
||||
EventNames: eventNames,
|
||||
Pattern: pattern,
|
||||
TargetID: targetID,
|
||||
Addr: addr,
|
||||
}
|
||||
reply := VoidReply{}
|
||||
return rpcClient.Call(peerServiceName+".ListenBucketNotification", &args, &reply)
|
||||
}
|
||||
|
||||
// RemoteTargetExist - calls remote target ID exist RPC.
|
||||
func (rpcClient *PeerRPCClient) RemoteTargetExist(bucketName string, targetID event.TargetID) (bool, error) {
|
||||
args := RemoteTargetExistArgs{
|
||||
BucketName: bucketName,
|
||||
TargetID: targetID,
|
||||
}
|
||||
var reply bool
|
||||
|
||||
err := rpcClient.Call(peerServiceName+".RemoteTargetExist", &args, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// SendEvent - calls send event RPC.
|
||||
func (rpcClient *PeerRPCClient) SendEvent(bucketName string, targetID, remoteTargetID event.TargetID, eventData event.Event) error {
|
||||
args := SendEventArgs{
|
||||
BucketName: bucketName,
|
||||
TargetID: remoteTargetID,
|
||||
Event: eventData,
|
||||
}
|
||||
var reply bool
|
||||
|
||||
err := rpcClient.Call(peerServiceName+".SendEvent", &args, &reply)
|
||||
if err != nil && !reply {
|
||||
reqInfo := &logger.ReqInfo{BucketName: bucketName}
|
||||
reqInfo.AppendTags("targetID", targetID.Name)
|
||||
reqInfo.AppendTags("event", eventData.EventName.String())
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
globalNotificationSys.RemoveRemoteTarget(bucketName, targetID)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// SetCredentials - calls set credentials RPC.
|
||||
func (rpcClient *PeerRPCClient) SetCredentials(credentials auth.Credentials) error {
|
||||
args := SetCredentialsArgs{Credentials: credentials}
|
||||
reply := VoidReply{}
|
||||
|
||||
return rpcClient.Call(peerServiceName+".SetCredentials", &args, &reply)
|
||||
}
|
||||
|
||||
// NewPeerRPCClient - returns new peer RPC client.
|
||||
func NewPeerRPCClient(host *xnet.Host) (*PeerRPCClient, error) {
|
||||
scheme := "http"
|
||||
if globalIsSSL {
|
||||
scheme = "https"
|
||||
}
|
||||
|
||||
serviceURL := &xnet.URL{
|
||||
Scheme: scheme,
|
||||
Host: host.String(),
|
||||
Path: peerServicePath,
|
||||
}
|
||||
|
||||
var tlsConfig *tls.Config
|
||||
if globalIsSSL {
|
||||
tlsConfig = &tls.Config{
|
||||
ServerName: host.Name,
|
||||
RootCAs: globalRootCAs,
|
||||
}
|
||||
}
|
||||
|
||||
rpcClient, err := NewRPCClient(
|
||||
RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: peerServiceName,
|
||||
ServiceURL: serviceURL,
|
||||
TLSConfig: tlsConfig,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &PeerRPCClient{rpcClient}, nil
|
||||
}
|
||||
|
||||
// makeRemoteRPCClients - creates Peer RPCClients for given endpoint list.
|
||||
func makeRemoteRPCClients(endpoints EndpointList) map[xnet.Host]*PeerRPCClient {
|
||||
peerRPCClientMap := make(map[xnet.Host]*PeerRPCClient)
|
||||
for _, hostStr := range GetRemotePeers(endpoints) {
|
||||
host, err := xnet.ParseHost(hostStr)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
rpcClient, err := NewPeerRPCClient(host)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
peerRPCClientMap[*host] = rpcClient
|
||||
}
|
||||
|
||||
return peerRPCClientMap
|
||||
}
|
|
@ -0,0 +1,207 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
"github.com/minio/minio/pkg/auth"
|
||||
"github.com/minio/minio/pkg/event"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
"github.com/minio/minio/pkg/policy"
|
||||
)
|
||||
|
||||
const peerServiceName = "Peer"
|
||||
const peerServiceSubPath = "/s3/remote"
|
||||
|
||||
var peerServicePath = path.Join(minioReservedBucketPath, peerServiceSubPath)
|
||||
|
||||
// peerRPCReceiver - Peer RPC receiver for peer RPC server.
|
||||
type peerRPCReceiver struct{}
|
||||
|
||||
// DeleteBucketArgs - delete bucket RPC arguments.
|
||||
type DeleteBucketArgs struct {
|
||||
AuthArgs
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// DeleteBucket - handles delete bucket RPC call which removes all values of given bucket in global NotificationSys object.
|
||||
func (receiver *peerRPCReceiver) DeleteBucket(args *DeleteBucketArgs, reply *VoidReply) error {
|
||||
globalNotificationSys.RemoveNotification(args.BucketName)
|
||||
globalPolicySys.Remove(args.BucketName)
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetBucketPolicyArgs - set bucket policy RPC arguments.
|
||||
type SetBucketPolicyArgs struct {
|
||||
AuthArgs
|
||||
BucketName string
|
||||
Policy policy.Policy
|
||||
}
|
||||
|
||||
// SetBucketPolicy - handles set bucket policy RPC call which adds bucket policy to globalPolicySys.
|
||||
func (receiver *peerRPCReceiver) SetBucketPolicy(args *SetBucketPolicyArgs, reply *VoidReply) error {
|
||||
globalPolicySys.Set(args.BucketName, args.Policy)
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoveBucketPolicyArgs - delete bucket policy RPC arguments.
|
||||
type RemoveBucketPolicyArgs struct {
|
||||
AuthArgs
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// RemoveBucketPolicy - handles delete bucket policy RPC call which removes bucket policy to globalPolicySys.
|
||||
func (receiver *peerRPCReceiver) RemoveBucketPolicy(args *RemoveBucketPolicyArgs, reply *VoidReply) error {
|
||||
globalPolicySys.Remove(args.BucketName)
|
||||
return nil
|
||||
}
|
||||
|
||||
// PutBucketNotificationArgs - put bucket notification RPC arguments.
|
||||
type PutBucketNotificationArgs struct {
|
||||
AuthArgs
|
||||
BucketName string
|
||||
RulesMap event.RulesMap
|
||||
}
|
||||
|
||||
// PutBucketNotification - handles put bucket notification RPC call which adds rules to given bucket to global NotificationSys object.
|
||||
func (receiver *peerRPCReceiver) PutBucketNotification(args *PutBucketNotificationArgs, reply *VoidReply) error {
|
||||
globalNotificationSys.AddRulesMap(args.BucketName, args.RulesMap)
|
||||
return nil
|
||||
}
|
||||
|
||||
// ListenBucketNotificationArgs - listen bucket notification RPC arguments.
|
||||
type ListenBucketNotificationArgs struct {
|
||||
AuthArgs `json:"-"`
|
||||
BucketName string `json:"-"`
|
||||
EventNames []event.Name `json:"eventNames"`
|
||||
Pattern string `json:"pattern"`
|
||||
TargetID event.TargetID `json:"targetId"`
|
||||
Addr xnet.Host `json:"addr"`
|
||||
}
|
||||
|
||||
// ListenBucketNotification - handles listen bucket notification RPC call. It creates PeerRPCClient target which pushes requested events to target in remote peer.
|
||||
func (receiver *peerRPCReceiver) ListenBucketNotification(args *ListenBucketNotificationArgs, reply *VoidReply) error {
|
||||
rpcClient := globalNotificationSys.GetPeerRPCClient(args.Addr)
|
||||
if rpcClient == nil {
|
||||
return fmt.Errorf("unable to find PeerRPCClient for provided address %v. This happens only if remote and this minio run with different set of endpoints", args.Addr)
|
||||
}
|
||||
|
||||
target := NewPeerRPCClientTarget(args.BucketName, args.TargetID, rpcClient)
|
||||
rulesMap := event.NewRulesMap(args.EventNames, args.Pattern, target.ID())
|
||||
if err := globalNotificationSys.AddRemoteTarget(args.BucketName, target, rulesMap); err != nil {
|
||||
reqInfo := &logger.ReqInfo{BucketName: target.bucketName}
|
||||
reqInfo.AppendTags("target", target.id.Name)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoteTargetExistArgs - remote target ID exist RPC arguments.
|
||||
type RemoteTargetExistArgs struct {
|
||||
AuthArgs
|
||||
BucketName string
|
||||
TargetID event.TargetID
|
||||
}
|
||||
|
||||
// RemoteTargetExist - handles target ID exist RPC call which checks whether given target ID is a HTTP client target or not.
|
||||
func (receiver *peerRPCReceiver) RemoteTargetExist(args *RemoteTargetExistArgs, reply *bool) error {
|
||||
*reply = globalNotificationSys.RemoteTargetExist(args.BucketName, args.TargetID)
|
||||
return nil
|
||||
}
|
||||
|
||||
// SendEventArgs - send event RPC arguments.
|
||||
type SendEventArgs struct {
|
||||
AuthArgs
|
||||
Event event.Event
|
||||
TargetID event.TargetID
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// SendEvent - handles send event RPC call which sends given event to target by given target ID.
|
||||
func (receiver *peerRPCReceiver) SendEvent(args *SendEventArgs, reply *bool) error {
|
||||
// Set default to true to keep the target.
|
||||
*reply = true
|
||||
errs := globalNotificationSys.send(args.BucketName, args.Event, args.TargetID)
|
||||
|
||||
for i := range errs {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("Event", args.Event.EventName.String())
|
||||
reqInfo.AppendTags("targetName", args.TargetID.Name)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, errs[i].Err)
|
||||
|
||||
*reply = false // send failed i.e. do not keep the target.
|
||||
return errs[i].Err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetCredentialsArgs - set credentials RPC arguments.
|
||||
type SetCredentialsArgs struct {
|
||||
AuthArgs
|
||||
Credentials auth.Credentials
|
||||
}
|
||||
|
||||
// SetCredentials - handles set credentials RPC call.
|
||||
func (receiver *peerRPCReceiver) SetCredentials(args *SetCredentialsArgs, reply *VoidReply) error {
|
||||
if !args.Credentials.IsValid() {
|
||||
return fmt.Errorf("invalid credentials passed")
|
||||
}
|
||||
|
||||
// Acquire lock before updating global configuration.
|
||||
globalServerConfigMu.Lock()
|
||||
defer globalServerConfigMu.Unlock()
|
||||
|
||||
// Update credentials in memory
|
||||
prevCred := globalServerConfig.SetCredential(args.Credentials)
|
||||
|
||||
// Save credentials to config file
|
||||
if err := globalServerConfig.Save(); err != nil {
|
||||
// As saving configurstion failed, restore previous credential in memory.
|
||||
globalServerConfig.SetCredential(prevCred)
|
||||
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewPeerRPCServer - returns new peer RPC server.
|
||||
func NewPeerRPCServer() (*xrpc.Server, error) {
|
||||
rpcServer := xrpc.NewServer()
|
||||
if err := rpcServer.RegisterName(peerServiceName, &peerRPCReceiver{}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return rpcServer, nil
|
||||
}
|
||||
|
||||
// registerPeerRPCRouter - creates and registers Peer RPC server and its router.
|
||||
func registerPeerRPCRouter(router *mux.Router) {
|
||||
rpcServer, err := NewPeerRPCServer()
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
subrouter.Path(peerServiceSubPath).Handler(rpcServer)
|
||||
}
|
338
cmd/peer-rpc.go
338
cmd/peer-rpc.go
|
@ -1,338 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/event"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
"github.com/minio/minio/pkg/policy"
|
||||
)
|
||||
|
||||
const s3Path = "/s3/remote"
|
||||
|
||||
// PeerRPCReceiver - Peer RPC receiver for peer RPC server.
|
||||
type PeerRPCReceiver struct {
|
||||
AuthRPCServer
|
||||
}
|
||||
|
||||
// DeleteBucketArgs - delete bucket RPC arguments.
|
||||
type DeleteBucketArgs struct {
|
||||
AuthRPCArgs
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// DeleteBucket - handles delete bucket RPC call which removes all values of given bucket in global NotificationSys object.
|
||||
func (receiver *PeerRPCReceiver) DeleteBucket(args *DeleteBucketArgs, reply *AuthRPCArgs) error {
|
||||
globalNotificationSys.RemoveNotification(args.BucketName)
|
||||
globalPolicySys.Remove(args.BucketName)
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetBucketPolicyArgs - set bucket policy RPC arguments.
|
||||
type SetBucketPolicyArgs struct {
|
||||
AuthRPCArgs
|
||||
BucketName string
|
||||
Policy policy.Policy
|
||||
}
|
||||
|
||||
// SetBucketPolicy - handles set bucket policy RPC call which adds bucket policy to globalPolicySys.
|
||||
func (receiver *PeerRPCReceiver) SetBucketPolicy(args *SetBucketPolicyArgs, reply *AuthRPCArgs) error {
|
||||
globalPolicySys.Set(args.BucketName, args.Policy)
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoveBucketPolicyArgs - delete bucket policy RPC arguments.
|
||||
type RemoveBucketPolicyArgs struct {
|
||||
AuthRPCArgs
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// RemoveBucketPolicy - handles delete bucket policy RPC call which removes bucket policy to globalPolicySys.
|
||||
func (receiver *PeerRPCReceiver) RemoveBucketPolicy(args *RemoveBucketPolicyArgs, reply *AuthRPCArgs) error {
|
||||
globalPolicySys.Remove(args.BucketName)
|
||||
return nil
|
||||
}
|
||||
|
||||
// PutBucketNotificationArgs - put bucket notification RPC arguments.
|
||||
type PutBucketNotificationArgs struct {
|
||||
AuthRPCArgs
|
||||
BucketName string
|
||||
RulesMap event.RulesMap
|
||||
}
|
||||
|
||||
// PutBucketNotification - handles put bucket notification RPC call which adds rules to given bucket to global NotificationSys object.
|
||||
func (receiver *PeerRPCReceiver) PutBucketNotification(args *PutBucketNotificationArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
globalNotificationSys.AddRulesMap(args.BucketName, args.RulesMap)
|
||||
return nil
|
||||
}
|
||||
|
||||
// ListenBucketNotificationArgs - listen bucket notification RPC arguments.
|
||||
type ListenBucketNotificationArgs struct {
|
||||
AuthRPCArgs `json:"-"`
|
||||
BucketName string `json:"-"`
|
||||
EventNames []event.Name `json:"eventNames"`
|
||||
Pattern string `json:"pattern"`
|
||||
TargetID event.TargetID `json:"targetId"`
|
||||
Addr xnet.Host `json:"addr"`
|
||||
}
|
||||
|
||||
// ListenBucketNotification - handles listen bucket notification RPC call. It creates PeerRPCClient target which pushes requested events to target in remote peer.
|
||||
func (receiver *PeerRPCReceiver) ListenBucketNotification(args *ListenBucketNotificationArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
rpcClient := globalNotificationSys.GetPeerRPCClient(args.Addr)
|
||||
if rpcClient == nil {
|
||||
return fmt.Errorf("unable to find PeerRPCClient for provided address %v. This happens only if remote and this minio run with different set of endpoints", args.Addr)
|
||||
}
|
||||
|
||||
target := NewPeerRPCClientTarget(args.BucketName, args.TargetID, rpcClient)
|
||||
rulesMap := event.NewRulesMap(args.EventNames, args.Pattern, target.ID())
|
||||
if err := globalNotificationSys.AddRemoteTarget(args.BucketName, target, rulesMap); err != nil {
|
||||
reqInfo := &logger.ReqInfo{BucketName: target.bucketName}
|
||||
reqInfo.AppendTags("targetName", target.id.Name)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoteTargetExistArgs - remote target ID exist RPC arguments.
|
||||
type RemoteTargetExistArgs struct {
|
||||
AuthRPCArgs
|
||||
BucketName string
|
||||
TargetID event.TargetID
|
||||
}
|
||||
|
||||
// RemoteTargetExistReply - remote target ID exist RPC reply.
|
||||
type RemoteTargetExistReply struct {
|
||||
AuthRPCReply
|
||||
Exist bool
|
||||
}
|
||||
|
||||
// RemoteTargetExist - handles target ID exist RPC call which checks whether given target ID is a HTTP client target or not.
|
||||
func (receiver *PeerRPCReceiver) RemoteTargetExist(args *RemoteTargetExistArgs, reply *RemoteTargetExistReply) error {
|
||||
reply.Exist = globalNotificationSys.RemoteTargetExist(args.BucketName, args.TargetID)
|
||||
return nil
|
||||
}
|
||||
|
||||
// SendEventArgs - send event RPC arguments.
|
||||
type SendEventArgs struct {
|
||||
AuthRPCArgs
|
||||
Event event.Event
|
||||
TargetID event.TargetID
|
||||
BucketName string
|
||||
}
|
||||
|
||||
// SendEventReply - send event RPC reply.
|
||||
type SendEventReply struct {
|
||||
AuthRPCReply
|
||||
Error error
|
||||
}
|
||||
|
||||
// SendEvent - handles send event RPC call which sends given event to target by given target ID.
|
||||
func (receiver *PeerRPCReceiver) SendEvent(args *SendEventArgs, reply *SendEventReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
var err error
|
||||
for _, terr := range globalNotificationSys.send(args.BucketName, args.Event, args.TargetID) {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("Event", args.Event.EventName.String())
|
||||
reqInfo.AppendTags("targetName", args.TargetID.Name)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, terr.Err)
|
||||
err = terr.Err
|
||||
}
|
||||
|
||||
reply.Error = err
|
||||
return nil
|
||||
}
|
||||
|
||||
// registerS3PeerRPCRouter - creates and registers Peer RPC server and its router.
|
||||
func registerS3PeerRPCRouter(router *mux.Router) error {
|
||||
peerRPCServer := newRPCServer()
|
||||
if err := peerRPCServer.RegisterName("Peer", &PeerRPCReceiver{}); err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
subrouter.Path(s3Path).Handler(peerRPCServer)
|
||||
return nil
|
||||
}
|
||||
|
||||
// PeerRPCClient - peer RPC client talks to peer RPC server.
|
||||
type PeerRPCClient struct {
|
||||
*AuthRPCClient
|
||||
}
|
||||
|
||||
// DeleteBucket - calls delete bucket RPC.
|
||||
func (rpcClient *PeerRPCClient) DeleteBucket(bucketName string) error {
|
||||
args := DeleteBucketArgs{BucketName: bucketName}
|
||||
reply := AuthRPCReply{}
|
||||
return rpcClient.Call("Peer.DeleteBucket", &args, &reply)
|
||||
}
|
||||
|
||||
// SetBucketPolicy - calls set bucket policy RPC.
|
||||
func (rpcClient *PeerRPCClient) SetBucketPolicy(bucketName string, bucketPolicy *policy.Policy) error {
|
||||
args := SetBucketPolicyArgs{
|
||||
BucketName: bucketName,
|
||||
Policy: *bucketPolicy,
|
||||
}
|
||||
reply := AuthRPCReply{}
|
||||
return rpcClient.Call("Peer.SetBucketPolicy", &args, &reply)
|
||||
}
|
||||
|
||||
// RemoveBucketPolicy - calls remove bucket policy RPC.
|
||||
func (rpcClient *PeerRPCClient) RemoveBucketPolicy(bucketName string) error {
|
||||
args := RemoveBucketPolicyArgs{
|
||||
BucketName: bucketName,
|
||||
}
|
||||
reply := AuthRPCReply{}
|
||||
return rpcClient.Call("Peer.RemoveBucketPolicy", &args, &reply)
|
||||
}
|
||||
|
||||
// PutBucketNotification - calls put bukcet notification RPC.
|
||||
func (rpcClient *PeerRPCClient) PutBucketNotification(bucketName string, rulesMap event.RulesMap) error {
|
||||
args := PutBucketNotificationArgs{
|
||||
BucketName: bucketName,
|
||||
RulesMap: rulesMap,
|
||||
}
|
||||
reply := AuthRPCReply{}
|
||||
return rpcClient.Call("Peer.PutBucketNotification", &args, &reply)
|
||||
}
|
||||
|
||||
// ListenBucketNotification - calls listen bucket notification RPC.
|
||||
func (rpcClient *PeerRPCClient) ListenBucketNotification(bucketName string, eventNames []event.Name,
|
||||
pattern string, targetID event.TargetID, addr xnet.Host) error {
|
||||
args := ListenBucketNotificationArgs{
|
||||
BucketName: bucketName,
|
||||
EventNames: eventNames,
|
||||
Pattern: pattern,
|
||||
TargetID: targetID,
|
||||
Addr: addr,
|
||||
}
|
||||
reply := AuthRPCReply{}
|
||||
return rpcClient.Call("Peer.ListenBucketNotification", &args, &reply)
|
||||
}
|
||||
|
||||
// RemoteTargetExist - calls remote target ID exist RPC.
|
||||
func (rpcClient *PeerRPCClient) RemoteTargetExist(bucketName string, targetID event.TargetID) (bool, error) {
|
||||
args := RemoteTargetExistArgs{
|
||||
BucketName: bucketName,
|
||||
TargetID: targetID,
|
||||
}
|
||||
|
||||
reply := RemoteTargetExistReply{}
|
||||
if err := rpcClient.Call("Peer.RemoteTargetExist", &args, &reply); err != nil {
|
||||
return false, err
|
||||
}
|
||||
|
||||
return reply.Exist, nil
|
||||
}
|
||||
|
||||
// SendEvent - calls send event RPC.
|
||||
func (rpcClient *PeerRPCClient) SendEvent(bucketName string, targetID, remoteTargetID event.TargetID, eventData event.Event) error {
|
||||
args := SendEventArgs{
|
||||
BucketName: bucketName,
|
||||
TargetID: remoteTargetID,
|
||||
Event: eventData,
|
||||
}
|
||||
reply := SendEventReply{}
|
||||
if err := rpcClient.Call("Peer.SendEvent", &args, &reply); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if reply.Error != nil {
|
||||
reqInfo := &logger.ReqInfo{BucketName: bucketName}
|
||||
reqInfo.AppendTags("targetID", targetID.Name)
|
||||
reqInfo.AppendTags("event", eventData.EventName.String())
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, reply.Error)
|
||||
globalNotificationSys.RemoveRemoteTarget(bucketName, targetID)
|
||||
}
|
||||
|
||||
return reply.Error
|
||||
}
|
||||
|
||||
// makeRemoteRPCClients - creates Peer RPCClients for given endpoint list.
|
||||
func makeRemoteRPCClients(endpoints EndpointList) map[xnet.Host]*PeerRPCClient {
|
||||
peerRPCClientMap := make(map[xnet.Host]*PeerRPCClient)
|
||||
|
||||
cred := globalServerConfig.GetCredential()
|
||||
serviceEndpoint := path.Join(minioReservedBucketPath, s3Path)
|
||||
for _, hostStr := range GetRemotePeers(endpoints) {
|
||||
host, err := xnet.ParseHost(hostStr)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
peerRPCClientMap[*host] = &PeerRPCClient{newAuthRPCClient(authConfig{
|
||||
accessKey: cred.AccessKey,
|
||||
secretKey: cred.SecretKey,
|
||||
serverAddr: hostStr,
|
||||
serviceEndpoint: serviceEndpoint,
|
||||
secureConn: globalIsSSL,
|
||||
serviceName: "Peer",
|
||||
})}
|
||||
}
|
||||
|
||||
return peerRPCClientMap
|
||||
}
|
||||
|
||||
// PeerRPCClientTarget - RPCClient is an event.Target which sends event to target of remote peer.
|
||||
type PeerRPCClientTarget struct {
|
||||
id event.TargetID
|
||||
remoteTargetID event.TargetID
|
||||
rpcClient *PeerRPCClient
|
||||
bucketName string
|
||||
}
|
||||
|
||||
// ID - returns target ID.
|
||||
func (target *PeerRPCClientTarget) ID() event.TargetID {
|
||||
return target.id
|
||||
}
|
||||
|
||||
// Send - sends event to remote peer by making RPC call.
|
||||
func (target *PeerRPCClientTarget) Send(eventData event.Event) error {
|
||||
return target.rpcClient.SendEvent(target.bucketName, target.id, target.remoteTargetID, eventData)
|
||||
}
|
||||
|
||||
// Close - does nothing and available for interface compatibility.
|
||||
func (target *PeerRPCClientTarget) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// NewPeerRPCClientTarget - creates RPCClient target with given target ID available in remote peer.
|
||||
func NewPeerRPCClientTarget(bucketName string, targetID event.TargetID, rpcClient *PeerRPCClient) *PeerRPCClientTarget {
|
||||
return &PeerRPCClientTarget{
|
||||
id: event.TargetID{targetID.ID, targetID.Name + "+" + mustGetUUID()},
|
||||
remoteTargetID: targetID,
|
||||
bucketName: bucketName,
|
||||
rpcClient: rpcClient,
|
||||
}
|
||||
}
|
15
cmd/posix.go
15
cmd/posix.go
|
@ -137,7 +137,6 @@ func isDirEmpty(dirname string) bool {
|
|||
return false
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
// List one entry.
|
||||
_, err = f.Readdirnames(1)
|
||||
if err != io.EOF {
|
||||
|
@ -152,14 +151,15 @@ func isDirEmpty(dirname string) bool {
|
|||
}
|
||||
|
||||
// Initialize a new storage disk.
|
||||
func newPosix(path string) (StorageAPI, error) {
|
||||
func newPosix(path string) (*posix, error) {
|
||||
var err error
|
||||
if path, err = getValidPath(path); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
st := &posix{
|
||||
diskPath: path,
|
||||
p := &posix{
|
||||
connected: true,
|
||||
diskPath: path,
|
||||
// 1MiB buffer pool for posix internal operations.
|
||||
pool: sync.Pool{
|
||||
New: func() interface{} {
|
||||
|
@ -170,12 +170,10 @@ func newPosix(path string) (StorageAPI, error) {
|
|||
stopUsageCh: make(chan struct{}),
|
||||
}
|
||||
|
||||
st.connected = true
|
||||
|
||||
go st.diskUsage(globalServiceDoneCh)
|
||||
go p.diskUsage(globalServiceDoneCh)
|
||||
|
||||
// Success.
|
||||
return st, nil
|
||||
return p, nil
|
||||
}
|
||||
|
||||
// getDiskInfo returns given disk information.
|
||||
|
@ -285,6 +283,7 @@ func (s *posix) DiskInfo() (info DiskInfo, err error) {
|
|||
Free: di.Free,
|
||||
Used: atomic.LoadUint64(&s.totalUsed),
|
||||
}, nil
|
||||
|
||||
}
|
||||
|
||||
// getVolDir - will convert incoming volume names to
|
||||
|
|
|
@ -127,13 +127,6 @@ const (
|
|||
defaultRetryCap = 30 * time.Second // 30 seconds.
|
||||
)
|
||||
|
||||
// newRetryTimer creates a timer with exponentially increasing delays
|
||||
// until the maximum retry attempts are reached. - this function provides
|
||||
// resulting retry values to be of maximum jitter.
|
||||
func newRetryTimer(unit time.Duration, cap time.Duration, doneCh chan struct{}) <-chan int {
|
||||
return newRetryTimerWithJitter(unit, cap, MaxJitter, doneCh)
|
||||
}
|
||||
|
||||
// newRetryTimerSimple creates a timer with exponentially increasing delays
|
||||
// until the maximum retry attempts are reached. - this function is a
|
||||
// simpler version with all default values.
|
||||
|
|
|
@ -34,27 +34,15 @@ func newCacheObjectsFn() CacheObjectLayer {
|
|||
}
|
||||
|
||||
// Composed function registering routers for only distributed XL setup.
|
||||
func registerDistXLRouters(router *mux.Router, endpoints EndpointList) error {
|
||||
func registerDistXLRouters(router *mux.Router, endpoints EndpointList) {
|
||||
// Register storage rpc router only if its a distributed setup.
|
||||
err := registerStorageRPCRouters(router, endpoints)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
registerStorageRPCRouters(router, endpoints)
|
||||
|
||||
// Register distributed namespace lock.
|
||||
err = registerDistNSLockRouter(router, endpoints)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
registerDistNSLockRouter(router)
|
||||
|
||||
// Register S3 peer communication router.
|
||||
err = registerS3PeerRPCRouter(router)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Register RPC router for web related calls.
|
||||
return registerBrowserPeerRPCRouter(router)
|
||||
registerPeerRPCRouter(router)
|
||||
}
|
||||
|
||||
// List of some generic handlers which are applied for all incoming requests.
|
||||
|
@ -108,10 +96,7 @@ func configureServerHandler(endpoints EndpointList) (http.Handler, error) {
|
|||
}
|
||||
|
||||
// Add Admin RPC router
|
||||
err := registerAdminRPCRouter(router)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
registerAdminRPCRouter(router)
|
||||
|
||||
// Add Admin router.
|
||||
registerAdminRouter(router)
|
||||
|
|
|
@ -1,157 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2016, 2017, 2018 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 (
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/minio/dsync"
|
||||
)
|
||||
|
||||
// Allow any RPC call request time should be no more/less than 15 minutes.
|
||||
// 15 minute is chosen to be best for majority use cases.
|
||||
const rpcSkewTimeAllowed = 15 * time.Minute
|
||||
|
||||
// RPC V1 - Initial version
|
||||
// RPC V2 - format.json XL version changed to 2
|
||||
// RPC V3 - format.json XL version changed to 3
|
||||
// Current RPC version
|
||||
var globalRPCAPIVersion = semVersion{3, 0, 0}
|
||||
|
||||
func isRequestTimeAllowed(requestTime time.Time) bool {
|
||||
// Check whether request time is within acceptable skew time.
|
||||
utcNow := UTCNow()
|
||||
return !(requestTime.Sub(utcNow) > rpcSkewTimeAllowed ||
|
||||
utcNow.Sub(requestTime) > rpcSkewTimeAllowed)
|
||||
}
|
||||
|
||||
// semVersion - RPC semantic versioning.
|
||||
type semVersion struct {
|
||||
Major uint64
|
||||
Minor uint64
|
||||
Patch uint64
|
||||
}
|
||||
|
||||
// semver comparator implementation based on the semver 2.0.0 https://semver.org/.
|
||||
func (v semVersion) Compare(o semVersion) int {
|
||||
if v.Major != o.Major {
|
||||
if v.Major > o.Major {
|
||||
return 1
|
||||
}
|
||||
return -1
|
||||
}
|
||||
if v.Minor != o.Minor {
|
||||
if v.Minor > o.Minor {
|
||||
return 1
|
||||
}
|
||||
return -1
|
||||
}
|
||||
if v.Patch != o.Patch {
|
||||
if v.Patch > o.Patch {
|
||||
return 1
|
||||
}
|
||||
return -1
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (v semVersion) String() string {
|
||||
b := make([]byte, 0, 5)
|
||||
b = strconv.AppendUint(b, v.Major, 10)
|
||||
b = append(b, '.')
|
||||
b = strconv.AppendUint(b, v.Minor, 10)
|
||||
b = append(b, '.')
|
||||
b = strconv.AppendUint(b, v.Patch, 10)
|
||||
return string(b)
|
||||
}
|
||||
|
||||
// AuthRPCArgs represents minimum required arguments to make any authenticated RPC call.
|
||||
type AuthRPCArgs struct {
|
||||
// Authentication token to be verified by the server for every RPC call.
|
||||
AuthToken string
|
||||
Version semVersion
|
||||
}
|
||||
|
||||
// SetAuthToken - sets the token to the supplied value.
|
||||
func (args *AuthRPCArgs) SetAuthToken(authToken string) {
|
||||
args.AuthToken = authToken
|
||||
}
|
||||
|
||||
// SetRPCAPIVersion - sets the rpc version to the supplied value.
|
||||
func (args *AuthRPCArgs) SetRPCAPIVersion(version semVersion) {
|
||||
args.Version = version
|
||||
}
|
||||
|
||||
// IsAuthenticated - validated whether this auth RPC args are already authenticated or not.
|
||||
func (args AuthRPCArgs) IsAuthenticated() error {
|
||||
// checks if rpc Version is not equal to current server rpc version.
|
||||
// this is fine for now, but in future when we add backward compatible
|
||||
// APIs we need to make sure to allow lesser versioned clients to
|
||||
// talk over RPC, until then we are fine with this check.
|
||||
if args.Version.Compare(globalRPCAPIVersion) != 0 {
|
||||
return errRPCAPIVersionUnsupported
|
||||
}
|
||||
|
||||
// Check whether the token is valid
|
||||
if !isAuthTokenValid(args.AuthToken) {
|
||||
return errInvalidToken
|
||||
}
|
||||
|
||||
// Good to go.
|
||||
return nil
|
||||
}
|
||||
|
||||
// AuthRPCReply represents minimum required reply for any authenticated RPC call.
|
||||
type AuthRPCReply struct{}
|
||||
|
||||
// LoginRPCArgs - login username and password for RPC.
|
||||
type LoginRPCArgs struct {
|
||||
AuthToken string
|
||||
Version semVersion
|
||||
RequestTime time.Time
|
||||
}
|
||||
|
||||
// IsValid - validates whether this LoginRPCArgs are valid for authentication.
|
||||
func (args LoginRPCArgs) IsValid() error {
|
||||
// checks if rpc Version is not equal to current server rpc version.
|
||||
// this is fine for now, but in future when we add backward compatible
|
||||
// APIs we need to make sure to allow lesser versioned clients to
|
||||
// talk over RPC, until then we are fine with this check.
|
||||
if args.Version.Compare(globalRPCAPIVersion) != 0 {
|
||||
return errRPCAPIVersionUnsupported
|
||||
}
|
||||
|
||||
if !isRequestTimeAllowed(args.RequestTime) {
|
||||
return errServerTimeMismatch
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// LoginRPCReply - login reply is a dummy struct perhaps for future use.
|
||||
type LoginRPCReply struct{}
|
||||
|
||||
// LockArgs represents arguments for any authenticated lock RPC call.
|
||||
type LockArgs struct {
|
||||
AuthRPCArgs
|
||||
LockArgs dsync.LockArgs
|
||||
}
|
||||
|
||||
func newLockArgs(args dsync.LockArgs) LockArgs {
|
||||
return LockArgs{LockArgs: args}
|
||||
}
|
|
@ -1,56 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 Minio, Inc.
|
||||
*
|
||||
* Licensed under the Apache License, semVersion 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 "testing"
|
||||
|
||||
// Tests version comparator.
|
||||
func TestCompare(t *testing.T) {
|
||||
type compareTest struct {
|
||||
v1 semVersion
|
||||
v2 semVersion
|
||||
result int
|
||||
}
|
||||
|
||||
var compareTests = []compareTest{
|
||||
{semVersion{1, 0, 0}, semVersion{1, 0, 0}, 0},
|
||||
{semVersion{2, 0, 0}, semVersion{1, 0, 0}, 1},
|
||||
{semVersion{0, 1, 0}, semVersion{0, 1, 0}, 0},
|
||||
{semVersion{0, 2, 0}, semVersion{0, 1, 0}, 1},
|
||||
{semVersion{0, 0, 1}, semVersion{0, 0, 1}, 0},
|
||||
{semVersion{0, 0, 2}, semVersion{0, 0, 1}, 1},
|
||||
{semVersion{1, 2, 3}, semVersion{1, 2, 3}, 0},
|
||||
{semVersion{2, 2, 4}, semVersion{1, 2, 4}, 1},
|
||||
{semVersion{1, 3, 3}, semVersion{1, 2, 3}, 1},
|
||||
{semVersion{1, 2, 4}, semVersion{1, 2, 3}, 1},
|
||||
|
||||
// Spec Examples #11
|
||||
{semVersion{1, 0, 0}, semVersion{2, 0, 0}, -1},
|
||||
{semVersion{2, 0, 0}, semVersion{2, 1, 0}, -1},
|
||||
{semVersion{2, 1, 0}, semVersion{2, 1, 1}, -1},
|
||||
}
|
||||
|
||||
for _, test := range compareTests {
|
||||
if res := test.v1.Compare(test.v2); res != test.result {
|
||||
t.Errorf("Comparing %q : %q, expected %d but got %d", test.v1, test.v2, test.result, res)
|
||||
}
|
||||
// Test if reverse is true as well.
|
||||
if res := test.v2.Compare(test.v1); res != -test.result {
|
||||
t.Errorf("Comparing %q : %q, expected %d but got %d", test.v2, test.v1, -test.result, res)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,62 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"context"
|
||||
"io"
|
||||
"net/http"
|
||||
"net/rpc"
|
||||
|
||||
miniohttp "github.com/minio/minio/cmd/http"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
)
|
||||
|
||||
// ServeHTTP implements an http.Handler that answers RPC requests,
|
||||
// hijacks the underlying connection and clears all deadlines if any.
|
||||
func (server *rpcServer) ServeHTTP(w http.ResponseWriter, req *http.Request) {
|
||||
if req.Method != http.MethodConnect {
|
||||
w.WriteHeader(http.StatusMethodNotAllowed)
|
||||
return
|
||||
}
|
||||
|
||||
conn, _, err := w.(http.Hijacker).Hijack()
|
||||
if err != nil {
|
||||
reqInfo := (&logger.ReqInfo{}).AppendTags("remoteaddr", req.RemoteAddr)
|
||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||
logger.LogIf(ctx, err)
|
||||
return
|
||||
}
|
||||
|
||||
// Overrides Read/Write deadlines if any.
|
||||
bufConn, ok := conn.(*miniohttp.BufConn)
|
||||
if ok {
|
||||
bufConn.RemoveTimeout()
|
||||
conn = bufConn
|
||||
}
|
||||
|
||||
// Can connect to RPC service using HTTP CONNECT to rpcPath.
|
||||
io.WriteString(conn, "HTTP/1.0 200 Connected to Go RPC\n\n")
|
||||
server.ServeConn(conn)
|
||||
}
|
||||
|
||||
type rpcServer struct{ *rpc.Server }
|
||||
|
||||
// Similar to rpc.NewServer() provides a custom ServeHTTP override.
|
||||
func newRPCServer() *rpcServer {
|
||||
return &rpcServer{rpc.NewServer()}
|
||||
}
|
|
@ -1,97 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
)
|
||||
|
||||
type ArithArgs struct {
|
||||
A, B int
|
||||
}
|
||||
|
||||
type ArithReply struct {
|
||||
C int
|
||||
}
|
||||
|
||||
type Arith struct {
|
||||
AuthRPCServer
|
||||
}
|
||||
|
||||
// Some of Arith's methods have value args, some have pointer args. That's deliberate.
|
||||
func (t *Arith) Add(args ArithArgs, reply *ArithReply) error {
|
||||
reply.C = args.A + args.B
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestGoHTTPRPC(t *testing.T) {
|
||||
newServer := newRPCServer()
|
||||
newServer.Register(&Arith{
|
||||
AuthRPCServer: AuthRPCServer{},
|
||||
})
|
||||
|
||||
router := mux.NewRouter().SkipClean(true)
|
||||
router.Path("/foo").Handler(newServer)
|
||||
|
||||
httpServer := httptest.NewServer(router)
|
||||
defer httpServer.Close()
|
||||
|
||||
rootPath, err := newTestConfig("us-east-1")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer os.RemoveAll(rootPath)
|
||||
|
||||
creds := globalServerConfig.GetCredential()
|
||||
client := newAuthRPCClient(authConfig{
|
||||
serverAddr: httpServer.Listener.Addr().String(),
|
||||
serviceName: "Arith",
|
||||
serviceEndpoint: "/foo",
|
||||
accessKey: creds.AccessKey,
|
||||
secretKey: creds.SecretKey,
|
||||
})
|
||||
defer client.Close()
|
||||
|
||||
if err = client.Login(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Synchronous calls
|
||||
args := &ArithArgs{7, 8}
|
||||
reply := new(ArithReply)
|
||||
if err = client.rpcClient.Call("Arith.Add", args, reply); err != nil {
|
||||
t.Errorf("Add: expected no error but got string %v", err)
|
||||
}
|
||||
|
||||
if reply.C != args.A+args.B {
|
||||
t.Errorf("Add: expected %d got %d", reply.C, args.A+args.B)
|
||||
}
|
||||
|
||||
resp, err := http.Get(httpServer.URL + "/foo")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
if resp.StatusCode != http.StatusMethodNotAllowed {
|
||||
t.Errorf("Expected %d, got %d", http.StatusMethodNotAllowed, resp.StatusCode)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,257 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"crypto/tls"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/url"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
// DefaultSkewTime - skew time is 15 minutes between minio peers.
|
||||
const DefaultSkewTime = 15 * time.Minute
|
||||
|
||||
var errRPCRetry = fmt.Errorf("rpc: retry error")
|
||||
|
||||
func isNetError(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if uerr, isURLError := err.(*url.Error); isURLError {
|
||||
if uerr.Timeout() {
|
||||
return true
|
||||
}
|
||||
|
||||
err = uerr.Err
|
||||
}
|
||||
|
||||
_, isNetOpError := err.(*net.OpError)
|
||||
return isNetOpError
|
||||
}
|
||||
|
||||
// RPCVersion - RPC semantic version based on semver 2.0.0 https://semver.org/.
|
||||
type RPCVersion struct {
|
||||
Major uint64
|
||||
Minor uint64
|
||||
Patch uint64
|
||||
}
|
||||
|
||||
// Compare - compares given version with this version.
|
||||
func (v RPCVersion) Compare(o RPCVersion) int {
|
||||
compare := func(v1, v2 uint64) int {
|
||||
if v1 == v2 {
|
||||
return 0
|
||||
}
|
||||
|
||||
if v1 > v2 {
|
||||
return 1
|
||||
}
|
||||
|
||||
return -1
|
||||
}
|
||||
|
||||
if r := compare(v.Major, o.Major); r != 0 {
|
||||
return r
|
||||
}
|
||||
|
||||
if r := compare(v.Minor, o.Minor); r != 0 {
|
||||
return r
|
||||
}
|
||||
|
||||
return compare(v.Patch, o.Patch)
|
||||
}
|
||||
|
||||
func (v RPCVersion) String() string {
|
||||
return fmt.Sprintf("%v.%v.%v", v.Major, v.Minor, v.Patch)
|
||||
}
|
||||
|
||||
// AuthArgs - base argument for any RPC call for authentication.
|
||||
type AuthArgs struct {
|
||||
Token string
|
||||
RPCVersion RPCVersion
|
||||
RequestTime time.Time
|
||||
}
|
||||
|
||||
// Authenticate - checks if given arguments are valid to allow RPC call.
|
||||
// This is xrpc.Authenticator and is called in RPC server.
|
||||
func (args AuthArgs) Authenticate() error {
|
||||
// Check whether request time is within acceptable skew time.
|
||||
utcNow := time.Now().UTC()
|
||||
if args.RequestTime.Sub(utcNow) > DefaultSkewTime || utcNow.Sub(args.RequestTime) > DefaultSkewTime {
|
||||
return fmt.Errorf("client time %v is too apart with server time %v", args.RequestTime, utcNow)
|
||||
}
|
||||
|
||||
if globalRPCAPIVersion.Compare(args.RPCVersion) != 0 {
|
||||
return fmt.Errorf("version mismatch. expected: %v, received: %v", globalRPCAPIVersion, args.RPCVersion)
|
||||
}
|
||||
|
||||
if !isAuthTokenValid(args.Token) {
|
||||
return errAuthentication
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// SetAuthArgs - sets given authentication arguments to this args. This is called in RPC client.
|
||||
func (args *AuthArgs) SetAuthArgs(authArgs AuthArgs) {
|
||||
*args = authArgs
|
||||
}
|
||||
|
||||
// VoidReply - void (empty) RPC reply.
|
||||
type VoidReply struct{}
|
||||
|
||||
// RPCClientArgs - RPC client arguments.
|
||||
type RPCClientArgs struct {
|
||||
NewAuthTokenFunc func() string
|
||||
RPCVersion RPCVersion
|
||||
ServiceName string
|
||||
ServiceURL *xnet.URL
|
||||
TLSConfig *tls.Config
|
||||
}
|
||||
|
||||
// validate - checks whether given args are valid or not.
|
||||
func (args RPCClientArgs) validate() error {
|
||||
if args.NewAuthTokenFunc == nil {
|
||||
return fmt.Errorf("NewAuthTokenFunc must not be empty")
|
||||
}
|
||||
|
||||
if args.ServiceName == "" {
|
||||
return fmt.Errorf("ServiceName must not be empty")
|
||||
}
|
||||
|
||||
if args.ServiceURL.Scheme != "http" && args.ServiceURL.Scheme != "https" {
|
||||
return fmt.Errorf("unknown RPC URL %v", args.ServiceURL)
|
||||
}
|
||||
|
||||
if args.ServiceURL.User != nil || args.ServiceURL.ForceQuery || args.ServiceURL.RawQuery != "" || args.ServiceURL.Fragment != "" {
|
||||
return fmt.Errorf("unknown RPC URL %v", args.ServiceURL)
|
||||
}
|
||||
|
||||
if args.ServiceURL.Scheme == "https" && args.TLSConfig == nil {
|
||||
return fmt.Errorf("tls configuration must not be empty for https url %v", args.ServiceURL)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// RPCClient - base RPC client.
|
||||
type RPCClient struct {
|
||||
sync.RWMutex
|
||||
args RPCClientArgs
|
||||
authToken string
|
||||
rpcClient *xrpc.Client
|
||||
retryTicker *time.Ticker
|
||||
}
|
||||
|
||||
func (client *RPCClient) setRetryTicker(ticker *time.Ticker) {
|
||||
client.Lock()
|
||||
defer client.Unlock()
|
||||
|
||||
if client.retryTicker != nil {
|
||||
client.retryTicker.Stop()
|
||||
}
|
||||
|
||||
client.retryTicker = ticker
|
||||
}
|
||||
|
||||
// Call - calls servicemethod on remote server.
|
||||
func (client *RPCClient) Call(serviceMethod string, args interface {
|
||||
SetAuthArgs(args AuthArgs)
|
||||
}, reply interface{}) (err error) {
|
||||
lockedCall := func() error {
|
||||
client.RLock()
|
||||
defer client.RUnlock()
|
||||
|
||||
if client.retryTicker != nil {
|
||||
select {
|
||||
case <-client.retryTicker.C:
|
||||
default:
|
||||
return errRPCRetry
|
||||
}
|
||||
}
|
||||
|
||||
// Make RPC call.
|
||||
args.SetAuthArgs(AuthArgs{client.authToken, client.args.RPCVersion, time.Now().UTC()})
|
||||
return client.rpcClient.Call(serviceMethod, args, reply)
|
||||
}
|
||||
|
||||
call := func() error {
|
||||
err = lockedCall()
|
||||
|
||||
if err == errRPCRetry {
|
||||
return err
|
||||
}
|
||||
|
||||
if isNetError(err) {
|
||||
client.setRetryTicker(time.NewTicker(xrpc.DefaultRPCTimeout))
|
||||
} else {
|
||||
client.setRetryTicker(nil)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// If authentication error is received, retry the same call only once
|
||||
// with new authentication token.
|
||||
if err = call(); err == nil {
|
||||
return nil
|
||||
}
|
||||
if err.Error() != errAuthentication.Error() {
|
||||
return err
|
||||
}
|
||||
|
||||
client.Lock()
|
||||
client.authToken = client.args.NewAuthTokenFunc()
|
||||
client.Unlock()
|
||||
return call()
|
||||
}
|
||||
|
||||
// Close - closes underneath RPC client.
|
||||
func (client *RPCClient) Close() error {
|
||||
client.Lock()
|
||||
defer client.Unlock()
|
||||
|
||||
client.authToken = ""
|
||||
return client.rpcClient.Close()
|
||||
}
|
||||
|
||||
// ServiceURL - returns service URL used for RPC call.
|
||||
func (client *RPCClient) ServiceURL() *xnet.URL {
|
||||
// Take copy of ServiceURL
|
||||
u := *(client.args.ServiceURL)
|
||||
return &u
|
||||
}
|
||||
|
||||
// NewRPCClient - returns new RPC client.
|
||||
func NewRPCClient(args RPCClientArgs) (*RPCClient, error) {
|
||||
if err := args.validate(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &RPCClient{
|
||||
args: args,
|
||||
authToken: args.NewAuthTokenFunc(),
|
||||
rpcClient: xrpc.NewClient(args.ServiceURL, args.TLSConfig, xrpc.DefaultRPCTimeout),
|
||||
}, nil
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 rpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"encoding/gob"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
xhttp "github.com/minio/minio/cmd/http"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
// DefaultRPCTimeout - default RPC timeout is one minute.
|
||||
const DefaultRPCTimeout = 1 * time.Minute
|
||||
|
||||
// Client - http based RPC client.
|
||||
type Client struct {
|
||||
httpClient *http.Client
|
||||
serviceURL *xnet.URL
|
||||
}
|
||||
|
||||
// Call - calls service method on RPC server.
|
||||
func (client *Client) Call(serviceMethod string, args, reply interface{}) error {
|
||||
replyKind := reflect.TypeOf(reply).Kind()
|
||||
if replyKind != reflect.Ptr {
|
||||
return fmt.Errorf("rpc reply must be a pointer type, but found %v", replyKind)
|
||||
}
|
||||
|
||||
data, err := gobEncode(args)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
callRequest := CallRequest{
|
||||
Method: serviceMethod,
|
||||
ArgBytes: data,
|
||||
}
|
||||
|
||||
var buf bytes.Buffer
|
||||
if err = gob.NewEncoder(&buf).Encode(callRequest); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
response, err := client.httpClient.Post(client.serviceURL.String(), "", &buf)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer response.Body.Close()
|
||||
|
||||
if response.StatusCode != http.StatusOK {
|
||||
return fmt.Errorf("%v rpc call failed with error code %v", serviceMethod, response.StatusCode)
|
||||
}
|
||||
|
||||
var callResponse CallResponse
|
||||
if err := gob.NewDecoder(response.Body).Decode(&callResponse); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if callResponse.Error != "" {
|
||||
return errors.New(callResponse.Error)
|
||||
}
|
||||
|
||||
return gobDecode(callResponse.ReplyBytes, reply)
|
||||
}
|
||||
|
||||
// Close - does nothing and presents for interface compatibility.
|
||||
func (client *Client) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func newCustomDialContext(timeout time.Duration) func(ctx context.Context, network, addr string) (net.Conn, error) {
|
||||
return func(ctx context.Context, network, addr string) (net.Conn, error) {
|
||||
dialer := &net.Dialer{
|
||||
Timeout: timeout,
|
||||
KeepAlive: timeout,
|
||||
DualStack: true,
|
||||
}
|
||||
|
||||
conn, err := dialer.DialContext(ctx, network, addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return xhttp.NewTimeoutConn(conn, timeout, timeout), nil
|
||||
}
|
||||
}
|
||||
|
||||
// NewClient - returns new RPC client.
|
||||
func NewClient(serviceURL *xnet.URL, tlsConfig *tls.Config, timeout time.Duration) *Client {
|
||||
return &Client{
|
||||
httpClient: &http.Client{
|
||||
// Transport is exactly same as Go default in https://golang.org/pkg/net/http/#RoundTripper
|
||||
// except custom DialContext and TLSClientConfig.
|
||||
Transport: &http.Transport{
|
||||
Proxy: http.ProxyFromEnvironment,
|
||||
DialContext: newCustomDialContext(timeout),
|
||||
MaxIdleConns: 100,
|
||||
IdleConnTimeout: 90 * time.Second,
|
||||
TLSHandshakeTimeout: 10 * time.Second,
|
||||
ExpectContinueTimeout: 1 * time.Second,
|
||||
TLSClientConfig: tlsConfig,
|
||||
},
|
||||
},
|
||||
serviceURL: serviceURL,
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 rpc
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
func TestClientCall(t *testing.T) {
|
||||
rpcServer := NewServer()
|
||||
if err := rpcServer.RegisterName("Arith", &Arith{}); err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
httpServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
rpcServer.ServeHTTP(w, r)
|
||||
}))
|
||||
defer httpServer.Close()
|
||||
|
||||
url, err := xnet.ParseURL(httpServer.URL)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
rpcClient := NewClient(url, nil, DefaultRPCTimeout)
|
||||
|
||||
var reply int
|
||||
var boolReply bool
|
||||
var intArg int
|
||||
|
||||
testCases := []struct {
|
||||
serviceMethod string
|
||||
args interface{}
|
||||
reply interface{}
|
||||
expectErr bool
|
||||
}{
|
||||
{"Arith.Multiply", Args{7, 8}, &reply, false},
|
||||
{"Arith.Multiply", &Args{7, 8}, &reply, false},
|
||||
// rpc reply must be a pointer type but found int error.
|
||||
{"Arith.Multiply", &Args{7, 8}, reply, true},
|
||||
// gob: type mismatch in decoder: want struct type rpc.Args; got non-struct error.
|
||||
{"Arith.Multiply", intArg, &reply, true},
|
||||
// gob: decoding into local type *bool, received remote type int error.
|
||||
{"Arith.Multiply", &Args{7, 8}, &boolReply, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := rpcClient.Call(testCase.serviceMethod, testCase.args, testCase.reply)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,264 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 rpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/gob"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/http"
|
||||
"reflect"
|
||||
"strings"
|
||||
"unicode"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
// Authenticator - validator of first argument of any RPC call.
|
||||
type Authenticator interface {
|
||||
// Method to validate first argument of any RPC call.
|
||||
Authenticate() error
|
||||
}
|
||||
|
||||
// reflect.Type of error interface.
|
||||
var errorType = reflect.TypeOf((*error)(nil)).Elem()
|
||||
|
||||
// reflect.Type of Authenticator interface.
|
||||
var authenticatorType = reflect.TypeOf((*Authenticator)(nil)).Elem()
|
||||
|
||||
func gobEncode(e interface{}) ([]byte, error) {
|
||||
var buf bytes.Buffer
|
||||
|
||||
if err := gob.NewEncoder(&buf).Encode(e); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return buf.Bytes(), nil
|
||||
}
|
||||
|
||||
func gobDecode(data []byte, e interface{}) error {
|
||||
return gob.NewDecoder(bytes.NewReader(data)).Decode(e)
|
||||
}
|
||||
|
||||
// Returns whether given type is exported or builin type or not.
|
||||
func isExportedOrBuiltinType(t reflect.Type) bool {
|
||||
for t.Kind() == reflect.Ptr {
|
||||
t = t.Elem()
|
||||
}
|
||||
|
||||
rune, _ := utf8.DecodeRuneInString(t.Name())
|
||||
return unicode.IsUpper(rune) || t.PkgPath() == ""
|
||||
}
|
||||
|
||||
// Makes method name map from given type.
|
||||
func getMethodMap(receiverType reflect.Type) map[string]reflect.Method {
|
||||
methodMap := make(map[string]reflect.Method)
|
||||
for i := 0; i < receiverType.NumMethod(); i++ {
|
||||
// Method.PkgPath is empty for this package.
|
||||
method := receiverType.Method(i)
|
||||
|
||||
// Methods must have three arguments (receiver, args, reply)
|
||||
if method.Type.NumIn() != 3 {
|
||||
continue
|
||||
}
|
||||
|
||||
// First argument must be exported.
|
||||
if !isExportedOrBuiltinType(method.Type.In(1)) {
|
||||
continue
|
||||
}
|
||||
|
||||
// First argument must be Authenticator.
|
||||
if !method.Type.In(1).Implements(authenticatorType) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Second argument must be exported or builtin type.
|
||||
if !isExportedOrBuiltinType(method.Type.In(2)) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Second argument must be a pointer.
|
||||
if method.Type.In(2).Kind() != reflect.Ptr {
|
||||
continue
|
||||
}
|
||||
|
||||
// Method must return one value.
|
||||
if method.Type.NumOut() != 1 {
|
||||
continue
|
||||
}
|
||||
|
||||
// The return type of the method must be error.
|
||||
if method.Type.Out(0) != errorType {
|
||||
continue
|
||||
}
|
||||
|
||||
methodMap[method.Name] = method
|
||||
}
|
||||
|
||||
return methodMap
|
||||
}
|
||||
|
||||
// Server - HTTP based RPC server.
|
||||
type Server struct {
|
||||
serviceName string
|
||||
receiverValue reflect.Value
|
||||
methodMap map[string]reflect.Method
|
||||
}
|
||||
|
||||
// RegisterName - registers receiver with given name to handle RPC requests.
|
||||
func (server *Server) RegisterName(name string, receiver interface{}) error {
|
||||
server.serviceName = name
|
||||
|
||||
server.receiverValue = reflect.ValueOf(receiver)
|
||||
if !reflect.Indirect(server.receiverValue).IsValid() {
|
||||
return fmt.Errorf("nil receiver")
|
||||
}
|
||||
|
||||
receiverName := reflect.Indirect(server.receiverValue).Type().Name()
|
||||
receiverType := reflect.TypeOf(receiver)
|
||||
server.methodMap = getMethodMap(receiverType)
|
||||
if len(server.methodMap) == 0 {
|
||||
str := "rpc.Register: type " + receiverName + " has no exported methods of suitable type"
|
||||
|
||||
// To help the user, see if a pointer receiver would work.
|
||||
if len(getMethodMap(reflect.PtrTo(receiverType))) != 0 {
|
||||
str += " (hint: pass a pointer to value of that type)"
|
||||
}
|
||||
|
||||
return errors.New(str)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// call - call service method in receiver.
|
||||
func (server *Server) call(serviceMethod string, argBytes []byte) (replyBytes []byte, err error) {
|
||||
tokens := strings.SplitN(serviceMethod, ".", 2)
|
||||
if len(tokens) != 2 {
|
||||
return nil, fmt.Errorf("invalid service/method request ill-formed %v", serviceMethod)
|
||||
}
|
||||
|
||||
serviceName := tokens[0]
|
||||
if serviceName != server.serviceName {
|
||||
return nil, fmt.Errorf("can't find service %v", serviceName)
|
||||
}
|
||||
|
||||
methodName := tokens[1]
|
||||
method, found := server.methodMap[methodName]
|
||||
if !found {
|
||||
return nil, fmt.Errorf("can't find method %v", methodName)
|
||||
}
|
||||
|
||||
var argv reflect.Value
|
||||
|
||||
// Decode the argument value.
|
||||
argIsValue := false // if true, need to indirect before calling.
|
||||
if method.Type.In(1).Kind() == reflect.Ptr {
|
||||
argv = reflect.New(method.Type.In(1).Elem())
|
||||
} else {
|
||||
argv = reflect.New(method.Type.In(1))
|
||||
argIsValue = true
|
||||
}
|
||||
|
||||
if err = gobDecode(argBytes, argv.Interface()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if argIsValue {
|
||||
argv = argv.Elem()
|
||||
}
|
||||
|
||||
// call Authenticate() method.
|
||||
authMethod, ok := method.Type.In(1).MethodByName("Authenticate")
|
||||
if !ok {
|
||||
panic("Authenticate() method not found. This should not happen.")
|
||||
}
|
||||
returnValues := authMethod.Func.Call([]reflect.Value{argv})
|
||||
errInter := returnValues[0].Interface()
|
||||
if errInter != nil {
|
||||
err = errInter.(error)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
replyv := reflect.New(method.Type.In(2).Elem())
|
||||
|
||||
switch method.Type.In(2).Elem().Kind() {
|
||||
case reflect.Map:
|
||||
replyv.Elem().Set(reflect.MakeMap(method.Type.In(2).Elem()))
|
||||
case reflect.Slice:
|
||||
replyv.Elem().Set(reflect.MakeSlice(method.Type.In(2).Elem(), 0, 0))
|
||||
}
|
||||
|
||||
returnValues = method.Func.Call([]reflect.Value{server.receiverValue, argv, replyv})
|
||||
errInter = returnValues[0].Interface()
|
||||
if errInter != nil {
|
||||
err = errInter.(error)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return gobEncode(replyv.Interface())
|
||||
}
|
||||
|
||||
// CallRequest - RPC call request parameters.
|
||||
type CallRequest struct {
|
||||
Method string
|
||||
ArgBytes []byte
|
||||
}
|
||||
|
||||
// CallResponse - RPC call response parameters.
|
||||
type CallResponse struct {
|
||||
Error string
|
||||
ReplyBytes []byte
|
||||
}
|
||||
|
||||
// ServeHTTP - handles RPC on HTTP request.
|
||||
func (server *Server) ServeHTTP(w http.ResponseWriter, req *http.Request) {
|
||||
if req.Method != http.MethodPost {
|
||||
w.WriteHeader(http.StatusMethodNotAllowed)
|
||||
return
|
||||
}
|
||||
|
||||
var callRequest CallRequest
|
||||
if err := gob.NewDecoder(req.Body).Decode(&callRequest); err != nil {
|
||||
w.WriteHeader(http.StatusBadRequest)
|
||||
return
|
||||
}
|
||||
|
||||
var callResponse CallResponse
|
||||
var err error
|
||||
callResponse.ReplyBytes, err = server.call(callRequest.Method, callRequest.ArgBytes)
|
||||
if err != nil {
|
||||
callResponse.Error = err.Error()
|
||||
}
|
||||
|
||||
data, err := gobEncode(callResponse)
|
||||
if err != nil {
|
||||
w.WriteHeader(http.StatusInternalServerError)
|
||||
return
|
||||
}
|
||||
|
||||
w.Write(data)
|
||||
}
|
||||
|
||||
// NewServer - returns new RPC server.
|
||||
func NewServer() *Server {
|
||||
return &Server{}
|
||||
}
|
|
@ -0,0 +1,345 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 rpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
type Args struct {
|
||||
A, B int
|
||||
}
|
||||
|
||||
func (a *Args) Authenticate() (err error) {
|
||||
if a.A == 0 && a.B == 0 {
|
||||
err = errors.New("authenticated failed")
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
type Quotient struct {
|
||||
Quo, Rem int
|
||||
}
|
||||
|
||||
type Arith struct{}
|
||||
|
||||
func (t *Arith) Multiply(args *Args, reply *int) error {
|
||||
*reply = args.A * args.B
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Arith) Divide(args *Args, quo *Quotient) error {
|
||||
if args.B == 0 {
|
||||
return errors.New("divide by zero")
|
||||
}
|
||||
quo.Quo = args.A / args.B
|
||||
quo.Rem = args.A % args.B
|
||||
return nil
|
||||
}
|
||||
|
||||
type mytype int
|
||||
|
||||
type Auth struct{}
|
||||
|
||||
func (a Auth) Authenticate() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// exported method.
|
||||
func (t mytype) Foo(a *Auth, b *int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// incompatible method because of unexported method.
|
||||
func (t mytype) foo(a *Auth, b *int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// incompatible method because of first argument is not Authenticator.
|
||||
func (t *mytype) Bar(a, b *int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// incompatible method because of error is not returned.
|
||||
func (t mytype) IncompatFoo(a, b *int) {
|
||||
}
|
||||
|
||||
// incompatible method because of second argument is not a pointer.
|
||||
func (t *mytype) IncompatBar(a *int, b int) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestIsExportedOrBuiltinType(t *testing.T) {
|
||||
var i int
|
||||
case1Type := reflect.TypeOf(i)
|
||||
|
||||
var iptr *int
|
||||
case2Type := reflect.TypeOf(iptr)
|
||||
|
||||
var a Arith
|
||||
case3Type := reflect.TypeOf(a)
|
||||
|
||||
var aptr *Arith
|
||||
case4Type := reflect.TypeOf(aptr)
|
||||
|
||||
var m mytype
|
||||
case5Type := reflect.TypeOf(m)
|
||||
|
||||
var mptr *mytype
|
||||
case6Type := reflect.TypeOf(mptr)
|
||||
|
||||
testCases := []struct {
|
||||
t reflect.Type
|
||||
expectedResult bool
|
||||
}{
|
||||
{case1Type, true},
|
||||
{case2Type, true},
|
||||
{case3Type, true},
|
||||
{case4Type, true},
|
||||
// Type.Name() starts with lower case and Type.PkgPath() is not empty.
|
||||
{case5Type, false},
|
||||
// Type.Name() starts with lower case and Type.PkgPath() is not empty.
|
||||
{case6Type, false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result := isExportedOrBuiltinType(testCase.t)
|
||||
|
||||
if result != testCase.expectedResult {
|
||||
t.Fatalf("case %v: expected: %v, got: %v\n", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetMethodMap(t *testing.T) {
|
||||
var a Arith
|
||||
case1Type := reflect.TypeOf(a)
|
||||
|
||||
var aptr *Arith
|
||||
case2Type := reflect.TypeOf(aptr)
|
||||
|
||||
var m mytype
|
||||
case3Type := reflect.TypeOf(m)
|
||||
|
||||
var mptr *mytype
|
||||
case4Type := reflect.TypeOf(mptr)
|
||||
|
||||
testCases := []struct {
|
||||
t reflect.Type
|
||||
expectedResult int
|
||||
}{
|
||||
// No methods exported.
|
||||
{case1Type, 0},
|
||||
// Multiply and Divide methods are exported.
|
||||
{case2Type, 2},
|
||||
// Foo method is exported.
|
||||
{case3Type, 1},
|
||||
// Foo method is exported.
|
||||
{case4Type, 1},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
m := getMethodMap(testCase.t)
|
||||
result := len(m)
|
||||
|
||||
if result != testCase.expectedResult {
|
||||
t.Fatalf("case %v: expected: %v, got: %v\n", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerRegisterName(t *testing.T) {
|
||||
case1Receiver := &Arith{}
|
||||
var case2Receiver mytype
|
||||
var case3Receiver *Arith
|
||||
i := 0
|
||||
var case4Receiver = &i
|
||||
var case5Receiver Arith
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
receiver interface{}
|
||||
expectErr bool
|
||||
}{
|
||||
{"Arith", case1Receiver, false},
|
||||
{"arith", case1Receiver, false},
|
||||
{"Arith", case2Receiver, false},
|
||||
// nil receiver error.
|
||||
{"Arith", nil, true},
|
||||
// nil receiver error.
|
||||
{"Arith", case3Receiver, true},
|
||||
// rpc.Register: type Arith has no exported methods of suitable type error.
|
||||
{"Arith", case4Receiver, true},
|
||||
// rpc.Register: type Arith has no exported methods of suitable type (hint: pass a pointer to value of that type) error.
|
||||
{"Arith", case5Receiver, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := NewServer().RegisterName(testCase.name, testCase.receiver)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v\n", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerCall(t *testing.T) {
|
||||
server1 := NewServer()
|
||||
if err := server1.RegisterName("Arith", &Arith{}); err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
server2 := NewServer()
|
||||
if err := server2.RegisterName("arith", &Arith{}); err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
case1ArgBytes, err := gobEncode(&Args{7, 8})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
reply := 7 * 8
|
||||
case1ExpectedResult, err := gobEncode(&reply)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
case2ArgBytes, err := gobEncode(&Args{})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
server *Server
|
||||
serviceMethod string
|
||||
argBytes []byte
|
||||
expectedResult []byte
|
||||
expectErr bool
|
||||
}{
|
||||
{server1, "Arith.Multiply", case1ArgBytes, case1ExpectedResult, false},
|
||||
{server2, "arith.Multiply", case1ArgBytes, case1ExpectedResult, false},
|
||||
// invalid service/method request ill-formed error.
|
||||
{server1, "Multiply", nil, nil, true},
|
||||
// can't find service error.
|
||||
{server1, "arith.Multiply", nil, nil, true},
|
||||
// can't find method error.
|
||||
{server1, "Arith.Add", nil, nil, true},
|
||||
// gob decode error.
|
||||
{server1, "Arith.Multiply", []byte{10}, nil, true},
|
||||
// authentication error.
|
||||
{server1, "Arith.Multiply", case2ArgBytes, nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result, err := testCase.server.call(testCase.serviceMethod, testCase.argBytes)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v\n", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %v, got: %v\n", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServerServeHTTP(t *testing.T) {
|
||||
server1 := NewServer()
|
||||
if err := server1.RegisterName("Arith", &Arith{}); err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
argBytes, err := gobEncode(&Args{7, 8})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
requestBodyData, err := gobEncode(CallRequest{Method: "Arith.Multiply", ArgBytes: argBytes})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case1Request, err := http.NewRequest("POST", "http://localhost:12345/", bytes.NewReader(requestBodyData))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
reply := 7 * 8
|
||||
replyBytes, err := gobEncode(&reply)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case1Result, err := gobEncode(CallResponse{ReplyBytes: replyBytes})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
case2Request, err := http.NewRequest("GET", "http://localhost:12345/", bytes.NewReader([]byte{}))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
case3Request, err := http.NewRequest("POST", "http://localhost:12345/", bytes.NewReader([]byte{10, 20}))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
requestBodyData, err = gobEncode(CallRequest{Method: "Arith.Add", ArgBytes: argBytes})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case4Request, err := http.NewRequest("POST", "http://localhost:12345/", bytes.NewReader(requestBodyData))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case4Result, err := gobEncode(CallResponse{Error: "can't find method Add"})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
server *Server
|
||||
httpRequest *http.Request
|
||||
expectedCode int
|
||||
expectedResult []byte
|
||||
}{
|
||||
{server1, case1Request, http.StatusOK, case1Result},
|
||||
{server1, case2Request, http.StatusMethodNotAllowed, nil},
|
||||
{server1, case3Request, http.StatusBadRequest, nil},
|
||||
{server1, case4Request, http.StatusOK, case4Result},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
writer := httptest.NewRecorder()
|
||||
testCase.server.ServeHTTP(writer, testCase.httpRequest)
|
||||
if writer.Code != testCase.expectedCode {
|
||||
t.Fatalf("case %v: code: expected: %v, got: %v\n", i+1, testCase.expectedCode, writer.Code)
|
||||
}
|
||||
|
||||
if testCase.expectedCode == http.StatusOK {
|
||||
result := writer.Body.Bytes()
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %v, got: %v\n", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,404 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"crypto/tls"
|
||||
"errors"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
func TestAuthArgsAuthenticate(t *testing.T) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
case1Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
|
||||
case2Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow().Add(15 * time.Minute),
|
||||
}
|
||||
|
||||
case3Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow().Add(-16 * time.Minute),
|
||||
}
|
||||
|
||||
case4Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: RPCVersion{99, 99, 99},
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
|
||||
case5Args := AuthArgs{
|
||||
Token: "invalid-token",
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
args AuthArgs
|
||||
expectErr bool
|
||||
}{
|
||||
{case1Args, false},
|
||||
{case2Args, false},
|
||||
{case3Args, true},
|
||||
{case4Args, true},
|
||||
{case5Args, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := testCase.args.Authenticate()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestAuthArgsSetAuthArgs(t *testing.T) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
case1Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow(),
|
||||
}
|
||||
|
||||
case2Args := AuthArgs{
|
||||
Token: newAuthToken(),
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
RequestTime: UTCNow().Add(15 * time.Minute),
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
args *AuthArgs
|
||||
authArgs AuthArgs
|
||||
expectedResult *AuthArgs
|
||||
}{
|
||||
{&AuthArgs{}, case1Args, &case1Args},
|
||||
{&case2Args, case1Args, &case1Args},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
testCase.args.SetAuthArgs(testCase.authArgs)
|
||||
result := testCase.args
|
||||
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestRPCClientArgsValidate(t *testing.T) {
|
||||
case1URL, err := xnet.ParseURL("http://localhost:12345/rpc")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case1Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case1URL,
|
||||
TLSConfig: nil,
|
||||
}
|
||||
|
||||
case2URL, err := xnet.ParseURL("https://localhost:12345/rpc")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case2Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case1URL,
|
||||
TLSConfig: &tls.Config{},
|
||||
}
|
||||
|
||||
case3Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: nil,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case1URL,
|
||||
TLSConfig: &tls.Config{},
|
||||
}
|
||||
|
||||
case4Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceURL: case1URL,
|
||||
TLSConfig: &tls.Config{},
|
||||
}
|
||||
|
||||
case5URL, err := xnet.ParseURL("ftp://localhost:12345/rpc")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case5Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case5URL,
|
||||
TLSConfig: &tls.Config{},
|
||||
}
|
||||
|
||||
case6URL, err := xnet.ParseURL("http://localhost:12345/rpc?location")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
case6Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case6URL,
|
||||
TLSConfig: &tls.Config{},
|
||||
}
|
||||
|
||||
case7Args := RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: case2URL,
|
||||
TLSConfig: nil,
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
args RPCClientArgs
|
||||
expectErr bool
|
||||
}{
|
||||
{case1Args, false},
|
||||
{case2Args, false},
|
||||
// NewAuthTokenFunc must not be empty error.
|
||||
{case3Args, true},
|
||||
// ServiceName must not be empty.
|
||||
{case4Args, true},
|
||||
// unknown RPC URL error.
|
||||
{case5Args, true},
|
||||
// unknown RPC URL error.
|
||||
{case6Args, true},
|
||||
// tls configuration must not be empty for https url error.
|
||||
{case7Args, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := testCase.args.validate()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type Args struct {
|
||||
AuthArgs
|
||||
A, B int
|
||||
}
|
||||
|
||||
type Quotient struct {
|
||||
Quo, Rem int
|
||||
}
|
||||
|
||||
type Arith struct{}
|
||||
|
||||
func (t *Arith) Multiply(args *Args, reply *int) error {
|
||||
*reply = args.A * args.B
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Arith) Divide(args *Args, quo *Quotient) error {
|
||||
if args.B == 0 {
|
||||
return errors.New("divide by zero")
|
||||
}
|
||||
quo.Quo = args.A / args.B
|
||||
quo.Rem = args.A % args.B
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestRPCClientCall(t *testing.T) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
rpcServer := xrpc.NewServer()
|
||||
if err := rpcServer.RegisterName("Arith", &Arith{}); err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
httpServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
rpcServer.ServeHTTP(w, r)
|
||||
}))
|
||||
defer httpServer.Close()
|
||||
|
||||
url, err := xnet.ParseURL(httpServer.URL)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
rpcClient, err := NewRPCClient(RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: url,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
var case1Result int
|
||||
case1ExpectedResult := 19 * 8
|
||||
|
||||
testCases := []struct {
|
||||
serviceMethod string
|
||||
args interface {
|
||||
SetAuthArgs(args AuthArgs)
|
||||
}
|
||||
result interface{}
|
||||
changeConfig bool
|
||||
expectedResult interface{}
|
||||
expectErr bool
|
||||
}{
|
||||
{"Arith.Multiply", &Args{A: 19, B: 8}, &case1Result, false, &case1ExpectedResult, false},
|
||||
{"Arith.Divide", &Args{A: 19, B: 8}, &Quotient{}, false, &Quotient{2, 3}, false},
|
||||
{"Arith.Multiply", &Args{A: 19, B: 8}, &case1Result, true, &case1ExpectedResult, false},
|
||||
{"Arith.Divide", &Args{A: 19, B: 8}, &Quotient{}, true, &Quotient{2, 3}, false},
|
||||
{"Arith.Divide", &Args{A: 19, B: 0}, &Quotient{}, false, nil, true},
|
||||
{"Arith.Divide", &Args{A: 19, B: 8}, &case1Result, false, nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
if testCase.changeConfig {
|
||||
globalServerConfig = newServerConfig()
|
||||
}
|
||||
|
||||
err := rpcClient.Call(testCase.serviceMethod, testCase.args, testCase.result)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if !reflect.DeepEqual(testCase.result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectedResult, testCase.result)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestRPCClientClose(t *testing.T) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
url, err := xnet.ParseURL("http://localhost:12345")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
rpcClient, err := NewRPCClient(RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: url,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
rpcClient *RPCClient
|
||||
expectErr bool
|
||||
}{
|
||||
{rpcClient, false},
|
||||
// Double close.
|
||||
{rpcClient, false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := testCase.rpcClient.Close()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestRPCClientServiceURL(t *testing.T) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
url, err := xnet.ParseURL("http://localhost:12345")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
rpcClient, err := NewRPCClient(RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: "Arith",
|
||||
ServiceURL: url,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
case1Result, err := xnet.ParseURL("http://localhost:12345")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
testCases := []struct {
|
||||
rpcClient *RPCClient
|
||||
expectedResult *xnet.URL
|
||||
}{
|
||||
{rpcClient, case1Result},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result := testCase.rpcClient.ServiceURL()
|
||||
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: expected: %v, got: %v", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -282,8 +282,6 @@ func serverMain(ctx *cli.Context) {
|
|||
}
|
||||
|
||||
globalHTTPServer = xhttp.NewServer([]string{globalMinioAddr}, handler, getCert)
|
||||
globalHTTPServer.ReadTimeout = globalConnReadTimeout
|
||||
globalHTTPServer.WriteTimeout = globalConnWriteTimeout
|
||||
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
||||
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
||||
go func() {
|
||||
|
|
|
@ -18,33 +18,33 @@ package cmd
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"io"
|
||||
"net"
|
||||
"net/rpc"
|
||||
"net/url"
|
||||
"path"
|
||||
"strings"
|
||||
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
type networkStorage struct {
|
||||
rpcClient *AuthRPCClient
|
||||
connected bool
|
||||
}
|
||||
|
||||
const (
|
||||
storageRPCPath = "/storage"
|
||||
)
|
||||
|
||||
func isErrorNetworkDisconnect(err error) bool {
|
||||
func isNetworkDisconnectError(err error) bool {
|
||||
if err == nil {
|
||||
return false
|
||||
}
|
||||
if _, ok := err.(*net.OpError); ok {
|
||||
return true
|
||||
|
||||
if uerr, isURLError := err.(*url.Error); isURLError {
|
||||
if uerr.Timeout() {
|
||||
return true
|
||||
}
|
||||
|
||||
err = uerr.Err
|
||||
}
|
||||
if err == rpc.ErrShutdown {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
|
||||
_, isNetOpError := err.(*net.OpError)
|
||||
return isNetOpError
|
||||
}
|
||||
|
||||
// Converts rpc.ServerError to underlying error. This function is
|
||||
|
@ -55,7 +55,7 @@ func toStorageErr(err error) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
if isErrorNetworkDisconnect(err) {
|
||||
if isNetworkDisconnectError(err) {
|
||||
return errDiskNotFound
|
||||
}
|
||||
|
||||
|
@ -100,162 +100,128 @@ func toStorageErr(err error) error {
|
|||
return err
|
||||
}
|
||||
|
||||
// Initialize new storage rpc client.
|
||||
func newStorageRPC(endpoint Endpoint) StorageAPI {
|
||||
// Dial minio rpc storage http path.
|
||||
rpcPath := path.Join(minioReservedBucketPath, storageRPCPath, endpoint.Path)
|
||||
serverCred := globalServerConfig.GetCredential()
|
||||
|
||||
disk := &networkStorage{
|
||||
rpcClient: newAuthRPCClient(authConfig{
|
||||
accessKey: serverCred.AccessKey,
|
||||
secretKey: serverCred.SecretKey,
|
||||
serverAddr: endpoint.Host,
|
||||
serviceEndpoint: rpcPath,
|
||||
secureConn: globalIsSSL,
|
||||
serviceName: "Storage",
|
||||
disableReconnect: true,
|
||||
}),
|
||||
}
|
||||
// Attempt a remote login.
|
||||
disk.connected = disk.rpcClient.Login() == nil
|
||||
return disk
|
||||
// StorageRPCClient - storage RPC client.
|
||||
type StorageRPCClient struct {
|
||||
*RPCClient
|
||||
connected bool
|
||||
}
|
||||
|
||||
// Stringer provides a canonicalized representation of network device.
|
||||
func (n *networkStorage) String() string {
|
||||
// Remove the storage RPC path prefix, internal paths are meaningless.
|
||||
serviceEndpoint := strings.TrimPrefix(n.rpcClient.ServiceEndpoint(),
|
||||
path.Join(minioReservedBucketPath, storageRPCPath))
|
||||
// Check for the transport layer being used.
|
||||
scheme := "http"
|
||||
if n.rpcClient.config.secureConn {
|
||||
scheme = "https"
|
||||
}
|
||||
// Finally construct the disk endpoint in http://<server>/<path> form.
|
||||
return scheme + "://" + n.rpcClient.ServerAddr() + path.Join("/", serviceEndpoint)
|
||||
func (client *StorageRPCClient) String() string {
|
||||
url := client.ServiceURL()
|
||||
// Remove the storage RPC path prefix, internal paths are meaningless. why?
|
||||
url.Path = strings.TrimPrefix(url.Path, storageServicePath)
|
||||
return url.String()
|
||||
}
|
||||
|
||||
func (n *networkStorage) Close() error {
|
||||
n.connected = false
|
||||
return toStorageErr(n.rpcClient.Close())
|
||||
// Close - closes underneath RPC client.
|
||||
func (client *StorageRPCClient) Close() error {
|
||||
client.connected = false
|
||||
return toStorageErr(client.RPCClient.Close())
|
||||
}
|
||||
|
||||
func (n *networkStorage) IsOnline() bool {
|
||||
return n.connected
|
||||
// IsOnline - returns whether RPC client failed to connect or not.
|
||||
func (client *StorageRPCClient) IsOnline() bool {
|
||||
return client.connected
|
||||
}
|
||||
|
||||
func (n *networkStorage) call(handler string, args interface {
|
||||
SetAuthToken(string)
|
||||
SetRPCAPIVersion(semVersion)
|
||||
func (client *StorageRPCClient) call(handler string, args interface {
|
||||
SetAuthArgs(args AuthArgs)
|
||||
}, reply interface{}) error {
|
||||
if !n.connected {
|
||||
if !client.connected {
|
||||
return errDiskNotFound
|
||||
}
|
||||
if err := n.rpcClient.Call(handler, args, reply); err != nil {
|
||||
if isErrorNetworkDisconnect(err) {
|
||||
n.connected = false
|
||||
}
|
||||
return toStorageErr(err)
|
||||
|
||||
err := client.Call(handler, args, reply)
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
|
||||
if isNetworkDisconnectError(err) {
|
||||
client.connected = false
|
||||
}
|
||||
|
||||
return toStorageErr(err)
|
||||
}
|
||||
|
||||
// DiskInfo - fetch disk information for a remote disk.
|
||||
func (n *networkStorage) DiskInfo() (info DiskInfo, err error) {
|
||||
args := AuthRPCArgs{}
|
||||
if err = n.call("Storage.DiskInfoHandler", &args, &info); err != nil {
|
||||
return DiskInfo{}, err
|
||||
}
|
||||
return info, nil
|
||||
func (client *StorageRPCClient) DiskInfo() (info DiskInfo, err error) {
|
||||
err = client.call(storageServiceName+".DiskInfo", &AuthArgs{}, &info)
|
||||
return info, err
|
||||
}
|
||||
|
||||
// MakeVol - create a volume on a remote disk.
|
||||
func (n *networkStorage) MakeVol(volume string) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
args := GenericVolArgs{Vol: volume}
|
||||
return n.call("Storage.MakeVolHandler", &args, &reply)
|
||||
func (client *StorageRPCClient) MakeVol(volume string) (err error) {
|
||||
return client.call(storageServiceName+".MakeVol", &VolArgs{Vol: volume}, &VoidReply{})
|
||||
}
|
||||
|
||||
// ListVols - List all volumes on a remote disk.
|
||||
func (n *networkStorage) ListVols() (vols []VolInfo, err error) {
|
||||
ListVols := ListVolsReply{}
|
||||
if err = n.call("Storage.ListVolsHandler", &AuthRPCArgs{}, &ListVols); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return ListVols.Vols, nil
|
||||
func (client *StorageRPCClient) ListVols() ([]VolInfo, error) {
|
||||
var reply []VolInfo
|
||||
err := client.call(storageServiceName+".ListVols", &AuthArgs{}, &reply)
|
||||
return reply, err
|
||||
}
|
||||
|
||||
// StatVol - get volume info over the network.
|
||||
func (n *networkStorage) StatVol(volume string) (volInfo VolInfo, err error) {
|
||||
args := GenericVolArgs{Vol: volume}
|
||||
if err = n.call("Storage.StatVolHandler", &args, &volInfo); err != nil {
|
||||
return VolInfo{}, err
|
||||
}
|
||||
return volInfo, nil
|
||||
func (client *StorageRPCClient) StatVol(volume string) (volInfo VolInfo, err error) {
|
||||
err = client.call(storageServiceName+".StatVol", &VolArgs{Vol: volume}, &volInfo)
|
||||
return volInfo, err
|
||||
}
|
||||
|
||||
// DeleteVol - Deletes a volume over the network.
|
||||
func (n *networkStorage) DeleteVol(volume string) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
args := GenericVolArgs{Vol: volume}
|
||||
return n.call("Storage.DeleteVolHandler", &args, &reply)
|
||||
func (client *StorageRPCClient) DeleteVol(volume string) (err error) {
|
||||
return client.call(storageServiceName+".DeleteVol", &VolArgs{Vol: volume}, &VoidReply{})
|
||||
}
|
||||
|
||||
// File operations.
|
||||
|
||||
func (n *networkStorage) PrepareFile(volume, path string, length int64) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
return n.call("Storage.PrepareFileHandler", &PrepareFileArgs{
|
||||
// PrepareFile - calls PrepareFile RPC.
|
||||
func (client *StorageRPCClient) PrepareFile(volume, path string, length int64) (err error) {
|
||||
args := PrepareFileArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
Size: length,
|
||||
}, &reply)
|
||||
}
|
||||
reply := VoidReply{}
|
||||
|
||||
return client.call(storageServiceName+".PrepareFile", &args, &reply)
|
||||
}
|
||||
|
||||
// AppendFile - append file writes buffer to a remote network path.
|
||||
func (n *networkStorage) AppendFile(volume, path string, buffer []byte) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
return n.call("Storage.AppendFileHandler", &AppendFileArgs{
|
||||
func (client *StorageRPCClient) AppendFile(volume, path string, buffer []byte) (err error) {
|
||||
args := AppendFileArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
Buffer: buffer,
|
||||
}, &reply)
|
||||
}
|
||||
reply := VoidReply{}
|
||||
|
||||
return client.call(storageServiceName+".AppendFile", &args, &reply)
|
||||
}
|
||||
|
||||
// StatFile - get latest Stat information for a file at path.
|
||||
func (n *networkStorage) StatFile(volume, path string) (fileInfo FileInfo, err error) {
|
||||
if err = n.call("Storage.StatFileHandler", &StatFileArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
}, &fileInfo); err != nil {
|
||||
return FileInfo{}, err
|
||||
}
|
||||
return fileInfo, nil
|
||||
func (client *StorageRPCClient) StatFile(volume, path string) (fileInfo FileInfo, err error) {
|
||||
err = client.call(storageServiceName+".StatFile", &StatFileArgs{Vol: volume, Path: path}, &fileInfo)
|
||||
return fileInfo, err
|
||||
}
|
||||
|
||||
// ReadAll - reads entire contents of the file at path until EOF, returns the
|
||||
// contents in a byte slice. Returns buf == nil if err != nil.
|
||||
// This API is meant to be used on files which have small memory footprint, do
|
||||
// not use this on large files as it would cause server to crash.
|
||||
func (n *networkStorage) ReadAll(volume, path string) (buf []byte, err error) {
|
||||
if err = n.call("Storage.ReadAllHandler", &ReadAllArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
}, &buf); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return buf, nil
|
||||
func (client *StorageRPCClient) ReadAll(volume, path string) (buf []byte, err error) {
|
||||
err = client.call(storageServiceName+".ReadAll", &ReadAllArgs{Vol: volume, Path: path}, &buf)
|
||||
return buf, err
|
||||
}
|
||||
|
||||
// ReadFile - reads a file at remote path and fills the buffer.
|
||||
func (n *networkStorage) ReadFile(volume string, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (m int64, err error) {
|
||||
func (client *StorageRPCClient) ReadFile(volume string, path string, offset int64, buffer []byte, verifier *BitrotVerifier) (m int64, err error) {
|
||||
// Recover from any panic and return error.
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
// Recover any panic from allocation, and return error.
|
||||
err = bytes.ErrTooLarge
|
||||
}
|
||||
}() // Do not crash the server.
|
||||
}()
|
||||
|
||||
args := ReadFileArgs{
|
||||
Vol: volume,
|
||||
|
@ -269,46 +235,90 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
|
|||
args.ExpectedHash = verifier.sum
|
||||
args.Verified = verifier.IsVerified()
|
||||
}
|
||||
var reply []byte
|
||||
|
||||
var result []byte
|
||||
err = n.call("Storage.ReadFileHandler", &args, &result)
|
||||
err = client.call(storageServiceName+".ReadFile", &args, &reply)
|
||||
|
||||
// Copy results to buffer.
|
||||
copy(buffer, result)
|
||||
// Copy reply to buffer.
|
||||
copy(buffer, reply)
|
||||
|
||||
// Return length of result, err if any.
|
||||
return int64(len(result)), err
|
||||
return int64(len(reply)), err
|
||||
}
|
||||
|
||||
// ListDir - list all entries at prefix.
|
||||
func (n *networkStorage) ListDir(volume, path string, count int) (entries []string, err error) {
|
||||
if err = n.call("Storage.ListDirHandler", &ListDirArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
Count: count,
|
||||
}, &entries); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Return successfully unmarshalled results.
|
||||
return entries, nil
|
||||
func (client *StorageRPCClient) ListDir(volume, path string, count int) (entries []string, err error) {
|
||||
err = client.call(storageServiceName+".ListDir", &ListDirArgs{Vol: volume, Path: path, Count: count}, &entries)
|
||||
return entries, err
|
||||
}
|
||||
|
||||
// DeleteFile - Delete a file at path.
|
||||
func (n *networkStorage) DeleteFile(volume, path string) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
return n.call("Storage.DeleteFileHandler", &DeleteFileArgs{
|
||||
func (client *StorageRPCClient) DeleteFile(volume, path string) (err error) {
|
||||
args := DeleteFileArgs{
|
||||
Vol: volume,
|
||||
Path: path,
|
||||
}, &reply)
|
||||
}
|
||||
reply := VoidReply{}
|
||||
|
||||
return client.call(storageServiceName+".DeleteFile", &args, &reply)
|
||||
}
|
||||
|
||||
// RenameFile - rename a remote file from source to destination.
|
||||
func (n *networkStorage) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
|
||||
reply := AuthRPCReply{}
|
||||
return n.call("Storage.RenameFileHandler", &RenameFileArgs{
|
||||
func (client *StorageRPCClient) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
|
||||
args := RenameFileArgs{
|
||||
SrcVol: srcVolume,
|
||||
SrcPath: srcPath,
|
||||
DstVol: dstVolume,
|
||||
DstPath: dstPath,
|
||||
}, &reply)
|
||||
}
|
||||
reply := VoidReply{}
|
||||
|
||||
return client.call(storageServiceName+".RenameFile", &args, &reply)
|
||||
}
|
||||
|
||||
// NewStorageRPCClient - returns new storage RPC client.
|
||||
func NewStorageRPCClient(host *xnet.Host, endpointPath string) (*StorageRPCClient, error) {
|
||||
scheme := "http"
|
||||
if globalIsSSL {
|
||||
scheme = "https"
|
||||
}
|
||||
|
||||
serviceURL := &xnet.URL{
|
||||
Scheme: scheme,
|
||||
Host: host.String(),
|
||||
Path: path.Join(storageServicePath, endpointPath),
|
||||
}
|
||||
|
||||
var tlsConfig *tls.Config
|
||||
if globalIsSSL {
|
||||
tlsConfig = &tls.Config{
|
||||
ServerName: host.Name,
|
||||
RootCAs: globalRootCAs,
|
||||
}
|
||||
}
|
||||
|
||||
rpcClient, err := NewRPCClient(
|
||||
RPCClientArgs{
|
||||
NewAuthTokenFunc: newAuthToken,
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
ServiceName: storageServiceName,
|
||||
ServiceURL: serviceURL,
|
||||
TLSConfig: tlsConfig,
|
||||
},
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &StorageRPCClient{RPCClient: rpcClient}, nil
|
||||
}
|
||||
|
||||
// Initialize new storage rpc client.
|
||||
func newStorageRPC(endpoint Endpoint) *StorageRPCClient {
|
||||
host, err := xnet.ParseHost(endpoint.Host)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
rpcClient, err := NewStorageRPCClient(host, endpoint.Path)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
rpcClient.connected = rpcClient.Call(storageServiceName+".Connect", &AuthArgs{}, &VoidReply{}) == nil
|
||||
return rpcClient
|
||||
}
|
||||
|
|
|
@ -1,471 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"net/rpc"
|
||||
"runtime"
|
||||
"testing"
|
||||
|
||||
"golang.org/x/crypto/blake2b"
|
||||
)
|
||||
|
||||
// Tests the construction of canonical string by the
|
||||
// Stringer method for StorageAPI.
|
||||
func TestStorageCanonicalStrings(t *testing.T) {
|
||||
testCases := []struct {
|
||||
storageAPI StorageAPI
|
||||
canonicalPath string
|
||||
}{
|
||||
// Canonicalized name as unix path.
|
||||
{
|
||||
storageAPI: &posix{
|
||||
diskPath: "/tmp",
|
||||
},
|
||||
canonicalPath: "/tmp",
|
||||
},
|
||||
// Canonicalized name as windows path.
|
||||
{
|
||||
storageAPI: &posix{
|
||||
diskPath: "C:/tmp",
|
||||
},
|
||||
canonicalPath: "C:/tmp",
|
||||
},
|
||||
// Canonicalized name as unix path.
|
||||
{
|
||||
storageAPI: &networkStorage{
|
||||
rpcClient: newAuthRPCClient(authConfig{
|
||||
accessKey: "",
|
||||
secretKey: "",
|
||||
serverAddr: "localhost:9000",
|
||||
serviceEndpoint: "/tmp",
|
||||
secureConn: false,
|
||||
serviceName: "Storage",
|
||||
disableReconnect: true,
|
||||
}),
|
||||
},
|
||||
canonicalPath: "http://localhost:9000/tmp",
|
||||
},
|
||||
// Canonicalized name as non TLS.
|
||||
{
|
||||
storageAPI: &networkStorage{
|
||||
rpcClient: newAuthRPCClient(authConfig{
|
||||
accessKey: "",
|
||||
secretKey: "",
|
||||
serverAddr: "localhost:9000",
|
||||
serviceEndpoint: "C:/tmp",
|
||||
secureConn: false,
|
||||
serviceName: "Storage",
|
||||
disableReconnect: true,
|
||||
}),
|
||||
},
|
||||
canonicalPath: "http://localhost:9000/C:/tmp",
|
||||
},
|
||||
// Canonicalized name as TLS.
|
||||
{
|
||||
storageAPI: &networkStorage{
|
||||
rpcClient: newAuthRPCClient(authConfig{
|
||||
accessKey: "",
|
||||
secretKey: "",
|
||||
serverAddr: "localhost:9000",
|
||||
serviceEndpoint: "C:/tmp",
|
||||
secureConn: true,
|
||||
serviceName: "Storage",
|
||||
disableReconnect: true,
|
||||
}),
|
||||
},
|
||||
canonicalPath: "https://localhost:9000/C:/tmp",
|
||||
},
|
||||
}
|
||||
|
||||
// Validate all the test cases.
|
||||
for i, testCase := range testCases {
|
||||
p := testCase.storageAPI
|
||||
if p.String() != testCase.canonicalPath {
|
||||
t.Errorf("Test %d: Expected %s, got %s", i+1, testCase.canonicalPath, p.String())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Tests storage error transformation.
|
||||
func TestStorageErr(t *testing.T) {
|
||||
unknownErr := errors.New("Unknown error")
|
||||
testCases := []struct {
|
||||
expectedErr error
|
||||
err error
|
||||
}{
|
||||
{
|
||||
expectedErr: nil,
|
||||
err: nil,
|
||||
},
|
||||
{
|
||||
expectedErr: io.EOF,
|
||||
err: fmt.Errorf("%s", io.EOF.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: io.ErrUnexpectedEOF,
|
||||
err: fmt.Errorf("%s", io.ErrUnexpectedEOF.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errDiskNotFound,
|
||||
err: &net.OpError{},
|
||||
},
|
||||
{
|
||||
expectedErr: errDiskNotFound,
|
||||
err: rpc.ErrShutdown,
|
||||
},
|
||||
{
|
||||
expectedErr: errUnexpected,
|
||||
err: fmt.Errorf("%s", errUnexpected.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errDiskFull,
|
||||
err: fmt.Errorf("%s", errDiskFull.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errVolumeNotFound,
|
||||
err: fmt.Errorf("%s", errVolumeNotFound.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errVolumeExists,
|
||||
err: fmt.Errorf("%s", errVolumeExists.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errFileNotFound,
|
||||
err: fmt.Errorf("%s", errFileNotFound.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errFileAccessDenied,
|
||||
err: fmt.Errorf("%s", errFileAccessDenied.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errIsNotRegular,
|
||||
err: fmt.Errorf("%s", errIsNotRegular.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errVolumeNotEmpty,
|
||||
err: fmt.Errorf("%s", errVolumeNotEmpty.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errVolumeAccessDenied,
|
||||
err: fmt.Errorf("%s", errVolumeAccessDenied.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errCorruptedFormat,
|
||||
err: fmt.Errorf("%s", errCorruptedFormat.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errUnformattedDisk,
|
||||
err: fmt.Errorf("%s", errUnformattedDisk.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errFileNameTooLong,
|
||||
err: fmt.Errorf("%s", errFileNameTooLong.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errInvalidAccessKeyID,
|
||||
err: fmt.Errorf("%s", errInvalidAccessKeyID.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errAuthentication,
|
||||
err: fmt.Errorf("%s", errAuthentication.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errRPCAPIVersionUnsupported,
|
||||
err: fmt.Errorf("%s", errRPCAPIVersionUnsupported.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: errServerTimeMismatch,
|
||||
err: fmt.Errorf("%s", errServerTimeMismatch.Error()),
|
||||
},
|
||||
{
|
||||
expectedErr: unknownErr,
|
||||
err: unknownErr,
|
||||
},
|
||||
}
|
||||
for i, testCase := range testCases {
|
||||
resultErr := toStorageErr(testCase.err)
|
||||
if testCase.expectedErr != resultErr {
|
||||
t.Errorf("Test %d: Expected %s, got %s", i+1, testCase.expectedErr, resultErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// API suite container common to both FS and XL.
|
||||
type TestRPCStorageSuite struct {
|
||||
serverType string
|
||||
testServer TestServer
|
||||
remoteDisks []StorageAPI
|
||||
}
|
||||
|
||||
// Setting up the test suite.
|
||||
// Starting the Test server with temporary FS backend.
|
||||
func (s *TestRPCStorageSuite) SetUpSuite(t *testing.T) {
|
||||
s.testServer = StartTestStorageRPCServer(t, s.serverType, 1)
|
||||
listenAddress := s.testServer.Server.Listener.Addr().String()
|
||||
|
||||
for _, ep := range s.testServer.Disks {
|
||||
// Eventhough s.testServer.Disks is EndpointList, we would need a URLEndpointType here.
|
||||
endpoint := ep
|
||||
if endpoint.Type() == PathEndpointType {
|
||||
endpoint.Scheme = "http"
|
||||
}
|
||||
endpoint.Host = listenAddress
|
||||
storageDisk := newStorageRPC(endpoint)
|
||||
s.remoteDisks = append(s.remoteDisks, storageDisk)
|
||||
}
|
||||
}
|
||||
|
||||
// No longer used with gocheck, but used in explicit teardown code in
|
||||
// each test function. Called implicitly by after all tests are run.
|
||||
func (s *TestRPCStorageSuite) TearDownSuite(t *testing.T) {
|
||||
s.testServer.Stop()
|
||||
}
|
||||
|
||||
func TestRPCStorageClient(t *testing.T) {
|
||||
// Setup code
|
||||
s := &TestRPCStorageSuite{serverType: "XL"}
|
||||
s.SetUpSuite(t)
|
||||
|
||||
// Run the test.
|
||||
s.testRPCStorageClient(t)
|
||||
|
||||
// Teardown code
|
||||
s.TearDownSuite(t)
|
||||
}
|
||||
|
||||
func (s *TestRPCStorageSuite) testRPCStorageClient(t *testing.T) {
|
||||
// TODO - Fix below tests to run on windows.
|
||||
if runtime.GOOS == globalWindowsOSName {
|
||||
return
|
||||
}
|
||||
s.testRPCStorageDisksInfo(t)
|
||||
s.testRPCStorageVolOps(t)
|
||||
s.testRPCStorageFileOps(t)
|
||||
s.testRPCStorageListDir(t)
|
||||
}
|
||||
|
||||
// Test storage disks info.
|
||||
func (s *TestRPCStorageSuite) testRPCStorageDisksInfo(t *testing.T) {
|
||||
for _, storageDisk := range s.remoteDisks {
|
||||
diskInfo, err := storageDisk.DiskInfo()
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DiskInfo", err)
|
||||
}
|
||||
if diskInfo.Total == 0 {
|
||||
t.Error("Invalid diskInfo total")
|
||||
}
|
||||
if storageDisk.String() == "" {
|
||||
t.Error("String representation of storageAPI should not be empty")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Test storage vol operations.
|
||||
func (s *TestRPCStorageSuite) testRPCStorageVolOps(t *testing.T) {
|
||||
for _, storageDisk := range s.remoteDisks {
|
||||
numVols := 0
|
||||
err := storageDisk.MakeVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate MakeVol", err)
|
||||
}
|
||||
numVols++
|
||||
volInfo, err := storageDisk.StatVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate StatVol", err)
|
||||
}
|
||||
if volInfo.Name != "myvol" {
|
||||
t.Errorf("Expected `myvol` found %s instead", volInfo.Name)
|
||||
}
|
||||
if volInfo.Created.IsZero() {
|
||||
t.Error("Expected created time to be non zero")
|
||||
}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
err = storageDisk.MakeVol(fmt.Sprintf("myvol-%d", i))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate MakeVol", err)
|
||||
}
|
||||
numVols++
|
||||
}
|
||||
vols, err := storageDisk.ListVols()
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate ListVol")
|
||||
}
|
||||
if len(vols) != numVols {
|
||||
t.Errorf("Expected %d volumes but found only %d", numVols, len(vols))
|
||||
}
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
err = storageDisk.DeleteVol(fmt.Sprintf("myvol-%d", i))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteVol", err)
|
||||
}
|
||||
}
|
||||
|
||||
err = storageDisk.DeleteVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteVol", err)
|
||||
}
|
||||
vols, err = storageDisk.ListVols()
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate ListVol")
|
||||
}
|
||||
if len(vols) > 0 {
|
||||
t.Errorf("Expected no volumes but found %d", len(vols))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Tests all file operations.
|
||||
func (s *TestRPCStorageSuite) testRPCStorageFileOps(t *testing.T) {
|
||||
for _, storageDisk := range s.remoteDisks {
|
||||
err := storageDisk.MakeVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate MakeVol", err)
|
||||
}
|
||||
err = storageDisk.PrepareFile("myvol", "file1", int64(len([]byte("Hello, world"))))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate AppendFile", err)
|
||||
}
|
||||
err = storageDisk.AppendFile("myvol", "file1", []byte("Hello, world"))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate AppendFile", err)
|
||||
}
|
||||
fi, err := storageDisk.StatFile("myvol", "file1")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate StatFile", err)
|
||||
}
|
||||
if fi.Name != "file1" {
|
||||
t.Errorf("Expected `file1` but got %s", fi.Name)
|
||||
}
|
||||
if fi.Volume != "myvol" {
|
||||
t.Errorf("Expected `myvol` but got %s", fi.Volume)
|
||||
}
|
||||
if fi.Size != 12 {
|
||||
t.Errorf("Expected 12 but got %d", fi.Size)
|
||||
}
|
||||
if !fi.Mode.IsRegular() {
|
||||
t.Error("Expected file to be regular found", fi.Mode)
|
||||
}
|
||||
if fi.ModTime.IsZero() {
|
||||
t.Error("Expected created time to be non zero")
|
||||
}
|
||||
buf, err := storageDisk.ReadAll("myvol", "file1")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate ReadAll", err)
|
||||
}
|
||||
if !bytes.Equal(buf, []byte("Hello, world")) {
|
||||
t.Errorf("Expected `Hello, world`, got %s", string(buf))
|
||||
}
|
||||
buf1 := make([]byte, 5)
|
||||
n, err := storageDisk.ReadFile("myvol", "file1", 4, buf1, nil)
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate ReadFile", err)
|
||||
}
|
||||
if n != 5 {
|
||||
t.Errorf("Expected `5`, got %d", n)
|
||||
}
|
||||
if !bytes.Equal(buf[4:9], buf1) {
|
||||
t.Errorf("Expected %s, got %s", string(buf[4:9]), string(buf1))
|
||||
}
|
||||
|
||||
blakeHash := func(b []byte) []byte {
|
||||
k := blake2b.Sum512(b)
|
||||
return k[:]
|
||||
}
|
||||
verifier := NewBitrotVerifier(BLAKE2b512, blakeHash(buf))
|
||||
buf2 := make([]byte, 2)
|
||||
n, err = storageDisk.ReadFile("myvol", "file1", 1, buf2, verifier)
|
||||
if err != nil {
|
||||
t.Error("Error in ReadFile with bitrot verification", err)
|
||||
}
|
||||
if n != 2 {
|
||||
t.Errorf("Expected `2`, got %d", n)
|
||||
}
|
||||
if !bytes.Equal(buf[1:3], buf2) {
|
||||
t.Errorf("Expected %s, got %s", string(buf[1:3]), string(buf2))
|
||||
}
|
||||
|
||||
err = storageDisk.RenameFile("myvol", "file1", "myvol", "file2")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate RenameFile", err)
|
||||
}
|
||||
err = storageDisk.DeleteFile("myvol", "file2")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteFile", err)
|
||||
}
|
||||
err = storageDisk.DeleteVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteVol", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Tests for ListDirHandler.
|
||||
func (s *TestRPCStorageSuite) testRPCStorageListDir(t *testing.T) {
|
||||
for _, storageDisk := range s.remoteDisks {
|
||||
err := storageDisk.MakeVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate MakeVol", err)
|
||||
}
|
||||
dirCount := 10
|
||||
for i := 0; i < dirCount; i++ {
|
||||
err = storageDisk.MakeVol(fmt.Sprintf("myvol/mydir-%d", i))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate MakeVol", err)
|
||||
}
|
||||
}
|
||||
dirs, err := storageDisk.ListDir("myvol", "", -1)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
if len(dirs) != dirCount {
|
||||
t.Errorf("Expected %d directories but found only %d", dirCount, len(dirs))
|
||||
}
|
||||
for i := 0; i < dirCount; i++ {
|
||||
err = storageDisk.DeleteVol(fmt.Sprintf("myvol/mydir-%d", i))
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteVol", err)
|
||||
}
|
||||
}
|
||||
dirs, err = storageDisk.ListDir("myvol", "", -1)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
if len(dirs) != 0 {
|
||||
t.Errorf("Expected no directories but found %d", dirCount)
|
||||
}
|
||||
|
||||
err = storageDisk.DeleteVol("myvol")
|
||||
if err != nil {
|
||||
t.Error("Unable to initiate DeleteVol", err)
|
||||
}
|
||||
vols, err := storageDisk.ListVols()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
if len(vols) != 0 {
|
||||
t.Errorf("Expected no volumes but found %d", dirCount)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,158 +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
|
||||
|
||||
// GenericVolArgs - generic volume args.
|
||||
type GenericVolArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
}
|
||||
|
||||
// ListVolsReply represents list of vols RPC reply.
|
||||
type ListVolsReply struct {
|
||||
// List of volumes stat information.
|
||||
Vols []VolInfo
|
||||
}
|
||||
|
||||
// ReadAllArgs represents read all RPC arguments.
|
||||
type ReadAllArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
}
|
||||
|
||||
// ReadFileArgs represents read file RPC arguments.
|
||||
type ReadFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
|
||||
// Starting offset to start reading into Buffer.
|
||||
Offset int64
|
||||
|
||||
// Data buffer read from the path at offset.
|
||||
Buffer []byte
|
||||
|
||||
// Algorithm used in bit-rot hash computation.
|
||||
Algo BitrotAlgorithm
|
||||
|
||||
// Stored hash value used to compare with computed value.
|
||||
ExpectedHash []byte
|
||||
|
||||
// Indicates whether the disk has already been verified
|
||||
Verified bool
|
||||
}
|
||||
|
||||
// PrepareFileArgs represents append file RPC arguments.
|
||||
type PrepareFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
|
||||
// Size of the file to be prepared
|
||||
Size int64
|
||||
}
|
||||
|
||||
// AppendFileArgs represents append file RPC arguments.
|
||||
type AppendFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
|
||||
// Data buffer to be saved at path.
|
||||
Buffer []byte
|
||||
}
|
||||
|
||||
// StatFileArgs represents stat file RPC arguments.
|
||||
type StatFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
}
|
||||
|
||||
// DeleteFileArgs represents delete file RPC arguments.
|
||||
type DeleteFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
}
|
||||
|
||||
// ListDirArgs represents list contents RPC arguments.
|
||||
type ListDirArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of the volume.
|
||||
Vol string
|
||||
|
||||
// Name of the path.
|
||||
Path string
|
||||
|
||||
// Number of wanted results
|
||||
Count int
|
||||
}
|
||||
|
||||
// RenameFileArgs represents rename file RPC arguments.
|
||||
type RenameFileArgs struct {
|
||||
// Authentication token generated by Login.
|
||||
AuthRPCArgs
|
||||
|
||||
// Name of source volume.
|
||||
SrcVol string
|
||||
|
||||
// Source path to be renamed.
|
||||
SrcPath string
|
||||
|
||||
// Name of destination volume.
|
||||
DstVol string
|
||||
|
||||
// Destination path of renamed file.
|
||||
DstPath string
|
||||
}
|
|
@ -20,225 +20,212 @@ import (
|
|||
"context"
|
||||
"io"
|
||||
"path"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
xrpc "github.com/minio/minio/cmd/rpc"
|
||||
)
|
||||
|
||||
// Storage server implements rpc primitives to facilitate exporting a
|
||||
// disk over a network.
|
||||
type storageServer struct {
|
||||
AuthRPCServer
|
||||
storage StorageAPI
|
||||
path string
|
||||
timestamp time.Time
|
||||
const storageServiceName = "Storage"
|
||||
const storageServiceSubPath = "/storage"
|
||||
|
||||
var storageServicePath = path.Join(minioReservedBucketPath, storageServiceSubPath)
|
||||
|
||||
// storageRPCReceiver - Storage RPC receiver for storage RPC server
|
||||
type storageRPCReceiver struct {
|
||||
local *posix
|
||||
}
|
||||
|
||||
// VolArgs - generic volume args.
|
||||
type VolArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
}
|
||||
|
||||
/// Storage operations handlers.
|
||||
|
||||
// DiskInfoHandler - disk info handler is rpc wrapper for DiskInfo operation.
|
||||
func (s *storageServer) DiskInfoHandler(args *AuthRPCArgs, reply *DiskInfo) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
// Connect - authenticates remote connection.
|
||||
func (receiver *storageRPCReceiver) Connect(args *AuthArgs, reply *VoidReply) (err error) {
|
||||
return args.Authenticate()
|
||||
}
|
||||
|
||||
info, err := s.storage.DiskInfo()
|
||||
*reply = info
|
||||
// DiskInfo - disk info handler is rpc wrapper for DiskInfo operation.
|
||||
func (receiver *storageRPCReceiver) DiskInfo(args *AuthArgs, reply *DiskInfo) (err error) {
|
||||
*reply, err = receiver.local.DiskInfo()
|
||||
return err
|
||||
}
|
||||
|
||||
/// Volume operations handlers.
|
||||
|
||||
// MakeVolHandler - make vol handler is rpc wrapper for MakeVol operation.
|
||||
func (s *storageServer) MakeVolHandler(args *GenericVolArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.MakeVol(args.Vol)
|
||||
// MakeVol - make vol handler is rpc wrapper for MakeVol operation.
|
||||
func (receiver *storageRPCReceiver) MakeVol(args *VolArgs, reply *VoidReply) error {
|
||||
return receiver.local.MakeVol(args.Vol)
|
||||
}
|
||||
|
||||
// ListVolsHandler - list vols handler is rpc wrapper for ListVols operation.
|
||||
func (s *storageServer) ListVolsHandler(args *AuthRPCArgs, reply *ListVolsReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
vols, err := s.storage.ListVols()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
reply.Vols = vols
|
||||
return nil
|
||||
// ListVols - list vols handler is rpc wrapper for ListVols operation.
|
||||
func (receiver *storageRPCReceiver) ListVols(args *AuthArgs, reply *[]VolInfo) (err error) {
|
||||
*reply, err = receiver.local.ListVols()
|
||||
return err
|
||||
}
|
||||
|
||||
// StatVolHandler - stat vol handler is a rpc wrapper for StatVol operation.
|
||||
func (s *storageServer) StatVolHandler(args *GenericVolArgs, reply *VolInfo) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
volInfo, err := s.storage.StatVol(args.Vol)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*reply = volInfo
|
||||
return nil
|
||||
// StatVol - stat vol handler is a rpc wrapper for StatVol operation.
|
||||
func (receiver *storageRPCReceiver) StatVol(args *VolArgs, reply *VolInfo) (err error) {
|
||||
*reply, err = receiver.local.StatVol(args.Vol)
|
||||
return err
|
||||
}
|
||||
|
||||
// DeleteVolHandler - delete vol handler is a rpc wrapper for
|
||||
// DeleteVol - delete vol handler is a rpc wrapper for
|
||||
// DeleteVol operation.
|
||||
func (s *storageServer) DeleteVolHandler(args *GenericVolArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.DeleteVol(args.Vol)
|
||||
func (receiver *storageRPCReceiver) DeleteVol(args *VolArgs, reply *VoidReply) error {
|
||||
return receiver.local.DeleteVol(args.Vol)
|
||||
}
|
||||
|
||||
/// File operations
|
||||
|
||||
// StatFileHandler - stat file handler is rpc wrapper to stat file.
|
||||
func (s *storageServer) StatFileHandler(args *StatFileArgs, reply *FileInfo) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
fileInfo, err := s.storage.StatFile(args.Vol, args.Path)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*reply = fileInfo
|
||||
return nil
|
||||
// StatFileArgs represents stat file RPC arguments.
|
||||
type StatFileArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
}
|
||||
|
||||
// ListDirHandler - list directory handler is rpc wrapper to list dir.
|
||||
func (s *storageServer) ListDirHandler(args *ListDirArgs, reply *[]string) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
entries, err := s.storage.ListDir(args.Vol, args.Path, args.Count)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*reply = entries
|
||||
return nil
|
||||
// StatFile - stat file handler is rpc wrapper to stat file.
|
||||
func (receiver *storageRPCReceiver) StatFile(args *StatFileArgs, reply *FileInfo) (err error) {
|
||||
*reply, err = receiver.local.StatFile(args.Vol, args.Path)
|
||||
return err
|
||||
}
|
||||
|
||||
// ReadAllHandler - read all handler is rpc wrapper to read all storage API.
|
||||
func (s *storageServer) ReadAllHandler(args *ReadFileArgs, reply *[]byte) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
buf, err := s.storage.ReadAll(args.Vol, args.Path)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
*reply = buf
|
||||
return nil
|
||||
// ListDirArgs represents list contents RPC arguments.
|
||||
type ListDirArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
Count int
|
||||
}
|
||||
|
||||
// ReadFileHandler - read file handler is rpc wrapper to read file.
|
||||
func (s *storageServer) ReadFileHandler(args *ReadFileArgs, reply *[]byte) (err error) {
|
||||
if err = args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
// ListDir - list directory handler is rpc wrapper to list dir.
|
||||
func (receiver *storageRPCReceiver) ListDir(args *ListDirArgs, reply *[]string) (err error) {
|
||||
*reply, err = receiver.local.ListDir(args.Vol, args.Path, args.Count)
|
||||
return err
|
||||
}
|
||||
|
||||
// ReadAllArgs represents read all RPC arguments.
|
||||
type ReadAllArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
}
|
||||
|
||||
// ReadAll - read all handler is rpc wrapper to read all storage API.
|
||||
func (receiver *storageRPCReceiver) ReadAll(args *ReadAllArgs, reply *[]byte) (err error) {
|
||||
*reply, err = receiver.local.ReadAll(args.Vol, args.Path)
|
||||
return err
|
||||
}
|
||||
|
||||
// ReadFileArgs represents read file RPC arguments.
|
||||
type ReadFileArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
Offset int64
|
||||
Buffer []byte
|
||||
Algo BitrotAlgorithm
|
||||
ExpectedHash []byte
|
||||
Verified bool
|
||||
}
|
||||
|
||||
// ReadFile - read file handler is rpc wrapper to read file.
|
||||
func (receiver *storageRPCReceiver) ReadFile(args *ReadFileArgs, reply *[]byte) error {
|
||||
var verifier *BitrotVerifier
|
||||
if !args.Verified {
|
||||
verifier = NewBitrotVerifier(args.Algo, args.ExpectedHash)
|
||||
}
|
||||
|
||||
var n int64
|
||||
n, err = s.storage.ReadFile(args.Vol, args.Path, args.Offset, args.Buffer, verifier)
|
||||
// Sending an error over the rpc layer, would cause unmarshalling to fail. In situations
|
||||
// when we have short read i.e `io.ErrUnexpectedEOF` treat it as good condition and copy
|
||||
// the buffer properly.
|
||||
n, err := receiver.local.ReadFile(args.Vol, args.Path, args.Offset, args.Buffer, verifier)
|
||||
// Ignore io.ErrEnexpectedEOF for short reads i.e. less content available than requested.
|
||||
if err == io.ErrUnexpectedEOF {
|
||||
// Reset to nil as good condition.
|
||||
err = nil
|
||||
}
|
||||
|
||||
*reply = args.Buffer[0:n]
|
||||
return err
|
||||
}
|
||||
|
||||
// PrepareFileHandler - prepare file handler is rpc wrapper to prepare file.
|
||||
func (s *storageServer) PrepareFileHandler(args *PrepareFileArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.PrepareFile(args.Vol, args.Path, args.Size)
|
||||
// PrepareFileArgs represents append file RPC arguments.
|
||||
type PrepareFileArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
Size int64
|
||||
}
|
||||
|
||||
// AppendFileHandler - append file handler is rpc wrapper to append file.
|
||||
func (s *storageServer) AppendFileHandler(args *AppendFileArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.AppendFile(args.Vol, args.Path, args.Buffer)
|
||||
// PrepareFile - prepare file handler is rpc wrapper to prepare file.
|
||||
func (receiver *storageRPCReceiver) PrepareFile(args *PrepareFileArgs, reply *VoidReply) error {
|
||||
return receiver.local.PrepareFile(args.Vol, args.Path, args.Size)
|
||||
}
|
||||
|
||||
// DeleteFileHandler - delete file handler is rpc wrapper to delete file.
|
||||
func (s *storageServer) DeleteFileHandler(args *DeleteFileArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.DeleteFile(args.Vol, args.Path)
|
||||
// AppendFileArgs represents append file RPC arguments.
|
||||
type AppendFileArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
Buffer []byte
|
||||
}
|
||||
|
||||
// RenameFileHandler - rename file handler is rpc wrapper to rename file.
|
||||
func (s *storageServer) RenameFileHandler(args *RenameFileArgs, reply *AuthRPCReply) error {
|
||||
if err := args.IsAuthenticated(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.storage.RenameFile(args.SrcVol, args.SrcPath, args.DstVol, args.DstPath)
|
||||
// AppendFile - append file handler is rpc wrapper to append file.
|
||||
func (receiver *storageRPCReceiver) AppendFile(args *AppendFileArgs, reply *VoidReply) error {
|
||||
return receiver.local.AppendFile(args.Vol, args.Path, args.Buffer)
|
||||
}
|
||||
|
||||
// Initialize new storage rpc.
|
||||
func newStorageRPCServer(endpoints EndpointList) (servers []*storageServer, err error) {
|
||||
for _, endpoint := range endpoints {
|
||||
if endpoint.IsLocal {
|
||||
storage, err := newPosix(endpoint.Path)
|
||||
if err != nil && err != errDiskNotFound {
|
||||
return nil, err
|
||||
}
|
||||
// DeleteFileArgs represents delete file RPC arguments.
|
||||
type DeleteFileArgs struct {
|
||||
AuthArgs
|
||||
Vol string
|
||||
Path string
|
||||
}
|
||||
|
||||
servers = append(servers, &storageServer{
|
||||
storage: storage,
|
||||
path: endpoint.Path,
|
||||
})
|
||||
}
|
||||
// DeleteFile - delete file handler is rpc wrapper to delete file.
|
||||
func (receiver *storageRPCReceiver) DeleteFile(args *DeleteFileArgs, reply *VoidReply) error {
|
||||
return receiver.local.DeleteFile(args.Vol, args.Path)
|
||||
}
|
||||
|
||||
// RenameFileArgs represents rename file RPC arguments.
|
||||
type RenameFileArgs struct {
|
||||
AuthArgs
|
||||
SrcVol string
|
||||
SrcPath string
|
||||
DstVol string
|
||||
DstPath string
|
||||
}
|
||||
|
||||
// RenameFile - rename file handler is rpc wrapper to rename file.
|
||||
func (receiver *storageRPCReceiver) RenameFile(args *RenameFileArgs, reply *VoidReply) error {
|
||||
return receiver.local.RenameFile(args.SrcVol, args.SrcPath, args.DstVol, args.DstPath)
|
||||
}
|
||||
|
||||
// NewStorageRPCServer - returns new storage RPC server.
|
||||
func NewStorageRPCServer(endpointPath string) (*xrpc.Server, error) {
|
||||
storage, err := newPosix(endpointPath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return servers, nil
|
||||
rpcServer := xrpc.NewServer()
|
||||
if err = rpcServer.RegisterName(storageServiceName, &storageRPCReceiver{storage}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return rpcServer, nil
|
||||
}
|
||||
|
||||
// registerStorageRPCRouter - register storage rpc router.
|
||||
func registerStorageRPCRouters(router *mux.Router, endpoints EndpointList) error {
|
||||
// Initialize storage rpc servers for every disk that is hosted on this node.
|
||||
storageRPCs, err := newStorageRPCServer(endpoints)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Create a unique route for each disk exported from this node.
|
||||
for _, stServer := range storageRPCs {
|
||||
storageRPCServer := newRPCServer()
|
||||
err = storageRPCServer.RegisterName("Storage", stServer)
|
||||
if err != nil {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return err
|
||||
func registerStorageRPCRouters(router *mux.Router, endpoints EndpointList) {
|
||||
for _, endpoint := range endpoints {
|
||||
if endpoint.IsLocal {
|
||||
rpcServer, err := NewStorageRPCServer(endpoint.Path)
|
||||
logger.CriticalIf(context.Background(), err)
|
||||
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
subrouter.Path(path.Join(storageServiceSubPath, endpoint.Path)).Handler(rpcServer)
|
||||
}
|
||||
// Add minio storage routes.
|
||||
storageRouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||
storageRouter.Path(path.Join(storageRPCPath, stServer.path)).Handler(storageRPCServer)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -1,178 +0,0 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 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 (
|
||||
"os"
|
||||
"testing"
|
||||
)
|
||||
|
||||
type testStorageRPCServer struct {
|
||||
configDir string
|
||||
token string
|
||||
diskDirs []string
|
||||
stServer *storageServer
|
||||
endpoints EndpointList
|
||||
}
|
||||
|
||||
func createTestStorageServer(t *testing.T) *testStorageRPCServer {
|
||||
testPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("unable initialize config file, %s", err)
|
||||
}
|
||||
|
||||
serverCred := globalServerConfig.GetCredential()
|
||||
token, err := authenticateNode(serverCred.AccessKey, serverCred.SecretKey)
|
||||
if err != nil {
|
||||
t.Fatalf("unable for JWT to generate token, %s", err)
|
||||
}
|
||||
|
||||
fsDirs, err := getRandomDisks(1)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to create FS backend, %s", err)
|
||||
}
|
||||
|
||||
endpoints := mustGetNewEndpointList(fsDirs...)
|
||||
storageDisks, err := initStorageDisks(endpoints)
|
||||
if err != nil {
|
||||
t.Fatalf("unable to initialize storage disks, %s", err)
|
||||
}
|
||||
stServer := &storageServer{
|
||||
storage: storageDisks[0],
|
||||
path: "/disk1",
|
||||
}
|
||||
return &testStorageRPCServer{
|
||||
token: token,
|
||||
configDir: testPath,
|
||||
diskDirs: fsDirs,
|
||||
endpoints: endpoints,
|
||||
stServer: stServer,
|
||||
}
|
||||
}
|
||||
|
||||
func errorIfInvalidToken(t *testing.T, err error) {
|
||||
if err != errInvalidToken {
|
||||
t.Errorf("Expected to fail with %s but failed with %s", errInvalidToken, err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestStorageRPCInvalidToken(t *testing.T) {
|
||||
st := createTestStorageServer(t)
|
||||
defer removeRoots(st.diskDirs)
|
||||
defer os.RemoveAll(st.configDir)
|
||||
|
||||
storageRPC := st.stServer
|
||||
|
||||
// Following test cases are meant to exercise the invalid
|
||||
// token code path of the storage RPC methods.
|
||||
var err error
|
||||
badAuthRPCArgs := AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
AuthToken: "invalidToken",
|
||||
}
|
||||
badGenericVolArgs := GenericVolArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
Vol: "myvol",
|
||||
}
|
||||
// 1. DiskInfoHandler
|
||||
diskInfoReply := &DiskInfo{}
|
||||
err = storageRPC.DiskInfoHandler(&badAuthRPCArgs, diskInfoReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 2. MakeVolHandler
|
||||
makeVolArgs := &badGenericVolArgs
|
||||
makeVolReply := &AuthRPCReply{}
|
||||
err = storageRPC.MakeVolHandler(makeVolArgs, makeVolReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 3. ListVolsHandler
|
||||
listVolReply := &ListVolsReply{}
|
||||
err = storageRPC.ListVolsHandler(&badAuthRPCArgs, listVolReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 4. StatVolHandler
|
||||
statVolReply := &VolInfo{}
|
||||
statVolArgs := &badGenericVolArgs
|
||||
err = storageRPC.StatVolHandler(statVolArgs, statVolReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 5. DeleteVolHandler
|
||||
deleteVolArgs := &badGenericVolArgs
|
||||
deleteVolReply := &AuthRPCReply{}
|
||||
err = storageRPC.DeleteVolHandler(deleteVolArgs, deleteVolReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 6. StatFileHandler
|
||||
statFileArgs := &StatFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
statReply := &FileInfo{}
|
||||
err = storageRPC.StatFileHandler(statFileArgs, statReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 7. ListDirHandler
|
||||
listDirArgs := &ListDirArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
listDirReply := &[]string{}
|
||||
err = storageRPC.ListDirHandler(listDirArgs, listDirReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 8. ReadAllHandler
|
||||
readFileArgs := &ReadFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
readFileReply := &[]byte{}
|
||||
err = storageRPC.ReadAllHandler(readFileArgs, readFileReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 9. ReadFileHandler
|
||||
err = storageRPC.ReadFileHandler(readFileArgs, readFileReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 10. PrepareFileHandler
|
||||
prepFileArgs := &PrepareFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
prepFileReply := &AuthRPCReply{}
|
||||
err = storageRPC.PrepareFileHandler(prepFileArgs, prepFileReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 11. AppendFileHandler
|
||||
appendArgs := &AppendFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
appendReply := &AuthRPCReply{}
|
||||
err = storageRPC.AppendFileHandler(appendArgs, appendReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 12. DeleteFileHandler
|
||||
delFileArgs := &DeleteFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
delFileRely := &AuthRPCReply{}
|
||||
err = storageRPC.DeleteFileHandler(delFileArgs, delFileRely)
|
||||
errorIfInvalidToken(t, err)
|
||||
|
||||
// 13. RenameFileHandler
|
||||
renameArgs := &RenameFileArgs{
|
||||
AuthRPCArgs: badAuthRPCArgs,
|
||||
}
|
||||
renameReply := &AuthRPCReply{}
|
||||
err = storageRPC.RenameFileHandler(renameArgs, renameReply)
|
||||
errorIfInvalidToken(t, err)
|
||||
}
|
|
@ -0,0 +1,701 @@
|
|||
/*
|
||||
* Minio Cloud Storage, (C) 2018 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 (
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"os"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
xnet "github.com/minio/minio/pkg/net"
|
||||
)
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
//
|
||||
// Storage RPC server, storageRPCReceiver and StorageRPCClient are
|
||||
// inter-dependent, below test functions are sufficient to test all of them.
|
||||
//
|
||||
///////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
func testStorageAPIDiskInfo(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
testCases := []struct {
|
||||
expectErr bool
|
||||
}{
|
||||
{false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
_, err := storage.DiskInfo()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIMakeVol(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", false},
|
||||
// volume exists error.
|
||||
{"foo", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.MakeVol(testCase.volumeName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIListVols(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
testCases := []struct {
|
||||
volumeNames []string
|
||||
expectedResult []VolInfo
|
||||
expectErr bool
|
||||
}{
|
||||
{nil, []VolInfo{}, false},
|
||||
{[]string{"foo"}, []VolInfo{{Name: "foo"}}, false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
for _, volumeName := range testCase.volumeNames {
|
||||
err := storage.MakeVol(volumeName)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
result, err := storage.ListVols()
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if len(result) != len(testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %+v, got: %+v", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIStatVol(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", false},
|
||||
// volume not found error.
|
||||
{"bar", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result, err := storage.StatVol(testCase.volumeName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if result.Name != testCase.volumeName {
|
||||
t.Fatalf("case %v: result: expected: %+v, got: %+v", i+1, testCase.volumeName, result.Name)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIDeleteVol(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", false},
|
||||
// volume not found error.
|
||||
{"bar", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.DeleteVol(testCase.volumeName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIStatFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
err = storage.AppendFile("foo", "myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", false},
|
||||
// file not found error.
|
||||
{"foo", "yourobject", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result, err := storage.StatFile(testCase.volumeName, testCase.objectName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if result.Name != testCase.objectName {
|
||||
t.Fatalf("case %v: result: expected: %+v, got: %+v", i+1, testCase.objectName, result.Name)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIListDir(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
err = storage.AppendFile("foo", "path/to/myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
prefix string
|
||||
expectedResult []string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "path", []string{"to/"}, false},
|
||||
// prefix not found error.
|
||||
{"foo", "nodir", nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result, err := storage.ListDir(testCase.volumeName, testCase.prefix, -1)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %v, got: %v", i+1, testCase.expectedResult, result)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIReadAll(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
err = storage.AppendFile("foo", "myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
expectedResult []byte
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", []byte("foo"), false},
|
||||
// file not found error.
|
||||
{"foo", "yourobject", nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result, err := storage.ReadAll(testCase.volumeName, testCase.objectName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %v, got: %v", i+1, string(testCase.expectedResult), string(result))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIReadFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
err = storage.AppendFile("foo", "myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
offset int64
|
||||
expectedResult []byte
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", 0, []byte("foo"), false},
|
||||
{"foo", "myobject", 1, []byte("oo"), false},
|
||||
// file not found error.
|
||||
{"foo", "yourobject", 0, nil, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
result := make([]byte, 100)
|
||||
n, err := storage.ReadFile(testCase.volumeName, testCase.objectName, testCase.offset, result, nil)
|
||||
expectErr := (err != nil)
|
||||
result = result[:n]
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
|
||||
if !testCase.expectErr {
|
||||
if !reflect.DeepEqual(result, testCase.expectedResult) {
|
||||
t.Fatalf("case %v: result: expected: %v, got: %v", i+1, string(testCase.expectedResult), string(result))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIPrepareFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", false},
|
||||
// volume not found error.
|
||||
{"bar", "myobject", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.PrepareFile(testCase.volumeName, testCase.objectName, 1)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIAppendFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
data []byte
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", []byte("foo"), false},
|
||||
{"foo", "myobject", []byte{}, false},
|
||||
// volume not found error.
|
||||
{"bar", "myobject", []byte{}, true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.AppendFile(testCase.volumeName, testCase.objectName, testCase.data)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIDeleteFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
err = storage.AppendFile("foo", "myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", false},
|
||||
// should removed by above case.
|
||||
{"foo", "myobject", true},
|
||||
// file not found error
|
||||
{"foo", "yourobject", true},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.DeleteFile(testCase.volumeName, testCase.objectName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStorageAPIRenameFile(t *testing.T, storage StorageAPI) {
|
||||
tmpGlobalServerConfig := globalServerConfig
|
||||
defer func() {
|
||||
globalServerConfig = tmpGlobalServerConfig
|
||||
}()
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
err := storage.MakeVol("foo")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
err = storage.MakeVol("bar")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
err = storage.AppendFile("foo", "myobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
err = storage.AppendFile("foo", "otherobject", []byte("foo"))
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
volumeName string
|
||||
objectName string
|
||||
destVolumeName string
|
||||
destObjectName string
|
||||
expectErr bool
|
||||
}{
|
||||
{"foo", "myobject", "foo", "yourobject", false},
|
||||
{"foo", "yourobject", "bar", "myobject", false},
|
||||
// overwrite.
|
||||
{"foo", "otherobject", "bar", "myobject", false},
|
||||
}
|
||||
|
||||
for i, testCase := range testCases {
|
||||
err := storage.RenameFile(testCase.volumeName, testCase.objectName, testCase.destVolumeName, testCase.destObjectName)
|
||||
expectErr := (err != nil)
|
||||
|
||||
if expectErr != testCase.expectErr {
|
||||
t.Fatalf("case %v: error: expected: %v, got: %v", i+1, testCase.expectErr, expectErr)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func newStorageRPCHTTPServerClient(t *testing.T) (*httptest.Server, *StorageRPCClient, *serverConfig, string) {
|
||||
endpointPath, err := ioutil.TempDir("", ".TestStorageRPC.")
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
rpcServer, err := NewStorageRPCServer(endpointPath)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
httpServer := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
rpcServer.ServeHTTP(w, r)
|
||||
}))
|
||||
|
||||
url, err := xnet.ParseURL(httpServer.URL)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
host, err := xnet.ParseHost(url.Host)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
|
||||
prevGlobalServerConfig := globalServerConfig
|
||||
globalServerConfig = newServerConfig()
|
||||
|
||||
rpcClient, err := NewStorageRPCClient(host, endpointPath)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error %v", err)
|
||||
}
|
||||
rpcClient.connected = true
|
||||
|
||||
return httpServer, rpcClient, prevGlobalServerConfig, endpointPath
|
||||
}
|
||||
|
||||
func TestStorageRPCClientDiskInfo(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIDiskInfo(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientMakeVol(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIMakeVol(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientListVols(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIListVols(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientStatVol(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIStatVol(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientDeleteVol(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIDeleteVol(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientStatFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIStatFile(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientListDir(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIListDir(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientReadAll(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIReadAll(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientReadFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIReadFile(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientPrepareFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIPrepareFile(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientAppendFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIAppendFile(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientDeleteFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIDeleteFile(t, rpcClient)
|
||||
}
|
||||
|
||||
func TestStorageRPCClientRenameFile(t *testing.T) {
|
||||
httpServer, rpcClient, prevGlobalServerConfig, endpointPath := newStorageRPCHTTPServerClient(t)
|
||||
defer httpServer.Close()
|
||||
defer func() {
|
||||
globalServerConfig = prevGlobalServerConfig
|
||||
}()
|
||||
defer os.RemoveAll(endpointPath)
|
||||
|
||||
testStorageAPIRenameFile(t, rpcClient)
|
||||
}
|
|
@ -479,7 +479,7 @@ func StartTestPeersRPCServer(t TestErrHandler, instanceType string) TestServer {
|
|||
// need API layer to send requests, etc.
|
||||
registerAPIRouter(router)
|
||||
// module being tested is Peer RPCs router.
|
||||
registerS3PeerRPCRouter(router)
|
||||
registerPeerRPCRouter(router)
|
||||
|
||||
// Run TestServer.
|
||||
testRPCServer.Server = httptest.NewServer(router)
|
||||
|
@ -2265,35 +2265,6 @@ func initTestWebRPCEndPoint(objLayer ObjectLayer) http.Handler {
|
|||
return muxRouter
|
||||
}
|
||||
|
||||
// Initialize browser RPC endpoint.
|
||||
func initTestBrowserPeerRPCEndPoint() http.Handler {
|
||||
// Initialize router.
|
||||
muxRouter := mux.NewRouter().SkipClean(true)
|
||||
registerBrowserPeerRPCRouter(muxRouter)
|
||||
return muxRouter
|
||||
}
|
||||
|
||||
func StartTestBrowserPeerRPCServer(t TestErrHandler, instanceType string) TestServer {
|
||||
root, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
t.Fatalf("%s", err)
|
||||
}
|
||||
|
||||
// Create an instance of TestServer.
|
||||
testRPCServer := TestServer{}
|
||||
|
||||
// Fetch credentials for the test server.
|
||||
credentials := globalServerConfig.GetCredential()
|
||||
|
||||
testRPCServer.Root = root
|
||||
testRPCServer.AccessKey = credentials.AccessKey
|
||||
testRPCServer.SecretKey = credentials.SecretKey
|
||||
|
||||
// Initialize and run the TestServer.
|
||||
testRPCServer.Server = httptest.NewServer(initTestBrowserPeerRPCEndPoint())
|
||||
return testRPCServer
|
||||
}
|
||||
|
||||
func StartTestS3PeerRPCServer(t TestErrHandler) (TestServer, []string) {
|
||||
root, err := newTestConfig(globalMinioDefaultRegion)
|
||||
if err != nil {
|
||||
|
@ -2323,10 +2294,7 @@ func StartTestS3PeerRPCServer(t TestErrHandler) (TestServer, []string) {
|
|||
|
||||
// Register router on a new mux
|
||||
muxRouter := mux.NewRouter().SkipClean(true)
|
||||
err = registerS3PeerRPCRouter(muxRouter)
|
||||
if err != nil {
|
||||
t.Fatalf("%s", err)
|
||||
}
|
||||
registerPeerRPCRouter(muxRouter)
|
||||
|
||||
// Initialize and run the TestServer.
|
||||
testRPCServer.Server = httptest.NewServer(muxRouter)
|
||||
|
|
|
@ -26,9 +26,6 @@ var errInvalidArgument = errors.New("Invalid arguments specified")
|
|||
// errSignatureMismatch means signature did not match.
|
||||
var errSignatureMismatch = errors.New("Signature does not match")
|
||||
|
||||
// used when token used for authentication by the MinioBrowser has expired
|
||||
var errInvalidToken = errors.New("Invalid token")
|
||||
|
||||
// used when we deal with data larger than expected
|
||||
var errSizeUnexpected = errors.New("Data size larger than expected")
|
||||
|
||||
|
|
|
@ -102,7 +102,7 @@ type StorageInfoRep struct {
|
|||
}
|
||||
|
||||
// StorageInfo - web call to gather storage usage statistics.
|
||||
func (web *webAPIHandlers) StorageInfo(r *http.Request, args *AuthRPCArgs, reply *StorageInfoRep) error {
|
||||
func (web *webAPIHandlers) StorageInfo(r *http.Request, args *AuthArgs, reply *StorageInfoRep) error {
|
||||
objectAPI := web.ObjectAPI()
|
||||
if objectAPI == nil {
|
||||
return toJSONError(errServerNotInitialized)
|
||||
|
@ -470,53 +470,29 @@ func (web *webAPIHandlers) SetAuth(r *http.Request, args *SetAuthArgs, reply *Se
|
|||
globalServerConfigMu.Lock()
|
||||
defer globalServerConfigMu.Unlock()
|
||||
|
||||
// Notify all other Minio peers to update credentials
|
||||
errsMap := updateCredsOnPeers(creds)
|
||||
|
||||
// Update local credentials
|
||||
// Update credentials in memory
|
||||
prevCred := globalServerConfig.SetCredential(creds)
|
||||
|
||||
// Persist updated credentials.
|
||||
if err = globalServerConfig.Save(); err != nil {
|
||||
// Save the current creds when failed to update.
|
||||
// Save credentials to config file
|
||||
if err := globalServerConfig.Save(); err != nil {
|
||||
// As saving configurstion failed, restore previous credential in memory.
|
||||
globalServerConfig.SetCredential(prevCred)
|
||||
|
||||
errsMap[globalMinioAddr] = err
|
||||
}
|
||||
|
||||
// Log all the peer related error messages, and populate the
|
||||
// PeerErrMsgs map.
|
||||
reply.PeerErrMsgs = make(map[string]string)
|
||||
for svr, errVal := range errsMap {
|
||||
tErr := fmt.Errorf("Unable to change credentials on %s: %v", svr, errVal)
|
||||
logger.LogIf(context.Background(), tErr)
|
||||
reply.PeerErrMsgs[svr] = errVal.Error()
|
||||
}
|
||||
|
||||
// If we were unable to update locally, we return an error to the user/browser.
|
||||
if errsMap[globalMinioAddr] != nil {
|
||||
// Since the error message may be very long to display
|
||||
// on the browser, we tell the user to check the
|
||||
// server logs.
|
||||
return toJSONError(fmt.Errorf("unexpected error(s) occurred - please check minio server logs"))
|
||||
}
|
||||
|
||||
// As we have updated access/secret key, generate new auth token.
|
||||
token, err := authenticateWeb(creds.AccessKey, creds.SecretKey)
|
||||
if err != nil {
|
||||
// Did we have peer errors?
|
||||
if len(errsMap) > 0 {
|
||||
err = fmt.Errorf(
|
||||
"we gave up due to: '%s', but there were more errors. Please check minio server logs",
|
||||
err.Error(),
|
||||
)
|
||||
}
|
||||
|
||||
logger.LogIf(context.Background(), err)
|
||||
return toJSONError(err)
|
||||
}
|
||||
|
||||
reply.Token = token
|
||||
reply.UIVersion = browser.UIVersion
|
||||
if errs := globalNotificationSys.SetCredentials(creds); len(errs) != 0 {
|
||||
reply.PeerErrMsgs = make(map[string]string)
|
||||
for host, err := range errs {
|
||||
err = fmt.Errorf("Unable to update credentials on server %v: %v", host, err)
|
||||
logger.LogIf(context.Background(), err)
|
||||
reply.PeerErrMsgs[host.String()] = err.Error()
|
||||
}
|
||||
} else {
|
||||
reply.Token = newAuthToken()
|
||||
reply.UIVersion = browser.UIVersion
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,6 @@ import (
|
|||
"crypto/md5"
|
||||
"encoding/hex"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
|
@ -130,25 +129,7 @@ func TestWriteWebErrorResponse(t *testing.T) {
|
|||
|
||||
// Authenticate and get JWT token - will be called before every webrpc handler invocation
|
||||
func getWebRPCToken(apiRouter http.Handler, accessKey, secretKey string) (token string, err error) {
|
||||
rec := httptest.NewRecorder()
|
||||
request := LoginArgs{Username: accessKey, Password: secretKey}
|
||||
reply := &LoginRep{}
|
||||
req, err := newTestWebRPCRequest("Web"+loginMethodName, "", request)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
apiRouter.ServeHTTP(rec, req)
|
||||
if rec.Code != http.StatusOK {
|
||||
return "", errors.New("Auth failed")
|
||||
}
|
||||
err = getTestWebRPCResponse(rec, &reply)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
if reply.Token == "" {
|
||||
return "", errors.New("Auth failed")
|
||||
}
|
||||
return reply.Token, nil
|
||||
return authenticateWeb(accessKey, secretKey)
|
||||
}
|
||||
|
||||
// Wrapper for calling Login Web Handler
|
||||
|
@ -209,8 +190,8 @@ func testStorageInfoWebHandler(obj ObjectLayer, instanceType string, t TestErrHa
|
|||
|
||||
rec := httptest.NewRecorder()
|
||||
|
||||
storageInfoRequest := AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
storageInfoRequest := AuthArgs{
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
}
|
||||
storageInfoReply := &StorageInfoRep{}
|
||||
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
||||
|
@ -244,8 +225,8 @@ func testServerInfoWebHandler(obj ObjectLayer, instanceType string, t TestErrHan
|
|||
|
||||
rec := httptest.NewRecorder()
|
||||
|
||||
serverInfoRequest := AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
serverInfoRequest := AuthArgs{
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
}
|
||||
serverInfoReply := &ServerInfoRep{}
|
||||
req, err := newTestWebRPCRequest("Web.ServerInfo", authorization, serverInfoRequest)
|
||||
|
@ -1545,8 +1526,8 @@ func TestWebCheckAuthorization(t *testing.T) {
|
|||
"PresignedGet",
|
||||
}
|
||||
for _, rpcCall := range webRPCs {
|
||||
args := &AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
args := &AuthArgs{
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
}
|
||||
reply := &WebGenericRep{}
|
||||
req, nerr := newTestWebRPCRequest("Web."+rpcCall, "Bearer fooauthorization", args)
|
||||
|
@ -1631,8 +1612,8 @@ func TestWebObjectLayerNotReady(t *testing.T) {
|
|||
webRPCs := []string{"StorageInfo", "MakeBucket", "ListBuckets", "ListObjects", "RemoveObject",
|
||||
"GetBucketPolicy", "SetBucketPolicy", "ListAllBucketPolicies"}
|
||||
for _, rpcCall := range webRPCs {
|
||||
args := &AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
args := &AuthArgs{
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
}
|
||||
reply := &WebGenericRep{}
|
||||
req, nerr := newTestWebRPCRequest("Web."+rpcCall, authorization, args)
|
||||
|
@ -1746,7 +1727,7 @@ func TestWebObjectLayerFaultyDisks(t *testing.T) {
|
|||
RepArgs interface{}
|
||||
}{
|
||||
{"MakeBucket", MakeBucketArgs{BucketName: bucketName}, WebGenericRep{}},
|
||||
{"ListBuckets", AuthRPCArgs{Version: globalRPCAPIVersion}, ListBucketsRep{}},
|
||||
{"ListBuckets", AuthArgs{RPCVersion: globalRPCAPIVersion}, ListBucketsRep{}},
|
||||
{"ListObjects", ListObjectsArgs{BucketName: bucketName, Prefix: ""}, ListObjectsRep{}},
|
||||
{"GetBucketPolicy", GetBucketPolicyArgs{BucketName: bucketName, Prefix: ""}, GetBucketPolicyRep{}},
|
||||
{"SetBucketPolicy", SetBucketPolicyWebArgs{BucketName: bucketName, Prefix: "", Policy: "none"}, WebGenericRep{}},
|
||||
|
@ -1770,8 +1751,8 @@ func TestWebObjectLayerFaultyDisks(t *testing.T) {
|
|||
}
|
||||
|
||||
// Test Web.StorageInfo
|
||||
storageInfoRequest := AuthRPCArgs{
|
||||
Version: globalRPCAPIVersion,
|
||||
storageInfoRequest := AuthArgs{
|
||||
RPCVersion: globalRPCAPIVersion,
|
||||
}
|
||||
storageInfoReply := &StorageInfoRep{}
|
||||
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
||||
|
|
Loading…
Reference in New Issue