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:
Nitish Tiwari 2019-12-06 12:46:06 +05:30 committed by Harshavardhana
parent d2dc964cb5
commit 3df7285c3c
32 changed files with 400 additions and 86 deletions

View file

@ -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
}

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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
View 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{}
}

View file

@ -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)
},

View file

@ -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
}

View file

@ -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"
}

View file

@ -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)

View file

@ -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
View 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{}
}

View file

@ -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

View file

@ -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)

View file

@ -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")

View file

@ -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 {

View file

@ -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 {

View file

@ -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,

View file

@ -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)
}

View file

@ -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))
}

View file

@ -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

View file

@ -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

View file

@ -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 {

View file

@ -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)
}
}

View file

@ -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 {

View file

@ -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{}
}

View file

@ -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)
}

View file

@ -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
}
}

View file

@ -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)

View file

@ -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{}
}

View file

@ -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{}
}

View file

@ -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