mirror of
https://github.com/minio/minio.git
synced 2025-04-04 03:40:30 -04:00
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
|
// Initialize server info at index
|
||||||
reply[idx] = ServerInfo{Addr: peer.addr}
|
reply[idx] = ServerInfo{Addr: peer.addr}
|
||||||
|
|
||||||
serverInfoData, err := peer.cmdRunner.ServerInfoData()
|
serverInfoData, err := peer.cmdRunner.ServerInfo()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", peer.addr)
|
reqInfo := (&logger.ReqInfo{}).AppendTags("peerAddress", peer.addr)
|
||||||
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
ctx := logger.SetReqInfo(context.Background(), reqInfo)
|
||||||
@ -808,14 +808,6 @@ func (a adminAPIHandlers) UpdateCredentialsHandler(w http.ResponseWriter,
|
|||||||
globalServerConfigMu.Lock()
|
globalServerConfigMu.Lock()
|
||||||
defer globalServerConfigMu.Unlock()
|
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.
|
// Update local credentials in memory.
|
||||||
globalServerConfig.SetCredential(creds)
|
globalServerConfig.SetCredential(creds)
|
||||||
if err = globalServerConfig.Save(); err != nil {
|
if err = globalServerConfig.Save(); err != nil {
|
||||||
@ -823,6 +815,13 @@ func (a adminAPIHandlers) UpdateCredentialsHandler(w http.ResponseWriter,
|
|||||||
return
|
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
|
// At this stage, the operation is successful, return 200 OK
|
||||||
w.WriteHeader(http.StatusOK)
|
w.WriteHeader(http.StatusOK)
|
||||||
}
|
}
|
||||||
|
@ -18,39 +18,128 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"crypto/tls"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"fmt"
|
"fmt"
|
||||||
"net"
|
"net"
|
||||||
"os"
|
|
||||||
"path"
|
|
||||||
"path/filepath"
|
|
||||||
"sort"
|
"sort"
|
||||||
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/minio/minio-go/pkg/set"
|
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
var errUnsupportedSignal = fmt.Errorf("unsupported signal: only restart and stop signals are supported")
|
||||||
// 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.
|
// AdminRPCClient - admin RPC client talks to admin RPC server.
|
||||||
type localAdminClient struct {
|
type AdminRPCClient struct {
|
||||||
|
*RPCClient
|
||||||
}
|
}
|
||||||
|
|
||||||
// remoteAdminClient - represents admin operation to be executed
|
// SignalService - calls SignalService RPC.
|
||||||
// remotely, via RPC.
|
func (rpcClient *AdminRPCClient) SignalService(signal serviceSignal) (err error) {
|
||||||
type remoteAdminClient struct {
|
args := SignalServiceArgs{Sig: signal}
|
||||||
*AuthRPCClient
|
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
|
// adminCmdRunner - abstracts local and remote execution of admin
|
||||||
@ -59,189 +148,12 @@ type adminCmdRunner interface {
|
|||||||
SignalService(s serviceSignal) error
|
SignalService(s serviceSignal) error
|
||||||
ReInitFormat(dryRun bool) error
|
ReInitFormat(dryRun bool) error
|
||||||
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
ListLocks(bucket, prefix string, duration time.Duration) ([]VolumeLockInfo, error)
|
||||||
ServerInfoData() (ServerInfoData, error)
|
ServerInfo() (ServerInfoData, error)
|
||||||
GetConfig() ([]byte, error)
|
GetConfig() ([]byte, error)
|
||||||
WriteTmpConfig(tmpFileName string, configBytes []byte) error
|
WriteTmpConfig(tmpFileName string, configBytes []byte) error
|
||||||
CommitConfig(tmpFileName string) 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.
|
// adminPeer - represents an entity that implements admin API RPCs.
|
||||||
type adminPeer struct {
|
type adminPeer struct {
|
||||||
addr string
|
addr string
|
||||||
@ -254,36 +166,25 @@ type adminPeers []adminPeer
|
|||||||
|
|
||||||
// makeAdminPeers - helper function to construct a collection of adminPeer.
|
// makeAdminPeers - helper function to construct a collection of adminPeer.
|
||||||
func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
|
func makeAdminPeers(endpoints EndpointList) (adminPeerList adminPeers) {
|
||||||
thisPeer := globalMinioAddr
|
localAddr := GetLocalPeer(endpoints)
|
||||||
if globalMinioHost == "" {
|
if strings.HasPrefix(localAddr, "127.0.0.1:") {
|
||||||
// When host is not explicitly provided simply
|
// Use first IPv4 instead of loopback address.
|
||||||
// use the first IPv4.
|
localAddr = net.JoinHostPort(sortIPs(localIP4.ToSlice())[0], globalMinioPort)
|
||||||
thisPeer = net.JoinHostPort(sortIPs(localIP4.ToSlice())[0], globalMinioPort)
|
|
||||||
}
|
}
|
||||||
adminPeerList = append(adminPeerList, adminPeer{
|
adminPeerList = append(adminPeerList, adminPeer{
|
||||||
thisPeer,
|
addr: localAddr,
|
||||||
localAdminClient{},
|
cmdRunner: localAdminClient{},
|
||||||
true,
|
isLocal: true,
|
||||||
})
|
})
|
||||||
|
|
||||||
hostSet := set.CreateStringSet(globalMinioAddr)
|
for _, hostStr := range GetRemotePeers(endpoints) {
|
||||||
cred := globalServerConfig.GetCredential()
|
host, err := xnet.ParseHost(hostStr)
|
||||||
serviceEndpoint := path.Join(minioReservedBucketPath, adminPath)
|
logger.CriticalIf(context.Background(), err)
|
||||||
for _, host := range GetRemotePeers(endpoints) {
|
rpcClient, err := NewAdminRPCClient(host)
|
||||||
if hostSet.Contains(host) {
|
logger.CriticalIf(context.Background(), err)
|
||||||
continue
|
|
||||||
}
|
|
||||||
hostSet.Add(host)
|
|
||||||
adminPeerList = append(adminPeerList, adminPeer{
|
adminPeerList = append(adminPeerList, adminPeer{
|
||||||
addr: host,
|
addr: hostStr,
|
||||||
cmdRunner: &remoteAdminClient{newAuthRPCClient(authConfig{
|
cmdRunner: rpcClient,
|
||||||
accessKey: cred.AccessKey,
|
|
||||||
secretKey: cred.SecretKey,
|
|
||||||
serverAddr: host,
|
|
||||||
serviceEndpoint: serviceEndpoint,
|
|
||||||
secureConn: globalIsSSL,
|
|
||||||
serviceName: "Admin",
|
|
||||||
})},
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -429,7 +330,7 @@ func getPeerUptimes(peers adminPeers) (time.Duration, error) {
|
|||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
go func(idx int, peer adminPeer) {
|
go func(idx int, peer adminPeer) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
serverInfoData, rpcErr := peer.cmdRunner.ServerInfoData()
|
serverInfoData, rpcErr := peer.cmdRunner.ServerInfo()
|
||||||
uptimes[idx].uptime, uptimes[idx].err = serverInfoData.Properties.Uptime, rpcErr
|
uptimes[idx].uptime, uptimes[idx].err = serverInfoData.Properties.Uptime, rpcErr
|
||||||
}(i, peer)
|
}(i, peer)
|
||||||
}
|
}
|
||||||
|
650
cmd/admin-rpc-client.go.orig
Normal file
650
cmd/admin-rpc-client.go.orig
Normal file
@ -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 (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"encoding/json"
|
"path"
|
||||||
"fmt"
|
|
||||||
"io/ioutil"
|
|
||||||
"os"
|
|
||||||
"path/filepath"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/gorilla/mux"
|
"github.com/gorilla/mux"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"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
|
var adminServicePath = path.Join(minioReservedBucketPath, adminServiceSubPath)
|
||||||
// restart commands.
|
|
||||||
type adminCmd struct {
|
// adminRPCReceiver - Admin RPC receiver for admin RPC server.
|
||||||
AuthRPCServer
|
type adminRPCReceiver struct {
|
||||||
|
local *localAdminClient
|
||||||
}
|
}
|
||||||
|
|
||||||
// SignalServiceArgs - provides the signal argument to SignalService RPC
|
// SignalServiceArgs - provides the signal argument to SignalService RPC
|
||||||
type SignalServiceArgs struct {
|
type SignalServiceArgs struct {
|
||||||
AuthRPCArgs
|
AuthArgs
|
||||||
Sig serviceSignal
|
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.
|
// ListLocksQuery - wraps ListLocks API's query values to send over RPC.
|
||||||
type ListLocksQuery struct {
|
type ListLocksQuery struct {
|
||||||
AuthRPCArgs
|
AuthArgs
|
||||||
Bucket string
|
Bucket string
|
||||||
Prefix string
|
Prefix string
|
||||||
Duration time.Duration
|
Duration time.Duration
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListLocksReply - wraps ListLocks response over RPC.
|
// ListLocks - lists locks held by requests handled by this server instance.
|
||||||
type ListLocksReply struct {
|
func (receiver *adminRPCReceiver) ListLocks(args *ListLocksQuery, reply *[]VolumeLockInfo) (err error) {
|
||||||
AuthRPCReply
|
*reply, err = receiver.local.ListLocks(args.Bucket, args.Prefix, args.Duration)
|
||||||
VolLocks []VolumeLockInfo
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// ServerInfoDataReply - wraps the server info response over RPC.
|
// ServerInfo - returns the server info when object layer was initialized on this server.
|
||||||
type ServerInfoDataReply struct {
|
func (receiver *adminRPCReceiver) ServerInfo(args *AuthArgs, reply *ServerInfoData) (err error) {
|
||||||
AuthRPCReply
|
*reply, err = receiver.local.ServerInfo()
|
||||||
ServerInfoData ServerInfoData
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// ConfigReply - wraps the server config response over RPC.
|
// GetConfig - returns the config.json of this server.
|
||||||
type ConfigReply struct {
|
func (receiver *adminRPCReceiver) GetConfig(args *AuthArgs, reply *[]byte) (err error) {
|
||||||
AuthRPCReply
|
*reply, err = receiver.local.GetConfig()
|
||||||
Config []byte // json-marshalled bytes of serverConfigV13
|
return err
|
||||||
}
|
|
||||||
|
|
||||||
// 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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReInitFormatArgs - provides dry-run information to re-initialize format.json
|
// ReInitFormatArgs - provides dry-run information to re-initialize format.json
|
||||||
type ReInitFormatArgs struct {
|
type ReInitFormatArgs struct {
|
||||||
AuthRPCArgs
|
AuthArgs
|
||||||
DryRun bool
|
DryRun bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReInitFormat - re-init 'format.json'
|
// ReInitFormat - re-init 'format.json'
|
||||||
func (s *adminCmd) ReInitFormat(args *ReInitFormatArgs, reply *AuthRPCReply) error {
|
func (receiver *adminRPCReceiver) ReInitFormat(args *ReInitFormatArgs, reply *VoidReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return receiver.local.ReInitFormat(args.DryRun)
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// WriteConfigArgs - wraps the bytes to be written and temporary file name.
|
// WriteConfigArgs - wraps the bytes to be written and temporary file name.
|
||||||
type WriteConfigArgs struct {
|
type WriteConfigArgs struct {
|
||||||
AuthRPCArgs
|
AuthArgs
|
||||||
TmpFileName string
|
TmpFileName string
|
||||||
Buf []byte
|
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
|
// WriteTmpConfig - writes the supplied config contents onto the
|
||||||
// supplied temporary file.
|
// supplied temporary file.
|
||||||
func (s *adminCmd) WriteTmpConfig(wArgs *WriteConfigArgs, wReply *WriteConfigReply) error {
|
func (receiver *adminRPCReceiver) WriteTmpConfig(args *WriteConfigArgs, reply *VoidReply) error {
|
||||||
if err := wArgs.IsAuthenticated(); err != nil {
|
return receiver.local.WriteTmpConfig(args.TmpFileName, args.Buf)
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
return writeTmpConfigCommon(wArgs.TmpFileName, wArgs.Buf)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// CommitConfigArgs - wraps the config file name that needs to be
|
// CommitConfigArgs - wraps the config file name that needs to be
|
||||||
// committed into config.json on this node.
|
// committed into config.json on this node.
|
||||||
type CommitConfigArgs struct {
|
type CommitConfigArgs struct {
|
||||||
AuthRPCArgs
|
AuthArgs
|
||||||
FileName string
|
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.
|
// CommitConfig - Renames the temporary file into config.json on this node.
|
||||||
func (s *adminCmd) CommitConfig(cArgs *CommitConfigArgs, cReply *CommitConfigReply) error {
|
func (receiver *adminRPCReceiver) CommitConfig(args *CommitConfigArgs, reply *VoidReply) error {
|
||||||
configFile := getConfigFile()
|
return receiver.local.CommitConfig(args.FileName)
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// registerAdminRPCRouter - registers RPC methods for service status,
|
// NewAdminRPCServer - returns new admin RPC server.
|
||||||
// stop and restart commands.
|
func NewAdminRPCServer() (*xrpc.Server, error) {
|
||||||
func registerAdminRPCRouter(router *mux.Router) error {
|
rpcServer := xrpc.NewServer()
|
||||||
adminRPCHandler := &adminCmd{}
|
if err := rpcServer.RegisterName(adminServiceName, &adminRPCReceiver{&localAdminClient{}}); err != nil {
|
||||||
adminRPCServer := newRPCServer()
|
return nil, err
|
||||||
err := adminRPCServer.RegisterName("Admin", adminRPCHandler)
|
|
||||||
if err != nil {
|
|
||||||
logger.LogIf(context.Background(), err)
|
|
||||||
return err
|
|
||||||
}
|
}
|
||||||
adminRouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
return rpcServer, nil
|
||||||
adminRouter.Path(adminPath).Handler(adminRPCServer)
|
}
|
||||||
return 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)
|
|
||||||
}
|
|
||||||
}
|
|
613
cmd/admin-rpc_test.go
Normal file
613
cmd/admin-rpc_test.go
Normal file
@ -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
|
|
||||||
}
|
|
165
cmd/dummy-object-layer_test.go
Normal file
165
cmd/dummy-object-layer_test.go
Normal file
@ -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 = xhttp.NewServer([]string{gatewayAddr}, registerHandlers(router, globalHandlers...), getCert)
|
||||||
globalHTTPServer.ReadTimeout = globalConnReadTimeout
|
|
||||||
globalHTTPServer.WriteTimeout = globalConnWriteTimeout
|
|
||||||
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
||||||
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
||||||
go func() {
|
go func() {
|
||||||
|
@ -222,7 +222,7 @@ func guessIsRPCReq(req *http.Request) bool {
|
|||||||
if req == nil {
|
if req == nil {
|
||||||
return false
|
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) {
|
func (h redirectHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||||
|
@ -76,7 +76,7 @@ func TestGuessIsRPC(t *testing.T) {
|
|||||||
}
|
}
|
||||||
r := &http.Request{
|
r := &http.Request{
|
||||||
Proto: "HTTP/1.0",
|
Proto: "HTTP/1.0",
|
||||||
Method: http.MethodConnect,
|
Method: http.MethodPost,
|
||||||
}
|
}
|
||||||
if !guessIsRPCReq(r) {
|
if !guessIsRPCReq(r) {
|
||||||
t.Fatal("Test shouldn't fail for a possible net/rpc request.")
|
t.Fatal("Test shouldn't fail for a possible net/rpc request.")
|
||||||
|
@ -68,10 +68,6 @@ const (
|
|||||||
// date and server date during signature verification.
|
// date and server date during signature verification.
|
||||||
globalMaxSkewTime = 15 * time.Minute // 15 minutes skew allowed.
|
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.
|
// Expiry duration after which the multipart uploads are deemed stale.
|
||||||
globalMultipartExpiry = time.Hour * 24 * 14 // 2 weeks.
|
globalMultipartExpiry = time.Hour * 24 * 14 // 2 weeks.
|
||||||
// Cleanup interval when the stale multipart cleanup is initiated.
|
// Cleanup interval when the stale multipart cleanup is initiated.
|
||||||
@ -186,6 +182,13 @@ var (
|
|||||||
globalCacheExcludes []string
|
globalCacheExcludes []string
|
||||||
// Disk cache expiry
|
// Disk cache expiry
|
||||||
globalCacheExpiry = 90
|
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.
|
// Add new variable global values here.
|
||||||
|
|
||||||
// Default usage check interval value.
|
// Default usage check interval value.
|
||||||
|
62
cmd/http/timeoutconn.go
Normal file
62
cmd/http/timeoutconn.go
Normal file
@ -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
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func newAuthToken() string {
|
||||||
|
cred := globalServerConfig.GetCredential()
|
||||||
|
token, err := authenticateNode(cred.AccessKey, cred.SecretKey)
|
||||||
|
logger.CriticalIf(context.Background(), err)
|
||||||
|
return token
|
||||||
|
}
|
||||||
|
123
cmd/local-admin-client.go
Normal file
123
cmd/local-admin-client.go
Normal file
@ -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");
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
* you may not use this file except in compliance with the License.
|
* you may not use this file except in compliance with the License.
|
||||||
@ -17,14 +17,40 @@
|
|||||||
package cmd
|
package cmd
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
|
||||||
"fmt"
|
"fmt"
|
||||||
"os"
|
"os"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"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) {
|
func TestListLocksInfo(t *testing.T) {
|
||||||
// reset global variables to start afresh.
|
// reset global variables to start afresh.
|
||||||
resetTestGlobals()
|
resetTestGlobals()
|
||||||
@ -70,6 +96,8 @@ func TestListLocksInfo(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
client := &localAdminClient{}
|
||||||
|
|
||||||
testCases := []struct {
|
testCases := []struct {
|
||||||
bucket string
|
bucket string
|
||||||
prefix string
|
prefix string
|
||||||
@ -100,9 +128,9 @@ func TestListLocksInfo(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
for i, test := range testCases {
|
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 {
|
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 {
|
if len(actual) != test.numLocks {
|
||||||
t.Errorf("Test %d - Expected %d locks but observed %d locks",
|
t.Errorf("Test %d - Expected %d locks but observed %d locks",
|
151
cmd/local-locker.go
Normal file
151
cmd/local-locker.go
Normal file
@ -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
|
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
|
// LockRPCClient is authenticable lock RPC client compatible to dsync.NetLocker
|
||||||
type LockRPCClient struct {
|
type LockRPCClient struct {
|
||||||
*AuthRPCClient
|
*RPCClient
|
||||||
}
|
}
|
||||||
|
|
||||||
// newLockRPCClient returns new lock RPC client object.
|
// ServerAddr - dsync.NetLocker interface compatible method.
|
||||||
func newLockRPCClient(config authConfig) *LockRPCClient {
|
func (lockRPC *LockRPCClient) ServerAddr() string {
|
||||||
return &LockRPCClient{newAuthRPCClient(config)}
|
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.
|
// RLock calls read lock RPC.
|
||||||
func (lockRPCClient *LockRPCClient) RLock(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) RLock(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".RLock", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.RLock", &lockArgs, &reply)
|
|
||||||
return reply, err
|
return reply, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Lock calls write lock RPC.
|
// Lock calls write lock RPC.
|
||||||
func (lockRPCClient *LockRPCClient) Lock(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) Lock(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".Lock", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Lock", &lockArgs, &reply)
|
|
||||||
return reply, err
|
return reply, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// RUnlock calls read unlock RPC.
|
// RUnlock calls read unlock RPC.
|
||||||
func (lockRPCClient *LockRPCClient) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) RUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".RUnlock", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.RUnlock", &lockArgs, &reply)
|
|
||||||
return reply, err
|
return reply, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unlock calls write unlock RPC.
|
// Unlock calls write unlock RPC.
|
||||||
func (lockRPCClient *LockRPCClient) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) Unlock(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".Unlock", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Unlock", &lockArgs, &reply)
|
|
||||||
return reply, err
|
return reply, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// ForceUnlock calls force unlock RPC.
|
// ForceUnlock calls force unlock RPC.
|
||||||
func (lockRPCClient *LockRPCClient) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) ForceUnlock(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".ForceUnlock", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.ForceUnlock", &lockArgs, &reply)
|
|
||||||
return reply, err
|
return reply, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Expired calls expired RPC.
|
// Expired calls expired RPC.
|
||||||
func (lockRPCClient *LockRPCClient) Expired(args dsync.LockArgs) (reply bool, err error) {
|
func (lockRPC *LockRPCClient) Expired(args dsync.LockArgs) (reply bool, err error) {
|
||||||
lockArgs := newLockArgs(args)
|
err = lockRPC.Call(lockServiceName+".Expired", &LockArgs{LockArgs: args}, &reply)
|
||||||
err = lockRPCClient.AuthRPCClient.Call("Dsync.Expired", &lockArgs, &reply)
|
|
||||||
return reply, err
|
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
|
package cmd
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/minio/dsync"
|
"github.com/minio/dsync"
|
||||||
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Tests lock rpc client.
|
// Tests lock rpc client.
|
||||||
func TestLockRPCClient(t *testing.T) {
|
func TestLockRPCClient(t *testing.T) {
|
||||||
lkClient := newLockRPCClient(authConfig{
|
host, err := xnet.ParseHost("localhost:9000")
|
||||||
accessKey: "abcd",
|
if err != nil {
|
||||||
secretKey: "abcd123",
|
t.Fatalf("unexpected error %v", err)
|
||||||
serverAddr: fmt.Sprintf("%X", UTCNow().UnixNano()),
|
}
|
||||||
serviceEndpoint: pathJoin(lockServicePath, "/test/1"),
|
lkClient, err := NewLockRPCClient(host)
|
||||||
secureConn: false,
|
if err != nil {
|
||||||
serviceName: lockServiceName,
|
t.Fatalf("unexpected error %v", err)
|
||||||
})
|
}
|
||||||
|
|
||||||
// Attempt all calls.
|
// Attempt all calls.
|
||||||
_, err := lkClient.RLock(dsync.LockArgs{})
|
_, err = lkClient.RLock(dsync.LockArgs{})
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatal("Expected for Rlock to fail")
|
t.Fatal("Expected for Rlock to fail")
|
||||||
}
|
}
|
||||||
|
@ -24,6 +24,12 @@ import (
|
|||||||
"github.com/minio/minio/cmd/logger"
|
"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.
|
// Similar to removeEntry but only removes an entry only if the lock entry exists in map.
|
||||||
func (l *localLocker) removeEntryIfExists(nlrip nameLockRequesterInfoPair) {
|
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)
|
// 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 (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"sync"
|
"path"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/gorilla/mux"
|
"github.com/gorilla/mux"
|
||||||
"github.com/minio/dsync"
|
"github.com/minio/dsync"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
xrpc "github.com/minio/minio/cmd/rpc"
|
||||||
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
// Lock rpc server endpoint.
|
// Lock rpc server endpoint.
|
||||||
lockServicePath = "/lock"
|
lockServiceSubPath = "/lock"
|
||||||
|
|
||||||
// Lock rpc service name.
|
// Lock rpc service name.
|
||||||
lockServiceName = "Dsync"
|
lockServiceName = "Dsync"
|
||||||
|
|
||||||
// Lock maintenance interval.
|
// Lock maintenance interval.
|
||||||
lockMaintenanceInterval = 1 * time.Minute // 1 minute.
|
lockMaintenanceInterval = 1 * time.Minute
|
||||||
|
|
||||||
// Lock validity check interval.
|
// 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.
|
var lockServicePath = path.Join(minioReservedBucketPath, lockServiceSubPath)
|
||||||
type lockRequesterInfo struct {
|
|
||||||
writer bool // Bool whether write or read lock.
|
// LockArgs represents arguments for any authenticated lock RPC call.
|
||||||
node string // Network address of client claiming lock.
|
type LockArgs struct {
|
||||||
serviceEndpoint string // RPC path of client claiming lock.
|
AuthArgs
|
||||||
uid string // UID to uniquely identify request of client.
|
LockArgs dsync.LockArgs
|
||||||
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.
|
// lockRPCReceiver is type for RPC handlers
|
||||||
func isWriteLock(lri []lockRequesterInfo) bool {
|
type lockRPCReceiver struct {
|
||||||
return len(lri) == 1 && lri[0].writer
|
|
||||||
}
|
|
||||||
|
|
||||||
// lockServer is type for RPC handlers
|
|
||||||
type lockServer struct {
|
|
||||||
AuthRPCServer
|
|
||||||
ll localLocker
|
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.
|
// Lock - rpc handler for (single) write lock operation.
|
||||||
func (l *lockServer) Lock(args *LockArgs, reply *bool) (err error) {
|
func (l *lockRPCReceiver) Lock(args *LockArgs, reply *bool) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply, err = l.ll.Lock(args.LockArgs)
|
*reply, err = l.ll.Lock(args.LockArgs)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unlock - rpc handler for (single) write unlock operation.
|
// Unlock - rpc handler for (single) write unlock operation.
|
||||||
func (l *lockServer) Unlock(args *LockArgs, reply *bool) (err error) {
|
func (l *lockRPCReceiver) Unlock(args *LockArgs, reply *bool) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply, err = l.ll.Unlock(args.LockArgs)
|
*reply, err = l.ll.Unlock(args.LockArgs)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// RLock - rpc handler for read lock operation.
|
// RLock - rpc handler for read lock operation.
|
||||||
func (l *lockServer) RLock(args *LockArgs, reply *bool) (err error) {
|
func (l *lockRPCReceiver) RLock(args *LockArgs, reply *bool) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply, err = l.ll.RLock(args.LockArgs)
|
*reply, err = l.ll.RLock(args.LockArgs)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// RUnlock - rpc handler for read unlock operation.
|
// RUnlock - rpc handler for read unlock operation.
|
||||||
func (l *lockServer) RUnlock(args *LockArgs, reply *bool) (err error) {
|
func (l *lockRPCReceiver) RUnlock(args *LockArgs, reply *bool) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply, err = l.ll.RUnlock(args.LockArgs)
|
*reply, err = l.ll.RUnlock(args.LockArgs)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// ForceUnlock - rpc handler for force unlock operation.
|
// ForceUnlock - rpc handler for force unlock operation.
|
||||||
func (l *lockServer) ForceUnlock(args *LockArgs, reply *bool) (err error) {
|
func (l *lockRPCReceiver) ForceUnlock(args *LockArgs, reply *bool) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply, err = l.ll.ForceUnlock(args.LockArgs)
|
*reply, err = l.ll.ForceUnlock(args.LockArgs)
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Expired - rpc handler for expired lock status.
|
// Expired - rpc handler for expired lock status.
|
||||||
func (l *lockServer) Expired(args *LockArgs, reply *bool) error {
|
func (l *lockRPCReceiver) Expired(args *LockArgs, reply *bool) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
l.ll.mutex.Lock()
|
l.ll.mutex.Lock()
|
||||||
defer l.ll.mutex.Unlock()
|
defer l.ll.mutex.Unlock()
|
||||||
// Lock found, proceed to verify if belongs to given uid.
|
// 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
|
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
|
// 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
|
// 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)
|
// - 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
|
// 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()
|
l.ll.mutex.Lock()
|
||||||
// Get list of long lived locks to check for staleness.
|
// Get list of long lived locks to check for staleness.
|
||||||
nlripLongLived := getLongLivedLocks(l.ll.lockMap, interval)
|
nlripLongLived := getLongLivedLocks(l.ll.lockMap, interval)
|
||||||
l.ll.mutex.Unlock()
|
l.ll.mutex.Unlock()
|
||||||
|
|
||||||
serverCred := globalServerConfig.GetCredential()
|
|
||||||
// Validate if long lived locks are indeed clean.
|
// Validate if long lived locks are indeed clean.
|
||||||
for _, nlrip := range nlripLongLived {
|
for _, nlrip := range nlripLongLived {
|
||||||
// Initialize client based on the long live locks.
|
// Initialize client based on the long live locks.
|
||||||
c := newLockRPCClient(authConfig{
|
host, err := xnet.ParseHost(nlrip.lri.node)
|
||||||
accessKey: serverCred.AccessKey,
|
logger.CriticalIf(context.Background(), err)
|
||||||
secretKey: serverCred.SecretKey,
|
c, err := NewLockRPCClient(host)
|
||||||
serverAddr: nlrip.lri.node,
|
if err != nil {
|
||||||
secureConn: globalIsSSL,
|
logger.LogIf(context.Background(), err)
|
||||||
serviceEndpoint: nlrip.lri.serviceEndpoint,
|
continue
|
||||||
serviceName: lockServiceName,
|
}
|
||||||
})
|
|
||||||
|
|
||||||
// Call back to original server verify whether the lock is still active (based on name & uid)
|
// Call back to original server verify whether the lock is still active (based on name & uid)
|
||||||
expired, _ := c.Expired(dsync.LockArgs{
|
expired, _ := c.Expired(dsync.LockArgs{
|
||||||
@ -328,7 +138,7 @@ func (l *lockServer) lockMaintenance(interval time.Duration) {
|
|||||||
})
|
})
|
||||||
|
|
||||||
// Close the connection regardless of the call response.
|
// Close the connection regardless of the call response.
|
||||||
c.AuthRPCClient.Close()
|
c.Close()
|
||||||
|
|
||||||
// For successful response, verify if lock is indeed active or stale.
|
// For successful response, verify if lock is indeed active or stale.
|
||||||
if expired {
|
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
|
// 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)
|
testPath, err := newTestConfig(globalMinioDefaultRegion)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("unable initialize config file, %s", err)
|
t.Fatalf("unable initialize config file, %s", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
locker := &lockServer{
|
locker := &lockRPCReceiver{
|
||||||
AuthRPCServer: AuthRPCServer{},
|
|
||||||
ll: localLocker{
|
ll: localLocker{
|
||||||
mutex: sync.Mutex{},
|
mutex: sync.Mutex{},
|
||||||
serviceEndpoint: "rpc-path",
|
serviceEndpoint: "rpc-path",
|
||||||
@ -62,16 +61,6 @@ func createLockTestServer(t *testing.T) (string, *lockServer, string) {
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatal(err)
|
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
|
return testPath, locker, token
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -80,14 +69,18 @@ func TestLockRpcServerLock(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "0123-4567",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// Claim a lock
|
// Claim a lock
|
||||||
var result bool
|
var result bool
|
||||||
@ -114,14 +107,18 @@ func TestLockRpcServerLock(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Try to claim same lock again (will fail)
|
// Try to claim same lock again (will fail)
|
||||||
la2 := newLockArgs(dsync.LockArgs{
|
la2 := LockArgs{
|
||||||
UID: "89ab-cdef",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la2.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "89ab-cdef",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
err = locker.Lock(&la2, &result)
|
err = locker.Lock(&la2, &result)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -138,14 +135,18 @@ func TestLockRpcServerUnlock(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
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
|
// First test return of error when attempting to unlock a lock that does not exist
|
||||||
var result bool
|
var result bool
|
||||||
@ -184,14 +185,18 @@ func TestLockRpcServerRLock(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "0123-4567",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// Claim a lock
|
// Claim a lock
|
||||||
var result bool
|
var result bool
|
||||||
@ -218,14 +223,18 @@ func TestLockRpcServerRLock(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Try to claim same again (will succeed)
|
// Try to claim same again (will succeed)
|
||||||
la2 := newLockArgs(dsync.LockArgs{
|
la2 := LockArgs{
|
||||||
UID: "89ab-cdef",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la2.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "89ab-cdef",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
err = locker.RLock(&la2, &result)
|
err = locker.RLock(&la2, &result)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -242,14 +251,18 @@ func TestLockRpcServerRUnlock(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
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
|
// First test return of error when attempting to unlock a read-lock that does not exist
|
||||||
var result bool
|
var result bool
|
||||||
@ -267,14 +280,18 @@ func TestLockRpcServerRUnlock(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Try to claim same again (will succeed)
|
// Try to claim same again (will succeed)
|
||||||
la2 := newLockArgs(dsync.LockArgs{
|
la2 := LockArgs{
|
||||||
UID: "89ab-cdef",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la2.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la2.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "89ab-cdef",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// ... and create a second lock on same resource
|
// ... and create a second lock on same resource
|
||||||
err = locker.RLock(&la2, &result)
|
err = locker.RLock(&la2, &result)
|
||||||
@ -330,14 +347,18 @@ func TestLockRpcServerForceUnlock(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
laForce := newLockArgs(dsync.LockArgs{
|
laForce := LockArgs{
|
||||||
UID: "1234-5678",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
laForce.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
laForce.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "1234-5678",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// First test that UID should be empty
|
// First test that UID should be empty
|
||||||
var result bool
|
var result bool
|
||||||
@ -353,14 +374,18 @@ func TestLockRpcServerForceUnlock(t *testing.T) {
|
|||||||
t.Errorf("Expected no error, got %#v", err)
|
t.Errorf("Expected no error, got %#v", err)
|
||||||
}
|
}
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "0123-4567",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// Create lock ... (so that we can force unlock)
|
// Create lock ... (so that we can force unlock)
|
||||||
err = locker.Lock(&la, &result)
|
err = locker.Lock(&la, &result)
|
||||||
@ -396,14 +421,18 @@ func TestLockRpcServerExpired(t *testing.T) {
|
|||||||
testPath, locker, token := createLockTestServer(t)
|
testPath, locker, token := createLockTestServer(t)
|
||||||
defer os.RemoveAll(testPath)
|
defer os.RemoveAll(testPath)
|
||||||
|
|
||||||
la := newLockArgs(dsync.LockArgs{
|
la := LockArgs{
|
||||||
UID: "0123-4567",
|
AuthArgs: AuthArgs{
|
||||||
Resource: "name",
|
Token: token,
|
||||||
ServerAddr: "node",
|
RPCVersion: globalRPCAPIVersion,
|
||||||
ServiceEndpoint: "rpc-path",
|
RequestTime: UTCNow(),
|
||||||
})
|
},
|
||||||
la.SetAuthToken(token)
|
LockArgs: dsync.LockArgs{
|
||||||
la.SetRPCAPIVersion(globalRPCAPIVersion)
|
UID: "0123-4567",
|
||||||
|
Resource: "name",
|
||||||
|
ServerAddr: "node",
|
||||||
|
ServiceEndpoint: "rpc-path",
|
||||||
|
}}
|
||||||
|
|
||||||
// Unknown lock at server will return expired = true
|
// Unknown lock at server will return expired = true
|
||||||
var expired bool
|
var expired bool
|
||||||
@ -500,7 +529,7 @@ func TestLockServerInit(t *testing.T) {
|
|||||||
t.Fatalf("Got unexpected error initializing lock servers: %v", err)
|
t.Fatalf("Got unexpected error initializing lock servers: %v", err)
|
||||||
}
|
}
|
||||||
if globalLockServer == nil && testCase.isDistXL {
|
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/lsync"
|
||||||
"github.com/minio/minio-go/pkg/set"
|
"github.com/minio/minio-go/pkg/set"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Global name space lock.
|
// Global name space lock.
|
||||||
var globalNSMutex *nsLockMap
|
var globalNSMutex *nsLockMap
|
||||||
|
|
||||||
// Global lock server one per server.
|
// Global lock server one per server.
|
||||||
var globalLockServer *lockServer
|
var globalLockServer *lockRPCReceiver
|
||||||
|
|
||||||
// Instance of dsync for distributed clients.
|
// Instance of dsync for distributed clients.
|
||||||
var globalDsync *dsync.Dsync
|
var globalDsync *dsync.Dsync
|
||||||
@ -61,7 +62,6 @@ type RWLockerSync interface {
|
|||||||
// Initialize distributed locking only in case of distributed setup.
|
// Initialize distributed locking only in case of distributed setup.
|
||||||
// Returns lock clients and the node index for the current server.
|
// Returns lock clients and the node index for the current server.
|
||||||
func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int) {
|
func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int) {
|
||||||
cred := globalServerConfig.GetCredential()
|
|
||||||
myNode = -1
|
myNode = -1
|
||||||
seenHosts := set.NewStringSet()
|
seenHosts := set.NewStringSet()
|
||||||
for _, endpoint := range endpoints {
|
for _, endpoint := range endpoints {
|
||||||
@ -69,35 +69,29 @@ func newDsyncNodes(endpoints EndpointList) (clnts []dsync.NetLocker, myNode int)
|
|||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
seenHosts.Add(endpoint.Host)
|
seenHosts.Add(endpoint.Host)
|
||||||
if !endpoint.IsLocal {
|
|
||||||
// For a remote endpoints setup a lock RPC client.
|
var locker dsync.NetLocker
|
||||||
clnts = append(clnts, newLockRPCClient(authConfig{
|
if endpoint.IsLocal {
|
||||||
accessKey: cred.AccessKey,
|
myNode = len(clnts)
|
||||||
secretKey: cred.SecretKey,
|
|
||||||
serverAddr: endpoint.Host,
|
receiver := &lockRPCReceiver{
|
||||||
secureConn: globalIsSSL,
|
ll: localLocker{
|
||||||
serviceEndpoint: pathutil.Join(minioReservedBucketPath, lockServicePath),
|
serverAddr: endpoint.Host,
|
||||||
serviceName: lockServiceName,
|
serviceEndpoint: lockServicePath,
|
||||||
}))
|
lockMap: make(map[string][]lockRequesterInfo),
|
||||||
continue
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
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
|
clnts = append(clnts, locker)
|
||||||
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))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return clnts, myNode
|
return clnts, myNode
|
||||||
|
@ -26,8 +26,10 @@ import (
|
|||||||
"net/url"
|
"net/url"
|
||||||
"path"
|
"path"
|
||||||
"sync"
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
"github.com/minio/minio/pkg/auth"
|
||||||
"github.com/minio/minio/pkg/event"
|
"github.com/minio/minio/pkg/event"
|
||||||
"github.com/minio/minio/pkg/hash"
|
"github.com/minio/minio/pkg/hash"
|
||||||
xnet "github.com/minio/minio/pkg/net"
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
@ -90,6 +92,33 @@ func (sys *NotificationSys) DeleteBucket(bucketName string) <-chan NotificationP
|
|||||||
return errCh
|
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.
|
// SetBucketPolicy - calls SetBucketPolicy RPC call on all peers.
|
||||||
func (sys *NotificationSys) SetBucketPolicy(bucketName string, bucketPolicy *policy.Policy) <-chan NotificationPeerErr {
|
func (sys *NotificationSys) SetBucketPolicy(bucketName string, bucketPolicy *policy.Policy) <-chan NotificationPeerErr {
|
||||||
errCh := make(chan NotificationPeerErr)
|
errCh := make(chan NotificationPeerErr)
|
||||||
|
52
cmd/peer-rpc-client-target.go
Normal file
52
cmd/peer-rpc-client-target.go
Normal file
@ -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,
|
||||||
|
}
|
||||||
|
}
|
176
cmd/peer-rpc-client.go
Normal file
176
cmd/peer-rpc-client.go
Normal file
@ -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
|
||||||
|
}
|
207
cmd/peer-rpc-server.go
Normal file
207
cmd/peer-rpc-server.go
Normal file
@ -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
|
return false
|
||||||
}
|
}
|
||||||
defer f.Close()
|
defer f.Close()
|
||||||
|
|
||||||
// List one entry.
|
// List one entry.
|
||||||
_, err = f.Readdirnames(1)
|
_, err = f.Readdirnames(1)
|
||||||
if err != io.EOF {
|
if err != io.EOF {
|
||||||
@ -152,14 +151,15 @@ func isDirEmpty(dirname string) bool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Initialize a new storage disk.
|
// Initialize a new storage disk.
|
||||||
func newPosix(path string) (StorageAPI, error) {
|
func newPosix(path string) (*posix, error) {
|
||||||
var err error
|
var err error
|
||||||
if path, err = getValidPath(path); err != nil {
|
if path, err = getValidPath(path); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
st := &posix{
|
p := &posix{
|
||||||
diskPath: path,
|
connected: true,
|
||||||
|
diskPath: path,
|
||||||
// 1MiB buffer pool for posix internal operations.
|
// 1MiB buffer pool for posix internal operations.
|
||||||
pool: sync.Pool{
|
pool: sync.Pool{
|
||||||
New: func() interface{} {
|
New: func() interface{} {
|
||||||
@ -170,12 +170,10 @@ func newPosix(path string) (StorageAPI, error) {
|
|||||||
stopUsageCh: make(chan struct{}),
|
stopUsageCh: make(chan struct{}),
|
||||||
}
|
}
|
||||||
|
|
||||||
st.connected = true
|
go p.diskUsage(globalServiceDoneCh)
|
||||||
|
|
||||||
go st.diskUsage(globalServiceDoneCh)
|
|
||||||
|
|
||||||
// Success.
|
// Success.
|
||||||
return st, nil
|
return p, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// getDiskInfo returns given disk information.
|
// getDiskInfo returns given disk information.
|
||||||
@ -285,6 +283,7 @@ func (s *posix) DiskInfo() (info DiskInfo, err error) {
|
|||||||
Free: di.Free,
|
Free: di.Free,
|
||||||
Used: atomic.LoadUint64(&s.totalUsed),
|
Used: atomic.LoadUint64(&s.totalUsed),
|
||||||
}, nil
|
}, nil
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// getVolDir - will convert incoming volume names to
|
// getVolDir - will convert incoming volume names to
|
||||||
|
@ -127,13 +127,6 @@ const (
|
|||||||
defaultRetryCap = 30 * time.Second // 30 seconds.
|
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
|
// newRetryTimerSimple creates a timer with exponentially increasing delays
|
||||||
// until the maximum retry attempts are reached. - this function is a
|
// until the maximum retry attempts are reached. - this function is a
|
||||||
// simpler version with all default values.
|
// simpler version with all default values.
|
||||||
|
@ -34,27 +34,15 @@ func newCacheObjectsFn() CacheObjectLayer {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Composed function registering routers for only distributed XL setup.
|
// 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.
|
// Register storage rpc router only if its a distributed setup.
|
||||||
err := registerStorageRPCRouters(router, endpoints)
|
registerStorageRPCRouters(router, endpoints)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Register distributed namespace lock.
|
// Register distributed namespace lock.
|
||||||
err = registerDistNSLockRouter(router, endpoints)
|
registerDistNSLockRouter(router)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Register S3 peer communication router.
|
// Register S3 peer communication router.
|
||||||
err = registerS3PeerRPCRouter(router)
|
registerPeerRPCRouter(router)
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Register RPC router for web related calls.
|
|
||||||
return registerBrowserPeerRPCRouter(router)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// List of some generic handlers which are applied for all incoming requests.
|
// 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
|
// Add Admin RPC router
|
||||||
err := registerAdminRPCRouter(router)
|
registerAdminRPCRouter(router)
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Add Admin router.
|
// Add Admin router.
|
||||||
registerAdminRouter(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)
|
|
||||||
}
|
|
||||||
}
|
|
257
cmd/rpc.go
Normal file
257
cmd/rpc.go
Normal file
@ -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
|
||||||
|
}
|
128
cmd/rpc/client.go
Normal file
128
cmd/rpc/client.go
Normal file
@ -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,
|
||||||
|
}
|
||||||
|
}
|
72
cmd/rpc/client_test.go
Normal file
72
cmd/rpc/client_test.go
Normal file
@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
264
cmd/rpc/server.go
Normal file
264
cmd/rpc/server.go
Normal file
@ -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{}
|
||||||
|
}
|
345
cmd/rpc/server_test.go
Normal file
345
cmd/rpc/server_test.go
Normal file
@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
404
cmd/rpc_test.go
Normal file
404
cmd/rpc_test.go
Normal file
@ -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 = xhttp.NewServer([]string{globalMinioAddr}, handler, getCert)
|
||||||
globalHTTPServer.ReadTimeout = globalConnReadTimeout
|
|
||||||
globalHTTPServer.WriteTimeout = globalConnWriteTimeout
|
|
||||||
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
globalHTTPServer.UpdateBytesReadFunc = globalConnStats.incInputBytes
|
||||||
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
globalHTTPServer.UpdateBytesWrittenFunc = globalConnStats.incOutputBytes
|
||||||
go func() {
|
go func() {
|
||||||
|
@ -18,33 +18,33 @@ package cmd
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"context"
|
||||||
|
"crypto/tls"
|
||||||
"io"
|
"io"
|
||||||
"net"
|
"net"
|
||||||
"net/rpc"
|
"net/url"
|
||||||
"path"
|
"path"
|
||||||
"strings"
|
"strings"
|
||||||
|
|
||||||
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
xnet "github.com/minio/minio/pkg/net"
|
||||||
)
|
)
|
||||||
|
|
||||||
type networkStorage struct {
|
func isNetworkDisconnectError(err error) bool {
|
||||||
rpcClient *AuthRPCClient
|
|
||||||
connected bool
|
|
||||||
}
|
|
||||||
|
|
||||||
const (
|
|
||||||
storageRPCPath = "/storage"
|
|
||||||
)
|
|
||||||
|
|
||||||
func isErrorNetworkDisconnect(err error) bool {
|
|
||||||
if err == nil {
|
if err == nil {
|
||||||
return false
|
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
|
_, isNetOpError := err.(*net.OpError)
|
||||||
}
|
return isNetOpError
|
||||||
return false
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Converts rpc.ServerError to underlying error. This function is
|
// Converts rpc.ServerError to underlying error. This function is
|
||||||
@ -55,7 +55,7 @@ func toStorageErr(err error) error {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
if isErrorNetworkDisconnect(err) {
|
if isNetworkDisconnectError(err) {
|
||||||
return errDiskNotFound
|
return errDiskNotFound
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -100,162 +100,128 @@ func toStorageErr(err error) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize new storage rpc client.
|
// StorageRPCClient - storage RPC client.
|
||||||
func newStorageRPC(endpoint Endpoint) StorageAPI {
|
type StorageRPCClient struct {
|
||||||
// Dial minio rpc storage http path.
|
*RPCClient
|
||||||
rpcPath := path.Join(minioReservedBucketPath, storageRPCPath, endpoint.Path)
|
connected bool
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Stringer provides a canonicalized representation of network device.
|
// Stringer provides a canonicalized representation of network device.
|
||||||
func (n *networkStorage) String() string {
|
func (client *StorageRPCClient) String() string {
|
||||||
// Remove the storage RPC path prefix, internal paths are meaningless.
|
url := client.ServiceURL()
|
||||||
serviceEndpoint := strings.TrimPrefix(n.rpcClient.ServiceEndpoint(),
|
// Remove the storage RPC path prefix, internal paths are meaningless. why?
|
||||||
path.Join(minioReservedBucketPath, storageRPCPath))
|
url.Path = strings.TrimPrefix(url.Path, storageServicePath)
|
||||||
// Check for the transport layer being used.
|
return url.String()
|
||||||
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 (n *networkStorage) Close() error {
|
// Close - closes underneath RPC client.
|
||||||
n.connected = false
|
func (client *StorageRPCClient) Close() error {
|
||||||
return toStorageErr(n.rpcClient.Close())
|
client.connected = false
|
||||||
|
return toStorageErr(client.RPCClient.Close())
|
||||||
}
|
}
|
||||||
|
|
||||||
func (n *networkStorage) IsOnline() bool {
|
// IsOnline - returns whether RPC client failed to connect or not.
|
||||||
return n.connected
|
func (client *StorageRPCClient) IsOnline() bool {
|
||||||
|
return client.connected
|
||||||
}
|
}
|
||||||
|
|
||||||
func (n *networkStorage) call(handler string, args interface {
|
func (client *StorageRPCClient) call(handler string, args interface {
|
||||||
SetAuthToken(string)
|
SetAuthArgs(args AuthArgs)
|
||||||
SetRPCAPIVersion(semVersion)
|
|
||||||
}, reply interface{}) error {
|
}, reply interface{}) error {
|
||||||
if !n.connected {
|
if !client.connected {
|
||||||
return errDiskNotFound
|
return errDiskNotFound
|
||||||
}
|
}
|
||||||
if err := n.rpcClient.Call(handler, args, reply); err != nil {
|
|
||||||
if isErrorNetworkDisconnect(err) {
|
err := client.Call(handler, args, reply)
|
||||||
n.connected = false
|
if err == nil {
|
||||||
}
|
return nil
|
||||||
return toStorageErr(err)
|
|
||||||
}
|
}
|
||||||
return nil
|
|
||||||
|
if isNetworkDisconnectError(err) {
|
||||||
|
client.connected = false
|
||||||
|
}
|
||||||
|
|
||||||
|
return toStorageErr(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
// DiskInfo - fetch disk information for a remote disk.
|
// DiskInfo - fetch disk information for a remote disk.
|
||||||
func (n *networkStorage) DiskInfo() (info DiskInfo, err error) {
|
func (client *StorageRPCClient) DiskInfo() (info DiskInfo, err error) {
|
||||||
args := AuthRPCArgs{}
|
err = client.call(storageServiceName+".DiskInfo", &AuthArgs{}, &info)
|
||||||
if err = n.call("Storage.DiskInfoHandler", &args, &info); err != nil {
|
return info, err
|
||||||
return DiskInfo{}, err
|
|
||||||
}
|
|
||||||
return info, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// MakeVol - create a volume on a remote disk.
|
// MakeVol - create a volume on a remote disk.
|
||||||
func (n *networkStorage) MakeVol(volume string) (err error) {
|
func (client *StorageRPCClient) MakeVol(volume string) (err error) {
|
||||||
reply := AuthRPCReply{}
|
return client.call(storageServiceName+".MakeVol", &VolArgs{Vol: volume}, &VoidReply{})
|
||||||
args := GenericVolArgs{Vol: volume}
|
|
||||||
return n.call("Storage.MakeVolHandler", &args, &reply)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListVols - List all volumes on a remote disk.
|
// ListVols - List all volumes on a remote disk.
|
||||||
func (n *networkStorage) ListVols() (vols []VolInfo, err error) {
|
func (client *StorageRPCClient) ListVols() ([]VolInfo, error) {
|
||||||
ListVols := ListVolsReply{}
|
var reply []VolInfo
|
||||||
if err = n.call("Storage.ListVolsHandler", &AuthRPCArgs{}, &ListVols); err != nil {
|
err := client.call(storageServiceName+".ListVols", &AuthArgs{}, &reply)
|
||||||
return nil, err
|
return reply, err
|
||||||
}
|
|
||||||
return ListVols.Vols, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// StatVol - get volume info over the network.
|
// StatVol - get volume info over the network.
|
||||||
func (n *networkStorage) StatVol(volume string) (volInfo VolInfo, err error) {
|
func (client *StorageRPCClient) StatVol(volume string) (volInfo VolInfo, err error) {
|
||||||
args := GenericVolArgs{Vol: volume}
|
err = client.call(storageServiceName+".StatVol", &VolArgs{Vol: volume}, &volInfo)
|
||||||
if err = n.call("Storage.StatVolHandler", &args, &volInfo); err != nil {
|
return volInfo, err
|
||||||
return VolInfo{}, err
|
|
||||||
}
|
|
||||||
return volInfo, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// DeleteVol - Deletes a volume over the network.
|
// DeleteVol - Deletes a volume over the network.
|
||||||
func (n *networkStorage) DeleteVol(volume string) (err error) {
|
func (client *StorageRPCClient) DeleteVol(volume string) (err error) {
|
||||||
reply := AuthRPCReply{}
|
return client.call(storageServiceName+".DeleteVol", &VolArgs{Vol: volume}, &VoidReply{})
|
||||||
args := GenericVolArgs{Vol: volume}
|
|
||||||
return n.call("Storage.DeleteVolHandler", &args, &reply)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// File operations.
|
// File operations.
|
||||||
|
|
||||||
func (n *networkStorage) PrepareFile(volume, path string, length int64) (err error) {
|
// PrepareFile - calls PrepareFile RPC.
|
||||||
reply := AuthRPCReply{}
|
func (client *StorageRPCClient) PrepareFile(volume, path string, length int64) (err error) {
|
||||||
return n.call("Storage.PrepareFileHandler", &PrepareFileArgs{
|
args := PrepareFileArgs{
|
||||||
Vol: volume,
|
Vol: volume,
|
||||||
Path: path,
|
Path: path,
|
||||||
Size: length,
|
Size: length,
|
||||||
}, &reply)
|
}
|
||||||
|
reply := VoidReply{}
|
||||||
|
|
||||||
|
return client.call(storageServiceName+".PrepareFile", &args, &reply)
|
||||||
}
|
}
|
||||||
|
|
||||||
// AppendFile - append file writes buffer to a remote network path.
|
// AppendFile - append file writes buffer to a remote network path.
|
||||||
func (n *networkStorage) AppendFile(volume, path string, buffer []byte) (err error) {
|
func (client *StorageRPCClient) AppendFile(volume, path string, buffer []byte) (err error) {
|
||||||
reply := AuthRPCReply{}
|
args := AppendFileArgs{
|
||||||
return n.call("Storage.AppendFileHandler", &AppendFileArgs{
|
|
||||||
Vol: volume,
|
Vol: volume,
|
||||||
Path: path,
|
Path: path,
|
||||||
Buffer: buffer,
|
Buffer: buffer,
|
||||||
}, &reply)
|
}
|
||||||
|
reply := VoidReply{}
|
||||||
|
|
||||||
|
return client.call(storageServiceName+".AppendFile", &args, &reply)
|
||||||
}
|
}
|
||||||
|
|
||||||
// StatFile - get latest Stat information for a file at path.
|
// StatFile - get latest Stat information for a file at path.
|
||||||
func (n *networkStorage) StatFile(volume, path string) (fileInfo FileInfo, err error) {
|
func (client *StorageRPCClient) StatFile(volume, path string) (fileInfo FileInfo, err error) {
|
||||||
if err = n.call("Storage.StatFileHandler", &StatFileArgs{
|
err = client.call(storageServiceName+".StatFile", &StatFileArgs{Vol: volume, Path: path}, &fileInfo)
|
||||||
Vol: volume,
|
return fileInfo, err
|
||||||
Path: path,
|
|
||||||
}, &fileInfo); err != nil {
|
|
||||||
return FileInfo{}, err
|
|
||||||
}
|
|
||||||
return fileInfo, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReadAll - reads entire contents of the file at path until EOF, returns the
|
// ReadAll - reads entire contents of the file at path until EOF, returns the
|
||||||
// contents in a byte slice. Returns buf == nil if err != nil.
|
// 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
|
// 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.
|
// not use this on large files as it would cause server to crash.
|
||||||
func (n *networkStorage) ReadAll(volume, path string) (buf []byte, err error) {
|
func (client *StorageRPCClient) ReadAll(volume, path string) (buf []byte, err error) {
|
||||||
if err = n.call("Storage.ReadAllHandler", &ReadAllArgs{
|
err = client.call(storageServiceName+".ReadAll", &ReadAllArgs{Vol: volume, Path: path}, &buf)
|
||||||
Vol: volume,
|
return buf, err
|
||||||
Path: path,
|
|
||||||
}, &buf); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
return buf, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReadFile - reads a file at remote path and fills the buffer.
|
// 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() {
|
defer func() {
|
||||||
if r := recover(); r != nil {
|
if r := recover(); r != nil {
|
||||||
// Recover any panic from allocation, and return error.
|
|
||||||
err = bytes.ErrTooLarge
|
err = bytes.ErrTooLarge
|
||||||
}
|
}
|
||||||
}() // Do not crash the server.
|
}()
|
||||||
|
|
||||||
args := ReadFileArgs{
|
args := ReadFileArgs{
|
||||||
Vol: volume,
|
Vol: volume,
|
||||||
@ -269,46 +235,90 @@ func (n *networkStorage) ReadFile(volume string, path string, offset int64, buff
|
|||||||
args.ExpectedHash = verifier.sum
|
args.ExpectedHash = verifier.sum
|
||||||
args.Verified = verifier.IsVerified()
|
args.Verified = verifier.IsVerified()
|
||||||
}
|
}
|
||||||
|
var reply []byte
|
||||||
|
|
||||||
var result []byte
|
err = client.call(storageServiceName+".ReadFile", &args, &reply)
|
||||||
err = n.call("Storage.ReadFileHandler", &args, &result)
|
|
||||||
|
|
||||||
// Copy results to buffer.
|
// Copy reply to buffer.
|
||||||
copy(buffer, result)
|
copy(buffer, reply)
|
||||||
|
|
||||||
// Return length of result, err if any.
|
// Return length of result, err if any.
|
||||||
return int64(len(result)), err
|
return int64(len(reply)), err
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListDir - list all entries at prefix.
|
// ListDir - list all entries at prefix.
|
||||||
func (n *networkStorage) ListDir(volume, path string, count int) (entries []string, err error) {
|
func (client *StorageRPCClient) ListDir(volume, path string, count int) (entries []string, err error) {
|
||||||
if err = n.call("Storage.ListDirHandler", &ListDirArgs{
|
err = client.call(storageServiceName+".ListDir", &ListDirArgs{Vol: volume, Path: path, Count: count}, &entries)
|
||||||
Vol: volume,
|
return entries, err
|
||||||
Path: path,
|
|
||||||
Count: count,
|
|
||||||
}, &entries); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
// Return successfully unmarshalled results.
|
|
||||||
return entries, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// DeleteFile - Delete a file at path.
|
// DeleteFile - Delete a file at path.
|
||||||
func (n *networkStorage) DeleteFile(volume, path string) (err error) {
|
func (client *StorageRPCClient) DeleteFile(volume, path string) (err error) {
|
||||||
reply := AuthRPCReply{}
|
args := DeleteFileArgs{
|
||||||
return n.call("Storage.DeleteFileHandler", &DeleteFileArgs{
|
|
||||||
Vol: volume,
|
Vol: volume,
|
||||||
Path: path,
|
Path: path,
|
||||||
}, &reply)
|
}
|
||||||
|
reply := VoidReply{}
|
||||||
|
|
||||||
|
return client.call(storageServiceName+".DeleteFile", &args, &reply)
|
||||||
}
|
}
|
||||||
|
|
||||||
// RenameFile - rename a remote file from source to destination.
|
// RenameFile - rename a remote file from source to destination.
|
||||||
func (n *networkStorage) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
|
func (client *StorageRPCClient) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err error) {
|
||||||
reply := AuthRPCReply{}
|
args := RenameFileArgs{
|
||||||
return n.call("Storage.RenameFileHandler", &RenameFileArgs{
|
|
||||||
SrcVol: srcVolume,
|
SrcVol: srcVolume,
|
||||||
SrcPath: srcPath,
|
SrcPath: srcPath,
|
||||||
DstVol: dstVolume,
|
DstVol: dstVolume,
|
||||||
DstPath: dstPath,
|
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"
|
"context"
|
||||||
"io"
|
"io"
|
||||||
"path"
|
"path"
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/gorilla/mux"
|
"github.com/gorilla/mux"
|
||||||
"github.com/minio/minio/cmd/logger"
|
"github.com/minio/minio/cmd/logger"
|
||||||
|
xrpc "github.com/minio/minio/cmd/rpc"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Storage server implements rpc primitives to facilitate exporting a
|
const storageServiceName = "Storage"
|
||||||
// disk over a network.
|
const storageServiceSubPath = "/storage"
|
||||||
type storageServer struct {
|
|
||||||
AuthRPCServer
|
var storageServicePath = path.Join(minioReservedBucketPath, storageServiceSubPath)
|
||||||
storage StorageAPI
|
|
||||||
path string
|
// storageRPCReceiver - Storage RPC receiver for storage RPC server
|
||||||
timestamp time.Time
|
type storageRPCReceiver struct {
|
||||||
|
local *posix
|
||||||
|
}
|
||||||
|
|
||||||
|
// VolArgs - generic volume args.
|
||||||
|
type VolArgs struct {
|
||||||
|
AuthArgs
|
||||||
|
Vol string
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Storage operations handlers.
|
/// Storage operations handlers.
|
||||||
|
|
||||||
// DiskInfoHandler - disk info handler is rpc wrapper for DiskInfo operation.
|
// Connect - authenticates remote connection.
|
||||||
func (s *storageServer) DiskInfoHandler(args *AuthRPCArgs, reply *DiskInfo) error {
|
func (receiver *storageRPCReceiver) Connect(args *AuthArgs, reply *VoidReply) (err error) {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return args.Authenticate()
|
||||||
return err
|
}
|
||||||
}
|
|
||||||
|
|
||||||
info, err := s.storage.DiskInfo()
|
// DiskInfo - disk info handler is rpc wrapper for DiskInfo operation.
|
||||||
*reply = info
|
func (receiver *storageRPCReceiver) DiskInfo(args *AuthArgs, reply *DiskInfo) (err error) {
|
||||||
|
*reply, err = receiver.local.DiskInfo()
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Volume operations handlers.
|
/// Volume operations handlers.
|
||||||
|
|
||||||
// MakeVolHandler - make vol handler is rpc wrapper for MakeVol operation.
|
// MakeVol - make vol handler is rpc wrapper for MakeVol operation.
|
||||||
func (s *storageServer) MakeVolHandler(args *GenericVolArgs, reply *AuthRPCReply) error {
|
func (receiver *storageRPCReceiver) MakeVol(args *VolArgs, reply *VoidReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return receiver.local.MakeVol(args.Vol)
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
return s.storage.MakeVol(args.Vol)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListVolsHandler - list vols handler is rpc wrapper for ListVols operation.
|
// ListVols - list vols handler is rpc wrapper for ListVols operation.
|
||||||
func (s *storageServer) ListVolsHandler(args *AuthRPCArgs, reply *ListVolsReply) error {
|
func (receiver *storageRPCReceiver) ListVols(args *AuthArgs, reply *[]VolInfo) (err error) {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
*reply, err = receiver.local.ListVols()
|
||||||
return err
|
return err
|
||||||
}
|
|
||||||
|
|
||||||
vols, err := s.storage.ListVols()
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
reply.Vols = vols
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// StatVolHandler - stat vol handler is a rpc wrapper for StatVol operation.
|
// StatVol - stat vol handler is a rpc wrapper for StatVol operation.
|
||||||
func (s *storageServer) StatVolHandler(args *GenericVolArgs, reply *VolInfo) error {
|
func (receiver *storageRPCReceiver) StatVol(args *VolArgs, reply *VolInfo) (err error) {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
*reply, err = receiver.local.StatVol(args.Vol)
|
||||||
return err
|
return err
|
||||||
}
|
|
||||||
|
|
||||||
volInfo, err := s.storage.StatVol(args.Vol)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply = volInfo
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// DeleteVolHandler - delete vol handler is a rpc wrapper for
|
// DeleteVol - delete vol handler is a rpc wrapper for
|
||||||
// DeleteVol operation.
|
// DeleteVol operation.
|
||||||
func (s *storageServer) DeleteVolHandler(args *GenericVolArgs, reply *AuthRPCReply) error {
|
func (receiver *storageRPCReceiver) DeleteVol(args *VolArgs, reply *VoidReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return receiver.local.DeleteVol(args.Vol)
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
return s.storage.DeleteVol(args.Vol)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// File operations
|
/// File operations
|
||||||
|
|
||||||
// StatFileHandler - stat file handler is rpc wrapper to stat file.
|
// StatFileArgs represents stat file RPC arguments.
|
||||||
func (s *storageServer) StatFileHandler(args *StatFileArgs, reply *FileInfo) error {
|
type StatFileArgs struct {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
AuthArgs
|
||||||
return err
|
Vol string
|
||||||
}
|
Path string
|
||||||
|
|
||||||
fileInfo, err := s.storage.StatFile(args.Vol, args.Path)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply = fileInfo
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListDirHandler - list directory handler is rpc wrapper to list dir.
|
// StatFile - stat file handler is rpc wrapper to stat file.
|
||||||
func (s *storageServer) ListDirHandler(args *ListDirArgs, reply *[]string) error {
|
func (receiver *storageRPCReceiver) StatFile(args *StatFileArgs, reply *FileInfo) (err error) {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
*reply, err = receiver.local.StatFile(args.Vol, args.Path)
|
||||||
return err
|
return err
|
||||||
}
|
|
||||||
|
|
||||||
entries, err := s.storage.ListDir(args.Vol, args.Path, args.Count)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply = entries
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReadAllHandler - read all handler is rpc wrapper to read all storage API.
|
// ListDirArgs represents list contents RPC arguments.
|
||||||
func (s *storageServer) ReadAllHandler(args *ReadFileArgs, reply *[]byte) error {
|
type ListDirArgs struct {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
AuthArgs
|
||||||
return err
|
Vol string
|
||||||
}
|
Path string
|
||||||
|
Count int
|
||||||
buf, err := s.storage.ReadAll(args.Vol, args.Path)
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
*reply = buf
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ReadFileHandler - read file handler is rpc wrapper to read file.
|
// ListDir - list directory handler is rpc wrapper to list dir.
|
||||||
func (s *storageServer) ReadFileHandler(args *ReadFileArgs, reply *[]byte) (err error) {
|
func (receiver *storageRPCReceiver) ListDir(args *ListDirArgs, reply *[]string) (err error) {
|
||||||
if err = args.IsAuthenticated(); err != nil {
|
*reply, err = receiver.local.ListDir(args.Vol, args.Path, args.Count)
|
||||||
return err
|
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
|
var verifier *BitrotVerifier
|
||||||
if !args.Verified {
|
if !args.Verified {
|
||||||
verifier = NewBitrotVerifier(args.Algo, args.ExpectedHash)
|
verifier = NewBitrotVerifier(args.Algo, args.ExpectedHash)
|
||||||
}
|
}
|
||||||
|
|
||||||
var n int64
|
n, err := receiver.local.ReadFile(args.Vol, args.Path, args.Offset, args.Buffer, verifier)
|
||||||
n, err = s.storage.ReadFile(args.Vol, args.Path, args.Offset, args.Buffer, verifier)
|
// Ignore io.ErrEnexpectedEOF for short reads i.e. less content available than requested.
|
||||||
// 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.
|
|
||||||
if err == io.ErrUnexpectedEOF {
|
if err == io.ErrUnexpectedEOF {
|
||||||
// Reset to nil as good condition.
|
|
||||||
err = nil
|
err = nil
|
||||||
}
|
}
|
||||||
|
|
||||||
*reply = args.Buffer[0:n]
|
*reply = args.Buffer[0:n]
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
// PrepareFileHandler - prepare file handler is rpc wrapper to prepare file.
|
// PrepareFileArgs represents append file RPC arguments.
|
||||||
func (s *storageServer) PrepareFileHandler(args *PrepareFileArgs, reply *AuthRPCReply) error {
|
type PrepareFileArgs struct {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
AuthArgs
|
||||||
return err
|
Vol string
|
||||||
}
|
Path string
|
||||||
|
Size int64
|
||||||
return s.storage.PrepareFile(args.Vol, args.Path, args.Size)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// AppendFileHandler - append file handler is rpc wrapper to append file.
|
// PrepareFile - prepare file handler is rpc wrapper to prepare file.
|
||||||
func (s *storageServer) AppendFileHandler(args *AppendFileArgs, reply *AuthRPCReply) error {
|
func (receiver *storageRPCReceiver) PrepareFile(args *PrepareFileArgs, reply *VoidReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return receiver.local.PrepareFile(args.Vol, args.Path, args.Size)
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
return s.storage.AppendFile(args.Vol, args.Path, args.Buffer)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// DeleteFileHandler - delete file handler is rpc wrapper to delete file.
|
// AppendFileArgs represents append file RPC arguments.
|
||||||
func (s *storageServer) DeleteFileHandler(args *DeleteFileArgs, reply *AuthRPCReply) error {
|
type AppendFileArgs struct {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
AuthArgs
|
||||||
return err
|
Vol string
|
||||||
}
|
Path string
|
||||||
|
Buffer []byte
|
||||||
return s.storage.DeleteFile(args.Vol, args.Path)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// RenameFileHandler - rename file handler is rpc wrapper to rename file.
|
// AppendFile - append file handler is rpc wrapper to append file.
|
||||||
func (s *storageServer) RenameFileHandler(args *RenameFileArgs, reply *AuthRPCReply) error {
|
func (receiver *storageRPCReceiver) AppendFile(args *AppendFileArgs, reply *VoidReply) error {
|
||||||
if err := args.IsAuthenticated(); err != nil {
|
return receiver.local.AppendFile(args.Vol, args.Path, args.Buffer)
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
return s.storage.RenameFile(args.SrcVol, args.SrcPath, args.DstVol, args.DstPath)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize new storage rpc.
|
// DeleteFileArgs represents delete file RPC arguments.
|
||||||
func newStorageRPCServer(endpoints EndpointList) (servers []*storageServer, err error) {
|
type DeleteFileArgs struct {
|
||||||
for _, endpoint := range endpoints {
|
AuthArgs
|
||||||
if endpoint.IsLocal {
|
Vol string
|
||||||
storage, err := newPosix(endpoint.Path)
|
Path string
|
||||||
if err != nil && err != errDiskNotFound {
|
}
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
servers = append(servers, &storageServer{
|
// DeleteFile - delete file handler is rpc wrapper to delete file.
|
||||||
storage: storage,
|
func (receiver *storageRPCReceiver) DeleteFile(args *DeleteFileArgs, reply *VoidReply) error {
|
||||||
path: endpoint.Path,
|
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.
|
// registerStorageRPCRouter - register storage rpc router.
|
||||||
func registerStorageRPCRouters(router *mux.Router, endpoints EndpointList) error {
|
func registerStorageRPCRouters(router *mux.Router, endpoints EndpointList) {
|
||||||
// Initialize storage rpc servers for every disk that is hosted on this node.
|
for _, endpoint := range endpoints {
|
||||||
storageRPCs, err := newStorageRPCServer(endpoints)
|
if endpoint.IsLocal {
|
||||||
if err != nil {
|
rpcServer, err := NewStorageRPCServer(endpoint.Path)
|
||||||
logger.LogIf(context.Background(), err)
|
logger.CriticalIf(context.Background(), err)
|
||||||
return err
|
subrouter := router.PathPrefix(minioReservedBucketPath).Subrouter()
|
||||||
}
|
subrouter.Path(path.Join(storageServiceSubPath, endpoint.Path)).Handler(rpcServer)
|
||||||
|
|
||||||
// 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
|
|
||||||
}
|
}
|
||||||
// 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)
|
|
||||||
}
|
|
701
cmd/storage-rpc_test.go
Normal file
701
cmd/storage-rpc_test.go
Normal file
@ -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.
|
// need API layer to send requests, etc.
|
||||||
registerAPIRouter(router)
|
registerAPIRouter(router)
|
||||||
// module being tested is Peer RPCs router.
|
// module being tested is Peer RPCs router.
|
||||||
registerS3PeerRPCRouter(router)
|
registerPeerRPCRouter(router)
|
||||||
|
|
||||||
// Run TestServer.
|
// Run TestServer.
|
||||||
testRPCServer.Server = httptest.NewServer(router)
|
testRPCServer.Server = httptest.NewServer(router)
|
||||||
@ -2265,35 +2265,6 @@ func initTestWebRPCEndPoint(objLayer ObjectLayer) http.Handler {
|
|||||||
return muxRouter
|
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) {
|
func StartTestS3PeerRPCServer(t TestErrHandler) (TestServer, []string) {
|
||||||
root, err := newTestConfig(globalMinioDefaultRegion)
|
root, err := newTestConfig(globalMinioDefaultRegion)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -2323,10 +2294,7 @@ func StartTestS3PeerRPCServer(t TestErrHandler) (TestServer, []string) {
|
|||||||
|
|
||||||
// Register router on a new mux
|
// Register router on a new mux
|
||||||
muxRouter := mux.NewRouter().SkipClean(true)
|
muxRouter := mux.NewRouter().SkipClean(true)
|
||||||
err = registerS3PeerRPCRouter(muxRouter)
|
registerPeerRPCRouter(muxRouter)
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("%s", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Initialize and run the TestServer.
|
// Initialize and run the TestServer.
|
||||||
testRPCServer.Server = httptest.NewServer(muxRouter)
|
testRPCServer.Server = httptest.NewServer(muxRouter)
|
||||||
|
@ -26,9 +26,6 @@ var errInvalidArgument = errors.New("Invalid arguments specified")
|
|||||||
// errSignatureMismatch means signature did not match.
|
// errSignatureMismatch means signature did not match.
|
||||||
var errSignatureMismatch = errors.New("Signature does 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
|
// used when we deal with data larger than expected
|
||||||
var errSizeUnexpected = errors.New("Data size 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.
|
// 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()
|
objectAPI := web.ObjectAPI()
|
||||||
if objectAPI == nil {
|
if objectAPI == nil {
|
||||||
return toJSONError(errServerNotInitialized)
|
return toJSONError(errServerNotInitialized)
|
||||||
@ -470,53 +470,29 @@ func (web *webAPIHandlers) SetAuth(r *http.Request, args *SetAuthArgs, reply *Se
|
|||||||
globalServerConfigMu.Lock()
|
globalServerConfigMu.Lock()
|
||||||
defer globalServerConfigMu.Unlock()
|
defer globalServerConfigMu.Unlock()
|
||||||
|
|
||||||
// Notify all other Minio peers to update credentials
|
// Update credentials in memory
|
||||||
errsMap := updateCredsOnPeers(creds)
|
|
||||||
|
|
||||||
// Update local credentials
|
|
||||||
prevCred := globalServerConfig.SetCredential(creds)
|
prevCred := globalServerConfig.SetCredential(creds)
|
||||||
|
|
||||||
// Persist updated credentials.
|
// Save credentials to config file
|
||||||
if err = globalServerConfig.Save(); err != nil {
|
if err := globalServerConfig.Save(); err != nil {
|
||||||
// Save the current creds when failed to update.
|
// As saving configurstion failed, restore previous credential in memory.
|
||||||
globalServerConfig.SetCredential(prevCred)
|
globalServerConfig.SetCredential(prevCred)
|
||||||
|
logger.LogIf(context.Background(), err)
|
||||||
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(),
|
|
||||||
)
|
|
||||||
}
|
|
||||||
|
|
||||||
return toJSONError(err)
|
return toJSONError(err)
|
||||||
}
|
}
|
||||||
|
|
||||||
reply.Token = token
|
if errs := globalNotificationSys.SetCredentials(creds); len(errs) != 0 {
|
||||||
reply.UIVersion = browser.UIVersion
|
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
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -23,7 +23,6 @@ import (
|
|||||||
"crypto/md5"
|
"crypto/md5"
|
||||||
"encoding/hex"
|
"encoding/hex"
|
||||||
"encoding/json"
|
"encoding/json"
|
||||||
"errors"
|
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
@ -130,25 +129,7 @@ func TestWriteWebErrorResponse(t *testing.T) {
|
|||||||
|
|
||||||
// Authenticate and get JWT token - will be called before every webrpc handler invocation
|
// 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) {
|
func getWebRPCToken(apiRouter http.Handler, accessKey, secretKey string) (token string, err error) {
|
||||||
rec := httptest.NewRecorder()
|
return authenticateWeb(accessKey, secretKey)
|
||||||
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
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wrapper for calling Login Web Handler
|
// Wrapper for calling Login Web Handler
|
||||||
@ -209,8 +190,8 @@ func testStorageInfoWebHandler(obj ObjectLayer, instanceType string, t TestErrHa
|
|||||||
|
|
||||||
rec := httptest.NewRecorder()
|
rec := httptest.NewRecorder()
|
||||||
|
|
||||||
storageInfoRequest := AuthRPCArgs{
|
storageInfoRequest := AuthArgs{
|
||||||
Version: globalRPCAPIVersion,
|
RPCVersion: globalRPCAPIVersion,
|
||||||
}
|
}
|
||||||
storageInfoReply := &StorageInfoRep{}
|
storageInfoReply := &StorageInfoRep{}
|
||||||
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
||||||
@ -244,8 +225,8 @@ func testServerInfoWebHandler(obj ObjectLayer, instanceType string, t TestErrHan
|
|||||||
|
|
||||||
rec := httptest.NewRecorder()
|
rec := httptest.NewRecorder()
|
||||||
|
|
||||||
serverInfoRequest := AuthRPCArgs{
|
serverInfoRequest := AuthArgs{
|
||||||
Version: globalRPCAPIVersion,
|
RPCVersion: globalRPCAPIVersion,
|
||||||
}
|
}
|
||||||
serverInfoReply := &ServerInfoRep{}
|
serverInfoReply := &ServerInfoRep{}
|
||||||
req, err := newTestWebRPCRequest("Web.ServerInfo", authorization, serverInfoRequest)
|
req, err := newTestWebRPCRequest("Web.ServerInfo", authorization, serverInfoRequest)
|
||||||
@ -1545,8 +1526,8 @@ func TestWebCheckAuthorization(t *testing.T) {
|
|||||||
"PresignedGet",
|
"PresignedGet",
|
||||||
}
|
}
|
||||||
for _, rpcCall := range webRPCs {
|
for _, rpcCall := range webRPCs {
|
||||||
args := &AuthRPCArgs{
|
args := &AuthArgs{
|
||||||
Version: globalRPCAPIVersion,
|
RPCVersion: globalRPCAPIVersion,
|
||||||
}
|
}
|
||||||
reply := &WebGenericRep{}
|
reply := &WebGenericRep{}
|
||||||
req, nerr := newTestWebRPCRequest("Web."+rpcCall, "Bearer fooauthorization", args)
|
req, nerr := newTestWebRPCRequest("Web."+rpcCall, "Bearer fooauthorization", args)
|
||||||
@ -1631,8 +1612,8 @@ func TestWebObjectLayerNotReady(t *testing.T) {
|
|||||||
webRPCs := []string{"StorageInfo", "MakeBucket", "ListBuckets", "ListObjects", "RemoveObject",
|
webRPCs := []string{"StorageInfo", "MakeBucket", "ListBuckets", "ListObjects", "RemoveObject",
|
||||||
"GetBucketPolicy", "SetBucketPolicy", "ListAllBucketPolicies"}
|
"GetBucketPolicy", "SetBucketPolicy", "ListAllBucketPolicies"}
|
||||||
for _, rpcCall := range webRPCs {
|
for _, rpcCall := range webRPCs {
|
||||||
args := &AuthRPCArgs{
|
args := &AuthArgs{
|
||||||
Version: globalRPCAPIVersion,
|
RPCVersion: globalRPCAPIVersion,
|
||||||
}
|
}
|
||||||
reply := &WebGenericRep{}
|
reply := &WebGenericRep{}
|
||||||
req, nerr := newTestWebRPCRequest("Web."+rpcCall, authorization, args)
|
req, nerr := newTestWebRPCRequest("Web."+rpcCall, authorization, args)
|
||||||
@ -1746,7 +1727,7 @@ func TestWebObjectLayerFaultyDisks(t *testing.T) {
|
|||||||
RepArgs interface{}
|
RepArgs interface{}
|
||||||
}{
|
}{
|
||||||
{"MakeBucket", MakeBucketArgs{BucketName: bucketName}, WebGenericRep{}},
|
{"MakeBucket", MakeBucketArgs{BucketName: bucketName}, WebGenericRep{}},
|
||||||
{"ListBuckets", AuthRPCArgs{Version: globalRPCAPIVersion}, ListBucketsRep{}},
|
{"ListBuckets", AuthArgs{RPCVersion: globalRPCAPIVersion}, ListBucketsRep{}},
|
||||||
{"ListObjects", ListObjectsArgs{BucketName: bucketName, Prefix: ""}, ListObjectsRep{}},
|
{"ListObjects", ListObjectsArgs{BucketName: bucketName, Prefix: ""}, ListObjectsRep{}},
|
||||||
{"GetBucketPolicy", GetBucketPolicyArgs{BucketName: bucketName, Prefix: ""}, GetBucketPolicyRep{}},
|
{"GetBucketPolicy", GetBucketPolicyArgs{BucketName: bucketName, Prefix: ""}, GetBucketPolicyRep{}},
|
||||||
{"SetBucketPolicy", SetBucketPolicyWebArgs{BucketName: bucketName, Prefix: "", Policy: "none"}, WebGenericRep{}},
|
{"SetBucketPolicy", SetBucketPolicyWebArgs{BucketName: bucketName, Prefix: "", Policy: "none"}, WebGenericRep{}},
|
||||||
@ -1770,8 +1751,8 @@ func TestWebObjectLayerFaultyDisks(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Test Web.StorageInfo
|
// Test Web.StorageInfo
|
||||||
storageInfoRequest := AuthRPCArgs{
|
storageInfoRequest := AuthArgs{
|
||||||
Version: globalRPCAPIVersion,
|
RPCVersion: globalRPCAPIVersion,
|
||||||
}
|
}
|
||||||
storageInfoReply := &StorageInfoRep{}
|
storageInfoReply := &StorageInfoRep{}
|
||||||
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
req, err := newTestWebRPCRequest("Web.StorageInfo", authorization, storageInfoRequest)
|
||||||
|
Loading…
x
Reference in New Issue
Block a user