mirror of
https://github.com/minio/minio.git
synced 2025-11-07 21:02:58 -05:00
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:
@@ -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 {
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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
83
cmd/grid.go
Normal 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
|
||||
}
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
|
||||
@@ -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())}
|
||||
}
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
285
cmd/metacache-walk_gen.go
Normal 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
|
||||
}
|
||||
123
cmd/metacache-walk_gen_test.go
Normal file
123
cmd/metacache-walk_gen_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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())
|
||||
}
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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))).
|
||||
|
||||
@@ -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
@@ -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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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"`
|
||||
}
|
||||
|
||||
418
cmd/storage-rest-common_gen.go
Normal file
418
cmd/storage-rest-common_gen.go
Normal 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
|
||||
}
|
||||
236
cmd/storage-rest-common_gen_test.go
Normal file
236
cmd/storage-rest-common_gen_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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 {
|
||||
|
||||
Reference in New Issue
Block a user