perf: websocket grid connectivity for all internode communication (#18461)

This PR adds a WebSocket grid feature that allows servers to communicate via 
a single two-way connection.

There are two request types:

* Single requests, which are `[]byte => ([]byte, error)`. This is for efficient small
  roundtrips with small payloads.

* Streaming requests which are `[]byte, chan []byte => chan []byte (and error)`,
  which allows for different combinations of full two-way streams with an initial payload.

Only a single stream is created between two machines - and there is, as such, no
server/client relation since both sides can initiate and handle requests. Which server
initiates the request is decided deterministically on the server names.

Requests are made through a mux client and server, which handles message
passing, congestion, cancelation, timeouts, etc.

If a connection is lost, all requests are canceled, and the calling server will try
to reconnect. Registered handlers can operate directly on byte 
slices or use a higher-level generics abstraction.

There is no versioning of handlers/clients, and incompatible changes should
be handled by adding new handlers.

The request path can be changed to a new one for any protocol changes.

First, all servers create a "Manager." The manager must know its address 
as well as all remote addresses. This will manage all connections.
To get a connection to any remote, ask the manager to provide it given
the remote address using.

```
func (m *Manager) Connection(host string) *Connection
```

All serverside handlers must also be registered on the manager. This will
make sure that all incoming requests are served. The number of in-flight 
requests and responses must also be given for streaming requests.

The "Connection" returned manages the mux-clients. Requests issued
to the connection will be sent to the remote.

* `func (c *Connection) Request(ctx context.Context, h HandlerID, req []byte) ([]byte, error)`
   performs a single request and returns the result. Any deadline provided on the request is
   forwarded to the server, and canceling the context will make the function return at once.

* `func (c *Connection) NewStream(ctx context.Context, h HandlerID, payload []byte) (st *Stream, err error)`
   will initiate a remote call and send the initial payload.

```Go
// A Stream is a two-way stream.
// All responses *must* be read by the caller.
// If the call is canceled through the context,
//The appropriate error will be returned.
type Stream struct {
	// Responses from the remote server.
	// Channel will be closed after an error or when the remote closes.
	// All responses *must* be read by the caller until either an error is returned or the channel is closed.
	// Canceling the context will cause the context cancellation error to be returned.
	Responses <-chan Response

	// Requests sent to the server.
	// If the handler is defined with 0 incoming capacity this will be nil.
	// Channel *must* be closed to signal the end of the stream.
	// If the request context is canceled, the stream will no longer process requests.
	Requests chan<- []byte
}

type Response struct {
	Msg []byte
	Err error
}
```

There are generic versions of the server/client handlers that allow the use of type
safe implementations for data types that support msgpack marshal/unmarshal.
This commit is contained in:
Klaus Post
2023-11-20 17:09:35 -08:00
committed by GitHub
parent 8bedb419a9
commit 51aa59a737
65 changed files with 13519 additions and 1022 deletions

View File

@@ -96,6 +96,11 @@ func (endpoint Endpoint) HTTPS() bool {
return endpoint.Scheme == "https"
}
// GridHost returns the host to be used for grid connections.
func (endpoint Endpoint) GridHost() string {
return fmt.Sprintf("%s://%s", endpoint.Scheme, endpoint.Host)
}
// UpdateIsLocal - resolves the host and updates if it is local or not.
func (endpoint *Endpoint) UpdateIsLocal() (err error) {
if !endpoint.IsLocal {

View File

@@ -31,6 +31,7 @@ import (
"github.com/minio/minio/internal/color"
"github.com/minio/minio/internal/config"
"github.com/minio/minio/internal/config/storageclass"
"github.com/minio/minio/internal/grid"
xioutil "github.com/minio/minio/internal/ioutil"
"github.com/minio/minio/internal/logger"
"github.com/minio/pkg/v2/sync/errgroup"
@@ -388,6 +389,12 @@ func saveFormatErasure(disk StorageAPI, format *formatErasureV3, healID string)
// loadFormatErasure - loads format.json from disk.
func loadFormatErasure(disk StorageAPI) (format *formatErasureV3, err error) {
// Ensure that the grid is online.
if _, err := disk.DiskInfo(context.Background(), false); err != nil {
if errors.Is(err, grid.ErrDisconnected) {
return nil, err
}
}
buf, err := disk.ReadAll(context.TODO(), minioMetaBucket, formatConfigFile)
if err != nil {
// 'file not found' and 'volume not found' as

View File

@@ -31,6 +31,7 @@ import (
"github.com/dustin/go-humanize"
"github.com/minio/minio-go/v7/pkg/s3utils"
"github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio/internal/grid"
xnet "github.com/minio/pkg/v2/net"
"github.com/minio/minio/internal/amztime"
@@ -240,6 +241,10 @@ func guessIsRPCReq(req *http.Request) bool {
if req == nil {
return false
}
if req.Method == http.MethodGet && req.URL != nil && req.URL.Path == grid.RoutePath {
return true
}
return req.Method == http.MethodPost &&
strings.HasPrefix(req.URL.Path, minioReservedBucketPath+SlashSeparator)
}

View File

@@ -25,6 +25,7 @@ import (
"testing"
"github.com/minio/minio/internal/crypto"
"github.com/minio/minio/internal/grid"
xhttp "github.com/minio/minio/internal/http"
)
@@ -54,6 +55,14 @@ func TestGuessIsRPC(t *testing.T) {
if guessIsRPCReq(r) {
t.Fatal("Test shouldn't report as net/rpc for a non net/rpc request.")
}
r = &http.Request{
Proto: "HTTP/1.1",
Method: http.MethodGet,
URL: &url.URL{Path: grid.RoutePath},
}
if !guessIsRPCReq(r) {
t.Fatal("Grid RPC path not detected")
}
}
var isHTTPHeaderSizeTooLargeTests = []struct {

83
cmd/grid.go Normal file
View File

@@ -0,0 +1,83 @@
// Copyright (c) 2015-2023 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package cmd
import (
"context"
"crypto/tls"
"sync/atomic"
"github.com/minio/minio-go/v7/pkg/set"
"github.com/minio/minio/internal/fips"
"github.com/minio/minio/internal/grid"
xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/rest"
)
// globalGrid is the global grid manager.
var globalGrid atomic.Pointer[grid.Manager]
// globalGridStart is a channel that will block startup of grid connections until closed.
var globalGridStart = make(chan struct{})
func initGlobalGrid(ctx context.Context, eps EndpointServerPools) error {
seenHosts := set.NewStringSet()
var hosts []string
var local string
for _, ep := range eps {
for _, endpoint := range ep.Endpoints {
u := endpoint.GridHost()
if seenHosts.Contains(u) {
continue
}
seenHosts.Add(u)
// Set local endpoint
if endpoint.IsLocal {
local = u
}
hosts = append(hosts, u)
}
}
lookupHost := globalDNSCache.LookupHost
if IsKubernetes() || IsDocker() {
lookupHost = nil
}
g, err := grid.NewManager(ctx, grid.ManagerOptions{
Dialer: grid.ContextDialer(xhttp.DialContextWithLookupHost(lookupHost, xhttp.NewInternodeDialContext(rest.DefaultTimeout, globalTCPOptions))),
Local: local,
Hosts: hosts,
AddAuth: newCachedAuthToken(),
AuthRequest: storageServerRequestValidate,
BlockConnect: globalGridStart,
TLSConfig: &tls.Config{
RootCAs: globalRootCAs,
CipherSuites: fips.TLSCiphers(),
CurvePreferences: fips.TLSCurveIDs(),
},
// Record incoming and outgoing bytes.
Incoming: globalConnStats.incInternodeInputBytes,
Outgoing: globalConnStats.incInternodeOutputBytes,
TraceTo: globalTrace,
})
if err != nil {
return err
}
globalGrid.Store(g)
return nil
}

View File

@@ -389,12 +389,6 @@ func errorResponseHandler(w http.ResponseWriter, r *http.Request) {
Description: desc,
HTTPStatusCode: http.StatusUpgradeRequired,
}, r.URL)
case strings.HasPrefix(r.URL.Path, lockRESTPrefix):
writeErrorResponseString(r.Context(), w, APIError{
Code: "XMinioLockVersionMismatch",
Description: desc,
HTTPStatusCode: http.StatusUpgradeRequired,
}, r.URL)
case strings.HasPrefix(r.URL.Path, adminPathPrefix):
var desc string
version := extractAPIVersion(r)

View File

@@ -55,6 +55,7 @@ func getOpName(name string) (op string) {
op = strings.Replace(op, "(*peerRESTServer)", "peer", 1)
op = strings.Replace(op, "(*lockRESTServer)", "lockR", 1)
op = strings.Replace(op, "(*stsAPIHandlers)", "sts", 1)
op = strings.Replace(op, "(*peerS3Server)", "s3", 1)
op = strings.Replace(op, "ClusterCheckHandler", "health.Cluster", 1)
op = strings.Replace(op, "ClusterReadCheckHandler", "health.ClusterRead", 1)
op = strings.Replace(op, "LivenessCheckHandler", "health.Liveness", 1)

View File

@@ -159,7 +159,7 @@ func (l *localLocker) removeEntry(name string, args dsync.LockArgs, lri *[]lockR
}
func (l *localLocker) RLock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
if len(args.Resources) > 1 {
if len(args.Resources) != 1 {
return false, fmt.Errorf("internal error: localLocker.RLock called with more than one resource")
}

View File

@@ -18,121 +18,86 @@
package cmd
import (
"bytes"
"context"
"io"
"net/url"
"errors"
"github.com/minio/minio/internal/dsync"
xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/rest"
"github.com/minio/minio/internal/grid"
"github.com/minio/minio/internal/logger"
)
// lockRESTClient is authenticable lock REST client
type lockRESTClient struct {
restClient *rest.Client
u *url.URL
}
func toLockError(err error) error {
if err == nil {
return nil
}
switch err.Error() {
case errLockConflict.Error():
return errLockConflict
case errLockNotFound.Error():
return errLockNotFound
}
return err
}
// String stringer *dsync.NetLocker* interface compatible method.
func (client *lockRESTClient) String() string {
return client.u.String()
}
// Wrapper to restClient.Call to handle network errors, in case of network error the connection is marked disconnected
// permanently. The only way to restore the connection is at the xl-sets layer by xlsets.monitorAndConnectEndpoints()
// after verifying format.json
func (client *lockRESTClient) callWithContext(ctx context.Context, method string, values url.Values, body io.Reader, length int64) (respBody io.ReadCloser, err error) {
if values == nil {
values = make(url.Values)
}
respBody, err = client.restClient.Call(ctx, method, values, body, length)
if err == nil {
return respBody, nil
}
return nil, toLockError(err)
connection *grid.Connection
}
// IsOnline - returns whether REST client failed to connect or not.
func (client *lockRESTClient) IsOnline() bool {
return client.restClient.IsOnline()
func (c *lockRESTClient) IsOnline() bool {
return c.connection.State() == grid.StateConnected
}
// Not a local locker
func (client *lockRESTClient) IsLocal() bool {
func (c *lockRESTClient) IsLocal() bool {
return false
}
// Close - marks the client as closed.
func (client *lockRESTClient) Close() error {
client.restClient.Close()
func (c *lockRESTClient) Close() error {
return nil
}
// restCall makes a call to the lock REST server.
func (client *lockRESTClient) restCall(ctx context.Context, call string, args dsync.LockArgs) (reply bool, err error) {
argsBytes, err := args.MarshalMsg(metaDataPoolGet()[:0])
// String - returns the remote host of the connection.
func (c *lockRESTClient) String() string {
return c.connection.Remote
}
func (c *lockRESTClient) call(ctx context.Context, h *grid.SingleHandler[*dsync.LockArgs, *dsync.LockResp], args *dsync.LockArgs) (ok bool, err error) {
r, err := h.Call(ctx, c.connection, args)
if err != nil {
logger.LogIfNot(ctx, err, grid.ErrDisconnected)
return false, err
}
defer metaDataPoolPut(argsBytes)
body := bytes.NewReader(argsBytes)
respBody, err := client.callWithContext(ctx, call, nil, body, body.Size())
defer xhttp.DrainBody(respBody)
switch err {
case nil:
return true, nil
case errLockConflict, errLockNotFound:
return false, nil
defer h.PutResponse(r)
ok = r.Code == dsync.RespOK
switch r.Code {
case dsync.RespLockConflict, dsync.RespLockNotFound, dsync.RespOK:
// no error
case dsync.RespLockNotInitialized:
err = errLockNotInitialized
default:
return false, err
err = errors.New(r.Err)
}
return ok, err
}
// RLock calls read lock REST API.
func (client *lockRESTClient) RLock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodRLock, args)
func (c *lockRESTClient) RLock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCRLock, &args)
}
// Lock calls lock REST API.
func (client *lockRESTClient) Lock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodLock, args)
func (c *lockRESTClient) Lock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCLock, &args)
}
// RUnlock calls read unlock REST API.
func (client *lockRESTClient) RUnlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodRUnlock, args)
func (c *lockRESTClient) RUnlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCRUnlock, &args)
}
// RUnlock calls read unlock REST API.
func (client *lockRESTClient) Refresh(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodRefresh, args)
// Refresh calls Refresh REST API.
func (c *lockRESTClient) Refresh(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCRefresh, &args)
}
// Unlock calls write unlock RPC.
func (client *lockRESTClient) Unlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodUnlock, args)
func (c *lockRESTClient) Unlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCUnlock, &args)
}
// ForceUnlock calls force unlock handler to forcibly unlock an active lock.
func (client *lockRESTClient) ForceUnlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return client.restCall(ctx, lockRESTMethodForceUnlock, args)
func (c *lockRESTClient) ForceUnlock(ctx context.Context, args dsync.LockArgs) (reply bool, err error) {
return c.call(ctx, lockRPCForceUnlock, &args)
}
func newLockAPI(endpoint Endpoint) dsync.NetLocker {
@@ -143,27 +108,6 @@ func newLockAPI(endpoint Endpoint) dsync.NetLocker {
}
// Returns a lock rest client.
func newlockRESTClient(endpoint Endpoint) *lockRESTClient {
serverURL := &url.URL{
Scheme: endpoint.Scheme,
Host: endpoint.Host,
Path: pathJoin(lockRESTPrefix, lockRESTVersion),
}
restClient := rest.NewClient(serverURL, globalInternodeTransport, newCachedAuthToken())
// Use a separate client to avoid recursive calls.
healthClient := rest.NewClient(serverURL, globalInternodeTransport, newCachedAuthToken())
healthClient.NoMetrics = true
restClient.HealthCheckFn = func() bool {
ctx, cancel := context.WithTimeout(context.Background(), restClient.HealthCheckTimeout)
defer cancel()
respBody, err := healthClient.Call(ctx, lockRESTMethodHealth, nil, nil, -1)
xhttp.DrainBody(respBody)
return !isNetworkError(err)
}
return &lockRESTClient{u: &url.URL{
Scheme: endpoint.Scheme,
Host: endpoint.Host,
}, restClient: restClient}
func newlockRESTClient(ep Endpoint) *lockRESTClient {
return &lockRESTClient{globalGrid.Load().Connection(ep.GridHost())}
}

View File

@@ -26,14 +26,27 @@ import (
// Tests lock rpc client.
func TestLockRESTlient(t *testing.T) {
endpoint, err := NewEndpoint("http://localhost:9000")
// These should not be connectable.
endpoint, err := NewEndpoint("http://localhost:9876")
if err != nil {
t.Fatalf("unexpected error %v", err)
}
endpointLocal, err := NewEndpoint("http://localhost:9012")
if err != nil {
t.Fatalf("unexpected error %v", err)
}
endpointLocal.IsLocal = true
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
err = initGlobalGrid(ctx, []PoolEndpoints{{Endpoints: Endpoints{endpoint, endpointLocal}}})
if err != nil {
t.Fatal(err)
}
lkClient := newlockRESTClient(endpoint)
if !lkClient.IsOnline() {
t.Fatalf("unexpected error. connection failed")
if lkClient.IsOnline() {
t.Fatalf("unexpected result. connection was online")
}
// Attempt all calls.

View File

@@ -21,22 +21,6 @@ import (
"errors"
)
const (
lockRESTVersion = "v7" // Add msgp for lockArgs
lockRESTVersionPrefix = SlashSeparator + lockRESTVersion
lockRESTPrefix = minioReservedBucketPath + "/lock"
)
const (
lockRESTMethodHealth = "/health"
lockRESTMethodRefresh = "/refresh"
lockRESTMethodLock = "/lock"
lockRESTMethodRLock = "/rlock"
lockRESTMethodUnlock = "/unlock"
lockRESTMethodRUnlock = "/runlock"
lockRESTMethodForceUnlock = "/force-unlock"
)
var (
errLockConflict = errors.New("lock conflict")
errLockNotInitialized = errors.New("lock not initialized")

View File

@@ -19,16 +19,132 @@ package cmd
import (
"context"
"errors"
"io"
"net/http"
"time"
"github.com/dustin/go-humanize"
"github.com/minio/minio/internal/dsync"
"github.com/minio/mux"
"github.com/minio/minio/internal/grid"
"github.com/minio/minio/internal/logger"
)
// To abstract a node over network.
type lockRESTServer struct {
ll *localLocker
}
// RefreshHandler - refresh the current lock
func (l *lockRESTServer) RefreshHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCRefresh.NewResponse()
refreshed, err := l.ll.Refresh(context.Background(), *args)
if err != nil {
return l.makeResp(resp, err)
}
if !refreshed {
return l.makeResp(resp, errLockNotFound)
}
return l.makeResp(resp, err)
}
// LockHandler - Acquires a lock.
func (l *lockRESTServer) LockHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCLock.NewResponse()
success, err := l.ll.Lock(context.Background(), *args)
if err == nil && !success {
return l.makeResp(resp, errLockConflict)
}
return l.makeResp(resp, err)
}
// UnlockHandler - releases the acquired lock.
func (l *lockRESTServer) UnlockHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCUnlock.NewResponse()
_, err := l.ll.Unlock(context.Background(), *args)
// Ignore the Unlock() "reply" return value because if err == nil, "reply" is always true
// Consequently, if err != nil, reply is always false
return l.makeResp(resp, err)
}
// RLockHandler - Acquires an RLock.
func (l *lockRESTServer) RLockHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCRLock.NewResponse()
success, err := l.ll.RLock(context.Background(), *args)
if err == nil && !success {
err = errLockConflict
}
return l.makeResp(resp, err)
}
// RUnlockHandler - releases the acquired read lock.
func (l *lockRESTServer) RUnlockHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCRUnlock.NewResponse()
// Ignore the RUnlock() "reply" return value because if err == nil, "reply" is always true.
// Consequently, if err != nil, reply is always false
_, err := l.ll.RUnlock(context.Background(), *args)
return l.makeResp(resp, err)
}
// ForceUnlockHandler - query expired lock status.
func (l *lockRESTServer) ForceUnlockHandler(args *dsync.LockArgs) (*dsync.LockResp, *grid.RemoteErr) {
resp := lockRPCForceUnlock.NewResponse()
_, err := l.ll.ForceUnlock(context.Background(), *args)
return l.makeResp(resp, err)
}
var (
// Static lock handlers.
// All have the same signature.
lockRPCForceUnlock = newLockHandler(grid.HandlerLockForceUnlock)
lockRPCRefresh = newLockHandler(grid.HandlerLockRefresh)
lockRPCLock = newLockHandler(grid.HandlerLockLock)
lockRPCUnlock = newLockHandler(grid.HandlerLockUnlock)
lockRPCRLock = newLockHandler(grid.HandlerLockRLock)
lockRPCRUnlock = newLockHandler(grid.HandlerLockRUnlock)
)
func newLockHandler(h grid.HandlerID) *grid.SingleHandler[*dsync.LockArgs, *dsync.LockResp] {
return grid.NewSingleHandler[*dsync.LockArgs, *dsync.LockResp](h, func() *dsync.LockArgs {
return &dsync.LockArgs{}
}, func() *dsync.LockResp {
return &dsync.LockResp{}
})
}
// registerLockRESTHandlers - register lock rest router.
func registerLockRESTHandlers() {
lockServer := &lockRESTServer{
ll: newLocker(),
}
logger.FatalIf(lockRPCForceUnlock.Register(globalGrid.Load(), lockServer.ForceUnlockHandler), "unable to register handler")
logger.FatalIf(lockRPCRefresh.Register(globalGrid.Load(), lockServer.RefreshHandler), "unable to register handler")
logger.FatalIf(lockRPCLock.Register(globalGrid.Load(), lockServer.LockHandler), "unable to register handler")
logger.FatalIf(lockRPCUnlock.Register(globalGrid.Load(), lockServer.UnlockHandler), "unable to register handler")
logger.FatalIf(lockRPCRLock.Register(globalGrid.Load(), lockServer.RLockHandler), "unable to register handler")
logger.FatalIf(lockRPCRUnlock.Register(globalGrid.Load(), lockServer.RUnlockHandler), "unable to register handler")
globalLockServer = lockServer.ll
go lockMaintenance(GlobalContext)
}
func (l *lockRESTServer) makeResp(dst *dsync.LockResp, err error) (*dsync.LockResp, *grid.RemoteErr) {
*dst = dsync.LockResp{Code: dsync.RespOK}
switch err {
case nil:
case errLockNotInitialized:
dst.Code = dsync.RespLockNotInitialized
case errLockConflict:
dst.Code = dsync.RespLockConflict
case errLockNotFound:
dst.Code = dsync.RespLockNotFound
default:
dst.Code = dsync.RespErr
dst.Err = err.Error()
}
return dst, nil
}
const (
// Lock maintenance interval.
lockMaintenanceInterval = 1 * time.Minute
@@ -37,185 +153,6 @@ const (
lockValidityDuration = 1 * time.Minute
)
// To abstract a node over network.
type lockRESTServer struct {
ll *localLocker
}
func (l *lockRESTServer) writeErrorResponse(w http.ResponseWriter, err error) {
statusCode := http.StatusForbidden
switch err {
case errLockNotInitialized:
// Return 425 instead of 5xx, otherwise this node will be marked offline
statusCode = http.StatusTooEarly
case errLockConflict:
statusCode = http.StatusConflict
case errLockNotFound:
statusCode = http.StatusNotFound
}
w.WriteHeader(statusCode)
w.Write([]byte(err.Error()))
}
// IsValid - To authenticate and verify the time difference.
func (l *lockRESTServer) IsValid(w http.ResponseWriter, r *http.Request) bool {
if l.ll == nil {
l.writeErrorResponse(w, errLockNotInitialized)
return false
}
if err := storageServerRequestValidate(r); err != nil {
l.writeErrorResponse(w, err)
return false
}
return true
}
func getLockArgs(r *http.Request) (args dsync.LockArgs, err error) {
dec := msgpNewReader(io.LimitReader(r.Body, 1000*humanize.KiByte))
defer readMsgpReaderPoolPut(dec)
err = args.DecodeMsg(dec)
return args, err
}
// HealthHandler returns success if request is authenticated.
func (l *lockRESTServer) HealthHandler(w http.ResponseWriter, r *http.Request) {
l.IsValid(w, r)
}
// RefreshHandler - refresh the current lock
func (l *lockRESTServer) RefreshHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
refreshed, err := l.ll.Refresh(r.Context(), args)
if err != nil {
l.writeErrorResponse(w, err)
return
}
if !refreshed {
l.writeErrorResponse(w, errLockNotFound)
return
}
}
// LockHandler - Acquires a lock.
func (l *lockRESTServer) LockHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
success, err := l.ll.Lock(r.Context(), args)
if err == nil && !success {
err = errLockConflict
}
if err != nil {
l.writeErrorResponse(w, err)
return
}
}
// UnlockHandler - releases the acquired lock.
func (l *lockRESTServer) UnlockHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
_, err = l.ll.Unlock(context.Background(), args)
// Ignore the Unlock() "reply" return value because if err == nil, "reply" is always true
// Consequently, if err != nil, reply is always false
if err != nil {
l.writeErrorResponse(w, err)
return
}
}
// LockHandler - Acquires an RLock.
func (l *lockRESTServer) RLockHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
success, err := l.ll.RLock(r.Context(), args)
if err == nil && !success {
err = errLockConflict
}
if err != nil {
l.writeErrorResponse(w, err)
return
}
}
// RUnlockHandler - releases the acquired read lock.
func (l *lockRESTServer) RUnlockHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
// Ignore the RUnlock() "reply" return value because if err == nil, "reply" is always true.
// Consequently, if err != nil, reply is always false
if _, err = l.ll.RUnlock(context.Background(), args); err != nil {
l.writeErrorResponse(w, err)
return
}
}
// ForceUnlockHandler - query expired lock status.
func (l *lockRESTServer) ForceUnlockHandler(w http.ResponseWriter, r *http.Request) {
if !l.IsValid(w, r) {
l.writeErrorResponse(w, errors.New("invalid request"))
return
}
args, err := getLockArgs(r)
if err != nil {
l.writeErrorResponse(w, err)
return
}
if _, err = l.ll.ForceUnlock(r.Context(), args); err != nil {
l.writeErrorResponse(w, err)
return
}
}
// lockMaintenance loops over all locks and discards locks
// that have not been refreshed for some time.
func lockMaintenance(ctx context.Context) {
@@ -241,27 +178,3 @@ func lockMaintenance(ctx context.Context) {
}
}
}
// registerLockRESTHandlers - register lock rest router.
func registerLockRESTHandlers(router *mux.Router) {
h := func(f http.HandlerFunc) http.HandlerFunc {
return collectInternodeStats(httpTraceHdrs(f))
}
lockServer := &lockRESTServer{
ll: newLocker(),
}
subrouter := router.PathPrefix(lockRESTPrefix).Subrouter()
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodHealth).HandlerFunc(h(lockServer.HealthHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRefresh).HandlerFunc(h(lockServer.RefreshHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodLock).HandlerFunc(h(lockServer.LockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRLock).HandlerFunc(h(lockServer.RLockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodUnlock).HandlerFunc(h(lockServer.UnlockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodRUnlock).HandlerFunc(h(lockServer.RUnlockHandler))
subrouter.Methods(http.MethodPost).Path(lockRESTVersionPrefix + lockRESTMethodForceUnlock).HandlerFunc(h(lockServer.ForceUnlockHandler))
globalLockServer = lockServer.ll
go lockMaintenance(GlobalContext)
}

View File

@@ -1,103 +0,0 @@
// Copyright (c) 2015-2021 MinIO, Inc.
//
// This file is part of MinIO Object Storage stack
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
package cmd
import (
"bufio"
"bytes"
"io"
"net/http"
"net/url"
"sort"
"strconv"
"testing"
"github.com/minio/minio/internal/dsync"
)
func BenchmarkLockArgs(b *testing.B) {
args := dsync.LockArgs{
Owner: "minio",
UID: "uid",
Source: "lockArgs.go",
Quorum: 3,
Resources: []string{"obj.txt"},
}
argBytes, err := args.MarshalMsg(nil)
if err != nil {
b.Fatal(err)
}
req := &http.Request{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
req.Body = io.NopCloser(bytes.NewReader(argBytes))
getLockArgs(req)
}
}
func BenchmarkLockArgsOld(b *testing.B) {
values := url.Values{}
values.Set("owner", "minio")
values.Set("uid", "uid")
values.Set("source", "lockArgs.go")
values.Set("quorum", "3")
req := &http.Request{
Form: values,
}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
req.Body = io.NopCloser(bytes.NewReader([]byte(`obj.txt`)))
getLockArgsOld(req)
}
}
func getLockArgsOld(r *http.Request) (args dsync.LockArgs, err error) {
values := r.Form
quorum, err := strconv.Atoi(values.Get("quorum"))
if err != nil {
return args, err
}
args = dsync.LockArgs{
Owner: values.Get("onwer"),
UID: values.Get("uid"),
Source: values.Get("source"),
Quorum: quorum,
}
var resources []string
bio := bufio.NewScanner(r.Body)
for bio.Scan() {
resources = append(resources, bio.Text())
}
if err := bio.Err(); err != nil {
return args, err
}
sort.Strings(resources)
args.Resources = resources
return args, nil
}

View File

@@ -19,21 +19,18 @@ package cmd
import (
"context"
"fmt"
"io"
"net/http"
"net/url"
"runtime/debug"
"sort"
"strconv"
"strings"
xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/grid"
xioutil "github.com/minio/minio/internal/ioutil"
"github.com/minio/minio/internal/logger"
"github.com/valyala/bytebufferpool"
)
//go:generate msgp -file $GOFILE
// WalkDirOptions provides options for WalkDir operations.
type WalkDirOptions struct {
// Bucket to scanner
@@ -57,6 +54,10 @@ type WalkDirOptions struct {
// Limit the number of returned objects if > 0.
Limit int
// DiskID contains the disk ID of the disk.
// Leave empty to not check disk ID.
DiskID string
}
// WalkDir will traverse a directory and return all entries found.
@@ -387,6 +388,9 @@ func (s *xlStorage) WalkDir(ctx context.Context, opts WalkDirOptions, wr io.Writ
}
func (p *xlStorageDiskIDCheck) WalkDir(ctx context.Context, opts WalkDirOptions, wr io.Writer) (err error) {
if err := p.checkID(opts.DiskID); err != nil {
return err
}
ctx, done, err := p.TrackDiskHealth(ctx, storageMetricWalkDir, opts.Bucket, opts.BaseDir)
if err != nil {
return err
@@ -399,59 +403,32 @@ func (p *xlStorageDiskIDCheck) WalkDir(ctx context.Context, opts WalkDirOptions,
// WalkDir will traverse a directory and return all entries found.
// On success a meta cache stream will be returned, that should be closed when done.
func (client *storageRESTClient) WalkDir(ctx context.Context, opts WalkDirOptions, wr io.Writer) error {
values := make(url.Values)
values.Set(storageRESTVolume, opts.Bucket)
values.Set(storageRESTDirPath, opts.BaseDir)
values.Set(storageRESTRecursive, strconv.FormatBool(opts.Recursive))
values.Set(storageRESTReportNotFound, strconv.FormatBool(opts.ReportNotFound))
values.Set(storageRESTPrefixFilter, opts.FilterPrefix)
values.Set(storageRESTForwardFilter, opts.ForwardTo)
respBody, err := client.call(ctx, storageRESTMethodWalkDir, values, nil, -1)
// Ensure remote has the same disk ID.
opts.DiskID = client.diskID
b, err := opts.MarshalMsg(grid.GetByteBuffer()[:0])
if err != nil {
logger.LogIf(ctx, err)
return err
}
defer xhttp.DrainBody(respBody)
return waitForHTTPStream(respBody, wr)
st, err := client.gridConn.NewStream(ctx, grid.HandlerWalkDir, b)
if err != nil {
return err
}
return toStorageErr(st.Results(func(in []byte) error {
_, err := wr.Write(in)
return err
}))
}
// WalkDirHandler - remote caller to list files and folders in a requested directory path.
func (s *storageRESTServer) WalkDirHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
dirPath := r.Form.Get(storageRESTDirPath)
recursive, err := strconv.ParseBool(r.Form.Get(storageRESTRecursive))
func (s *storageRESTServer) WalkDirHandler(ctx context.Context, payload []byte, _ <-chan []byte, out chan<- []byte) (gerr *grid.RemoteErr) {
var opts WalkDirOptions
_, err := opts.UnmarshalMsg(payload)
if err != nil {
s.writeErrorResponse(w, err)
return
return grid.NewRemoteErr(err)
}
var reportNotFound bool
if v := r.Form.Get(storageRESTReportNotFound); v != "" {
reportNotFound, err = strconv.ParseBool(v)
if err != nil {
s.writeErrorResponse(w, err)
return
}
}
prefix := r.Form.Get(storageRESTPrefixFilter)
forward := r.Form.Get(storageRESTForwardFilter)
writer := streamHTTPResponse(w)
defer func() {
if r := recover(); r != nil {
debug.PrintStack()
writer.CloseWithError(fmt.Errorf("panic: %v", r))
}
}()
writer.CloseWithError(s.storage.WalkDir(r.Context(), WalkDirOptions{
Bucket: volume,
BaseDir: dirPath,
Recursive: recursive,
ReportNotFound: reportNotFound,
FilterPrefix: prefix,
ForwardTo: forward,
}, writer))
ctx, cancel := context.WithCancel(ctx)
defer cancel()
return grid.NewRemoteErr(s.storage.WalkDir(ctx, opts, grid.WriterToChannel(ctx, out)))
}

285
cmd/metacache-walk_gen.go Normal file
View File

@@ -0,0 +1,285 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"github.com/tinylib/msgp/msgp"
)
// DecodeMsg implements msgp.Decodable
func (z *WalkDirOptions) DecodeMsg(dc *msgp.Reader) (err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, err = dc.ReadMapHeader()
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, err = dc.ReadMapKeyPtr()
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "Bucket":
z.Bucket, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "Bucket")
return
}
case "BaseDir":
z.BaseDir, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "BaseDir")
return
}
case "Recursive":
z.Recursive, err = dc.ReadBool()
if err != nil {
err = msgp.WrapError(err, "Recursive")
return
}
case "ReportNotFound":
z.ReportNotFound, err = dc.ReadBool()
if err != nil {
err = msgp.WrapError(err, "ReportNotFound")
return
}
case "FilterPrefix":
z.FilterPrefix, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "FilterPrefix")
return
}
case "ForwardTo":
z.ForwardTo, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "ForwardTo")
return
}
case "Limit":
z.Limit, err = dc.ReadInt()
if err != nil {
err = msgp.WrapError(err, "Limit")
return
}
case "DiskID":
z.DiskID, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
default:
err = dc.Skip()
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
return
}
// EncodeMsg implements msgp.Encodable
func (z *WalkDirOptions) EncodeMsg(en *msgp.Writer) (err error) {
// map header, size 8
// write "Bucket"
err = en.Append(0x88, 0xa6, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74)
if err != nil {
return
}
err = en.WriteString(z.Bucket)
if err != nil {
err = msgp.WrapError(err, "Bucket")
return
}
// write "BaseDir"
err = en.Append(0xa7, 0x42, 0x61, 0x73, 0x65, 0x44, 0x69, 0x72)
if err != nil {
return
}
err = en.WriteString(z.BaseDir)
if err != nil {
err = msgp.WrapError(err, "BaseDir")
return
}
// write "Recursive"
err = en.Append(0xa9, 0x52, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65)
if err != nil {
return
}
err = en.WriteBool(z.Recursive)
if err != nil {
err = msgp.WrapError(err, "Recursive")
return
}
// write "ReportNotFound"
err = en.Append(0xae, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64)
if err != nil {
return
}
err = en.WriteBool(z.ReportNotFound)
if err != nil {
err = msgp.WrapError(err, "ReportNotFound")
return
}
// write "FilterPrefix"
err = en.Append(0xac, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78)
if err != nil {
return
}
err = en.WriteString(z.FilterPrefix)
if err != nil {
err = msgp.WrapError(err, "FilterPrefix")
return
}
// write "ForwardTo"
err = en.Append(0xa9, 0x46, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x54, 0x6f)
if err != nil {
return
}
err = en.WriteString(z.ForwardTo)
if err != nil {
err = msgp.WrapError(err, "ForwardTo")
return
}
// write "Limit"
err = en.Append(0xa5, 0x4c, 0x69, 0x6d, 0x69, 0x74)
if err != nil {
return
}
err = en.WriteInt(z.Limit)
if err != nil {
err = msgp.WrapError(err, "Limit")
return
}
// write "DiskID"
err = en.Append(0xa6, 0x44, 0x69, 0x73, 0x6b, 0x49, 0x44)
if err != nil {
return
}
err = en.WriteString(z.DiskID)
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *WalkDirOptions) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// map header, size 8
// string "Bucket"
o = append(o, 0x88, 0xa6, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74)
o = msgp.AppendString(o, z.Bucket)
// string "BaseDir"
o = append(o, 0xa7, 0x42, 0x61, 0x73, 0x65, 0x44, 0x69, 0x72)
o = msgp.AppendString(o, z.BaseDir)
// string "Recursive"
o = append(o, 0xa9, 0x52, 0x65, 0x63, 0x75, 0x72, 0x73, 0x69, 0x76, 0x65)
o = msgp.AppendBool(o, z.Recursive)
// string "ReportNotFound"
o = append(o, 0xae, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4e, 0x6f, 0x74, 0x46, 0x6f, 0x75, 0x6e, 0x64)
o = msgp.AppendBool(o, z.ReportNotFound)
// string "FilterPrefix"
o = append(o, 0xac, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78)
o = msgp.AppendString(o, z.FilterPrefix)
// string "ForwardTo"
o = append(o, 0xa9, 0x46, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x54, 0x6f)
o = msgp.AppendString(o, z.ForwardTo)
// string "Limit"
o = append(o, 0xa5, 0x4c, 0x69, 0x6d, 0x69, 0x74)
o = msgp.AppendInt(o, z.Limit)
// string "DiskID"
o = append(o, 0xa6, 0x44, 0x69, 0x73, 0x6b, 0x49, 0x44)
o = msgp.AppendString(o, z.DiskID)
return
}
// UnmarshalMsg implements msgp.Unmarshaler
func (z *WalkDirOptions) UnmarshalMsg(bts []byte) (o []byte, err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, bts, err = msgp.ReadMapHeaderBytes(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, bts, err = msgp.ReadMapKeyZC(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "Bucket":
z.Bucket, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "Bucket")
return
}
case "BaseDir":
z.BaseDir, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "BaseDir")
return
}
case "Recursive":
z.Recursive, bts, err = msgp.ReadBoolBytes(bts)
if err != nil {
err = msgp.WrapError(err, "Recursive")
return
}
case "ReportNotFound":
z.ReportNotFound, bts, err = msgp.ReadBoolBytes(bts)
if err != nil {
err = msgp.WrapError(err, "ReportNotFound")
return
}
case "FilterPrefix":
z.FilterPrefix, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "FilterPrefix")
return
}
case "ForwardTo":
z.ForwardTo, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "ForwardTo")
return
}
case "Limit":
z.Limit, bts, err = msgp.ReadIntBytes(bts)
if err != nil {
err = msgp.WrapError(err, "Limit")
return
}
case "DiskID":
z.DiskID, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
default:
bts, err = msgp.Skip(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
o = bts
return
}
// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message
func (z *WalkDirOptions) Msgsize() (s int) {
s = 1 + 7 + msgp.StringPrefixSize + len(z.Bucket) + 8 + msgp.StringPrefixSize + len(z.BaseDir) + 10 + msgp.BoolSize + 15 + msgp.BoolSize + 13 + msgp.StringPrefixSize + len(z.FilterPrefix) + 10 + msgp.StringPrefixSize + len(z.ForwardTo) + 6 + msgp.IntSize + 7 + msgp.StringPrefixSize + len(z.DiskID)
return
}

View File

@@ -0,0 +1,123 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"bytes"
"testing"
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalWalkDirOptions(t *testing.T) {
v := WalkDirOptions{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgWalkDirOptions(b *testing.B) {
v := WalkDirOptions{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgWalkDirOptions(b *testing.B) {
v := WalkDirOptions{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalWalkDirOptions(b *testing.B) {
v := WalkDirOptions{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeWalkDirOptions(t *testing.T) {
v := WalkDirOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeWalkDirOptions Msgsize() is inaccurate")
}
vn := WalkDirOptions{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeWalkDirOptions(b *testing.B) {
v := WalkDirOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeWalkDirOptions(b *testing.B) {
v := WalkDirOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@@ -67,5 +67,5 @@ func newStorageAPI(endpoint Endpoint, opts storageOpts) (storage StorageAPI, err
return newXLStorageDiskIDCheck(storage, opts.healthCheck), nil
}
return newStorageRESTClient(endpoint, opts.healthCheck), nil
return newStorageRESTClient(endpoint, opts.healthCheck, globalGrid.Load())
}

View File

@@ -20,13 +20,14 @@ package cmd
import (
"net/http"
"github.com/minio/minio/internal/grid"
"github.com/minio/mux"
)
// Composed function registering routers for only distributed Erasure setup.
func registerDistErasureRouters(router *mux.Router, endpointServerPools EndpointServerPools) {
// Register storage REST router only if its a distributed setup.
registerStorageRESTHandlers(router, endpointServerPools)
registerStorageRESTHandlers(router, endpointServerPools, globalGrid.Load())
// Register peer REST router only if its a distributed setup.
registerPeerRESTHandlers(router)
@@ -38,7 +39,10 @@ func registerDistErasureRouters(router *mux.Router, endpointServerPools Endpoint
registerBootstrapRESTHandlers(router)
// Register distributed namespace lock routers.
registerLockRESTHandlers(router)
registerLockRESTHandlers()
// Add grid to router
router.Handle(grid.RoutePath, adminMiddleware(globalGrid.Load().Handler(), noGZFlag, noObjLayerFlag))
}
// List of some generic middlewares which are applied for all incoming requests.

View File

@@ -666,12 +666,19 @@ func serverMain(ctx *cli.Context) {
getCert = globalTLSCerts.GetCertificate
}
// Initialize grid
bootstrapTrace("initGrid", func() {
logger.FatalIf(initGlobalGrid(GlobalContext, globalEndpoints), "Unable to configure server grid RPC services")
})
// Configure server.
bootstrapTrace("configureServer", func() {
handler, err := configureServerHandler(globalEndpoints)
if err != nil {
logger.Fatal(config.ErrUnexpectedError(err), "Unable to configure one of server's RPC services")
}
// Allow grid to start after registering all services.
close(globalGridStart)
httpServer := xhttp.NewServer(getServerListenAddrs()).
UseHandler(setCriticalErrorHandler(corsHandler(handler))).

View File

@@ -22,11 +22,9 @@ import (
)
// DeleteOptions represents the disk level delete options available for the APIs
//
//msgp:ignore DeleteOptions
type DeleteOptions struct {
Recursive bool
Force bool
Recursive bool `msg:"r"`
Force bool `msg:"f"`
}
//go:generate msgp -file=$GOFILE
@@ -143,7 +141,7 @@ func (f *FileInfoVersions) findVersionIndex(v string) int {
// Make sure to bump the internode version at storage-rest-common.go
type RawFileInfo struct {
// Content of entire xl.meta (may contain data depending on what was requested by the caller.
Buf []byte `msg:"b"`
Buf []byte `msg:"b,allownil"`
// DiskMTime indicates the mtime of the xl.meta on disk
// This is mainly used for detecting a particular issue
@@ -349,3 +347,57 @@ type ReadMultipleResp struct {
Data []byte // Contains all data of file.
Modtime time.Time // Modtime of file on disk.
}
// DeleteVersionHandlerParams are parameters for DeleteVersionHandler
type DeleteVersionHandlerParams struct {
DiskID string `msg:"id"`
Volume string `msg:"v"`
FilePath string `msg:"fp"`
ForceDelMarker bool `msg:"fdm"`
FI FileInfo `msg:"fi"`
}
// MetadataHandlerParams is request info for UpdateMetadataHandle and WriteMetadataHandler.
type MetadataHandlerParams struct {
DiskID string `msg:"id"`
Volume string `msg:"v"`
FilePath string `msg:"fp"`
UpdateOpts UpdateMetadataOpts `msg:"uo"`
FI FileInfo `msg:"fi"`
}
// UpdateMetadataOpts provides an optional input to indicate if xl.meta updates need to be fully synced to disk.
type UpdateMetadataOpts struct {
NoPersistence bool `msg:"np"`
}
// CheckPartsHandlerParams are parameters for CheckPartsHandler
type CheckPartsHandlerParams struct {
DiskID string `msg:"id"`
Volume string `msg:"v"`
FilePath string `msg:"fp"`
FI FileInfo `msg:"fi"`
}
// DeleteFileHandlerParams are parameters for DeleteFileHandler
type DeleteFileHandlerParams struct {
DiskID string `msg:"id"`
Volume string `msg:"v"`
FilePath string `msg:"fp"`
Opts DeleteOptions `msg:"do"`
}
// RenameDataHandlerParams are parameters for RenameDataHandler.
type RenameDataHandlerParams struct {
DiskID string `msg:"id"`
SrcVolume string `msg:"sv"`
SrcPath string `msg:"sp"`
DstVolume string `msg:"dv"`
DstPath string `msg:"dp"`
FI FileInfo `msg:"fi"`
}
// RenameDataResp - RenameData()'s response.
type RenameDataResp struct {
Signature uint64 `msg:"sig"`
}

File diff suppressed because it is too large Load Diff

View File

@@ -9,6 +9,458 @@ import (
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalCheckPartsHandlerParams(t *testing.T) {
v := CheckPartsHandlerParams{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgCheckPartsHandlerParams(b *testing.B) {
v := CheckPartsHandlerParams{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgCheckPartsHandlerParams(b *testing.B) {
v := CheckPartsHandlerParams{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalCheckPartsHandlerParams(b *testing.B) {
v := CheckPartsHandlerParams{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeCheckPartsHandlerParams(t *testing.T) {
v := CheckPartsHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeCheckPartsHandlerParams Msgsize() is inaccurate")
}
vn := CheckPartsHandlerParams{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeCheckPartsHandlerParams(b *testing.B) {
v := CheckPartsHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeCheckPartsHandlerParams(b *testing.B) {
v := CheckPartsHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalDeleteFileHandlerParams(t *testing.T) {
v := DeleteFileHandlerParams{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgDeleteFileHandlerParams(b *testing.B) {
v := DeleteFileHandlerParams{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgDeleteFileHandlerParams(b *testing.B) {
v := DeleteFileHandlerParams{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalDeleteFileHandlerParams(b *testing.B) {
v := DeleteFileHandlerParams{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeDeleteFileHandlerParams(t *testing.T) {
v := DeleteFileHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeDeleteFileHandlerParams Msgsize() is inaccurate")
}
vn := DeleteFileHandlerParams{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeDeleteFileHandlerParams(b *testing.B) {
v := DeleteFileHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeDeleteFileHandlerParams(b *testing.B) {
v := DeleteFileHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalDeleteOptions(t *testing.T) {
v := DeleteOptions{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgDeleteOptions(b *testing.B) {
v := DeleteOptions{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgDeleteOptions(b *testing.B) {
v := DeleteOptions{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalDeleteOptions(b *testing.B) {
v := DeleteOptions{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeDeleteOptions(t *testing.T) {
v := DeleteOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeDeleteOptions Msgsize() is inaccurate")
}
vn := DeleteOptions{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeDeleteOptions(b *testing.B) {
v := DeleteOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeDeleteOptions(b *testing.B) {
v := DeleteOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalDeleteVersionHandlerParams(t *testing.T) {
v := DeleteVersionHandlerParams{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgDeleteVersionHandlerParams(b *testing.B) {
v := DeleteVersionHandlerParams{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgDeleteVersionHandlerParams(b *testing.B) {
v := DeleteVersionHandlerParams{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalDeleteVersionHandlerParams(b *testing.B) {
v := DeleteVersionHandlerParams{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeDeleteVersionHandlerParams(t *testing.T) {
v := DeleteVersionHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeDeleteVersionHandlerParams Msgsize() is inaccurate")
}
vn := DeleteVersionHandlerParams{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeDeleteVersionHandlerParams(b *testing.B) {
v := DeleteVersionHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeDeleteVersionHandlerParams(b *testing.B) {
v := DeleteVersionHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalDiskInfo(t *testing.T) {
v := DiskInfo{}
bts, err := v.MarshalMsg(nil)
@@ -574,6 +1026,119 @@ func BenchmarkDecodeFilesInfo(b *testing.B) {
}
}
func TestMarshalUnmarshalMetadataHandlerParams(t *testing.T) {
v := MetadataHandlerParams{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgMetadataHandlerParams(b *testing.B) {
v := MetadataHandlerParams{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgMetadataHandlerParams(b *testing.B) {
v := MetadataHandlerParams{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalMetadataHandlerParams(b *testing.B) {
v := MetadataHandlerParams{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeMetadataHandlerParams(t *testing.T) {
v := MetadataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeMetadataHandlerParams Msgsize() is inaccurate")
}
vn := MetadataHandlerParams{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeMetadataHandlerParams(b *testing.B) {
v := MetadataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeMetadataHandlerParams(b *testing.B) {
v := MetadataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalRawFileInfo(t *testing.T) {
v := RawFileInfo{}
bts, err := v.MarshalMsg(nil)
@@ -913,6 +1478,345 @@ func BenchmarkDecodeReadMultipleResp(b *testing.B) {
}
}
func TestMarshalUnmarshalRenameDataHandlerParams(t *testing.T) {
v := RenameDataHandlerParams{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgRenameDataHandlerParams(b *testing.B) {
v := RenameDataHandlerParams{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgRenameDataHandlerParams(b *testing.B) {
v := RenameDataHandlerParams{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalRenameDataHandlerParams(b *testing.B) {
v := RenameDataHandlerParams{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeRenameDataHandlerParams(t *testing.T) {
v := RenameDataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeRenameDataHandlerParams Msgsize() is inaccurate")
}
vn := RenameDataHandlerParams{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeRenameDataHandlerParams(b *testing.B) {
v := RenameDataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeRenameDataHandlerParams(b *testing.B) {
v := RenameDataHandlerParams{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalRenameDataResp(t *testing.T) {
v := RenameDataResp{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgRenameDataResp(b *testing.B) {
v := RenameDataResp{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgRenameDataResp(b *testing.B) {
v := RenameDataResp{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalRenameDataResp(b *testing.B) {
v := RenameDataResp{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeRenameDataResp(t *testing.T) {
v := RenameDataResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeRenameDataResp Msgsize() is inaccurate")
}
vn := RenameDataResp{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeRenameDataResp(b *testing.B) {
v := RenameDataResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeRenameDataResp(b *testing.B) {
v := RenameDataResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalUpdateMetadataOpts(t *testing.T) {
v := UpdateMetadataOpts{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgUpdateMetadataOpts(b *testing.B) {
v := UpdateMetadataOpts{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgUpdateMetadataOpts(b *testing.B) {
v := UpdateMetadataOpts{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalUpdateMetadataOpts(b *testing.B) {
v := UpdateMetadataOpts{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodeUpdateMetadataOpts(t *testing.T) {
v := UpdateMetadataOpts{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodeUpdateMetadataOpts Msgsize() is inaccurate")
}
vn := UpdateMetadataOpts{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodeUpdateMetadataOpts(b *testing.B) {
v := UpdateMetadataOpts{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodeUpdateMetadataOpts(b *testing.B) {
v := UpdateMetadataOpts{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalVolInfo(t *testing.T) {
v := VolInfo{}
bts, err := v.MarshalMsg(nil)

View File

@@ -23,6 +23,7 @@ import (
"encoding/gob"
"encoding/hex"
"errors"
"fmt"
"io"
"net/http"
"net/url"
@@ -34,6 +35,7 @@ import (
"time"
"github.com/minio/madmin-go/v3"
"github.com/minio/minio/internal/grid"
xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/logger"
"github.com/minio/minio/internal/rest"
@@ -52,7 +54,9 @@ func isNetworkError(err error) bool {
return true
}
}
if errors.Is(err, grid.ErrDisconnected) {
return true
}
// More corner cases suitable for storage REST API
switch {
// A peer node can be in shut down phase and proactively
@@ -139,6 +143,7 @@ type storageRESTClient struct {
endpoint Endpoint
restClient *rest.Client
gridConn *grid.Subroute
diskID string
// Indexes, will be -1 until assigned a set.
@@ -184,7 +189,7 @@ func (client *storageRESTClient) String() string {
// IsOnline - returns whether RPC client failed to connect or not.
func (client *storageRESTClient) IsOnline() bool {
return client.restClient.IsOnline()
return client.restClient.IsOnline() && client.gridConn.State() == grid.StateConnected
}
// LastConn - returns when the disk is seen to be connected the last time
@@ -213,57 +218,37 @@ func (client *storageRESTClient) Healing() *healingTracker {
func (client *storageRESTClient) NSScanner(ctx context.Context, cache dataUsageCache, updates chan<- dataUsageEntry, scanMode madmin.HealScanMode) (dataUsageCache, error) {
atomic.AddInt32(&client.scanning, 1)
defer atomic.AddInt32(&client.scanning, -1)
defer close(updates)
pr, pw := io.Pipe()
go func() {
pw.CloseWithError(cache.serializeTo(pw))
}()
vals := make(url.Values)
vals.Set(storageRESTScanMode, strconv.Itoa(int(scanMode)))
respBody, err := client.call(ctx, storageRESTMethodNSScanner, vals, pr, -1)
defer xhttp.DrainBody(respBody)
pr.CloseWithError(err)
st, err := storageNSScannerHandler.Call(ctx, client.gridConn, &nsScannerOptions{
DiskID: client.diskID,
ScanMode: int(scanMode),
Cache: &cache,
})
if err != nil {
return cache, err
return cache, toStorageErr(err)
}
rr, rw := io.Pipe()
go func() {
rw.CloseWithError(waitForHTTPStream(respBody, rw))
}()
ms := msgpNewReader(rr)
defer readMsgpReaderPoolPut(ms)
for {
// Read whether it is an update.
upd, err := ms.ReadBool()
if err != nil {
rr.CloseWithError(err)
return cache, err
var final *dataUsageCache
err = st.Results(func(resp *nsScannerResp) error {
if resp.Update != nil {
select {
case <-ctx.Done():
case updates <- *resp.Update:
}
}
if !upd {
// No more updates... New cache follows.
break
}
var update dataUsageEntry
err = update.DecodeMsg(ms)
if err != nil || err == io.EOF {
rr.CloseWithError(err)
return cache, err
}
select {
case <-ctx.Done():
case updates <- update:
if resp.Final != nil {
final = resp.Final
}
// We can't reuse the response since it is sent upstream.
return nil
})
if err != nil {
return cache, toStorageErr(err)
}
var newCache dataUsageCache
err = newCache.DecodeMsg(ms)
rr.CloseWithError(err)
if err == io.EOF {
err = nil
if final == nil {
return cache, errors.New("no final cache")
}
return newCache, err
return *final, nil
}
func (client *storageRESTClient) GetDiskID() (string, error) {
@@ -278,77 +263,44 @@ func (client *storageRESTClient) SetDiskID(id string) {
client.diskID = id
}
// DiskInfo - fetch disk information for a remote disk.
func (client *storageRESTClient) DiskInfo(_ context.Context, metrics bool) (info DiskInfo, err error) {
if !client.IsOnline() {
func (client *storageRESTClient) DiskInfo(ctx context.Context, metrics bool) (info DiskInfo, err error) {
if client.gridConn.State() != grid.StateConnected {
// make sure to check if the disk is offline, since the underlying
// value is cached we should attempt to invalidate it if such calls
// were attempted. This can lead to false success under certain conditions
// - this change attempts to avoid stale information if the underlying
// transport is already down.
return info, errDiskNotFound
return info, grid.ErrDisconnected
}
// Do not cache results from atomic variables
scanning := atomic.LoadInt32(&client.scanning) == 1
if metrics {
client.diskInfoCacheMetrics.Once.Do(func() {
client.diskInfoCacheMetrics.TTL = time.Second
client.diskInfoCacheMetrics.Update = func() (interface{}, error) {
var info DiskInfo
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
vals := make(url.Values)
vals.Set(storageRESTMetrics, "true")
respBody, err := client.call(ctx, storageRESTMethodDiskInfo, vals, nil, -1)
if err != nil {
return info, err
}
defer xhttp.DrainBody(respBody)
if err = msgp.Decode(respBody, &info); err != nil {
return info, err
}
if info.Error != "" {
return info, toStorageErr(errors.New(info.Error))
}
return info, nil
fetchDI := func(di *timedValue, metrics bool) {
di.TTL = time.Second
di.Update = func() (interface{}, error) {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
info, err := storageDiskInfoHandler.Call(ctx, client.gridConn, grid.NewMSSWith(map[string]string{
storageRESTDiskID: client.diskID,
// Always request metrics, since we are caching the result.
storageRESTMetrics: strconv.FormatBool(metrics),
}))
if err != nil {
return info, err
}
})
} else {
client.diskInfoCache.Once.Do(func() {
client.diskInfoCache.TTL = time.Second
client.diskInfoCache.Update = func() (interface{}, error) {
var info DiskInfo
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
vals := make(url.Values)
respBody, err := client.call(ctx, storageRESTMethodDiskInfo, vals, nil, -1)
if err != nil {
return info, err
}
defer xhttp.DrainBody(respBody)
if err = msgp.Decode(respBody, &info); err != nil {
return info, err
}
if info.Error != "" {
return info, toStorageErr(errors.New(info.Error))
}
return info, nil
if info.Error != "" {
return info, toStorageErr(errors.New(info.Error))
}
})
return info, nil
}
}
var val interface{}
// Fetch disk info from appropriate cache.
dic := &client.diskInfoCache
if metrics {
val, err = client.diskInfoCacheMetrics.Get()
} else {
val, err = client.diskInfoCache.Get()
dic = &client.diskInfoCacheMetrics
}
if val != nil {
info = val.(DiskInfo)
dic.Once.Do(func() { fetchDI(dic, metrics) })
val, err := dic.Get()
if di, ok := val.(*DiskInfo); di != nil && ok {
info = *di
}
info.Scanning = scanning
return info, err
}
@@ -384,15 +336,16 @@ func (client *storageRESTClient) ListVols(ctx context.Context) (vols []VolInfo,
// StatVol - get volume info over the network.
func (client *storageRESTClient) StatVol(ctx context.Context, volume string) (vol VolInfo, err error) {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
respBody, err := client.call(ctx, storageRESTMethodStatVol, values, nil, -1)
v, err := storageStatVolHandler.Call(ctx, client.gridConn, grid.NewMSSWith(map[string]string{
storageRESTDiskID: client.diskID,
storageRESTVolume: volume,
}))
if err != nil {
return
return vol, toStorageErr(err)
}
defer xhttp.DrainBody(respBody)
err = msgp.Decode(respBody, &vol)
return vol, err
vol = *v
storageStatVolHandler.PutResponse(v)
return vol, nil
}
// DeleteVol - Deletes a volume over the network.
@@ -433,50 +386,35 @@ func (client *storageRESTClient) CreateFile(ctx context.Context, volume, path st
}
func (client *storageRESTClient) WriteMetadata(ctx context.Context, volume, path string, fi FileInfo) error {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
var reader bytes.Buffer
if err := msgp.Encode(&reader, &fi); err != nil {
return err
}
respBody, err := client.call(ctx, storageRESTMethodWriteMetadata, values, &reader, -1)
defer xhttp.DrainBody(respBody)
return err
_, err := storageWriteMetadataHandler.Call(ctx, client.gridConn, &MetadataHandlerParams{
DiskID: client.diskID,
Volume: volume,
FilePath: path,
FI: fi,
})
return toStorageErr(err)
}
func (client *storageRESTClient) UpdateMetadata(ctx context.Context, volume, path string, fi FileInfo, opts UpdateMetadataOpts) error {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
values.Set(storageRESTNoPersistence, strconv.FormatBool(opts.NoPersistence))
var reader bytes.Buffer
if err := msgp.Encode(&reader, &fi); err != nil {
return err
}
respBody, err := client.call(ctx, storageRESTMethodUpdateMetadata, values, &reader, -1)
defer xhttp.DrainBody(respBody)
return err
_, err := storageUpdateMetadataHandler.Call(ctx, client.gridConn, &MetadataHandlerParams{
DiskID: client.diskID,
Volume: volume,
FilePath: path,
UpdateOpts: opts,
FI: fi,
})
return toStorageErr(err)
}
func (client *storageRESTClient) DeleteVersion(ctx context.Context, volume, path string, fi FileInfo, forceDelMarker bool) error {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
values.Set(storageRESTForceDelMarker, strconv.FormatBool(forceDelMarker))
var buffer bytes.Buffer
if err := msgp.Encode(&buffer, &fi); err != nil {
return err
}
respBody, err := client.call(ctx, storageRESTMethodDeleteVersion, values, &buffer, -1)
defer xhttp.DrainBody(respBody)
return err
_, err := storageDeleteVersionHandler.Call(ctx, client.gridConn, &DeleteVersionHandlerParams{
DiskID: client.diskID,
Volume: volume,
FilePath: path,
ForceDelMarker: forceDelMarker,
FI: fi,
})
return toStorageErr(err)
}
// WriteAll - write all data to a file.
@@ -491,51 +429,32 @@ func (client *storageRESTClient) WriteAll(ctx context.Context, volume string, pa
// CheckParts - stat all file parts.
func (client *storageRESTClient) CheckParts(ctx context.Context, volume string, path string, fi FileInfo) error {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
var reader bytes.Buffer
if err := msgp.Encode(&reader, &fi); err != nil {
logger.LogIf(context.Background(), err)
return err
}
respBody, err := client.call(ctx, storageRESTMethodCheckParts, values, &reader, -1)
defer xhttp.DrainBody(respBody)
return err
_, err := storageCheckPartsHandler.Call(ctx, client.gridConn, &CheckPartsHandlerParams{
DiskID: client.diskID,
Volume: volume,
FilePath: path,
FI: fi,
})
return toStorageErr(err)
}
// RenameData - rename source path to destination path atomically, metadata and data file.
func (client *storageRESTClient) RenameData(ctx context.Context, srcVolume, srcPath string, fi FileInfo, dstVolume, dstPath string) (sign uint64, err error) {
values := make(url.Values)
values.Set(storageRESTSrcVolume, srcVolume)
values.Set(storageRESTSrcPath, srcPath)
values.Set(storageRESTDstVolume, dstVolume)
values.Set(storageRESTDstPath, dstPath)
var reader bytes.Buffer
if err = msgp.Encode(&reader, &fi); err != nil {
return 0, err
}
respBody, err := client.call(ctx, storageRESTMethodRenameData, values, &reader, -1)
defer xhttp.DrainBody(respBody)
// Set a very long timeout for rename data.
ctx, cancel := context.WithTimeout(ctx, 5*time.Minute)
defer cancel()
resp, err := storageRenameDataHandler.Call(ctx, client.gridConn, &RenameDataHandlerParams{
DiskID: client.diskID,
SrcVolume: srcVolume,
SrcPath: srcPath,
DstPath: dstPath,
DstVolume: dstVolume,
FI: fi,
})
if err != nil {
return 0, err
return 0, toStorageErr(err)
}
respReader, err := waitForHTTPResponse(respBody)
if err != nil {
return 0, err
}
resp := &RenameDataResp{}
if err = gob.NewDecoder(respReader).Decode(resp); err != nil {
return 0, err
}
return resp.Signature, toStorageErr(resp.Err)
return resp.Signature, nil
}
// where we keep old *Readers
@@ -562,6 +481,21 @@ func readMsgpReaderPoolPut(r *msgp.Reader) {
}
func (client *storageRESTClient) ReadVersion(ctx context.Context, volume, path, versionID string, readData bool) (fi FileInfo, err error) {
// Use websocket when not reading data.
if !readData {
resp, err := storageReadVersionHandler.Call(ctx, client.gridConn, grid.NewMSSWith(map[string]string{
storageRESTDiskID: client.diskID,
storageRESTVolume: volume,
storageRESTFilePath: path,
storageRESTVersionID: versionID,
storageRESTReadData: "false",
}))
if err != nil {
return fi, toStorageErr(err)
}
return *resp, nil
}
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
@@ -583,13 +517,27 @@ func (client *storageRESTClient) ReadVersion(ctx context.Context, volume, path,
// ReadXL - reads all contents of xl.meta of a file.
func (client *storageRESTClient) ReadXL(ctx context.Context, volume string, path string, readData bool) (rf RawFileInfo, err error) {
// Use websocket when not reading data.
if !readData {
resp, err := storageReadXLHandler.Call(ctx, client.gridConn, grid.NewMSSWith(map[string]string{
storageRESTDiskID: client.diskID,
storageRESTVolume: volume,
storageRESTFilePath: path,
storageRESTReadData: "false",
}))
if err != nil {
return rf, toStorageErr(err)
}
return *resp, nil
}
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
values.Set(storageRESTReadData, strconv.FormatBool(readData))
respBody, err := client.call(ctx, storageRESTMethodReadXL, values, nil, -1)
if err != nil {
return rf, err
return rf, toStorageErr(err)
}
defer xhttp.DrainBody(respBody)
@@ -667,15 +615,13 @@ func (client *storageRESTClient) ListDir(ctx context.Context, volume, dirPath st
// DeleteFile - deletes a file.
func (client *storageRESTClient) Delete(ctx context.Context, volume string, path string, deleteOpts DeleteOptions) error {
values := make(url.Values)
values.Set(storageRESTVolume, volume)
values.Set(storageRESTFilePath, path)
values.Set(storageRESTRecursive, strconv.FormatBool(deleteOpts.Recursive))
values.Set(storageRESTForceDelete, strconv.FormatBool(deleteOpts.Force))
respBody, err := client.call(ctx, storageRESTMethodDeleteFile, values, nil, -1)
defer xhttp.DrainBody(respBody)
return err
_, err := storageDeleteFileHandler.Call(ctx, client.gridConn, &DeleteFileHandlerParams{
DiskID: client.diskID,
Volume: volume,
FilePath: path,
Opts: deleteOpts,
})
return toStorageErr(err)
}
// DeleteVersions - deletes list of specified versions if present
@@ -867,7 +813,7 @@ func (client *storageRESTClient) Close() error {
}
// Returns a storage rest client.
func newStorageRESTClient(endpoint Endpoint, healthCheck bool) *storageRESTClient {
func newStorageRESTClient(endpoint Endpoint, healthCheck bool, gm *grid.Manager) (*storageRESTClient, error) {
serverURL := &url.URL{
Scheme: endpoint.Scheme,
Host: endpoint.Host,
@@ -888,6 +834,12 @@ func newStorageRESTClient(endpoint Endpoint, healthCheck bool) *storageRESTClien
return toStorageErr(err) != errDiskNotFound
}
}
return &storageRESTClient{endpoint: endpoint, restClient: restClient, poolIndex: -1, setIndex: -1, diskIndex: -1}
conn := gm.Connection(endpoint.GridHost()).Subroute(endpoint.Path)
if conn == nil {
return nil, fmt.Errorf("unable to find connection for %s in targets: %v", endpoint.GridHost(), gm.Targets())
}
return &storageRESTClient{
endpoint: endpoint, restClient: restClient, poolIndex: -1, setIndex: -1, diskIndex: -1,
gridConn: conn,
}, nil
}

View File

@@ -17,6 +17,8 @@
package cmd
//go:generate msgp -file $GOFILE -unexported
const (
storageRESTVersion = "v50" // Added DiskInfo metrics query
storageRESTVersionPrefix = SlashSeparator + storageRESTVersion
@@ -25,64 +27,58 @@ const (
const (
storageRESTMethodHealth = "/health"
storageRESTMethodDiskInfo = "/diskinfo"
storageRESTMethodNSScanner = "/nsscanner"
storageRESTMethodMakeVol = "/makevol"
storageRESTMethodMakeVolBulk = "/makevolbulk"
storageRESTMethodStatVol = "/statvol"
storageRESTMethodDeleteVol = "/deletevol"
storageRESTMethodListVols = "/listvols"
storageRESTMethodAppendFile = "/appendfile"
storageRESTMethodCreateFile = "/createfile"
storageRESTMethodWriteAll = "/writeall"
storageRESTMethodWriteMetadata = "/writemetadata"
storageRESTMethodUpdateMetadata = "/updatemetadata"
storageRESTMethodDeleteVersion = "/deleteversion"
storageRESTMethodReadVersion = "/readversion"
storageRESTMethodReadXL = "/readxl"
storageRESTMethodRenameData = "/renamedata"
storageRESTMethodCheckParts = "/checkparts"
storageRESTMethodReadAll = "/readall"
storageRESTMethodReadFile = "/readfile"
storageRESTMethodReadFileStream = "/readfilestream"
storageRESTMethodListDir = "/listdir"
storageRESTMethodDeleteFile = "/deletefile"
storageRESTMethodDeleteVersions = "/deleteverions"
storageRESTMethodRenameFile = "/renamefile"
storageRESTMethodVerifyFile = "/verifyfile"
storageRESTMethodWalkDir = "/walkdir"
storageRESTMethodStatInfoFile = "/statfile"
storageRESTMethodReadMultiple = "/readmultiple"
storageRESTMethodCleanAbandoned = "/cleanabandoned"
)
const (
storageRESTVolume = "volume"
storageRESTVolumes = "volumes"
storageRESTDirPath = "dir-path"
storageRESTFilePath = "file-path"
storageRESTForceDelMarker = "force-delete-marker"
storageRESTVersionID = "version-id"
storageRESTReadData = "read-data"
storageRESTTotalVersions = "total-versions"
storageRESTSrcVolume = "source-volume"
storageRESTSrcPath = "source-path"
storageRESTDstVolume = "destination-volume"
storageRESTDstPath = "destination-path"
storageRESTOffset = "offset"
storageRESTLength = "length"
storageRESTCount = "count"
storageRESTPrefixFilter = "prefix"
storageRESTForwardFilter = "forward"
storageRESTRecursive = "recursive"
storageRESTReportNotFound = "report-notfound"
storageRESTBitrotAlgo = "bitrot-algo"
storageRESTBitrotHash = "bitrot-hash"
storageRESTDiskID = "disk-id"
storageRESTForceDelete = "force-delete"
storageRESTGlob = "glob"
storageRESTScanMode = "scan-mode"
storageRESTMetrics = "metrics"
storageRESTNoPersistence = "no-persistence"
storageRESTVolume = "volume"
storageRESTVolumes = "volumes"
storageRESTDirPath = "dir-path"
storageRESTFilePath = "file-path"
storageRESTVersionID = "version-id"
storageRESTReadData = "read-data"
storageRESTTotalVersions = "total-versions"
storageRESTSrcVolume = "source-volume"
storageRESTSrcPath = "source-path"
storageRESTDstVolume = "destination-volume"
storageRESTDstPath = "destination-path"
storageRESTOffset = "offset"
storageRESTLength = "length"
storageRESTCount = "count"
storageRESTBitrotAlgo = "bitrot-algo"
storageRESTBitrotHash = "bitrot-hash"
storageRESTDiskID = "disk-id"
storageRESTForceDelete = "force-delete"
storageRESTGlob = "glob"
storageRESTMetrics = "metrics"
)
type nsScannerOptions struct {
DiskID string `msg:"id"`
ScanMode int `msg:"m"`
Cache *dataUsageCache `msg:"c"`
}
type nsScannerResp struct {
Update *dataUsageEntry `msg:"u"`
Final *dataUsageCache `msg:"f"`
}

View File

@@ -0,0 +1,418 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"github.com/tinylib/msgp/msgp"
)
// DecodeMsg implements msgp.Decodable
func (z *nsScannerOptions) DecodeMsg(dc *msgp.Reader) (err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, err = dc.ReadMapHeader()
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, err = dc.ReadMapKeyPtr()
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "id":
z.DiskID, err = dc.ReadString()
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
case "m":
z.ScanMode, err = dc.ReadInt()
if err != nil {
err = msgp.WrapError(err, "ScanMode")
return
}
case "c":
if dc.IsNil() {
err = dc.ReadNil()
if err != nil {
err = msgp.WrapError(err, "Cache")
return
}
z.Cache = nil
} else {
if z.Cache == nil {
z.Cache = new(dataUsageCache)
}
err = z.Cache.DecodeMsg(dc)
if err != nil {
err = msgp.WrapError(err, "Cache")
return
}
}
default:
err = dc.Skip()
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
return
}
// EncodeMsg implements msgp.Encodable
func (z *nsScannerOptions) EncodeMsg(en *msgp.Writer) (err error) {
// map header, size 3
// write "id"
err = en.Append(0x83, 0xa2, 0x69, 0x64)
if err != nil {
return
}
err = en.WriteString(z.DiskID)
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
// write "m"
err = en.Append(0xa1, 0x6d)
if err != nil {
return
}
err = en.WriteInt(z.ScanMode)
if err != nil {
err = msgp.WrapError(err, "ScanMode")
return
}
// write "c"
err = en.Append(0xa1, 0x63)
if err != nil {
return
}
if z.Cache == nil {
err = en.WriteNil()
if err != nil {
return
}
} else {
err = z.Cache.EncodeMsg(en)
if err != nil {
err = msgp.WrapError(err, "Cache")
return
}
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *nsScannerOptions) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// map header, size 3
// string "id"
o = append(o, 0x83, 0xa2, 0x69, 0x64)
o = msgp.AppendString(o, z.DiskID)
// string "m"
o = append(o, 0xa1, 0x6d)
o = msgp.AppendInt(o, z.ScanMode)
// string "c"
o = append(o, 0xa1, 0x63)
if z.Cache == nil {
o = msgp.AppendNil(o)
} else {
o, err = z.Cache.MarshalMsg(o)
if err != nil {
err = msgp.WrapError(err, "Cache")
return
}
}
return
}
// UnmarshalMsg implements msgp.Unmarshaler
func (z *nsScannerOptions) UnmarshalMsg(bts []byte) (o []byte, err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, bts, err = msgp.ReadMapHeaderBytes(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, bts, err = msgp.ReadMapKeyZC(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "id":
z.DiskID, bts, err = msgp.ReadStringBytes(bts)
if err != nil {
err = msgp.WrapError(err, "DiskID")
return
}
case "m":
z.ScanMode, bts, err = msgp.ReadIntBytes(bts)
if err != nil {
err = msgp.WrapError(err, "ScanMode")
return
}
case "c":
if msgp.IsNil(bts) {
bts, err = msgp.ReadNilBytes(bts)
if err != nil {
return
}
z.Cache = nil
} else {
if z.Cache == nil {
z.Cache = new(dataUsageCache)
}
bts, err = z.Cache.UnmarshalMsg(bts)
if err != nil {
err = msgp.WrapError(err, "Cache")
return
}
}
default:
bts, err = msgp.Skip(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
o = bts
return
}
// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message
func (z *nsScannerOptions) Msgsize() (s int) {
s = 1 + 3 + msgp.StringPrefixSize + len(z.DiskID) + 2 + msgp.IntSize + 2
if z.Cache == nil {
s += msgp.NilSize
} else {
s += z.Cache.Msgsize()
}
return
}
// DecodeMsg implements msgp.Decodable
func (z *nsScannerResp) DecodeMsg(dc *msgp.Reader) (err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, err = dc.ReadMapHeader()
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, err = dc.ReadMapKeyPtr()
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "u":
if dc.IsNil() {
err = dc.ReadNil()
if err != nil {
err = msgp.WrapError(err, "Update")
return
}
z.Update = nil
} else {
if z.Update == nil {
z.Update = new(dataUsageEntry)
}
err = z.Update.DecodeMsg(dc)
if err != nil {
err = msgp.WrapError(err, "Update")
return
}
}
case "f":
if dc.IsNil() {
err = dc.ReadNil()
if err != nil {
err = msgp.WrapError(err, "Final")
return
}
z.Final = nil
} else {
if z.Final == nil {
z.Final = new(dataUsageCache)
}
err = z.Final.DecodeMsg(dc)
if err != nil {
err = msgp.WrapError(err, "Final")
return
}
}
default:
err = dc.Skip()
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
return
}
// EncodeMsg implements msgp.Encodable
func (z *nsScannerResp) EncodeMsg(en *msgp.Writer) (err error) {
// map header, size 2
// write "u"
err = en.Append(0x82, 0xa1, 0x75)
if err != nil {
return
}
if z.Update == nil {
err = en.WriteNil()
if err != nil {
return
}
} else {
err = z.Update.EncodeMsg(en)
if err != nil {
err = msgp.WrapError(err, "Update")
return
}
}
// write "f"
err = en.Append(0xa1, 0x66)
if err != nil {
return
}
if z.Final == nil {
err = en.WriteNil()
if err != nil {
return
}
} else {
err = z.Final.EncodeMsg(en)
if err != nil {
err = msgp.WrapError(err, "Final")
return
}
}
return
}
// MarshalMsg implements msgp.Marshaler
func (z *nsScannerResp) MarshalMsg(b []byte) (o []byte, err error) {
o = msgp.Require(b, z.Msgsize())
// map header, size 2
// string "u"
o = append(o, 0x82, 0xa1, 0x75)
if z.Update == nil {
o = msgp.AppendNil(o)
} else {
o, err = z.Update.MarshalMsg(o)
if err != nil {
err = msgp.WrapError(err, "Update")
return
}
}
// string "f"
o = append(o, 0xa1, 0x66)
if z.Final == nil {
o = msgp.AppendNil(o)
} else {
o, err = z.Final.MarshalMsg(o)
if err != nil {
err = msgp.WrapError(err, "Final")
return
}
}
return
}
// UnmarshalMsg implements msgp.Unmarshaler
func (z *nsScannerResp) UnmarshalMsg(bts []byte) (o []byte, err error) {
var field []byte
_ = field
var zb0001 uint32
zb0001, bts, err = msgp.ReadMapHeaderBytes(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
for zb0001 > 0 {
zb0001--
field, bts, err = msgp.ReadMapKeyZC(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
switch msgp.UnsafeString(field) {
case "u":
if msgp.IsNil(bts) {
bts, err = msgp.ReadNilBytes(bts)
if err != nil {
return
}
z.Update = nil
} else {
if z.Update == nil {
z.Update = new(dataUsageEntry)
}
bts, err = z.Update.UnmarshalMsg(bts)
if err != nil {
err = msgp.WrapError(err, "Update")
return
}
}
case "f":
if msgp.IsNil(bts) {
bts, err = msgp.ReadNilBytes(bts)
if err != nil {
return
}
z.Final = nil
} else {
if z.Final == nil {
z.Final = new(dataUsageCache)
}
bts, err = z.Final.UnmarshalMsg(bts)
if err != nil {
err = msgp.WrapError(err, "Final")
return
}
}
default:
bts, err = msgp.Skip(bts)
if err != nil {
err = msgp.WrapError(err)
return
}
}
}
o = bts
return
}
// Msgsize returns an upper bound estimate of the number of bytes occupied by the serialized message
func (z *nsScannerResp) Msgsize() (s int) {
s = 1 + 2
if z.Update == nil {
s += msgp.NilSize
} else {
s += z.Update.Msgsize()
}
s += 2
if z.Final == nil {
s += msgp.NilSize
} else {
s += z.Final.Msgsize()
}
return
}

View File

@@ -0,0 +1,236 @@
package cmd
// Code generated by github.com/tinylib/msgp DO NOT EDIT.
import (
"bytes"
"testing"
"github.com/tinylib/msgp/msgp"
)
func TestMarshalUnmarshalnsScannerOptions(t *testing.T) {
v := nsScannerOptions{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgnsScannerOptions(b *testing.B) {
v := nsScannerOptions{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgnsScannerOptions(b *testing.B) {
v := nsScannerOptions{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalnsScannerOptions(b *testing.B) {
v := nsScannerOptions{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodensScannerOptions(t *testing.T) {
v := nsScannerOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodensScannerOptions Msgsize() is inaccurate")
}
vn := nsScannerOptions{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodensScannerOptions(b *testing.B) {
v := nsScannerOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodensScannerOptions(b *testing.B) {
v := nsScannerOptions{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}
func TestMarshalUnmarshalnsScannerResp(t *testing.T) {
v := nsScannerResp{}
bts, err := v.MarshalMsg(nil)
if err != nil {
t.Fatal(err)
}
left, err := v.UnmarshalMsg(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after UnmarshalMsg(): %q", len(left), left)
}
left, err = msgp.Skip(bts)
if err != nil {
t.Fatal(err)
}
if len(left) > 0 {
t.Errorf("%d bytes left over after Skip(): %q", len(left), left)
}
}
func BenchmarkMarshalMsgnsScannerResp(b *testing.B) {
v := nsScannerResp{}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.MarshalMsg(nil)
}
}
func BenchmarkAppendMsgnsScannerResp(b *testing.B) {
v := nsScannerResp{}
bts := make([]byte, 0, v.Msgsize())
bts, _ = v.MarshalMsg(bts[0:0])
b.SetBytes(int64(len(bts)))
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
bts, _ = v.MarshalMsg(bts[0:0])
}
}
func BenchmarkUnmarshalnsScannerResp(b *testing.B) {
v := nsScannerResp{}
bts, _ := v.MarshalMsg(nil)
b.ReportAllocs()
b.SetBytes(int64(len(bts)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := v.UnmarshalMsg(bts)
if err != nil {
b.Fatal(err)
}
}
}
func TestEncodeDecodensScannerResp(t *testing.T) {
v := nsScannerResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
m := v.Msgsize()
if buf.Len() > m {
t.Log("WARNING: TestEncodeDecodensScannerResp Msgsize() is inaccurate")
}
vn := nsScannerResp{}
err := msgp.Decode(&buf, &vn)
if err != nil {
t.Error(err)
}
buf.Reset()
msgp.Encode(&buf, &v)
err = msgp.NewReader(&buf).Skip()
if err != nil {
t.Error(err)
}
}
func BenchmarkEncodensScannerResp(b *testing.B) {
v := nsScannerResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
en := msgp.NewWriter(msgp.Nowhere)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
v.EncodeMsg(en)
}
en.Flush()
}
func BenchmarkDecodensScannerResp(b *testing.B) {
v := nsScannerResp{}
var buf bytes.Buffer
msgp.Encode(&buf, &v)
b.SetBytes(int64(buf.Len()))
rd := msgp.NewEndlessReader(buf.Bytes(), b)
dc := msgp.NewReader(rd)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := v.DecodeMsg(dc)
if err != nil {
b.Fatal(err)
}
}
}

View File

@@ -33,8 +33,10 @@ import (
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/minio/minio/internal/grid"
"github.com/tinylib/msgp/msgp"
jwtreq "github.com/golang-jwt/jwt/v4/request"
@@ -116,7 +118,7 @@ func storageServerRequestValidate(r *http.Request) error {
return nil
}
// IsValid - To authenticate and verify the time difference.
// IsAuthValid - To authenticate and verify the time difference.
func (s *storageRESTServer) IsAuthValid(w http.ResponseWriter, r *http.Request) bool {
if s.storage == nil {
s.writeErrorResponse(w, errDiskNotFound)
@@ -165,56 +167,63 @@ func (s *storageRESTServer) IsValid(w http.ResponseWriter, r *http.Request) bool
return true
}
// checkID - check if the disk-id in the request corresponds to the underlying disk.
func (s *storageRESTServer) checkID(wantID string) bool {
if s.storage == nil {
return false
}
if wantID == "" {
// Request sent empty disk-id, we allow the request
// as the peer might be coming up and trying to read format.json
// or create format.json
return true
}
storedDiskID, err := s.storage.GetDiskID()
if err != nil {
return false
}
return wantID == storedDiskID
}
// HealthHandler handler checks if disk is stale
func (s *storageRESTServer) HealthHandler(w http.ResponseWriter, r *http.Request) {
s.IsValid(w, r)
}
// DiskInfo types.
// DiskInfo.Metrics elements are shared, so we cannot reuse.
var storageDiskInfoHandler = grid.NewSingleHandler[*grid.MSS, *DiskInfo](grid.HandlerDiskInfo, grid.NewMSS, func() *DiskInfo { return &DiskInfo{} }).WithSharedResponse()
// DiskInfoHandler - returns disk info.
func (s *storageRESTServer) DiskInfoHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsAuthValid(w, r) {
return
func (s *storageRESTServer) DiskInfoHandler(params *grid.MSS) (*DiskInfo, *grid.RemoteErr) {
if !s.checkID(params.Get(storageRESTDiskID)) {
return nil, grid.NewRemoteErr(errDiskNotFound)
}
info, err := s.storage.DiskInfo(r.Context(), r.Form.Get(storageRESTMetrics) == "true")
withMetrics := params.Get(storageRESTMetrics) == "true"
info, err := s.storage.DiskInfo(context.Background(), withMetrics)
if err != nil {
info.Error = err.Error()
}
logger.LogIf(r.Context(), msgp.Encode(w, &info))
info.Scanning = s.storage != nil && s.storage.storage != nil && atomic.LoadInt32(&s.storage.storage.scanning) > 0
return &info, nil
}
func (s *storageRESTServer) NSScannerHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
// scanner rpc handler.
var storageNSScannerHandler = grid.NewStream[*nsScannerOptions, grid.NoPayload, *nsScannerResp](grid.HandlerNSScanner,
func() *nsScannerOptions { return &nsScannerOptions{} },
nil,
func() *nsScannerResp { return &nsScannerResp{} })
func (s *storageRESTServer) NSScannerHandler(ctx context.Context, params *nsScannerOptions, out chan<- *nsScannerResp) *grid.RemoteErr {
if !s.checkID(params.DiskID) {
return grid.NewRemoteErr(errDiskNotFound)
}
scanMode, err := strconv.Atoi(r.Form.Get(storageRESTScanMode))
if err != nil {
logger.LogIf(r.Context(), err)
s.writeErrorResponse(w, err)
return
if params.Cache == nil {
return grid.NewRemoteErrString("NSScannerHandler: provided cache is nil")
}
setEventStreamHeaders(w)
var cache dataUsageCache
err = cache.deserialize(r.Body)
if err != nil {
logger.LogIf(r.Context(), err)
s.writeErrorResponse(w, err)
return
}
ctx, cancel := context.WithCancel(r.Context())
defer cancel()
resp := streamHTTPResponse(w)
defer func() {
if r := recover(); r != nil {
debug.PrintStack()
resp.CloseWithError(fmt.Errorf("panic: %v", r))
}
}()
respW := msgp.NewWriter(resp)
// Collect updates, stream them before the full cache is sent.
updates := make(chan dataUsageEntry, 1)
var wg sync.WaitGroup
@@ -222,36 +231,21 @@ func (s *storageRESTServer) NSScannerHandler(w http.ResponseWriter, r *http.Requ
go func() {
defer wg.Done()
for update := range updates {
// Write true bool to indicate update.
var err error
if err = respW.WriteBool(true); err == nil {
err = update.EncodeMsg(respW)
}
respW.Flush()
if err != nil {
cancel()
resp.CloseWithError(err)
return
}
resp := storageNSScannerHandler.NewResponse()
resp.Update = &update
out <- resp
}
}()
usageInfo, err := s.storage.NSScanner(ctx, cache, updates, madmin.HealScanMode(scanMode))
if err != nil {
respW.Flush()
resp.CloseWithError(err)
return
}
// Write false bool to indicate we finished.
ui, err := s.storage.NSScanner(ctx, *params.Cache, updates, madmin.HealScanMode(params.ScanMode))
wg.Wait()
if err = respW.WriteBool(false); err == nil {
err = usageInfo.EncodeMsg(respW)
}
if err != nil {
resp.CloseWithError(err)
return
return grid.NewRemoteErr(err)
}
resp.CloseWithError(respW.Flush())
// Send final response.
resp := storageNSScannerHandler.NewResponse()
resp.Final = &ui
out <- resp
return nil
}
// MakeVolHandler - make a volume.
@@ -291,21 +285,22 @@ func (s *storageRESTServer) ListVolsHandler(w http.ResponseWriter, r *http.Reque
logger.LogIf(r.Context(), msgp.Encode(w, VolsInfo(infos)))
}
// statvol types.
var storageStatVolHandler = grid.NewSingleHandler[*grid.MSS, *VolInfo](grid.HandlerStatVol, grid.NewMSS, func() *VolInfo { return &VolInfo{} })
// StatVolHandler - stat a volume.
func (s *storageRESTServer) StatVolHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
func (s *storageRESTServer) StatVolHandler(params *grid.MSS) (*VolInfo, *grid.RemoteErr) {
if !s.checkID(params.Get(storageRESTDiskID)) {
return nil, grid.NewRemoteErr(errDiskNotFound)
}
volume := r.Form.Get(storageRESTVolume)
info, err := s.storage.StatVol(r.Context(), volume)
info, err := s.storage.StatVol(context.Background(), params.Get(storageRESTVolume))
if err != nil {
s.writeErrorResponse(w, err)
return
return nil, grid.NewRemoteErr(err)
}
logger.LogIf(r.Context(), msgp.Encode(w, &info))
return &info, nil
}
// DeleteVolumeHandler - delete a volume.
// DeleteVolHandler - delete a volume.
func (s *storageRESTServer) DeleteVolHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
@@ -357,37 +352,48 @@ func (s *storageRESTServer) CreateFileHandler(w http.ResponseWriter, r *http.Req
done(s.storage.CreateFile(r.Context(), volume, filePath, int64(fileSize), body))
}
// DeleteVersion delete updated metadata.
func (s *storageRESTServer) DeleteVersionHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
filePath := r.Form.Get(storageRESTFilePath)
forceDelMarker, err := strconv.ParseBool(r.Form.Get(storageRESTForceDelMarker))
if err != nil {
s.writeErrorResponse(w, errInvalidArgument)
return
}
var storageDeleteVersionHandler = grid.NewSingleHandler[*DeleteVersionHandlerParams, grid.NoPayload](grid.HandlerDeleteVersion, func() *DeleteVersionHandlerParams {
return &DeleteVersionHandlerParams{}
}, grid.NewNoPayload)
if r.ContentLength < 0 {
s.writeErrorResponse(w, errInvalidArgument)
return
// DeleteVersionHandler delete updated metadata.
func (s *storageRESTServer) DeleteVersionHandler(p *DeleteVersionHandlerParams) (np grid.NoPayload, gerr *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return np, grid.NewRemoteErr(errDiskNotFound)
}
volume := p.Volume
filePath := p.FilePath
forceDelMarker := p.ForceDelMarker
var fi FileInfo
if err := msgp.Decode(r.Body, &fi); err != nil {
s.writeErrorResponse(w, err)
return
}
err = s.storage.DeleteVersion(r.Context(), volume, filePath, fi, forceDelMarker)
if err != nil {
s.writeErrorResponse(w, err)
}
err := s.storage.DeleteVersion(context.Background(), volume, filePath, p.FI, forceDelMarker)
return np, grid.NewRemoteErr(err)
}
// ReadVersion read metadata of versionID
var storageReadVersionHandler = grid.NewSingleHandler[*grid.MSS, *FileInfo](grid.HandlerReadVersion, grid.NewMSS, func() *FileInfo {
return &FileInfo{}
})
// ReadVersionHandlerWS read metadata of versionID
func (s *storageRESTServer) ReadVersionHandlerWS(params *grid.MSS) (*FileInfo, *grid.RemoteErr) {
if !s.checkID(params.Get(storageRESTDiskID)) {
return nil, grid.NewRemoteErr(errDiskNotFound)
}
volume := params.Get(storageRESTVolume)
filePath := params.Get(storageRESTFilePath)
versionID := params.Get(storageRESTVersionID)
readData, err := strconv.ParseBool(params.Get(storageRESTReadData))
if err != nil {
return nil, grid.NewRemoteErr(err)
}
fi, err := s.storage.ReadVersion(context.Background(), volume, filePath, versionID, readData)
if err != nil {
return nil, grid.NewRemoteErr(err)
}
return &fi, nil
}
// ReadVersionHandler read metadata of versionID
func (s *storageRESTServer) ReadVersionHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
@@ -410,55 +416,35 @@ func (s *storageRESTServer) ReadVersionHandler(w http.ResponseWriter, r *http.Re
logger.LogIf(r.Context(), msgp.Encode(w, &fi))
}
// WriteMetadata write new updated metadata.
func (s *storageRESTServer) WriteMetadataHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
filePath := r.Form.Get(storageRESTFilePath)
var storageWriteMetadataHandler = grid.NewSingleHandler[*MetadataHandlerParams, grid.NoPayload](grid.HandlerWriteMetadata, func() *MetadataHandlerParams {
return &MetadataHandlerParams{}
}, grid.NewNoPayload)
if r.ContentLength < 0 {
s.writeErrorResponse(w, errInvalidArgument)
return
// WriteMetadataHandler rpc handler to write new updated metadata.
func (s *storageRESTServer) WriteMetadataHandler(p *MetadataHandlerParams) (np grid.NoPayload, gerr *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return grid.NewNPErr(errDiskNotFound)
}
volume := p.Volume
filePath := p.FilePath
var fi FileInfo
if err := msgp.Decode(r.Body, &fi); err != nil {
s.writeErrorResponse(w, err)
return
}
err := s.storage.WriteMetadata(r.Context(), volume, filePath, fi)
if err != nil {
s.writeErrorResponse(w, err)
}
err := s.storage.WriteMetadata(context.Background(), volume, filePath, p.FI)
return np, grid.NewRemoteErr(err)
}
// UpdateMetadata update new updated metadata.
func (s *storageRESTServer) UpdateMetadataHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
filePath := r.Form.Get(storageRESTFilePath)
noPersistence := r.Form.Get(storageRESTNoPersistence) == "true"
var storageUpdateMetadataHandler = grid.NewSingleHandler[*MetadataHandlerParams, grid.NoPayload](grid.HandlerUpdateMetadata, func() *MetadataHandlerParams {
return &MetadataHandlerParams{}
}, grid.NewNoPayload)
if r.ContentLength < 0 {
s.writeErrorResponse(w, errInvalidArgument)
return
// UpdateMetadataHandler update new updated metadata.
func (s *storageRESTServer) UpdateMetadataHandler(p *MetadataHandlerParams) (grid.NoPayload, *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return grid.NewNPErr(errDiskNotFound)
}
volume := p.Volume
filePath := p.FilePath
var fi FileInfo
if err := msgp.Decode(r.Body, &fi); err != nil {
s.writeErrorResponse(w, err)
return
}
err := s.storage.UpdateMetadata(r.Context(), volume, filePath, fi, UpdateMetadataOpts{NoPersistence: noPersistence})
if err != nil {
s.writeErrorResponse(w, err)
}
return grid.NewNPErr(s.storage.UpdateMetadata(context.Background(), volume, filePath, p.FI, p.UpdateOpts))
}
// WriteAllHandler - write to file all content.
@@ -485,28 +471,18 @@ func (s *storageRESTServer) WriteAllHandler(w http.ResponseWriter, r *http.Reque
}
}
var storageCheckPartsHandler = grid.NewSingleHandler[*CheckPartsHandlerParams, grid.NoPayload](grid.HandlerCheckParts, func() *CheckPartsHandlerParams {
return &CheckPartsHandlerParams{}
}, grid.NewNoPayload)
// CheckPartsHandler - check if a file metadata exists.
func (s *storageRESTServer) CheckPartsHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
filePath := r.Form.Get(storageRESTFilePath)
if r.ContentLength < 0 {
s.writeErrorResponse(w, errInvalidArgument)
return
}
var fi FileInfo
if err := msgp.Decode(r.Body, &fi); err != nil {
s.writeErrorResponse(w, err)
return
}
if err := s.storage.CheckParts(r.Context(), volume, filePath, fi); err != nil {
s.writeErrorResponse(w, err)
func (s *storageRESTServer) CheckPartsHandler(p *CheckPartsHandlerParams) (grid.NoPayload, *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return grid.NewNPErr(errDiskNotFound)
}
volume := p.Volume
filePath := p.FilePath
return grid.NewNPErr(s.storage.CheckParts(context.Background(), volume, filePath, p.FI))
}
// ReadAllHandler - read all the contents of a file.
@@ -550,6 +526,30 @@ func (s *storageRESTServer) ReadXLHandler(w http.ResponseWriter, r *http.Request
logger.LogIf(r.Context(), msgp.Encode(w, &rf))
}
var storageReadXLHandler = grid.NewSingleHandler[*grid.MSS, *RawFileInfo](grid.HandlerReadXL, grid.NewMSS, func() *RawFileInfo {
return &RawFileInfo{}
})
// ReadXLHandlerWS - read xl.meta for an object at path.
func (s *storageRESTServer) ReadXLHandlerWS(params *grid.MSS) (*RawFileInfo, *grid.RemoteErr) {
if !s.checkID(params.Get(storageRESTDiskID)) {
return nil, grid.NewRemoteErr(errDiskNotFound)
}
volume := params.Get(storageRESTVolume)
filePath := params.Get(storageRESTFilePath)
readData, err := strconv.ParseBool(params.Get(storageRESTReadData))
if err != nil {
return nil, grid.NewRemoteErr(err)
}
rf, err := s.storage.ReadXL(context.Background(), volume, filePath, readData)
if err != nil {
return nil, grid.NewRemoteErr(err)
}
return &rf, nil
}
// ReadFileHandler - read section of a file.
func (s *storageRESTServer) ReadFileHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
@@ -593,7 +593,7 @@ func (s *storageRESTServer) ReadFileHandler(w http.ResponseWriter, r *http.Reque
w.Write(buf)
}
// ReadFileHandler - read section of a file.
// ReadFileStreamHandler - read section of a file.
func (s *storageRESTServer) ReadFileStreamHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
@@ -666,30 +666,16 @@ func (s *storageRESTServer) ListDirHandler(w http.ResponseWriter, r *http.Reques
gob.NewEncoder(w).Encode(&entries)
}
var storageDeleteFileHandler = grid.NewSingleHandler[*DeleteFileHandlerParams, grid.NoPayload](grid.HandlerDeleteFile, func() *DeleteFileHandlerParams {
return &DeleteFileHandlerParams{}
}, grid.NewNoPayload)
// DeleteFileHandler - delete a file.
func (s *storageRESTServer) DeleteFileHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
}
volume := r.Form.Get(storageRESTVolume)
filePath := r.Form.Get(storageRESTFilePath)
recursive, err := strconv.ParseBool(r.Form.Get(storageRESTRecursive))
if err != nil {
s.writeErrorResponse(w, err)
return
}
force, err := strconv.ParseBool(r.Form.Get(storageRESTForceDelete))
if err != nil {
s.writeErrorResponse(w, err)
return
}
err = s.storage.Delete(r.Context(), volume, filePath, DeleteOptions{
Recursive: recursive,
Force: force,
})
if err != nil {
s.writeErrorResponse(w, err)
func (s *storageRESTServer) DeleteFileHandler(p *DeleteFileHandlerParams) (grid.NoPayload, *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return grid.NewNPErr(errDiskNotFound)
}
return grid.NewNPErr(s.storage.Delete(context.Background(), p.Volume, p.FilePath, p.Opts))
}
// DeleteVersionsErrsResp - collection of delete errors
@@ -737,48 +723,23 @@ func (s *storageRESTServer) DeleteVersionsHandler(w http.ResponseWriter, r *http
encoder.Encode(dErrsResp)
}
// RenameDataResp - RenameData()'s response.
type RenameDataResp struct {
Signature uint64
Err error
}
var storageRenameDataHandler = grid.NewSingleHandler[*RenameDataHandlerParams, *RenameDataResp](grid.HandlerRenamedata, func() *RenameDataHandlerParams {
return &RenameDataHandlerParams{}
}, func() *RenameDataResp {
return &RenameDataResp{}
})
// RenameDataHandler - renames a meta object and data dir to destination.
func (s *storageRESTServer) RenameDataHandler(w http.ResponseWriter, r *http.Request) {
if !s.IsValid(w, r) {
return
func (s *storageRESTServer) RenameDataHandler(p *RenameDataHandlerParams) (*RenameDataResp, *grid.RemoteErr) {
if !s.checkID(p.DiskID) {
return nil, grid.NewRemoteErr(errDiskNotFound)
}
srcVolume := r.Form.Get(storageRESTSrcVolume)
srcFilePath := r.Form.Get(storageRESTSrcPath)
dstVolume := r.Form.Get(storageRESTDstVolume)
dstFilePath := r.Form.Get(storageRESTDstPath)
if r.ContentLength < 0 {
s.writeErrorResponse(w, errInvalidArgument)
return
}
var fi FileInfo
if err := msgp.Decode(r.Body, &fi); err != nil {
s.writeErrorResponse(w, err)
return
}
setEventStreamHeaders(w)
encoder := gob.NewEncoder(w)
done := keepHTTPResponseAlive(w)
sign, err := s.storage.RenameData(r.Context(), srcVolume, srcFilePath, fi, dstVolume, dstFilePath)
done(nil)
sign, err := s.storage.RenameData(context.Background(), p.SrcVolume, p.SrcPath, p.FI, p.DstVolume, p.DstPath)
resp := &RenameDataResp{
Signature: sign,
}
if err != nil {
resp.Err = StorageErr(err.Error())
}
encoder.Encode(resp)
return resp, grid.NewRemoteErr(err)
}
// RenameFileHandler - rename a file.
@@ -1349,8 +1310,8 @@ func (s *storageRESTServer) ReadMultiple(w http.ResponseWriter, r *http.Request)
rw.CloseWithError(err)
}
// registerStorageRPCRouter - register storage rpc router.
func registerStorageRESTHandlers(router *mux.Router, endpointServerPools EndpointServerPools) {
// registerStorageRESTHandlers - register storage rpc router.
func registerStorageRESTHandlers(router *mux.Router, endpointServerPools EndpointServerPools, gm *grid.Manager) {
storageDisks := make([][]*xlStorage, len(endpointServerPools))
for poolIdx, ep := range endpointServerPools {
storageDisks[poolIdx] = make([]*xlStorage, len(ep.Endpoints))
@@ -1394,38 +1355,44 @@ func registerStorageRESTHandlers(router *mux.Router, endpointServerPools Endpoin
subrouter := router.PathPrefix(path.Join(storageRESTPrefix, endpoint.Path)).Subrouter()
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodHealth).HandlerFunc(h(server.HealthHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodDiskInfo).HandlerFunc(h(server.DiskInfoHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodNSScanner).HandlerFunc(h(server.NSScannerHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodMakeVol).HandlerFunc(h(server.MakeVolHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodMakeVolBulk).HandlerFunc(h(server.MakeVolBulkHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodStatVol).HandlerFunc(h(server.StatVolHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodDeleteVol).HandlerFunc(h(server.DeleteVolHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodListVols).HandlerFunc(h(server.ListVolsHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodAppendFile).HandlerFunc(h(server.AppendFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodWriteAll).HandlerFunc(h(server.WriteAllHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodWriteMetadata).HandlerFunc(h(server.WriteMetadataHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodUpdateMetadata).HandlerFunc(h(server.UpdateMetadataHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodDeleteVersion).HandlerFunc(h(server.DeleteVersionHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadVersion).HandlerFunc(h(server.ReadVersionHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadXL).HandlerFunc(h(server.ReadXLHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodRenameData).HandlerFunc(h(server.RenameDataHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodCreateFile).HandlerFunc(h(server.CreateFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodCheckParts).HandlerFunc(h(server.CheckPartsHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadAll).HandlerFunc(h(server.ReadAllHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadFile).HandlerFunc(h(server.ReadFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadFileStream).HandlerFunc(h(server.ReadFileStreamHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodListDir).HandlerFunc(h(server.ListDirHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodDeleteVersions).HandlerFunc(h(server.DeleteVersionsHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodDeleteFile).HandlerFunc(h(server.DeleteFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodRenameFile).HandlerFunc(h(server.RenameFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodVerifyFile).HandlerFunc(h(server.VerifyFileHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodWalkDir).HandlerFunc(h(server.WalkDirHandler))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodStatInfoFile).HandlerFunc(h(server.StatInfoFile))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodReadMultiple).HandlerFunc(h(server.ReadMultiple))
subrouter.Methods(http.MethodPost).Path(storageRESTVersionPrefix + storageRESTMethodCleanAbandoned).HandlerFunc(h(server.CleanAbandonedDataHandler))
logger.FatalIf(storageRenameDataHandler.Register(gm, server.RenameDataHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageDeleteFileHandler.Register(gm, server.DeleteFileHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageCheckPartsHandler.Register(gm, server.CheckPartsHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageReadVersionHandler.Register(gm, server.ReadVersionHandlerWS, endpoint.Path), "unable to register handler")
logger.FatalIf(storageWriteMetadataHandler.Register(gm, server.WriteMetadataHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageUpdateMetadataHandler.Register(gm, server.UpdateMetadataHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageDeleteVersionHandler.Register(gm, server.DeleteVersionHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageReadXLHandler.Register(gm, server.ReadXLHandlerWS, endpoint.Path), "unable to register handler")
logger.FatalIf(storageNSScannerHandler.RegisterNoInput(gm, server.NSScannerHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageDiskInfoHandler.Register(gm, server.DiskInfoHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(storageStatVolHandler.Register(gm, server.StatVolHandler, endpoint.Path), "unable to register handler")
logger.FatalIf(gm.RegisterStreamingHandler(grid.HandlerWalkDir, grid.StreamHandler{
Subroute: endpoint.Path,
Handle: server.WalkDirHandler,
OutCapacity: 1,
}), "unable to register handler")
}
}
}

View File

@@ -20,12 +20,11 @@ package cmd
import (
"bytes"
"context"
"net/http/httptest"
"reflect"
"runtime"
"testing"
"github.com/minio/mux"
"github.com/minio/minio/internal/grid"
xnet "github.com/minio/pkg/v2/net"
)
@@ -437,17 +436,21 @@ func testStorageAPIRenameFile(t *testing.T, storage StorageAPI) {
}
}
func newStorageRESTHTTPServerClient(t *testing.T) *storageRESTClient {
func newStorageRESTHTTPServerClient(t testing.TB) *storageRESTClient {
// Grid with 2 hosts
tg, err := grid.SetupTestGrid(2)
if err != nil {
t.Fatalf("SetupTestGrid: %v", err)
}
t.Cleanup(tg.Cleanup)
prevHost, prevPort := globalMinioHost, globalMinioPort
defer func() {
globalMinioHost, globalMinioPort = prevHost, prevPort
}()
// tg[0] = local, tg[1] = remote
router := mux.NewRouter()
httpServer := httptest.NewServer(router)
t.Cleanup(httpServer.Close)
url, err := xnet.ParseHTTPURL(httpServer.URL)
// Remote URL
url, err := xnet.ParseHTTPURL(tg.Servers[1].URL)
if err != nil {
t.Fatalf("unexpected error %v", err)
}
@@ -464,11 +467,18 @@ func newStorageRESTHTTPServerClient(t *testing.T) *storageRESTClient {
t.Fatalf("UpdateIsLocal failed %v", err)
}
registerStorageRESTHandlers(router, []PoolEndpoints{{
// Register handlers on newly created servers
registerStorageRESTHandlers(tg.Mux[0], []PoolEndpoints{{
Endpoints: Endpoints{endpoint},
}})
}}, tg.Managers[0])
registerStorageRESTHandlers(tg.Mux[1], []PoolEndpoints{{
Endpoints: Endpoints{endpoint},
}}, tg.Managers[1])
restClient := newStorageRESTClient(endpoint, false)
restClient, err := newStorageRESTClient(endpoint, false, tg.Managers[0])
if err != nil {
t.Fatal(err)
}
return restClient
}

View File

@@ -1121,6 +1121,21 @@ func (p *xlStorageDiskIDCheck) monitorDiskWritable(ctx context.Context) {
}
}
// checkID will check if the disk ID matches the provided ID.
func (p *xlStorageDiskIDCheck) checkID(wantID string) (err error) {
if wantID == "" {
return nil
}
id, err := p.storage.GetDiskID()
if err != nil {
return err
}
if id != wantID {
return fmt.Errorf("disk ID %s does not match. disk reports %s", wantID, id)
}
return nil
}
// diskHealthCheckOK will check if the provided error is nil
// and update disk status if good.
// For convenience a bool is returned to indicate any error state

View File

@@ -1226,11 +1226,6 @@ func (s *xlStorage) DeleteVersion(ctx context.Context, volume, path string, fi F
return s.deleteFile(volumeDir, filePath, true, false)
}
// UpdateMetadataOpts provides an optional input to indicate if xl.meta updates need to be fully synced to disk.
type UpdateMetadataOpts struct {
NoPersistence bool
}
// Updates only metadata for a given version.
func (s *xlStorage) UpdateMetadata(ctx context.Context, volume, path string, fi FileInfo, opts UpdateMetadataOpts) error {
if len(fi.Metadata) == 0 {