mirror of
https://github.com/minio/minio.git
synced 2025-01-11 15:03:22 -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 {
|
||||
return false
|
||||
}
|
||||
trace := trcAll || !hasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath+SlashSeparator)
|
||||
trace := trcAll || !HasPrefix(trcInfo.ReqInfo.Path, minioReservedBucketPath+SlashSeparator)
|
||||
if errOnly {
|
||||
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.
|
||||
for k, v := range objInfo.UserDefined {
|
||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
||||
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||
// Do not need to send any internal metadata
|
||||
// values to client.
|
||||
continue
|
||||
|
@ -526,7 +526,7 @@ func generateListObjectsV2Response(bucket, prefix, token, nextToken, startAfter,
|
||||
if metadata {
|
||||
content.UserMetadata = make(StringMap)
|
||||
for k, v := range CleanMinioInternalMetadataKeys(object.UserDefined) {
|
||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
||||
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||
// Do not need to send any internal metadata
|
||||
// values to client.
|
||||
continue
|
||||
|
@ -214,7 +214,7 @@ func (api objectAPIHandlers) ListMultipartUploadsHandler(w http.ResponseWriter,
|
||||
|
||||
if keyMarker != "" {
|
||||
// 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))
|
||||
return
|
||||
}
|
||||
@ -750,7 +750,7 @@ func (api objectAPIHandlers) PostPolicyBucketHandler(w http.ResponseWriter, r *h
|
||||
return
|
||||
}
|
||||
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) {
|
||||
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
||||
return
|
||||
|
@ -334,14 +334,14 @@ func (c *diskCache) updateMetadataIfChanged(ctx context.Context, bucket, object
|
||||
bkMeta := make(map[string]string)
|
||||
cacheMeta := make(map[string]string)
|
||||
for k, v := range bkObjectInfo.UserDefined {
|
||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
||||
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||
// Do not need to send any internal metadata
|
||||
continue
|
||||
}
|
||||
bkMeta[http.CanonicalHeaderKey(k)] = v
|
||||
}
|
||||
for k, v := range cacheObjInfo.UserDefined {
|
||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
||||
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||
// Do not need to send any internal metadata
|
||||
continue
|
||||
}
|
||||
@ -602,7 +602,7 @@ func (c *diskCache) Get(ctx context.Context, bucket, object string, rs *HTTPRang
|
||||
|
||||
var nsUnlocker = func() {}
|
||||
// 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
|
||||
// objReader.Close() is called by the caller.
|
||||
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)
|
||||
// Storage operations.
|
||||
StorageInfo(ctx context.Context) CacheStorageInfo
|
||||
CacheStats() *CacheStats
|
||||
}
|
||||
|
||||
// Abstracts disk caching - used by the S3 layer
|
||||
@ -74,6 +75,9 @@ type cacheObjects struct {
|
||||
// nsMutex namespace lock
|
||||
nsMutex *nsLockMap
|
||||
|
||||
// Cache stats
|
||||
cacheStats *CacheStats
|
||||
|
||||
// Object functions pointing to the corresponding functions of backend implementation.
|
||||
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)
|
||||
@ -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)
|
||||
if cacheErr == nil {
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
// Reaching here implies cache miss
|
||||
c.cacheStats.incMiss()
|
||||
|
||||
objInfo, err := c.GetObjectInfoFn(ctx, bucket, object, opts)
|
||||
if backendDownError(err) && cacheErr == 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)
|
||||
if cerr == nil {
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
// Reaching here implies cache miss
|
||||
c.cacheStats.incMiss()
|
||||
|
||||
objInfo, err := getObjectInfoFn(ctx, bucket, object, opts)
|
||||
if err != nil {
|
||||
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
|
||||
func (c *cacheObjects) skipCache() bool {
|
||||
c.migMutex.Lock()
|
||||
@ -572,11 +593,12 @@ func newServerCacheObjects(ctx context.Context, config cache.Config) (CacheObjec
|
||||
}
|
||||
|
||||
c := &cacheObjects{
|
||||
cache: cache,
|
||||
exclude: config.Exclude,
|
||||
migrating: migrateSw,
|
||||
migMutex: sync.Mutex{},
|
||||
nsMutex: newNSLock(false),
|
||||
cache: cache,
|
||||
exclude: config.Exclude,
|
||||
migrating: migrateSw,
|
||||
migMutex: sync.Mutex{},
|
||||
nsMutex: newNSLock(false),
|
||||
cacheStats: newCacheStats(),
|
||||
GetObjectInfoFn: func(ctx context.Context, bucket, object string, opts ObjectOptions) (ObjectInfo, error) {
|
||||
return newObjectLayerFn().GetObjectInfo(ctx, bucket, object, opts)
|
||||
},
|
||||
|
@ -34,7 +34,7 @@ func walk(ctx context.Context, path string, usageFn usageFunc) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if !hasSuffix(path, SlashSeparator) {
|
||||
if !HasSuffix(path, SlashSeparator) {
|
||||
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))
|
||||
}
|
||||
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
m.Meta["etag"] = emptyETag // For directories etag is d41d8cd98f00b204e9800998ecf8427e
|
||||
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)
|
||||
}
|
||||
// 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
|
||||
// objReader.Close() is called by the caller.
|
||||
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 hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
_, err = writer.Write([]byte(""))
|
||||
logger.LogIf(ctx, err)
|
||||
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.
|
||||
func (fs *FSObjects) getObjectInfo(ctx context.Context, bucket, object string) (oi ObjectInfo, e error) {
|
||||
fsMeta := fsMetaV1{}
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
fi, err := fsStatDir(ctx, pathJoin(fs.fsPath, bucket, object))
|
||||
if err != nil {
|
||||
return oi, err
|
||||
@ -1167,6 +1167,12 @@ func (fs *FSObjects) ListObjectsHeal(ctx context.Context, bucket, prefix, marker
|
||||
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
|
||||
func (fs *FSObjects) SetBucketPolicy(ctx context.Context, bucket string, policy *policy.Policy) error {
|
||||
return savePolicyConfig(ctx, fs, bucket, policy)
|
||||
|
@ -226,7 +226,6 @@ func StartGateway(ctx *cli.Context, gw Gateway) {
|
||||
globalHTTPServer.Shutdown()
|
||||
logger.FatalIf(err, "Unable to initialize gateway backend")
|
||||
}
|
||||
|
||||
newObject = NewGatewayLayerWithLocker(newObject)
|
||||
|
||||
// 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{}
|
||||
}
|
||||
|
||||
// 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.
|
||||
func (a GatewayUnsupported) IsNotificationSupported() bool {
|
||||
return false
|
||||
|
@ -132,9 +132,11 @@ func (g *S3) Name() string {
|
||||
|
||||
const letterBytes = "abcdefghijklmnopqrstuvwxyz01234569"
|
||||
const (
|
||||
letterIdxBits = 6 // 6 bits to represent a letter index
|
||||
letterIdxMask = 1<<letterIdxBits - 1 // All 1-bits, as many as letterIdxBits
|
||||
letterIdxMax = 63 / letterIdxBits // # of letter indices fitting in 63 bits
|
||||
letterIdxBits = 6 // 6 bits to represent a letter index
|
||||
letterIdxMask = 1<<letterIdxBits - 1 // All 1-bits, as many as letterIdxBits
|
||||
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.
|
||||
@ -182,6 +184,32 @@ var defaultAWSCredProviders = []credentials.Provider{
|
||||
&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.
|
||||
func newS3(urlStr string) (*miniogo.Core, error) {
|
||||
if urlStr == "" {
|
||||
@ -221,18 +249,6 @@ func newS3(urlStr string) (*miniogo.Core, error) {
|
||||
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
|
||||
}
|
||||
|
||||
@ -245,10 +261,30 @@ func (g *S3) NewGatewayLayer(creds auth.Credentials) (minio.ObjectLayer, error)
|
||||
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{
|
||||
Client: clnt,
|
||||
Client: clnt,
|
||||
Metrics: metrics,
|
||||
HTTPClient: &http.Client{
|
||||
Transport: minio.NewCustomHTTPTransport(),
|
||||
Transport: t,
|
||||
},
|
||||
}
|
||||
|
||||
@ -275,6 +311,12 @@ type s3Objects struct {
|
||||
minio.GatewayUnsupported
|
||||
Client *miniogo.Core
|
||||
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
|
||||
@ -302,7 +344,6 @@ func (l *s3Objects) MakeBucketWithLocation(ctx context.Context, bucket, location
|
||||
if s3utils.CheckValidBucketName(bucket) != nil {
|
||||
return minio.BucketNameInvalid{Bucket: bucket}
|
||||
}
|
||||
|
||||
err := l.Client.MakeBucket(bucket, location)
|
||||
if err != nil {
|
||||
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
|
||||
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)
|
||||
if err != nil {
|
||||
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,
|
||||
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
|
||||
|
||||
oi, err := l.Client.PutObject(bucket, object, data, data.Size(), data.MD5Base64String(), data.SHA256HexString(), minio.ToMinioClientMetadata(opts.UserDefined), opts.ServerSideEncryption)
|
||||
if err != nil {
|
||||
return objInfo, minio.ErrorRespToObjectError(err, bucket, object)
|
||||
|
@ -102,7 +102,7 @@ func isHTTPHeaderSizeTooLarge(header http.Header) bool {
|
||||
length := len(key) + len(header.Get(key))
|
||||
size += length
|
||||
for _, prefix := range userMetadataKeyPrefixes {
|
||||
if hasPrefix(key, prefix) {
|
||||
if HasPrefix(key, prefix) {
|
||||
usersize += length
|
||||
break
|
||||
}
|
||||
@ -141,7 +141,7 @@ func (h reservedMetadataHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques
|
||||
// and must not set by clients
|
||||
func containsReservedMetadata(header http.Header) bool {
|
||||
for key := range header {
|
||||
if hasPrefix(key, ReservedMetadataPrefix) {
|
||||
if HasPrefix(key, ReservedMetadataPrefix) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
@ -259,8 +259,8 @@ func setBrowserCacheControlHandler(h http.Handler) http.Handler {
|
||||
func (h cacheControlHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
|
||||
if r.Method == http.MethodGet && guessIsBrowserReq(r) {
|
||||
// For all browser requests set appropriate Cache-Control policies
|
||||
if hasPrefix(r.URL.Path, minioReservedBucketPath+SlashSeparator) {
|
||||
if hasSuffix(r.URL.Path, ".js") || r.URL.Path == minioReservedBucketPath+"/favicon.ico" {
|
||||
if HasPrefix(r.URL.Path, minioReservedBucketPath+SlashSeparator) {
|
||||
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
|
||||
// of the asset name will change and hence it can not be served from cache.
|
||||
w.Header().Set(xhttp.CacheControl, "max-age=31536000")
|
||||
|
@ -167,7 +167,6 @@ func (c *minioCollector) Collect(ch chan<- prometheus.Metric) {
|
||||
}
|
||||
|
||||
connStats := globalConnStats.toServerConnStats()
|
||||
httpStats := globalHTTPStats.toServerHTTPStats()
|
||||
|
||||
// Network Sent/Received Bytes (internode)
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
@ -207,6 +206,8 @@ func (c *minioCollector) Collect(ch chan<- prometheus.Metric) {
|
||||
float64(connStats.S3InputBytes),
|
||||
)
|
||||
|
||||
httpStats := globalHTTPStats.toServerHTTPStats()
|
||||
|
||||
for api, value := range httpStats.CurrentS3Requests.APIStats {
|
||||
ch <- prometheus.MustNewConstMetric(
|
||||
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 {
|
||||
|
@ -59,7 +59,7 @@ func init() {
|
||||
// if size == 0 and object ends with SlashSeparator then
|
||||
// returns true.
|
||||
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.
|
||||
@ -114,7 +114,7 @@ func cleanupDir(ctx context.Context, storage StorageAPI, volume, dirPath string)
|
||||
var delFunc func(string) error
|
||||
// Function to delete entries recursively.
|
||||
delFunc = func(entryPath string) error {
|
||||
if !hasSuffix(entryPath, SlashSeparator) {
|
||||
if !HasSuffix(entryPath, SlashSeparator) {
|
||||
// Delete the file entry.
|
||||
err := storage.DeleteFile(volume, entryPath)
|
||||
logger.LogIf(ctx, err)
|
||||
@ -161,7 +161,7 @@ func cleanupObjectsBulk(storage StorageAPI, volume string, objsPaths []string, e
|
||||
var traverse func(string) ([]string, error)
|
||||
traverse = func(entryPath string) ([]string, error) {
|
||||
var output = make([]string, 0)
|
||||
if !hasSuffix(entryPath, SlashSeparator) {
|
||||
if !HasSuffix(entryPath, SlashSeparator) {
|
||||
output = append(output, entryPath)
|
||||
return output, nil
|
||||
}
|
||||
@ -335,7 +335,7 @@ func listObjects(ctx context.Context, obj ObjectLayer, bucket, prefix, marker, d
|
||||
// Marker is set validate pre-condition.
|
||||
if marker != "" {
|
||||
// Marker not common with prefix is not implemented. Send an empty response
|
||||
if !hasPrefix(marker, prefix) {
|
||||
if !HasPrefix(marker, prefix) {
|
||||
return loi, nil
|
||||
}
|
||||
}
|
||||
@ -386,7 +386,7 @@ func listObjects(ctx context.Context, obj ObjectLayer, bucket, prefix, marker, d
|
||||
|
||||
var objInfo ObjectInfo
|
||||
var err error
|
||||
if hasSuffix(walkResult.entry, SlashSeparator) {
|
||||
if HasSuffix(walkResult.entry, SlashSeparator) {
|
||||
for _, getObjectInfoDir := range getObjectInfoDirs {
|
||||
objInfo, err = getObjectInfoDir(ctx, bucket, walkResult.entry)
|
||||
if err == nil {
|
||||
|
@ -75,7 +75,7 @@ func checkListObjsArgs(ctx context.Context, bucket, prefix, marker, delimiter st
|
||||
}
|
||||
}
|
||||
// Verify if marker has prefix.
|
||||
if marker != "" && !hasPrefix(marker, prefix) {
|
||||
if marker != "" && !HasPrefix(marker, prefix) {
|
||||
logger.LogIf(ctx, InvalidMarkerPrefixCombination{
|
||||
Marker: marker,
|
||||
Prefix: prefix,
|
||||
@ -94,7 +94,7 @@ func checkListMultipartArgs(ctx context.Context, bucket, prefix, keyMarker, uplo
|
||||
return err
|
||||
}
|
||||
if uploadIDMarker != "" {
|
||||
if hasSuffix(keyMarker, SlashSeparator) {
|
||||
if HasSuffix(keyMarker, SlashSeparator) {
|
||||
|
||||
logger.LogIf(ctx, InvalidUploadIDKeyCombination{
|
||||
UploadIDMarker: uploadIDMarker,
|
||||
@ -188,7 +188,7 @@ func checkPutObjectArgs(ctx context.Context, bucket, object string, obj ObjectLa
|
||||
return err
|
||||
}
|
||||
if len(object) == 0 ||
|
||||
(hasSuffix(object, SlashSeparator) && size != 0) ||
|
||||
(HasSuffix(object, SlashSeparator) && size != 0) ||
|
||||
!IsValidObjectPrefix(object) {
|
||||
return ObjectNameInvalid{
|
||||
Bucket: bucket,
|
||||
|
@ -119,4 +119,7 @@ type ObjectLayer interface {
|
||||
SetBucketLifecycle(context.Context, string, *lifecycle.Lifecycle) error
|
||||
GetBucketLifecycle(context.Context, string) (*lifecycle.Lifecycle, 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 {
|
||||
return false
|
||||
}
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
return false
|
||||
}
|
||||
return IsValidObjectPrefix(object)
|
||||
@ -177,7 +177,7 @@ func checkObjectNameForLengthAndSlash(bucket, object string) error {
|
||||
}
|
||||
}
|
||||
// Check for slash as prefix in object name
|
||||
if hasPrefix(object, SlashSeparator) {
|
||||
if HasPrefix(object, SlashSeparator) {
|
||||
return ObjectNamePrefixAsSlash{
|
||||
Bucket: bucket,
|
||||
Object: object,
|
||||
@ -198,7 +198,7 @@ func retainSlash(s string) string {
|
||||
func pathJoin(elem ...string) string {
|
||||
trailingSlash := ""
|
||||
if len(elem) > 0 {
|
||||
if hasSuffix(elem[len(elem)-1], SlashSeparator) {
|
||||
if HasSuffix(elem[len(elem)-1], SlashSeparator) {
|
||||
trailingSlash = SlashSeparator
|
||||
}
|
||||
}
|
||||
@ -271,20 +271,20 @@ func extractETag(metadata map[string]string) string {
|
||||
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
|
||||
// to do case insensitive checks.
|
||||
func hasPrefix(s string, prefix string) bool {
|
||||
func HasPrefix(s string, prefix string) bool {
|
||||
if runtime.GOOS == globalWindowsOSName {
|
||||
return strings.HasPrefix(strings.ToLower(s), strings.ToLower(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
|
||||
// to do case insensitive checks.
|
||||
func hasSuffix(s string, suffix string) bool {
|
||||
func HasSuffix(s string, suffix string) bool {
|
||||
if runtime.GOOS == globalWindowsOSName {
|
||||
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 {
|
||||
if hasPrefix(k, ReservedMetadataPrefix) {
|
||||
if HasPrefix(k, ReservedMetadataPrefix) {
|
||||
encMetadata[k] = v
|
||||
}
|
||||
}
|
||||
@ -1254,7 +1254,7 @@ func (api objectAPIHandlers) PutObjectHandler(w http.ResponseWriter, r *http.Req
|
||||
|
||||
var objectEncryptionKey []byte
|
||||
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) {
|
||||
writeErrorResponse(ctx, w, toAPIError(ctx, errInvalidEncryptionParameters), r.URL, guessIsBrowserReq(r))
|
||||
return
|
||||
|
@ -604,7 +604,7 @@ func listVols(dirPath string) ([]VolInfo, error) {
|
||||
}
|
||||
var volsInfo []VolInfo
|
||||
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.
|
||||
continue
|
||||
}
|
||||
@ -752,7 +752,7 @@ func (s *posix) Walk(volume, dirPath, marker string, recursive bool, leafFile st
|
||||
return
|
||||
}
|
||||
var fi FileInfo
|
||||
if hasSuffix(walkResult.entry, SlashSeparator) {
|
||||
if HasSuffix(walkResult.entry, SlashSeparator) {
|
||||
fi = FileInfo{
|
||||
Volume: volume,
|
||||
Name: walkResult.entry,
|
||||
@ -1492,8 +1492,8 @@ func (s *posix) RenameFile(srcVolume, srcPath, dstVolume, dstPath string) (err e
|
||||
}
|
||||
}
|
||||
|
||||
srcIsDir := hasSuffix(srcPath, SlashSeparator)
|
||||
dstIsDir := hasSuffix(dstPath, SlashSeparator)
|
||||
srcIsDir := HasSuffix(srcPath, SlashSeparator)
|
||||
dstIsDir := HasSuffix(dstPath, SlashSeparator)
|
||||
// Either src and dst have to be directories or files, else return error.
|
||||
if !(srcIsDir && dstIsDir || !srcIsDir && !dstIsDir) {
|
||||
return errFileAccessDenied
|
||||
|
@ -37,7 +37,7 @@ func filterMatchingPrefix(entries []string, prefixEntry string) []string {
|
||||
if start == end {
|
||||
break
|
||||
}
|
||||
if hasPrefix(entries[start], prefixEntry) {
|
||||
if HasPrefix(entries[start], prefixEntry) {
|
||||
break
|
||||
}
|
||||
start++
|
||||
@ -46,7 +46,7 @@ func filterMatchingPrefix(entries []string, prefixEntry string) []string {
|
||||
if start == end {
|
||||
break
|
||||
}
|
||||
if hasPrefix(entries[end-1], prefixEntry) {
|
||||
if HasPrefix(entries[end-1], prefixEntry) {
|
||||
break
|
||||
}
|
||||
end--
|
||||
@ -95,7 +95,7 @@ func doTreeWalk(ctx context.Context, bucket, prefixDir, entryPrefixMatch, marker
|
||||
|
||||
for i, entry := range entries {
|
||||
pentry := pathJoin(prefixDir, entry)
|
||||
isDir := hasSuffix(pentry, SlashSeparator)
|
||||
isDir := HasSuffix(pentry, SlashSeparator)
|
||||
|
||||
if i == 0 && markerDir == entry {
|
||||
if !recursive {
|
||||
|
@ -93,7 +93,7 @@ func testTreeWalkPrefix(t *testing.T, listDir ListDirFunc) {
|
||||
|
||||
// Check if all entries received on the channel match the prefix.
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@ -669,7 +669,7 @@ func (web *webAPIHandlers) RemoveObject(r *http.Request, args *RemoveObjectArgs,
|
||||
next:
|
||||
for _, objectName := range args.Objects {
|
||||
// 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
|
||||
// non-anonymous login. For anonymous login, policy has already
|
||||
// been checked.
|
||||
@ -1043,7 +1043,7 @@ func (web *webAPIHandlers) Upload(w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
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
|
||||
var objectEncryptionKey []byte
|
||||
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
|
||||
}
|
||||
|
||||
if !hasSuffix(object, SlashSeparator) {
|
||||
if !HasSuffix(object, SlashSeparator) {
|
||||
// If not a directory, compress the file and write it to response.
|
||||
err := zipit(pathJoin(args.Prefix, object))
|
||||
if err != nil {
|
||||
|
@ -28,6 +28,7 @@ import (
|
||||
|
||||
"github.com/minio/minio/cmd/config/storageclass"
|
||||
xhttp "github.com/minio/minio/cmd/http"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/bpool"
|
||||
"github.com/minio/minio/pkg/dsync"
|
||||
"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.
|
||||
if marker != "" {
|
||||
// Marker not common with prefix is not implemented. Send an empty response
|
||||
if !hasPrefix(marker, prefix) {
|
||||
if !HasPrefix(marker, prefix) {
|
||||
return loi, nil
|
||||
}
|
||||
}
|
||||
@ -1092,7 +1093,7 @@ func (s *xlSets) listObjects(ctx context.Context, bucket, prefix, marker, delimi
|
||||
|
||||
for _, entry := range entries.Files {
|
||||
var objInfo ObjectInfo
|
||||
if hasSuffix(entry.Name, SlashSeparator) {
|
||||
if HasSuffix(entry.Name, SlashSeparator) {
|
||||
if !recursive {
|
||||
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
||||
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) {
|
||||
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)
|
||||
|
||||
// Healing directories handle it separately.
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
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
|
||||
var objInfo ObjectInfo
|
||||
if hasSuffix(entry, SlashSeparator) {
|
||||
if HasSuffix(entry, SlashSeparator) {
|
||||
// Object name needs to be full path.
|
||||
objInfo.Bucket = bucket
|
||||
objInfo.Name = entry
|
||||
@ -156,7 +156,7 @@ func (xl xlObjects) ListObjects(ctx context.Context, bucket, prefix, marker, del
|
||||
// Marker is set validate pre-condition.
|
||||
if marker != "" {
|
||||
// Marker not common with prefix is not implemented.Send an empty response
|
||||
if !hasPrefix(marker, prefix) {
|
||||
if !HasPrefix(marker, prefix) {
|
||||
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
|
||||
// returns no bytes.
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
var objInfo ObjectInfo
|
||||
if objInfo, err = xl.getObjectInfoDir(ctx, bucket, object); err != nil {
|
||||
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 hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
_, err := writer.Write([]byte(""))
|
||||
logger.LogIf(ctx, err)
|
||||
return toObjectErr(err, bucket, object)
|
||||
@ -344,7 +344,7 @@ func (xl xlObjects) GetObjectInfo(ctx context.Context, bucket, object string, op
|
||||
return oi, err
|
||||
}
|
||||
|
||||
if hasSuffix(object, SlashSeparator) {
|
||||
if HasSuffix(object, SlashSeparator) {
|
||||
info, err := xl.getObjectInfoDir(ctx, bucket, object)
|
||||
if err != nil {
|
||||
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 {
|
||||
isObjectDirs[i] = hasSuffix(object, SlashSeparator)
|
||||
isObjectDirs[i] = HasSuffix(object, SlashSeparator)
|
||||
}
|
||||
|
||||
for i, object := range objects {
|
||||
@ -903,7 +903,7 @@ func (xl xlObjects) DeleteObject(ctx context.Context, bucket, object string) (er
|
||||
}
|
||||
|
||||
var writeQuorum int
|
||||
var isObjectDir = hasSuffix(object, SlashSeparator)
|
||||
var isObjectDir = HasSuffix(object, SlashSeparator)
|
||||
|
||||
if isObjectDir {
|
||||
_, err = xl.getObjectInfoDir(ctx, bucket, object)
|
||||
|
@ -200,3 +200,9 @@ func getStorageInfo(disks []StorageAPI) StorageInfo {
|
||||
func (xl xlObjects) StorageInfo(ctx context.Context) StorageInfo {
|
||||
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.
|
||||
if marker != "" {
|
||||
// Marker not common with prefix is not implemented. Send an empty response
|
||||
if !hasPrefix(marker, prefix) {
|
||||
if !HasPrefix(marker, prefix) {
|
||||
return loi, nil
|
||||
}
|
||||
}
|
||||
@ -682,7 +682,7 @@ func (z *xlZones) listObjects(ctx context.Context, bucket, prefix, marker, delim
|
||||
|
||||
for _, entry := range entries.Files {
|
||||
var objInfo ObjectInfo
|
||||
if hasSuffix(entry.Name, SlashSeparator) {
|
||||
if HasSuffix(entry.Name, SlashSeparator) {
|
||||
if !recursive {
|
||||
loi.Prefixes = append(loi.Prefixes, entry.Name)
|
||||
continue
|
||||
@ -1312,3 +1312,9 @@ func (z *xlZones) ListBucketsHeal(ctx context.Context) ([]BucketInfo, error) {
|
||||
}
|
||||
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.
|
||||
> 2. `s3 requests` exclude internode requests.
|
||||
|
||||
|
||||
- standard go runtime metrics prefixed by `go_`
|
||||
- process level metrics prefixed with `process_`
|
||||
- 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.
|
||||
- `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
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user