From 7478521ac76f30201c65af2b3d5eec11831990dc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 14:48:08 +0000 Subject: [PATCH 01/37] WIP --- x-pack/apm-server/main.go | 22 ++++++++++++++++++++-- x-pack/apm-server/sampling/config.go | 3 +-- x-pack/apm-server/sampling/processor.go | 14 ++++++++++++-- 3 files changed, 33 insertions(+), 6 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index caab7b061e2..7283db7c0ab 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -11,6 +11,10 @@ import ( "os" "sync" + "github.com/cockroachdb/pebble" + + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstoragepebble" + "github.com/dgraph-io/badger/v2" "github.com/gofrs/uuid/v5" "golang.org/x/sync/errgroup" @@ -46,7 +50,7 @@ var ( badgerDB *badger.DB storageMu sync.Mutex - storage *eventstorage.ShardedReadWriter + storage sampling.RW // samplerUUID is a UUID used to identify sampled trace ID documents // published by this process. @@ -179,7 +183,7 @@ func getBadgerDB(storageDir string) (*badger.DB, error) { return badgerDB, nil } -func getStorage(db *badger.DB) *eventstorage.ShardedReadWriter { +func getStorage(db *badger.DB) sampling.RW { storageMu.Lock() defer storageMu.Unlock() if storage == nil { @@ -189,6 +193,20 @@ func getStorage(db *badger.DB) *eventstorage.ShardedReadWriter { return storage } +func getPebbleDB(storageDir string) (*pebble.DB, error) { + return pebble.Open(storageDir, &pebble.Options{}) +} + +func getPebbleStorage(db *pebble.DB) sampling.RW { + storageMu.Lock() + defer storageMu.Unlock() + if storage == nil { + eventCodec := eventstoragepebble.ProtobufCodec{} + storage = eventstoragepebble.New(db, eventCodec).NewReadWriter() + } + return storage +} + // runServerWithProcessors runs the APM Server and the given list of processors. // // newProcessors returns a list of processors which will process events in diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 43d5c34d96e..bdffce4bee7 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -11,7 +11,6 @@ import ( "github.com/pkg/errors" "github.com/elastic/apm-data/model/modelpb" - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" "github.com/elastic/apm-server/x-pack/apm-server/sampling/pubsub" "github.com/elastic/go-elasticsearch/v8" ) @@ -105,7 +104,7 @@ type StorageConfig struct { // // Storage lives outside processor lifecycle and will not be closed when processor // is closed - Storage *eventstorage.ShardedReadWriter + Storage RW // StorageDir holds the directory in which event storage will be maintained. StorageDir string diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 9402408b299..7de49996a14 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -623,9 +623,19 @@ const ( storageLimitThreshold = 0.90 // Allow 90% of the quota to be used. ) +type RW interface { + Close() + Flush() error + WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error + IsTraceSampled(traceID string) (bool, error) + WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error + DeleteTraceEvent(traceID, id string) error + ReadTraceEvents(traceID string, out *modelpb.Batch) error +} + // wrappedRW wraps configurable write options for global ShardedReadWriter type wrappedRW struct { - rw *eventstorage.ShardedReadWriter + rw RW writerOpts eventstorage.WriterOpts } @@ -634,7 +644,7 @@ type wrappedRW struct { // limit value greater than zero. The hard limit on storage is set to 90% of // the limit to account for delay in the size reporting by badger. // https://github.com/dgraph-io/badger/blob/82b00f27e3827022082225221ae05c03f0d37620/db.go#L1302-L1319. -func newWrappedRW(rw *eventstorage.ShardedReadWriter, ttl time.Duration, limit int64) *wrappedRW { +func newWrappedRW(rw RW, ttl time.Duration, limit int64) *wrappedRW { if limit > 1 { limit = int64(float64(limit) * storageLimitThreshold) } From 29b3a6339c71dfc0437e364be1c8e718fefee4c4 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 14:49:14 +0000 Subject: [PATCH 02/37] Add pebble --- .../sampling/eventstoragepebble/logger.go | 50 +++ .../eventstoragepebble/protobufcodec.go | 22 ++ .../sampling/eventstoragepebble/storage.go | 284 ++++++++++++++++++ 3 files changed, 356 insertions(+) create mode 100644 x-pack/apm-server/sampling/eventstoragepebble/logger.go create mode 100644 x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go create mode 100644 x-pack/apm-server/sampling/eventstoragepebble/storage.go diff --git a/x-pack/apm-server/sampling/eventstoragepebble/logger.go b/x-pack/apm-server/sampling/eventstoragepebble/logger.go new file mode 100644 index 00000000000..fb9f2d69756 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstoragepebble/logger.go @@ -0,0 +1,50 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License 2.0; +// you may not use this file except in compliance with the Elastic License 2.0. + +package eventstoragepebble + +import ( + "fmt" + "sync" + + "github.com/elastic/elastic-agent-libs/logp" +) + +// LogpAdaptor adapts logp.Logger to the badger.Logger interface. +type LogpAdaptor struct { + *logp.Logger + + mu sync.RWMutex + last string +} + +// Errorf prints the log message when the current message isn't the same as the +// previously logged message. +func (a *LogpAdaptor) Errorf(format string, args ...interface{}) { + msg := fmt.Sprintf(format, args...) + if a.setLast(msg) { + a.Logger.Errorf(format, args...) + } +} + +func (a *LogpAdaptor) setLast(msg string) bool { + a.mu.RLock() + if msg != a.last { + a.mu.RUnlock() + return false + } + a.mu.RUnlock() + a.mu.Lock() + defer a.mu.Unlock() + shouldSet := msg != a.last + if shouldSet { + a.last = msg + } + return shouldSet +} + +// Warningf adapts badger.Logger.Warningf to logp.Logger.Warngf. +func (a *LogpAdaptor) Warningf(format string, args ...interface{}) { + a.Warnf(format, args...) +} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go b/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go new file mode 100644 index 00000000000..40862ae8f23 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go @@ -0,0 +1,22 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License 2.0; +// you may not use this file except in compliance with the Elastic License 2.0. + +package eventstoragepebble + +import ( + "github.com/elastic/apm-data/model/modelpb" +) + +// ProtobufCodec is an implementation of Codec, using protobuf encoding. +type ProtobufCodec struct{} + +// DecodeEvent decodes data as protobuf into event. +func (ProtobufCodec) DecodeEvent(data []byte, event *modelpb.APMEvent) error { + return event.UnmarshalVT(data) +} + +// EncodeEvent encodes event as protobuf. +func (ProtobufCodec) EncodeEvent(event *modelpb.APMEvent) ([]byte, error) { + return event.MarshalVT() +} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go new file mode 100644 index 00000000000..0b90e02a63f --- /dev/null +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -0,0 +1,284 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License 2.0; +// you may not use this file except in compliance with the Elastic License 2.0. + +package eventstoragepebble + +import ( + "bytes" + "errors" + "fmt" + "github.com/cockroachdb/pebble" + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" + "sync/atomic" + "time" + + "github.com/elastic/apm-data/model/modelpb" +) + +const ( + // NOTE(axw) these values (and their meanings) must remain stable + // over time, to avoid misinterpreting historical data. + entryMetaTraceSampled = 's' + entryMetaTraceUnsampled = 'u' + entryMetaTraceEvent = 'e' + + // Initial transaction size + // len(txnKey) + 10 + baseTransactionSize = 10 + 11 +) + +var ( + // ErrNotFound is returned by by the Storage.IsTraceSampled method, + // for non-existing trace IDs. + ErrNotFound = errors.New("key not found") + + // ErrLimitReached is returned by the ReadWriter.Flush method when + // the configured StorageLimiter.Limit is true. + ErrLimitReached = errors.New("configured storage limit reached") +) + +// Storage provides storage for sampled transactions and spans, +// and for recording trace sampling decisions. +type Storage struct { + db *pebble.DB + // pendingSize tracks the total size of pending writes across ReadWriters + pendingSize *atomic.Int64 + codec Codec +} + +// Codec provides methods for encoding and decoding events. +type Codec interface { + DecodeEvent([]byte, *modelpb.APMEvent) error + EncodeEvent(*modelpb.APMEvent) ([]byte, error) +} + +// New returns a new Storage using db and codec. +func New(db *pebble.DB, codec Codec) *Storage { + return &Storage{db: db, pendingSize: &atomic.Int64{}, codec: codec} +} + +// NewReadWriter returns a new ReadWriter for reading events from and +// writing events to storage. +// +// The returned ReadWriter must be closed when it is no longer needed. +func (s *Storage) NewReadWriter() *ReadWriter { + //s.pendingSize.Add(baseTransactionSize) + return &ReadWriter{ + s: s, + //pendingSize: baseTransactionSize, + } +} + +// WriterOpts provides configuration options for writes to storage +type WriterOpts struct { + TTL time.Duration + StorageLimitInBytes int64 +} + +// ReadWriter provides a means of reading events from storage, and batched +// writing of events to storage. +// +// ReadWriter is not safe for concurrent access. All operations that involve +// a given trace ID should be performed with the same ReadWriter in order to +// avoid conflicts, e.g. by using consistent hashing to distribute to one of +// a set of ReadWriters, such as implemented by ShardedReadWriter. +type ReadWriter struct { + s *Storage + + // readKeyBuf is a reusable buffer for keys used in read operations. + // This must not be used in write operations, as keys are expected to + // be unmodified until the end of a transaction. + readKeyBuf []byte +} + +// Close closes the writer. Any writes that have not been flushed may be lost. +// +// This must be called when the writer is no longer needed, in order to reclaim +// resources. +func (rw *ReadWriter) Close() { + //rw.txn.Discard() +} + +// Flush waits for preceding writes to be committed to storage. +// +// Flush must be called to ensure writes are committed to storage. +// If Flush is not called before the writer is closed, then writes +// may be lost. +func (rw *ReadWriter) Flush() error { + //const flushErrFmt = "failed to flush pending writes: %w" + //err := rw.txn.Commit() + //rw.txn = rw.s.db.NewTransaction(true) + //rw.s.pendingSize.Add(-rw.pendingSize) + //rw.pendingWrites = 0 + //rw.pendingSize = baseTransactionSize + //rw.s.pendingSize.Add(baseTransactionSize) + //if err != nil { + // return fmt.Errorf(flushErrFmt, err) + //} + return nil +} + +// WriteTraceSampled records the tail-sampling decision for the given trace ID. +func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { + //key := []byte(traceID) + //var meta uint8 = entryMetaTraceUnsampled + //if sampled { + // meta = entryMetaTraceSampled + //} + //return rw.writeEntry(badger.NewEntry(key[:], nil).WithMeta(meta), opts) + return nil +} + +// IsTraceSampled reports whether traceID belongs to a trace that is sampled +// or unsampled. If no sampling decision has been recorded, IsTraceSampled +// returns ErrNotFound. +func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { + //rw.readKeyBuf = append(rw.readKeyBuf[:0], traceID...) + //item, err := rw.txn.Get(rw.readKeyBuf) + //if err != nil { + // if err == badger.ErrKeyNotFound { + // return false, ErrNotFound + // } + // return false, err + //} + //return item.UserMeta() == entryMetaTraceSampled, nil + return false, nil +} + +// WriteTraceEvent writes a trace event to storage. +// +// WriteTraceEvent may return before the write is committed to storage. +// Call Flush to ensure the write is committed. +func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { + data, err := rw.s.codec.EncodeEvent(event) + if err != nil { + return err + } + var buf bytes.Buffer + buf.Grow(len(traceID) + 1 + len(id)) + buf.WriteString(traceID) + buf.WriteByte(':') + buf.WriteString(id) + key := buf.Bytes() + return rw.s.db.Set(key, data, pebble.NoSync) + //rw.writeEntry(badger.NewEntry(key, data).WithMeta(entryMetaTraceEvent), opts) +} + +//func (rw *ReadWriter) writeEntry(e *badger.Entry, opts WriterOpts) error { +// rw.pendingWrites++ +// entrySize := estimateSize(e) +// // The badger database has an async size reconciliation, with a 1 minute +// // ticker that keeps the lsm and vlog sizes updated in an in-memory map. +// // It's OK to call call s.db.Size() on the hot path, since the memory +// // lookup is cheap. +// lsm, vlog := rw.s.db.Size() +// +// // there are multiple ReadWriters writing to the same storage so add +// // the entry size and consider the new value to avoid TOCTOU issues. +// pendingSize := rw.s.pendingSize.Add(entrySize) +// rw.pendingSize += entrySize +// +// if current := pendingSize + lsm + vlog; opts.StorageLimitInBytes != 0 && current >= opts.StorageLimitInBytes { +// // flush what we currently have and discard the current entry +// if err := rw.Flush(); err != nil { +// return err +// } +// return fmt.Errorf("%w (current: %d, limit: %d)", ErrLimitReached, current, opts.StorageLimitInBytes) +// } +// +// if rw.pendingWrites >= 200 { +// // Attempt to flush if there are 200 or more uncommitted writes. +// // This ensures calls to ReadTraceEvents are not slowed down; +// // ReadTraceEvents uses an iterator, which must sort all keys +// // of uncommitted writes. +// // The 200 value yielded a good balance between read and write speed: +// // https://github.com/elastic/apm-server/pull/8407#issuecomment-1162994643 +// if err := rw.Flush(); err != nil { +// return err +// } +// +// // the current ReadWriter flushed the transaction and reset the pendingSize so add +// // the entrySize again. +// rw.pendingSize += entrySize +// rw.s.pendingSize.Add(entrySize) +// } +// +// err := rw.txn.SetEntry(e.WithTTL(opts.TTL)) +// +// // If the transaction is already too big to accommodate the new entry, flush +// // the existing transaction and set the entry on a new one, otherwise, +// // returns early. +// if err != badger.ErrTxnTooBig { +// return err +// } +// if err := rw.Flush(); err != nil { +// return err +// } +// rw.pendingSize += entrySize +// rw.s.pendingSize.Add(entrySize) +// return rw.txn.SetEntry(e.WithTTL(opts.TTL)) +//} +// +//func estimateSize(e *badger.Entry) int64 { +// // See badger WithValueThreshold option +// // An storage usage of an entry depends on its size +// // +// // if len(e.Value) < threshold { +// // return len(e.Key) + len(e.Value) + 2 // Meta, UserMeta +// // } +// // return len(e.Key) + 12 + 2 // 12 for ValuePointer, 2 for metas. +// // +// // Make a good estimate by reserving more space +// estimate := len(e.Key) + len(e.Value) + 12 + 2 +// // Extra bytes for the version in key. +// return int64(estimate) + 10 +//} + +// DeleteTraceEvent deletes the trace event from storage. +func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { + //var buf bytes.Buffer + //buf.Grow(len(traceID) + 1 + len(id)) + //buf.WriteString(traceID) + //buf.WriteByte(':') + //buf.WriteString(id) + //key := buf.Bytes() + // + //err := rw.txn.Delete(key) + //// If the transaction is already too big to accommodate the new entry, flush + //// the existing transaction and set the entry on a new one, otherwise, + //// returns early. + //if err != badger.ErrTxnTooBig { + // return err + //} + //if err := rw.Flush(); err != nil { + // return err + //} + // + //return rw.txn.Delete(key) + return nil +} + +// ReadTraceEvents reads trace events with the given trace ID from storage into out. +func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + iter, err := rw.s.db.NewIter(&pebble.IterOptions{ + LowerBound: append([]byte(traceID), ':'), + UpperBound: append([]byte(traceID), ';'), + }) + if err != nil { + return err + } + defer iter.Close() + for iter.First(); iter.Valid(); iter.Next() { + event := &modelpb.APMEvent{} + data, err := iter.ValueAndErr() + if err != nil { + return err + } + if err := rw.s.codec.DecodeEvent(data, event); err != nil { + return fmt.Errorf("codec failed to decode event: %w", err) + } + *out = append(*out, event) + } + return nil +} From 452c0ca6823088f60cefbb6ee2498b476f09dc41 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 16:16:58 +0000 Subject: [PATCH 03/37] Use pebble --- x-pack/apm-server/main.go | 4 +- x-pack/apm-server/sampling/config.go | 6 +- .../sampling/eventstoragepebble/storage.go | 145 +++++++++++------- x-pack/apm-server/sampling/processor.go | 79 +++++----- 4 files changed, 131 insertions(+), 103 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 7283db7c0ab..319656e15fe 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -122,11 +122,11 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er } storageDir := paths.Resolve(paths.Data, tailSamplingStorageDir) - badgerDB, err = getBadgerDB(storageDir) + db, err := getPebbleDB(storageDir) if err != nil { return nil, fmt.Errorf("failed to get Badger database: %w", err) } - readWriters := getStorage(badgerDB) + readWriters := getPebbleStorage(db) policies := make([]sampling.Policy, len(tailSamplingConfig.Policies)) for i, in := range tailSamplingConfig.Policies { diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index bdffce4bee7..e8364c9dca5 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -234,9 +234,9 @@ func (config DataStreamConfig) validate() error { } func (config StorageConfig) validate() error { - if config.DB == nil { - return errors.New("DB unspecified") - } + //if config.DB == nil { + // return errors.New("DB unspecified") + //} if config.Storage == nil { return errors.New("Storage unspecified") } diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index 0b90e02a63f..feec4e222c7 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -10,6 +10,7 @@ import ( "fmt" "github.com/cockroachdb/pebble" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" + "sync" "sync/atomic" "time" @@ -90,6 +91,9 @@ type ReadWriter struct { // This must not be used in write operations, as keys are expected to // be unmodified until the end of a transaction. readKeyBuf []byte + + mu sync.Mutex + batch *pebble.Batch } // Close closes the writer. Any writes that have not been flushed may be lost. @@ -143,7 +147,7 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // return false, err //} //return item.UserMeta() == entryMetaTraceSampled, nil - return false, nil + return false, eventstorage.ErrNotFound } // WriteTraceEvent writes a trace event to storage. @@ -161,64 +165,84 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. buf.WriteByte(':') buf.WriteString(id) key := buf.Bytes() - return rw.s.db.Set(key, data, pebble.NoSync) - //rw.writeEntry(badger.NewEntry(key, data).WithMeta(entryMetaTraceEvent), opts) + + //return rw.s.db.Set(key, data, pebble.NoSync) + return rw.writeEntry(key, data) +} + +func (rw *ReadWriter) writeEntry(key, data []byte) error { + rw.mu.Lock() + defer rw.mu.Unlock() + if rw.batch == nil { + rw.batch = rw.s.db.NewIndexedBatch() + } + if err := rw.batch.Set(key, data, pebble.NoSync); err != nil { + return err + } + + if rw.batch.Len() > 2000 { + err := rw.batch.Commit(pebble.Sync) + rw.batch.Close() + rw.batch = nil + return err + } + return nil + + // + //rw.pendingWrites++ + //entrySize := estimateSize(e) + //// The badger database has an async size reconciliation, with a 1 minute + //// ticker that keeps the lsm and vlog sizes updated in an in-memory map. + //// It's OK to call call s.db.Size() on the hot path, since the memory + //// lookup is cheap. + //lsm, vlog := rw.s.db.Size() + // + //// there are multiple ReadWriters writing to the same storage so add + //// the entry size and consider the new value to avoid TOCTOU issues. + //pendingSize := rw.s.pendingSize.Add(entrySize) + //rw.pendingSize += entrySize + // + //if current := pendingSize + lsm + vlog; opts.StorageLimitInBytes != 0 && current >= opts.StorageLimitInBytes { + // // flush what we currently have and discard the current entry + // if err := rw.Flush(); err != nil { + // return err + // } + // return fmt.Errorf("%w (current: %d, limit: %d)", ErrLimitReached, current, opts.StorageLimitInBytes) + //} + // + //if rw.pendingWrites >= 200 { + // // Attempt to flush if there are 200 or more uncommitted writes. + // // This ensures calls to ReadTraceEvents are not slowed down; + // // ReadTraceEvents uses an iterator, which must sort all keys + // // of uncommitted writes. + // // The 200 value yielded a good balance between read and write speed: + // // https://github.com/elastic/apm-server/pull/8407#issuecomment-1162994643 + // if err := rw.Flush(); err != nil { + // return err + // } + // + // // the current ReadWriter flushed the transaction and reset the pendingSize so add + // // the entrySize again. + // rw.pendingSize += entrySize + // rw.s.pendingSize.Add(entrySize) + //} + // + //err := rw.txn.SetEntry(e.WithTTL(opts.TTL)) + // + //// If the transaction is already too big to accommodate the new entry, flush + //// the existing transaction and set the entry on a new one, otherwise, + //// returns early. + //if err != badger.ErrTxnTooBig { + // return err + //} + //if err := rw.Flush(); err != nil { + // return err + //} + //rw.pendingSize += entrySize + //rw.s.pendingSize.Add(entrySize) + //return rw.txn.SetEntry(e.WithTTL(opts.TTL)) } -//func (rw *ReadWriter) writeEntry(e *badger.Entry, opts WriterOpts) error { -// rw.pendingWrites++ -// entrySize := estimateSize(e) -// // The badger database has an async size reconciliation, with a 1 minute -// // ticker that keeps the lsm and vlog sizes updated in an in-memory map. -// // It's OK to call call s.db.Size() on the hot path, since the memory -// // lookup is cheap. -// lsm, vlog := rw.s.db.Size() -// -// // there are multiple ReadWriters writing to the same storage so add -// // the entry size and consider the new value to avoid TOCTOU issues. -// pendingSize := rw.s.pendingSize.Add(entrySize) -// rw.pendingSize += entrySize -// -// if current := pendingSize + lsm + vlog; opts.StorageLimitInBytes != 0 && current >= opts.StorageLimitInBytes { -// // flush what we currently have and discard the current entry -// if err := rw.Flush(); err != nil { -// return err -// } -// return fmt.Errorf("%w (current: %d, limit: %d)", ErrLimitReached, current, opts.StorageLimitInBytes) -// } -// -// if rw.pendingWrites >= 200 { -// // Attempt to flush if there are 200 or more uncommitted writes. -// // This ensures calls to ReadTraceEvents are not slowed down; -// // ReadTraceEvents uses an iterator, which must sort all keys -// // of uncommitted writes. -// // The 200 value yielded a good balance between read and write speed: -// // https://github.com/elastic/apm-server/pull/8407#issuecomment-1162994643 -// if err := rw.Flush(); err != nil { -// return err -// } -// -// // the current ReadWriter flushed the transaction and reset the pendingSize so add -// // the entrySize again. -// rw.pendingSize += entrySize -// rw.s.pendingSize.Add(entrySize) -// } -// -// err := rw.txn.SetEntry(e.WithTTL(opts.TTL)) -// -// // If the transaction is already too big to accommodate the new entry, flush -// // the existing transaction and set the entry on a new one, otherwise, -// // returns early. -// if err != badger.ErrTxnTooBig { -// return err -// } -// if err := rw.Flush(); err != nil { -// return err -// } -// rw.pendingSize += entrySize -// rw.s.pendingSize.Add(entrySize) -// return rw.txn.SetEntry(e.WithTTL(opts.TTL)) -//} // //func estimateSize(e *badger.Entry) int64 { // // See badger WithValueThreshold option @@ -261,7 +285,12 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - iter, err := rw.s.db.NewIter(&pebble.IterOptions{ + rw.mu.Lock() + defer rw.mu.Unlock() + if rw.batch == nil { + rw.batch = rw.s.db.NewIndexedBatch() + } + iter, err := rw.batch.NewIter(&pebble.IterOptions{ LowerBound: append([]byte(traceID), ':'), UpperBound: append([]byte(traceID), ';'), }) diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 7de49996a14..7f5bb85ed4d 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -14,7 +14,6 @@ import ( "sync/atomic" "time" - "github.com/dgraph-io/badger/v2" "github.com/pkg/errors" "golang.org/x/sync/errgroup" @@ -117,11 +116,11 @@ func (p *Processor) CollectMonitoring(_ monitoring.Mode, V monitoring.Visitor) { p.groups.mu.RUnlock() monitoring.ReportInt(V, "dynamic_service_groups", int64(numDynamicGroups)) - monitoring.ReportNamespace(V, "storage", func() { - lsmSize, valueLogSize := p.config.DB.Size() - monitoring.ReportInt(V, "lsm_size", int64(lsmSize)) - monitoring.ReportInt(V, "value_log_size", int64(valueLogSize)) - }) + //monitoring.ReportNamespace(V, "storage", func() { + // lsmSize, valueLogSize := p.config.DB.Size() + // monitoring.ReportInt(V, "lsm_size", int64(lsmSize)) + // monitoring.ReportInt(V, "value_log_size", int64(valueLogSize)) + //}) monitoring.ReportNamespace(V, "events", func() { monitoring.ReportInt(V, "processed", atomic.LoadInt64(&p.eventMetrics.processed)) monitoring.ReportInt(V, "dropped", atomic.LoadInt64(&p.eventMetrics.dropped)) @@ -390,40 +389,40 @@ func (p *Processor) Run() error { } } }) - g.Go(func() error { - // Protect this goroutine from running concurrently when 2 TBS processors are active - // as badger GC is not concurrent safe. - select { - case <-p.stopping: - return nil - case gcCh <- struct{}{}: - } - defer func() { - <-gcCh - }() - // This goroutine is responsible for periodically garbage - // collecting the Badger value log, using the recommended - // discard ratio of 0.5. - ticker := time.NewTicker(p.config.StorageGCInterval) - defer ticker.Stop() - for { - select { - case <-p.stopping: - return nil - case <-ticker.C: - const discardRatio = 0.5 - var err error - for err == nil { - // Keep garbage collecting until there are no more rewrites, - // or garbage collection fails. - err = p.config.DB.RunValueLogGC(discardRatio) - } - if err != nil && err != badger.ErrNoRewrite { - return err - } - } - } - }) + //g.Go(func() error { + // // Protect this goroutine from running concurrently when 2 TBS processors are active + // // as badger GC is not concurrent safe. + // select { + // case <-p.stopping: + // return nil + // case gcCh <- struct{}{}: + // } + // defer func() { + // <-gcCh + // }() + // // This goroutine is responsible for periodically garbage + // // collecting the Badger value log, using the recommended + // // discard ratio of 0.5. + // ticker := time.NewTicker(p.config.StorageGCInterval) + // defer ticker.Stop() + // for { + // select { + // case <-p.stopping: + // return nil + // case <-ticker.C: + // const discardRatio = 0.5 + // var err error + // for err == nil { + // // Keep garbage collecting until there are no more rewrites, + // // or garbage collection fails. + // err = p.config.DB.RunValueLogGC(discardRatio) + // } + // if err != nil && err != badger.ErrNoRewrite { + // return err + // } + // } + // } + //}) g.Go(func() error { // Subscribe to remotely sampled trace IDs. This is cancelled immediately when // Stop is called. But it is possible that both old and new subscriber goroutines From e30ee49244ba73e2e295a7477853d39d20fe702e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 16:33:52 +0000 Subject: [PATCH 04/37] Implement sampling decision --- .../sampling/eventstoragepebble/storage.go | 149 ++++-------------- 1 file changed, 33 insertions(+), 116 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index feec4e222c7..ac3b8ce3e23 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -6,37 +6,24 @@ package eventstoragepebble import ( "bytes" - "errors" "fmt" - "github.com/cockroachdb/pebble" - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" "sync" "sync/atomic" "time" + "github.com/cockroachdb/pebble" + + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" + "github.com/elastic/apm-data/model/modelpb" ) const ( // NOTE(axw) these values (and their meanings) must remain stable // over time, to avoid misinterpreting historical data. - entryMetaTraceSampled = 's' - entryMetaTraceUnsampled = 'u' - entryMetaTraceEvent = 'e' - - // Initial transaction size - // len(txnKey) + 10 - baseTransactionSize = 10 + 11 -) - -var ( - // ErrNotFound is returned by by the Storage.IsTraceSampled method, - // for non-existing trace IDs. - ErrNotFound = errors.New("key not found") - - // ErrLimitReached is returned by the ReadWriter.Flush method when - // the configured StorageLimiter.Limit is true. - ErrLimitReached = errors.New("configured storage limit reached") + entryMetaTraceSampled byte = 's' + entryMetaTraceUnsampled byte = 'u' + entryMetaTraceEvent byte = 'e' ) // Storage provides storage for sampled transactions and spans, @@ -66,7 +53,8 @@ func New(db *pebble.DB, codec Codec) *Storage { func (s *Storage) NewReadWriter() *ReadWriter { //s.pendingSize.Add(baseTransactionSize) return &ReadWriter{ - s: s, + s: s, + batch: s.db.NewIndexedBatch(), //pendingSize: baseTransactionSize, } } @@ -110,44 +98,36 @@ func (rw *ReadWriter) Close() { // If Flush is not called before the writer is closed, then writes // may be lost. func (rw *ReadWriter) Flush() error { - //const flushErrFmt = "failed to flush pending writes: %w" - //err := rw.txn.Commit() - //rw.txn = rw.s.db.NewTransaction(true) - //rw.s.pendingSize.Add(-rw.pendingSize) - //rw.pendingWrites = 0 - //rw.pendingSize = baseTransactionSize - //rw.s.pendingSize.Add(baseTransactionSize) - //if err != nil { - // return fmt.Errorf(flushErrFmt, err) - //} + err := rw.batch.Commit(pebble.Sync) + rw.batch.Close() + rw.batch = rw.s.db.NewIndexedBatch() + const flushErrFmt = "failed to flush pending writes: %w" + if err != nil { + return fmt.Errorf(flushErrFmt, err) + } return nil } // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { - //key := []byte(traceID) - //var meta uint8 = entryMetaTraceUnsampled - //if sampled { - // meta = entryMetaTraceSampled - //} - //return rw.writeEntry(badger.NewEntry(key[:], nil).WithMeta(meta), opts) - return nil + key := []byte(traceID) + meta := entryMetaTraceUnsampled + if sampled { + meta = entryMetaTraceSampled + } + return rw.batch.Set(key, []byte{meta}, pebble.NoSync) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - //rw.readKeyBuf = append(rw.readKeyBuf[:0], traceID...) - //item, err := rw.txn.Get(rw.readKeyBuf) - //if err != nil { - // if err == badger.ErrKeyNotFound { - // return false, ErrNotFound - // } - // return false, err - //} - //return item.UserMeta() == entryMetaTraceSampled, nil - return false, eventstorage.ErrNotFound + item, closer, err := rw.batch.Get([]byte(traceID)) + if err == pebble.ErrNotFound { + return false, eventstorage.ErrNotFound + } + defer closer.Close() + return item[0] == entryMetaTraceSampled, nil } // WriteTraceEvent writes a trace event to storage. @@ -165,82 +145,22 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. buf.WriteByte(':') buf.WriteString(id) key := buf.Bytes() - - //return rw.s.db.Set(key, data, pebble.NoSync) return rw.writeEntry(key, data) } func (rw *ReadWriter) writeEntry(key, data []byte) error { rw.mu.Lock() defer rw.mu.Unlock() - if rw.batch == nil { - rw.batch = rw.s.db.NewIndexedBatch() - } - if err := rw.batch.Set(key, data, pebble.NoSync); err != nil { + if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { return err } if rw.batch.Len() > 2000 { - err := rw.batch.Commit(pebble.Sync) - rw.batch.Close() - rw.batch = nil - return err + if err := rw.Flush(); err != nil { + return err + } } return nil - - // - //rw.pendingWrites++ - //entrySize := estimateSize(e) - //// The badger database has an async size reconciliation, with a 1 minute - //// ticker that keeps the lsm and vlog sizes updated in an in-memory map. - //// It's OK to call call s.db.Size() on the hot path, since the memory - //// lookup is cheap. - //lsm, vlog := rw.s.db.Size() - // - //// there are multiple ReadWriters writing to the same storage so add - //// the entry size and consider the new value to avoid TOCTOU issues. - //pendingSize := rw.s.pendingSize.Add(entrySize) - //rw.pendingSize += entrySize - // - //if current := pendingSize + lsm + vlog; opts.StorageLimitInBytes != 0 && current >= opts.StorageLimitInBytes { - // // flush what we currently have and discard the current entry - // if err := rw.Flush(); err != nil { - // return err - // } - // return fmt.Errorf("%w (current: %d, limit: %d)", ErrLimitReached, current, opts.StorageLimitInBytes) - //} - // - //if rw.pendingWrites >= 200 { - // // Attempt to flush if there are 200 or more uncommitted writes. - // // This ensures calls to ReadTraceEvents are not slowed down; - // // ReadTraceEvents uses an iterator, which must sort all keys - // // of uncommitted writes. - // // The 200 value yielded a good balance between read and write speed: - // // https://github.com/elastic/apm-server/pull/8407#issuecomment-1162994643 - // if err := rw.Flush(); err != nil { - // return err - // } - // - // // the current ReadWriter flushed the transaction and reset the pendingSize so add - // // the entrySize again. - // rw.pendingSize += entrySize - // rw.s.pendingSize.Add(entrySize) - //} - // - //err := rw.txn.SetEntry(e.WithTTL(opts.TTL)) - // - //// If the transaction is already too big to accommodate the new entry, flush - //// the existing transaction and set the entry on a new one, otherwise, - //// returns early. - //if err != badger.ErrTxnTooBig { - // return err - //} - //if err := rw.Flush(); err != nil { - // return err - //} - //rw.pendingSize += entrySize - //rw.s.pendingSize.Add(entrySize) - //return rw.txn.SetEntry(e.WithTTL(opts.TTL)) } // @@ -287,9 +207,6 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { rw.mu.Lock() defer rw.mu.Unlock() - if rw.batch == nil { - rw.batch = rw.s.db.NewIndexedBatch() - } iter, err := rw.batch.NewIter(&pebble.IterOptions{ LowerBound: append([]byte(traceID), ':'), UpperBound: append([]byte(traceID), ';'), @@ -304,7 +221,7 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error if err != nil { return err } - if err := rw.s.codec.DecodeEvent(data, event); err != nil { + if err := rw.s.codec.DecodeEvent(data[1:], event); err != nil { return fmt.Errorf("codec failed to decode event: %w", err) } *out = append(*out, event) From 3467415821672b5d46f8db666d63ad4010315c68 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 16:50:03 +0000 Subject: [PATCH 05/37] Fix locking, use NoSync --- x-pack/apm-server/sampling/eventstoragepebble/storage.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index ac3b8ce3e23..edffa08990e 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -98,7 +98,7 @@ func (rw *ReadWriter) Close() { // If Flush is not called before the writer is closed, then writes // may be lost. func (rw *ReadWriter) Flush() error { - err := rw.batch.Commit(pebble.Sync) + err := rw.batch.Commit(pebble.NoSync) rw.batch.Close() rw.batch = rw.s.db.NewIndexedBatch() const flushErrFmt = "failed to flush pending writes: %w" @@ -110,6 +110,8 @@ func (rw *ReadWriter) Flush() error { // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { + rw.mu.Lock() + defer rw.mu.Unlock() key := []byte(traceID) meta := entryMetaTraceUnsampled if sampled { @@ -122,6 +124,8 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts event // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { + rw.mu.Lock() + defer rw.mu.Unlock() item, closer, err := rw.batch.Get([]byte(traceID)) if err == pebble.ErrNotFound { return false, eventstorage.ErrNotFound From 886643f06e74d4499b6281486cd7c823abaaf4b3 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 17:03:40 +0000 Subject: [PATCH 06/37] Tune batch flush size, add delete --- .../sampling/eventstoragepebble/storage.go | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index edffa08990e..e4f468e0e75 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -159,7 +159,7 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { return err } - if rw.batch.Len() > 2000 { + if rw.batch.Len() > 1<<20 { if err := rw.Flush(); err != nil { return err } @@ -185,25 +185,25 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { // DeleteTraceEvent deletes the trace event from storage. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { - //var buf bytes.Buffer - //buf.Grow(len(traceID) + 1 + len(id)) - //buf.WriteString(traceID) - //buf.WriteByte(':') - //buf.WriteString(id) - //key := buf.Bytes() - // - //err := rw.txn.Delete(key) - //// If the transaction is already too big to accommodate the new entry, flush - //// the existing transaction and set the entry on a new one, otherwise, - //// returns early. - //if err != badger.ErrTxnTooBig { - // return err - //} - //if err := rw.Flush(); err != nil { - // return err - //} - // - //return rw.txn.Delete(key) + rw.mu.Lock() + defer rw.mu.Unlock() + //FIXME: use delete range + var buf bytes.Buffer + buf.Grow(len(traceID) + 1 + len(id)) + buf.WriteString(traceID) + buf.WriteByte(':') + buf.WriteString(id) + key := buf.Bytes() + + err := rw.batch.Delete(key, pebble.NoSync) + if err != nil { + return err + } + if rw.batch.Len() > 1<<20 { + if err := rw.Flush(); err != nil { + return err + } + } return nil } From 7d2618d20223743b22c69cec8ed629ede07110a8 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 17:40:49 +0000 Subject: [PATCH 07/37] Sharded --- x-pack/apm-server/main.go | 2 +- .../sampling/eventstoragepebble/sharded.go | 137 ++++++++++++++++++ .../sampling/eventstoragepebble/storage.go | 19 +-- 3 files changed, 148 insertions(+), 10 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstoragepebble/sharded.go diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 319656e15fe..82a2b1cfe38 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -202,7 +202,7 @@ func getPebbleStorage(db *pebble.DB) sampling.RW { defer storageMu.Unlock() if storage == nil { eventCodec := eventstoragepebble.ProtobufCodec{} - storage = eventstoragepebble.New(db, eventCodec).NewReadWriter() + storage = eventstoragepebble.New(db, eventCodec).NewShardedReadWriter() } return storage } diff --git a/x-pack/apm-server/sampling/eventstoragepebble/sharded.go b/x-pack/apm-server/sampling/eventstoragepebble/sharded.go new file mode 100644 index 00000000000..73dd1ea05d3 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstoragepebble/sharded.go @@ -0,0 +1,137 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License 2.0; +// you may not use this file except in compliance with the Elastic License 2.0. + +package eventstoragepebble + +import ( + "errors" + "runtime" + "sync" + + "github.com/cespare/xxhash/v2" + "github.com/elastic/apm-data/model/modelpb" + + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" +) + +// ShardedReadWriter provides sharded, locked, access to a Storage. +// +// ShardedReadWriter shards on trace ID. +type ShardedReadWriter struct { + readWriters []lockedReadWriter +} + +func newShardedReadWriter(storage *Storage) *ShardedReadWriter { + s := &ShardedReadWriter{ + // Create as many ReadWriters as there are GOMAXPROCS, which considers + // cgroup quotas, so we can ideally minimise lock contention, and scale + // up accordingly with more CPU. + readWriters: make([]lockedReadWriter, runtime.GOMAXPROCS(0)), + } + for i := range s.readWriters { + s.readWriters[i].rw = storage.NewReadWriter() + } + return s +} + +// Close closes all sharded storage readWriters. +func (s *ShardedReadWriter) Close() { + for i := range s.readWriters { + s.readWriters[i].Close() + } +} + +// Flush flushes all sharded storage readWriters. +func (s *ShardedReadWriter) Flush() error { + var errs []error + for i := range s.readWriters { + if err := s.readWriters[i].Flush(); err != nil { + errs = append(errs, err) + } + } + return errors.Join(errs...) +} + +// ReadTraceEvents calls Writer.ReadTraceEvents, using a sharded, locked, Writer. +func (s *ShardedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + return s.getWriter(traceID).ReadTraceEvents(traceID, out) +} + +// WriteTraceEvent calls Writer.WriteTraceEvent, using a sharded, locked, Writer. +func (s *ShardedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { + return s.getWriter(traceID).WriteTraceEvent(traceID, id, event, opts) +} + +// WriteTraceSampled calls Writer.WriteTraceSampled, using a sharded, locked, Writer. +func (s *ShardedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { + return s.getWriter(traceID).WriteTraceSampled(traceID, sampled, opts) +} + +// IsTraceSampled calls Writer.IsTraceSampled, using a sharded, locked, Writer. +func (s *ShardedReadWriter) IsTraceSampled(traceID string) (bool, error) { + return s.getWriter(traceID).IsTraceSampled(traceID) +} + +// DeleteTraceEvent calls Writer.DeleteTraceEvent, using a sharded, locked, Writer. +func (s *ShardedReadWriter) DeleteTraceEvent(traceID, id string) error { + return s.getWriter(traceID).DeleteTraceEvent(traceID, id) +} + +// getWriter returns an event storage writer for the given trace ID. +// +// This method is idempotent, which is necessary to avoid transaction +// conflicts and ensure all events are reported once a sampling decision +// has been recorded. +func (s *ShardedReadWriter) getWriter(traceID string) *lockedReadWriter { + var h xxhash.Digest + h.WriteString(traceID) + return &s.readWriters[h.Sum64()%uint64(len(s.readWriters))] +} + +type lockedReadWriter struct { + mu sync.Mutex + rw *ReadWriter +} + +func (rw *lockedReadWriter) Close() { + rw.mu.Lock() + defer rw.mu.Unlock() + rw.rw.Close() +} + +func (rw *lockedReadWriter) Flush() error { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.Flush() +} + +func (rw *lockedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.ReadTraceEvents(traceID, out) +} + +func (rw *lockedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.WriteTraceEvent(traceID, id, event, opts) +} + +func (rw *lockedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.WriteTraceSampled(traceID, sampled, opts) +} + +func (rw *lockedReadWriter) IsTraceSampled(traceID string) (bool, error) { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.IsTraceSampled(traceID) +} + +func (rw *lockedReadWriter) DeleteTraceEvent(traceID, id string) error { + rw.mu.Lock() + defer rw.mu.Unlock() + return rw.rw.DeleteTraceEvent(traceID, id) +} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index e4f468e0e75..2aee9a66619 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -9,13 +9,11 @@ import ( "fmt" "sync" "sync/atomic" - "time" "github.com/cockroachdb/pebble" + "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" - - "github.com/elastic/apm-data/model/modelpb" ) const ( @@ -46,6 +44,15 @@ func New(db *pebble.DB, codec Codec) *Storage { return &Storage{db: db, pendingSize: &atomic.Int64{}, codec: codec} } +// NewShardedReadWriter returns a new ShardedReadWriter, for sharded +// reading and writing. +// +// The returned ShardedReadWriter must be closed when it is no longer +// needed. +func (s *Storage) NewShardedReadWriter() *ShardedReadWriter { + return newShardedReadWriter(s) +} + // NewReadWriter returns a new ReadWriter for reading events from and // writing events to storage. // @@ -59,12 +66,6 @@ func (s *Storage) NewReadWriter() *ReadWriter { } } -// WriterOpts provides configuration options for writes to storage -type WriterOpts struct { - TTL time.Duration - StorageLimitInBytes int64 -} - // ReadWriter provides a means of reading events from storage, and batched // writing of events to storage. // From 79e32c0d2e32391167cd297d6ec08654ce06e067 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 17:51:54 +0000 Subject: [PATCH 08/37] Increase flush threshold to 5MB --- x-pack/apm-server/sampling/eventstoragepebble/storage.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go index 2aee9a66619..438181c7459 100644 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ b/x-pack/apm-server/sampling/eventstoragepebble/storage.go @@ -22,6 +22,8 @@ const ( entryMetaTraceSampled byte = 's' entryMetaTraceUnsampled byte = 'u' entryMetaTraceEvent byte = 'e' + + flushThreshold = 5 * 1024 * 1024 ) // Storage provides storage for sampled transactions and spans, @@ -160,7 +162,7 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { return err } - if rw.batch.Len() > 1<<20 { + if rw.batch.Len() > flushThreshold { if err := rw.Flush(); err != nil { return err } @@ -200,7 +202,7 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { if err != nil { return err } - if rw.batch.Len() > 1<<20 { + if rw.batch.Len() > flushThreshold { if err := rw.Flush(); err != nil { return err } From d2b1080f3d8a4dcc36c8376d8ba0af364fbab9ae Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:07:27 +0000 Subject: [PATCH 09/37] Replace badger --- x-pack/apm-server/main.go | 51 ++-- x-pack/apm-server/sampling/config.go | 7 +- .../sampling/eventstorage/badger.go | 45 ---- .../sampling/eventstorage/pebble.go | 7 + .../sampling/eventstorage/storage.go | 203 +++++---------- .../sampling/eventstorage/storage_manager.go | 18 +- .../sampling/eventstoragepebble/logger.go | 50 ---- .../eventstoragepebble/protobufcodec.go | 22 -- .../sampling/eventstoragepebble/sharded.go | 137 ---------- .../sampling/eventstoragepebble/storage.go | 237 ------------------ 10 files changed, 94 insertions(+), 683 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/badger.go create mode 100644 x-pack/apm-server/sampling/eventstorage/pebble.go delete mode 100644 x-pack/apm-server/sampling/eventstoragepebble/logger.go delete mode 100644 x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go delete mode 100644 x-pack/apm-server/sampling/eventstoragepebble/sharded.go delete mode 100644 x-pack/apm-server/sampling/eventstoragepebble/storage.go diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 085d88906fd..60e74a5f04f 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -11,11 +11,6 @@ import ( "os" "sync" - "github.com/cockroachdb/pebble" - - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstoragepebble" - - "github.com/dgraph-io/badger/v2" "github.com/gofrs/uuid/v5" "golang.org/x/sync/errgroup" @@ -45,9 +40,9 @@ var ( // will hopefully disappear in the future, when agents no longer send unsampled transactions. samplingMonitoringRegistry = monitoring.Default.GetRegistry("apm-server.sampling") - // badgerDB holds the badger database to use when tail-based sampling is configured. - badgerMu sync.Mutex - badgerDB *eventstorage.StorageManager + // db holds the database to use when tail-based sampling is configured. + dbMu sync.Mutex + db *eventstorage.StorageManager storageMu sync.Mutex storage *eventstorage.ManagedReadWriter @@ -122,11 +117,11 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er } storageDir := paths.Resolve(paths.Data, tailSamplingStorageDir) - db, err := getPebbleDB(storageDir) + db, err := getDB(storageDir) if err != nil { return nil, fmt.Errorf("failed to get Badger database: %w", err) } - readWriter := getPebbleStorage(db) + readWriter := getStorage(db) policies := make([]sampling.Policy, len(tailSamplingConfig.Policies)) for i, in := range tailSamplingConfig.Policies { @@ -160,7 +155,7 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er UUID: samplerUUID.String(), }, StorageConfig: sampling.StorageConfig{ - DB: badgerDB, + DB: db, Storage: readWriter, StorageDir: storageDir, StorageGCInterval: tailSamplingConfig.StorageGCInterval, @@ -171,17 +166,17 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er }) } -func getBadgerDB(storageDir string) (*eventstorage.StorageManager, error) { - badgerMu.Lock() - defer badgerMu.Unlock() - if badgerDB == nil { +func getDB(storageDir string) (*eventstorage.StorageManager, error) { + dbMu.Lock() + defer dbMu.Unlock() + if db == nil { sm, err := eventstorage.NewStorageManager(storageDir) if err != nil { return nil, err } - badgerDB = sm + db = sm } - return badgerDB, nil + return db, nil } func getStorage(sm *eventstorage.StorageManager) *eventstorage.ManagedReadWriter { @@ -193,20 +188,6 @@ func getStorage(sm *eventstorage.StorageManager) *eventstorage.ManagedReadWriter return storage } -func getPebbleDB(storageDir string) (*pebble.DB, error) { - return pebble.Open(storageDir, &pebble.Options{}) -} - -func getPebbleStorage(db *pebble.DB) sampling.RW { - storageMu.Lock() - defer storageMu.Unlock() - if storage == nil { - eventCodec := eventstoragepebble.ProtobufCodec{} - storage = eventstoragepebble.New(db, eventCodec).NewShardedReadWriter() - } - return storage -} - // runServerWithProcessors runs the APM Server and the given list of processors. // // newProcessors returns a list of processors which will process events in @@ -270,11 +251,11 @@ func wrapServer(args beater.ServerParams, runServer beater.RunServerFunc) (beate // closeBadger is called at process exit time to close the badger.DB opened // by the tail-based sampling processor constructor, if any. This is never -// called concurrently with opening badger.DB/accessing the badgerDB global, -// so it does not need to hold badgerMu. +// called concurrently with opening badger.DB/accessing the db global, +// so it does not need to hold dbMu. func closeBadger() error { - if badgerDB != nil { - return badgerDB.Close() + if db != nil { + return db.Close() } return nil } diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 8e0e2f7cad2..b6d4d6ce252 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -10,6 +10,7 @@ import ( "github.com/pkg/errors" "github.com/elastic/apm-data/model/modelpb" + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" "github.com/elastic/apm-server/x-pack/apm-server/sampling/pubsub" "github.com/elastic/go-elasticsearch/v8" ) @@ -238,9 +239,9 @@ func (config DataStreamConfig) validate() error { } func (config StorageConfig) validate() error { - //if config.DB == nil { - // return errors.New("DB unspecified") - //} + if config.DB == nil { + return errors.New("DB unspecified") + } if config.Storage == nil { return errors.New("Storage unspecified") } diff --git a/x-pack/apm-server/sampling/eventstorage/badger.go b/x-pack/apm-server/sampling/eventstorage/badger.go deleted file mode 100644 index 99219262d4f..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/badger.go +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstorage - -import ( - "github.com/dgraph-io/badger/v2" - - "github.com/elastic/apm-server/internal/logs" - "github.com/elastic/elastic-agent-libs/logp" -) - -const ( - defaultValueLogFileSize = 64 * 1024 * 1024 -) - -// OpenBadger creates or opens a Badger database with the specified location -// and value log file size. If the value log file size is <= 0, the default -// of 64MB will be used. -// -// NOTE(axw) only one badger.DB for a given storage directory may be open at any given time. -func OpenBadger(storageDir string, valueLogFileSize int64) (*badger.DB, error) { - logger := logp.NewLogger(logs.Sampling) - // Tunable memory options: - // - NumMemtables - default 5 in-mem tables (MaxTableSize default) - // - NumLevelZeroTables - default 5 - number of L0 tables before compaction starts. - // - NumLevelZeroTablesStall - number of L0 tables before writing stalls (waiting for compaction). - // - IndexCacheSize - default all in mem, Each table has its own bloom filter and each bloom filter is approximately of 5 MB. - // - MaxTableSize - Default 64MB - if valueLogFileSize <= 0 { - valueLogFileSize = defaultValueLogFileSize - } - const tableLimit = 4 - badgerOpts := badger.DefaultOptions(storageDir). - WithLogger(&LogpAdaptor{Logger: logger}). - WithTruncate(true). // Truncate unreadable files which cannot be read. - WithNumMemtables(tableLimit). // in-memory tables. - WithNumLevelZeroTables(tableLimit). // L0 tables. - WithNumLevelZeroTablesStall(tableLimit * 3). // Maintain the default 1-to-3 ratio before stalling. - WithMaxTableSize(int64(16 << 20)). // Max LSM table or file size. - WithValueLogFileSize(valueLogFileSize) // vlog file size. - - return badger.Open(badgerOpts) -} diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go new file mode 100644 index 00000000000..c5099f023e5 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -0,0 +1,7 @@ +package eventstorage + +import "github.com/cockroachdb/pebble" + +func OpenPebble(storageDir string) (*pebble.DB, error) { + return pebble.Open(storageDir, &pebble.Options{}) +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index c11c89d647e..73c15c19168 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -11,21 +11,18 @@ import ( "sync/atomic" "time" - "github.com/dgraph-io/badger/v2" - + "github.com/cockroachdb/pebble" "github.com/elastic/apm-data/model/modelpb" ) const ( // NOTE(axw) these values (and their meanings) must remain stable // over time, to avoid misinterpreting historical data. - entryMetaTraceSampled = 's' - entryMetaTraceUnsampled = 'u' - entryMetaTraceEvent = 'e' + entryMetaTraceSampled byte = 's' + entryMetaTraceUnsampled byte = 'u' + entryMetaTraceEvent byte = 'e' - // Initial transaction size - // len(txnKey) + 10 - baseTransactionSize = 10 + 11 + flushThreshold = 5 * 1024 * 1024 ) var ( @@ -39,7 +36,7 @@ var ( ) type db interface { - NewTransaction(update bool) *badger.Txn + NewIndexedBatch() *pebble.Batch Size() (lsm, vlog int64) Close() error } @@ -78,11 +75,11 @@ func (s *Storage) NewShardedReadWriter() *ShardedReadWriter { // // The returned ReadWriter must be closed when it is no longer needed. func (s *Storage) NewReadWriter() *ReadWriter { - s.pendingSize.Add(baseTransactionSize) + //s.pendingSize.Add(baseTransactionSize) return &ReadWriter{ - s: s, - txn: nil, // lazy init to avoid deadlock in storage manager - pendingSize: baseTransactionSize, + s: s, + //txn: nil, // lazy init to avoid deadlock in storage manager + //pendingSize: baseTransactionSize, } } @@ -100,8 +97,8 @@ type WriterOpts struct { // avoid conflicts, e.g. by using consistent hashing to distribute to one of // a set of ReadWriters, such as implemented by ShardedReadWriter. type ReadWriter struct { - s *Storage - txn *badger.Txn + s *Storage + batch *pebble.Batch // readKeyBuf is a reusable buffer for keys used in read operations. // This must not be used in write operations, as keys are expected to @@ -113,8 +110,8 @@ type ReadWriter struct { } func (rw *ReadWriter) lazyInit() { - if rw.txn == nil { - rw.txn = rw.s.db.NewTransaction(true) + if rw.batch == nil { + rw.batch = rw.s.db.NewIndexedBatch() } } @@ -123,8 +120,8 @@ func (rw *ReadWriter) lazyInit() { // This must be called when the writer is no longer needed, in order to reclaim // resources. func (rw *ReadWriter) Close() { - if rw.txn != nil { - rw.txn.Discard() + if rw.batch != nil { + rw.batch.Close() } } @@ -137,12 +134,13 @@ func (rw *ReadWriter) Flush() error { rw.lazyInit() const flushErrFmt = "failed to flush pending writes: %w" - err := rw.txn.Commit() - rw.txn = rw.s.db.NewTransaction(true) - rw.s.pendingSize.Add(-rw.pendingSize) - rw.pendingWrites = 0 - rw.pendingSize = baseTransactionSize - rw.s.pendingSize.Add(baseTransactionSize) + err := rw.batch.Commit(pebble.NoSync) + rw.batch.Close() + //rw.txn = rw.s.db.NewTransaction(true) + //rw.s.pendingSize.Add(-rw.pendingSize) + //rw.pendingWrites = 0 + //rw.pendingSize = baseTransactionSize + //rw.s.pendingSize.Add(baseTransactionSize) if err != nil { return fmt.Errorf(flushErrFmt, err) } @@ -154,11 +152,11 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write rw.lazyInit() key := []byte(traceID) - var meta uint8 = entryMetaTraceUnsampled + meta := entryMetaTraceUnsampled if sampled { meta = entryMetaTraceSampled } - return rw.writeEntry(badger.NewEntry(key[:], nil).WithMeta(meta), opts) + return rw.batch.Set(key, []byte{meta}, pebble.NoSync) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -167,15 +165,12 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { rw.lazyInit() - rw.readKeyBuf = append(rw.readKeyBuf[:0], traceID...) - item, err := rw.txn.Get(rw.readKeyBuf) - if err != nil { - if err == badger.ErrKeyNotFound { - return false, ErrNotFound - } - return false, err + item, closer, err := rw.batch.Get([]byte(traceID)) + if err == pebble.ErrNotFound { + return false, ErrNotFound } - return item.UserMeta() == entryMetaTraceSampled, nil + defer closer.Close() + return item[0] == entryMetaTraceSampled, nil } // WriteTraceEvent writes a trace event to storage. @@ -195,77 +190,20 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. buf.WriteByte(':') buf.WriteString(id) key := buf.Bytes() - return rw.writeEntry(badger.NewEntry(key, data).WithMeta(entryMetaTraceEvent), opts) + return rw.writeEntry(key, data) } -func (rw *ReadWriter) writeEntry(e *badger.Entry, opts WriterOpts) error { - rw.pendingWrites++ - entrySize := estimateSize(e) - // The badger database has an async size reconciliation, with a 1 minute - // ticker that keeps the lsm and vlog sizes updated in an in-memory map. - // It's OK to call call s.db.Size() on the hot path, since the memory - // lookup is cheap. - lsm, vlog := rw.s.db.Size() - - // there are multiple ReadWriters writing to the same storage so add - // the entry size and consider the new value to avoid TOCTOU issues. - pendingSize := rw.s.pendingSize.Add(entrySize) - rw.pendingSize += entrySize - - if current := pendingSize + lsm + vlog; opts.StorageLimitInBytes != 0 && current >= opts.StorageLimitInBytes { - // flush what we currently have and discard the current entry - if err := rw.Flush(); err != nil { - return err - } - return fmt.Errorf("%w (current: %d, limit: %d)", ErrLimitReached, current, opts.StorageLimitInBytes) +func (rw *ReadWriter) writeEntry(key, data []byte) error { + if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { + return err } - if rw.pendingWrites >= 200 { - // Attempt to flush if there are 200 or more uncommitted writes. - // This ensures calls to ReadTraceEvents are not slowed down; - // ReadTraceEvents uses an iterator, which must sort all keys - // of uncommitted writes. - // The 200 value yielded a good balance between read and write speed: - // https://github.com/elastic/apm-server/pull/8407#issuecomment-1162994643 + if rw.batch.Len() > flushThreshold { if err := rw.Flush(); err != nil { return err } - - // the current ReadWriter flushed the transaction and reset the pendingSize so add - // the entrySize again. - rw.pendingSize += entrySize - rw.s.pendingSize.Add(entrySize) - } - - err := rw.txn.SetEntry(e.WithTTL(opts.TTL)) - - // If the transaction is already too big to accommodate the new entry, flush - // the existing transaction and set the entry on a new one, otherwise, - // returns early. - if err != badger.ErrTxnTooBig { - return err - } - if err := rw.Flush(); err != nil { - return err } - rw.pendingSize += entrySize - rw.s.pendingSize.Add(entrySize) - return rw.txn.SetEntry(e.WithTTL(opts.TTL)) -} - -func estimateSize(e *badger.Entry) int64 { - // See badger WithValueThreshold option - // An storage usage of an entry depends on its size - // - // if len(e.Value) < threshold { - // return len(e.Key) + len(e.Value) + 2 // Meta, UserMeta - // } - // return len(e.Key) + 12 + 2 // 12 for ValuePointer, 2 for metas. - // - // Make a good estimate by reserving more space - estimate := len(e.Key) + len(e.Value) + 12 + 2 - // Extra bytes for the version in key. - return int64(estimate) + 10 + return nil } // DeleteTraceEvent deletes the trace event from storage. @@ -279,67 +217,40 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { buf.WriteString(id) key := buf.Bytes() - err := rw.txn.Delete(key) - // If the transaction is already too big to accommodate the new entry, flush - // the existing transaction and set the entry on a new one, otherwise, - // returns early. - if err != badger.ErrTxnTooBig { + err := rw.batch.Delete(key, pebble.NoSync) + if err != nil { return err } - if err := rw.Flush(); err != nil { - return err + if rw.batch.Len() > flushThreshold { + if err := rw.Flush(); err != nil { + return err + } } - - return rw.txn.Delete(key) + return nil } // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { rw.lazyInit() - opts := badger.DefaultIteratorOptions - rw.readKeyBuf = append(append(rw.readKeyBuf[:0], traceID...), ':') - opts.Prefix = rw.readKeyBuf - - // 1st pass: check whether there exist keys matching the prefix. - // Do not prefetch values so that the check is done in-memory. - // This is to optimize for cases when it is a miss. - opts.PrefetchValues = false - iter := rw.txn.NewIterator(opts) - iter.Rewind() - if !iter.Valid() { - iter.Close() - return nil + iter, err := rw.batch.NewIter(&pebble.IterOptions{ + LowerBound: append([]byte(traceID), ':'), + UpperBound: append([]byte(traceID), ';'), + }) + if err != nil { + return err } - iter.Close() - - // 2nd pass: this is only done when there exist keys matching the prefix. - // Fetch the events with PrefetchValues for performance. - // This is to optimize for cases when it is a hit. - opts.PrefetchValues = true - iter = rw.txn.NewIterator(opts) defer iter.Close() - for iter.Rewind(); iter.Valid(); iter.Next() { - item := iter.Item() - if item.IsDeletedOrExpired() { - continue + for iter.First(); iter.Valid(); iter.Next() { + event := &modelpb.APMEvent{} + data, err := iter.ValueAndErr() + if err != nil { + return err } - switch item.UserMeta() { - case entryMetaTraceEvent: - event := &modelpb.APMEvent{} - if err := item.Value(func(data []byte) error { - if err := rw.s.codec.DecodeEvent(data, event); err != nil { - return fmt.Errorf("codec failed to decode event: %w", err) - } - return nil - }); err != nil { - return err - } - *out = append(*out, event) - default: - // Unknown entry meta: ignore. - continue + if err := rw.s.codec.DecodeEvent(data[1:], event); err != nil { + return fmt.Errorf("codec failed to decode event: %w", err) } + *out = append(*out, event) } return nil } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 8446f8fbf89..73e0df9458b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -14,6 +14,7 @@ import ( "sync" "time" + "github.com/cockroachdb/pebble" "github.com/dgraph-io/badger/v2" "golang.org/x/sync/errgroup" @@ -32,14 +33,14 @@ var ( errDropAndRecreateInProgress = errors.New("db drop and recreate in progress") ) -// StorageManager encapsulates badger.DB. +// StorageManager encapsulates pebble.DB. // It is to provide file system access, simplify synchronization and enable underlying db swaps. -// It assumes exclusive access to badger DB at storageDir. +// It assumes exclusive access to pebble DB at storageDir. type StorageManager struct { storageDir string logger *logp.Logger - db *badger.DB + db *pebble.DB storage *Storage rw *ShardedReadWriter @@ -53,7 +54,7 @@ type StorageManager struct { runCh chan struct{} } -// NewStorageManager returns a new StorageManager with badger DB at storageDir. +// NewStorageManager returns a new StorageManager with pebble DB at storageDir. func NewStorageManager(storageDir string) (*StorageManager, error) { sm := &StorageManager{ storageDir: storageDir, @@ -69,7 +70,7 @@ func NewStorageManager(storageDir string) (*StorageManager, error) { // reset initializes db, storage, and rw. func (s *StorageManager) reset() error { - db, err := OpenBadger(s.storageDir, -1) + db, err := OpenPebble(s.storageDir) if err != nil { return err } @@ -91,13 +92,14 @@ func (s *StorageManager) Close() error { func (s *StorageManager) Size() (lsm, vlog int64) { s.mu.RLock() defer s.mu.RUnlock() - return s.db.Size() + return 0, 0 + //return s.db.Size() } -func (s *StorageManager) NewTransaction(update bool) *badger.Txn { +func (s *StorageManager) NewIndexedBatch() *pebble.Batch { s.mu.RLock() defer s.mu.RUnlock() - return s.db.NewTransaction(update) + return s.db.NewIndexedBatch() } // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. diff --git a/x-pack/apm-server/sampling/eventstoragepebble/logger.go b/x-pack/apm-server/sampling/eventstoragepebble/logger.go deleted file mode 100644 index fb9f2d69756..00000000000 --- a/x-pack/apm-server/sampling/eventstoragepebble/logger.go +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstoragepebble - -import ( - "fmt" - "sync" - - "github.com/elastic/elastic-agent-libs/logp" -) - -// LogpAdaptor adapts logp.Logger to the badger.Logger interface. -type LogpAdaptor struct { - *logp.Logger - - mu sync.RWMutex - last string -} - -// Errorf prints the log message when the current message isn't the same as the -// previously logged message. -func (a *LogpAdaptor) Errorf(format string, args ...interface{}) { - msg := fmt.Sprintf(format, args...) - if a.setLast(msg) { - a.Logger.Errorf(format, args...) - } -} - -func (a *LogpAdaptor) setLast(msg string) bool { - a.mu.RLock() - if msg != a.last { - a.mu.RUnlock() - return false - } - a.mu.RUnlock() - a.mu.Lock() - defer a.mu.Unlock() - shouldSet := msg != a.last - if shouldSet { - a.last = msg - } - return shouldSet -} - -// Warningf adapts badger.Logger.Warningf to logp.Logger.Warngf. -func (a *LogpAdaptor) Warningf(format string, args ...interface{}) { - a.Warnf(format, args...) -} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go b/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go deleted file mode 100644 index 40862ae8f23..00000000000 --- a/x-pack/apm-server/sampling/eventstoragepebble/protobufcodec.go +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstoragepebble - -import ( - "github.com/elastic/apm-data/model/modelpb" -) - -// ProtobufCodec is an implementation of Codec, using protobuf encoding. -type ProtobufCodec struct{} - -// DecodeEvent decodes data as protobuf into event. -func (ProtobufCodec) DecodeEvent(data []byte, event *modelpb.APMEvent) error { - return event.UnmarshalVT(data) -} - -// EncodeEvent encodes event as protobuf. -func (ProtobufCodec) EncodeEvent(event *modelpb.APMEvent) ([]byte, error) { - return event.MarshalVT() -} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/sharded.go b/x-pack/apm-server/sampling/eventstoragepebble/sharded.go deleted file mode 100644 index 73dd1ea05d3..00000000000 --- a/x-pack/apm-server/sampling/eventstoragepebble/sharded.go +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstoragepebble - -import ( - "errors" - "runtime" - "sync" - - "github.com/cespare/xxhash/v2" - "github.com/elastic/apm-data/model/modelpb" - - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" -) - -// ShardedReadWriter provides sharded, locked, access to a Storage. -// -// ShardedReadWriter shards on trace ID. -type ShardedReadWriter struct { - readWriters []lockedReadWriter -} - -func newShardedReadWriter(storage *Storage) *ShardedReadWriter { - s := &ShardedReadWriter{ - // Create as many ReadWriters as there are GOMAXPROCS, which considers - // cgroup quotas, so we can ideally minimise lock contention, and scale - // up accordingly with more CPU. - readWriters: make([]lockedReadWriter, runtime.GOMAXPROCS(0)), - } - for i := range s.readWriters { - s.readWriters[i].rw = storage.NewReadWriter() - } - return s -} - -// Close closes all sharded storage readWriters. -func (s *ShardedReadWriter) Close() { - for i := range s.readWriters { - s.readWriters[i].Close() - } -} - -// Flush flushes all sharded storage readWriters. -func (s *ShardedReadWriter) Flush() error { - var errs []error - for i := range s.readWriters { - if err := s.readWriters[i].Flush(); err != nil { - errs = append(errs, err) - } - } - return errors.Join(errs...) -} - -// ReadTraceEvents calls Writer.ReadTraceEvents, using a sharded, locked, Writer. -func (s *ShardedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - return s.getWriter(traceID).ReadTraceEvents(traceID, out) -} - -// WriteTraceEvent calls Writer.WriteTraceEvent, using a sharded, locked, Writer. -func (s *ShardedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { - return s.getWriter(traceID).WriteTraceEvent(traceID, id, event, opts) -} - -// WriteTraceSampled calls Writer.WriteTraceSampled, using a sharded, locked, Writer. -func (s *ShardedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { - return s.getWriter(traceID).WriteTraceSampled(traceID, sampled, opts) -} - -// IsTraceSampled calls Writer.IsTraceSampled, using a sharded, locked, Writer. -func (s *ShardedReadWriter) IsTraceSampled(traceID string) (bool, error) { - return s.getWriter(traceID).IsTraceSampled(traceID) -} - -// DeleteTraceEvent calls Writer.DeleteTraceEvent, using a sharded, locked, Writer. -func (s *ShardedReadWriter) DeleteTraceEvent(traceID, id string) error { - return s.getWriter(traceID).DeleteTraceEvent(traceID, id) -} - -// getWriter returns an event storage writer for the given trace ID. -// -// This method is idempotent, which is necessary to avoid transaction -// conflicts and ensure all events are reported once a sampling decision -// has been recorded. -func (s *ShardedReadWriter) getWriter(traceID string) *lockedReadWriter { - var h xxhash.Digest - h.WriteString(traceID) - return &s.readWriters[h.Sum64()%uint64(len(s.readWriters))] -} - -type lockedReadWriter struct { - mu sync.Mutex - rw *ReadWriter -} - -func (rw *lockedReadWriter) Close() { - rw.mu.Lock() - defer rw.mu.Unlock() - rw.rw.Close() -} - -func (rw *lockedReadWriter) Flush() error { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.Flush() -} - -func (rw *lockedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.ReadTraceEvents(traceID, out) -} - -func (rw *lockedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.WriteTraceEvent(traceID, id, event, opts) -} - -func (rw *lockedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.WriteTraceSampled(traceID, sampled, opts) -} - -func (rw *lockedReadWriter) IsTraceSampled(traceID string) (bool, error) { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.IsTraceSampled(traceID) -} - -func (rw *lockedReadWriter) DeleteTraceEvent(traceID, id string) error { - rw.mu.Lock() - defer rw.mu.Unlock() - return rw.rw.DeleteTraceEvent(traceID, id) -} diff --git a/x-pack/apm-server/sampling/eventstoragepebble/storage.go b/x-pack/apm-server/sampling/eventstoragepebble/storage.go deleted file mode 100644 index 438181c7459..00000000000 --- a/x-pack/apm-server/sampling/eventstoragepebble/storage.go +++ /dev/null @@ -1,237 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstoragepebble - -import ( - "bytes" - "fmt" - "sync" - "sync/atomic" - - "github.com/cockroachdb/pebble" - "github.com/elastic/apm-data/model/modelpb" - - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" -) - -const ( - // NOTE(axw) these values (and their meanings) must remain stable - // over time, to avoid misinterpreting historical data. - entryMetaTraceSampled byte = 's' - entryMetaTraceUnsampled byte = 'u' - entryMetaTraceEvent byte = 'e' - - flushThreshold = 5 * 1024 * 1024 -) - -// Storage provides storage for sampled transactions and spans, -// and for recording trace sampling decisions. -type Storage struct { - db *pebble.DB - // pendingSize tracks the total size of pending writes across ReadWriters - pendingSize *atomic.Int64 - codec Codec -} - -// Codec provides methods for encoding and decoding events. -type Codec interface { - DecodeEvent([]byte, *modelpb.APMEvent) error - EncodeEvent(*modelpb.APMEvent) ([]byte, error) -} - -// New returns a new Storage using db and codec. -func New(db *pebble.DB, codec Codec) *Storage { - return &Storage{db: db, pendingSize: &atomic.Int64{}, codec: codec} -} - -// NewShardedReadWriter returns a new ShardedReadWriter, for sharded -// reading and writing. -// -// The returned ShardedReadWriter must be closed when it is no longer -// needed. -func (s *Storage) NewShardedReadWriter() *ShardedReadWriter { - return newShardedReadWriter(s) -} - -// NewReadWriter returns a new ReadWriter for reading events from and -// writing events to storage. -// -// The returned ReadWriter must be closed when it is no longer needed. -func (s *Storage) NewReadWriter() *ReadWriter { - //s.pendingSize.Add(baseTransactionSize) - return &ReadWriter{ - s: s, - batch: s.db.NewIndexedBatch(), - //pendingSize: baseTransactionSize, - } -} - -// ReadWriter provides a means of reading events from storage, and batched -// writing of events to storage. -// -// ReadWriter is not safe for concurrent access. All operations that involve -// a given trace ID should be performed with the same ReadWriter in order to -// avoid conflicts, e.g. by using consistent hashing to distribute to one of -// a set of ReadWriters, such as implemented by ShardedReadWriter. -type ReadWriter struct { - s *Storage - - // readKeyBuf is a reusable buffer for keys used in read operations. - // This must not be used in write operations, as keys are expected to - // be unmodified until the end of a transaction. - readKeyBuf []byte - - mu sync.Mutex - batch *pebble.Batch -} - -// Close closes the writer. Any writes that have not been flushed may be lost. -// -// This must be called when the writer is no longer needed, in order to reclaim -// resources. -func (rw *ReadWriter) Close() { - //rw.txn.Discard() -} - -// Flush waits for preceding writes to be committed to storage. -// -// Flush must be called to ensure writes are committed to storage. -// If Flush is not called before the writer is closed, then writes -// may be lost. -func (rw *ReadWriter) Flush() error { - err := rw.batch.Commit(pebble.NoSync) - rw.batch.Close() - rw.batch = rw.s.db.NewIndexedBatch() - const flushErrFmt = "failed to flush pending writes: %w" - if err != nil { - return fmt.Errorf(flushErrFmt, err) - } - return nil -} - -// WriteTraceSampled records the tail-sampling decision for the given trace ID. -func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { - rw.mu.Lock() - defer rw.mu.Unlock() - key := []byte(traceID) - meta := entryMetaTraceUnsampled - if sampled { - meta = entryMetaTraceSampled - } - return rw.batch.Set(key, []byte{meta}, pebble.NoSync) -} - -// IsTraceSampled reports whether traceID belongs to a trace that is sampled -// or unsampled. If no sampling decision has been recorded, IsTraceSampled -// returns ErrNotFound. -func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - rw.mu.Lock() - defer rw.mu.Unlock() - item, closer, err := rw.batch.Get([]byte(traceID)) - if err == pebble.ErrNotFound { - return false, eventstorage.ErrNotFound - } - defer closer.Close() - return item[0] == entryMetaTraceSampled, nil -} - -// WriteTraceEvent writes a trace event to storage. -// -// WriteTraceEvent may return before the write is committed to storage. -// Call Flush to ensure the write is committed. -func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { - data, err := rw.s.codec.EncodeEvent(event) - if err != nil { - return err - } - var buf bytes.Buffer - buf.Grow(len(traceID) + 1 + len(id)) - buf.WriteString(traceID) - buf.WriteByte(':') - buf.WriteString(id) - key := buf.Bytes() - return rw.writeEntry(key, data) -} - -func (rw *ReadWriter) writeEntry(key, data []byte) error { - rw.mu.Lock() - defer rw.mu.Unlock() - if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { - return err - } - - if rw.batch.Len() > flushThreshold { - if err := rw.Flush(); err != nil { - return err - } - } - return nil -} - -// -//func estimateSize(e *badger.Entry) int64 { -// // See badger WithValueThreshold option -// // An storage usage of an entry depends on its size -// // -// // if len(e.Value) < threshold { -// // return len(e.Key) + len(e.Value) + 2 // Meta, UserMeta -// // } -// // return len(e.Key) + 12 + 2 // 12 for ValuePointer, 2 for metas. -// // -// // Make a good estimate by reserving more space -// estimate := len(e.Key) + len(e.Value) + 12 + 2 -// // Extra bytes for the version in key. -// return int64(estimate) + 10 -//} - -// DeleteTraceEvent deletes the trace event from storage. -func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { - rw.mu.Lock() - defer rw.mu.Unlock() - //FIXME: use delete range - var buf bytes.Buffer - buf.Grow(len(traceID) + 1 + len(id)) - buf.WriteString(traceID) - buf.WriteByte(':') - buf.WriteString(id) - key := buf.Bytes() - - err := rw.batch.Delete(key, pebble.NoSync) - if err != nil { - return err - } - if rw.batch.Len() > flushThreshold { - if err := rw.Flush(); err != nil { - return err - } - } - return nil -} - -// ReadTraceEvents reads trace events with the given trace ID from storage into out. -func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - rw.mu.Lock() - defer rw.mu.Unlock() - iter, err := rw.batch.NewIter(&pebble.IterOptions{ - LowerBound: append([]byte(traceID), ':'), - UpperBound: append([]byte(traceID), ';'), - }) - if err != nil { - return err - } - defer iter.Close() - for iter.First(); iter.Valid(); iter.Next() { - event := &modelpb.APMEvent{} - data, err := iter.ValueAndErr() - if err != nil { - return err - } - if err := rw.s.codec.DecodeEvent(data[1:], event); err != nil { - return fmt.Errorf("codec failed to decode event: %w", err) - } - *out = append(*out, event) - } - return nil -} From 640ee0c2c8a27971c866a98ccca231354ab213b6 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:08:54 +0000 Subject: [PATCH 10/37] Remove gc loop and drop loop --- .../sampling/eventstorage/storage_manager.go | 167 ------------------ 1 file changed, 167 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 73e0df9458b..5f34278df1d 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -6,18 +6,12 @@ package eventstorage import ( "errors" - "fmt" - "io/fs" "os" "path/filepath" - "strings" "sync" "time" "github.com/cockroachdb/pebble" - "github.com/dgraph-io/badger/v2" - "golang.org/x/sync/errgroup" - "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" @@ -104,170 +98,9 @@ func (s *StorageManager) NewIndexedBatch() *pebble.Batch { // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. func (s *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration, ttl time.Duration, storageLimit uint64, storageLimitThreshold float64) error { - select { - case <-stopping: - return nil - case s.runCh <- struct{}{}: - } - defer func() { - <-s.runCh - }() - - g := errgroup.Group{} - g.Go(func() error { - return s.runGCLoop(stopping, gcInterval) - }) - g.Go(func() error { - return s.runDropLoop(stopping, ttl, storageLimit, storageLimitThreshold) - }) - return g.Wait() -} - -// runGCLoop runs a loop that calls badger DB RunValueLogGC every gcInterval. -func (s *StorageManager) runGCLoop(stopping <-chan struct{}, gcInterval time.Duration) error { - // This goroutine is responsible for periodically garbage - // collecting the Badger value log, using the recommended - // discard ratio of 0.5. - ticker := time.NewTicker(gcInterval) - defer ticker.Stop() - for { - select { - case <-stopping: - return nil - case <-ticker.C: - const discardRatio = 0.5 - var err error - for err == nil { - // Keep garbage collecting until there are no more rewrites, - // or garbage collection fails. - err = s.runValueLogGC(discardRatio) - } - if err != nil && err != badger.ErrNoRewrite { - return err - } - } - } -} - -func (s *StorageManager) runValueLogGC(discardRatio float64) error { - s.mu.RLock() - defer s.mu.RUnlock() - return s.db.RunValueLogGC(discardRatio) -} - -// runDropLoop runs a loop that detects if storage limit has been exceeded for at least ttl. -// If so, it drops and recreates the underlying badger DB. -// This is a mitigation for issue https://github.com/elastic/apm-server/issues/14923 -func (s *StorageManager) runDropLoop(stopping <-chan struct{}, ttl time.Duration, storageLimitInBytes uint64, storageLimitThreshold float64) error { - if storageLimitInBytes == 0 { - return nil - } - - var firstExceeded time.Time - checkAndFix := func() error { - lsm, vlog := s.Size() - // add buffer to avoid edge case storageLimitInBytes-lsm-vlog < buffer, when writes are still always rejected - buffer := int64(baseTransactionSize * len(s.rw.readWriters)) - if uint64(lsm+vlog+buffer) >= storageLimitInBytes { - now := time.Now() - if firstExceeded.IsZero() { - firstExceeded = now - s.logger.Warnf( - "badger db size (%d+%d=%d) has exceeded storage limit (%d*%.1f=%d); db will be dropped and recreated if problem persists for `sampling.tail.ttl` (%s)", - lsm, vlog, lsm+vlog, storageLimitInBytes, storageLimitThreshold, int64(float64(storageLimitInBytes)*storageLimitThreshold), ttl.String()) - } - if now.Sub(firstExceeded) >= ttl { - s.logger.Warnf("badger db size has exceeded storage limit for over `sampling.tail.ttl` (%s), please consider increasing `sampling.tail.storage_limit`; dropping and recreating badger db to recover", ttl.String()) - err := s.dropAndRecreate() - if err != nil { - s.logger.With(logp.Error(err)).Error("error dropping and recreating badger db to recover storage space") - } else { - s.logger.Info("badger db dropped and recreated") - } - firstExceeded = time.Time{} - } - } else { - firstExceeded = time.Time{} - } - return nil - } - - timer := time.NewTicker(time.Minute) // Eval db size every minute as badger reports them with 1m lag - defer timer.Stop() - for { - if err := checkAndFix(); err != nil { - return err - } - - select { - case <-stopping: - return nil - case <-timer.C: - continue - } - } -} - -// dropAndRecreate deletes the underlying badger DB files at the file system level, and replaces it with a new badger DB. -func (s *StorageManager) dropAndRecreate() (retErr error) { - s.mu.Lock() - defer s.mu.Unlock() - - defer func() { - // In any case (errors or not), reset StorageManager while lock is held - err := s.reset() - if err != nil { - retErr = errors.Join(retErr, fmt.Errorf("error reopening badger db: %w", err)) - } - }() - - // Intentionally not flush rw, as storage is full. - s.rw.Close() - err := s.db.Close() - if err != nil { - return fmt.Errorf("error closing badger db: %w", err) - } - - err = s.deleteBadgerFiles() - if err != nil { - return fmt.Errorf("error deleting badger db files: %w", err) - } - return nil } -func (s *StorageManager) deleteBadgerFiles() error { - // Although removing the files in place can be slower, it is less error-prone than rename-and-delete. - // Delete every file except subscriber position file - var ( - rootVisited bool - sstFiles, vlogFiles int - otherFilenames []string - ) - err := filepath.WalkDir(s.storageDir, func(path string, d fs.DirEntry, _ error) error { - if !rootVisited { - rootVisited = true - return nil - } - filename := filepath.Base(path) - if filename == subscriberPositionFile { - return nil - } - switch ext := filepath.Ext(filename); ext { - case ".sst": - sstFiles++ - case ".vlog": - vlogFiles++ - default: - otherFilenames = append(otherFilenames, filename) - } - return os.RemoveAll(path) - }) - s.logger.Infof("deleted badger files: %d SST files, %d VLOG files, %d other files: [%s]", - sstFiles, vlogFiles, len(otherFilenames), strings.Join(otherFilenames, ", ")) - return err -} - func (s *StorageManager) ReadSubscriberPosition() ([]byte, error) { s.subscriberPosMu.Lock() defer s.subscriberPosMu.Unlock() From 85963176dfbb73c3c24c80bfca896677ab9c4155 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:18:39 +0000 Subject: [PATCH 11/37] Fix test --- .../sampling/eventstorage/storage.go | 2 +- .../eventstorage/storage_manager_test.go | 70 --------------- x-pack/apm-server/sampling/processor_test.go | 87 +------------------ 3 files changed, 5 insertions(+), 154 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/storage_manager_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 73c15c19168..1526637c19b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -136,7 +136,7 @@ func (rw *ReadWriter) Flush() error { const flushErrFmt = "failed to flush pending writes: %w" err := rw.batch.Commit(pebble.NoSync) rw.batch.Close() - //rw.txn = rw.s.db.NewTransaction(true) + rw.batch = rw.s.db.NewIndexedBatch() //rw.s.pendingSize.Add(-rw.pendingSize) //rw.pendingWrites = 0 //rw.pendingSize = baseTransactionSize diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go deleted file mode 100644 index 3dccac1ed38..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstorage - -import ( - "fmt" - "os" - "path/filepath" - "testing" - "time" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func badgerModTime(dir string) time.Time { - oldest := time.Now() - filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { - ext := filepath.Ext(path) - if (ext == ".vlog" || ext == ".sst") && info.ModTime().Before(oldest) { - oldest = info.ModTime() - } - return nil - }) - return oldest -} - -func TestDropAndRecreate_filesRecreated(t *testing.T) { - tempDir := t.TempDir() - sm, err := NewStorageManager(tempDir) - require.NoError(t, err) - defer sm.Close() - - oldModTime := badgerModTime(tempDir) - - err = sm.dropAndRecreate() - assert.NoError(t, err) - - newModTime := badgerModTime(tempDir) - - assert.Greater(t, newModTime, oldModTime) -} - -func TestDropAndRecreate_subscriberPositionFile(t *testing.T) { - for _, exists := range []bool{true, false} { - t.Run(fmt.Sprintf("exists=%t", exists), func(t *testing.T) { - tempDir := t.TempDir() - sm, err := NewStorageManager(tempDir) - require.NoError(t, err) - defer sm.Close() - - if exists { - err := sm.WriteSubscriberPosition([]byte("{}")) - require.NoError(t, err) - } - - err = sm.dropAndRecreate() - assert.NoError(t, err) - - data, err := sm.ReadSubscriberPosition() - if exists { - assert.Equal(t, "{}", string(data)) - } else { - assert.ErrorIs(t, err, os.ErrNotExist) - } - }) - } -} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 5a301e208cf..efb0c5e551c 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -646,85 +646,6 @@ func TestStorageMonitoring(t *testing.T) { assert.NotZero(t, metrics.Ints, "sampling.storage.value_log_size") } -func TestStorageGC(t *testing.T) { - if testing.Short() { - t.Skip("skipping slow test") - } - - config := newTempdirConfig(t) - config.TTL = 10 * time.Millisecond - config.FlushInterval = 10 * time.Millisecond - - writeBatch := func(n int) { - config.StorageGCInterval = time.Hour // effectively disable - processor, err := sampling.NewProcessor(config) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - for i := 0; i < n; i++ { - traceID := uuid.Must(uuid.NewV4()).String() - // Create a larger event to fill up the vlog faster, especially when it is above ValueThreshold - batch := modelpb.Batch{{ - Trace: &modelpb.Trace{Id: traceID}, - Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, - Span: &modelpb.Span{ - Type: strings.Repeat("a", 1000), - Subtype: strings.Repeat("b", 1000), - Id: traceID, - Name: strings.Repeat("c", 1000), - }, - }} - err := processor.ProcessBatch(context.Background(), &batch) - require.NoError(t, err) - assert.Empty(t, batch) - } - } - - // Process spans until value log files have been created. - // Garbage collection is disabled at this time. - for len(vlogFilenames(config.StorageDir)) < 3 { - writeBatch(2000) - } - - config.StorageGCInterval = 10 * time.Millisecond - processor, err := sampling.NewProcessor(config) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - - // Wait for the first value log file to be garbage collected. - var vlogs []string - assert.Eventually(t, func() bool { - vlogs = vlogFilenames(config.StorageDir) - return len(vlogs) == 0 || vlogs[0] != "000000.vlog" - }, 10*time.Second, 100*time.Millisecond, vlogs) -} - -func TestStorageGCConcurrency(t *testing.T) { - // This test ensures that TBS processor does not return an error - // even when run concurrently e.g. in hot reload - if testing.Short() { - t.Skip("skipping slow test") - } - - config := newTempdirConfig(t) - config.TTL = 10 * time.Millisecond - config.FlushInterval = 10 * time.Millisecond - config.StorageGCInterval = 10 * time.Millisecond - - g := errgroup.Group{} - for i := 0; i < 2; i++ { - processor, err := sampling.NewProcessor(config) - require.NoError(t, err) - g.Go(processor.Run) - go func() { - time.Sleep(time.Second) - assert.NoError(t, processor.Stop(context.Background())) - }() - } - assert.NoError(t, g.Wait()) -} - func TestStorageLimit(t *testing.T) { // This test ensures that when tail sampling is configured with a hard // storage limit, the limit is respected once the size is available. @@ -998,11 +919,11 @@ func newTempdirConfig(tb testing.TB) sampling.Config { require.NoError(tb, err) tb.Cleanup(func() { os.RemoveAll(tempdir) }) - badgerDB, err := eventstorage.NewStorageManager(tempdir) + db, err := eventstorage.NewStorageManager(tempdir) require.NoError(tb, err) - tb.Cleanup(func() { badgerDB.Close() }) + tb.Cleanup(func() { db.Close() }) - storage := badgerDB.NewReadWriter() + storage := db.NewReadWriter() return sampling.Config{ BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), @@ -1024,7 +945,7 @@ func newTempdirConfig(tb testing.TB) sampling.Config { UUID: "local-apm-server", }, StorageConfig: sampling.StorageConfig{ - DB: badgerDB, + DB: db, Storage: storage, StorageDir: tempdir, StorageGCInterval: time.Second, From 3a6f72760204a7b5655b76f671aaa4d583e2f6ce Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:23:24 +0000 Subject: [PATCH 12/37] Restore storage monitoring --- x-pack/apm-server/sampling/processor.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 8e40de74712..4289a991597 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -99,11 +99,11 @@ func (p *Processor) CollectMonitoring(_ monitoring.Mode, V monitoring.Visitor) { p.groups.mu.RUnlock() monitoring.ReportInt(V, "dynamic_service_groups", int64(numDynamicGroups)) - //monitoring.ReportNamespace(V, "storage", func() { - // lsmSize, valueLogSize := p.config.DB.Size() - // monitoring.ReportInt(V, "lsm_size", int64(lsmSize)) - // monitoring.ReportInt(V, "value_log_size", int64(valueLogSize)) - //}) + monitoring.ReportNamespace(V, "storage", func() { + lsmSize, valueLogSize := p.config.DB.Size() + monitoring.ReportInt(V, "lsm_size", int64(lsmSize)) + monitoring.ReportInt(V, "value_log_size", int64(valueLogSize)) + }) monitoring.ReportNamespace(V, "events", func() { monitoring.ReportInt(V, "processed", atomic.LoadInt64(&p.eventMetrics.processed)) monitoring.ReportInt(V, "dropped", atomic.LoadInt64(&p.eventMetrics.dropped)) From fd9abbc28308c411f77dbc6c1b2013d4f2321ba2 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:25:16 +0000 Subject: [PATCH 13/37] Update processor test --- x-pack/apm-server/sampling/processor_test.go | 279 +++++-------------- 1 file changed, 76 insertions(+), 203 deletions(-) diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index efb0c5e551c..247895c2b91 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -11,7 +11,6 @@ import ( "os" "path" "path/filepath" - "runtime" "sort" "strings" "testing" @@ -22,7 +21,6 @@ import ( "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "golang.org/x/sync/errgroup" "google.golang.org/protobuf/testing/protocmp" "github.com/elastic/apm-data/model/modelpb" @@ -646,82 +644,82 @@ func TestStorageMonitoring(t *testing.T) { assert.NotZero(t, metrics.Ints, "sampling.storage.value_log_size") } -func TestStorageLimit(t *testing.T) { - // This test ensures that when tail sampling is configured with a hard - // storage limit, the limit is respected once the size is available. - // To update the database size during our test without waiting a full - // minute, we store some span events, close and re-open the database, so - // the size is updated. - if testing.Short() { - t.Skip("skipping slow test") - } - - writeBatch := func(n int, c sampling.Config, assertBatch func(b modelpb.Batch)) *sampling.Processor { - processor, err := sampling.NewProcessor(c) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - batch := make(modelpb.Batch, 0, n) - for i := 0; i < n; i++ { - traceID := uuid.Must(uuid.NewV4()).String() - batch = append(batch, &modelpb.APMEvent{ - Trace: &modelpb.Trace{Id: traceID}, - Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, - Span: &modelpb.Span{ - Type: "type", - Id: traceID, - }, - }) - } - err = processor.ProcessBatch(context.Background(), &batch) - require.NoError(t, err) - assertBatch(batch) - return processor - } - - config := newTempdirConfig(t) - // Write 5K span events and close the DB to persist to disk the storage - // size and assert that none are reported immediately. - writeBatch(5000, config, func(b modelpb.Batch) { - assert.Empty(t, b, fmt.Sprintf("expected empty but size is %d", len(b))) - }) - assert.NoError(t, config.Storage.Flush()) - assert.NoError(t, config.DB.Close()) - - // Open a new instance of the badgerDB and check the size. - var err error - config.DB, err = eventstorage.NewStorageManager(config.StorageDir) - require.NoError(t, err) - t.Cleanup(func() { config.DB.Close() }) - config.Storage = config.DB.NewReadWriter() - - lsm, vlog := config.DB.Size() - assert.GreaterOrEqual(t, lsm+vlog, int64(1024)) - - config.StorageLimit = 1024 // Set the storage limit to 1024 bytes. - // Create a massive 150K span batch (per CPU) to trigger the badger error - // Transaction too big, causing the ProcessBatch to report the some traces - // immediately. - // Rather than setting a static threshold, use the runtime.NumCPU as a - // multiplier since the sharded writers use that variable and the more CPUs - // we have, the more sharded writes we'll have, resulting in a greater buffer. - // To avoid huge test time on large systems do this incrementally - for i := 1; i < runtime.NumCPU(); i++ { - processor := writeBatch(150_000*i, config, func(b modelpb.Batch) { - assert.NotEmpty(t, b) - }) - - failedWrites := collectProcessorMetrics(processor).Ints["sampling.events.failed_writes"] - t.Log(failedWrites) - // Ensure that there are some failed writes. - - if failedWrites >= 1 { - return - } - } - - t.Fatal("badger error never thrown") -} +//func TestStorageLimit(t *testing.T) { +// // This test ensures that when tail sampling is configured with a hard +// // storage limit, the limit is respected once the size is available. +// // To update the database size during our test without waiting a full +// // minute, we store some span events, close and re-open the database, so +// // the size is updated. +// if testing.Short() { +// t.Skip("skipping slow test") +// } +// +// writeBatch := func(n int, c sampling.Config, assertBatch func(b modelpb.Batch)) *sampling.Processor { +// processor, err := sampling.NewProcessor(c) +// require.NoError(t, err) +// go processor.Run() +// defer processor.Stop(context.Background()) +// batch := make(modelpb.Batch, 0, n) +// for i := 0; i < n; i++ { +// traceID := uuid.Must(uuid.NewV4()).String() +// batch = append(batch, &modelpb.APMEvent{ +// Trace: &modelpb.Trace{Id: traceID}, +// Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, +// Span: &modelpb.Span{ +// Type: "type", +// Id: traceID, +// }, +// }) +// } +// err = processor.ProcessBatch(context.Background(), &batch) +// require.NoError(t, err) +// assertBatch(batch) +// return processor +// } +// +// config := newTempdirConfig(t) +// // Write 5K span events and close the DB to persist to disk the storage +// // size and assert that none are reported immediately. +// writeBatch(5000, config, func(b modelpb.Batch) { +// assert.Empty(t, b, fmt.Sprintf("expected empty but size is %d", len(b))) +// }) +// assert.NoError(t, config.Storage.Flush()) +// assert.NoError(t, config.DB.Close()) +// +// // Open a new instance of the badgerDB and check the size. +// var err error +// config.DB, err = eventstorage.NewStorageManager(config.StorageDir) +// require.NoError(t, err) +// t.Cleanup(func() { config.DB.Close() }) +// config.Storage = config.DB.NewReadWriter() +// +// lsm, vlog := config.DB.Size() +// assert.GreaterOrEqual(t, lsm+vlog, int64(1024)) +// +// config.StorageLimit = 1024 // Set the storage limit to 1024 bytes. +// // Create a massive 150K span batch (per CPU) to trigger the badger error +// // Transaction too big, causing the ProcessBatch to report the some traces +// // immediately. +// // Rather than setting a static threshold, use the runtime.NumCPU as a +// // multiplier since the sharded writers use that variable and the more CPUs +// // we have, the more sharded writes we'll have, resulting in a greater buffer. +// // To avoid huge test time on large systems do this incrementally +// for i := 1; i < runtime.NumCPU(); i++ { +// processor := writeBatch(150_000*i, config, func(b modelpb.Batch) { +// assert.NotEmpty(t, b) +// }) +// +// failedWrites := collectProcessorMetrics(processor).Ints["sampling.events.failed_writes"] +// t.Log(failedWrites) +// // Ensure that there are some failed writes. +// +// if failedWrites >= 1 { +// return +// } +// } +// +// t.Fatal("badger error never thrown") +//} func TestProcessRemoteTailSamplingPersistence(t *testing.T) { config := newTempdirConfig(t) @@ -747,131 +745,6 @@ func TestProcessRemoteTailSamplingPersistence(t *testing.T) { assert.Equal(t, `{"index_name":1}`, string(data)) } -func TestDropLoop(t *testing.T) { - // This test ensures that if badger is stuck at storage limit for TTL, - // DB is dropped and recreated. - if testing.Short() { - t.Skip("skipping slow test") - } - - makeBatch := func(n int) modelpb.Batch { - batch := make(modelpb.Batch, 0, n) - for i := 0; i < n; i++ { - traceID := uuid.Must(uuid.NewV4()).String() - batch = append(batch, &modelpb.APMEvent{ - Trace: &modelpb.Trace{Id: traceID}, - Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, - Span: &modelpb.Span{ - Type: "type", - Id: traceID, - }, - }) - } - return batch - } - - writeBatch := func(t *testing.T, n int, c sampling.Config, assertBatch func(b modelpb.Batch)) *sampling.Processor { - processor, err := sampling.NewProcessor(c) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - batch := makeBatch(n) - err = processor.ProcessBatch(context.Background(), &batch) - require.NoError(t, err) - assertBatch(batch) - return processor - } - - for _, tc := range []struct { - name string - subscriberPosExists bool - }{ - { - name: "subscriber_position_not_exist", - subscriberPosExists: false, - }, - { - name: "subscriber_position_exists", - subscriberPosExists: true, - }, - } { - t.Run(tc.name, func(t *testing.T) { - t.Parallel() - config := newTempdirConfig(t) - config.StorageGCInterval = time.Hour // effectively disable GC - - config.FlushInterval = 10 * time.Millisecond - subscriberChan := make(chan string) - subscriber := pubsubtest.SubscriberChan(subscriberChan) - config.Elasticsearch = pubsubtest.Client(nil, subscriber) - subscriberPositionFile := filepath.Join(config.StorageDir, "subscriber_position.json") - - // Write 5K span events and close the DB to persist to disk the storage - // size and assert that none are reported immediately. - writeBatch(t, 5000, config, func(b modelpb.Batch) { - assert.Empty(t, b, fmt.Sprintf("expected empty but size is %d", len(b))) - - subscriberChan <- "0102030405060708090a0b0c0d0e0f10" - assert.Eventually(t, func() bool { - data, err := config.DB.ReadSubscriberPosition() - return err == nil && string(data) == `{"index_name":1}` - }, time.Second, 100*time.Millisecond) - }) - assert.NoError(t, config.Storage.Flush()) - assert.NoError(t, config.DB.Close()) - - if !tc.subscriberPosExists { - err := os.Remove(subscriberPositionFile) - assert.NoError(t, err) - } - - func() { - // Open a new instance of the badgerDB and check the size. - var err error - config.DB, err = eventstorage.NewStorageManager(config.StorageDir) - require.NoError(t, err) - t.Cleanup(func() { config.DB.Close() }) - config.Storage = config.DB.NewReadWriter() - - lsm, vlog := config.DB.Size() - assert.Greater(t, lsm+vlog, int64(1024*1024)) - - config.Elasticsearch = pubsubtest.Client(nil, nil) // disable pubsub - - config.StorageLimit = 100 * 1024 // lower limit to trigger storage limit error - config.TTL = time.Second - processor, err := sampling.NewProcessor(config) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - - // wait for up to 1 minute for dropAndRecreate to kick in - // no SST files after dropping DB and before first write - var filenames []string - assert.Eventually(t, func() bool { - filenames = sstFilenames(config.StorageDir) - return len(filenames) == 0 - }, 90*time.Second, 200*time.Millisecond, filenames) - - data, err := config.DB.ReadSubscriberPosition() - assert.NoError(t, err) - if tc.subscriberPosExists { - assert.Equal(t, `{"index_name":1}`, string(data)) - } else { - assert.Equal(t, "{}", string(data)) - } - - // try to write to new DB - batch := makeBatch(10) - err = processor.ProcessBatch(context.Background(), &batch) - require.NoError(t, err) - }() - assert.NoError(t, config.DB.Close()) - assert.Greater(t, len(sstFilenames(config.StorageDir)), 0) - }) - } -} - func TestGracefulShutdown(t *testing.T) { config := newTempdirConfig(t) sampleRate := 0.5 From 896dedbc181a1415a1ee9830f92fb249f8d1b8bb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:28:58 +0000 Subject: [PATCH 14/37] Explain AlreadyTailSampled failure --- x-pack/apm-server/sampling/processor_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 247895c2b91..9acf9dad5c8 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -55,6 +55,8 @@ func TestProcessUnsampled(t *testing.T) { } func TestProcessAlreadyTailSampled(t *testing.T) { + t.FailNow() // Expected to fail as TTL is not handled as strictly in pebble + config := newTempdirConfig(t) // Seed event storage with a tail-sampling decisions, to show that From 2c70ad5696579e990f9e12895feee54d9d12e12d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:32:54 +0000 Subject: [PATCH 15/37] Update size estimation --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 5f34278df1d..65bf4032a24 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -86,8 +86,7 @@ func (s *StorageManager) Close() error { func (s *StorageManager) Size() (lsm, vlog int64) { s.mu.RLock() defer s.mu.RUnlock() - return 0, 0 - //return s.db.Size() + return int64(s.db.Metrics().DiskSpaceUsage()), 0 // FIXME } func (s *StorageManager) NewIndexedBatch() *pebble.Batch { From cc38dbb9ee753982d70e3722144e4cb71edb0a5e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:39:34 +0000 Subject: [PATCH 16/37] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 1526637c19b..4ef3465bb8c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -194,6 +194,7 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. } func (rw *ReadWriter) writeEntry(key, data []byte) error { + // FIXME: possibly change key structure, because the append is going to be expensive if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { return err } @@ -235,7 +236,7 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error iter, err := rw.batch.NewIter(&pebble.IterOptions{ LowerBound: append([]byte(traceID), ':'), - UpperBound: append([]byte(traceID), ';'), + UpperBound: append([]byte(traceID), ';'), // This is a hack to stop before next ID }) if err != nil { return err From 8841d41f1d8bc67cca62d3de9e01bd7d751fcd5b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 14:55:38 +0000 Subject: [PATCH 17/37] Flush by number of pending writes for fair perf comparison --- .../apm-server/sampling/eventstorage/storage.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 4ef3465bb8c..b2ed4564e8b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -22,7 +22,7 @@ const ( entryMetaTraceUnsampled byte = 'u' entryMetaTraceEvent byte = 'e' - flushThreshold = 5 * 1024 * 1024 + //flushThreshold = 5 * 1024 * 1024 ) var ( @@ -138,7 +138,7 @@ func (rw *ReadWriter) Flush() error { rw.batch.Close() rw.batch = rw.s.db.NewIndexedBatch() //rw.s.pendingSize.Add(-rw.pendingSize) - //rw.pendingWrites = 0 + rw.pendingWrites = 0 //rw.pendingSize = baseTransactionSize //rw.s.pendingSize.Add(baseTransactionSize) if err != nil { @@ -194,12 +194,13 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. } func (rw *ReadWriter) writeEntry(key, data []byte) error { + rw.pendingWrites++ // FIXME: possibly change key structure, because the append is going to be expensive if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { return err } - if rw.batch.Len() > flushThreshold { + if rw.pendingWrites >= 200 { if err := rw.Flush(); err != nil { return err } @@ -222,11 +223,11 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { if err != nil { return err } - if rw.batch.Len() > flushThreshold { - if err := rw.Flush(); err != nil { - return err - } - } + //if rw.batch.Len() > flushThreshold { + // if err := rw.Flush(); err != nil { + // return err + // } + //} return nil } From adac2a25d7b1a32206b1fceb5112ee6a2407769a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 15:07:26 +0000 Subject: [PATCH 18/37] Try disable WAL --- x-pack/apm-server/sampling/eventstorage/pebble.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index c5099f023e5..e5220a29ce7 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -3,5 +3,7 @@ package eventstorage import "github.com/cockroachdb/pebble" func OpenPebble(storageDir string) (*pebble.DB, error) { - return pebble.Open(storageDir, &pebble.Options{}) + return pebble.Open(storageDir, &pebble.Options{ + DisableWAL: true, + }) } From a71fab0d6197aadd7e4054127f7e5fe3a376eb3e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 15:09:29 +0000 Subject: [PATCH 19/37] Try in-memory mode --- x-pack/apm-server/sampling/eventstorage/pebble.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e5220a29ce7..6c51b495c12 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,9 +1,12 @@ package eventstorage -import "github.com/cockroachdb/pebble" +import ( + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/vfs" +) func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ - DisableWAL: true, + FS: vfs.NewMem(), }) } From 15387c803d4882ed23bd24044483d8ff11258a21 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 15:11:59 +0000 Subject: [PATCH 20/37] Revert "Try in-memory mode" This reverts commit a71fab0d6197aadd7e4054127f7e5fe3a376eb3e. --- x-pack/apm-server/sampling/eventstorage/pebble.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 6c51b495c12..e5220a29ce7 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,12 +1,9 @@ package eventstorage -import ( - "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/vfs" -) +import "github.com/cockroachdb/pebble" func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ - FS: vfs.NewMem(), + DisableWAL: true, }) } From d73809db20cec18950dd513c91ac97d1be5342ea Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 15:16:37 +0000 Subject: [PATCH 21/37] Revert "Try disable WAL" This reverts commit adac2a25d7b1a32206b1fceb5112ee6a2407769a. --- x-pack/apm-server/sampling/eventstorage/pebble.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e5220a29ce7..c5099f023e5 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -3,7 +3,5 @@ package eventstorage import "github.com/cockroachdb/pebble" func OpenPebble(storageDir string) (*pebble.DB, error) { - return pebble.Open(storageDir, &pebble.Options{ - DisableWAL: true, - }) + return pebble.Open(storageDir, &pebble.Options{}) } From b2948779c300b9c05a1ffd67d878b3f1ae74cbaa Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:05:53 +0000 Subject: [PATCH 22/37] Use fork for pebble batch config --- go.mod | 4 ++- go.sum | 4 +-- .../sampling/eventstorage/pebble.go | 33 ++++++++++++++++++- 3 files changed, 37 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index b6f4d255361..6983b99b941 100644 --- a/go.mod +++ b/go.mod @@ -5,6 +5,7 @@ go 1.23.0 require ( github.com/KimMachineGun/automemlimit v0.7.0-pre.3 github.com/cespare/xxhash/v2 v2.3.0 + github.com/cockroachdb/pebble v1.1.2 github.com/dgraph-io/badger/v2 v2.2007.4 github.com/dustin/go-humanize v1.0.1 github.com/elastic/apm-aggregation v1.2.0 @@ -68,7 +69,6 @@ require ( github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect - github.com/cockroachdb/pebble v1.1.2 // indirect github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect @@ -159,3 +159,5 @@ require ( ) replace github.com/dop251/goja => github.com/elastic/goja v0.0.0-20190128172624-dd2ac4456e20 // pin to version used by beats + +replace github.com/cockroachdb/pebble => github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe diff --git a/go.sum b/go.sum index bb3ad3ed4c6..d926fb7f5f7 100644 --- a/go.sum +++ b/go.sum @@ -60,6 +60,8 @@ github.com/axiomhq/hyperloglog v0.2.0/go.mod h1:GcgMjz9gaDKZ3G0UMS6Fq/VkZ4l7uGgc github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe h1:/Bc5YzIaN9hbtu/Xffa3S59j0sHxMAGCYRkp6dDSwlQ= +github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe/go.mod h1:4exszw1r40423ZsmkG/09AFEG83I0uDgfujJdbL6kYU= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= @@ -75,8 +77,6 @@ github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 h1:pU88SPhIFid6/k github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0/go.mod h1:9/y3cnZ5GKakj/H4y9r9GTjCvAFta7KLgSHPJJYc52M= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v1.1.2 h1:CUh2IPtR4swHlEj48Rhfzw6l/d0qA31fItcIszQVIsA= -github.com/cockroachdb/pebble v1.1.2/go.mod h1:4exszw1r40423ZsmkG/09AFEG83I0uDgfujJdbL6kYU= github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index c5099f023e5..7fed0721204 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -2,6 +2,37 @@ package eventstorage import "github.com/cockroachdb/pebble" +const ( + // Batch grows in multiples of 2 based on the initial size. For + // example, if the initial size is 1MB then the batch will grow as + // {2, 4, 8, 16, ...}. If a batch of size greater than 4MBs is + // consistently committed then that batch will never be retained + // if the max retained size is smaller than 8MBs as the batch capacity + // will always grow to 8MB. + initialPebbleBatchSize = 64 << 10 // 64KB + maxRetainedPebbleBatchSize = 8 << 20 // 8MB + + // pebbleMemTableSize defines the max stead state size of a memtable. + // There can be more than 1 memtable in memory at a time as it takes + // time for old memtable to flush. The memtable size also defines + // the size for large batches. A large batch is a batch which will + // take atleast half of the memtable size. Note that the Batch#Len + // is not the same as the memtable size that the batch will occupy + // as data in batches are encoded differently. In general, the + // memtable size of the batch will be higher than the length of the + // batch data. + // + // On commit, data in the large batch maybe kept by pebble and thus + // large batches will need to be reallocated. Note that large batch + // classification uses the memtable size that a batch will occupy + // rather than the length of data slice backing the batch. + pebbleMemTableSize = 32 << 20 // 32MB +) + func OpenPebble(storageDir string) (*pebble.DB, error) { - return pebble.Open(storageDir, &pebble.Options{}) + return pebble.Open(storageDir, &pebble.Options{ + BatchInitialSize: initialPebbleBatchSize, + BatchMaxRetainedSize: maxRetainedPebbleBatchSize, + MemTableSize: pebbleMemTableSize, + }) } From e2d2cdf3914d935116aed62e80876c672e2c3283 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:10:14 +0000 Subject: [PATCH 23/37] Add db commit threshold bytes --- x-pack/apm-server/sampling/eventstorage/pebble.go | 7 +++++++ x-pack/apm-server/sampling/eventstorage/storage.go | 4 +--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 7fed0721204..0695b06e471 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -27,6 +27,13 @@ const ( // classification uses the memtable size that a batch will occupy // rather than the length of data slice backing the batch. pebbleMemTableSize = 32 << 20 // 32MB + + // dbCommitThresholdBytes is a soft limit and the batch is committed + // to the DB as soon as it crosses this threshold. To make sure that + // the commit threshold plays will with the max retained batch size + // the threshold should be kept smaller than the sum of max retained + // batch size and encoded size of aggregated data to be committed. + dbCommitThresholdBytes = 8000 << 10 // 8000KB ) func OpenPebble(storageDir string) (*pebble.DB, error) { diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index b2ed4564e8b..ecca22cc589 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -21,8 +21,6 @@ const ( entryMetaTraceSampled byte = 's' entryMetaTraceUnsampled byte = 'u' entryMetaTraceEvent byte = 'e' - - //flushThreshold = 5 * 1024 * 1024 ) var ( @@ -200,7 +198,7 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { return err } - if rw.pendingWrites >= 200 { + if rw.batch.Len() >= dbCommitThresholdBytes { if err := rw.Flush(); err != nil { return err } From 60ab9cabfa0433fee92ed1d2d4798cbc9430de00 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:26:09 +0000 Subject: [PATCH 24/37] Use BatchOption --- go.mod | 3 +-- go.sum | 6 ++---- x-pack/apm-server/sampling/eventstorage/pebble.go | 4 +--- x-pack/apm-server/sampling/eventstorage/storage.go | 10 +++++++--- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index 6983b99b941..7657acc631a 100644 --- a/go.mod +++ b/go.mod @@ -67,7 +67,6 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash v1.1.0 // indirect github.com/cockroachdb/errors v1.11.3 // indirect - github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect github.com/cockroachdb/redact v1.1.5 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect @@ -160,4 +159,4 @@ require ( replace github.com/dop251/goja => github.com/elastic/goja v0.0.0-20190128172624-dd2ac4456e20 // pin to version used by beats -replace github.com/cockroachdb/pebble => github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe +replace github.com/cockroachdb/pebble => github.com/carsonip/pebble v0.0.0-20250114162318-fa34738bbef0 diff --git a/go.sum b/go.sum index d926fb7f5f7..bdf93fc0ce9 100644 --- a/go.sum +++ b/go.sum @@ -60,8 +60,8 @@ github.com/axiomhq/hyperloglog v0.2.0/go.mod h1:GcgMjz9gaDKZ3G0UMS6Fq/VkZ4l7uGgc github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe h1:/Bc5YzIaN9hbtu/Xffa3S59j0sHxMAGCYRkp6dDSwlQ= -github.com/carsonip/pebble v0.0.0-20250114155901-9c153a2df5fe/go.mod h1:4exszw1r40423ZsmkG/09AFEG83I0uDgfujJdbL6kYU= +github.com/carsonip/pebble v0.0.0-20250114162318-fa34738bbef0 h1:FSsCMsR/nTCbTsfbxQu2Xy5VArWxzgjBXRe0uEJiMMI= +github.com/carsonip/pebble v0.0.0-20250114162318-fa34738bbef0/go.mod h1:sEHm5NOXxyiAoKWhoFxT8xMgd/f3RA6qUqQ1BXKrh2E= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= @@ -73,8 +73,6 @@ github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaY github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= github.com/cockroachdb/errors v1.11.3 h1:5bA+k2Y6r+oz/6Z/RFlNeVCesGARKuC6YymtcDrbC/I= github.com/cockroachdb/errors v1.11.3/go.mod h1:m4UIW4CDjx+R5cybPsNrRbreomiFqt8o1h1wUVazSd8= -github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 h1:pU88SPhIFid6/k0egdR5V6eALQYq2qbSmukrkgIh/0A= -github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0/go.mod h1:9/y3cnZ5GKakj/H4y9r9GTjCvAFta7KLgSHPJJYc52M= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 0695b06e471..7395eaacc78 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -38,8 +38,6 @@ const ( func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ - BatchInitialSize: initialPebbleBatchSize, - BatchMaxRetainedSize: maxRetainedPebbleBatchSize, - MemTableSize: pebbleMemTableSize, + MemTableSize: pebbleMemTableSize, }) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index ecca22cc589..63a85d25fcb 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -34,7 +34,7 @@ var ( ) type db interface { - NewIndexedBatch() *pebble.Batch + NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch Size() (lsm, vlog int64) Close() error } @@ -109,7 +109,10 @@ type ReadWriter struct { func (rw *ReadWriter) lazyInit() { if rw.batch == nil { - rw.batch = rw.s.db.NewIndexedBatch() + rw.batch = rw.s.db.NewIndexedBatch( + pebble.WithInitialSizeBytes(initialPebbleBatchSize), + pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), + ) } } @@ -134,7 +137,8 @@ func (rw *ReadWriter) Flush() error { const flushErrFmt = "failed to flush pending writes: %w" err := rw.batch.Commit(pebble.NoSync) rw.batch.Close() - rw.batch = rw.s.db.NewIndexedBatch() + rw.batch = nil + rw.lazyInit() //rw.s.pendingSize.Add(-rw.pendingSize) rw.pendingWrites = 0 //rw.pendingSize = baseTransactionSize From ef1b422b575922b40c072bcf6c550cf2089be88a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:31:08 +0000 Subject: [PATCH 25/37] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 63a85d25fcb..8936dea0804 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -138,7 +138,7 @@ func (rw *ReadWriter) Flush() error { err := rw.batch.Commit(pebble.NoSync) rw.batch.Close() rw.batch = nil - rw.lazyInit() + rw.lazyInit() // FIXME: this shouldn't be needed //rw.s.pendingSize.Add(-rw.pendingSize) rw.pendingWrites = 0 //rw.pendingSize = baseTransactionSize From 4b1cbdd9c2a0290668f46c6a9a507436176ed410 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:32:32 +0000 Subject: [PATCH 26/37] Fix compile error --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 65bf4032a24..aa0888cee4a 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -89,10 +89,10 @@ func (s *StorageManager) Size() (lsm, vlog int64) { return int64(s.db.Metrics().DiskSpaceUsage()), 0 // FIXME } -func (s *StorageManager) NewIndexedBatch() *pebble.Batch { +func (s *StorageManager) NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch { s.mu.RLock() defer s.mu.RUnlock() - return s.db.NewIndexedBatch() + return s.db.NewIndexedBatch(opts...) } // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. From c0e679cb5f5cd531ba89b959f9e0d813742b939e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 16:51:32 +0000 Subject: [PATCH 27/37] Add logger to pebble --- x-pack/apm-server/sampling/eventstorage/pebble.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 7395eaacc78..c126ec13fa1 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,6 +1,11 @@ package eventstorage -import "github.com/cockroachdb/pebble" +import ( + "github.com/cockroachdb/pebble" + + "github.com/elastic/apm-server/internal/logs" + "github.com/elastic/elastic-agent-libs/logp" +) const ( // Batch grows in multiples of 2 based on the initial size. For @@ -38,6 +43,7 @@ const ( func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ + Logger: logp.NewLogger(logs.Sampling), MemTableSize: pebbleMemTableSize, }) } From c883c353a04439f376a7e9f971bbcb3cb67825a9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 17:13:24 +0000 Subject: [PATCH 28/37] Sort imports, remove logger.go --- .../sampling/eventstorage/logger.go | 50 ------------------- .../sampling/eventstorage/storage.go | 1 + .../sampling/eventstorage/storage_manager.go | 1 + 3 files changed, 2 insertions(+), 50 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/logger.go diff --git a/x-pack/apm-server/sampling/eventstorage/logger.go b/x-pack/apm-server/sampling/eventstorage/logger.go deleted file mode 100644 index 695a68e8a78..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/logger.go +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one -// or more contributor license agreements. Licensed under the Elastic License 2.0; -// you may not use this file except in compliance with the Elastic License 2.0. - -package eventstorage - -import ( - "fmt" - "sync" - - "github.com/elastic/elastic-agent-libs/logp" -) - -// LogpAdaptor adapts logp.Logger to the badger.Logger interface. -type LogpAdaptor struct { - *logp.Logger - - mu sync.RWMutex - last string -} - -// Errorf prints the log message when the current message isn't the same as the -// previously logged message. -func (a *LogpAdaptor) Errorf(format string, args ...interface{}) { - msg := fmt.Sprintf(format, args...) - if a.setLast(msg) { - a.Logger.Errorf(format, args...) - } -} - -func (a *LogpAdaptor) setLast(msg string) bool { - a.mu.RLock() - if msg != a.last { - a.mu.RUnlock() - return false - } - a.mu.RUnlock() - a.mu.Lock() - defer a.mu.Unlock() - shouldSet := msg != a.last - if shouldSet { - a.last = msg - } - return shouldSet -} - -// Warningf adapts badger.Logger.Warningf to logp.Logger.Warngf. -func (a *LogpAdaptor) Warningf(format string, args ...interface{}) { - a.Warnf(format, args...) -} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 8936dea0804..70f57b8e1b7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -12,6 +12,7 @@ import ( "time" "github.com/cockroachdb/pebble" + "github.com/elastic/apm-data/model/modelpb" ) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index aa0888cee4a..911feb81a62 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -12,6 +12,7 @@ import ( "time" "github.com/cockroachdb/pebble" + "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" From 08c92052e61abdce284116b5f6722318b386371a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 18:20:36 +0000 Subject: [PATCH 29/37] Disable pebble level compression --- x-pack/apm-server/sampling/eventstorage/pebble.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index c126ec13fa1..7f536423708 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -45,5 +45,10 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ Logger: logp.NewLogger(logs.Sampling), MemTableSize: pebbleMemTableSize, + Levels: []pebble.LevelOptions{ + { + Compression: pebble.NoCompression, + }, + }, }) } From 1c89735ab309774a7db95cc1eaa7f82af9858bdc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 18:30:51 +0000 Subject: [PATCH 30/37] Add table bloom filter --- x-pack/apm-server/sampling/eventstorage/pebble.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 7f536423708..74fcc566b25 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -2,6 +2,7 @@ package eventstorage import ( "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/bloom" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" @@ -47,7 +48,9 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { - Compression: pebble.NoCompression, + Compression: pebble.NoCompression, + FilterPolicy: bloom.FilterPolicy(10), + FilterType: pebble.TableFilter, }, }, }) From c35dcb9718170f0fe992605404fb2bb171a0be2f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 14 Jan 2025 18:57:34 +0000 Subject: [PATCH 31/37] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 70f57b8e1b7..dea061a8e6b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -168,6 +168,10 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { rw.lazyInit() + // FIXME: this needs to be fast, as it is in the hot path + // It should minimize disk IO on miss due to + // 1. (pubsub) remote sampling decision + // 2. (hot path) sampling decision not made yet item, closer, err := rw.batch.Get([]byte(traceID)) if err == pebble.ErrNotFound { return false, ErrNotFound From ea7b481edc982c7d702f63055a607e0cb1ad76cc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 15:11:17 +0000 Subject: [PATCH 32/37] Add prefix to sampling decision key to separate from events to improve perf --- x-pack/apm-server/sampling/eventstorage/storage.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index dea061a8e6b..f7f4c586d3c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -17,6 +17,7 @@ import ( ) const ( + prefixSamplingDecision string = "!" // NOTE(axw) these values (and their meanings) must remain stable // over time, to avoid misinterpreting historical data. entryMetaTraceSampled byte = 's' @@ -154,7 +155,7 @@ func (rw *ReadWriter) Flush() error { func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { rw.lazyInit() - key := []byte(traceID) + key := []byte(prefixSamplingDecision + traceID) meta := entryMetaTraceUnsampled if sampled { meta = entryMetaTraceSampled @@ -172,7 +173,7 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // It should minimize disk IO on miss due to // 1. (pubsub) remote sampling decision // 2. (hot path) sampling decision not made yet - item, closer, err := rw.batch.Get([]byte(traceID)) + item, closer, err := rw.batch.Get([]byte(prefixSamplingDecision + traceID)) if err == pebble.ErrNotFound { return false, ErrNotFound } @@ -203,7 +204,7 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. func (rw *ReadWriter) writeEntry(key, data []byte) error { rw.pendingWrites++ // FIXME: possibly change key structure, because the append is going to be expensive - if err := rw.batch.Set(key, append([]byte{entryMetaTraceEvent}, data...), pebble.NoSync); err != nil { + if err := rw.batch.Set(key, data, pebble.NoSync); err != nil { return err } @@ -256,7 +257,7 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error if err != nil { return err } - if err := rw.s.codec.DecodeEvent(data[1:], event); err != nil { + if err := rw.s.codec.DecodeEvent(data, event); err != nil { return fmt.Errorf("codec failed to decode event: %w", err) } *out = append(*out, event) From 5dde3d048d61746d12eb3acffa44f58acf8961b9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 15:55:48 +0000 Subject: [PATCH 33/37] Use a sync.Map for sampling decision to establish baseline perf --- .../sampling/eventstorage/storage.go | 46 +++++++++++-------- 1 file changed, 28 insertions(+), 18 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index f7f4c586d3c..b5d68320c0c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -8,6 +8,7 @@ import ( "bytes" "errors" "fmt" + "sync" "sync/atomic" "time" @@ -48,6 +49,8 @@ type Storage struct { // pendingSize tracks the total size of pending writes across ReadWriters pendingSize *atomic.Int64 codec Codec + + sampled sync.Map } // Codec provides methods for encoding and decoding events. @@ -153,32 +156,39 @@ func (rw *ReadWriter) Flush() error { // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - rw.lazyInit() - - key := []byte(prefixSamplingDecision + traceID) - meta := entryMetaTraceUnsampled - if sampled { - meta = entryMetaTraceSampled - } - return rw.batch.Set(key, []byte{meta}, pebble.NoSync) + rw.s.sampled.Store(traceID, sampled) + return nil + //rw.lazyInit() + // + //key := []byte(prefixSamplingDecision + traceID) + //meta := entryMetaTraceUnsampled + //if sampled { + // meta = entryMetaTraceSampled + //} + //return rw.batch.Set(key, []byte{meta}, pebble.NoSync) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - rw.lazyInit() - - // FIXME: this needs to be fast, as it is in the hot path - // It should minimize disk IO on miss due to - // 1. (pubsub) remote sampling decision - // 2. (hot path) sampling decision not made yet - item, closer, err := rw.batch.Get([]byte(prefixSamplingDecision + traceID)) - if err == pebble.ErrNotFound { + if sampled, ok := rw.s.sampled.Load(traceID); !ok { return false, ErrNotFound + } else { + return sampled.(bool), nil } - defer closer.Close() - return item[0] == entryMetaTraceSampled, nil + //rw.lazyInit() + // + //// FIXME: this needs to be fast, as it is in the hot path + //// It should minimize disk IO on miss due to + //// 1. (pubsub) remote sampling decision + //// 2. (hot path) sampling decision not made yet + //item, closer, err := rw.batch.Get([]byte(prefixSamplingDecision + traceID)) + //if err == pebble.ErrNotFound { + // return false, ErrNotFound + //} + //defer closer.Close() + //return item[0] == entryMetaTraceSampled, nil } // WriteTraceEvent writes a trace event to storage. From b8f50f3187a53d39779165c41e4b7fcaf54a43ff Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 16:54:43 +0000 Subject: [PATCH 34/37] Try enabling snappy to trade cpu for reduced disk IO / usage --- x-pack/apm-server/sampling/eventstorage/pebble.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 74fcc566b25..90da8e4c577 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -48,7 +48,7 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { - Compression: pebble.NoCompression, + Compression: pebble.SnappyCompression, FilterPolicy: bloom.FilterPolicy(10), FilterType: pebble.TableFilter, }, From 554e93daff84557507df5c747a8f5550c16caa4f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 17:17:02 +0000 Subject: [PATCH 35/37] Try 16KB block size for better compression --- x-pack/apm-server/sampling/eventstorage/pebble.go | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 90da8e4c577..e888000b00b 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -48,6 +48,7 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { + BlockSize: 16 << 10, Compression: pebble.SnappyCompression, FilterPolicy: bloom.FilterPolicy(10), FilterType: pebble.TableFilter, From e1694a14da9cf0bcf71c765f19a4cce335a99653 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 17:30:55 +0000 Subject: [PATCH 36/37] Try FormatMajorVersion --- x-pack/apm-server/sampling/eventstorage/pebble.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e888000b00b..95aa4ae13b5 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -44,8 +44,10 @@ const ( func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(storageDir, &pebble.Options{ - Logger: logp.NewLogger(logs.Sampling), - MemTableSize: pebbleMemTableSize, + // FIXME: Specify FormatMajorVersion to use value blocks? + FormatMajorVersion: pebble.FormatNewest, + Logger: logp.NewLogger(logs.Sampling), + MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { BlockSize: 16 << 10, From a1e9010dfe31ef82367e21f3b97e7b0494510cbb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 15 Jan 2025 22:06:21 +0000 Subject: [PATCH 37/37] Revert "Use a sync.Map for sampling decision to establish baseline perf" This reverts commit 5dde3d048d61746d12eb3acffa44f58acf8961b9. --- .../sampling/eventstorage/storage.go | 46 ++++++++----------- 1 file changed, 18 insertions(+), 28 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index b5d68320c0c..f7f4c586d3c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -8,7 +8,6 @@ import ( "bytes" "errors" "fmt" - "sync" "sync/atomic" "time" @@ -49,8 +48,6 @@ type Storage struct { // pendingSize tracks the total size of pending writes across ReadWriters pendingSize *atomic.Int64 codec Codec - - sampled sync.Map } // Codec provides methods for encoding and decoding events. @@ -156,39 +153,32 @@ func (rw *ReadWriter) Flush() error { // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - rw.s.sampled.Store(traceID, sampled) - return nil - //rw.lazyInit() - // - //key := []byte(prefixSamplingDecision + traceID) - //meta := entryMetaTraceUnsampled - //if sampled { - // meta = entryMetaTraceSampled - //} - //return rw.batch.Set(key, []byte{meta}, pebble.NoSync) + rw.lazyInit() + + key := []byte(prefixSamplingDecision + traceID) + meta := entryMetaTraceUnsampled + if sampled { + meta = entryMetaTraceSampled + } + return rw.batch.Set(key, []byte{meta}, pebble.NoSync) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - if sampled, ok := rw.s.sampled.Load(traceID); !ok { + rw.lazyInit() + + // FIXME: this needs to be fast, as it is in the hot path + // It should minimize disk IO on miss due to + // 1. (pubsub) remote sampling decision + // 2. (hot path) sampling decision not made yet + item, closer, err := rw.batch.Get([]byte(prefixSamplingDecision + traceID)) + if err == pebble.ErrNotFound { return false, ErrNotFound - } else { - return sampled.(bool), nil } - //rw.lazyInit() - // - //// FIXME: this needs to be fast, as it is in the hot path - //// It should minimize disk IO on miss due to - //// 1. (pubsub) remote sampling decision - //// 2. (hot path) sampling decision not made yet - //item, closer, err := rw.batch.Get([]byte(prefixSamplingDecision + traceID)) - //if err == pebble.ErrNotFound { - // return false, ErrNotFound - //} - //defer closer.Close() - //return item[0] == entryMetaTraceSampled, nil + defer closer.Close() + return item[0] == entryMetaTraceSampled, nil } // WriteTraceEvent writes a trace event to storage.