mirror of
https://github.com/minio/minio.git
synced 2025-02-04 02:15:59 -05:00
Add Support for Cache and S3 related metrics in Prometheus endpoint (#8591)
This PR adds support below metrics - Cache Hit Count - Cache Miss Count - Data served from Cache (in Bytes) - Bytes received from AWS S3 - Bytes sent to AWS S3 - Number of requests sent to AWS S3 Fixes #8549
This commit is contained in:
parent
d2dc964cb5
commit
3df7285c3c
@ -1028,7 +1028,7 @@ func mustTrace(entry interface{}, trcAll, errOnly bool) bool {
|
|||||||
if !ok {
|
if !ok {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
trace := trcAll || !hasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath+SlashSeparator)
|
trace := trcAll || !HasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath+SlashSeparator)
|
||||||
if errOnly {
|
if errOnly {
|
||||||
return trace && trcInfo.RespInfo.StatusCode >= http.StatusBadRequest
|
return trace && trcInfo.RespInfo.StatusCode >= http.StatusBadRequest
|
||||||
}
|
}
|
||||||
|
@ -94,7 +94,7 @@ func setObjectHeaders(w http.ResponseWriter, objInfo ObjectInfo, rs *HTTPRangeSp
|
|||||||
|
|
||||||
// Set all other user defined metadata.
|
// Set all other user defined metadata.
|
||||||
for k, v := range objInfo.UserDefined {
|
for k, v := range objInfo.UserDefined {
|
||||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||||
// Do not need to send any internal metadata
|
// Do not need to send any internal metadata
|
||||||
// values to client.
|
// values to client.
|
||||||
continue
|
continue
|
||||||
|
@ -526,7 +526,7 @@ func generateListObjectsV2Response(bucket, prefix, token, nextToken, startAfter,
|
|||||||
if metadata {
|
if metadata {
|
||||||
content.UserMetadata = make(StringMap)
|
content.UserMetadata = make(StringMap)
|
||||||
for k, v := range CleanMinioInternalMetadataKeys(object.UserDefined) {
|
for k, v := range CleanMinioInternalMetadataKeys(object.UserDefined) {
|
||||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||||
// Do not need to send any internal metadata
|
// Do not need to send any internal metadata
|
||||||
// values to client.
|
// values to client.
|
||||||
continue
|
continue
|
||||||
|
@ -214,7 +214,7 @@ func (api objectAPIHandlers) ListMultipartUploadsHandler(w http.ResponseWriter,
|
|||||||
|
|
||||||
if keyMarker != "" {
|
if keyMarker != "" {
|
||||||
// Marker not common with prefix is not implemented.
|
// Marker not common with prefix is not implemented.
|
||||||
if !hasPrefix(keyMarker, prefix) {
|
if !HasPrefix(keyMarker, prefix) {
|
||||||
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, errorCodes.ToAPIErr(ErrNotImplemented), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -750,7 +750,7 @@ func (api objectAPIHandlers) PostPolicyBucketHandler(w http.ResponseWriter, r *h
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
if objectAPI.IsEncryptionSupported() {
|
if objectAPI.IsEncryptionSupported() {
|
||||||
if crypto.IsRequested(formValues) && !hasSuffix(object, SlashSeparator) { // handle SSE requests
|
if crypto.IsRequested(formValues) && !HasSuffix(object, SlashSeparator) { // handle SSE requests
|
||||||
if crypto.SSECopy.IsRequested(r.Header) {
|
if crypto.SSECopy.IsRequested(r.Header) {
|
||||||
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
|
@ -334,14 +334,14 @@ func (c *diskCache) updateMetadataIfChanged(ctx context.Context, bucket, object
|
|||||||
bkMeta := make(map[string]string)
|
bkMeta := make(map[string]string)
|
||||||
cacheMeta := make(map[string]string)
|
cacheMeta := make(map[string]string)
|
||||||
for k, v := range bkObjectInfo.UserDefined {
|
for k, v := range bkObjectInfo.UserDefined {
|
||||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||||
// Do not need to send any internal metadata
|
// Do not need to send any internal metadata
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
bkMeta[http.CanonicalHeaderKey(k)] = v
|
bkMeta[http.CanonicalHeaderKey(k)] = v
|
||||||
}
|
}
|
||||||
for k, v := range cacheObjInfo.UserDefined {
|
for k, v := range cacheObjInfo.UserDefined {
|
||||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||||
// Do not need to send any internal metadata
|
// Do not need to send any internal metadata
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
@ -602,7 +602,7 @@ func (c *diskCache) Get(ctx context.Context, bucket, object string, rs *HTTPRang
|
|||||||
|
|
||||||
var nsUnlocker = func() {}
|
var nsUnlocker = func() {}
|
||||||
// For a directory, we need to send an reader that returns no bytes.
|
// For a directory, we need to send an reader that returns no bytes.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
// The lock taken above is released when
|
// The lock taken above is released when
|
||||||
// objReader.Close() is called by the caller.
|
// objReader.Close() is called by the caller.
|
||||||
return NewGetObjectReaderFromReader(bytes.NewBuffer(nil), objInfo, opts.CheckCopyPrecondFn, nsUnlocker)
|
return NewGetObjectReaderFromReader(bytes.NewBuffer(nil), objInfo, opts.CheckCopyPrecondFn, nsUnlocker)
|
||||||
|
64
cmd/disk-cache-stats.go
Normal file
64
cmd/disk-cache-stats.go
Normal file
@ -0,0 +1,64 @@
|
|||||||
|
/*
|
||||||
|
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
* you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package cmd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"go.uber.org/atomic"
|
||||||
|
)
|
||||||
|
|
||||||
|
// CacheStats - represents bytes served from cache,
|
||||||
|
// cache hits and cache misses.
|
||||||
|
type CacheStats struct {
|
||||||
|
BytesServed atomic.Uint64
|
||||||
|
Hits atomic.Uint64
|
||||||
|
Misses atomic.Uint64
|
||||||
|
}
|
||||||
|
|
||||||
|
// Increase total bytes served from cache
|
||||||
|
func (s *CacheStats) incBytesServed(n int64) {
|
||||||
|
s.BytesServed.Add(uint64(n))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Increase cache hit by 1
|
||||||
|
func (s *CacheStats) incHit() {
|
||||||
|
s.Hits.Add(uint64(1))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Increase cache miss by 1
|
||||||
|
func (s *CacheStats) incMiss() {
|
||||||
|
s.Misses.Add(uint64(1))
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get total bytes served
|
||||||
|
func (s *CacheStats) getBytesServed() uint64 {
|
||||||
|
return s.BytesServed.Load()
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get total cache hits
|
||||||
|
func (s *CacheStats) getHits() uint64 {
|
||||||
|
return s.Hits.Load()
|
||||||
|
}
|
||||||
|
|
||||||
|
// Get total cache misses
|
||||||
|
func (s *CacheStats) getMisses() uint64 {
|
||||||
|
return s.Misses.Load()
|
||||||
|
}
|
||||||
|
|
||||||
|
// Prepare new CacheStats structure
|
||||||
|
func newCacheStats() *CacheStats {
|
||||||
|
return &CacheStats{}
|
||||||
|
}
|
@ -57,6 +57,7 @@ type CacheObjectLayer interface {
|
|||||||
PutObject(ctx context.Context, bucket, object string, data *PutObjReader, opts ObjectOptions) (objInfo ObjectInfo, err error)
|
PutObject(ctx context.Context, bucket, object string, data *PutObjReader, opts ObjectOptions) (objInfo ObjectInfo, err error)
|
||||||
// Storage operations.
|
// Storage operations.
|
||||||
StorageInfo(ctx context.Context) CacheStorageInfo
|
StorageInfo(ctx context.Context) CacheStorageInfo
|
||||||
|
CacheStats() *CacheStats
|
||||||
}
|
}
|
||||||
|
|
||||||
// Abstracts disk caching - used by the S3 layer
|
// Abstracts disk caching - used by the S3 layer
|
||||||
@ -74,6 +75,9 @@ type cacheObjects struct {
|
|||||||
// nsMutex namespace lock
|
// nsMutex namespace lock
|
||||||
nsMutex *nsLockMap
|
nsMutex *nsLockMap
|
||||||
|
|
||||||
|
// Cache stats
|
||||||
|
cacheStats *CacheStats
|
||||||
|
|
||||||
// Object functions pointing to the corresponding functions of backend implementation.
|
// Object functions pointing to the corresponding functions of backend implementation.
|
||||||
NewNSLockFn func(ctx context.Context, bucket, object string) RWLocker
|
NewNSLockFn func(ctx context.Context, bucket, object string) RWLocker
|
||||||
GetObjectNInfoFn func(ctx context.Context, bucket, object string, rs *HTTPRangeSpec, h http.Header, lockType LockType, opts ObjectOptions) (gr *GetObjectReader, err error)
|
GetObjectNInfoFn func(ctx context.Context, bucket, object string, rs *HTTPRangeSpec, h http.Header, lockType LockType, opts ObjectOptions) (gr *GetObjectReader, err error)
|
||||||
@ -181,11 +185,17 @@ func (c *cacheObjects) GetObjectNInfo(ctx context.Context, bucket, object string
|
|||||||
cacheReader, cacheErr := c.get(ctx, dcache, bucket, object, rs, h, opts)
|
cacheReader, cacheErr := c.get(ctx, dcache, bucket, object, rs, h, opts)
|
||||||
if cacheErr == nil {
|
if cacheErr == nil {
|
||||||
cc = cacheControlOpts(cacheReader.ObjInfo)
|
cc = cacheControlOpts(cacheReader.ObjInfo)
|
||||||
if !cc.isEmpty() && !cc.isStale(cacheReader.ObjInfo.ModTime) {
|
if !cc.isStale(cacheReader.ObjInfo.ModTime) {
|
||||||
|
// This is a cache hit, mark it so
|
||||||
|
c.cacheStats.incHit()
|
||||||
|
c.cacheStats.incBytesServed(cacheReader.ObjInfo.Size)
|
||||||
return cacheReader, nil
|
return cacheReader, nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Reaching here implies cache miss
|
||||||
|
c.cacheStats.incMiss()
|
||||||
|
|
||||||
objInfo, err := c.GetObjectInfoFn(ctx, bucket, object, opts)
|
objInfo, err := c.GetObjectInfoFn(ctx, bucket, object, opts)
|
||||||
if backendDownError(err) && cacheErr == nil {
|
if backendDownError(err) && cacheErr == nil {
|
||||||
return cacheReader, nil
|
return cacheReader, nil
|
||||||
@ -282,10 +292,16 @@ func (c *cacheObjects) GetObjectInfo(ctx context.Context, bucket, object string,
|
|||||||
cachedObjInfo, cerr := c.stat(ctx, dcache, bucket, object)
|
cachedObjInfo, cerr := c.stat(ctx, dcache, bucket, object)
|
||||||
if cerr == nil {
|
if cerr == nil {
|
||||||
cc = cacheControlOpts(cachedObjInfo)
|
cc = cacheControlOpts(cachedObjInfo)
|
||||||
if !cc.isEmpty() && !cc.isStale(cachedObjInfo.ModTime) {
|
if !cc.isStale(cachedObjInfo.ModTime) {
|
||||||
|
// This is a cache hit, mark it so
|
||||||
|
c.cacheStats.incHit()
|
||||||
return cachedObjInfo, nil
|
return cachedObjInfo, nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Reaching here implies cache miss
|
||||||
|
c.cacheStats.incMiss()
|
||||||
|
|
||||||
objInfo, err := getObjectInfoFn(ctx, bucket, object, opts)
|
objInfo, err := getObjectInfoFn(ctx, bucket, object, opts)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if _, ok := err.(ObjectNotFound); ok {
|
if _, ok := err.(ObjectNotFound); ok {
|
||||||
@ -332,6 +348,11 @@ func (c *cacheObjects) StorageInfo(ctx context.Context) (cInfo CacheStorageInfo)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// CacheStats - returns underlying storage statistics.
|
||||||
|
func (c *cacheObjects) CacheStats() (cs *CacheStats) {
|
||||||
|
return c.cacheStats
|
||||||
|
}
|
||||||
|
|
||||||
// skipCache() returns true if cache migration is in progress
|
// skipCache() returns true if cache migration is in progress
|
||||||
func (c *cacheObjects) skipCache() bool {
|
func (c *cacheObjects) skipCache() bool {
|
||||||
c.migMutex.Lock()
|
c.migMutex.Lock()
|
||||||
@ -577,6 +598,7 @@ func newServerCacheObjects(ctx context.Context, config cache.Config) (CacheObjec
|
|||||||
migrating: migrateSw,
|
migrating: migrateSw,
|
||||||
migMutex: sync.Mutex{},
|
migMutex: sync.Mutex{},
|
||||||
nsMutex: newNSLock(false),
|
nsMutex: newNSLock(false),
|
||||||
|
cacheStats: newCacheStats(),
|
||||||
GetObjectInfoFn: func(ctx context.Context, bucket, object string, opts ObjectOptions) (ObjectInfo, error) {
|
GetObjectInfoFn: func(ctx context.Context, bucket, object string, opts ObjectOptions) (ObjectInfo, error) {
|
||||||
return newObjectLayerFn().GetObjectInfo(ctx, bucket, object, opts)
|
return newObjectLayerFn().GetObjectInfo(ctx, bucket, object, opts)
|
||||||
},
|
},
|
||||||
|
@ -34,7 +34,7 @@ func walk(ctx context.Context, path string, usageFn usageFunc) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
if !hasSuffix(path, SlashSeparator) {
|
if !HasSuffix(path, SlashSeparator) {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -142,7 +142,7 @@ func (m fsMetaV1) ToObjectInfo(bucket, object string, fi os.FileInfo) ObjectInfo
|
|||||||
m.Meta["content-type"] = mimedb.TypeByExtension(pathutil.Ext(object))
|
m.Meta["content-type"] = mimedb.TypeByExtension(pathutil.Ext(object))
|
||||||
}
|
}
|
||||||
|
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
m.Meta["etag"] = emptyETag // For directories etag is d41d8cd98f00b204e9800998ecf8427e
|
m.Meta["etag"] = emptyETag // For directories etag is d41d8cd98f00b204e9800998ecf8427e
|
||||||
m.Meta["content-type"] = "application/octet-stream"
|
m.Meta["content-type"] = "application/octet-stream"
|
||||||
}
|
}
|
||||||
|
12
cmd/fs-v1.go
12
cmd/fs-v1.go
@ -514,7 +514,7 @@ func (fs *FSObjects) GetObjectNInfo(ctx context.Context, bucket, object string,
|
|||||||
return nil, toObjectErr(err, bucket, object)
|
return nil, toObjectErr(err, bucket, object)
|
||||||
}
|
}
|
||||||
// For a directory, we need to send an reader that returns no bytes.
|
// For a directory, we need to send an reader that returns no bytes.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
// The lock taken above is released when
|
// The lock taken above is released when
|
||||||
// objReader.Close() is called by the caller.
|
// objReader.Close() is called by the caller.
|
||||||
return NewGetObjectReaderFromReader(bytes.NewBuffer(nil), objInfo, opts.CheckCopyPrecondFn, nsUnlocker)
|
return NewGetObjectReaderFromReader(bytes.NewBuffer(nil), objInfo, opts.CheckCopyPrecondFn, nsUnlocker)
|
||||||
@ -605,7 +605,7 @@ func (fs *FSObjects) getObject(ctx context.Context, bucket, object string, offse
|
|||||||
}
|
}
|
||||||
|
|
||||||
// If its a directory request, we return an empty body.
|
// If its a directory request, we return an empty body.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
_, err = writer.Write([]byte(""))
|
_, err = writer.Write([]byte(""))
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
return toObjectErr(err, bucket, object)
|
return toObjectErr(err, bucket, object)
|
||||||
@ -699,7 +699,7 @@ func (fs *FSObjects) defaultFsJSON(object string) fsMetaV1 {
|
|||||||
// getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo.
|
// getObjectInfo - wrapper for reading object metadata and constructs ObjectInfo.
|
||||||
func (fs *FSObjects) getObjectInfo(ctx context.Context, bucket, object string) (oi ObjectInfo, e error) {
|
func (fs *FSObjects) getObjectInfo(ctx context.Context, bucket, object string) (oi ObjectInfo, e error) {
|
||||||
fsMeta := fsMetaV1{}
|
fsMeta := fsMetaV1{}
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
fi, err := fsStatDir(ctx, pathJoin(fs.fsPath, bucket, object))
|
fi, err := fsStatDir(ctx, pathJoin(fs.fsPath, bucket, object))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return oi, err
|
return oi, err
|
||||||
@ -1167,6 +1167,12 @@ func (fs *FSObjects) ListObjectsHeal(ctx context.Context, bucket, prefix, marker
|
|||||||
return ListObjectsInfo{}, NotImplemented{}
|
return ListObjectsInfo{}, NotImplemented{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetMetrics - no op
|
||||||
|
func (fs *FSObjects) GetMetrics(ctx context.Context) (*Metrics, error) {
|
||||||
|
logger.LogIf(ctx, NotImplemented{})
|
||||||
|
return &Metrics{}, NotImplemented{}
|
||||||
|
}
|
||||||
|
|
||||||
// SetBucketPolicy sets policy on bucket
|
// SetBucketPolicy sets policy on bucket
|
||||||
func (fs *FSObjects) SetBucketPolicy(ctx context.Context, bucket string, policy *policy.Policy) error {
|
func (fs *FSObjects) SetBucketPolicy(ctx context.Context, bucket string, policy *policy.Policy) error {
|
||||||
return savePolicyConfig(ctx, fs, bucket, policy)
|
return savePolicyConfig(ctx, fs, bucket, policy)
|
||||||
|
@ -226,7 +226,6 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
|||||||
globalHTTPServer.Shutdown()
|
globalHTTPServer.Shutdown()
|
||||||
logger.FatalIf(err, "Unable to initialize gateway backend")
|
logger.FatalIf(err, "Unable to initialize gateway backend")
|
||||||
}
|
}
|
||||||
|
|
||||||
newObject = NewGatewayLayerWithLocker(newObject)
|
newObject = NewGatewayLayerWithLocker(newObject)
|
||||||
|
|
||||||
// Re-enable logging
|
// Re-enable logging
|
||||||
|
79
cmd/gateway-metrics.go
Normal file
79
cmd/gateway-metrics.go
Normal file
@ -0,0 +1,79 @@
|
|||||||
|
/*
|
||||||
|
* MinIO Cloud Storage, (C) 2019 MinIO, Inc.
|
||||||
|
*
|
||||||
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
* you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package cmd
|
||||||
|
|
||||||
|
import (
|
||||||
|
"sync"
|
||||||
|
|
||||||
|
"go.uber.org/atomic"
|
||||||
|
)
|
||||||
|
|
||||||
|
// Metrics - represents bytes served from backend
|
||||||
|
// only implemented for S3 Gateway
|
||||||
|
type Metrics struct {
|
||||||
|
BytesReceived atomic.Uint64
|
||||||
|
BytesSent atomic.Uint64
|
||||||
|
RequestStats map[string]int
|
||||||
|
sync.RWMutex
|
||||||
|
}
|
||||||
|
|
||||||
|
// IncBytesReceived - Increase total bytes received from gateway backend
|
||||||
|
func (s *Metrics) IncBytesReceived(n int64) {
|
||||||
|
s.BytesReceived.Add(uint64(n))
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetBytesReceived - Get total bytes received from gateway backend
|
||||||
|
func (s *Metrics) GetBytesReceived() uint64 {
|
||||||
|
return s.BytesReceived.Load()
|
||||||
|
}
|
||||||
|
|
||||||
|
// IncBytesSent - Increase total bytes sent to gateway backend
|
||||||
|
func (s *Metrics) IncBytesSent(n int64) {
|
||||||
|
s.BytesSent.Add(uint64(n))
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetBytesSent - Get total bytes received from gateway backend
|
||||||
|
func (s *Metrics) GetBytesSent() uint64 {
|
||||||
|
return s.BytesSent.Load()
|
||||||
|
}
|
||||||
|
|
||||||
|
// IncRequests - Increase request sent to gateway backend by 1
|
||||||
|
func (s *Metrics) IncRequests(method string) {
|
||||||
|
s.Lock()
|
||||||
|
defer s.Unlock()
|
||||||
|
if s == nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
if s.RequestStats == nil {
|
||||||
|
s.RequestStats = make(map[string]int)
|
||||||
|
}
|
||||||
|
if _, ok := s.RequestStats[method]; ok {
|
||||||
|
s.RequestStats[method]++
|
||||||
|
return
|
||||||
|
}
|
||||||
|
s.RequestStats[method] = 1
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetRequests - Get total number of requests sent to gateway backend
|
||||||
|
func (s *Metrics) GetRequests() map[string]int {
|
||||||
|
return s.RequestStats
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewMetrics - Prepare new Metrics structure
|
||||||
|
func NewMetrics() *Metrics {
|
||||||
|
return &Metrics{}
|
||||||
|
}
|
@ -167,6 +167,12 @@ func (a GatewayUnsupported) CopyObject(ctx context.Context, srcBucket string, sr
|
|||||||
return objInfo, NotImplemented{}
|
return objInfo, NotImplemented{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetMetrics - no op
|
||||||
|
func (a GatewayUnsupported) GetMetrics(ctx context.Context) (*Metrics, error) {
|
||||||
|
logger.LogIf(ctx, NotImplemented{})
|
||||||
|
return &Metrics{}, NotImplemented{}
|
||||||
|
}
|
||||||
|
|
||||||
// IsNotificationSupported returns whether bucket notification is applicable for this layer.
|
// IsNotificationSupported returns whether bucket notification is applicable for this layer.
|
||||||
func (a GatewayUnsupported) IsNotificationSupported() bool {
|
func (a GatewayUnsupported) IsNotificationSupported() bool {
|
||||||
return false
|
return false
|
||||||
|
@ -135,6 +135,8 @@ const (
|
|||||||
letterIdxBits = 6 // 6 bits to represent a letter index
|
letterIdxBits = 6 // 6 bits to represent a letter index
|
||||||
letterIdxMask = 1<<letterIdxBits - 1 // All 1-bits, as many as letterIdxBits
|
letterIdxMask = 1<<letterIdxBits - 1 // All 1-bits, as many as letterIdxBits
|
||||||
letterIdxMax = 63 / letterIdxBits // # of letter indices fitting in 63 bits
|
letterIdxMax = 63 / letterIdxBits // # of letter indices fitting in 63 bits
|
||||||
|
minioReservedBucket = "minio"
|
||||||
|
minioReservedBucketPath = minio.SlashSeparator + minioReservedBucket
|
||||||
)
|
)
|
||||||
|
|
||||||
// randString generates random names and prepends them with a known prefix.
|
// randString generates random names and prepends them with a known prefix.
|
||||||
@ -182,6 +184,32 @@ var defaultAWSCredProviders = []credentials.Provider{
|
|||||||
&credentials.EnvMinio{},
|
&credentials.EnvMinio{},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type metricsTransport struct {
|
||||||
|
transport *http.Transport
|
||||||
|
metrics *minio.Metrics
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s metricsTransport) RoundTrip(r *http.Request) (*http.Response, error) {
|
||||||
|
isS3Request := func() bool {
|
||||||
|
return !(minio.HasPrefix(r.URL.Path, minioReservedBucketPath) ||
|
||||||
|
minio.HasSuffix(r.URL.Path, ".js") || strings.Contains(r.URL.Path, "favicon.ico") ||
|
||||||
|
strings.Contains(r.URL.Path, ".html"))
|
||||||
|
}
|
||||||
|
if isS3Request() && (r.Method == http.MethodGet || r.Method == http.MethodHead) {
|
||||||
|
s.metrics.IncRequests(r.Method)
|
||||||
|
s.metrics.IncBytesSent(r.ContentLength)
|
||||||
|
}
|
||||||
|
// Make the request to the server.
|
||||||
|
resp, err := s.transport.RoundTrip(r)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if isS3Request() && (r.Method == http.MethodGet || r.Method == http.MethodHead) {
|
||||||
|
s.metrics.IncBytesReceived(resp.ContentLength)
|
||||||
|
}
|
||||||
|
return resp, nil
|
||||||
|
}
|
||||||
|
|
||||||
// newS3 - Initializes a new client by auto probing S3 server signature.
|
// newS3 - Initializes a new client by auto probing S3 server signature.
|
||||||
func newS3(urlStr string) (*miniogo.Core, error) {
|
func newS3(urlStr string) (*miniogo.Core, error) {
|
||||||
if urlStr == "" {
|
if urlStr == "" {
|
||||||
@ -221,18 +249,6 @@ func newS3(urlStr string) (*miniogo.Core, error) {
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// Set custom transport
|
|
||||||
clnt.SetCustomTransport(minio.NewCustomHTTPTransport())
|
|
||||||
|
|
||||||
probeBucketName := randString(60, rand.NewSource(time.Now().UnixNano()), "probe-bucket-sign-")
|
|
||||||
|
|
||||||
// Check if the provided keys are valid.
|
|
||||||
if _, err = clnt.BucketExists(probeBucketName); err != nil {
|
|
||||||
if miniogo.ToErrorResponse(err).Code != "AccessDenied" {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return &miniogo.Core{Client: clnt}, nil
|
return &miniogo.Core{Client: clnt}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -245,10 +261,30 @@ func (g *S3) NewGatewayLayer(creds auth.Credentials) (minio.ObjectLayer, error)
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
metrics := minio.NewMetrics()
|
||||||
|
|
||||||
|
t := &metricsTransport{
|
||||||
|
transport: minio.NewCustomHTTPTransport(),
|
||||||
|
metrics: metrics,
|
||||||
|
}
|
||||||
|
|
||||||
|
// Set custom transport
|
||||||
|
clnt.SetCustomTransport(t)
|
||||||
|
|
||||||
|
probeBucketName := randString(60, rand.NewSource(time.Now().UnixNano()), "probe-bucket-sign-")
|
||||||
|
|
||||||
|
// Check if the provided keys are valid.
|
||||||
|
if _, err = clnt.BucketExists(probeBucketName); err != nil {
|
||||||
|
if miniogo.ToErrorResponse(err).Code != "AccessDenied" {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
s := s3Objects{
|
s := s3Objects{
|
||||||
Client: clnt,
|
Client: clnt,
|
||||||
|
Metrics: metrics,
|
||||||
HTTPClient: &http.Client{
|
HTTPClient: &http.Client{
|
||||||
Transport: minio.NewCustomHTTPTransport(),
|
Transport: t,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -275,6 +311,12 @@ type s3Objects struct {
|
|||||||
minio.GatewayUnsupported
|
minio.GatewayUnsupported
|
||||||
Client *miniogo.Core
|
Client *miniogo.Core
|
||||||
HTTPClient *http.Client
|
HTTPClient *http.Client
|
||||||
|
Metrics *minio.Metrics
|
||||||
|
}
|
||||||
|
|
||||||
|
// GetMetrics returns this gateway's metrics
|
||||||
|
func (l *s3Objects) GetMetrics(ctx context.Context) (*minio.Metrics, error) {
|
||||||
|
return l.Metrics, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Shutdown saves any gateway metadata to disk
|
// Shutdown saves any gateway metadata to disk
|
||||||
@ -302,7 +344,6 @@ func (l *s3Objects) MakeBucketWithLocation(ctx context.Context, bucket, location
|
|||||||
if s3utils.CheckValidBucketName(bucket) != nil {
|
if s3utils.CheckValidBucketName(bucket) != nil {
|
||||||
return minio.BucketNameInvalid{Bucket: bucket}
|
return minio.BucketNameInvalid{Bucket: bucket}
|
||||||
}
|
}
|
||||||
|
|
||||||
err := l.Client.MakeBucket(bucket, location)
|
err := l.Client.MakeBucket(bucket, location)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return minio.ErrorRespToObjectError(err, bucket)
|
return minio.ErrorRespToObjectError(err, bucket)
|
||||||
@ -382,7 +423,6 @@ func (l *s3Objects) ListObjects(ctx context.Context, bucket string, prefix strin
|
|||||||
|
|
||||||
// ListObjectsV2 lists all blobs in S3 bucket filtered by prefix
|
// ListObjectsV2 lists all blobs in S3 bucket filtered by prefix
|
||||||
func (l *s3Objects) ListObjectsV2(ctx context.Context, bucket, prefix, continuationToken, delimiter string, maxKeys int, fetchOwner bool, startAfter string) (loi minio.ListObjectsV2Info, e error) {
|
func (l *s3Objects) ListObjectsV2(ctx context.Context, bucket, prefix, continuationToken, delimiter string, maxKeys int, fetchOwner bool, startAfter string) (loi minio.ListObjectsV2Info, e error) {
|
||||||
|
|
||||||
result, err := l.Client.ListObjectsV2(bucket, prefix, continuationToken, fetchOwner, delimiter, maxKeys, startAfter)
|
result, err := l.Client.ListObjectsV2(bucket, prefix, continuationToken, fetchOwner, delimiter, maxKeys, startAfter)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return loi, minio.ErrorRespToObjectError(err, bucket)
|
return loi, minio.ErrorRespToObjectError(err, bucket)
|
||||||
@ -463,6 +503,7 @@ func (l *s3Objects) GetObjectInfo(ctx context.Context, bucket string, object str
|
|||||||
// PutObject creates a new object with the incoming data,
|
// PutObject creates a new object with the incoming data,
|
||||||
func (l *s3Objects) PutObject(ctx context.Context, bucket string, object string, r *minio.PutObjReader, opts minio.ObjectOptions) (objInfo minio.ObjectInfo, err error) {
|
func (l *s3Objects) PutObject(ctx context.Context, bucket string, object string, r *minio.PutObjReader, opts minio.ObjectOptions) (objInfo minio.ObjectInfo, err error) {
|
||||||
data := r.Reader
|
data := r.Reader
|
||||||
|
|
||||||
oi, err := l.Client.PutObject(bucket, object, data, data.Size(), data.MD5Base64String(), data.SHA256HexString(), minio.ToMinioClientMetadata(opts.UserDefined), opts.ServerSideEncryption)
|
oi, err := l.Client.PutObject(bucket, object, data, data.Size(), data.MD5Base64String(), data.SHA256HexString(), minio.ToMinioClientMetadata(opts.UserDefined), opts.ServerSideEncryption)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return objInfo, minio.ErrorRespToObjectError(err, bucket, object)
|
return objInfo, minio.ErrorRespToObjectError(err, bucket, object)
|
||||||
|
@ -102,7 +102,7 @@ func isHTTPHeaderSizeTooLarge(header http.Header) bool {
|
|||||||
length := len(key) + len(header.Get(key))
|
length := len(key) + len(header.Get(key))
|
||||||
size += length
|
size += length
|
||||||
for _, prefix := range userMetadataKeyPrefixes {
|
for _, prefix := range userMetadataKeyPrefixes {
|
||||||
if hasPrefix(key, prefix) {
|
if HasPrefix(key, prefix) {
|
||||||
usersize += length
|
usersize += length
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
@ -141,7 +141,7 @@ func (h reservedMetadataHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
|
|||||||
// and must not set by clients
|
// and must not set by clients
|
||||||
func containsReservedMetadata(header http.Header) bool {
|
func containsReservedMetadata(header http.Header) bool {
|
||||||
for key := range header {
|
for key := range header {
|
||||||
if hasPrefix(key, ReservedMetadataPrefix) {
|
if HasPrefix(key, ReservedMetadataPrefix) {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -259,8 +259,8 @@ func setBrowserCacheControlHandler(h http.Handler) http.Handler {
|
|||||||
func (h cacheControlHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
func (h cacheControlHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||||
if r.Method == http.MethodGet && guessIsBrowserReq(r) {
|
if r.Method == http.MethodGet && guessIsBrowserReq(r) {
|
||||||
// For all browser requests set appropriate Cache-Control policies
|
// For all browser requests set appropriate Cache-Control policies
|
||||||
if hasPrefix(r.URL.Path, minioReservedBucketPath+SlashSeparator) {
|
if HasPrefix(r.URL.Path, minioReservedBucketPath+SlashSeparator) {
|
||||||
if hasSuffix(r.URL.Path, ".js") || r.URL.Path == minioReservedBucketPath+"/favicon.ico" {
|
if HasSuffix(r.URL.Path, ".js") || r.URL.Path == minioReservedBucketPath+"/favicon.ico" {
|
||||||
// For assets set cache expiry of one year. For each release, the name
|
// For assets set cache expiry of one year. For each release, the name
|
||||||
// of the asset name will change and hence it can not be served from cache.
|
// of the asset name will change and hence it can not be served from cache.
|
||||||
w.Header().Set(xhttp.CacheControl, "max-age=31536000")
|
w.Header().Set(xhttp.CacheControl, "max-age=31536000")
|
||||||
|
@ -167,7 +167,6 @@ func (c *minioCollector) Collect(ch chan<- prometheus.Metric) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
connStats := globalConnStats.toServerConnStats()
|
connStats := globalConnStats.toServerConnStats()
|
||||||
httpStats := globalHTTPStats.toServerHTTPStats()
|
|
||||||
|
|
||||||
// Network Sent/Received Bytes (internode)
|
// Network Sent/Received Bytes (internode)
|
||||||
ch <- prometheus.MustNewConstMetric(
|
ch <- prometheus.MustNewConstMetric(
|
||||||
@ -207,6 +206,8 @@ func (c *minioCollector) Collect(ch chan<- prometheus.Metric) {
|
|||||||
float64(connStats.S3InputBytes),
|
float64(connStats.S3InputBytes),
|
||||||
)
|
)
|
||||||
|
|
||||||
|
httpStats := globalHTTPStats.toServerHTTPStats()
|
||||||
|
|
||||||
for api, value := range httpStats.CurrentS3Requests.APIStats {
|
for api, value := range httpStats.CurrentS3Requests.APIStats {
|
||||||
ch <- prometheus.MustNewConstMetric(
|
ch <- prometheus.MustNewConstMetric(
|
||||||
prometheus.NewDesc(
|
prometheus.NewDesc(
|
||||||
@ -243,6 +244,64 @@ func (c *minioCollector) Collect(ch chan<- prometheus.Metric) {
|
|||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Cache related metrics
|
||||||
|
if globalCacheConfig.Enabled {
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("cache", "hits", "total"),
|
||||||
|
"Total number of disk cache hits in current MinIO instance",
|
||||||
|
nil, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(newCachedObjectLayerFn().CacheStats().getHits()),
|
||||||
|
)
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("cache", "misses", "total"),
|
||||||
|
"Total number of disk cache misses in current MinIO instance",
|
||||||
|
nil, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(newCachedObjectLayerFn().CacheStats().getMisses()),
|
||||||
|
)
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("cache", "data", "served"),
|
||||||
|
"Total number of bytes served from cache of current MinIO instance",
|
||||||
|
nil, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(newCachedObjectLayerFn().CacheStats().getBytesServed()),
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
if globalIsGateway && globalGatewayName == "s3" {
|
||||||
|
m, _ := globalObjectAPI.GetMetrics(context.Background())
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("gateway", globalGatewayName, "bytes_received"),
|
||||||
|
"Total number of bytes received by current MinIO S3 Gateway from AWS S3",
|
||||||
|
nil, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(m.GetBytesReceived()),
|
||||||
|
)
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("gateway", globalGatewayName, "bytes_sent"),
|
||||||
|
"Total number of bytes sent by current MinIO S3 Gateway to AWS S3",
|
||||||
|
nil, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(m.GetBytesSent()),
|
||||||
|
)
|
||||||
|
for method, count := range m.GetRequests() {
|
||||||
|
ch <- prometheus.MustNewConstMetric(
|
||||||
|
prometheus.NewDesc(
|
||||||
|
prometheus.BuildFQName("gateway", globalGatewayName, "requests"),
|
||||||
|
"Total number of requests made to AWS S3 by current MinIO S3 Gateway",
|
||||||
|
[]string{"method"}, nil),
|
||||||
|
prometheus.CounterValue,
|
||||||
|
float64(count),
|
||||||
|
method,
|
||||||
|
)
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func metricsHandler() http.Handler {
|
func metricsHandler() http.Handler {
|
||||||
|
@ -59,7 +59,7 @@ func init() {
|
|||||||
// if size == 0 and object ends with SlashSeparator then
|
// if size == 0 and object ends with SlashSeparator then
|
||||||
// returns true.
|
// returns true.
|
||||||
func isObjectDir(object string, size int64) bool {
|
func isObjectDir(object string, size int64) bool {
|
||||||
return hasSuffix(object, SlashSeparator) && size == 0
|
return HasSuffix(object, SlashSeparator) && size == 0
|
||||||
}
|
}
|
||||||
|
|
||||||
// Converts just bucket, object metadata into ObjectInfo datatype.
|
// Converts just bucket, object metadata into ObjectInfo datatype.
|
||||||
@ -114,7 +114,7 @@ func cleanupDir(ctx context.Context, storage StorageAPI, volume, dirPath string)
|
|||||||
var delFunc func(string) error
|
var delFunc func(string) error
|
||||||
// Function to delete entries recursively.
|
// Function to delete entries recursively.
|
||||||
delFunc = func(entryPath string) error {
|
delFunc = func(entryPath string) error {
|
||||||
if !hasSuffix(entryPath, SlashSeparator) {
|
if !HasSuffix(entryPath, SlashSeparator) {
|
||||||
// Delete the file entry.
|
// Delete the file entry.
|
||||||
err := storage.DeleteFile(volume, entryPath)
|
err := storage.DeleteFile(volume, entryPath)
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
@ -161,7 +161,7 @@ func cleanupObjectsBulk(storage StorageAPI, volume string, objsPaths []string, e
|
|||||||
var traverse func(string) ([]string, error)
|
var traverse func(string) ([]string, error)
|
||||||
traverse = func(entryPath string) ([]string, error) {
|
traverse = func(entryPath string) ([]string, error) {
|
||||||
var output = make([]string, 0)
|
var output = make([]string, 0)
|
||||||
if !hasSuffix(entryPath, SlashSeparator) {
|
if !HasSuffix(entryPath, SlashSeparator) {
|
||||||
output = append(output, entryPath)
|
output = append(output, entryPath)
|
||||||
return output, nil
|
return output, nil
|
||||||
}
|
}
|
||||||
@ -335,7 +335,7 @@ func listObjects(ctx context.Context, obj ObjectLayer, bucket, prefix, marker, d
|
|||||||
// Marker is set validate pre-condition.
|
// Marker is set validate pre-condition.
|
||||||
if marker != "" {
|
if marker != "" {
|
||||||
// Marker not common with prefix is not implemented. Send an empty response
|
// Marker not common with prefix is not implemented. Send an empty response
|
||||||
if !hasPrefix(marker, prefix) {
|
if !HasPrefix(marker, prefix) {
|
||||||
return loi, nil
|
return loi, nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -386,7 +386,7 @@ func listObjects(ctx context.Context, obj ObjectLayer, bucket, prefix, marker, d
|
|||||||
|
|
||||||
var objInfo ObjectInfo
|
var objInfo ObjectInfo
|
||||||
var err error
|
var err error
|
||||||
if hasSuffix(walkResult.entry, SlashSeparator) {
|
if HasSuffix(walkResult.entry, SlashSeparator) {
|
||||||
for _, getObjectInfoDir := range getObjectInfoDirs {
|
for _, getObjectInfoDir := range getObjectInfoDirs {
|
||||||
objInfo, err = getObjectInfoDir(ctx, bucket, walkResult.entry)
|
objInfo, err = getObjectInfoDir(ctx, bucket, walkResult.entry)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
|
@ -75,7 +75,7 @@ func checkListObjsArgs(ctx context.Context, bucket, prefix, marker, delimiter st
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Verify if marker has prefix.
|
// Verify if marker has prefix.
|
||||||
if marker != "" && !hasPrefix(marker, prefix) {
|
if marker != "" && !HasPrefix(marker, prefix) {
|
||||||
logger.LogIf(ctx, InvalidMarkerPrefixCombination{
|
logger.LogIf(ctx, InvalidMarkerPrefixCombination{
|
||||||
Marker: marker,
|
Marker: marker,
|
||||||
Prefix: prefix,
|
Prefix: prefix,
|
||||||
@ -94,7 +94,7 @@ func checkListMultipartArgs(ctx context.Context, bucket, prefix, keyMarker, uplo
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
if uploadIDMarker != "" {
|
if uploadIDMarker != "" {
|
||||||
if hasSuffix(keyMarker, SlashSeparator) {
|
if HasSuffix(keyMarker, SlashSeparator) {
|
||||||
|
|
||||||
logger.LogIf(ctx, InvalidUploadIDKeyCombination{
|
logger.LogIf(ctx, InvalidUploadIDKeyCombination{
|
||||||
UploadIDMarker: uploadIDMarker,
|
UploadIDMarker: uploadIDMarker,
|
||||||
@ -188,7 +188,7 @@ func checkPutObjectArgs(ctx context.Context, bucket, object string, obj ObjectLa
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
if len(object) == 0 ||
|
if len(object) == 0 ||
|
||||||
(hasSuffix(object, SlashSeparator) && size != 0) ||
|
(HasSuffix(object, SlashSeparator) && size != 0) ||
|
||||||
!IsValidObjectPrefix(object) {
|
!IsValidObjectPrefix(object) {
|
||||||
return ObjectNameInvalid{
|
return ObjectNameInvalid{
|
||||||
Bucket: bucket,
|
Bucket: bucket,
|
||||||
|
@ -119,4 +119,7 @@ type ObjectLayer interface {
|
|||||||
SetBucketLifecycle(context.Context, string, *lifecycle.Lifecycle) error
|
SetBucketLifecycle(context.Context, string, *lifecycle.Lifecycle) error
|
||||||
GetBucketLifecycle(context.Context, string) (*lifecycle.Lifecycle, error)
|
GetBucketLifecycle(context.Context, string) (*lifecycle.Lifecycle, error)
|
||||||
DeleteBucketLifecycle(context.Context, string) error
|
DeleteBucketLifecycle(context.Context, string) error
|
||||||
|
|
||||||
|
// Backend related metrics
|
||||||
|
GetMetrics(ctx context.Context) (*Metrics, error)
|
||||||
}
|
}
|
||||||
|
@ -145,7 +145,7 @@ func IsValidObjectName(object string) bool {
|
|||||||
if len(object) == 0 {
|
if len(object) == 0 {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
return IsValidObjectPrefix(object)
|
return IsValidObjectPrefix(object)
|
||||||
@ -177,7 +177,7 @@ func checkObjectNameForLengthAndSlash(bucket, object string) error {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Check for slash as prefix in object name
|
// Check for slash as prefix in object name
|
||||||
if hasPrefix(object, SlashSeparator) {
|
if HasPrefix(object, SlashSeparator) {
|
||||||
return ObjectNamePrefixAsSlash{
|
return ObjectNamePrefixAsSlash{
|
||||||
Bucket: bucket,
|
Bucket: bucket,
|
||||||
Object: object,
|
Object: object,
|
||||||
@ -198,7 +198,7 @@ func retainSlash(s string) string {
|
|||||||
func pathJoin(elem ...string) string {
|
func pathJoin(elem ...string) string {
|
||||||
trailingSlash := ""
|
trailingSlash := ""
|
||||||
if len(elem) > 0 {
|
if len(elem) > 0 {
|
||||||
if hasSuffix(elem[len(elem)-1], SlashSeparator) {
|
if HasSuffix(elem[len(elem)-1], SlashSeparator) {
|
||||||
trailingSlash = SlashSeparator
|
trailingSlash = SlashSeparator
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -271,20 +271,20 @@ func extractETag(metadata map[string]string) string {
|
|||||||
return etag
|
return etag
|
||||||
}
|
}
|
||||||
|
|
||||||
// Prefix matcher string matches prefix in a platform specific way.
|
// HasPrefix - Prefix matcher string matches prefix in a platform specific way.
|
||||||
// For example on windows since its case insensitive we are supposed
|
// For example on windows since its case insensitive we are supposed
|
||||||
// to do case insensitive checks.
|
// to do case insensitive checks.
|
||||||
func hasPrefix(s string, prefix string) bool {
|
func HasPrefix(s string, prefix string) bool {
|
||||||
if runtime.GOOS == globalWindowsOSName {
|
if runtime.GOOS == globalWindowsOSName {
|
||||||
return strings.HasPrefix(strings.ToLower(s), strings.ToLower(prefix))
|
return strings.HasPrefix(strings.ToLower(s), strings.ToLower(prefix))
|
||||||
}
|
}
|
||||||
return strings.HasPrefix(s, prefix)
|
return strings.HasPrefix(s, prefix)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Suffix matcher string matches suffix in a platform specific way.
|
// HasSuffix - Suffix matcher string matches suffix in a platform specific way.
|
||||||
// For example on windows since its case insensitive we are supposed
|
// For example on windows since its case insensitive we are supposed
|
||||||
// to do case insensitive checks.
|
// to do case insensitive checks.
|
||||||
func hasSuffix(s string, suffix string) bool {
|
func HasSuffix(s string, suffix string) bool {
|
||||||
if runtime.GOOS == globalWindowsOSName {
|
if runtime.GOOS == globalWindowsOSName {
|
||||||
return strings.HasSuffix(strings.ToLower(s), strings.ToLower(suffix))
|
return strings.HasSuffix(strings.ToLower(s), strings.ToLower(suffix))
|
||||||
}
|
}
|
||||||
|
@ -878,7 +878,7 @@ func (api objectAPIHandlers) CopyObjectHandler(w http.ResponseWriter, r *http.Re
|
|||||||
}
|
}
|
||||||
|
|
||||||
for k, v := range srcInfo.UserDefined {
|
for k, v := range srcInfo.UserDefined {
|
||||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||||
encMetadata[k] = v
|
encMetadata[k] = v
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1254,7 +1254,7 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
|
|||||||
|
|
||||||
var objectEncryptionKey []byte
|
var objectEncryptionKey []byte
|
||||||
if objectAPI.IsEncryptionSupported() {
|
if objectAPI.IsEncryptionSupported() {
|
||||||
if crypto.IsRequested(r.Header) && !hasSuffix(object, SlashSeparator) { // handle SSE requests
|
if crypto.IsRequested(r.Header) && !HasSuffix(object, SlashSeparator) { // handle SSE requests
|
||||||
if crypto.SSECopy.IsRequested(r.Header) {
|
if crypto.SSECopy.IsRequested(r.Header) {
|
||||||
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
||||||
return
|
return
|
||||||
|
@ -604,7 +604,7 @@ func listVols(dirPath string) ([]VolInfo, error) {
|
|||||||
}
|
}
|
||||||
var volsInfo []VolInfo
|
var volsInfo []VolInfo
|
||||||
for _, entry := range entries {
|
for _, entry := range entries {
|
||||||
if !hasSuffix(entry, SlashSeparator) || !isValidVolname(slashpath.Clean(entry)) {
|
if !HasSuffix(entry, SlashSeparator) || !isValidVolname(slashpath.Clean(entry)) {
|
||||||
// Skip if entry is neither a directory not a valid volume name.
|
// Skip if entry is neither a directory not a valid volume name.
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
@ -752,7 +752,7 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
var fi FileInfo
|
var fi FileInfo
|
||||||
if hasSuffix(walkResult.entry, SlashSeparator) {
|
if HasSuffix(walkResult.entry, SlashSeparator) {
|
||||||
fi = FileInfo{
|
fi = FileInfo{
|
||||||
Volume: volume,
|
Volume: volume,
|
||||||
Name: walkResult.entry,
|
Name: walkResult.entry,
|
||||||
@ -1492,8 +1492,8 @@ func (s *posix) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err e
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
srcIsDir := hasSuffix(srcPath, SlashSeparator)
|
srcIsDir := HasSuffix(srcPath, SlashSeparator)
|
||||||
dstIsDir := hasSuffix(dstPath, SlashSeparator)
|
dstIsDir := HasSuffix(dstPath, SlashSeparator)
|
||||||
// Either src and dst have to be directories or files, else return error.
|
// Either src and dst have to be directories or files, else return error.
|
||||||
if !(srcIsDir && dstIsDir || !srcIsDir && !dstIsDir) {
|
if !(srcIsDir && dstIsDir || !srcIsDir && !dstIsDir) {
|
||||||
return errFileAccessDenied
|
return errFileAccessDenied
|
||||||
|
@ -37,7 +37,7 @@ func filterMatchingPrefix(entries []string, prefixEntry string) []string {
|
|||||||
if start == end {
|
if start == end {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
if hasPrefix(entries[start], prefixEntry) {
|
if HasPrefix(entries[start], prefixEntry) {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
start++
|
start++
|
||||||
@ -46,7 +46,7 @@ func filterMatchingPrefix(entries []string, prefixEntry string) []string {
|
|||||||
if start == end {
|
if start == end {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
if hasPrefix(entries[end-1], prefixEntry) {
|
if HasPrefix(entries[end-1], prefixEntry) {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
end--
|
end--
|
||||||
@ -95,7 +95,7 @@ func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker
|
|||||||
|
|
||||||
for i, entry := range entries {
|
for i, entry := range entries {
|
||||||
pentry := pathJoin(prefixDir, entry)
|
pentry := pathJoin(prefixDir, entry)
|
||||||
isDir := hasSuffix(pentry, SlashSeparator)
|
isDir := HasSuffix(pentry, SlashSeparator)
|
||||||
|
|
||||||
if i == 0 && markerDir == entry {
|
if i == 0 && markerDir == entry {
|
||||||
if !recursive {
|
if !recursive {
|
||||||
|
@ -93,7 +93,7 @@ func testTreeWalkPrefix(t *testing.T, listDir ListDirFunc) {
|
|||||||
|
|
||||||
// Check if all entries received on the channel match the prefix.
|
// Check if all entries received on the channel match the prefix.
|
||||||
for res := range twResultCh {
|
for res := range twResultCh {
|
||||||
if !hasPrefix(res.entry, prefix) {
|
if !HasPrefix(res.entry, prefix) {
|
||||||
t.Errorf("Entry %s doesn't match prefix %s", res.entry, prefix)
|
t.Errorf("Entry %s doesn't match prefix %s", res.entry, prefix)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -669,7 +669,7 @@ func (web *webAPIHandlers) RemoveObject(r *http.Request, args *RemoveObjectArgs,
|
|||||||
next:
|
next:
|
||||||
for _, objectName := range args.Objects {
|
for _, objectName := range args.Objects {
|
||||||
// If not a directory, remove the object.
|
// If not a directory, remove the object.
|
||||||
if !hasSuffix(objectName, SlashSeparator) && objectName != "" {
|
if !HasSuffix(objectName, SlashSeparator) && objectName != "" {
|
||||||
// Check for permissions only in the case of
|
// Check for permissions only in the case of
|
||||||
// non-anonymous login. For anonymous login, policy has already
|
// non-anonymous login. For anonymous login, policy has already
|
||||||
// been checked.
|
// been checked.
|
||||||
@ -1043,7 +1043,7 @@ func (web *webAPIHandlers) Upload(w http.ResponseWriter, r *http.Request) {
|
|||||||
return
|
return
|
||||||
}
|
}
|
||||||
if objectAPI.IsEncryptionSupported() {
|
if objectAPI.IsEncryptionSupported() {
|
||||||
if crypto.IsRequested(r.Header) && !hasSuffix(object, SlashSeparator) { // handle SSE requests
|
if crypto.IsRequested(r.Header) && !HasSuffix(object, SlashSeparator) { // handle SSE requests
|
||||||
rawReader := hashReader
|
rawReader := hashReader
|
||||||
var objectEncryptionKey []byte
|
var objectEncryptionKey []byte
|
||||||
reader, objectEncryptionKey, err = EncryptRequest(hashReader, r, bucket, object, metadata)
|
reader, objectEncryptionKey, err = EncryptRequest(hashReader, r, bucket, object, metadata)
|
||||||
@ -1447,7 +1447,7 @@ func (web *webAPIHandlers) DownloadZip(w http.ResponseWriter, r *http.Request) {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
if !hasSuffix(object, SlashSeparator) {
|
if !HasSuffix(object, SlashSeparator) {
|
||||||
// If not a directory, compress the file and write it to response.
|
// If not a directory, compress the file and write it to response.
|
||||||
err := zipit(pathJoin(args.Prefix, object))
|
err := zipit(pathJoin(args.Prefix, object))
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -28,6 +28,7 @@ import (
|
|||||||
|
|
||||||
"github.com/minio/minio/cmd/config/storageclass"
|
"github.com/minio/minio/cmd/config/storageclass"
|
||||||
xhttp "github.com/minio/minio/cmd/http"
|
xhttp "github.com/minio/minio/cmd/http"
|
||||||
|
"github.com/minio/minio/cmd/logger"
|
||||||
"github.com/minio/minio/pkg/bpool"
|
"github.com/minio/minio/pkg/bpool"
|
||||||
"github.com/minio/minio/pkg/dsync"
|
"github.com/minio/minio/pkg/dsync"
|
||||||
"github.com/minio/minio/pkg/lifecycle"
|
"github.com/minio/minio/pkg/lifecycle"
|
||||||
@ -1039,7 +1040,7 @@ func (s *xlSets) listObjects(ctx context.Context, bucket, prefix, marker, delimi
|
|||||||
// Marker is set validate pre-condition.
|
// Marker is set validate pre-condition.
|
||||||
if marker != "" {
|
if marker != "" {
|
||||||
// Marker not common with prefix is not implemented. Send an empty response
|
// Marker not common with prefix is not implemented. Send an empty response
|
||||||
if !hasPrefix(marker, prefix) {
|
if !HasPrefix(marker, prefix) {
|
||||||
return loi, nil
|
return loi, nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1092,7 +1093,7 @@ func (s *xlSets) listObjects(ctx context.Context, bucket, prefix, marker, delimi
|
|||||||
|
|
||||||
for _, entry := range entries.Files {
|
for _, entry := range entries.Files {
|
||||||
var objInfo ObjectInfo
|
var objInfo ObjectInfo
|
||||||
if hasSuffix(entry.Name, SlashSeparator) {
|
if HasSuffix(entry.Name, SlashSeparator) {
|
||||||
if !recursive {
|
if !recursive {
|
||||||
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
||||||
continue
|
continue
|
||||||
@ -1655,3 +1656,9 @@ func (s *xlSets) HealObjects(ctx context.Context, bucket, prefix string, healObj
|
|||||||
func (s *xlSets) ListObjectsHeal(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, err error) {
|
func (s *xlSets) ListObjectsHeal(ctx context.Context, bucket, prefix, marker, delimiter string, maxKeys int) (loi ListObjectsInfo, err error) {
|
||||||
return s.listObjects(ctx, bucket, prefix, marker, delimiter, maxKeys, true)
|
return s.listObjects(ctx, bucket, prefix, marker, delimiter, maxKeys, true)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetMetrics - no op
|
||||||
|
func (s *xlSets) GetMetrics(ctx context.Context) (*Metrics, error) {
|
||||||
|
logger.LogIf(ctx, NotImplemented{})
|
||||||
|
return &Metrics{}, NotImplemented{}
|
||||||
|
}
|
||||||
|
@ -680,7 +680,7 @@ func (xl xlObjects) HealObject(ctx context.Context, bucket, object string, dryRu
|
|||||||
healCtx := logger.SetReqInfo(context.Background(), newReqInfo)
|
healCtx := logger.SetReqInfo(context.Background(), newReqInfo)
|
||||||
|
|
||||||
// Healing directories handle it separately.
|
// Healing directories handle it separately.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
return xl.healObjectDir(healCtx, bucket, object, dryRun)
|
return xl.healObjectDir(healCtx, bucket, object, dryRun)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -87,7 +87,7 @@ func (xl xlObjects) listObjects(ctx context.Context, bucket, prefix, marker, del
|
|||||||
}
|
}
|
||||||
entry := walkResult.entry
|
entry := walkResult.entry
|
||||||
var objInfo ObjectInfo
|
var objInfo ObjectInfo
|
||||||
if hasSuffix(entry, SlashSeparator) {
|
if HasSuffix(entry, SlashSeparator) {
|
||||||
// Object name needs to be full path.
|
// Object name needs to be full path.
|
||||||
objInfo.Bucket = bucket
|
objInfo.Bucket = bucket
|
||||||
objInfo.Name = entry
|
objInfo.Name = entry
|
||||||
@ -156,7 +156,7 @@ func (xl xlObjects) ListObjects(ctx context.Context, bucket, prefix, marker, del
|
|||||||
// Marker is set validate pre-condition.
|
// Marker is set validate pre-condition.
|
||||||
if marker != "" {
|
if marker != "" {
|
||||||
// Marker not common with prefix is not implemented.Send an empty response
|
// Marker not common with prefix is not implemented.Send an empty response
|
||||||
if !hasPrefix(marker, prefix) {
|
if !HasPrefix(marker, prefix) {
|
||||||
return ListObjectsInfo{}, e
|
return ListObjectsInfo{}, e
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -129,7 +129,7 @@ func (xl xlObjects) GetObjectNInfo(ctx context.Context, bucket, object string, r
|
|||||||
|
|
||||||
// Handler directory request by returning a reader that
|
// Handler directory request by returning a reader that
|
||||||
// returns no bytes.
|
// returns no bytes.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
var objInfo ObjectInfo
|
var objInfo ObjectInfo
|
||||||
if objInfo, err = xl.getObjectInfoDir(ctx, bucket, object); err != nil {
|
if objInfo, err = xl.getObjectInfoDir(ctx, bucket, object); err != nil {
|
||||||
return nil, toObjectErr(err, bucket, object)
|
return nil, toObjectErr(err, bucket, object)
|
||||||
@ -190,7 +190,7 @@ func (xl xlObjects) getObject(ctx context.Context, bucket, object string, startO
|
|||||||
}
|
}
|
||||||
|
|
||||||
// If its a directory request, we return an empty body.
|
// If its a directory request, we return an empty body.
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
_, err := writer.Write([]byte(""))
|
_, err := writer.Write([]byte(""))
|
||||||
logger.LogIf(ctx, err)
|
logger.LogIf(ctx, err)
|
||||||
return toObjectErr(err, bucket, object)
|
return toObjectErr(err, bucket, object)
|
||||||
@ -344,7 +344,7 @@ func (xl xlObjects) GetObjectInfo(ctx context.Context, bucket, object string, op
|
|||||||
return oi, err
|
return oi, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if hasSuffix(object, SlashSeparator) {
|
if HasSuffix(object, SlashSeparator) {
|
||||||
info, err := xl.getObjectInfoDir(ctx, bucket, object)
|
info, err := xl.getObjectInfoDir(ctx, bucket, object)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return oi, toObjectErr(err, bucket, object)
|
return oi, toObjectErr(err, bucket, object)
|
||||||
@ -804,7 +804,7 @@ func (xl xlObjects) deleteObjects(ctx context.Context, bucket string, objects []
|
|||||||
}
|
}
|
||||||
|
|
||||||
for i, object := range objects {
|
for i, object := range objects {
|
||||||
isObjectDirs[i] = hasSuffix(object, SlashSeparator)
|
isObjectDirs[i] = HasSuffix(object, SlashSeparator)
|
||||||
}
|
}
|
||||||
|
|
||||||
for i, object := range objects {
|
for i, object := range objects {
|
||||||
@ -903,7 +903,7 @@ func (xl xlObjects) DeleteObject(ctx context.Context, bucket, object string) (er
|
|||||||
}
|
}
|
||||||
|
|
||||||
var writeQuorum int
|
var writeQuorum int
|
||||||
var isObjectDir = hasSuffix(object, SlashSeparator)
|
var isObjectDir = HasSuffix(object, SlashSeparator)
|
||||||
|
|
||||||
if isObjectDir {
|
if isObjectDir {
|
||||||
_, err = xl.getObjectInfoDir(ctx, bucket, object)
|
_, err = xl.getObjectInfoDir(ctx, bucket, object)
|
||||||
|
@ -200,3 +200,9 @@ func getStorageInfo(disks []StorageAPI) StorageInfo {
|
|||||||
func (xl xlObjects) StorageInfo(ctx context.Context) StorageInfo {
|
func (xl xlObjects) StorageInfo(ctx context.Context) StorageInfo {
|
||||||
return getStorageInfo(xl.getDisks())
|
return getStorageInfo(xl.getDisks())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetMetrics - no op
|
||||||
|
func (xl xlObjects) GetMetrics(ctx context.Context) (*Metrics, error) {
|
||||||
|
logger.LogIf(ctx, NotImplemented{})
|
||||||
|
return &Metrics{}, NotImplemented{}
|
||||||
|
}
|
||||||
|
@ -617,7 +617,7 @@ func (z *xlZones) listObjects(ctx context.Context, bucket, prefix, marker, delim
|
|||||||
// Marker is set validate pre-condition.
|
// Marker is set validate pre-condition.
|
||||||
if marker != "" {
|
if marker != "" {
|
||||||
// Marker not common with prefix is not implemented. Send an empty response
|
// Marker not common with prefix is not implemented. Send an empty response
|
||||||
if !hasPrefix(marker, prefix) {
|
if !HasPrefix(marker, prefix) {
|
||||||
return loi, nil
|
return loi, nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -682,7 +682,7 @@ func (z *xlZones) listObjects(ctx context.Context, bucket, prefix, marker, delim
|
|||||||
|
|
||||||
for _, entry := range entries.Files {
|
for _, entry := range entries.Files {
|
||||||
var objInfo ObjectInfo
|
var objInfo ObjectInfo
|
||||||
if hasSuffix(entry.Name, SlashSeparator) {
|
if HasSuffix(entry.Name, SlashSeparator) {
|
||||||
if !recursive {
|
if !recursive {
|
||||||
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
||||||
continue
|
continue
|
||||||
@ -1312,3 +1312,9 @@ func (z *xlZones) ListBucketsHeal(ctx context.Context) ([]BucketInfo, error) {
|
|||||||
}
|
}
|
||||||
return healBuckets, nil
|
return healBuckets, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetMetrics - no op
|
||||||
|
func (z *xlZones) GetMetrics(ctx context.Context) (*Metrics, error) {
|
||||||
|
logger.LogIf(ctx, NotImplemented{})
|
||||||
|
return &Metrics{}, NotImplemented{}
|
||||||
|
}
|
||||||
|
@ -118,7 +118,6 @@ The list of metrics and its definition are as follows. (NOTE: instance here is o
|
|||||||
> 1. Instance here is one MinIO node.
|
> 1. Instance here is one MinIO node.
|
||||||
> 2. `s3 requests` exclude internode requests.
|
> 2. `s3 requests` exclude internode requests.
|
||||||
|
|
||||||
|
|
||||||
- standard go runtime metrics prefixed by `go_`
|
- standard go runtime metrics prefixed by `go_`
|
||||||
- process level metrics prefixed with `process_`
|
- process level metrics prefixed with `process_`
|
||||||
- prometheus scrap metrics prefixed with `promhttp_`
|
- prometheus scrap metrics prefixed with `promhttp_`
|
||||||
@ -138,6 +137,23 @@ The list of metrics and its definition are as follows. (NOTE: instance here is o
|
|||||||
- `minio_version_info`: Current MinIO version with commit-id.
|
- `minio_version_info`: Current MinIO version with commit-id.
|
||||||
- `s3_ttfb_seconds`: Histogram that holds the latency information of the requests.
|
- `s3_ttfb_seconds`: Histogram that holds the latency information of the requests.
|
||||||
|
|
||||||
|
Apart from above metrics, MinIO also exposes below mode specific metrics
|
||||||
|
|
||||||
|
### Cache specific metrics
|
||||||
|
|
||||||
|
MinIO Gateway instances enabled with Disk-Caching expose caching related metrics.
|
||||||
|
|
||||||
|
- `cache_data_served`: Total number of bytes served from cache.
|
||||||
|
- `cache_hits_total`: Total number of cache hits.
|
||||||
|
- `cache_misses_total`: Total number of cache misses.
|
||||||
|
|
||||||
|
### S3 Gateway & Cache specific metrics
|
||||||
|
|
||||||
|
MinIO S3 Gateway instance exposes metrics related to Gateway communication with AWS S3.
|
||||||
|
|
||||||
|
- `gateway_s3_requests`: Total number of GET & HEAD requests made to AWS S3. This metrics has a label `method` that identifies GET & HEAD Requests.
|
||||||
|
- `gateway_s3_bytes_sent`: Total number of bytes sent to AWS S3 (in GET & HEAD Requests).
|
||||||
|
- `gateway_s3_bytes_received`: Total number of bytes received from AWS S3 (in GET & HEAD Requests).
|
||||||
|
|
||||||
## Migration guide for the new set of metrics
|
## Migration guide for the new set of metrics
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user