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:
Harshavardhana
2021-02-24 15:51:41 -08:00
committed by GitHub
parent b912e9ab41
commit b23659927c
14 changed files with 683 additions and 605 deletions

View File

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