mirror of
https://github.com/minio/minio.git
synced 2025-11-09 05:34:56 -05:00
fix: remove persistence layer for metacache store in memory (#11538)
store the cache in-memory instead of disks to avoid large write amplifications for list heavy workloads, store in memory instead and let it auto expire.
This commit is contained in:
@@ -17,23 +17,18 @@
|
||||
package cmd
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/gob"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
jsoniter "github.com/json-iterator/go"
|
||||
"github.com/minio/minio/cmd/logger"
|
||||
"github.com/minio/minio/pkg/color"
|
||||
"github.com/minio/minio/pkg/console"
|
||||
"github.com/minio/minio/pkg/hash"
|
||||
)
|
||||
|
||||
type listPathOptions struct {
|
||||
@@ -91,11 +86,6 @@ type listPathOptions struct {
|
||||
// Include pure directories.
|
||||
IncludeDirectories bool
|
||||
|
||||
// Transient is set if the cache is transient due to an error or being a reserved bucket.
|
||||
// This means the cache metadata will not be persisted on disk.
|
||||
// A transient result will never be returned from the cache so knowing the list id is required.
|
||||
Transient bool
|
||||
|
||||
// discardResult will not persist the cache to storage.
|
||||
// When the initial results are returned listing will be canceled.
|
||||
discardResult bool
|
||||
@@ -196,74 +186,14 @@ func (o *listPathOptions) gatherResults(in <-chan metaCacheEntry) func() (metaCa
|
||||
}
|
||||
}
|
||||
|
||||
// findFirstPart will find the part with 0 being the first that corresponds to the marker in the options.
|
||||
// io.ErrUnexpectedEOF is returned if the place containing the marker hasn't been scanned yet.
|
||||
// io.EOF indicates the marker is beyond the end of the stream and does not exist.
|
||||
func (o *listPathOptions) findFirstPart(fi FileInfo) (int, error) {
|
||||
search := o.Marker
|
||||
if search == "" {
|
||||
search = o.Prefix
|
||||
}
|
||||
if search == "" {
|
||||
return 0, nil
|
||||
}
|
||||
o.debugln("searching for ", search)
|
||||
var tmp metacacheBlock
|
||||
var json = jsoniter.ConfigCompatibleWithStandardLibrary
|
||||
i := 0
|
||||
for {
|
||||
partKey := fmt.Sprintf("%s-metacache-part-%d", ReservedMetadataPrefixLower, i)
|
||||
v, ok := fi.Metadata[partKey]
|
||||
if !ok {
|
||||
o.debugln("no match in metadata, waiting")
|
||||
return -1, io.ErrUnexpectedEOF
|
||||
}
|
||||
err := json.Unmarshal([]byte(v), &tmp)
|
||||
if !ok {
|
||||
logger.LogIf(context.Background(), err)
|
||||
return -1, err
|
||||
}
|
||||
if tmp.First == "" && tmp.Last == "" && tmp.EOS {
|
||||
return 0, errFileNotFound
|
||||
}
|
||||
if tmp.First >= search {
|
||||
o.debugln("First >= search", v)
|
||||
return i, nil
|
||||
}
|
||||
if tmp.Last >= search {
|
||||
o.debugln("Last >= search", v)
|
||||
return i, nil
|
||||
}
|
||||
if tmp.EOS {
|
||||
o.debugln("no match, at EOS", v)
|
||||
return -3, io.EOF
|
||||
}
|
||||
o.debugln("First ", tmp.First, "<", search, " search", i)
|
||||
i++
|
||||
}
|
||||
}
|
||||
|
||||
// updateMetacacheListing will update the metacache listing.
|
||||
func (o *listPathOptions) updateMetacacheListing(m metacache, rpc *peerRESTClient) (metacache, error) {
|
||||
if o.Transient {
|
||||
return localMetacacheMgr.getTransient().updateCacheEntry(m)
|
||||
}
|
||||
if rpc == nil {
|
||||
return localMetacacheMgr.updateCacheEntry(m)
|
||||
}
|
||||
return rpc.UpdateMetacacheListing(context.Background(), m)
|
||||
}
|
||||
|
||||
func getMetacacheBlockInfo(fi FileInfo, block int) (*metacacheBlock, error) {
|
||||
var tmp metacacheBlock
|
||||
partKey := fmt.Sprintf("%s-metacache-part-%d", ReservedMetadataPrefixLower, block)
|
||||
v, ok := fi.Metadata[partKey]
|
||||
if !ok {
|
||||
return nil, io.ErrUnexpectedEOF
|
||||
}
|
||||
return &tmp, json.Unmarshal([]byte(v), &tmp)
|
||||
}
|
||||
|
||||
const metacachePrefix = ".metacache"
|
||||
|
||||
func metacachePrefixForID(bucket, id string) string {
|
||||
@@ -271,8 +201,8 @@ func metacachePrefixForID(bucket, id string) string {
|
||||
}
|
||||
|
||||
// objectPath returns the object path of the cache.
|
||||
func (o *listPathOptions) objectPath(block int) string {
|
||||
return pathJoin(metacachePrefixForID(o.Bucket, o.ID), "block-"+strconv.Itoa(block)+".s2")
|
||||
func (o *listPathOptions) objectPath() string {
|
||||
return pathJoin(metacachePrefixForID(o.Bucket, o.ID), "block.s2")
|
||||
}
|
||||
|
||||
func (o *listPathOptions) SetFilter() {
|
||||
@@ -357,187 +287,33 @@ func (r *metacacheReader) filter(o listPathOptions) (entries metaCacheEntriesSor
|
||||
}
|
||||
|
||||
func (er *erasureObjects) streamMetadataParts(ctx context.Context, o listPathOptions) (entries metaCacheEntriesSorted, err error) {
|
||||
retries := 0
|
||||
rpc := globalNotificationSys.restClientFromHash(o.Bucket)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return entries, ctx.Err()
|
||||
default:
|
||||
}
|
||||
|
||||
// If many failures, check the cache state.
|
||||
if retries > 10 {
|
||||
err := o.checkMetacacheState(ctx, rpc)
|
||||
if err != nil {
|
||||
return entries, fmt.Errorf("remote listing canceled: %w", err)
|
||||
}
|
||||
retries = 1
|
||||
}
|
||||
|
||||
const retryDelay = 500 * time.Millisecond
|
||||
// Load first part metadata...
|
||||
// All operations are performed without locks, so we must be careful and allow for failures.
|
||||
// Read metadata associated with the object from a disk.
|
||||
if retries > 0 {
|
||||
disks := er.getOnlineDisks()
|
||||
if len(disks) == 0 {
|
||||
time.Sleep(retryDelay)
|
||||
retries++
|
||||
continue
|
||||
}
|
||||
|
||||
_, err := disks[0].ReadVersion(ctx, minioMetaBucket, o.objectPath(0), "", false)
|
||||
if err != nil {
|
||||
time.Sleep(retryDelay)
|
||||
retries++
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// Read metadata associated with the object from all disks.
|
||||
fi, metaArr, onlineDisks, err := er.getObjectFileInfo(ctx, minioMetaBucket, o.objectPath(0), ObjectOptions{}, true)
|
||||
if err != nil {
|
||||
switch toObjectErr(err, minioMetaBucket, o.objectPath(0)).(type) {
|
||||
case ObjectNotFound:
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
continue
|
||||
case InsufficientReadQuorum:
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
continue
|
||||
default:
|
||||
return entries, fmt.Errorf("reading first part metadata: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
partN, err := o.findFirstPart(fi)
|
||||
switch {
|
||||
case err == nil:
|
||||
case errors.Is(err, io.ErrUnexpectedEOF):
|
||||
if retries == 10 {
|
||||
err := o.checkMetacacheState(ctx, rpc)
|
||||
if err != nil {
|
||||
return entries, fmt.Errorf("remote listing canceled: %w", err)
|
||||
}
|
||||
retries = -1
|
||||
}
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
continue
|
||||
case errors.Is(err, io.EOF):
|
||||
return entries, io.EOF
|
||||
}
|
||||
|
||||
// We got a stream to start at.
|
||||
loadedPart := 0
|
||||
buf := bufferPool.Get().(*bytes.Buffer)
|
||||
defer func() {
|
||||
buf.Reset()
|
||||
bufferPool.Put(buf)
|
||||
}()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return entries, ctx.Err()
|
||||
default:
|
||||
}
|
||||
|
||||
if partN != loadedPart {
|
||||
if retries > 10 {
|
||||
err := o.checkMetacacheState(ctx, rpc)
|
||||
if err != nil {
|
||||
return entries, fmt.Errorf("waiting for next part %d: %w", partN, err)
|
||||
}
|
||||
retries = 1
|
||||
}
|
||||
|
||||
if retries > 0 {
|
||||
// Load from one disk only
|
||||
disks := er.getOnlineDisks()
|
||||
if len(disks) == 0 {
|
||||
time.Sleep(retryDelay)
|
||||
retries++
|
||||
continue
|
||||
}
|
||||
|
||||
_, err := disks[0].ReadVersion(ctx, minioMetaBucket, o.objectPath(partN), "", false)
|
||||
if err != nil {
|
||||
time.Sleep(retryDelay)
|
||||
retries++
|
||||
continue
|
||||
}
|
||||
}
|
||||
// Load first part metadata...
|
||||
fi, metaArr, onlineDisks, err = er.getObjectFileInfo(ctx, minioMetaBucket, o.objectPath(partN), ObjectOptions{}, true)
|
||||
if err != nil {
|
||||
time.Sleep(retryDelay)
|
||||
retries++
|
||||
continue
|
||||
}
|
||||
loadedPart = partN
|
||||
bi, err := getMetacacheBlockInfo(fi, partN)
|
||||
logger.LogIf(ctx, err)
|
||||
if err == nil {
|
||||
if bi.pastPrefix(o.Prefix) {
|
||||
return entries, io.EOF
|
||||
}
|
||||
}
|
||||
}
|
||||
buf.Reset()
|
||||
err := er.getObjectWithFileInfo(ctx, minioMetaBucket, o.objectPath(partN), 0, fi.Size, buf, fi, metaArr, onlineDisks)
|
||||
if err != nil {
|
||||
switch toObjectErr(err, minioMetaBucket, o.objectPath(partN)).(type) {
|
||||
case ObjectNotFound:
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
continue
|
||||
case InsufficientReadQuorum:
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
continue
|
||||
default:
|
||||
logger.LogIf(ctx, err)
|
||||
return entries, err
|
||||
}
|
||||
}
|
||||
tmp, err := newMetacacheReader(buf)
|
||||
if err != nil {
|
||||
return entries, err
|
||||
}
|
||||
e, err := tmp.filter(o)
|
||||
entries.o = append(entries.o, e.o...)
|
||||
if o.Limit > 0 && entries.len() > o.Limit {
|
||||
entries.truncate(o.Limit)
|
||||
return entries, nil
|
||||
}
|
||||
if err == nil {
|
||||
// We stopped within the listing, we are done for now...
|
||||
return entries, nil
|
||||
}
|
||||
if !errors.Is(err, io.EOF) {
|
||||
logger.LogIf(ctx, err)
|
||||
return entries, err
|
||||
}
|
||||
|
||||
// We finished at the end of the block.
|
||||
// And should not expect any more results.
|
||||
bi, err := getMetacacheBlockInfo(fi, partN)
|
||||
logger.LogIf(ctx, err)
|
||||
if err != nil || bi.EOS {
|
||||
// We are done and there are no more parts.
|
||||
return entries, io.EOF
|
||||
}
|
||||
if bi.endedPrefix(o.Prefix) {
|
||||
// Nothing more for prefix.
|
||||
return entries, io.EOF
|
||||
}
|
||||
partN++
|
||||
retries = 0
|
||||
}
|
||||
r, err := er.metaCache.Open(pathJoin(minioMetaBucket, o.objectPath()), time.Now().Add(-time.Hour))
|
||||
if err != nil {
|
||||
return entries, io.EOF
|
||||
}
|
||||
|
||||
tmp, err := newMetacacheReader(r)
|
||||
if err != nil {
|
||||
return entries, err
|
||||
}
|
||||
|
||||
e, err := tmp.filter(o)
|
||||
entries.o = append(entries.o, e.o...)
|
||||
if o.Limit > 0 && entries.len() > o.Limit {
|
||||
entries.truncate(o.Limit)
|
||||
return entries, nil
|
||||
}
|
||||
|
||||
if err == nil {
|
||||
// We stopped within the listing, we are done for now...
|
||||
return entries, nil
|
||||
}
|
||||
|
||||
if !errors.Is(err, io.EOF) {
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
|
||||
return entries, err
|
||||
}
|
||||
|
||||
// Will return io.EOF if continuing would not yield more results.
|
||||
@@ -658,28 +434,14 @@ func (er *erasureObjects) listPath(ctx context.Context, o listPathOptions) (entr
|
||||
}
|
||||
}()
|
||||
|
||||
const retryDelay = 200 * time.Millisecond
|
||||
const maxTries = 5
|
||||
wc := er.metaCache.Create(pathJoin(minioMetaBucket, o.objectPath()))
|
||||
|
||||
var bw *metacacheBlockWriter
|
||||
// Don't save single object listings.
|
||||
if !o.discardResult {
|
||||
// Write results to disk.
|
||||
bw = newMetacacheBlockWriter(cacheCh, func(b *metacacheBlock) error {
|
||||
// if the block is 0 bytes and its a first block skip it.
|
||||
// skip only this for Transient caches.
|
||||
if len(b.data) == 0 && b.n == 0 && o.Transient {
|
||||
return nil
|
||||
}
|
||||
o.debugln(color.Green("listPath:")+" saving block", b.n, "to", o.objectPath(b.n))
|
||||
r, err := hash.NewReader(bytes.NewReader(b.data), int64(len(b.data)), "", "", int64(len(b.data)))
|
||||
logger.LogIf(ctx, err)
|
||||
custom := b.headerKV()
|
||||
_, err = er.putObject(ctx, minioMetaBucket, o.objectPath(b.n), NewPutObjReader(r), ObjectOptions{
|
||||
UserDefined: custom,
|
||||
NoLock: true, // No need to hold namespace lock, each prefix caches uniquely.
|
||||
ParentIsObject: nil,
|
||||
})
|
||||
n, err := wc.Write(b.data)
|
||||
if err != nil {
|
||||
metaMu.Lock()
|
||||
if meta.error != "" {
|
||||
@@ -690,29 +452,17 @@ func (er *erasureObjects) listPath(ctx context.Context, o listPathOptions) (entr
|
||||
cancel()
|
||||
return err
|
||||
}
|
||||
if b.n == 0 {
|
||||
return nil
|
||||
}
|
||||
// Update block 0 metadata.
|
||||
var retries int
|
||||
for {
|
||||
err := er.updateObjectMeta(ctx, minioMetaBucket, o.objectPath(0), b.headerKV(), ObjectOptions{})
|
||||
if err == nil {
|
||||
break
|
||||
}
|
||||
switch err.(type) {
|
||||
case ObjectNotFound:
|
||||
return err
|
||||
case InsufficientReadQuorum:
|
||||
default:
|
||||
logger.LogIf(ctx, err)
|
||||
}
|
||||
if retries >= maxTries {
|
||||
return err
|
||||
}
|
||||
retries++
|
||||
time.Sleep(retryDelay)
|
||||
if n != len(b.data) {
|
||||
metaMu.Lock()
|
||||
if meta.error != "" {
|
||||
meta.status = scanStateError
|
||||
meta.error = io.ErrShortWrite.Error()
|
||||
}
|
||||
metaMu.Unlock()
|
||||
cancel()
|
||||
return io.ErrShortWrite
|
||||
}
|
||||
o.debugln(color.Green("listPath:")+" saving block to", o.objectPath())
|
||||
return nil
|
||||
})
|
||||
}
|
||||
@@ -772,6 +522,13 @@ func (er *erasureObjects) listPath(ctx context.Context, o listPathOptions) (entr
|
||||
meta, _ = o.updateMetacacheListing(meta, rpc)
|
||||
metaMu.Unlock()
|
||||
}
|
||||
if err := wc.Close(); err != nil {
|
||||
metaMu.Lock()
|
||||
meta.error = err.Error()
|
||||
meta.status = scanStateError
|
||||
meta, _ = o.updateMetacacheListing(meta, rpc)
|
||||
metaMu.Unlock()
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
|
||||
Reference in New Issue
Block a user