From 7478521ac76f30201c65af2b3d5eec11831990dc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 16 Dec 2024 14:48:08 +0000 Subject: [PATCH 001/184] 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 002/184] 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 003/184] 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 004/184] 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 005/184] 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 006/184] 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 007/184] 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 008/184] 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 009/184] 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 010/184] 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 011/184] 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 012/184] 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 013/184] 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 014/184] 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 015/184] 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 016/184] 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 017/184] 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 018/184] 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 019/184] 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 020/184] 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 021/184] 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 022/184] 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 023/184] 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 024/184] 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 025/184] 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 026/184] 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 027/184] 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 028/184] 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 029/184] 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 030/184] 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 031/184] 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 032/184] 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 033/184] 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 034/184] 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 035/184] 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 036/184] 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 037/184] 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. From 92b49ec5891cc82576a4aec80288032f1b6d6b1f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 13:59:01 +0000 Subject: [PATCH 038/184] Use a separate DB for decision --- .../sampling/eventstorage/pebble.go | 21 ++++++- .../sampling/eventstorage/storage.go | 60 +++++++++++++------ .../sampling/eventstorage/storage_manager.go | 51 +++++++++++----- 3 files changed, 96 insertions(+), 36 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 95aa4ae13b5..375b8402ec6 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,6 +1,8 @@ package eventstorage import ( + "path/filepath" + "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/bloom" @@ -43,7 +45,7 @@ const ( ) func OpenPebble(storageDir string) (*pebble.DB, error) { - return pebble.Open(storageDir, &pebble.Options{ + return pebble.Open(filepath.Join(storageDir, "event"), &pebble.Options{ // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, Logger: logp.NewLogger(logs.Sampling), @@ -58,3 +60,20 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { }, }) } + +func OpenSamplingDecisionPebble(storageDir string) (*pebble.DB, error) { + return pebble.Open(filepath.Join(storageDir, "decision"), &pebble.Options{ + // FIXME: Specify FormatMajorVersion to use value blocks? + FormatMajorVersion: pebble.FormatNewest, + Logger: logp.NewLogger(logs.Sampling), + //MemTableSize: pebbleMemTableSize, + Levels: []pebble.LevelOptions{ + { + //BlockSize: 16 << 10, + Compression: pebble.NoCompression, + FilterPolicy: bloom.FilterPolicy(10), + FilterType: pebble.TableFilter, + }, + }, + }) +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index f7f4c586d3c..0ecdd7f472a 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -17,7 +17,7 @@ import ( ) const ( - prefixSamplingDecision string = "!" + //prefixSamplingDecision string = "!" // NOTE(axw) these values (and their meanings) must remain stable // over time, to avoid misinterpreting historical data. entryMetaTraceSampled byte = 's' @@ -38,13 +38,14 @@ var ( type db interface { NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch Size() (lsm, vlog int64) - Close() error + //Close() error } // Storage provides storage for sampled transactions and spans, // and for recording trace sampling decisions. type Storage struct { - db db + db db + decisionDB db // pendingSize tracks the total size of pending writes across ReadWriters pendingSize *atomic.Int64 codec Codec @@ -56,9 +57,14 @@ type Codec interface { EncodeEvent(*modelpb.APMEvent) ([]byte, error) } -// New returns a new Storage using db and codec. -func New(db db, codec Codec) *Storage { - return &Storage{db: db, pendingSize: &atomic.Int64{}, codec: codec} +// New returns a new Storage using db, decisionDB and codec. +func New(db db, decisionDB db, codec Codec) *Storage { + return &Storage{ + db: db, + decisionDB: decisionDB, + pendingSize: &atomic.Int64{}, + codec: codec, + } } // NewShardedReadWriter returns a new ShardedReadWriter, for sharded @@ -97,8 +103,9 @@ 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 - batch *pebble.Batch + s *Storage + batch *pebble.Batch + decisionBatch *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 @@ -118,6 +125,12 @@ func (rw *ReadWriter) lazyInit() { } } +func (rw *ReadWriter) decisionLazyInit() { + if rw.decisionBatch == nil { + rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch() //FIXME: tuning + } +} + // 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 @@ -126,6 +139,9 @@ func (rw *ReadWriter) Close() { if rw.batch != nil { rw.batch.Close() } + if rw.decisionBatch != nil { + rw.decisionBatch.Close() + } } // Flush waits for preceding writes to be committed to storage. @@ -134,13 +150,20 @@ func (rw *ReadWriter) Close() { // If Flush is not called before the writer is closed, then writes // may be lost. func (rw *ReadWriter) Flush() error { - rw.lazyInit() - const flushErrFmt = "failed to flush pending writes: %w" - err := rw.batch.Commit(pebble.NoSync) - rw.batch.Close() - rw.batch = nil - rw.lazyInit() // FIXME: this shouldn't be needed + var err error + if rw.batch != nil { + err = rw.batch.Commit(pebble.NoSync) + rw.batch.Close() + rw.batch = nil + } + + if rw.decisionBatch != nil { + err = errors.Join(err, rw.decisionBatch.Commit(pebble.NoSync)) + rw.decisionBatch.Close() + rw.decisionBatch = nil + } + //rw.s.pendingSize.Add(-rw.pendingSize) rw.pendingWrites = 0 //rw.pendingSize = baseTransactionSize @@ -153,27 +176,26 @@ 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() + rw.decisionLazyInit() - key := []byte(prefixSamplingDecision + traceID) meta := entryMetaTraceUnsampled if sampled { meta = entryMetaTraceSampled } - return rw.batch.Set(key, []byte{meta}, pebble.NoSync) + return rw.decisionBatch.Set([]byte(traceID), []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() + rw.decisionLazyInit() // 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)) + item, closer, err := rw.decisionBatch.Get([]byte(traceID)) if err == pebble.ErrNotFound { return false, ErrNotFound } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 911feb81a62..5ec341325af 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -28,6 +28,25 @@ var ( errDropAndRecreateInProgress = errors.New("db drop and recreate in progress") ) +type wrappedDB struct { + sm *StorageManager + db *pebble.DB +} + +// Size returns the db size +func (w *wrappedDB) Size() (lsm, vlog int64) { + // FIXME: we may want it to report the sum of 2 dbs + w.sm.mu.RLock() + defer w.sm.mu.RUnlock() + return int64(w.db.Metrics().DiskSpaceUsage()), 0 // FIXME +} + +func (w *wrappedDB) NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch { + w.sm.mu.RLock() + defer w.sm.mu.RUnlock() + return w.db.NewIndexedBatch(opts...) +} + // StorageManager encapsulates pebble.DB. // It is to provide file system access, simplify synchronization and enable underlying db swaps. // It assumes exclusive access to pebble DB at storageDir. @@ -35,9 +54,10 @@ type StorageManager struct { storageDir string logger *logp.Logger - db *pebble.DB - storage *Storage - rw *ShardedReadWriter + db *pebble.DB + decisionDB *pebble.DB + storage *Storage + rw *ShardedReadWriter // mu guards db, storage, and rw swaps. mu sync.RWMutex @@ -70,30 +90,29 @@ func (s *StorageManager) reset() error { return err } s.db = db - s.storage = New(s, ProtobufCodec{}) + decisionDB, err := OpenSamplingDecisionPebble(s.storageDir) + if err != nil { + return err + } + s.decisionDB = decisionDB + s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, ProtobufCodec{}) s.rw = s.storage.NewShardedReadWriter() return nil } -// Close closes StorageManager's underlying ShardedReadWriter and badger DB -func (s *StorageManager) Close() error { - s.mu.RLock() - defer s.mu.RUnlock() - s.rw.Close() - return s.db.Close() -} - -// Size returns the db size func (s *StorageManager) Size() (lsm, vlog int64) { s.mu.RLock() defer s.mu.RUnlock() - return int64(s.db.Metrics().DiskSpaceUsage()), 0 // FIXME + return int64(s.db.Metrics().DiskSpaceUsage() + s.decisionDB.Metrics().DiskSpaceUsage()), 0 } -func (s *StorageManager) NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch { +func (s *StorageManager) Close() error { s.mu.RLock() defer s.mu.RUnlock() - return s.db.NewIndexedBatch(opts...) + s.rw.Close() + _ = s.db.Close() // FIXME + _ = s.decisionDB.Close() // FIXME + return nil } // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. From 2643e508d671c6152268412da3021c29fbc58a3d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 16:48:29 +0000 Subject: [PATCH 039/184] First attempt to fix test and benchmarks --- .../eventstorage/sharded_bench_test.go | 109 ++-- .../eventstorage/storage_bench_test.go | 24 +- .../sampling/eventstorage/storage_manager.go | 16 +- .../sampling/eventstorage/storage_test.go | 595 +++++++++--------- .../eventstorage/storage_whitebox_test.go | 166 +++-- 5 files changed, 441 insertions(+), 469 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go b/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go index 7ef76309093..5f2b8b955f4 100644 --- a/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go @@ -4,62 +4,53 @@ package eventstorage_test -import ( - "testing" - "time" - - "github.com/gofrs/uuid/v5" - - "github.com/elastic/apm-data/model/modelpb" - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" -) - -func BenchmarkShardedWriteTransactionUncontended(b *testing.B) { - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - sharded := store.NewShardedReadWriter() - defer sharded.Close() - wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 0, - } - - b.RunParallel(func(pb *testing.PB) { - traceID := uuid.Must(uuid.NewV4()).String() - transaction := &modelpb.APMEvent{ - Transaction: &modelpb.Transaction{Id: traceID}, - } - for pb.Next() { - if err := sharded.WriteTraceEvent(traceID, traceID, transaction, wOpts); err != nil { - b.Fatal(err) - } - } - }) -} - -func BenchmarkShardedWriteTransactionContended(b *testing.B) { - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - sharded := store.NewShardedReadWriter() - defer sharded.Close() - wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 0, - } - - // Use a single trace ID, causing all events to go through - // the same sharded writer, contending for a single lock. - traceID := uuid.Must(uuid.NewV4()).String() - - b.RunParallel(func(pb *testing.PB) { - transactionID := uuid.Must(uuid.NewV4()).String() - transaction := &modelpb.APMEvent{ - Transaction: &modelpb.Transaction{Id: transactionID}, - } - for pb.Next() { - if err := sharded.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { - b.Fatal(err) - } - } - }) -} +// +//func BenchmarkShardedWriteTransactionUncontended(b *testing.B) { +// db := newBadgerDB(b, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// sharded := store.NewShardedReadWriter() +// defer sharded.Close() +// wOpts := eventstorage.WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 0, +// } +// +// b.RunParallel(func(pb *testing.PB) { +// traceID := uuid.Must(uuid.NewV4()).String() +// transaction := &modelpb.APMEvent{ +// Transaction: &modelpb.Transaction{Id: traceID}, +// } +// for pb.Next() { +// if err := sharded.WriteTraceEvent(traceID, traceID, transaction, wOpts); err != nil { +// b.Fatal(err) +// } +// } +// }) +//} +// +//func BenchmarkShardedWriteTransactionContended(b *testing.B) { +// db := newBadgerDB(b, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// sharded := store.NewShardedReadWriter() +// defer sharded.Close() +// wOpts := eventstorage.WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 0, +// } +// +// // Use a single trace ID, causing all events to go through +// // the same sharded writer, contending for a single lock. +// traceID := uuid.Must(uuid.NewV4()).String() +// +// b.RunParallel(func(pb *testing.PB) { +// transactionID := uuid.Must(uuid.NewV4()).String() +// transaction := &modelpb.APMEvent{ +// Transaction: &modelpb.Transaction{Id: transactionID}, +// } +// for pb.Next() { +// if err := sharded.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { +// b.Fatal(err) +// } +// } +// }) +//} diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index 90f58fbd268..b0b9af31e20 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -19,9 +19,8 @@ import ( func BenchmarkWriteTransaction(b *testing.B) { test := func(b *testing.B, codec eventstorage.Codec, bigTX bool) { - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, codec) - readWriter := store.NewReadWriter() + sm := newStorageManager(b, eventstorage.WithCodec(codec)) + readWriter := sm.NewBypassReadWriter() defer readWriter.Close() traceID := hex.EncodeToString([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}) @@ -87,9 +86,8 @@ func BenchmarkReadEvents(b *testing.B) { counts := []int{0, 1, 10, 100, 199, 399, 1000} for _, count := range counts { b.Run(fmt.Sprintf("%d events", count), func(b *testing.B) { - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, codec) - readWriter := store.NewReadWriter() + sm := newStorageManager(b, eventstorage.WithCodec(codec)) + readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ TTL: time.Minute, @@ -167,12 +165,11 @@ func BenchmarkReadEventsHit(b *testing.B) { // And causes next iteration setup to take a very long time. const txnCountInTrace = 5 - test := func(b *testing.B, codec eventstorage.Codec, bigTX bool) { + test := func(b *testing.B, bigTX bool) { for _, hit := range []bool{false, true} { b.Run(fmt.Sprintf("hit=%v", hit), func(b *testing.B) { - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, codec) - readWriter := store.NewReadWriter() + sm := newStorageManager(b) + readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ TTL: time.Hour, @@ -226,7 +223,7 @@ func BenchmarkReadEventsHit(b *testing.B) { for _, bigTX := range []bool{true, false} { b.Run(fmt.Sprintf("bigTX=%v", bigTX), func(b *testing.B) { - test(b, eventstorage.ProtobufCodec{}, bigTX) + test(b, bigTX) }) } } @@ -237,9 +234,8 @@ func BenchmarkIsTraceSampled(b *testing.B) { unknownTraceUUID := uuid.Must(uuid.NewV4()) // Test with varying numbers of events in the trace. - db := newBadgerDB(b, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - readWriter := store.NewReadWriter() + sm := newStorageManager(b) + readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ TTL: time.Minute, diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 5ec341325af..b43f7951ce1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -47,6 +47,14 @@ func (w *wrappedDB) NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch { return w.db.NewIndexedBatch(opts...) } +type StorageManagerOptions func(*StorageManager) + +func WithCodec(codec Codec) StorageManagerOptions { + return func(sm *StorageManager) { + sm.codec = codec + } +} + // StorageManager encapsulates pebble.DB. // It is to provide file system access, simplify synchronization and enable underlying db swaps. // It assumes exclusive access to pebble DB at storageDir. @@ -59,6 +67,8 @@ type StorageManager struct { storage *Storage rw *ShardedReadWriter + codec Codec + // mu guards db, storage, and rw swaps. mu sync.RWMutex // subscriberPosMu protects the subscriber file from concurrent RW. @@ -70,11 +80,15 @@ type StorageManager struct { } // NewStorageManager returns a new StorageManager with pebble DB at storageDir. -func NewStorageManager(storageDir string) (*StorageManager, error) { +func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*StorageManager, error) { sm := &StorageManager{ storageDir: storageDir, runCh: make(chan struct{}, 1), logger: logp.NewLogger(logs.Sampling), + codec: ProtobufCodec{}, + } + for _, opt := range opts { + opt(sm) } err := sm.reset() if err != nil { diff --git a/x-pack/apm-server/sampling/eventstorage/storage_test.go b/x-pack/apm-server/sampling/eventstorage/storage_test.go index 3f35924965e..93cdbb8ab88 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_test.go @@ -6,317 +6,300 @@ package eventstorage_test import ( "testing" - "time" - "github.com/dgraph-io/badger/v2" - "github.com/gofrs/uuid/v5" - "github.com/google/go-cmp/cmp" - "github.com/google/go-cmp/cmp/cmpopts" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "google.golang.org/protobuf/proto" - "google.golang.org/protobuf/testing/protocmp" - - "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) -func TestWriteEvents(t *testing.T) { - // Run two tests: - // - 1 transaction and 1 span - // - 1 transaction and 100 spans - // - // The latter test will cause ReadTraceEvents to implicitly call flush. - t.Run("no_flush", func(t *testing.T) { - testWriteEvents(t, 1) - }) - t.Run("implicit_flush", func(t *testing.T) { - testWriteEvents(t, 100) - }) -} - -func testWriteEvents(t *testing.T, numSpans int) { - db := newBadgerDB(t, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - readWriter := store.NewShardedReadWriter() - defer readWriter.Close() - - beforeWrite := time.Now() - traceID := uuid.Must(uuid.NewV4()).String() - transactionID := uuid.Must(uuid.NewV4()).String() - transaction := modelpb.APMEvent{ - Transaction: &modelpb.Transaction{Id: transactionID}, - } - wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 0, - } - assert.NoError(t, readWriter.WriteTraceEvent(traceID, transactionID, &transaction, wOpts)) - - var spanEvents []*modelpb.APMEvent - for i := 0; i < numSpans; i++ { - spanID := uuid.Must(uuid.NewV4()).String() - span := modelpb.APMEvent{ - Span: &modelpb.Span{Id: spanID}, - } - assert.NoError(t, readWriter.WriteTraceEvent(traceID, spanID, &span, wOpts)) - spanEvents = append(spanEvents, &span) - } - afterWrite := time.Now() - - // We can read our writes without flushing. - var batch modelpb.Batch - assert.NoError(t, readWriter.ReadTraceEvents(traceID, &batch)) - spanEvents = append(spanEvents, &transaction) - assert.Empty(t, cmp.Diff(modelpb.Batch(spanEvents), batch, - cmpopts.SortSlices(func(e1 *modelpb.APMEvent, e2 *modelpb.APMEvent) bool { - return e1.GetSpan().GetId() < e2.GetSpan().GetId() - }), - protocmp.Transform()), - ) - - // Flush in order for the writes to be visible to other readers. - assert.NoError(t, readWriter.Flush()) - - var recorded modelpb.Batch - assert.NoError(t, db.View(func(txn *badger.Txn) error { - iter := txn.NewIterator(badger.IteratorOptions{ - Prefix: []byte(traceID), - }) - defer iter.Close() - for iter.Rewind(); iter.Valid(); iter.Next() { - item := iter.Item() - expiresAt := item.ExpiresAt() - expiryTime := time.Unix(int64(expiresAt), 0) - - // The expiry time should be somewhere between when we - // started and finished writing + the TTL. The expiry time - // is recorded as seconds since the Unix epoch, hence the - // truncation. - lowerBound := beforeWrite.Add(wOpts.TTL).Truncate(time.Second) - upperBound := afterWrite.Add(wOpts.TTL).Truncate(time.Second) - assert.Condition(t, func() bool { - return !lowerBound.After(expiryTime) - }, "expiry time %s is before %s", expiryTime, lowerBound) - assert.Condition(t, func() bool { - return !expiryTime.After(upperBound) - }, "expiry time %s is after %s", expiryTime, upperBound) - - var event modelpb.APMEvent - require.Equal(t, "e", string(item.UserMeta())) - assert.NoError(t, item.Value(func(data []byte) error { - return proto.Unmarshal(data, &event) - })) - recorded = append(recorded, &event) - } - return nil - })) - assert.Empty(t, cmp.Diff(batch, recorded, protocmp.Transform())) -} - -func TestWriteTraceSampled(t *testing.T) { - db := newBadgerDB(t, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - readWriter := store.NewShardedReadWriter() - defer readWriter.Close() - wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 0, - } - - before := time.Now() - assert.NoError(t, readWriter.WriteTraceSampled("sampled_trace_id", true, wOpts)) - assert.NoError(t, readWriter.WriteTraceSampled("unsampled_trace_id", false, wOpts)) - - // We can read our writes without flushing. - isSampled, err := readWriter.IsTraceSampled("sampled_trace_id") - assert.NoError(t, err) - assert.True(t, isSampled) - - // Flush in order for the writes to be visible to other readers. - assert.NoError(t, readWriter.Flush()) - - sampled := make(map[string]bool) - assert.NoError(t, db.View(func(txn *badger.Txn) error { - iter := txn.NewIterator(badger.IteratorOptions{}) - defer iter.Close() - for iter.Rewind(); iter.Valid(); iter.Next() { - item := iter.Item() - expiresAt := item.ExpiresAt() - expiryTime := time.Unix(int64(expiresAt), 0) - assert.Condition(t, func() bool { - return !before.After(expiryTime) && !expiryTime.After(before.Add(wOpts.TTL)) - }) - - key := string(item.Key()) - switch meta := item.UserMeta(); meta { - case 's': - sampled[key] = true - case 'u': - sampled[key] = false - default: - t.Fatalf("invalid meta %q", meta) - } - assert.Zero(t, item.ValueSize()) - } - return nil - })) - assert.Equal(t, map[string]bool{ - "sampled_trace_id": true, - "unsampled_trace_id": false, - }, sampled) -} - -func TestReadTraceEvents(t *testing.T) { - db := newBadgerDB(t, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - - traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} - require.NoError(t, db.Update(func(txn *badger.Txn) error { - key := append(traceID[:], ":12345678"...) - value, err := proto.Marshal(&modelpb.APMEvent{Transaction: &modelpb.Transaction{Name: "transaction"}}) - if err != nil { - return err - } - if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { - return err - } - - key = append(traceID[:], ":87654321"...) - value, err = proto.Marshal(&modelpb.APMEvent{Span: &modelpb.Span{Name: "span"}}) - if err != nil { - return err - } - if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { - return err - } - - // Write an entry with the trace ID as a prefix, but with no - // proceeding colon, causing it to be ignored. - key = append(traceID[:], "nocolon"...) - value = []byte(`not-protobuf`) - if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { - return err - } - - // Write an entry with an unknown meta value. It will be ignored. - key = append(traceID[:], ":11111111"...) - value = []byte(`not-protobuf`) - if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('?')); err != nil { - return err - } - return nil - })) - - reader := store.NewShardedReadWriter() - defer reader.Close() - - var events modelpb.Batch - assert.NoError(t, reader.ReadTraceEvents(string(traceID[:]), &events)) - assert.Empty(t, cmp.Diff(modelpb.Batch{ - {Transaction: &modelpb.Transaction{Name: "transaction"}}, - {Span: &modelpb.Span{Name: "span"}}, - }, events, protocmp.Transform())) -} - -func TestReadTraceEventsDecodeError(t *testing.T) { - db := newBadgerDB(t, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - - traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} - require.NoError(t, db.Update(func(txn *badger.Txn) error { - key := append(traceID[:], ":12345678"...) - value := []byte(`wat`) - if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { - return err - } - return nil - })) - - reader := store.NewShardedReadWriter() - defer reader.Close() - - var events modelpb.Batch - err := reader.ReadTraceEvents(string(traceID[:]), &events) - assert.Error(t, err) -} - -func TestIsTraceSampled(t *testing.T) { - db := newBadgerDB(t, badgerOptions) - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - - require.NoError(t, db.Update(func(txn *badger.Txn) error { - if err := txn.SetEntry(badger.NewEntry([]byte("sampled_trace_id"), nil).WithMeta('s')); err != nil { - return err - } - if err := txn.SetEntry(badger.NewEntry([]byte("unsampled_trace_id"), nil).WithMeta('u')); err != nil { - return err - } - return nil - })) - - reader := store.NewShardedReadWriter() - defer reader.Close() - - sampled, err := reader.IsTraceSampled("sampled_trace_id") - assert.NoError(t, err) - assert.True(t, sampled) - - sampled, err = reader.IsTraceSampled("unsampled_trace_id") - assert.NoError(t, err) - assert.False(t, sampled) - - _, err = reader.IsTraceSampled("unknown_trace_id") - assert.Equal(t, err, eventstorage.ErrNotFound) -} - -func TestStorageLimit(t *testing.T) { - tempdir := t.TempDir() - opts := func() badger.Options { - opts := badgerOptions() - opts = opts.WithInMemory(false) - opts = opts.WithDir(tempdir).WithValueDir(tempdir) - return opts - } - - // Open and close the database to create a non-empty value log file, - // which will cause writes below to fail due to the storage limit being - // exceeded. We would otherwise have to rely on Badger's one minute - // timer to refresh the size. - db := newBadgerDB(t, opts) - db.Close() - db = newBadgerDB(t, opts) - - store := eventstorage.New(db, eventstorage.ProtobufCodec{}) - readWriter := store.NewReadWriter() - defer readWriter.Close() - - traceID := uuid.Must(uuid.NewV4()).String() - transactionID := uuid.Must(uuid.NewV4()).String() - transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} - err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, eventstorage.WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 1, - }) - assert.ErrorIs(t, err, eventstorage.ErrLimitReached) - - // Assert the stored write has been discarded. - var batch modelpb.Batch - readWriter.ReadTraceEvents(traceID, &batch) - assert.Equal(t, 0, len(batch)) -} - -func badgerOptions() badger.Options { - return badger.DefaultOptions("").WithInMemory(true).WithLogger(nil) -} - -type badgerOptionsFunc func() badger.Options - -func newBadgerDB(tb testing.TB, badgerOptions badgerOptionsFunc) *badger.DB { - db, err := badger.Open(badgerOptions()) +//func TestWriteEvents(t *testing.T) { +// // Run two tests: +// // - 1 transaction and 1 span +// // - 1 transaction and 100 spans +// // +// // The latter test will cause ReadTraceEvents to implicitly call flush. +// t.Run("no_flush", func(t *testing.T) { +// testWriteEvents(t, 1) +// }) +// t.Run("implicit_flush", func(t *testing.T) { +// testWriteEvents(t, 100) +// }) +//} +// +//func testWriteEvents(t *testing.T, numSpans int) { +// db := newBadgerDB(t, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// readWriter := store.NewShardedReadWriter() +// defer readWriter.Close() +// +// beforeWrite := time.Now() +// traceID := uuid.Must(uuid.NewV4()).String() +// transactionID := uuid.Must(uuid.NewV4()).String() +// transaction := modelpb.APMEvent{ +// Transaction: &modelpb.Transaction{Id: transactionID}, +// } +// wOpts := eventstorage.WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 0, +// } +// assert.NoError(t, readWriter.WriteTraceEvent(traceID, transactionID, &transaction, wOpts)) +// +// var spanEvents []*modelpb.APMEvent +// for i := 0; i < numSpans; i++ { +// spanID := uuid.Must(uuid.NewV4()).String() +// span := modelpb.APMEvent{ +// Span: &modelpb.Span{Id: spanID}, +// } +// assert.NoError(t, readWriter.WriteTraceEvent(traceID, spanID, &span, wOpts)) +// spanEvents = append(spanEvents, &span) +// } +// afterWrite := time.Now() +// +// // We can read our writes without flushing. +// var batch modelpb.Batch +// assert.NoError(t, readWriter.ReadTraceEvents(traceID, &batch)) +// spanEvents = append(spanEvents, &transaction) +// assert.Empty(t, cmp.Diff(modelpb.Batch(spanEvents), batch, +// cmpopts.SortSlices(func(e1 *modelpb.APMEvent, e2 *modelpb.APMEvent) bool { +// return e1.GetSpan().GetId() < e2.GetSpan().GetId() +// }), +// protocmp.Transform()), +// ) +// +// // Flush in order for the writes to be visible to other readers. +// assert.NoError(t, readWriter.Flush()) +// +// var recorded modelpb.Batch +// assert.NoError(t, db.View(func(txn *badger.Txn) error { +// iter := txn.NewIterator(badger.IteratorOptions{ +// Prefix: []byte(traceID), +// }) +// defer iter.Close() +// for iter.Rewind(); iter.Valid(); iter.Next() { +// item := iter.Item() +// expiresAt := item.ExpiresAt() +// expiryTime := time.Unix(int64(expiresAt), 0) +// +// // The expiry time should be somewhere between when we +// // started and finished writing + the TTL. The expiry time +// // is recorded as seconds since the Unix epoch, hence the +// // truncation. +// lowerBound := beforeWrite.Add(wOpts.TTL).Truncate(time.Second) +// upperBound := afterWrite.Add(wOpts.TTL).Truncate(time.Second) +// assert.Condition(t, func() bool { +// return !lowerBound.After(expiryTime) +// }, "expiry time %s is before %s", expiryTime, lowerBound) +// assert.Condition(t, func() bool { +// return !expiryTime.After(upperBound) +// }, "expiry time %s is after %s", expiryTime, upperBound) +// +// var event modelpb.APMEvent +// require.Equal(t, "e", string(item.UserMeta())) +// assert.NoError(t, item.Value(func(data []byte) error { +// return proto.Unmarshal(data, &event) +// })) +// recorded = append(recorded, &event) +// } +// return nil +// })) +// assert.Empty(t, cmp.Diff(batch, recorded, protocmp.Transform())) +//} +// +//func TestWriteTraceSampled(t *testing.T) { +// db := newBadgerDB(t, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// readWriter := store.NewShardedReadWriter() +// defer readWriter.Close() +// wOpts := eventstorage.WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 0, +// } +// +// before := time.Now() +// assert.NoError(t, readWriter.WriteTraceSampled("sampled_trace_id", true, wOpts)) +// assert.NoError(t, readWriter.WriteTraceSampled("unsampled_trace_id", false, wOpts)) +// +// // We can read our writes without flushing. +// isSampled, err := readWriter.IsTraceSampled("sampled_trace_id") +// assert.NoError(t, err) +// assert.True(t, isSampled) +// +// // Flush in order for the writes to be visible to other readers. +// assert.NoError(t, readWriter.Flush()) +// +// sampled := make(map[string]bool) +// assert.NoError(t, db.View(func(txn *badger.Txn) error { +// iter := txn.NewIterator(badger.IteratorOptions{}) +// defer iter.Close() +// for iter.Rewind(); iter.Valid(); iter.Next() { +// item := iter.Item() +// expiresAt := item.ExpiresAt() +// expiryTime := time.Unix(int64(expiresAt), 0) +// assert.Condition(t, func() bool { +// return !before.After(expiryTime) && !expiryTime.After(before.Add(wOpts.TTL)) +// }) +// +// key := string(item.Key()) +// switch meta := item.UserMeta(); meta { +// case 's': +// sampled[key] = true +// case 'u': +// sampled[key] = false +// default: +// t.Fatalf("invalid meta %q", meta) +// } +// assert.Zero(t, item.ValueSize()) +// } +// return nil +// })) +// assert.Equal(t, map[string]bool{ +// "sampled_trace_id": true, +// "unsampled_trace_id": false, +// }, sampled) +//} +// +//func TestReadTraceEvents(t *testing.T) { +// db := newBadgerDB(t, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// +// traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} +// require.NoError(t, db.Update(func(txn *badger.Txn) error { +// key := append(traceID[:], ":12345678"...) +// value, err := proto.Marshal(&modelpb.APMEvent{Transaction: &modelpb.Transaction{Name: "transaction"}}) +// if err != nil { +// return err +// } +// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { +// return err +// } +// +// key = append(traceID[:], ":87654321"...) +// value, err = proto.Marshal(&modelpb.APMEvent{Span: &modelpb.Span{Name: "span"}}) +// if err != nil { +// return err +// } +// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { +// return err +// } +// +// // Write an entry with the trace ID as a prefix, but with no +// // proceeding colon, causing it to be ignored. +// key = append(traceID[:], "nocolon"...) +// value = []byte(`not-protobuf`) +// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { +// return err +// } +// +// // Write an entry with an unknown meta value. It will be ignored. +// key = append(traceID[:], ":11111111"...) +// value = []byte(`not-protobuf`) +// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('?')); err != nil { +// return err +// } +// return nil +// })) +// +// reader := store.NewShardedReadWriter() +// defer reader.Close() +// +// var events modelpb.Batch +// assert.NoError(t, reader.ReadTraceEvents(string(traceID[:]), &events)) +// assert.Empty(t, cmp.Diff(modelpb.Batch{ +// {Transaction: &modelpb.Transaction{Name: "transaction"}}, +// {Span: &modelpb.Span{Name: "span"}}, +// }, events, protocmp.Transform())) +//} +// +//func TestReadTraceEventsDecodeError(t *testing.T) { +// db := newBadgerDB(t, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// +// traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} +// require.NoError(t, db.Update(func(txn *badger.Txn) error { +// key := append(traceID[:], ":12345678"...) +// value := []byte(`wat`) +// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { +// return err +// } +// return nil +// })) +// +// reader := store.NewShardedReadWriter() +// defer reader.Close() +// +// var events modelpb.Batch +// err := reader.ReadTraceEvents(string(traceID[:]), &events) +// assert.Error(t, err) +//} +// +//func TestIsTraceSampled(t *testing.T) { +// db := newBadgerDB(t, badgerOptions) +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// +// require.NoError(t, db.Update(func(txn *badger.Txn) error { +// if err := txn.SetEntry(badger.NewEntry([]byte("sampled_trace_id"), nil).WithMeta('s')); err != nil { +// return err +// } +// if err := txn.SetEntry(badger.NewEntry([]byte("unsampled_trace_id"), nil).WithMeta('u')); err != nil { +// return err +// } +// return nil +// })) +// +// reader := store.NewShardedReadWriter() +// defer reader.Close() +// +// sampled, err := reader.IsTraceSampled("sampled_trace_id") +// assert.NoError(t, err) +// assert.True(t, sampled) +// +// sampled, err = reader.IsTraceSampled("unsampled_trace_id") +// assert.NoError(t, err) +// assert.False(t, sampled) +// +// _, err = reader.IsTraceSampled("unknown_trace_id") +// assert.Equal(t, err, eventstorage.ErrNotFound) +//} +// +//func TestStorageLimit(t *testing.T) { +// tempdir := t.TempDir() +// opts := func() badger.Options { +// opts := badgerOptions() +// opts = opts.WithInMemory(false) +// opts = opts.WithDir(tempdir).WithValueDir(tempdir) +// return opts +// } +// +// // Open and close the database to create a non-empty value log file, +// // which will cause writes below to fail due to the storage limit being +// // exceeded. We would otherwise have to rely on Badger's one minute +// // timer to refresh the size. +// db := newBadgerDB(t, opts) +// db.Close() +// db = newBadgerDB(t, opts) +// +// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) +// readWriter := store.NewReadWriter() +// defer readWriter.Close() +// +// traceID := uuid.Must(uuid.NewV4()).String() +// transactionID := uuid.Must(uuid.NewV4()).String() +// transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} +// err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, eventstorage.WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 1, +// }) +// assert.ErrorIs(t, err, eventstorage.ErrLimitReached) +// +// // Assert the stored write has been discarded. +// var batch modelpb.Batch +// readWriter.ReadTraceEvents(traceID, &batch) +// assert.Equal(t, 0, len(batch)) +//} + +func newStorageManager(tb testing.TB, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { + sm, err := eventstorage.NewStorageManager(tb.TempDir(), opts...) if err != nil { panic(err) } - tb.Cleanup(func() { db.Close() }) - return db + tb.Cleanup(func() { sm.Close() }) + return sm } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go b/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go index 8e746aba823..e217875f84b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go @@ -4,92 +4,80 @@ package eventstorage -import ( - "testing" - "time" - - "github.com/dgraph-io/badger/v2" - "github.com/gofrs/uuid/v5" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/elastic/apm-data/model/modelpb" -) - -func newReadWriter(tb testing.TB) *ReadWriter { - tempdir := tb.TempDir() - opts := badger.DefaultOptions("").WithLogger(nil) - opts = opts.WithInMemory(false) - opts = opts.WithDir(tempdir).WithValueDir(tempdir) - - db, err := badger.Open(opts) - if err != nil { - panic(err) - } - tb.Cleanup(func() { db.Close() }) - - store := New(db, ProtobufCodec{}) - readWriter := store.NewReadWriter() - tb.Cleanup(func() { readWriter.Close() }) - - readWriter.lazyInit() - return readWriter -} - -func TestDeleteTraceEvent_ErrTxnTooBig(t *testing.T) { - readWriter := newReadWriter(t) - - traceID, transactionID := writeEvent(t, readWriter) - assert.True(t, eventExists(t, readWriter, traceID, transactionID)) - - fillTxnUntilTxnTooBig(readWriter.txn) - - err := readWriter.DeleteTraceEvent(traceID, transactionID) - assert.NoError(t, err) - - assert.False(t, eventExists(t, readWriter, traceID, transactionID)) -} - -func TestWriteTraceEvent_ErrTxnTooBig(t *testing.T) { - readWriter := newReadWriter(t) - - fillTxnUntilTxnTooBig(readWriter.txn) - - traceID, transactionID := writeEvent(t, readWriter) - assert.True(t, eventExists(t, readWriter, traceID, transactionID)) -} - -func writeEvent(t *testing.T, readWriter *ReadWriter) (traceID, transactionID string) { - traceID = uuid.Must(uuid.NewV4()).String() - transactionID = uuid.Must(uuid.NewV4()).String() - transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} - err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, WriterOpts{ - TTL: time.Minute, - StorageLimitInBytes: 0, - }) - assert.NoError(t, err) - return -} - -func eventExists(t *testing.T, readWriter *ReadWriter, traceID, transactionID string) (ok bool) { - var batch modelpb.Batch - err := readWriter.ReadTraceEvents(traceID, &batch) - require.NoError(t, err) - for _, e := range batch { - if e.GetTransaction().GetId() == transactionID { - ok = true - } - } - return -} - -func fillTxnUntilTxnTooBig(txn *badger.Txn) { - var err error - for { - if err == badger.ErrTxnTooBig { - break - } - entry := badger.NewEntry([]byte{0}, []byte{}) - err = txn.SetEntry(entry) - } -} +//func newReadWriter(tb testing.TB) *ReadWriter { +// tempdir := tb.TempDir() +// opts := badger.DefaultOptions("").WithLogger(nil) +// opts = opts.WithInMemory(false) +// opts = opts.WithDir(tempdir).WithValueDir(tempdir) +// +// db, err := badger.Open(opts) +// if err != nil { +// panic(err) +// } +// tb.Cleanup(func() { db.Close() }) +// +// store := New(db, ProtobufCodec{}) +// readWriter := store.NewReadWriter() +// tb.Cleanup(func() { readWriter.Close() }) +// +// readWriter.lazyInit() +// return readWriter +//} +// +//func TestDeleteTraceEvent_ErrTxnTooBig(t *testing.T) { +// readWriter := newReadWriter(t) +// +// traceID, transactionID := writeEvent(t, readWriter) +// assert.True(t, eventExists(t, readWriter, traceID, transactionID)) +// +// fillTxnUntilTxnTooBig(readWriter.txn) +// +// err := readWriter.DeleteTraceEvent(traceID, transactionID) +// assert.NoError(t, err) +// +// assert.False(t, eventExists(t, readWriter, traceID, transactionID)) +//} +// +//func TestWriteTraceEvent_ErrTxnTooBig(t *testing.T) { +// readWriter := newReadWriter(t) +// +// fillTxnUntilTxnTooBig(readWriter.txn) +// +// traceID, transactionID := writeEvent(t, readWriter) +// assert.True(t, eventExists(t, readWriter, traceID, transactionID)) +//} +// +//func writeEvent(t *testing.T, readWriter *ReadWriter) (traceID, transactionID string) { +// traceID = uuid.Must(uuid.NewV4()).String() +// transactionID = uuid.Must(uuid.NewV4()).String() +// transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} +// err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, WriterOpts{ +// TTL: time.Minute, +// StorageLimitInBytes: 0, +// }) +// assert.NoError(t, err) +// return +//} +// +//func eventExists(t *testing.T, readWriter *ReadWriter, traceID, transactionID string) (ok bool) { +// var batch modelpb.Batch +// err := readWriter.ReadTraceEvents(traceID, &batch) +// require.NoError(t, err) +// for _, e := range batch { +// if e.GetTransaction().GetId() == transactionID { +// ok = true +// } +// } +// return +//} +// +//func fillTxnUntilTxnTooBig(txn *badger.Txn) { +// var err error +// for { +// if err == badger.ErrTxnTooBig { +// break +// } +// entry := badger.NewEntry([]byte{0}, []byte{}) +// err = txn.SetEntry(entry) +// } +//} From e7ac29620646dcfb36e7f59b760380f71226ce67 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 16:49:31 +0000 Subject: [PATCH 040/184] Use configured codec --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index b43f7951ce1..2de80270390 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -109,7 +109,7 @@ func (s *StorageManager) reset() error { return err } s.decisionDB = decisionDB - s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, ProtobufCodec{}) + s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, s.codec) s.rw = s.storage.NewShardedReadWriter() return nil } From c128e75cdea6d17c0d2b8b73acf4cb74a036bb23 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 21:23:07 +0000 Subject: [PATCH 041/184] Use prefix bloom filter --- x-pack/apm-server/sampling/eventstorage/pebble.go | 11 +++++++++++ x-pack/apm-server/sampling/eventstorage/storage.go | 6 +++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 375b8402ec6..e26e43cbc7a 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,6 +1,7 @@ package eventstorage import ( + "bytes" "path/filepath" "github.com/cockroachdb/pebble" @@ -44,6 +45,15 @@ const ( dbCommitThresholdBytes = 8000 << 10 // 8000KB ) +func eventComparer() *pebble.Comparer { + comparer := *pebble.DefaultComparer + // Required for prefix bloom filter + comparer.Split = func(k []byte) int { + return bytes.IndexByte(k, ':') + } + return &comparer +} + func OpenPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "event"), &pebble.Options{ // FIXME: Specify FormatMajorVersion to use value blocks? @@ -58,6 +68,7 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { FilterType: pebble.TableFilter, }, }, + Comparer: eventComparer(), }) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 0ecdd7f472a..4311b091bb2 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -273,7 +273,11 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error return err } defer iter.Close() - for iter.First(); iter.Valid(); iter.Next() { + // SeekPrefixGE uses the prefix bloom filter, so that a miss will be much faster + if valid := iter.SeekPrefixGE(append([]byte(traceID), ':')); !valid { + return nil + } + for ; iter.Valid(); iter.Next() { event := &modelpb.APMEvent{} data, err := iter.ValueAndErr() if err != nil { From 86975b84c896a4fca950e8b558904f110f89ade6 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 21:49:30 +0000 Subject: [PATCH 042/184] Prefix bloom can only be tested when db flushes to disk --- .../eventstorage/storage_bench_test.go | 23 +++++++++++++++---- .../sampling/eventstorage/storage_manager.go | 16 +++++++++++++ 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index b0b9af31e20..a8842856e52 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -165,7 +165,7 @@ func BenchmarkReadEventsHit(b *testing.B) { // And causes next iteration setup to take a very long time. const txnCountInTrace = 5 - test := func(b *testing.B, bigTX bool) { + test := func(b *testing.B, bigTX bool, reloadDB bool) { for _, hit := range []bool{false, true} { b.Run(fmt.Sprintf("hit=%v", hit), func(b *testing.B) { sm := newStorageManager(b) @@ -202,6 +202,17 @@ func BenchmarkReadEventsHit(b *testing.B) { b.Fatal(err) } + readWriter.Close() + + if reloadDB { + if err := sm.Reload(); err != nil { + b.Fatal(err) + } + } + + readWriter = sm.NewBypassReadWriter() + defer readWriter.Close() + b.ResetTimer() var batch modelpb.Batch for i := 0; i < b.N; i++ { @@ -221,9 +232,13 @@ func BenchmarkReadEventsHit(b *testing.B) { } } - for _, bigTX := range []bool{true, false} { - b.Run(fmt.Sprintf("bigTX=%v", bigTX), func(b *testing.B) { - test(b, bigTX) + for _, reloadDB := range []bool{false, true} { + b.Run(fmt.Sprintf("reloadDB=%v", reloadDB), func(b *testing.B) { + for _, bigTX := range []bool{true, false} { + b.Run(fmt.Sprintf("bigTX=%v", bigTX), func(b *testing.B) { + test(b, bigTX, reloadDB) + }) + } }) } } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 2de80270390..a87d5e1d468 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -123,12 +123,28 @@ func (s *StorageManager) Size() (lsm, vlog int64) { func (s *StorageManager) Close() error { s.mu.RLock() defer s.mu.RUnlock() + return s.close() +} + +func (s *StorageManager) close() error { s.rw.Close() _ = s.db.Close() // FIXME _ = s.decisionDB.Close() // FIXME return nil } +// Reload flushes out pending disk writes to disk by reloading the database. +// It does not flush uncommitted writes. +// For testing only. +func (s *StorageManager) Reload() error { + s.mu.RLock() + defer s.mu.RUnlock() + if err := s.close(); err != nil { + return err + } + return s.reset() +} + // 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 { return nil From add0f9e9c7c03c23001b857f3613185ae4b4584c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 23:51:16 +0000 Subject: [PATCH 043/184] Add decision db flush --- .../sampling/eventstorage/pebble.go | 1 + .../sampling/eventstorage/storage.go | 47 ++++++++++++------- 2 files changed, 32 insertions(+), 16 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e26e43cbc7a..9ed71dbdba6 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -37,6 +37,7 @@ const ( // rather than the length of data slice backing the batch. pebbleMemTableSize = 32 << 20 // 32MB + // FIXME: fix comment // 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 diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 4311b091bb2..b4e3836dca1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -127,7 +127,10 @@ func (rw *ReadWriter) lazyInit() { func (rw *ReadWriter) decisionLazyInit() { if rw.decisionBatch == nil { - rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch() //FIXME: tuning + rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch( + pebble.WithInitialSizeBytes(initialPebbleBatchSize), + pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), + ) //FIXME: tuning } } @@ -151,27 +154,34 @@ func (rw *ReadWriter) Close() { // may be lost. func (rw *ReadWriter) Flush() error { const flushErrFmt = "failed to flush pending writes: %w" - var err error + err := errors.Join(rw.flushBatch(), rw.flushDecisionBatch()) + + //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 +} + +func (rw *ReadWriter) flushBatch() (err error) { if rw.batch != nil { err = rw.batch.Commit(pebble.NoSync) rw.batch.Close() rw.batch = nil } + return +} +func (rw *ReadWriter) flushDecisionBatch() (err error) { if rw.decisionBatch != nil { - err = errors.Join(err, rw.decisionBatch.Commit(pebble.NoSync)) + err = rw.decisionBatch.Commit(pebble.NoSync) rw.decisionBatch.Close() rw.decisionBatch = nil } - - //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 + return } // WriteTraceSampled records the tail-sampling decision for the given trace ID. @@ -182,7 +192,14 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write if sampled { meta = entryMetaTraceSampled } - return rw.decisionBatch.Set([]byte(traceID), []byte{meta}, pebble.NoSync) + err := rw.decisionBatch.Set([]byte(traceID), []byte{meta}, pebble.NoSync) + if err != nil { + return err + } + if rw.decisionBatch.Len() >= dbCommitThresholdBytes { + return rw.flushDecisionBatch() + } + return nil } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -231,9 +248,7 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { } if rw.batch.Len() >= dbCommitThresholdBytes { - if err := rw.Flush(); err != nil { - return err - } + return rw.flushBatch() } return nil } From f4049aff0e44ae44fab215d28759392cfbf8bbe8 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 20 Jan 2025 23:51:38 +0000 Subject: [PATCH 044/184] Set decision db block size to 2KB --- 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 9ed71dbdba6..10002b60514 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -81,7 +81,7 @@ func OpenSamplingDecisionPebble(storageDir string) (*pebble.DB, error) { //MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { - //BlockSize: 16 << 10, + BlockSize: 2 << 10, Compression: pebble.NoCompression, FilterPolicy: bloom.FilterPolicy(10), FilterType: pebble.TableFilter, From 0c8d662658fc8d96bae0f9bda94ebafedb17d2fd Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 00:03:52 +0000 Subject: [PATCH 045/184] Change sampling decision flush threshold --- 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 b4e3836dca1..f000e14976e 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -196,7 +196,7 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write if err != nil { return err } - if rw.decisionBatch.Len() >= dbCommitThresholdBytes { + if rw.decisionBatch.Len() >= 2<<20 { // FIXME: magic number return rw.flushDecisionBatch() } return nil From 6ba50f5955985c3f9fe36e7d3e98c39fa5ac76e0 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 00:09:23 +0000 Subject: [PATCH 046/184] Disable decision batch --- .../sampling/eventstorage/storage.go | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index f000e14976e..9ba8abf79b8 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -127,10 +127,10 @@ func (rw *ReadWriter) lazyInit() { func (rw *ReadWriter) decisionLazyInit() { if rw.decisionBatch == nil { - rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch( - pebble.WithInitialSizeBytes(initialPebbleBatchSize), - pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), - ) //FIXME: tuning + //rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch( + // pebble.WithInitialSizeBytes(initialPebbleBatchSize), + // pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), + //) //FIXME: tuning } } @@ -177,9 +177,9 @@ func (rw *ReadWriter) flushBatch() (err error) { func (rw *ReadWriter) flushDecisionBatch() (err error) { if rw.decisionBatch != nil { - err = rw.decisionBatch.Commit(pebble.NoSync) - rw.decisionBatch.Close() - rw.decisionBatch = nil + //err = rw.decisionBatch.Commit(pebble.NoSync) + //rw.decisionBatch.Close() + //rw.decisionBatch = nil } return } @@ -192,13 +192,15 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write if sampled { meta = entryMetaTraceSampled } - err := rw.decisionBatch.Set([]byte(traceID), []byte{meta}, pebble.NoSync) + //FIXME not using batch + //err := rw.decisionBatch.Set([]byte(traceID), []byte{meta}, pebble.NoSync) + err := rw.s.decisionDB.(*wrappedDB).db.Set([]byte(traceID), []byte{meta}, pebble.NoSync) if err != nil { return err } - if rw.decisionBatch.Len() >= 2<<20 { // FIXME: magic number - return rw.flushDecisionBatch() - } + //if rw.decisionBatch.Len() >= 2<<20 { // FIXME: magic number + // return rw.flushDecisionBatch() + //} return nil } @@ -212,7 +214,10 @@ 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.decisionBatch.Get([]byte(traceID)) + + //FIXME not using batch + //item, closer, err := rw.decisionBatch.Get([]byte(traceID)) + item, closer, err := rw.s.decisionDB.(*wrappedDB).db.Get([]byte(traceID)) if err == pebble.ErrNotFound { return false, ErrNotFound } From 779edceb27d8bc3f0939a1607b8b23b457643b2b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 00:21:14 +0000 Subject: [PATCH 047/184] Disable batch for events --- .../sampling/eventstorage/storage.go | 44 ++++++++++++------- 1 file changed, 29 insertions(+), 15 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 9ba8abf79b8..46ab1bb7e90 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -118,10 +118,10 @@ type ReadWriter struct { func (rw *ReadWriter) lazyInit() { if rw.batch == nil { - rw.batch = rw.s.db.NewIndexedBatch( - pebble.WithInitialSizeBytes(initialPebbleBatchSize), - pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), - ) + //rw.batch = rw.s.db.NewIndexedBatch( + // pebble.WithInitialSizeBytes(initialPebbleBatchSize), + // pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), + //) } } @@ -168,9 +168,9 @@ func (rw *ReadWriter) Flush() error { func (rw *ReadWriter) flushBatch() (err error) { if rw.batch != nil { - err = rw.batch.Commit(pebble.NoSync) - rw.batch.Close() - rw.batch = nil + //err = rw.batch.Commit(pebble.NoSync) + //rw.batch.Close() + //rw.batch = nil } return } @@ -247,14 +247,18 @@ 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, data, pebble.NoSync); err != nil { - return err - } - if rw.batch.Len() >= dbCommitThresholdBytes { - return rw.flushBatch() + // FIXME: disabled batch + if err := rw.s.db.(*wrappedDB).db.Set(key, data, pebble.NoSync); err != nil { + return err } + //if err := rw.batch.Set(key, data, pebble.NoSync); err != nil { + // return err + //} + // + //if rw.batch.Len() >= dbCommitThresholdBytes { + // return rw.flushBatch() + //} return nil } @@ -269,7 +273,12 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { buf.WriteString(id) key := buf.Bytes() - err := rw.batch.Delete(key, pebble.NoSync) + // FIXME: disabled batch + //err := rw.batch.Delete(key, pebble.NoSync) + //if err != nil { + // return err + //} + err := rw.s.db.(*wrappedDB).db.Delete(key, pebble.NoSync) if err != nil { return err } @@ -285,7 +294,12 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { rw.lazyInit() - iter, err := rw.batch.NewIter(&pebble.IterOptions{ + // FIXME: disabled batch + //iter, err := rw.batch.NewIter(&pebble.IterOptions{ + // LowerBound: append([]byte(traceID), ':'), + // UpperBound: append([]byte(traceID), ';'), // This is a hack to stop before next ID + //}) + iter, err := rw.s.db.(*wrappedDB).db.NewIter(&pebble.IterOptions{ LowerBound: append([]byte(traceID), ':'), UpperBound: append([]byte(traceID), ';'), // This is a hack to stop before next ID }) From 01f22763938aabd07f3920a81177ecd42023d2cb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 12:04:55 +0000 Subject: [PATCH 048/184] Remove all batch code --- .../sampling/eventstorage/storage.go | 114 ++---------------- .../sampling/eventstorage/storage_manager.go | 23 +++- 2 files changed, 28 insertions(+), 109 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 46ab1bb7e90..6d05e0940ab 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" + "io" "sync/atomic" "time" @@ -36,9 +37,11 @@ var ( ) type db interface { - NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch Size() (lsm, vlog int64) - //Close() error + Get(key []byte) ([]byte, io.Closer, error) + Set(key, value []byte, opts *pebble.WriteOptions) error + Delete(key []byte, opts *pebble.WriteOptions) error + NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) } // Storage provides storage for sampled transactions and spans, @@ -84,7 +87,6 @@ func (s *Storage) NewReadWriter() *ReadWriter { //s.pendingSize.Add(baseTransactionSize) return &ReadWriter{ s: s, - //txn: nil, // lazy init to avoid deadlock in storage manager //pendingSize: baseTransactionSize, } } @@ -116,36 +118,11 @@ type ReadWriter struct { pendingSize int64 } -func (rw *ReadWriter) lazyInit() { - if rw.batch == nil { - //rw.batch = rw.s.db.NewIndexedBatch( - // pebble.WithInitialSizeBytes(initialPebbleBatchSize), - // pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), - //) - } -} - -func (rw *ReadWriter) decisionLazyInit() { - if rw.decisionBatch == nil { - //rw.decisionBatch = rw.s.decisionDB.NewIndexedBatch( - // pebble.WithInitialSizeBytes(initialPebbleBatchSize), - // pebble.WithMaxRetainedSizeBytes(maxRetainedPebbleBatchSize), - //) //FIXME: tuning - } -} - // 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() { - if rw.batch != nil { - rw.batch.Close() - } - if rw.decisionBatch != nil { - rw.decisionBatch.Close() - } -} +func (rw *ReadWriter) Close() {} // Flush waits for preceding writes to be committed to storage. // @@ -153,54 +130,19 @@ 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 := errors.Join(rw.flushBatch(), rw.flushDecisionBatch()) - - //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 } -func (rw *ReadWriter) flushBatch() (err error) { - if rw.batch != nil { - //err = rw.batch.Commit(pebble.NoSync) - //rw.batch.Close() - //rw.batch = nil - } - return -} - -func (rw *ReadWriter) flushDecisionBatch() (err error) { - if rw.decisionBatch != nil { - //err = rw.decisionBatch.Commit(pebble.NoSync) - //rw.decisionBatch.Close() - //rw.decisionBatch = nil - } - return -} - // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - rw.decisionLazyInit() - meta := entryMetaTraceUnsampled if sampled { meta = entryMetaTraceSampled } - //FIXME not using batch - //err := rw.decisionBatch.Set([]byte(traceID), []byte{meta}, pebble.NoSync) - err := rw.s.decisionDB.(*wrappedDB).db.Set([]byte(traceID), []byte{meta}, pebble.NoSync) + err := rw.s.decisionDB.Set([]byte(traceID), []byte{meta}, pebble.NoSync) if err != nil { return err } - //if rw.decisionBatch.Len() >= 2<<20 { // FIXME: magic number - // return rw.flushDecisionBatch() - //} return nil } @@ -208,16 +150,12 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - rw.decisionLazyInit() - // 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 - //FIXME not using batch - //item, closer, err := rw.decisionBatch.Get([]byte(traceID)) - item, closer, err := rw.s.decisionDB.(*wrappedDB).db.Get([]byte(traceID)) + item, closer, err := rw.s.decisionDB.Get([]byte(traceID)) if err == pebble.ErrNotFound { return false, ErrNotFound } @@ -228,10 +166,7 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // 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 WriterOpts) error { - rw.lazyInit() - data, err := rw.s.codec.EncodeEvent(event) if err != nil { return err @@ -248,24 +183,14 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. func (rw *ReadWriter) writeEntry(key, data []byte) error { rw.pendingWrites++ - // FIXME: disabled batch - if err := rw.s.db.(*wrappedDB).db.Set(key, data, pebble.NoSync); err != nil { + if err := rw.s.db.Set(key, data, pebble.NoSync); err != nil { return err } - //if err := rw.batch.Set(key, data, pebble.NoSync); err != nil { - // return err - //} - // - //if rw.batch.Len() >= dbCommitThresholdBytes { - // return rw.flushBatch() - //} return nil } // DeleteTraceEvent deletes the trace event from storage. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { - rw.lazyInit() - var buf bytes.Buffer buf.Grow(len(traceID) + 1 + len(id)) buf.WriteString(traceID) @@ -273,33 +198,16 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { buf.WriteString(id) key := buf.Bytes() - // FIXME: disabled batch - //err := rw.batch.Delete(key, pebble.NoSync) - //if err != nil { - // return err - //} - err := rw.s.db.(*wrappedDB).db.Delete(key, pebble.NoSync) + err := rw.s.db.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.lazyInit() - - // FIXME: disabled batch - //iter, err := rw.batch.NewIter(&pebble.IterOptions{ - // LowerBound: append([]byte(traceID), ':'), - // UpperBound: append([]byte(traceID), ';'), // This is a hack to stop before next ID - //}) - iter, err := rw.s.db.(*wrappedDB).db.NewIter(&pebble.IterOptions{ + iter, err := rw.s.db.NewIter(&pebble.IterOptions{ LowerBound: append([]byte(traceID), ':'), UpperBound: append([]byte(traceID), ';'), // This is a hack to stop before next ID }) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index a87d5e1d468..1c64bfbf708 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -6,6 +6,7 @@ package eventstorage import ( "errors" + "io" "os" "path/filepath" "sync" @@ -33,6 +34,22 @@ type wrappedDB struct { db *pebble.DB } +func (w *wrappedDB) Get(key []byte) ([]byte, io.Closer, error) { + return w.db.Get(key) +} + +func (w *wrappedDB) Set(key, value []byte, opts *pebble.WriteOptions) error { + return w.db.Set(key, value, opts) +} + +func (w *wrappedDB) Delete(key []byte, opts *pebble.WriteOptions) error { + return w.db.Delete(key, opts) +} + +func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { + return w.db.NewIter(o) +} + // Size returns the db size func (w *wrappedDB) Size() (lsm, vlog int64) { // FIXME: we may want it to report the sum of 2 dbs @@ -41,12 +58,6 @@ func (w *wrappedDB) Size() (lsm, vlog int64) { return int64(w.db.Metrics().DiskSpaceUsage()), 0 // FIXME } -func (w *wrappedDB) NewIndexedBatch(opts ...pebble.BatchOption) *pebble.Batch { - w.sm.mu.RLock() - defer w.sm.mu.RUnlock() - return w.db.NewIndexedBatch(opts...) -} - type StorageManagerOptions func(*StorageManager) func WithCodec(codec Codec) StorageManagerOptions { From b1adee644d569230d99438cc12a74be983c7459b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 12:16:11 +0000 Subject: [PATCH 049/184] Add fixme --- x-pack/apm-server/sampling/eventstorage/storage.go | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 6d05e0940ab..6146f2d4252 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -191,6 +191,7 @@ func (rw *ReadWriter) writeEntry(key, data []byte) error { // DeleteTraceEvent deletes the trace event from storage. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { + // FIXME: use range delete var buf bytes.Buffer buf.Grow(len(traceID) + 1 + len(id)) buf.WriteString(traceID) From 985951f43a959283a61bcaa4d56325e0c5bb56b3 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 12:23:06 +0000 Subject: [PATCH 050/184] Try bypassing sharded rw --- 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 1c64bfbf708..55aaa5b43b1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -76,7 +76,7 @@ type StorageManager struct { db *pebble.DB decisionDB *pebble.DB storage *Storage - rw *ShardedReadWriter + rw *ReadWriter codec Codec @@ -121,7 +121,7 @@ func (s *StorageManager) reset() error { } s.decisionDB = decisionDB s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, s.codec) - s.rw = s.storage.NewShardedReadWriter() + s.rw = s.storage.NewReadWriter() return nil } From 24d9a9ce84374bb2222a4527ba3fc3a7a9d4a8c9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 12:23:18 +0000 Subject: [PATCH 051/184] Revert "Try bypassing sharded rw" This reverts commit 985951f43a959283a61bcaa4d56325e0c5bb56b3. --- 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 55aaa5b43b1..1c64bfbf708 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -76,7 +76,7 @@ type StorageManager struct { db *pebble.DB decisionDB *pebble.DB storage *Storage - rw *ReadWriter + rw *ShardedReadWriter codec Codec @@ -121,7 +121,7 @@ func (s *StorageManager) reset() error { } s.decisionDB = decisionDB s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, s.codec) - s.rw = s.storage.NewReadWriter() + s.rw = s.storage.NewShardedReadWriter() return nil } From 80c6cfe95ffd7c28ef4a687b402a15df49908300 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 15:04:13 +0000 Subject: [PATCH 052/184] WIP --- .../sampling/eventstorage/storage.go | 40 ++++++---- .../apm-server/sampling/eventstorage/ttlrw.go | 75 +++++++++++++++++++ .../sampling/eventstorage/ttlrw_test.go | 58 ++++++++++++++ x-pack/apm-server/sampling/processor_test.go | 5 +- 4 files changed, 161 insertions(+), 17 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/ttlrw.go create mode 100644 x-pack/apm-server/sampling/eventstorage/ttlrw_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 6146f2d4252..91d1e39243f 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -37,7 +37,7 @@ var ( ) type db interface { - Size() (lsm, vlog int64) + //Size() (lsm, vlog int64) Get(key []byte) ([]byte, io.Closer, error) Set(key, value []byte, opts *pebble.WriteOptions) error Delete(key []byte, opts *pebble.WriteOptions) error @@ -93,6 +93,7 @@ func (s *Storage) NewReadWriter() *ReadWriter { // WriterOpts provides configuration options for writes to storage type WriterOpts struct { + TimeNow func() time.Time TTL time.Duration StorageLimitInBytes int64 } @@ -135,15 +136,14 @@ 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 { - meta := entryMetaTraceUnsampled - if sampled { - meta = entryMetaTraceSampled + var now time.Time + if opts.TimeNow != nil { + now = opts.TimeNow() + } else { + now = time.Now() } - err := rw.s.decisionDB.Set([]byte(traceID), []byte{meta}, pebble.NoSync) - if err != nil { - return err - } - return nil + + return TTLReadWriter{truncatedTime: now.Truncate(opts.TTL), db: rw.s.decisionDB}.WriteTraceSampled(traceID, sampled, opts) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -154,13 +154,19 @@ 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 + now := time.Now() + ttl := time.Minute + sampled, err := TTLReadWriter{truncatedTime: now.Truncate(ttl), db: rw.s.decisionDB}.IsTraceSampled(traceID) + if err == nil { + return sampled, nil + } - item, closer, err := rw.s.decisionDB.Get([]byte(traceID)) - if err == pebble.ErrNotFound { - return false, ErrNotFound + sampled, err = TTLReadWriter{truncatedTime: now.Add(-ttl).Truncate(ttl), db: rw.s.decisionDB}.IsTraceSampled(traceID) + if err == nil { + return sampled, nil } - defer closer.Close() - return item[0] == entryMetaTraceSampled, nil + + return false, err } // WriteTraceEvent writes a trace event to storage. @@ -233,3 +239,9 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error } return nil } + +func timePrefix(t time.Time) string { + // FIXME: use TTL + // FIXME: convert int to bytes + return fmt.Sprintf("%d", t.Unix()) +} diff --git a/x-pack/apm-server/sampling/eventstorage/ttlrw.go b/x-pack/apm-server/sampling/eventstorage/ttlrw.go new file mode 100644 index 00000000000..cc207ac8e8d --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/ttlrw.go @@ -0,0 +1,75 @@ +package eventstorage + +import ( + "bytes" + "time" + + "github.com/cockroachdb/pebble" + + "github.com/elastic/apm-data/model/modelpb" +) + +func NewTTLReadWriter(truncatedTime time.Time, db db) TTLReadWriter { + return TTLReadWriter{truncatedTime: truncatedTime, db: db} +} + +type TTLReadWriter struct { + truncatedTime time.Time + db db +} + +func (rw TTLReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + panic("implement me") +} + +func (rw TTLReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { + //TODO implement me + panic("implement me") +} + +func (rw TTLReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + tp := timePrefix(rw.truncatedTime) + + var b bytes.Buffer + b.Grow(len(tp) + 1 + len(traceID)) + b.WriteString(tp) + b.WriteByte('@') + b.WriteString(traceID) + + meta := entryMetaTraceUnsampled + if sampled { + meta = entryMetaTraceSampled + } + err := rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) + if err != nil { + return err + } + return nil +} + +func (rw TTLReadWriter) IsTraceSampled(traceID string) (bool, error) { + tp := timePrefix(rw.truncatedTime) + + var b bytes.Buffer + b.Grow(len(tp) + 1 + len(traceID)) + b.WriteString(tp) + b.WriteByte('@') + b.WriteString(traceID) + + item, closer, err := rw.db.Get(b.Bytes()) + if err == pebble.ErrNotFound { + return false, ErrNotFound + } + defer closer.Close() + return item[0] == entryMetaTraceSampled, nil +} + +func (rw TTLReadWriter) DeleteTraceEvent(traceID, id string) error { + //TODO implement me + panic("implement me") +} + +func (rw TTLReadWriter) Flush() error { + //TODO implement me + panic("implement me") +} diff --git a/x-pack/apm-server/sampling/eventstorage/ttlrw_test.go b/x-pack/apm-server/sampling/eventstorage/ttlrw_test.go new file mode 100644 index 00000000000..11f4f407097 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/ttlrw_test.go @@ -0,0 +1,58 @@ +package eventstorage_test + +import ( + "fmt" + "testing" + "time" + + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/vfs" + "github.com/gofrs/uuid/v5" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" +) + +func newPebble(t *testing.T) *pebble.DB { + db, err := pebble.Open("", &pebble.Options{ + FS: vfs.NewMem(), + }) + require.NoError(t, err) + return db +} + +func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { + for _, tc := range []struct { + sampled bool + missing bool + }{ + { + sampled: true, + }, + { + sampled: false, + }, + { + missing: true, + }, + } { + t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { + tt := time.Unix(3600, 0) + db := newPebble(t) + rw := eventstorage.NewTTLReadWriter(tt, db) + traceID := uuid.Must(uuid.NewV4()).String() + if !tc.missing { + err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) + require.NoError(t, err) + } + sampled, err := rw.IsTraceSampled(traceID) + if tc.missing { + assert.ErrorIs(t, err, eventstorage.ErrNotFound) + } else { + assert.NoError(t, err) + assert.Equal(t, tc.sampled, sampled) + } + }) + } +} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 9acf9dad5c8..e5610a7671e 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -55,8 +55,6 @@ 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 @@ -65,6 +63,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} writer := config.DB.NewBypassReadWriter() wOpts := eventstorage.WriterOpts{ + TimeNow: time.Now, TTL: time.Minute, StorageLimitInBytes: 0, } @@ -72,7 +71,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { assert.NoError(t, writer.Flush()) writer.Close() - wOpts.TTL = -1 // expire immediately + wOpts.TimeNow = func() time.Time { return time.Now().Add(-2 * wOpts.TTL) } writer = config.DB.NewBypassReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true, wOpts)) assert.NoError(t, writer.Flush()) From 681d12e7233747401afdd0134083c7d20e008cb7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 15:47:15 +0000 Subject: [PATCH 053/184] Use partition --- .../sampling/eventstorage/partitioned.go | 70 +++++++++++++++++ .../{ttlrw_test.go => partitioned_test.go} | 18 ++++- .../sampling/eventstorage/storage.go | 20 ++--- .../sampling/eventstorage/storage_manager.go | 33 +++++++- .../apm-server/sampling/eventstorage/ttlrw.go | 75 ------------------- x-pack/apm-server/sampling/processor_test.go | 12 +-- 6 files changed, 128 insertions(+), 100 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/partitioned.go rename x-pack/apm-server/sampling/eventstorage/{ttlrw_test.go => partitioned_test.go} (78%) delete mode 100644 x-pack/apm-server/sampling/eventstorage/ttlrw.go diff --git a/x-pack/apm-server/sampling/eventstorage/partitioned.go b/x-pack/apm-server/sampling/eventstorage/partitioned.go new file mode 100644 index 00000000000..6973e5bd5d5 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/partitioned.go @@ -0,0 +1,70 @@ +package eventstorage + +import ( + "bytes" + + "github.com/cockroachdb/pebble" + + "github.com/elastic/apm-data/model/modelpb" +) + +func NewPartitionedReadWriter(db db, partitionID int32) PartitionedReadWriter { + return PartitionedReadWriter{db: db, partitionID: partitionID} +} + +type PartitionedReadWriter struct { + db db + partitionID int32 +} + +func (rw PartitionedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + panic("implement me") +} + +func (rw PartitionedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { + //TODO implement me + panic("implement me") +} + +func (rw PartitionedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + var b bytes.Buffer + b.Grow(1 + 1 + len(traceID)) + b.WriteByte(byte(rw.partitionID)) + b.WriteByte('@') + b.WriteString(traceID) + + meta := entryMetaTraceUnsampled + if sampled { + meta = entryMetaTraceSampled + } + err := rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) + if err != nil { + return err + } + return nil +} + +func (rw PartitionedReadWriter) IsTraceSampled(traceID string) (bool, error) { + var b bytes.Buffer + b.Grow(1 + 1 + len(traceID)) + b.WriteByte(byte(rw.partitionID)) + b.WriteByte('@') + b.WriteString(traceID) + + item, closer, err := rw.db.Get(b.Bytes()) + if err == pebble.ErrNotFound { + return false, ErrNotFound + } + defer closer.Close() + return item[0] == entryMetaTraceSampled, nil +} + +func (rw PartitionedReadWriter) DeleteTraceEvent(traceID, id string) error { + //TODO implement me + panic("implement me") +} + +func (rw PartitionedReadWriter) Flush() error { + //TODO implement me + panic("implement me") +} diff --git a/x-pack/apm-server/sampling/eventstorage/ttlrw_test.go b/x-pack/apm-server/sampling/eventstorage/partitioned_test.go similarity index 78% rename from x-pack/apm-server/sampling/eventstorage/ttlrw_test.go rename to x-pack/apm-server/sampling/eventstorage/partitioned_test.go index 11f4f407097..56d99a32bf1 100644 --- a/x-pack/apm-server/sampling/eventstorage/ttlrw_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioned_test.go @@ -3,7 +3,6 @@ package eventstorage_test import ( "fmt" "testing" - "time" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/vfs" @@ -14,6 +13,20 @@ import ( "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) +type testPartitionDB struct { + *pebble.DB + partitionID int32 + partitionCount int32 +} + +func (t testPartitionDB) PartitionID() int32 { + return t.partitionID +} + +func (t testPartitionDB) PartitionCount() int32 { + return t.partitionCount +} + func newPebble(t *testing.T) *pebble.DB { db, err := pebble.Open("", &pebble.Options{ FS: vfs.NewMem(), @@ -38,9 +51,8 @@ func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { }, } { t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { - tt := time.Unix(3600, 0) db := newPebble(t) - rw := eventstorage.NewTTLReadWriter(tt, db) + rw := eventstorage.NewPartitionedReadWriter(testPartitionDB{DB: db}, 1) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 91d1e39243f..52eec16bff7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -42,6 +42,8 @@ type db interface { Set(key, value []byte, opts *pebble.WriteOptions) error Delete(key []byte, opts *pebble.WriteOptions) error NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) + PartitionID() int32 + PartitionCount() int32 } // Storage provides storage for sampled transactions and spans, @@ -93,7 +95,6 @@ func (s *Storage) NewReadWriter() *ReadWriter { // WriterOpts provides configuration options for writes to storage type WriterOpts struct { - TimeNow func() time.Time TTL time.Duration StorageLimitInBytes int64 } @@ -136,32 +137,25 @@ 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 { - var now time.Time - if opts.TimeNow != nil { - now = opts.TimeNow() - } else { - now = time.Now() - } - - return TTLReadWriter{truncatedTime: now.Truncate(opts.TTL), db: rw.s.decisionDB}.WriteTraceSampled(traceID, sampled, opts) + return NewPartitionedReadWriter(rw.s.decisionDB, rw.s.decisionDB.PartitionID()).WriteTraceSampled(traceID, sampled, opts) } // 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) { + currentPID := rw.s.decisionDB.PartitionID() + prevPID := (currentPID - 1) % rw.s.decisionDB.PartitionCount() // 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 - now := time.Now() - ttl := time.Minute - sampled, err := TTLReadWriter{truncatedTime: now.Truncate(ttl), db: rw.s.decisionDB}.IsTraceSampled(traceID) + sampled, err := NewPartitionedReadWriter(rw.s.decisionDB, currentPID).IsTraceSampled(traceID) if err == nil { return sampled, nil } - sampled, err = TTLReadWriter{truncatedTime: now.Add(-ttl).Truncate(ttl), db: rw.s.decisionDB}.IsTraceSampled(traceID) + sampled, err = NewPartitionedReadWriter(rw.s.decisionDB, prevPID).IsTraceSampled(traceID) if err == nil { return sampled, nil } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 1c64bfbf708..2e59c9d293b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -10,6 +10,7 @@ import ( "os" "path/filepath" "sync" + "sync/atomic" "time" "github.com/cockroachdb/pebble" @@ -58,6 +59,14 @@ func (w *wrappedDB) Size() (lsm, vlog int64) { return int64(w.db.Metrics().DiskSpaceUsage()), 0 // FIXME } +func (w *wrappedDB) PartitionID() int32 { + return w.sm.partitionID.Load() +} + +func (w *wrappedDB) PartitionCount() int32 { + return w.sm.partitionCount +} + type StorageManagerOptions func(*StorageManager) func WithCodec(codec Codec) StorageManagerOptions { @@ -78,6 +87,9 @@ type StorageManager struct { storage *Storage rw *ShardedReadWriter + partitionID atomic.Int32 + partitionCount int32 + codec Codec // mu guards db, storage, and rw swaps. @@ -93,10 +105,11 @@ type StorageManager struct { // NewStorageManager returns a new StorageManager with pebble DB at storageDir. func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*StorageManager, error) { sm := &StorageManager{ - storageDir: storageDir, - runCh: make(chan struct{}, 1), - logger: logp.NewLogger(logs.Sampling), - codec: ProtobufCodec{}, + storageDir: storageDir, + runCh: make(chan struct{}, 1), + logger: logp.NewLogger(logs.Sampling), + codec: ProtobufCodec{}, + partitionCount: 3, } for _, opt := range opts { opt(sm) @@ -158,9 +171,21 @@ func (s *StorageManager) Reload() error { // 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 { + ttlTick := time.NewTicker(ttl) + defer ttlTick.Stop() + select { + case <-stopping: + return nil + case <-ttlTick.C: + s.IncrementPartition() + } return nil } +func (s *StorageManager) IncrementPartition() { + s.partitionID.Store((s.partitionID.Load() + 1) % s.partitionCount) +} + func (s *StorageManager) ReadSubscriberPosition() ([]byte, error) { s.subscriberPosMu.Lock() defer s.subscriberPosMu.Unlock() diff --git a/x-pack/apm-server/sampling/eventstorage/ttlrw.go b/x-pack/apm-server/sampling/eventstorage/ttlrw.go deleted file mode 100644 index cc207ac8e8d..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/ttlrw.go +++ /dev/null @@ -1,75 +0,0 @@ -package eventstorage - -import ( - "bytes" - "time" - - "github.com/cockroachdb/pebble" - - "github.com/elastic/apm-data/model/modelpb" -) - -func NewTTLReadWriter(truncatedTime time.Time, db db) TTLReadWriter { - return TTLReadWriter{truncatedTime: truncatedTime, db: db} -} - -type TTLReadWriter struct { - truncatedTime time.Time - db db -} - -func (rw TTLReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - panic("implement me") -} - -func (rw TTLReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { - //TODO implement me - panic("implement me") -} - -func (rw TTLReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - tp := timePrefix(rw.truncatedTime) - - var b bytes.Buffer - b.Grow(len(tp) + 1 + len(traceID)) - b.WriteString(tp) - b.WriteByte('@') - b.WriteString(traceID) - - meta := entryMetaTraceUnsampled - if sampled { - meta = entryMetaTraceSampled - } - err := rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) - if err != nil { - return err - } - return nil -} - -func (rw TTLReadWriter) IsTraceSampled(traceID string) (bool, error) { - tp := timePrefix(rw.truncatedTime) - - var b bytes.Buffer - b.Grow(len(tp) + 1 + len(traceID)) - b.WriteString(tp) - b.WriteByte('@') - b.WriteString(traceID) - - item, closer, err := rw.db.Get(b.Bytes()) - if err == pebble.ErrNotFound { - return false, ErrNotFound - } - defer closer.Close() - return item[0] == entryMetaTraceSampled, nil -} - -func (rw TTLReadWriter) DeleteTraceEvent(traceID, id string) error { - //TODO implement me - panic("implement me") -} - -func (rw TTLReadWriter) Flush() error { - //TODO implement me - panic("implement me") -} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index e5610a7671e..8e3af6212a9 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -63,17 +63,19 @@ func TestProcessAlreadyTailSampled(t *testing.T) { trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} writer := config.DB.NewBypassReadWriter() wOpts := eventstorage.WriterOpts{ - TimeNow: time.Now, - TTL: time.Minute, + TTL: time.Hour, StorageLimitInBytes: 0, } - assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true, wOpts)) + assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true, wOpts)) assert.NoError(t, writer.Flush()) writer.Close() - wOpts.TimeNow = func() time.Time { return time.Now().Add(-2 * wOpts.TTL) } + // simulate 2 TTL + config.DB.IncrementPartition() + config.DB.IncrementPartition() + writer = config.DB.NewBypassReadWriter() - assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true, wOpts)) + assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true, wOpts)) assert.NoError(t, writer.Flush()) writer.Close() From b7854e1537af8431f6db488494fba22dcca57a52 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:19:00 +0000 Subject: [PATCH 054/184] Implement TTL using partition --- .../sampling/eventstorage/partitioned.go | 81 +++++++++++++++-- .../sampling/eventstorage/partitioned_test.go | 2 +- .../sampling/eventstorage/storage.go | 83 ++++-------------- .../sampling/eventstorage/storage_manager.go | 13 ++- .../eventstorage/storage_manager_test.go | 86 +++++++++++++++++++ x-pack/apm-server/sampling/processor_test.go | 4 +- 6 files changed, 189 insertions(+), 80 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/storage_manager_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/partitioned.go b/x-pack/apm-server/sampling/eventstorage/partitioned.go index 6973e5bd5d5..364cc2b92ee 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioned.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioned.go @@ -2,28 +2,83 @@ package eventstorage import ( "bytes" + "fmt" "github.com/cockroachdb/pebble" "github.com/elastic/apm-data/model/modelpb" ) -func NewPartitionedReadWriter(db db, partitionID int32) PartitionedReadWriter { - return PartitionedReadWriter{db: db, partitionID: partitionID} +func NewPartitionedReadWriter(db db, partitionID int32, codec Codec) PartitionedReadWriter { + return PartitionedReadWriter{db: db, partitionID: partitionID, codec: codec} } type PartitionedReadWriter struct { db db partitionID int32 + codec Codec } func (rw PartitionedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - panic("implement me") + var lb bytes.Buffer + lb.Grow(1 + 1 + len(traceID) + 1) + lb.WriteByte(byte(rw.partitionID)) + lb.WriteByte('@') + lb.WriteString(traceID) + lb.WriteByte(':') + + var ub bytes.Buffer + ub.Grow(lb.Len()) + ub.Write(lb.Bytes()[:lb.Len()-1]) + ub.WriteByte(';') // This is a hack to stop before next ID + + iter, err := rw.db.NewIter(&pebble.IterOptions{ + LowerBound: lb.Bytes(), + UpperBound: ub.Bytes(), + }) + if err != nil { + return err + } + defer iter.Close() + // SeekPrefixGE uses the prefix bloom filter, so that a miss will be much faster + if valid := iter.SeekPrefixGE(lb.Bytes()); !valid { + return nil + } + for ; iter.Valid(); iter.Next() { + event := &modelpb.APMEvent{} + data, err := iter.ValueAndErr() + if err != nil { + return err + } + if err := rw.codec.DecodeEvent(data, event); err != nil { + return fmt.Errorf("codec failed to decode event: %w", err) + } + *out = append(*out, event) + } + return nil } func (rw PartitionedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { - //TODO implement me - panic("implement me") + data, err := rw.codec.EncodeEvent(event) + if err != nil { + return err + } + var b bytes.Buffer + b.Grow(1 + 1 + len(traceID) + 1 + len(id)) + b.WriteByte(byte(rw.partitionID)) + b.WriteByte('@') + b.WriteString(traceID) + b.WriteByte(':') + b.WriteString(id) + key := b.Bytes() + return rw.writeEntry(key, data) +} + +func (rw *PartitionedReadWriter) writeEntry(key, data []byte) error { + if err := rw.db.Set(key, data, pebble.NoSync); err != nil { + return err + } + return nil } func (rw PartitionedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { @@ -60,8 +115,20 @@ func (rw PartitionedReadWriter) IsTraceSampled(traceID string) (bool, error) { } func (rw PartitionedReadWriter) DeleteTraceEvent(traceID, id string) error { - //TODO implement me - panic("implement me") + var b bytes.Buffer + b.Grow(1 + 1 + len(traceID) + 1 + len(id)) + b.WriteByte(byte(rw.partitionID)) + b.WriteByte('@') + b.WriteString(traceID) + b.WriteByte(':') + b.WriteString(id) + key := b.Bytes() + + err := rw.db.Delete(key, pebble.NoSync) + if err != nil { + return err + } + return nil } func (rw PartitionedReadWriter) Flush() error { diff --git a/x-pack/apm-server/sampling/eventstorage/partitioned_test.go b/x-pack/apm-server/sampling/eventstorage/partitioned_test.go index 56d99a32bf1..bbd52651ad3 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioned_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioned_test.go @@ -52,7 +52,7 @@ func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { } { t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { db := newPebble(t) - rw := eventstorage.NewPartitionedReadWriter(testPartitionDB{DB: db}, 1) + rw := eventstorage.NewPartitionedReadWriter(testPartitionDB{DB: db}, 1, nopCodec{}) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 52eec16bff7..107eddf381a 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -5,9 +5,7 @@ package eventstorage import ( - "bytes" "errors" - "fmt" "io" "sync/atomic" "time" @@ -137,7 +135,7 @@ 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 { - return NewPartitionedReadWriter(rw.s.decisionDB, rw.s.decisionDB.PartitionID()).WriteTraceSampled(traceID, sampled, opts) + return NewPartitionedReadWriter(rw.s.decisionDB, rw.s.decisionDB.PartitionID(), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -150,12 +148,12 @@ 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 - sampled, err := NewPartitionedReadWriter(rw.s.decisionDB, currentPID).IsTraceSampled(traceID) + sampled, err := NewPartitionedReadWriter(rw.s.decisionDB, currentPID, rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } - sampled, err = NewPartitionedReadWriter(rw.s.decisionDB, prevPID).IsTraceSampled(traceID) + sampled, err = NewPartitionedReadWriter(rw.s.decisionDB, prevPID, rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } @@ -167,75 +165,26 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // // WriteTraceEvent may return before the write is committed to storage. func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts 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.pendingWrites++ - - if err := rw.s.db.Set(key, data, pebble.NoSync); err != nil { - return err - } - return nil + return NewPartitionedReadWriter(rw.s.db, rw.s.db.PartitionID(), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) } // DeleteTraceEvent deletes the trace event from storage. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { // FIXME: use range delete - var buf bytes.Buffer - buf.Grow(len(traceID) + 1 + len(id)) - buf.WriteString(traceID) - buf.WriteByte(':') - buf.WriteString(id) - key := buf.Bytes() - - err := rw.s.db.Delete(key, pebble.NoSync) - if err != nil { - return err - } - return nil + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID - 1) % rw.s.db.PartitionCount() + return errors.Join( + NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).DeleteTraceEvent(traceID, id), + NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).DeleteTraceEvent(traceID, id), + ) } // 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), ';'), // This is a hack to stop before next ID - }) - if err != nil { - return err - } - defer iter.Close() - // SeekPrefixGE uses the prefix bloom filter, so that a miss will be much faster - if valid := iter.SeekPrefixGE(append([]byte(traceID), ':')); !valid { - return nil - } - for ; 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 -} - -func timePrefix(t time.Time) string { - // FIXME: use TTL - // FIXME: convert int to bytes - return fmt.Sprintf("%d", t.Unix()) + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID - 1) % rw.s.db.PartitionCount() + return errors.Join( + NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).ReadTraceEvents(traceID, out), + NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).ReadTraceEvents(traceID, out), + ) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 2e59c9d293b..20c7618ece4 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -177,13 +177,20 @@ func (s *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration, case <-stopping: return nil case <-ttlTick.C: - s.IncrementPartition() + _ = s.IncrementPartition() } return nil } -func (s *StorageManager) IncrementPartition() { - s.partitionID.Store((s.partitionID.Load() + 1) % s.partitionCount) +func (s *StorageManager) IncrementPartition() error { + oldPID := s.partitionID.Load() + s.partitionID.Store((oldPID + 1) % s.partitionCount) + + pidToDelete := (oldPID - 1) % s.partitionCount + return errors.Join( + s.db.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), + s.decisionDB.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), + ) } func (s *StorageManager) ReadSubscriberPosition() ([]byte, error) { diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go new file mode 100644 index 00000000000..31b13fbaf3e --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -0,0 +1,86 @@ +package eventstorage_test + +import ( + "testing" + + "github.com/gofrs/uuid/v5" + "github.com/stretchr/testify/assert" + + "github.com/elastic/apm-data/model/modelpb" + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" +) + +func TestStorageManager_samplingDecisionTTL(t *testing.T) { + sm := newStorageManager(t) + rw := sm.NewBypassReadWriter() + traceID := uuid.Must(uuid.NewV4()).String() + err := rw.WriteTraceSampled(traceID, true, eventstorage.WriterOpts{}) + assert.NoError(t, err) + sampled, err := rw.IsTraceSampled(traceID) + assert.NoError(t, err) + assert.True(t, sampled) + + // after 1 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + sampled, err = rw.IsTraceSampled(traceID) + assert.NoError(t, err) + assert.True(t, sampled) + + // after 2 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + _, err = rw.IsTraceSampled(traceID) + assert.ErrorIs(t, err, eventstorage.ErrNotFound) + + // after 3 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + _, err = rw.IsTraceSampled(traceID) + assert.ErrorIs(t, err, eventstorage.ErrNotFound) +} + +func TestStorageManager_eventTTL(t *testing.T) { + sm := newStorageManager(t) + rw := sm.NewBypassReadWriter() + traceID := uuid.Must(uuid.NewV4()).String() + txnID := uuid.Must(uuid.NewV4()).String() + transaction := makeTransaction(txnID, traceID) + err := rw.WriteTraceEvent(traceID, txnID, transaction, eventstorage.WriterOpts{}) + assert.NoError(t, err) + + var out modelpb.Batch + err = rw.ReadTraceEvents(traceID, &out) + assert.NoError(t, err) + assert.Len(t, out, 1) + + // after 1 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + out = nil + err = rw.ReadTraceEvents(traceID, &out) + assert.NoError(t, err) + assert.Len(t, out, 1) + + // after 2 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + out = nil + err = rw.ReadTraceEvents(traceID, &out) + assert.NoError(t, err) + assert.Len(t, out, 0) + + // after 3 TTL + err = sm.IncrementPartition() + assert.NoError(t, err) + + out = nil + err = rw.ReadTraceEvents(traceID, &out) + assert.NoError(t, err) + assert.Len(t, out, 0) +} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 8e3af6212a9..3692027d119 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -71,8 +71,8 @@ func TestProcessAlreadyTailSampled(t *testing.T) { writer.Close() // simulate 2 TTL - config.DB.IncrementPartition() - config.DB.IncrementPartition() + assert.NoError(t, config.DB.IncrementPartition()) + assert.NoError(t, config.DB.IncrementPartition()) writer = config.DB.NewBypassReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true, wOpts)) From 5236c7c21d8d300cf0d4639fe7573987121ef80f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:34:04 +0000 Subject: [PATCH 055/184] Compact the DBs --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 20c7618ece4..c521bd0c82f 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -190,6 +190,8 @@ func (s *StorageManager) IncrementPartition() error { return errors.Join( s.db.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), s.decisionDB.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), + s.db.Compact([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, false), + s.decisionDB.Compact([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, false), ) } From 44ddc87c4bb612aadf703fca39e64921092c2df2 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:38:17 +0000 Subject: [PATCH 056/184] Clean up --- .../sampling/eventstorage/pebble.go | 18 ------------- .../sampling/eventstorage/storage.go | 27 ++++--------------- .../sampling/eventstorage/storage_manager.go | 8 ------ 3 files changed, 5 insertions(+), 48 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 10002b60514..74de03670d0 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -12,15 +12,6 @@ import ( ) 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 @@ -36,14 +27,6 @@ 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 - - // FIXME: fix comment - // 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 eventComparer() *pebble.Comparer { @@ -78,7 +61,6 @@ func OpenSamplingDecisionPebble(storageDir string) (*pebble.DB, error) { // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, Logger: logp.NewLogger(logs.Sampling), - //MemTableSize: pebbleMemTableSize, Levels: []pebble.LevelOptions{ { BlockSize: 2 << 10, diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 107eddf381a..fbe55b15946 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -7,7 +7,6 @@ package eventstorage import ( "errors" "io" - "sync/atomic" "time" "github.com/cockroachdb/pebble" @@ -35,7 +34,6 @@ var ( ) type db interface { - //Size() (lsm, vlog int64) Get(key []byte) ([]byte, io.Closer, error) Set(key, value []byte, opts *pebble.WriteOptions) error Delete(key []byte, opts *pebble.WriteOptions) error @@ -49,9 +47,7 @@ type db interface { type Storage struct { db db decisionDB db - // pendingSize tracks the total size of pending writes across ReadWriters - pendingSize *atomic.Int64 - codec Codec + codec Codec } // Codec provides methods for encoding and decoding events. @@ -63,10 +59,9 @@ type Codec interface { // New returns a new Storage using db, decisionDB and codec. func New(db db, decisionDB db, codec Codec) *Storage { return &Storage{ - db: db, - decisionDB: decisionDB, - pendingSize: &atomic.Int64{}, - codec: codec, + db: db, + decisionDB: decisionDB, + codec: codec, } } @@ -84,10 +79,8 @@ 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) return &ReadWriter{ s: s, - //pendingSize: baseTransactionSize, } } @@ -105,17 +98,7 @@ 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 - batch *pebble.Batch - decisionBatch *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 - // be unmodified until the end of a transaction. - readKeyBuf []byte - pendingWrites int - // pendingSize tracks the size of pending writes in the current ReadWriter - pendingSize int64 + s *Storage } // Close closes the writer. Any writes that have not been flushed may be lost. diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index c521bd0c82f..d3a5db8d036 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -51,14 +51,6 @@ func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { return w.db.NewIter(o) } -// Size returns the db size -func (w *wrappedDB) Size() (lsm, vlog int64) { - // FIXME: we may want it to report the sum of 2 dbs - w.sm.mu.RLock() - defer w.sm.mu.RUnlock() - return int64(w.db.Metrics().DiskSpaceUsage()), 0 // FIXME -} - func (w *wrappedDB) PartitionID() int32 { return w.sm.partitionID.Load() } From 0411dfa364e9d2294dbe5758db49ad9581d82d1c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:42:18 +0000 Subject: [PATCH 057/184] Fix TTL Loop --- .../sampling/eventstorage/storage_manager.go | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index d3a5db8d036..3dfbf7897b2 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 ( "time" "github.com/cockroachdb/pebble" + "golang.org/x/sync/errgroup" "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/internal/logs" @@ -163,15 +164,35 @@ func (s *StorageManager) Reload() error { // 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 { - ttlTick := time.NewTicker(ttl) - defer ttlTick.Stop() select { case <-stopping: return nil - case <-ttlTick.C: - _ = s.IncrementPartition() + case s.runCh <- struct{}{}: + } + defer func() { + <-s.runCh + }() + + g := errgroup.Group{} + g.Go(func() error { + return s.runTTLLoop(stopping, gcInterval) + }) + return g.Wait() +} + +func (s *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration) error { + ticker := time.NewTicker(ttl) + defer ticker.Stop() + for { + select { + case <-stopping: + return nil + case <-ticker.C: + if err := s.IncrementPartition(); err != nil { + s.logger.With(logp.Error(err)).Error("failed to increment partition") + } + } } - return nil } func (s *StorageManager) IncrementPartition() error { From 616bc0bb7791b188f85735bb6c1e22cbf5dae34a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:42:49 +0000 Subject: [PATCH 058/184] Fix close error --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 3dfbf7897b2..cdcaa619281 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -145,9 +145,7 @@ func (s *StorageManager) Close() error { func (s *StorageManager) close() error { s.rw.Close() - _ = s.db.Close() // FIXME - _ = s.decisionDB.Close() // FIXME - return nil + return errors.Join(s.db.Close(), s.decisionDB.Close()) } // Reload flushes out pending disk writes to disk by reloading the database. From e4c62b7ef3c0f0d9db84df9a4a93e97da4f8683c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 16:50:48 +0000 Subject: [PATCH 059/184] Remove ManagedReadWriter --- x-pack/apm-server/main.go | 4 +- x-pack/apm-server/sampling/config_test.go | 2 +- .../sampling/eventstorage/storage_manager.go | 67 +------------------ 3 files changed, 5 insertions(+), 68 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 60e74a5f04f..3e561104f00 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -45,7 +45,7 @@ var ( db *eventstorage.StorageManager storageMu sync.Mutex - storage *eventstorage.ManagedReadWriter + storage *eventstorage.ShardedReadWriter // samplerUUID is a UUID used to identify sampled trace ID documents // published by this process. @@ -179,7 +179,7 @@ func getDB(storageDir string) (*eventstorage.StorageManager, error) { return db, nil } -func getStorage(sm *eventstorage.StorageManager) *eventstorage.ManagedReadWriter { +func getStorage(sm *eventstorage.StorageManager) *eventstorage.ShardedReadWriter { storageMu.Lock() defer storageMu.Unlock() if storage == nil { diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index 697cafec588..12344e9637e 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -74,7 +74,7 @@ func TestNewProcessorConfigInvalid(t *testing.T) { config.DB = &eventstorage.StorageManager{} assertInvalidConfigError("invalid storage config: Storage unspecified") - config.Storage = &eventstorage.ManagedReadWriter{} + config.Storage = &eventstorage.ShardedReadWriter{} assertInvalidConfigError("invalid storage config: StorageDir unspecified") config.StorageDir = "tbs" diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index cdcaa619281..ff3e806ae60 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/pebble" "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" ) @@ -27,10 +26,6 @@ const ( subscriberPositionFile = "subscriber_position.json" ) -var ( - errDropAndRecreateInProgress = errors.New("db drop and recreate in progress") -) - type wrappedDB struct { sm *StorageManager db *pebble.DB @@ -85,8 +80,6 @@ type StorageManager struct { codec Codec - // mu guards db, storage, and rw swaps. - mu sync.RWMutex // subscriberPosMu protects the subscriber file from concurrent RW. subscriberPosMu sync.Mutex @@ -132,14 +125,10 @@ func (s *StorageManager) reset() error { } func (s *StorageManager) Size() (lsm, vlog int64) { - s.mu.RLock() - defer s.mu.RUnlock() return int64(s.db.Metrics().DiskSpaceUsage() + s.decisionDB.Metrics().DiskSpaceUsage()), 0 } func (s *StorageManager) Close() error { - s.mu.RLock() - defer s.mu.RUnlock() return s.close() } @@ -152,8 +141,6 @@ func (s *StorageManager) close() error { // It does not flush uncommitted writes. // For testing only. func (s *StorageManager) Reload() error { - s.mu.RLock() - defer s.mu.RUnlock() if err := s.close(); err != nil { return err } @@ -218,58 +205,8 @@ func (s *StorageManager) WriteSubscriberPosition(data []byte) error { return os.WriteFile(filepath.Join(s.storageDir, subscriberPositionFile), data, 0644) } -func (s *StorageManager) NewReadWriter() *ManagedReadWriter { - return &ManagedReadWriter{ - sm: s, - } -} - -// ManagedReadWriter is a read writer that is transparent to badger DB changes done by StorageManager. -// It is a wrapper of the ShardedReadWriter under StorageManager. -type ManagedReadWriter struct { - sm *StorageManager -} - -func (s *ManagedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - s.sm.mu.RLock() - defer s.sm.mu.RUnlock() - return s.sm.rw.ReadTraceEvents(traceID, out) -} - -func (s *ManagedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { - ok := s.sm.mu.TryRLock() - if !ok { - return errDropAndRecreateInProgress - } - defer s.sm.mu.RUnlock() - return s.sm.rw.WriteTraceEvent(traceID, id, event, opts) -} - -func (s *ManagedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - ok := s.sm.mu.TryRLock() - if !ok { - return errDropAndRecreateInProgress - } - defer s.sm.mu.RUnlock() - return s.sm.rw.WriteTraceSampled(traceID, sampled, opts) -} - -func (s *ManagedReadWriter) IsTraceSampled(traceID string) (bool, error) { - s.sm.mu.RLock() - defer s.sm.mu.RUnlock() - return s.sm.rw.IsTraceSampled(traceID) -} - -func (s *ManagedReadWriter) DeleteTraceEvent(traceID, id string) error { - s.sm.mu.RLock() - defer s.sm.mu.RUnlock() - return s.sm.rw.DeleteTraceEvent(traceID, id) -} - -func (s *ManagedReadWriter) Flush() error { - s.sm.mu.RLock() - defer s.sm.mu.RUnlock() - return s.sm.rw.Flush() +func (s *StorageManager) NewReadWriter() *ShardedReadWriter { + return s.rw } // NewBypassReadWriter returns a ReadWriter directly reading and writing to the database, From c085876f8b59e2ac31c7eea515d411fe150b8831 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:00:35 +0000 Subject: [PATCH 060/184] Fix modulo of negative number --- x-pack/apm-server/sampling/eventstorage/storage.go | 6 +++--- .../sampling/eventstorage/storage_manager.go | 12 +++++++----- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index fbe55b15946..223d2e3ac33 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -126,7 +126,7 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { currentPID := rw.s.decisionDB.PartitionID() - prevPID := (currentPID - 1) % rw.s.decisionDB.PartitionCount() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.decisionDB.PartitionCount() // 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 @@ -155,7 +155,7 @@ func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { // FIXME: use range delete currentPID := rw.s.db.PartitionID() - prevPID := (currentPID - 1) % rw.s.db.PartitionCount() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() return errors.Join( NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).DeleteTraceEvent(traceID, id), NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).DeleteTraceEvent(traceID, id), @@ -165,7 +165,7 @@ 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 { currentPID := rw.s.db.PartitionID() - prevPID := (currentPID - 1) % rw.s.db.PartitionCount() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() return errors.Join( NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).ReadTraceEvents(traceID, out), NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).ReadTraceEvents(traceID, out), diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index ff3e806ae60..9dfb412a4c6 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -184,12 +184,14 @@ func (s *StorageManager) IncrementPartition() error { oldPID := s.partitionID.Load() s.partitionID.Store((oldPID + 1) % s.partitionCount) - pidToDelete := (oldPID - 1) % s.partitionCount + pidToDelete := (oldPID + s.partitionCount - 1) % s.partitionCount + lbPrefix := byte(pidToDelete) + ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb return errors.Join( - s.db.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), - s.decisionDB.DeleteRange([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, pebble.NoSync), - s.db.Compact([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, false), - s.decisionDB.Compact([]byte{byte(pidToDelete)}, []byte{byte(oldPID)}, false), + s.db.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), + s.decisionDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), + s.db.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), + s.decisionDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), ) } From 3961e81244b1b9c5edaa2702a89e942a0f227240 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:26:37 +0000 Subject: [PATCH 061/184] Rename to prefix read writer --- .../{partitioned.go => prefix.go} | 51 +++++++++---------- .../{partitioned_test.go => prefix_test.go} | 2 +- .../sampling/eventstorage/storage.go | 19 +++---- 3 files changed, 34 insertions(+), 38 deletions(-) rename x-pack/apm-server/sampling/eventstorage/{partitioned.go => prefix.go} (59%) rename x-pack/apm-server/sampling/eventstorage/{partitioned_test.go => prefix_test.go} (94%) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioned.go b/x-pack/apm-server/sampling/eventstorage/prefix.go similarity index 59% rename from x-pack/apm-server/sampling/eventstorage/partitioned.go rename to x-pack/apm-server/sampling/eventstorage/prefix.go index 364cc2b92ee..a80fa44e2ea 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioned.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -9,21 +9,20 @@ import ( "github.com/elastic/apm-data/model/modelpb" ) -func NewPartitionedReadWriter(db db, partitionID int32, codec Codec) PartitionedReadWriter { - return PartitionedReadWriter{db: db, partitionID: partitionID, codec: codec} +func NewPrefixReadWriter(db db, prefix byte, codec Codec) PrefixReadWriter { + return PrefixReadWriter{db: db, prefix: prefix, codec: codec} } -type PartitionedReadWriter struct { - db db - partitionID int32 - codec Codec +type PrefixReadWriter struct { + db db + prefix byte + codec Codec } -func (rw PartitionedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { +func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { var lb bytes.Buffer - lb.Grow(1 + 1 + len(traceID) + 1) - lb.WriteByte(byte(rw.partitionID)) - lb.WriteByte('@') + lb.Grow(1 + len(traceID) + 1) + lb.WriteByte(rw.prefix) lb.WriteString(traceID) lb.WriteByte(':') @@ -58,15 +57,14 @@ func (rw PartitionedReadWriter) ReadTraceEvents(traceID string, out *modelpb.Bat return nil } -func (rw PartitionedReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { +func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { data, err := rw.codec.EncodeEvent(event) if err != nil { return err } var b bytes.Buffer - b.Grow(1 + 1 + len(traceID) + 1 + len(id)) - b.WriteByte(byte(rw.partitionID)) - b.WriteByte('@') + b.Grow(1 + len(traceID) + 1 + len(id)) + b.WriteByte(rw.prefix) b.WriteString(traceID) b.WriteByte(':') b.WriteString(id) @@ -74,18 +72,17 @@ func (rw PartitionedReadWriter) WriteTraceEvent(traceID, id string, event *model return rw.writeEntry(key, data) } -func (rw *PartitionedReadWriter) writeEntry(key, data []byte) error { +func (rw *PrefixReadWriter) writeEntry(key, data []byte) error { if err := rw.db.Set(key, data, pebble.NoSync); err != nil { return err } return nil } -func (rw PartitionedReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { +func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { var b bytes.Buffer - b.Grow(1 + 1 + len(traceID)) - b.WriteByte(byte(rw.partitionID)) - b.WriteByte('@') + b.Grow(1 + len(traceID)) + b.WriteByte(rw.prefix) b.WriteString(traceID) meta := entryMetaTraceUnsampled @@ -99,11 +96,10 @@ func (rw PartitionedReadWriter) WriteTraceSampled(traceID string, sampled bool, return nil } -func (rw PartitionedReadWriter) IsTraceSampled(traceID string) (bool, error) { +func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { var b bytes.Buffer - b.Grow(1 + 1 + len(traceID)) - b.WriteByte(byte(rw.partitionID)) - b.WriteByte('@') + b.Grow(1 + len(traceID)) + b.WriteByte(rw.prefix) b.WriteString(traceID) item, closer, err := rw.db.Get(b.Bytes()) @@ -114,11 +110,10 @@ func (rw PartitionedReadWriter) IsTraceSampled(traceID string) (bool, error) { return item[0] == entryMetaTraceSampled, nil } -func (rw PartitionedReadWriter) DeleteTraceEvent(traceID, id string) error { +func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { var b bytes.Buffer - b.Grow(1 + 1 + len(traceID) + 1 + len(id)) - b.WriteByte(byte(rw.partitionID)) - b.WriteByte('@') + b.Grow(1 + len(traceID) + 1 + len(id)) + b.WriteByte(rw.prefix) b.WriteString(traceID) b.WriteByte(':') b.WriteString(id) @@ -131,7 +126,7 @@ func (rw PartitionedReadWriter) DeleteTraceEvent(traceID, id string) error { return nil } -func (rw PartitionedReadWriter) Flush() error { +func (rw PrefixReadWriter) Flush() error { //TODO implement me panic("implement me") } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioned_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go similarity index 94% rename from x-pack/apm-server/sampling/eventstorage/partitioned_test.go rename to x-pack/apm-server/sampling/eventstorage/prefix_test.go index bbd52651ad3..2c81b51bb16 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioned_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -52,7 +52,7 @@ func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { } { t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { db := newPebble(t) - rw := eventstorage.NewPartitionedReadWriter(testPartitionDB{DB: db}, 1, nopCodec{}) + rw := eventstorage.NewPrefixReadWriter(testPartitionDB{DB: db}, 1, nopCodec{}) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 223d2e3ac33..897ca634c07 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -15,7 +15,6 @@ import ( ) const ( - //prefixSamplingDecision string = "!" // NOTE(axw) these values (and their meanings) must remain stable // over time, to avoid misinterpreting historical data. entryMetaTraceSampled byte = 's' @@ -118,7 +117,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 WriterOpts) error { - return NewPartitionedReadWriter(rw.s.decisionDB, rw.s.decisionDB.PartitionID(), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) + pid := rw.s.decisionDB.PartitionID() + return NewPrefixReadWriter(rw.s.decisionDB, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -131,12 +131,12 @@ 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 - sampled, err := NewPartitionedReadWriter(rw.s.decisionDB, currentPID, rw.s.codec).IsTraceSampled(traceID) + sampled, err := NewPrefixReadWriter(rw.s.decisionDB, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } - sampled, err = NewPartitionedReadWriter(rw.s.decisionDB, prevPID, rw.s.codec).IsTraceSampled(traceID) + sampled, err = NewPrefixReadWriter(rw.s.decisionDB, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } @@ -148,7 +148,8 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // // WriteTraceEvent may return before the write is committed to storage. func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { - return NewPartitionedReadWriter(rw.s.db, rw.s.db.PartitionID(), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) + pid := rw.s.db.PartitionID() + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) } // DeleteTraceEvent deletes the trace event from storage. @@ -157,8 +158,8 @@ func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { currentPID := rw.s.db.PartitionID() prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() return errors.Join( - NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).DeleteTraceEvent(traceID, id), - NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).DeleteTraceEvent(traceID, id), + NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).DeleteTraceEvent(traceID, id), + NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).DeleteTraceEvent(traceID, id), ) } @@ -167,7 +168,7 @@ func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error currentPID := rw.s.db.PartitionID() prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() return errors.Join( - NewPartitionedReadWriter(rw.s.db, currentPID, rw.s.codec).ReadTraceEvents(traceID, out), - NewPartitionedReadWriter(rw.s.db, prevPID, rw.s.codec).ReadTraceEvents(traceID, out), + NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).ReadTraceEvents(traceID, out), + NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).ReadTraceEvents(traceID, out), ) } From 9443b73c838d9be052e70c7a6fcf3bd67d0f18c5 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:31:19 +0000 Subject: [PATCH 062/184] Split interface --- .../sampling/eventstorage/prefix_test.go | 16 +--------------- .../apm-server/sampling/eventstorage/storage.go | 10 +++++++--- 2 files changed, 8 insertions(+), 18 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 2c81b51bb16..90bd71cd790 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -13,20 +13,6 @@ import ( "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) -type testPartitionDB struct { - *pebble.DB - partitionID int32 - partitionCount int32 -} - -func (t testPartitionDB) PartitionID() int32 { - return t.partitionID -} - -func (t testPartitionDB) PartitionCount() int32 { - return t.partitionCount -} - func newPebble(t *testing.T) *pebble.DB { db, err := pebble.Open("", &pebble.Options{ FS: vfs.NewMem(), @@ -52,7 +38,7 @@ func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { } { t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { db := newPebble(t) - rw := eventstorage.NewPrefixReadWriter(testPartitionDB{DB: db}, 1, nopCodec{}) + rw := eventstorage.NewPrefixReadWriter(db, 1, nopCodec{}) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 897ca634c07..153c5bc8016 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -37,6 +37,10 @@ type db interface { Set(key, value []byte, opts *pebble.WriteOptions) error Delete(key []byte, opts *pebble.WriteOptions) error NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) +} + +type partitionedDB interface { + db PartitionID() int32 PartitionCount() int32 } @@ -44,8 +48,8 @@ type db interface { // Storage provides storage for sampled transactions and spans, // and for recording trace sampling decisions. type Storage struct { - db db - decisionDB db + db partitionedDB + decisionDB partitionedDB codec Codec } @@ -56,7 +60,7 @@ type Codec interface { } // New returns a new Storage using db, decisionDB and codec. -func New(db db, decisionDB db, codec Codec) *Storage { +func New(db partitionedDB, decisionDB partitionedDB, codec Codec) *Storage { return &Storage{ db: db, decisionDB: decisionDB, From f3582ccec0998d8a3855463e5fcadc073068296b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:45:19 +0000 Subject: [PATCH 063/184] Add SplitReadWriter --- x-pack/apm-server/main.go | 4 +- x-pack/apm-server/sampling/config.go | 2 +- x-pack/apm-server/sampling/eventstorage/rw.go | 14 ++++ .../sampling/eventstorage/storage.go | 26 ++++---- .../sampling/eventstorage/storage_manager.go | 65 +++++++++++++++---- x-pack/apm-server/sampling/processor.go | 13 +--- 6 files changed, 84 insertions(+), 40 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/rw.go diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 3e561104f00..de9c0bcd038 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -45,7 +45,7 @@ var ( db *eventstorage.StorageManager storageMu sync.Mutex - storage *eventstorage.ShardedReadWriter + storage eventstorage.RW // samplerUUID is a UUID used to identify sampled trace ID documents // published by this process. @@ -179,7 +179,7 @@ func getDB(storageDir string) (*eventstorage.StorageManager, error) { return db, nil } -func getStorage(sm *eventstorage.StorageManager) *eventstorage.ShardedReadWriter { +func getStorage(sm *eventstorage.StorageManager) eventstorage.RW { storageMu.Lock() defer storageMu.Unlock() if storage == nil { diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index b6d4d6ce252..6dd3add8984 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -104,7 +104,7 @@ type StorageConfig struct { // // Storage lives outside processor lifecycle and will not be closed when processor // is closed - Storage rw + Storage eventstorage.RW // StorageDir holds the directory in which event storage will be maintained. StorageDir string diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go new file mode 100644 index 00000000000..4b2fae8d8c8 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -0,0 +1,14 @@ +package eventstorage + +import ( + "github.com/elastic/apm-data/model/modelpb" +) + +type RW interface { + ReadTraceEvents(traceID string, out *modelpb.Batch) error + WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error + WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error + IsTraceSampled(traceID string) (bool, error) + DeleteTraceEvent(traceID, id string) error + Flush() error +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 153c5bc8016..9cbec4bfc3a 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -48,9 +48,8 @@ type partitionedDB interface { // Storage provides storage for sampled transactions and spans, // and for recording trace sampling decisions. type Storage struct { - db partitionedDB - decisionDB partitionedDB - codec Codec + db partitionedDB + codec Codec } // Codec provides methods for encoding and decoding events. @@ -59,12 +58,11 @@ type Codec interface { EncodeEvent(*modelpb.APMEvent) ([]byte, error) } -// New returns a new Storage using db, decisionDB and codec. -func New(db partitionedDB, decisionDB partitionedDB, codec Codec) *Storage { +// New returns a new Storage using db and codec. +func New(db partitionedDB, codec Codec) *Storage { return &Storage{ - db: db, - decisionDB: decisionDB, - codec: codec, + db: db, + codec: codec, } } @@ -121,26 +119,26 @@ 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 { - pid := rw.s.decisionDB.PartitionID() - return NewPrefixReadWriter(rw.s.decisionDB, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) + pid := rw.s.db.PartitionID() + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) } // 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) { - currentPID := rw.s.decisionDB.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.decisionDB.PartitionCount() + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() // 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 - sampled, err := NewPrefixReadWriter(rw.s.decisionDB, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) + sampled, err := NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } - sampled, err = NewPrefixReadWriter(rw.s.decisionDB, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) + sampled, err = NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 9dfb412a4c6..f412944c7c6 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/pebble" "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" ) @@ -70,10 +71,11 @@ type StorageManager struct { storageDir string logger *logp.Logger - db *pebble.DB - decisionDB *pebble.DB - storage *Storage - rw *ShardedReadWriter + db *pebble.DB + decisionDB *pebble.DB + eventStorage *Storage + decisionStorage *Storage + rw *ShardedReadWriter partitionID atomic.Int32 partitionCount int32 @@ -114,13 +116,15 @@ func (s *StorageManager) reset() error { return err } s.db = db + s.eventStorage = New(&wrappedDB{sm: s, db: s.db}, s.codec) + decisionDB, err := OpenSamplingDecisionPebble(s.storageDir) if err != nil { return err } s.decisionDB = decisionDB - s.storage = New(&wrappedDB{sm: s, db: s.db}, &wrappedDB{sm: s, db: s.decisionDB}, s.codec) - s.rw = s.storage.NewShardedReadWriter() + s.decisionStorage = New(&wrappedDB{sm: s, db: s.decisionDB}, s.codec) + return nil } @@ -133,7 +137,6 @@ func (s *StorageManager) Close() error { } func (s *StorageManager) close() error { - s.rw.Close() return errors.Join(s.db.Close(), s.decisionDB.Close()) } @@ -207,13 +210,51 @@ func (s *StorageManager) WriteSubscriberPosition(data []byte) error { return os.WriteFile(filepath.Join(s.storageDir, subscriberPositionFile), data, 0644) } -func (s *StorageManager) NewReadWriter() *ShardedReadWriter { - return s.rw +func (s *StorageManager) NewReadWriter() SplitReadWriter { + return SplitReadWriter{ + eventRW: s.eventStorage.NewShardedReadWriter(), + decisionRW: s.decisionStorage.NewShardedReadWriter(), + } } -// NewBypassReadWriter returns a ReadWriter directly reading and writing to the database, +// NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, // bypassing any wrapper e.g. ShardedReadWriter. // This should be used for testing only, useful to check if data is actually persisted to the DB. -func (s *StorageManager) NewBypassReadWriter() *ReadWriter { - return s.storage.NewReadWriter() +func (s *StorageManager) NewBypassReadWriter() SplitReadWriter { + return SplitReadWriter{ + eventRW: s.eventStorage.NewReadWriter(), + decisionRW: s.decisionStorage.NewReadWriter(), + } +} + +type SplitReadWriter struct { + eventRW, decisionRW RW +} + +func (s SplitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + return s.eventRW.ReadTraceEvents(traceID, out) +} + +func (s SplitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { + return s.eventRW.WriteTraceEvent(traceID, id, event, opts) +} + +func (s SplitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + return s.decisionRW.WriteTraceSampled(traceID, sampled, opts) +} + +func (s SplitReadWriter) IsTraceSampled(traceID string) (bool, error) { + return s.decisionRW.IsTraceSampled(traceID) +} + +func (s SplitReadWriter) DeleteTraceEvent(traceID, id string) error { + return s.eventRW.DeleteTraceEvent(traceID, id) +} + +func (s SplitReadWriter) Flush() error { + return nil +} + +func (s SplitReadWriter) Close() error { + return nil } diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 4289a991597..407aa167225 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -566,18 +566,9 @@ const ( storageLimitThreshold = 0.90 // Allow 90% of the quota to be used. ) -type rw interface { - ReadTraceEvents(traceID string, out *modelpb.Batch) error - WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error - WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error - IsTraceSampled(traceID string) (bool, error) - DeleteTraceEvent(traceID, id string) error - Flush() error -} - // wrappedRW wraps configurable write options for global rw type wrappedRW struct { - rw rw + rw eventstorage.RW writerOpts eventstorage.WriterOpts } @@ -586,7 +577,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 rw, ttl time.Duration, limit int64) *wrappedRW { +func newWrappedRW(rw eventstorage.RW, ttl time.Duration, limit int64) *wrappedRW { if limit > 1 { limit = int64(float64(limit) * storageLimitThreshold) } From a0556b72185a3ce3b47773d069d324ab9c0f41e5 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:47:20 +0000 Subject: [PATCH 064/184] Rename s->sm --- .../sampling/eventstorage/storage_manager.go | 93 +++++++++---------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index f412944c7c6..ac790ec44a1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -65,13 +65,12 @@ func WithCodec(codec Codec) StorageManagerOptions { } // StorageManager encapsulates pebble.DB. -// It is to provide file system access, simplify synchronization and enable underlying db swaps. // It assumes exclusive access to pebble DB at storageDir. type StorageManager struct { storageDir string logger *logp.Logger - db *pebble.DB + eventDB *pebble.DB decisionDB *pebble.DB eventStorage *Storage decisionStorage *Storage @@ -110,65 +109,65 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora } // reset initializes db, storage, and rw. -func (s *StorageManager) reset() error { - db, err := OpenPebble(s.storageDir) +func (sm *StorageManager) reset() error { + db, err := OpenPebble(sm.storageDir) if err != nil { return err } - s.db = db - s.eventStorage = New(&wrappedDB{sm: s, db: s.db}, s.codec) + sm.eventDB = db + sm.eventStorage = New(&wrappedDB{sm: sm, db: sm.eventDB}, sm.codec) - decisionDB, err := OpenSamplingDecisionPebble(s.storageDir) + decisionDB, err := OpenSamplingDecisionPebble(sm.storageDir) if err != nil { return err } - s.decisionDB = decisionDB - s.decisionStorage = New(&wrappedDB{sm: s, db: s.decisionDB}, s.codec) + sm.decisionDB = decisionDB + sm.decisionStorage = New(&wrappedDB{sm: sm, db: sm.decisionDB}, sm.codec) return nil } -func (s *StorageManager) Size() (lsm, vlog int64) { - return int64(s.db.Metrics().DiskSpaceUsage() + s.decisionDB.Metrics().DiskSpaceUsage()), 0 +func (sm *StorageManager) Size() (lsm, vlog int64) { + return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 } -func (s *StorageManager) Close() error { - return s.close() +func (sm *StorageManager) Close() error { + return sm.close() } -func (s *StorageManager) close() error { - return errors.Join(s.db.Close(), s.decisionDB.Close()) +func (sm *StorageManager) close() error { + return errors.Join(sm.eventDB.Close(), sm.decisionDB.Close()) } // Reload flushes out pending disk writes to disk by reloading the database. // It does not flush uncommitted writes. // For testing only. -func (s *StorageManager) Reload() error { - if err := s.close(); err != nil { +func (sm *StorageManager) Reload() error { + if err := sm.close(); err != nil { return err } - return s.reset() + return sm.reset() } // 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 { +func (sm *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{}{}: + case sm.runCh <- struct{}{}: } defer func() { - <-s.runCh + <-sm.runCh }() g := errgroup.Group{} g.Go(func() error { - return s.runTTLLoop(stopping, gcInterval) + return sm.runTTLLoop(stopping, gcInterval) }) return g.Wait() } -func (s *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration) error { +func (sm *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration) error { ticker := time.NewTicker(ttl) defer ticker.Stop() for { @@ -176,54 +175,54 @@ func (s *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration) case <-stopping: return nil case <-ticker.C: - if err := s.IncrementPartition(); err != nil { - s.logger.With(logp.Error(err)).Error("failed to increment partition") + if err := sm.IncrementPartition(); err != nil { + sm.logger.With(logp.Error(err)).Error("failed to increment partition") } } } } -func (s *StorageManager) IncrementPartition() error { - oldPID := s.partitionID.Load() - s.partitionID.Store((oldPID + 1) % s.partitionCount) +func (sm *StorageManager) IncrementPartition() error { + oldPID := sm.partitionID.Load() + sm.partitionID.Store((oldPID + 1) % sm.partitionCount) - pidToDelete := (oldPID + s.partitionCount - 1) % s.partitionCount + pidToDelete := (oldPID + sm.partitionCount - 1) % sm.partitionCount lbPrefix := byte(pidToDelete) ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb return errors.Join( - s.db.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), - s.decisionDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), - s.db.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), - s.decisionDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), + sm.eventDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), + sm.decisionDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), + sm.eventDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), + sm.decisionDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), ) } -func (s *StorageManager) ReadSubscriberPosition() ([]byte, error) { - s.subscriberPosMu.Lock() - defer s.subscriberPosMu.Unlock() - return os.ReadFile(filepath.Join(s.storageDir, subscriberPositionFile)) +func (sm *StorageManager) ReadSubscriberPosition() ([]byte, error) { + sm.subscriberPosMu.Lock() + defer sm.subscriberPosMu.Unlock() + return os.ReadFile(filepath.Join(sm.storageDir, subscriberPositionFile)) } -func (s *StorageManager) WriteSubscriberPosition(data []byte) error { - s.subscriberPosMu.Lock() - defer s.subscriberPosMu.Unlock() - return os.WriteFile(filepath.Join(s.storageDir, subscriberPositionFile), data, 0644) +func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { + sm.subscriberPosMu.Lock() + defer sm.subscriberPosMu.Unlock() + return os.WriteFile(filepath.Join(sm.storageDir, subscriberPositionFile), data, 0644) } -func (s *StorageManager) NewReadWriter() SplitReadWriter { +func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ - eventRW: s.eventStorage.NewShardedReadWriter(), - decisionRW: s.decisionStorage.NewShardedReadWriter(), + eventRW: sm.eventStorage.NewShardedReadWriter(), + decisionRW: sm.decisionStorage.NewShardedReadWriter(), } } // NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, // bypassing any wrapper e.g. ShardedReadWriter. // This should be used for testing only, useful to check if data is actually persisted to the DB. -func (s *StorageManager) NewBypassReadWriter() SplitReadWriter { +func (sm *StorageManager) NewBypassReadWriter() SplitReadWriter { return SplitReadWriter{ - eventRW: s.eventStorage.NewReadWriter(), - decisionRW: s.decisionStorage.NewReadWriter(), + eventRW: sm.eventStorage.NewReadWriter(), + decisionRW: sm.decisionStorage.NewReadWriter(), } } From 39fe3404eca292d3fb32d2cd0e98816a8778f42b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:51:57 +0000 Subject: [PATCH 065/184] Remove TTL in WriterOps --- .../apm-server/sampling/eventstorage/storage_bench_test.go | 5 ----- x-pack/apm-server/sampling/processor.go | 1 - x-pack/apm-server/sampling/processor_test.go | 1 - 3 files changed, 7 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index a8842856e52..260b54b51e4 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -8,7 +8,6 @@ import ( "encoding/hex" "fmt" "testing" - "time" "github.com/gofrs/uuid/v5" "github.com/stretchr/testify/assert" @@ -39,7 +38,6 @@ func BenchmarkWriteTransaction(b *testing.B) { b.ResetTimer() wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, StorageLimitInBytes: 0, } for i := 0; i < b.N; i++ { @@ -90,7 +88,6 @@ func BenchmarkReadEvents(b *testing.B) { readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, StorageLimitInBytes: 0, } @@ -172,7 +169,6 @@ func BenchmarkReadEventsHit(b *testing.B) { readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ - TTL: time.Hour, StorageLimitInBytes: 0, } @@ -253,7 +249,6 @@ func BenchmarkIsTraceSampled(b *testing.B) { readWriter := sm.NewBypassReadWriter() defer readWriter.Close() wOpts := eventstorage.WriterOpts{ - TTL: time.Minute, StorageLimitInBytes: 0, } diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 407aa167225..a6c964e2a59 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -584,7 +584,6 @@ func newWrappedRW(rw eventstorage.RW, ttl time.Duration, limit int64) *wrappedRW return &wrappedRW{ rw: rw, writerOpts: eventstorage.WriterOpts{ - TTL: ttl, StorageLimitInBytes: limit, }, } diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 3692027d119..182305704bf 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -63,7 +63,6 @@ func TestProcessAlreadyTailSampled(t *testing.T) { trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} writer := config.DB.NewBypassReadWriter() wOpts := eventstorage.WriterOpts{ - TTL: time.Hour, StorageLimitInBytes: 0, } assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true, wOpts)) From 825be6c8cfb14e8ecf2457636ff7c2359aa78cc7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:53:17 +0000 Subject: [PATCH 066/184] Rename --- x-pack/apm-server/sampling/eventstorage/pebble.go | 4 ++-- .../apm-server/sampling/eventstorage/storage_manager.go | 9 ++++----- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 74de03670d0..e19c059dfff 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -38,7 +38,7 @@ func eventComparer() *pebble.Comparer { return &comparer } -func OpenPebble(storageDir string) (*pebble.DB, error) { +func OpenEventPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "event"), &pebble.Options{ // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, @@ -56,7 +56,7 @@ func OpenPebble(storageDir string) (*pebble.DB, error) { }) } -func OpenSamplingDecisionPebble(storageDir string) (*pebble.DB, error) { +func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "decision"), &pebble.Options{ // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index ac790ec44a1..944a74354b3 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -74,7 +74,6 @@ type StorageManager struct { decisionDB *pebble.DB eventStorage *Storage decisionStorage *Storage - rw *ShardedReadWriter partitionID atomic.Int32 partitionCount int32 @@ -108,16 +107,16 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora return sm, nil } -// reset initializes db, storage, and rw. +// reset initializes db and storage. func (sm *StorageManager) reset() error { - db, err := OpenPebble(sm.storageDir) + eventDB, err := OpenEventPebble(sm.storageDir) if err != nil { return err } - sm.eventDB = db + sm.eventDB = eventDB sm.eventStorage = New(&wrappedDB{sm: sm, db: sm.eventDB}, sm.codec) - decisionDB, err := OpenSamplingDecisionPebble(sm.storageDir) + decisionDB, err := OpenDecisionPebble(sm.storageDir) if err != nil { return err } From a67a31557464c0df43cdfbaddd401d491b250668 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:54:16 +0000 Subject: [PATCH 067/184] Remove pointer receiver --- x-pack/apm-server/sampling/eventstorage/prefix.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index a80fa44e2ea..c659b4e9152 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -72,7 +72,7 @@ func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.AP return rw.writeEntry(key, data) } -func (rw *PrefixReadWriter) writeEntry(key, data []byte) error { +func (rw PrefixReadWriter) writeEntry(key, data []byte) error { if err := rw.db.Set(key, data, pebble.NoSync); err != nil { return err } From 13b4031cc4b2c6b32c3c96efc4ee8efc1096f724 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 17:54:51 +0000 Subject: [PATCH 068/184] Fix PrefixReadWriter flush --- x-pack/apm-server/sampling/eventstorage/prefix.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index c659b4e9152..d8d4dbeb0da 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -127,6 +127,5 @@ func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { } func (rw PrefixReadWriter) Flush() error { - //TODO implement me - panic("implement me") + return nil } From 740c4f9b8a8e46c7000bf8811eb5a6d3164c3f22 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:44:19 +0000 Subject: [PATCH 069/184] make fmt --- x-pack/apm-server/sampling/eventstorage/pebble.go | 4 ++++ x-pack/apm-server/sampling/eventstorage/prefix.go | 4 ++++ x-pack/apm-server/sampling/eventstorage/prefix_test.go | 4 ++++ x-pack/apm-server/sampling/eventstorage/rw.go | 4 ++++ .../apm-server/sampling/eventstorage/storage_manager_test.go | 4 ++++ 5 files changed, 20 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e19c059dfff..5ef09494b38 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -1,3 +1,7 @@ +// 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 ( diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index d8d4dbeb0da..bed2a59c6d5 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -1,3 +1,7 @@ +// 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 ( diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 90bd71cd790..681c42d06cc 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -1,3 +1,7 @@ +// 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_test import ( diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 4b2fae8d8c8..8292d917255 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -1,3 +1,7 @@ +// 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 ( diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 31b13fbaf3e..a6cb33ae9e8 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -1,3 +1,7 @@ +// 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_test import ( From ec1a08700b7b0659d876d3818441c6bb9fc70551 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:45:20 +0000 Subject: [PATCH 070/184] Use upstream pebble, go mod tidy --- go.mod | 8 +------- go.sum | 51 ++++----------------------------------------------- 2 files changed, 5 insertions(+), 54 deletions(-) diff --git a/go.mod b/go.mod index 7657acc631a..a7a46ade280 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,6 @@ 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 github.com/elastic/apm-data v1.15.0 @@ -61,18 +60,15 @@ require ( require ( github.com/DataDog/zstd v1.5.6 // indirect github.com/Microsoft/go-winio v0.6.2 // indirect - github.com/OneOfOne/xxhash v1.2.8 // indirect github.com/armon/go-radix v1.0.0 // indirect github.com/axiomhq/hyperloglog v0.2.0 // indirect 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 github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect - github.com/dgraph-io/ristretto v0.2.0 // indirect - github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140 // indirect github.com/dlclark/regexp2 v1.8.1 // indirect github.com/docker/go-connections v0.5.0 // indirect @@ -158,5 +154,3 @@ 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-20250114162318-fa34738bbef0 diff --git a/go.sum b/go.sum index bdf93fc0ce9..dd03ef3ff81 100644 --- a/go.sum +++ b/go.sum @@ -21,10 +21,6 @@ github.com/KimMachineGun/automemlimit v0.7.0-pre.3 h1:aZVmBE7SmIsRoVHIzpgCJ6rvwn github.com/KimMachineGun/automemlimit v0.7.0-pre.3/go.mod h1:QZxpHaGOQoYvFhv/r4u3U0JTC2ZcOwbSr11UZF46UBM= github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU= -github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/OneOfOne/xxhash v1.2.8 h1:31czK/TI9sNkxIKfaUfGlU47BAxQ0ztGgd9vPyqimf8= -github.com/OneOfOne/xxhash v1.2.8/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/armon/go-radix v1.0.0 h1:F4z6KzEeeQIMeLFa97iZU6vupzoecKdU5TX24SNppXI= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/aws/aws-sdk-go-v2 v1.32.7 h1:ky5o35oENWi0JYWUZkB7WYvVPP+bcRF5/Iq7JWSb5Rw= @@ -60,11 +56,7 @@ 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-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= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= @@ -73,30 +65,22 @@ 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/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= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgraph-io/badger/v2 v2.2007.4 h1:TRWBQg8UrlUhaFdco01nO2uXwzKS7zd+HVdwV/GHc4o= -github.com/dgraph-io/badger/v2 v2.2007.4/go.mod h1:vSw/ax2qojzbN6eXHIx6KPKtCSHJN/Uz0X0VPruTIhk= -github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= -github.com/dgraph-io/ristretto v0.2.0 h1:XAfl+7cmoUDWW/2Lx8TGZQjjxIQ2Ley9DSf52dru4WE= -github.com/dgraph-io/ristretto v0.2.0/go.mod h1:8uBHCU/PBV4Ag0CJrP47b9Ofby5dqWNh4FicAdoqFNU= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140 h1:y7y0Oa6UawqTFPCDw9JG6pdKt4F9pAhHv0B7FMGaGD0= github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140/go.mod h1:c9O8+fpSOX1DM8cPNSkX/qsBWdkD4yd2dpciOWQjpBw= github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5QvfrDyIgxBk= @@ -111,7 +95,6 @@ github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4 github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dop251/goja_nodejs v0.0.0-20171011081505-adff31b136e6 h1:RrkoB0pT3gnjXhL/t10BSP1mcr/0Ldea2uMyuBr2SWk= github.com/dop251/goja_nodejs v0.0.0-20171011081505-adff31b136e6/go.mod h1:hn7BA7c8pLvoGndExHudxTDKZ84Pyvv+90pbBjbTz0Y= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/eapache/go-resiliency v1.7.0 h1:n3NRTnBn5N0Cbi/IeOHuQn9s2UwVUH7Ga0ZWcP+9JTA= @@ -178,7 +161,6 @@ github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2 github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/getsentry/sentry-go v0.29.1 h1:DyZuChN8Hz3ARxGVV8ePaNXh1dQ7d76AiB117xcREwA= github.com/getsentry/sentry-go v0.29.1/go.mod h1:x3AtIzN01d6SiWkderzaH28Tm0lgkafpJ5Bm3li39O0= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= @@ -212,12 +194,10 @@ github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVI github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= -github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/gomodule/redigo v1.8.9 h1:Sl3u+2BI/kk+VEatbj0scLdrFhjPmbxOc1myhDP41ws= @@ -255,10 +235,8 @@ github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/C github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v1.0.2 h1:dV3g9Z/unq5DpblPpw+Oqcv4dU/1omnb4Ok8iPY6p1c= github.com/hashicorp/golang-lru v1.0.2/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4= github.com/imdario/mergo v0.3.16/go.mod h1:WBLT9ZmE3lPoWsEzCh9LPo3TiwVN+ZKEjmz+hD27ysY= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= @@ -282,7 +260,6 @@ github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnr github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.12.3/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.17.11 h1:In6xLpyWOi1+C7tXUUWv2ot1QvBjxevKAaI6IXrJmUc= github.com/klauspost/compress v1.17.11/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -300,7 +277,6 @@ github.com/libp2p/go-reuseport v0.4.0 h1:nR5KU7hD0WxXCJbmw7r2rhRYruNRl2koHw8fQsc github.com/libp2p/go-reuseport v0.4.0/go.mod h1:ZtI03j/wO5hZVDFo2jKywN6bYKWLOy8Se6DrI2E1cLU= github.com/lufia/plan9stats v0.0.0-20220913051719-115f729f3c8c h1:VtwQ41oftZwlMnOEbMWQtSEUgU64U4s+GHk7hZK+jtY= github.com/lufia/plan9stats v0.0.0-20220913051719-115f729f3c8c/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= @@ -310,10 +286,8 @@ github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWE github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mileusna/useragent v1.3.4 h1:MiuRRuvGjEie1+yZHO88UBYg8YBC/ddF6T7F56i3PCk= github.com/mileusna/useragent v1.3.4/go.mod h1:3d8TOmwL/5I8pJjyVDteHtgDGcefrFUX4ccGOMKNYYc= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/hashstructure v1.1.0 h1:P6P1hdjqAAknpY/M1CGipelZgp+4y9ja9kmUZPXP+H0= github.com/mitchellh/hashstructure v1.1.0/go.mod h1:xUDAozZz0Wmdiufv0uyhnHkUTN6/6d8ulp4AwfLKrmA= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -333,7 +307,6 @@ github.com/patrickmn/go-cache v2.1.0+incompatible h1:HRMgzkcYKYpi3C8ajMPV8OFXaaR github.com/patrickmn/go-cache v2.1.0+incompatible/go.mod h1:3Qf8kWWT7OJRJbdiICTKqZju1ZixQ/KpMGzzAfe6+WQ= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0= github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pierrec/lz4/v4 v4.1.21 h1:yOVMLb6qSIDP67pl/5F7RepeKYu/VmTyEXvuMI5d9mQ= github.com/pierrec/lz4/v4 v4.1.21/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= @@ -367,26 +340,16 @@ github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqn github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= -github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryanuber/go-glob v1.0.0 h1:iQh3xXAumdQ+4Ufa5b25cRpC5TYKlno6hsv6Cb3pkBk= github.com/ryanuber/go-glob v1.0.0/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= github.com/shirou/gopsutil/v4 v4.24.9 h1:KIV+/HaHD5ka5f570RZq+2SaeFsb/pq+fp2DGNWYoOI= github.com/shirou/gopsutil/v4 v4.24.9/go.mod h1:3fkaHNeYsUFCGZ8+9vZVWtbyM1k2eRnlL+bWO8Bxa/Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= -github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.8.1 h1:e5/vxKd/rZsfSJMUX1agtjeTDf+qv1/JdBF8gg5k9ZM= github.com/spf13/cobra v1.8.1/go.mod h1:wHxEcudfqmLYa8iTfL+OuZPbBZkmvliBWKIezN3kD9Y= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= @@ -406,7 +369,6 @@ github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFA github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= @@ -419,7 +381,6 @@ github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415 h1:EzJWgHo github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= github.com/xeipuuv/gojsonschema v1.2.0 h1:LhYJRs+L4fBtjZUfuSZIKGeVu0QRy8e5Xi7D17UxZ74= github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= @@ -481,7 +442,6 @@ go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= go.uber.org/zap/exp v0.3.0 h1:6JYzdifzYkGmTdRR59oYH+Ng7k49H9qVpWwNSsGJj3U= go.uber.org/zap/exp v0.3.0/go.mod h1:5I384qq7XGxYyByIhHm6jg5CHkGY0nsTfbDLgDDlgJQ= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -536,11 +496,9 @@ golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -613,7 +571,6 @@ google.golang.org/protobuf v1.36.1 h1:yBPeRvTftaleIgM3PZ/WBIZ7XM/eEYAaEyCwvyjq/g google.golang.org/protobuf v1.36.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= From b2bf6b0a061638ad78278182f995a1ad35bd572a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:45:44 +0000 Subject: [PATCH 071/184] make notice --- NOTICE.txt | 491 +++-------------------------------------------------- 1 file changed, 28 insertions(+), 463 deletions(-) diff --git a/NOTICE.txt b/NOTICE.txt index 493ad689294..0379681fa6c 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -73,189 +73,40 @@ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- -Dependency : github.com/dgraph-io/badger/v2 -Version: v2.2007.4 -Licence type (autodetected): Apache-2.0 +Dependency : github.com/cockroachdb/pebble +Version: v1.1.2 +Licence type (autodetected): BSD-3-Clause -------------------------------------------------------------------------------- -Contents of probable licence file $GOMODCACHE/github.com/dgraph-io/badger/v2@v2.2007.4/LICENSE: - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. +Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/pebble@v1.1.2/LICENSE: - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. +Copyright (c) 2011 The LevelDB-Go Authors. All rights reserved. - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. - END OF TERMS AND CONDITIONS +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -------------------------------------------------------------------------------- @@ -6324,38 +6175,6 @@ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. --------------------------------------------------------------------------------- -Dependency : github.com/cespare/xxhash -Version: v1.1.0 -Licence type (autodetected): MIT --------------------------------------------------------------------------------- - -Contents of probable licence file $GOMODCACHE/github.com/cespare/xxhash@v1.1.0/LICENSE.txt: - -Copyright (c) 2016 Caleb Spare - -MIT License - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -"Software"), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND -NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE -LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION -OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION -WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - - -------------------------------------------------------------------------------- Dependency : github.com/cockroachdb/errors Version: v1.11.3 @@ -6989,43 +6808,6 @@ Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/logtags@v0. limitations under the License. --------------------------------------------------------------------------------- -Dependency : github.com/cockroachdb/pebble -Version: v1.1.2 -Licence type (autodetected): BSD-3-Clause --------------------------------------------------------------------------------- - -Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/pebble@v1.1.2/LICENSE: - -Copyright (c) 2011 The LevelDB-Go Authors. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - * Neither the name of Google Inc. nor the names of its -contributors may be used to endorse or promote products derived from -this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - - -------------------------------------------------------------------------------- Dependency : github.com/cockroachdb/redact Version: v1.1.5 @@ -7473,223 +7255,6 @@ ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. --------------------------------------------------------------------------------- -Dependency : github.com/dgraph-io/ristretto -Version: v0.2.0 -Licence type (autodetected): Apache-2.0 --------------------------------------------------------------------------------- - -Contents of probable licence file $GOMODCACHE/github.com/dgraph-io/ristretto@v0.2.0/LICENSE: - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - --------------------------------------------------------------------------------- -Dependency : github.com/dgryski/go-farm -Version: v0.0.0-20200201041132-a6ae2369ad13 -Licence type (autodetected): MIT --------------------------------------------------------------------------------- - -Contents of probable licence file $GOMODCACHE/github.com/dgryski/go-farm@v0.0.0-20200201041132-a6ae2369ad13/LICENSE: - -Copyright (c) 2014-2017 Damian Gryski -Copyright (c) 2016-2017 Nicola Asuni - Tecnick.com - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. - - - -------------------------------------------------------------------------------- Dependency : github.com/dgryski/go-metro Version: v0.0.0-20211217172704-adc40b04c140 From 1c31a6987f9acfaecb8909e9f3fd58288e9178c2 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:54:53 +0000 Subject: [PATCH 072/184] Delete storage whitebox test --- .../eventstorage/storage_whitebox_test.go | 83 ------------------- 1 file changed, 83 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go b/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go deleted file mode 100644 index e217875f84b..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/storage_whitebox_test.go +++ /dev/null @@ -1,83 +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 - -//func newReadWriter(tb testing.TB) *ReadWriter { -// tempdir := tb.TempDir() -// opts := badger.DefaultOptions("").WithLogger(nil) -// opts = opts.WithInMemory(false) -// opts = opts.WithDir(tempdir).WithValueDir(tempdir) -// -// db, err := badger.Open(opts) -// if err != nil { -// panic(err) -// } -// tb.Cleanup(func() { db.Close() }) -// -// store := New(db, ProtobufCodec{}) -// readWriter := store.NewReadWriter() -// tb.Cleanup(func() { readWriter.Close() }) -// -// readWriter.lazyInit() -// return readWriter -//} -// -//func TestDeleteTraceEvent_ErrTxnTooBig(t *testing.T) { -// readWriter := newReadWriter(t) -// -// traceID, transactionID := writeEvent(t, readWriter) -// assert.True(t, eventExists(t, readWriter, traceID, transactionID)) -// -// fillTxnUntilTxnTooBig(readWriter.txn) -// -// err := readWriter.DeleteTraceEvent(traceID, transactionID) -// assert.NoError(t, err) -// -// assert.False(t, eventExists(t, readWriter, traceID, transactionID)) -//} -// -//func TestWriteTraceEvent_ErrTxnTooBig(t *testing.T) { -// readWriter := newReadWriter(t) -// -// fillTxnUntilTxnTooBig(readWriter.txn) -// -// traceID, transactionID := writeEvent(t, readWriter) -// assert.True(t, eventExists(t, readWriter, traceID, transactionID)) -//} -// -//func writeEvent(t *testing.T, readWriter *ReadWriter) (traceID, transactionID string) { -// traceID = uuid.Must(uuid.NewV4()).String() -// transactionID = uuid.Must(uuid.NewV4()).String() -// transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} -// err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 0, -// }) -// assert.NoError(t, err) -// return -//} -// -//func eventExists(t *testing.T, readWriter *ReadWriter, traceID, transactionID string) (ok bool) { -// var batch modelpb.Batch -// err := readWriter.ReadTraceEvents(traceID, &batch) -// require.NoError(t, err) -// for _, e := range batch { -// if e.GetTransaction().GetId() == transactionID { -// ok = true -// } -// } -// return -//} -// -//func fillTxnUntilTxnTooBig(txn *badger.Txn) { -// var err error -// for { -// if err == badger.ErrTxnTooBig { -// break -// } -// entry := badger.NewEntry([]byte{0}, []byte{}) -// err = txn.SetEntry(entry) -// } -//} From 1fef6a748983bb694d67bfc689cd417325fc44bd Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:55:54 +0000 Subject: [PATCH 073/184] Rename test --- x-pack/apm-server/sampling/eventstorage/prefix_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 681c42d06cc..7b2ad40379c 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -25,7 +25,7 @@ func newPebble(t *testing.T) *pebble.DB { return db } -func TestTTLReadWriter_WriteTraceSampled(t *testing.T) { +func TestPrefixReadWriter_samplingDecision(t *testing.T) { for _, tc := range []struct { sampled bool missing bool From 19a6e7d791dd359f2e37b90f30755c26188a3542 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 18:58:00 +0000 Subject: [PATCH 074/184] Delete TTL in writerOpts --- x-pack/apm-server/sampling/eventstorage/storage.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 9cbec4bfc3a..f8bbc8c7cdd 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -7,7 +7,6 @@ package eventstorage import ( "errors" "io" - "time" "github.com/cockroachdb/pebble" @@ -19,7 +18,6 @@ const ( // over time, to avoid misinterpreting historical data. entryMetaTraceSampled byte = 's' entryMetaTraceUnsampled byte = 'u' - entryMetaTraceEvent byte = 'e' ) var ( @@ -87,7 +85,6 @@ func (s *Storage) NewReadWriter() *ReadWriter { // WriterOpts provides configuration options for writes to storage type WriterOpts struct { - TTL time.Duration StorageLimitInBytes int64 } From adcfc22a8a997fed021bc79ace27b3dea2924a37 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 19:00:10 +0000 Subject: [PATCH 075/184] Try bypassing ShardedReadWriter --- 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 944a74354b3..71223af1fc7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -210,8 +210,8 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ - eventRW: sm.eventStorage.NewShardedReadWriter(), - decisionRW: sm.decisionStorage.NewShardedReadWriter(), + eventRW: sm.eventStorage.NewReadWriter(), + decisionRW: sm.decisionStorage.NewReadWriter(), } } From 3675eef093fde5be8e09c0f624e87601b9582199 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 19:00:36 +0000 Subject: [PATCH 076/184] Revert "Try bypassing ShardedReadWriter" This reverts commit adcfc22a8a997fed021bc79ace27b3dea2924a37. --- 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 71223af1fc7..944a74354b3 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -210,8 +210,8 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ - eventRW: sm.eventStorage.NewReadWriter(), - decisionRW: sm.decisionStorage.NewReadWriter(), + eventRW: sm.eventStorage.NewShardedReadWriter(), + decisionRW: sm.decisionStorage.NewShardedReadWriter(), } } From 145d3921a286d67e9f2c8c81044da9810be90ec8 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 19:07:27 +0000 Subject: [PATCH 077/184] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 944a74354b3..ba38ee9ce61 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -75,7 +75,7 @@ type StorageManager struct { eventStorage *Storage decisionStorage *Storage - partitionID atomic.Int32 + partitionID atomic.Int32 // FIXME: load the correct partition ID on restart partitionCount int32 codec Codec From cd86c647b0c5fbbce80cb7101f799c95d00bfa53 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 19:08:19 +0000 Subject: [PATCH 078/184] Fix ttl loop param --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index ba38ee9ce61..d2d2ced3e17 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -161,7 +161,7 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration g := errgroup.Group{} g.Go(func() error { - return sm.runTTLLoop(stopping, gcInterval) + return sm.runTTLLoop(stopping, ttl) }) return g.Wait() } From 2e8e00b6e70177fceee254fb5af8a2886910ff02 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 21 Jan 2025 19:13:28 +0000 Subject: [PATCH 079/184] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index d2d2ced3e17..a821394fb6f 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -185,6 +185,8 @@ func (sm *StorageManager) IncrementPartition() error { oldPID := sm.partitionID.Load() sm.partitionID.Store((oldPID + 1) % sm.partitionCount) + // FIXME: potential race, wait for a bit before deleting? + pidToDelete := (oldPID + sm.partitionCount - 1) % sm.partitionCount lbPrefix := byte(pidToDelete) ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb From fc833fd5eab9d2040c7dafd2534d41066a554bc9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 22 Jan 2025 16:20:11 +0000 Subject: [PATCH 080/184] Do not use sharded rw --- 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 a821394fb6f..cebd943de11 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -212,8 +212,8 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ - eventRW: sm.eventStorage.NewShardedReadWriter(), - decisionRW: sm.decisionStorage.NewShardedReadWriter(), + eventRW: sm.eventStorage.NewReadWriter(), + decisionRW: sm.decisionStorage.NewReadWriter(), } } From 3d1b41be4df12e8ae505476097c31a6693f2645e Mon Sep 17 00:00:00 2001 From: kruskal <99559985+kruskall@users.noreply.github.com> Date: Wed, 22 Jan 2025 21:37:12 +0100 Subject: [PATCH 081/184] feat: update to badger v2 --- go.mod | 7 +++++-- go.sum | 20 +++++++++++++++---- .../sampling/eventstorage/pebble.go | 8 ++++---- .../sampling/eventstorage/prefix.go | 2 +- .../sampling/eventstorage/prefix_test.go | 4 ++-- .../sampling/eventstorage/storage.go | 2 +- .../sampling/eventstorage/storage_manager.go | 2 +- 7 files changed, 30 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index a7a46ade280..8d3413fffac 100644 --- a/go.mod +++ b/go.mod @@ -5,7 +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/cockroachdb/pebble/v2 v2.0.2 github.com/dustin/go-humanize v1.0.1 github.com/elastic/apm-aggregation v1.2.0 github.com/elastic/apm-data v1.15.0 @@ -63,10 +63,13 @@ require ( github.com/armon/go-radix v1.0.0 // indirect github.com/axiomhq/hyperloglog v0.2.0 // indirect github.com/beorn7/perks v1.0.1 // indirect + github.com/cockroachdb/crlib v0.0.0-20241015224233-894974b3ad94 // 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/pebble v1.1.2 // indirect github.com/cockroachdb/redact v1.1.5 // indirect + github.com/cockroachdb/swiss v0.0.0-20240612210725-f4de07ae6964 // indirect github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140 // indirect @@ -92,7 +95,7 @@ require ( github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/golang/protobuf v1.5.4 // indirect - github.com/golang/snappy v0.0.4 // indirect + github.com/golang/snappy v0.0.5-0.20231225225746-43d5d4cd4e0e // indirect github.com/gomodule/redigo v1.8.9 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect diff --git a/go.sum b/go.sum index dd03ef3ff81..334d64da71a 100644 --- a/go.sum +++ b/go.sum @@ -21,6 +21,8 @@ github.com/KimMachineGun/automemlimit v0.7.0-pre.3 h1:aZVmBE7SmIsRoVHIzpgCJ6rvwn github.com/KimMachineGun/automemlimit v0.7.0-pre.3/go.mod h1:QZxpHaGOQoYvFhv/r4u3U0JTC2ZcOwbSr11UZF46UBM= github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU= +github.com/aclements/go-perfevent v0.0.0-20240301234650-f7843625020f h1:JjxwchlOepwsUWcQwD2mLUAGE9aCp0/ehy6yCHFBOvo= +github.com/aclements/go-perfevent v0.0.0-20240301234650-f7843625020f/go.mod h1:tMDTce/yLLN/SK8gMOxQfnyeMeCg8KGzp0D1cbECEeo= github.com/armon/go-radix v1.0.0 h1:F4z6KzEeeQIMeLFa97iZU6vupzoecKdU5TX24SNppXI= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/aws/aws-sdk-go-v2 v1.32.7 h1:ky5o35oENWi0JYWUZkB7WYvVPP+bcRF5/Iq7JWSb5Rw= @@ -61,18 +63,26 @@ github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UF github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= -github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= +github.com/cockroachdb/crlib v0.0.0-20241015224233-894974b3ad94 h1:bvJv505UUfjzbaIPdNS4AEkHreDqQk6yuNpsdRHpwFA= +github.com/cockroachdb/crlib v0.0.0-20241015224233-894974b3ad94/go.mod h1:Gq51ZeKaFCXk6QwuGM0w1dnaOqc/F5zKT2zA9D6Xeac= +github.com/cockroachdb/datadriven v1.0.3-0.20240530155848-7682d40af056 h1:slXychO2uDM6hYRu4c0pD0udNI8uObfeKN6UInWViS8= +github.com/cockroachdb/datadriven v1.0.3-0.20240530155848-7682d40af056/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/metamorphic v0.0.0-20231108215700-4ba948b56895 h1:XANOgPYtvELQ/h4IrmPAohXqe2pWA8Bwhejr3VQoZsA= +github.com/cockroachdb/metamorphic v0.0.0-20231108215700-4ba948b56895/go.mod h1:aPd7gM9ov9M8v32Yy5NJrDyOcD8z642dqs+F0CeNXfA= 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/pebble/v2 v2.0.2 h1:PmmN7V/rDK+xgp5HiPV9e7ycAalyMjKwSIrcj/4HQz4= +github.com/cockroachdb/pebble/v2 v2.0.2/go.mod h1:NgxgNcWwyG/uxkLUZGM2aelshaLIZvc0hCX7SCfaO8s= 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/swiss v0.0.0-20240612210725-f4de07ae6964 h1:Ew0znI2JatzKy52N1iS5muUsHkf2UJuhocH7uFW7jjs= +github.com/cockroachdb/swiss v0.0.0-20240612210725-f4de07ae6964/go.mod h1:yBRu/cnL4ks9bgy4vAASdjIW+/xMlFwuHKqtmh3GZQg= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= @@ -163,6 +173,8 @@ github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8 github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/getsentry/sentry-go v0.29.1 h1:DyZuChN8Hz3ARxGVV8ePaNXh1dQ7d76AiB117xcREwA= github.com/getsentry/sentry-go v0.29.1/go.mod h1:x3AtIzN01d6SiWkderzaH28Tm0lgkafpJ5Bm3li39O0= +github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9 h1:r5GgOLGbza2wVHRzK7aAj6lWZjfbAwiu/RDCVOKjRyM= +github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= @@ -198,8 +210,8 @@ github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= -github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= -github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.5-0.20231225225746-43d5d4cd4e0e h1:4bw4WeyTYPp0smaXiJZCNnLrvVBqirQVreixayXezGc= +github.com/golang/snappy v0.0.5-0.20231225225746-43d5d4cd4e0e/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/gomodule/redigo v1.8.9 h1:Sl3u+2BI/kk+VEatbj0scLdrFhjPmbxOc1myhDP41ws= github.com/gomodule/redigo v1.8.9/go.mod h1:7ArFNvsTjH8GMMzB4uy1snslv2BwmginuMs06a1uzZE= github.com/google/gnostic-models v0.6.8 h1:yo/ABAfM5IMRsS1VnXjTBvUb61tFIHozhlYvRgGre9I= diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 5ef09494b38..9849c1e271a 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -8,8 +8,8 @@ import ( "bytes" "path/filepath" - "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/bloom" + "github.com/cockroachdb/pebble/v2" + "github.com/cockroachdb/pebble/v2/bloom" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" @@ -51,7 +51,7 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { Levels: []pebble.LevelOptions{ { BlockSize: 16 << 10, - Compression: pebble.SnappyCompression, + Compression: func() pebble.Compression { return pebble.SnappyCompression }, FilterPolicy: bloom.FilterPolicy(10), FilterType: pebble.TableFilter, }, @@ -68,7 +68,7 @@ func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { Levels: []pebble.LevelOptions{ { BlockSize: 2 << 10, - Compression: pebble.NoCompression, + Compression: func() pebble.Compression { return pebble.NoCompression }, FilterPolicy: bloom.FilterPolicy(10), FilterType: pebble.TableFilter, }, diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index bed2a59c6d5..e47d5b407ec 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -8,7 +8,7 @@ import ( "bytes" "fmt" - "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/v2" "github.com/elastic/apm-data/model/modelpb" ) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 7b2ad40379c..3e0cdd38c66 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -8,8 +8,8 @@ import ( "fmt" "testing" - "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/vfs" + "github.com/cockroachdb/pebble/v2" + "github.com/cockroachdb/pebble/v2/vfs" "github.com/gofrs/uuid/v5" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index f8bbc8c7cdd..1cab58d43c6 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -8,7 +8,7 @@ import ( "errors" "io" - "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/v2" "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 cebd943de11..3dcf2983499 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -13,7 +13,7 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/v2" "golang.org/x/sync/errgroup" "github.com/elastic/apm-data/model/modelpb" From 55e2de41f9344ec4ec681447159163f210896f6f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 00:48:31 +0000 Subject: [PATCH 082/184] Reduce memtable size by half --- .../sampling/eventstorage/pebble.go | 21 ++----------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 9849c1e271a..696fbff015d 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -15,24 +15,6 @@ import ( "github.com/elastic/elastic-agent-libs/logp" ) -const ( - // 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 eventComparer() *pebble.Comparer { comparer := *pebble.DefaultComparer // Required for prefix bloom filter @@ -47,7 +29,7 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, Logger: logp.NewLogger(logs.Sampling), - MemTableSize: pebbleMemTableSize, + MemTableSize: 16 << 20, Levels: []pebble.LevelOptions{ { BlockSize: 16 << 10, @@ -65,6 +47,7 @@ func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { // FIXME: Specify FormatMajorVersion to use value blocks? FormatMajorVersion: pebble.FormatNewest, Logger: logp.NewLogger(logs.Sampling), + MemTableSize: 2 << 20, Levels: []pebble.LevelOptions{ { BlockSize: 2 << 10, From ee69d3c33030b45a47140f1f3f04a7980f56b904 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 01:05:01 +0000 Subject: [PATCH 083/184] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 3dcf2983499..85086148bc7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -212,6 +212,7 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ + // FIXME: use sharded? eventRW: sm.eventStorage.NewReadWriter(), decisionRW: sm.decisionStorage.NewReadWriter(), } From 2cc8de4cbd3b905f29103679520e87af73a1552a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 01:06:29 +0000 Subject: [PATCH 084/184] Add FIXME --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 85086148bc7..7f7c58e659b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -127,6 +127,7 @@ func (sm *StorageManager) reset() error { } func (sm *StorageManager) Size() (lsm, vlog int64) { + // FIXME: separate WAL usage? return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 } From 44cb0f01c901292568ef1031f4ea77dedccb437c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 09:41:42 +0000 Subject: [PATCH 085/184] Move rw --- .../sampling/eventstorage/partition_rw.go | 95 +++++++++++++++++++ .../sampling/eventstorage/storage.go | 84 ---------------- 2 files changed, 95 insertions(+), 84 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/partition_rw.go diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go new file mode 100644 index 00000000000..68213064ec9 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -0,0 +1,95 @@ +// 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 ( + "errors" + + "github.com/elastic/apm-data/model/modelpb" +) + +// 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 +} + +// 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() {} + +// 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 { + return nil +} + +// WriteTraceSampled records the tail-sampling decision for the given trace ID. +func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + pid := rw.s.db.PartitionID() + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) +} + +// 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) { + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() + // 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 + sampled, err := NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) + if err == nil { + return sampled, nil + } + + sampled, err = NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) + if err == nil { + return sampled, nil + } + + return false, err +} + +// WriteTraceEvent writes a trace event to storage. +// +// WriteTraceEvent may return before the write is committed to storage. +func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { + pid := rw.s.db.PartitionID() + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) +} + +// DeleteTraceEvent deletes the trace event from storage. +func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { + // FIXME: use range delete + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() + return errors.Join( + NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).DeleteTraceEvent(traceID, id), + NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).DeleteTraceEvent(traceID, id), + ) +} + +// ReadTraceEvents reads trace events with the given trace ID from storage into out. +func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + currentPID := rw.s.db.PartitionID() + prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() + return errors.Join( + NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).ReadTraceEvents(traceID, out), + NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).ReadTraceEvents(traceID, out), + ) +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 1cab58d43c6..220b0fc8bc5 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -87,87 +87,3 @@ func (s *Storage) NewReadWriter() *ReadWriter { type WriterOpts struct { 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 -} - -// 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() {} - -// 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 { - return nil -} - -// WriteTraceSampled records the tail-sampling decision for the given trace ID. -func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - pid := rw.s.db.PartitionID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) -} - -// 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) { - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() - // 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 - sampled, err := NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) - if err == nil { - return sampled, nil - } - - sampled, err = NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) - if err == nil { - return sampled, nil - } - - return false, err -} - -// WriteTraceEvent writes a trace event to storage. -// -// WriteTraceEvent may return before the write is committed to storage. -func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { - pid := rw.s.db.PartitionID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) -} - -// DeleteTraceEvent deletes the trace event from storage. -func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { - // FIXME: use range delete - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() - return errors.Join( - NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).DeleteTraceEvent(traceID, id), - NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).DeleteTraceEvent(traceID, id), - ) -} - -// ReadTraceEvents reads trace events with the given trace ID from storage into out. -func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() - return errors.Join( - NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).ReadTraceEvents(traceID, out), - NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).ReadTraceEvents(traceID, out), - ) -} From 82e102bad8efc467065a7346a35f377f507c8d27 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 10:56:29 +0000 Subject: [PATCH 086/184] Add partitioner --- .../sampling/eventstorage/partitioner.go | 57 +++++++++++++++ .../sampling/eventstorage/partitioner_test.go | 71 +++++++++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 x-pack/apm-server/sampling/eventstorage/partitioner.go create mode 100644 x-pack/apm-server/sampling/eventstorage/partitioner_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go new file mode 100644 index 00000000000..0ed50390812 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -0,0 +1,57 @@ +// 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 "sync/atomic" + +type Partitioner struct { + total int // length of the ring + current atomic.Int32 +} + +func NewPartitioner(actives int) *Partitioner { + return &Partitioner{total: actives + 1} // actives + 1 inactive +} + +func (p *Partitioner) Rotate() { + p.current.Store(int32((int(p.current.Load()) + 1) % p.total)) +} + +func (p *Partitioner) Actives() PartitionIterator { + return PartitionIterator{ + id: int(p.current.Load()), + remaining: p.total - 1 - 1, + total: p.total, + } +} +func (p *Partitioner) Current() PartitionIterator { + return PartitionIterator{ + id: int(p.current.Load()), + remaining: 0, + total: p.total, + } +} + +type PartitionIterator struct { + id int + total int // length of the ring + remaining int +} + +func (it PartitionIterator) Prev() PartitionIterator { + return PartitionIterator{ + id: (it.id + it.total - 1) % it.total, + remaining: it.remaining - 1, + total: it.total, + } +} + +func (it PartitionIterator) Valid() bool { + return it.remaining >= 0 +} + +func (it PartitionIterator) ID() int { + return it.id +} diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go new file mode 100644 index 00000000000..5c96e2983f3 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -0,0 +1,71 @@ +package eventstorage_test + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" +) + +func TestPartitioner(t *testing.T) { + p := eventstorage.NewPartitioner(2) // partition id 0, 1, 2 + + cur := p.Current() + assert.True(t, cur.Valid()) + assert.Equal(t, 0, cur.ID()) + + active := p.Actives() + assert.True(t, active.Valid()) + assert.Equal(t, 0, active.ID()) + active = active.Prev() + assert.True(t, active.Valid()) + assert.Equal(t, 2, active.ID()) + active = active.Prev() + assert.False(t, active.Valid()) + + p.Rotate() + + cur = p.Current() + assert.True(t, cur.Valid()) + assert.Equal(t, 1, cur.ID()) + + active = p.Actives() + assert.True(t, active.Valid()) + assert.Equal(t, 1, active.ID()) + active = active.Prev() + assert.True(t, active.Valid()) + assert.Equal(t, 0, active.ID()) + active = active.Prev() + assert.False(t, active.Valid()) + + p.Rotate() + + cur = p.Current() + assert.True(t, cur.Valid()) + assert.Equal(t, 2, cur.ID()) + + active = p.Actives() + assert.True(t, active.Valid()) + assert.Equal(t, 2, active.ID()) + active = active.Prev() + assert.True(t, active.Valid()) + assert.Equal(t, 1, active.ID()) + active = active.Prev() + assert.False(t, active.Valid()) + + p.Rotate() + + cur = p.Current() + assert.True(t, cur.Valid()) + assert.Equal(t, 0, cur.ID()) + + active = p.Actives() + assert.True(t, active.Valid()) + assert.Equal(t, 0, active.ID()) + active = active.Prev() + assert.True(t, active.Valid()) + assert.Equal(t, 2, active.ID()) + active = active.Prev() + assert.False(t, active.Valid()) +} From 9edb4fc372e30eb86a390c57b46f3d56034e3d5c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 11:24:22 +0000 Subject: [PATCH 087/184] Use partitioner --- .../sampling/eventstorage/partition_rw.go | 54 ++++++++++--------- .../sampling/eventstorage/partitioner.go | 9 ++++ .../sampling/eventstorage/partitioner_test.go | 16 ++++++ .../sampling/eventstorage/storage.go | 4 +- .../sampling/eventstorage/storage_manager.go | 37 ++++++------- 5 files changed, 72 insertions(+), 48 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 68213064ec9..087c4958050 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -38,7 +38,7 @@ 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 { - pid := rw.s.db.PartitionID() + pid := rw.s.db.WritePartition().ID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) } @@ -46,50 +46,54 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() // 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 - sampled, err := NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).IsTraceSampled(traceID) - if err == nil { - return sampled, nil + var errs []error + for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { + sampled, err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).IsTraceSampled(traceID) + if err == nil { + return sampled, nil + } else if err != ErrNotFound { + errs = append(errs, err) + } } - - sampled, err = NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).IsTraceSampled(traceID) - if err == nil { - return sampled, nil + if len(errs) > 0 { + return false, errors.Join(errs...) } - - return false, err + return false, ErrNotFound } // WriteTraceEvent writes a trace event to storage. // // WriteTraceEvent may return before the write is committed to storage. func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { - pid := rw.s.db.PartitionID() + pid := rw.s.db.WritePartition().ID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) } // DeleteTraceEvent deletes the trace event from storage. func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { // FIXME: use range delete - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() - return errors.Join( - NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).DeleteTraceEvent(traceID, id), - NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).DeleteTraceEvent(traceID, id), - ) + var errs []error + for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { + err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).DeleteTraceEvent(traceID, id) + if err != nil { + errs = append(errs, err) + } + } + return errors.Join(errs...) } // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *ReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - currentPID := rw.s.db.PartitionID() - prevPID := (currentPID + rw.s.db.PartitionCount() - 1) % rw.s.db.PartitionCount() - return errors.Join( - NewPrefixReadWriter(rw.s.db, byte(currentPID), rw.s.codec).ReadTraceEvents(traceID, out), - NewPrefixReadWriter(rw.s.db, byte(prevPID), rw.s.codec).ReadTraceEvents(traceID, out), - ) + var errs []error + for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { + err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).ReadTraceEvents(traceID, out) + if err != nil { + errs = append(errs, err) + } + } + return errors.Join(errs...) } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 0ed50390812..c9d1bf7f04d 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -26,6 +26,15 @@ func (p *Partitioner) Actives() PartitionIterator { total: p.total, } } + +func (p *Partitioner) Inactive() PartitionIterator { + return PartitionIterator{ + id: (int(p.current.Load()) + 1) % p.total, + remaining: 0, + total: p.total, + } +} + func (p *Partitioner) Current() PartitionIterator { return PartitionIterator{ id: int(p.current.Load()), diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 5c96e2983f3..86c99e93a2a 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -15,6 +15,10 @@ func TestPartitioner(t *testing.T) { assert.True(t, cur.Valid()) assert.Equal(t, 0, cur.ID()) + inactive := p.Inactive() + assert.True(t, inactive.Valid()) + assert.Equal(t, 1, inactive.ID()) + active := p.Actives() assert.True(t, active.Valid()) assert.Equal(t, 0, active.ID()) @@ -30,6 +34,10 @@ func TestPartitioner(t *testing.T) { assert.True(t, cur.Valid()) assert.Equal(t, 1, cur.ID()) + inactive = p.Inactive() + assert.True(t, inactive.Valid()) + assert.Equal(t, 2, inactive.ID()) + active = p.Actives() assert.True(t, active.Valid()) assert.Equal(t, 1, active.ID()) @@ -45,6 +53,10 @@ func TestPartitioner(t *testing.T) { assert.True(t, cur.Valid()) assert.Equal(t, 2, cur.ID()) + inactive = p.Inactive() + assert.True(t, inactive.Valid()) + assert.Equal(t, 0, inactive.ID()) + active = p.Actives() assert.True(t, active.Valid()) assert.Equal(t, 2, active.ID()) @@ -60,6 +72,10 @@ func TestPartitioner(t *testing.T) { assert.True(t, cur.Valid()) assert.Equal(t, 0, cur.ID()) + inactive = p.Inactive() + assert.True(t, inactive.Valid()) + assert.Equal(t, 1, inactive.ID()) + active = p.Actives() assert.True(t, active.Valid()) assert.Equal(t, 0, active.ID()) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 220b0fc8bc5..ed357285581 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -39,8 +39,8 @@ type db interface { type partitionedDB interface { db - PartitionID() int32 - PartitionCount() int32 + ReadPartitions() PartitionIterator + WritePartition() PartitionIterator } // Storage provides storage for sampled transactions and spans, diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 7f7c58e659b..a58688526a9 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -10,7 +10,6 @@ import ( "os" "path/filepath" "sync" - "sync/atomic" "time" "github.com/cockroachdb/pebble/v2" @@ -28,8 +27,8 @@ const ( ) type wrappedDB struct { - sm *StorageManager - db *pebble.DB + partitioner *Partitioner + db *pebble.DB } func (w *wrappedDB) Get(key []byte) ([]byte, io.Closer, error) { @@ -48,12 +47,12 @@ func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { return w.db.NewIter(o) } -func (w *wrappedDB) PartitionID() int32 { - return w.sm.partitionID.Load() +func (w *wrappedDB) ReadPartitions() PartitionIterator { + return w.partitioner.Actives() } -func (w *wrappedDB) PartitionCount() int32 { - return w.sm.partitionCount +func (w *wrappedDB) WritePartition() PartitionIterator { + return w.partitioner.Current() } type StorageManagerOptions func(*StorageManager) @@ -75,8 +74,7 @@ type StorageManager struct { eventStorage *Storage decisionStorage *Storage - partitionID atomic.Int32 // FIXME: load the correct partition ID on restart - partitionCount int32 + partitioner *Partitioner // FIXME: load the correct partition ID on restart codec Codec @@ -91,11 +89,11 @@ type StorageManager struct { // NewStorageManager returns a new StorageManager with pebble DB at storageDir. func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*StorageManager, error) { sm := &StorageManager{ - storageDir: storageDir, - runCh: make(chan struct{}, 1), - logger: logp.NewLogger(logs.Sampling), - codec: ProtobufCodec{}, - partitionCount: 3, + storageDir: storageDir, + runCh: make(chan struct{}, 1), + logger: logp.NewLogger(logs.Sampling), + codec: ProtobufCodec{}, + partitioner: NewPartitioner(2), } for _, opt := range opts { opt(sm) @@ -114,14 +112,14 @@ func (sm *StorageManager) reset() error { return err } sm.eventDB = eventDB - sm.eventStorage = New(&wrappedDB{sm: sm, db: sm.eventDB}, sm.codec) + sm.eventStorage = New(&wrappedDB{partitioner: sm.partitioner, db: sm.eventDB}, sm.codec) decisionDB, err := OpenDecisionPebble(sm.storageDir) if err != nil { return err } sm.decisionDB = decisionDB - sm.decisionStorage = New(&wrappedDB{sm: sm, db: sm.decisionDB}, sm.codec) + sm.decisionStorage = New(&wrappedDB{partitioner: sm.partitioner, db: sm.decisionDB}, sm.codec) return nil } @@ -183,12 +181,9 @@ func (sm *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration } func (sm *StorageManager) IncrementPartition() error { - oldPID := sm.partitionID.Load() - sm.partitionID.Store((oldPID + 1) % sm.partitionCount) - + sm.partitioner.Rotate() // FIXME: potential race, wait for a bit before deleting? - - pidToDelete := (oldPID + sm.partitionCount - 1) % sm.partitionCount + pidToDelete := sm.partitioner.Inactive().ID() lbPrefix := byte(pidToDelete) ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb return errors.Join( From d08369ff713a05e3f7a065f52b3bab08e16bfc23 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 11:37:47 +0000 Subject: [PATCH 088/184] Add comments --- .../sampling/eventstorage/partitioner.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index c9d1bf7f04d..49d18c6ff3c 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -6,11 +6,21 @@ package eventstorage import "sync/atomic" +// Partitioner is a partitioned ring with `total` number of partitions. +// 1 of them is inactive while all the others are active. +// `current` points at the rightmost active partition. +// +// Example for total=4: +// (A: active, I: inactive) +// A-I-A-A +// ^ +// current type Partitioner struct { total int // length of the ring current atomic.Int32 } +// NewPartitioner returns a partitioner with `actives` number of active partitions. func NewPartitioner(actives int) *Partitioner { return &Partitioner{total: actives + 1} // actives + 1 inactive } @@ -43,6 +53,14 @@ func (p *Partitioner) Current() PartitionIterator { } } +// PartitionIterator is for iterating on partition results. +// In theory Partitioner could have returned a slice of partition IDs, +// but returning an iterator should avoid allocs. +// +// Example usage: +// for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { +// // do something with it.ID() +// } type PartitionIterator struct { id int total int // length of the ring From 7cf2002609060210755fc0b1717c0059117b16a9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 11:45:06 +0000 Subject: [PATCH 089/184] Naming --- .../sampling/eventstorage/storage_manager.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index a58688526a9..c70fcc66bcb 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -24,6 +24,11 @@ const ( // subscriberPositionFile holds the file name used for persisting // the subscriber position across server restarts. subscriberPositionFile = "subscriber_position.json" + + // partitionsPerTTL holds the number of partitions that events in 1 TTL should be stored over. + // Increasing partitionsPerTTL increases read amplification, but decreases storage overhead, + // as TTL GC can be performed sooner. + partitionsPerTTL = 1 ) type wrappedDB struct { @@ -93,7 +98,7 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora runCh: make(chan struct{}, 1), logger: logp.NewLogger(logs.Sampling), codec: ProtobufCodec{}, - partitioner: NewPartitioner(2), + partitioner: NewPartitioner(partitionsPerTTL + 1), } for _, opt := range opts { opt(sm) @@ -160,13 +165,13 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration g := errgroup.Group{} g.Go(func() error { - return sm.runTTLLoop(stopping, ttl) + return sm.runTTLGCLoop(stopping, ttl) }) return g.Wait() } -func (sm *StorageManager) runTTLLoop(stopping <-chan struct{}, ttl time.Duration) error { - ticker := time.NewTicker(ttl) +func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Duration) error { + ticker := time.NewTicker(ttl / partitionsPerTTL) defer ticker.Stop() for { select { From d6a94bd772d7ed259841dc7db05032a4fbf9a410 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 12:22:36 +0000 Subject: [PATCH 090/184] Add prefix test --- .../sampling/eventstorage/prefix_test.go | 103 +++++- .../eventstorage/storage_manager_test.go | 9 + .../sampling/eventstorage/storage_test.go | 305 ------------------ 3 files changed, 105 insertions(+), 312 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/storage_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 3e0cdd38c66..25c781c7ced 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -9,23 +9,112 @@ import ( "testing" "github.com/cockroachdb/pebble/v2" - "github.com/cockroachdb/pebble/v2/vfs" "github.com/gofrs/uuid/v5" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) -func newPebble(t *testing.T) *pebble.DB { - db, err := pebble.Open("", &pebble.Options{ - FS: vfs.NewMem(), - }) +func newEventPebble(t *testing.T) *pebble.DB { + db, err := eventstorage.OpenEventPebble(t.TempDir()) require.NoError(t, err) return db } -func TestPrefixReadWriter_samplingDecision(t *testing.T) { +func newDecisionPebble(t *testing.T) *pebble.DB { + db, err := eventstorage.OpenDecisionPebble(t.TempDir()) + require.NoError(t, err) + return db +} + +func TestPrefixReadWriter_WriteTraceEvent(t *testing.T) { + codec := eventstorage.ProtobufCodec{} + db := newEventPebble(t) + traceID := "foo" + txnID := "bar" + txn := makeTransaction(txnID, traceID) + rw := eventstorage.NewPrefixReadWriter(db, 1, codec) + err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + assert.NoError(t, err) + item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) + assert.NoError(t, err) + defer closer.Close() + var actual modelpb.APMEvent + err = codec.DecodeEvent(item, &actual) + assert.NoError(t, err) + assert.Equal(t, *txn, actual) +} + +func TestPrefixReadWriter_ReadTraceEvents(t *testing.T) { + codec := eventstorage.ProtobufCodec{} + db := newEventPebble(t) + rw := eventstorage.NewPrefixReadWriter(db, 1, codec) + + traceID := "foo" + for _, txnID := range []string{"bar", "baz"} { + txn := makeTransaction(txnID, traceID) + err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + require.NoError(t, err) + } + + var out modelpb.Batch + err := rw.ReadTraceEvents(traceID, &out) + assert.NoError(t, err) + assert.Equal(t, modelpb.Batch{ + makeTransaction("bar", traceID), + makeTransaction("baz", traceID), + }, out) +} + +func TestPrefixReadWriter_DeleteTraceEvent(t *testing.T) { + codec := eventstorage.ProtobufCodec{} + db := newEventPebble(t) + traceID := "foo" + txnID := "bar" + txn := makeTransaction(txnID, traceID) + rw := eventstorage.NewPrefixReadWriter(db, 1, codec) + err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + require.NoError(t, err) + + key := append([]byte{1}, []byte("foo:bar")...) + + _, closer, err := db.Get(key) + assert.NoError(t, err) + err = closer.Close() + assert.NoError(t, err) + + err = rw.DeleteTraceEvent(traceID, txnID) + assert.NoError(t, err) + + _, _, err = db.Get(key) + assert.ErrorIs(t, err, pebble.ErrNotFound) +} + +func TestPrefixReadWriter_WriteTraceSampled(t *testing.T) { + for _, sampled := range []bool{true, false} { + t.Run(fmt.Sprintf("sampled=%v", sampled), func(t *testing.T) { + codec := eventstorage.ProtobufCodec{} + db := newDecisionPebble(t) + traceID := "foo" + rw := eventstorage.NewPrefixReadWriter(db, 1, codec) + err := rw.WriteTraceSampled(traceID, sampled, eventstorage.WriterOpts{}) + assert.NoError(t, err) + item, closer, err := db.Get(append([]byte{1}, []byte("foo")...)) + assert.NoError(t, err) + defer closer.Close() + assert.NoError(t, err) + if sampled { + assert.Equal(t, []byte{'s'}, item) + } else { + assert.Equal(t, []byte{'u'}, item) + } + }) + } +} + +func TestPrefixReadWriter_IsTraceSampled(t *testing.T) { for _, tc := range []struct { sampled bool missing bool @@ -41,7 +130,7 @@ func TestPrefixReadWriter_samplingDecision(t *testing.T) { }, } { t.Run(fmt.Sprintf("sampled=%v,missing=%v", tc.sampled, tc.missing), func(t *testing.T) { - db := newPebble(t) + db := newDecisionPebble(t) rw := eventstorage.NewPrefixReadWriter(db, 1, nopCodec{}) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index a6cb33ae9e8..045c49d932c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -14,6 +14,15 @@ import ( "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) +func newStorageManager(tb testing.TB, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { + sm, err := eventstorage.NewStorageManager(tb.TempDir(), opts...) + if err != nil { + panic(err) + } + tb.Cleanup(func() { sm.Close() }) + return sm +} + func TestStorageManager_samplingDecisionTTL(t *testing.T) { sm := newStorageManager(t) rw := sm.NewBypassReadWriter() diff --git a/x-pack/apm-server/sampling/eventstorage/storage_test.go b/x-pack/apm-server/sampling/eventstorage/storage_test.go deleted file mode 100644 index 93cdbb8ab88..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/storage_test.go +++ /dev/null @@ -1,305 +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_test - -import ( - "testing" - - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" -) - -//func TestWriteEvents(t *testing.T) { -// // Run two tests: -// // - 1 transaction and 1 span -// // - 1 transaction and 100 spans -// // -// // The latter test will cause ReadTraceEvents to implicitly call flush. -// t.Run("no_flush", func(t *testing.T) { -// testWriteEvents(t, 1) -// }) -// t.Run("implicit_flush", func(t *testing.T) { -// testWriteEvents(t, 100) -// }) -//} -// -//func testWriteEvents(t *testing.T, numSpans int) { -// db := newBadgerDB(t, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// readWriter := store.NewShardedReadWriter() -// defer readWriter.Close() -// -// beforeWrite := time.Now() -// traceID := uuid.Must(uuid.NewV4()).String() -// transactionID := uuid.Must(uuid.NewV4()).String() -// transaction := modelpb.APMEvent{ -// Transaction: &modelpb.Transaction{Id: transactionID}, -// } -// wOpts := eventstorage.WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 0, -// } -// assert.NoError(t, readWriter.WriteTraceEvent(traceID, transactionID, &transaction, wOpts)) -// -// var spanEvents []*modelpb.APMEvent -// for i := 0; i < numSpans; i++ { -// spanID := uuid.Must(uuid.NewV4()).String() -// span := modelpb.APMEvent{ -// Span: &modelpb.Span{Id: spanID}, -// } -// assert.NoError(t, readWriter.WriteTraceEvent(traceID, spanID, &span, wOpts)) -// spanEvents = append(spanEvents, &span) -// } -// afterWrite := time.Now() -// -// // We can read our writes without flushing. -// var batch modelpb.Batch -// assert.NoError(t, readWriter.ReadTraceEvents(traceID, &batch)) -// spanEvents = append(spanEvents, &transaction) -// assert.Empty(t, cmp.Diff(modelpb.Batch(spanEvents), batch, -// cmpopts.SortSlices(func(e1 *modelpb.APMEvent, e2 *modelpb.APMEvent) bool { -// return e1.GetSpan().GetId() < e2.GetSpan().GetId() -// }), -// protocmp.Transform()), -// ) -// -// // Flush in order for the writes to be visible to other readers. -// assert.NoError(t, readWriter.Flush()) -// -// var recorded modelpb.Batch -// assert.NoError(t, db.View(func(txn *badger.Txn) error { -// iter := txn.NewIterator(badger.IteratorOptions{ -// Prefix: []byte(traceID), -// }) -// defer iter.Close() -// for iter.Rewind(); iter.Valid(); iter.Next() { -// item := iter.Item() -// expiresAt := item.ExpiresAt() -// expiryTime := time.Unix(int64(expiresAt), 0) -// -// // The expiry time should be somewhere between when we -// // started and finished writing + the TTL. The expiry time -// // is recorded as seconds since the Unix epoch, hence the -// // truncation. -// lowerBound := beforeWrite.Add(wOpts.TTL).Truncate(time.Second) -// upperBound := afterWrite.Add(wOpts.TTL).Truncate(time.Second) -// assert.Condition(t, func() bool { -// return !lowerBound.After(expiryTime) -// }, "expiry time %s is before %s", expiryTime, lowerBound) -// assert.Condition(t, func() bool { -// return !expiryTime.After(upperBound) -// }, "expiry time %s is after %s", expiryTime, upperBound) -// -// var event modelpb.APMEvent -// require.Equal(t, "e", string(item.UserMeta())) -// assert.NoError(t, item.Value(func(data []byte) error { -// return proto.Unmarshal(data, &event) -// })) -// recorded = append(recorded, &event) -// } -// return nil -// })) -// assert.Empty(t, cmp.Diff(batch, recorded, protocmp.Transform())) -//} -// -//func TestWriteTraceSampled(t *testing.T) { -// db := newBadgerDB(t, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// readWriter := store.NewShardedReadWriter() -// defer readWriter.Close() -// wOpts := eventstorage.WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 0, -// } -// -// before := time.Now() -// assert.NoError(t, readWriter.WriteTraceSampled("sampled_trace_id", true, wOpts)) -// assert.NoError(t, readWriter.WriteTraceSampled("unsampled_trace_id", false, wOpts)) -// -// // We can read our writes without flushing. -// isSampled, err := readWriter.IsTraceSampled("sampled_trace_id") -// assert.NoError(t, err) -// assert.True(t, isSampled) -// -// // Flush in order for the writes to be visible to other readers. -// assert.NoError(t, readWriter.Flush()) -// -// sampled := make(map[string]bool) -// assert.NoError(t, db.View(func(txn *badger.Txn) error { -// iter := txn.NewIterator(badger.IteratorOptions{}) -// defer iter.Close() -// for iter.Rewind(); iter.Valid(); iter.Next() { -// item := iter.Item() -// expiresAt := item.ExpiresAt() -// expiryTime := time.Unix(int64(expiresAt), 0) -// assert.Condition(t, func() bool { -// return !before.After(expiryTime) && !expiryTime.After(before.Add(wOpts.TTL)) -// }) -// -// key := string(item.Key()) -// switch meta := item.UserMeta(); meta { -// case 's': -// sampled[key] = true -// case 'u': -// sampled[key] = false -// default: -// t.Fatalf("invalid meta %q", meta) -// } -// assert.Zero(t, item.ValueSize()) -// } -// return nil -// })) -// assert.Equal(t, map[string]bool{ -// "sampled_trace_id": true, -// "unsampled_trace_id": false, -// }, sampled) -//} -// -//func TestReadTraceEvents(t *testing.T) { -// db := newBadgerDB(t, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// -// traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} -// require.NoError(t, db.Update(func(txn *badger.Txn) error { -// key := append(traceID[:], ":12345678"...) -// value, err := proto.Marshal(&modelpb.APMEvent{Transaction: &modelpb.Transaction{Name: "transaction"}}) -// if err != nil { -// return err -// } -// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { -// return err -// } -// -// key = append(traceID[:], ":87654321"...) -// value, err = proto.Marshal(&modelpb.APMEvent{Span: &modelpb.Span{Name: "span"}}) -// if err != nil { -// return err -// } -// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { -// return err -// } -// -// // Write an entry with the trace ID as a prefix, but with no -// // proceeding colon, causing it to be ignored. -// key = append(traceID[:], "nocolon"...) -// value = []byte(`not-protobuf`) -// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { -// return err -// } -// -// // Write an entry with an unknown meta value. It will be ignored. -// key = append(traceID[:], ":11111111"...) -// value = []byte(`not-protobuf`) -// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('?')); err != nil { -// return err -// } -// return nil -// })) -// -// reader := store.NewShardedReadWriter() -// defer reader.Close() -// -// var events modelpb.Batch -// assert.NoError(t, reader.ReadTraceEvents(string(traceID[:]), &events)) -// assert.Empty(t, cmp.Diff(modelpb.Batch{ -// {Transaction: &modelpb.Transaction{Name: "transaction"}}, -// {Span: &modelpb.Span{Name: "span"}}, -// }, events, protocmp.Transform())) -//} -// -//func TestReadTraceEventsDecodeError(t *testing.T) { -// db := newBadgerDB(t, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// -// traceID := [...]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16} -// require.NoError(t, db.Update(func(txn *badger.Txn) error { -// key := append(traceID[:], ":12345678"...) -// value := []byte(`wat`) -// if err := txn.SetEntry(badger.NewEntry(key, value).WithMeta('e')); err != nil { -// return err -// } -// return nil -// })) -// -// reader := store.NewShardedReadWriter() -// defer reader.Close() -// -// var events modelpb.Batch -// err := reader.ReadTraceEvents(string(traceID[:]), &events) -// assert.Error(t, err) -//} -// -//func TestIsTraceSampled(t *testing.T) { -// db := newBadgerDB(t, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// -// require.NoError(t, db.Update(func(txn *badger.Txn) error { -// if err := txn.SetEntry(badger.NewEntry([]byte("sampled_trace_id"), nil).WithMeta('s')); err != nil { -// return err -// } -// if err := txn.SetEntry(badger.NewEntry([]byte("unsampled_trace_id"), nil).WithMeta('u')); err != nil { -// return err -// } -// return nil -// })) -// -// reader := store.NewShardedReadWriter() -// defer reader.Close() -// -// sampled, err := reader.IsTraceSampled("sampled_trace_id") -// assert.NoError(t, err) -// assert.True(t, sampled) -// -// sampled, err = reader.IsTraceSampled("unsampled_trace_id") -// assert.NoError(t, err) -// assert.False(t, sampled) -// -// _, err = reader.IsTraceSampled("unknown_trace_id") -// assert.Equal(t, err, eventstorage.ErrNotFound) -//} -// -//func TestStorageLimit(t *testing.T) { -// tempdir := t.TempDir() -// opts := func() badger.Options { -// opts := badgerOptions() -// opts = opts.WithInMemory(false) -// opts = opts.WithDir(tempdir).WithValueDir(tempdir) -// return opts -// } -// -// // Open and close the database to create a non-empty value log file, -// // which will cause writes below to fail due to the storage limit being -// // exceeded. We would otherwise have to rely on Badger's one minute -// // timer to refresh the size. -// db := newBadgerDB(t, opts) -// db.Close() -// db = newBadgerDB(t, opts) -// -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// readWriter := store.NewReadWriter() -// defer readWriter.Close() -// -// traceID := uuid.Must(uuid.NewV4()).String() -// transactionID := uuid.Must(uuid.NewV4()).String() -// transaction := modelpb.APMEvent{Transaction: &modelpb.Transaction{Id: transactionID}} -// err := readWriter.WriteTraceEvent(traceID, transactionID, &transaction, eventstorage.WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 1, -// }) -// assert.ErrorIs(t, err, eventstorage.ErrLimitReached) -// -// // Assert the stored write has been discarded. -// var batch modelpb.Batch -// readWriter.ReadTraceEvents(traceID, &batch) -// assert.Equal(t, 0, len(batch)) -//} - -func newStorageManager(tb testing.TB, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { - sm, err := eventstorage.NewStorageManager(tb.TempDir(), opts...) - if err != nil { - panic(err) - } - tb.Cleanup(func() { sm.Close() }) - return sm -} From 69fff0d513d46159029af4795f7fae9a3c8f7858 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 12:27:11 +0000 Subject: [PATCH 091/184] Remove sharding completely --- x-pack/apm-server/sampling/config.go | 2 +- x-pack/apm-server/sampling/config_test.go | 2 +- .../sampling/eventstorage/partition_rw.go | 5 - .../sampling/eventstorage/sharded.go | 136 ------------------ .../eventstorage/sharded_bench_test.go | 56 -------- .../sampling/eventstorage/storage.go | 9 -- .../sampling/eventstorage/storage_manager.go | 3 +- 7 files changed, 3 insertions(+), 210 deletions(-) delete mode 100644 x-pack/apm-server/sampling/eventstorage/sharded.go delete mode 100644 x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 6dd3add8984..cc879063aa5 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -100,7 +100,7 @@ type StorageConfig struct { // DB will not be closed when the processor is closed. DB *eventstorage.StorageManager - // Storage holds the read writers which provide sharded, locked access to storage. + // Storage holds the read writers which provide access to storage. // // Storage lives outside processor lifecycle and will not be closed when processor // is closed diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index 12344e9637e..b244329604a 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -74,7 +74,7 @@ func TestNewProcessorConfigInvalid(t *testing.T) { config.DB = &eventstorage.StorageManager{} assertInvalidConfigError("invalid storage config: Storage unspecified") - config.Storage = &eventstorage.ShardedReadWriter{} + config.Storage = &eventstorage.ReadWriter{} assertInvalidConfigError("invalid storage config: StorageDir unspecified") config.StorageDir = "tbs" diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 087c4958050..0f10a0c5f0c 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -12,11 +12,6 @@ import ( // 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 } diff --git a/x-pack/apm-server/sampling/eventstorage/sharded.go b/x-pack/apm-server/sampling/eventstorage/sharded.go deleted file mode 100644 index 032f3fddad7..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/sharded.go +++ /dev/null @@ -1,136 +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 ( - "errors" - "runtime" - "sync" - - "github.com/cespare/xxhash/v2" - - "github.com/elastic/apm-data/model/modelpb" -) - -// 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 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 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 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 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/eventstorage/sharded_bench_test.go b/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go deleted file mode 100644 index 5f2b8b955f4..00000000000 --- a/x-pack/apm-server/sampling/eventstorage/sharded_bench_test.go +++ /dev/null @@ -1,56 +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_test - -// -//func BenchmarkShardedWriteTransactionUncontended(b *testing.B) { -// db := newBadgerDB(b, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// sharded := store.NewShardedReadWriter() -// defer sharded.Close() -// wOpts := eventstorage.WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 0, -// } -// -// b.RunParallel(func(pb *testing.PB) { -// traceID := uuid.Must(uuid.NewV4()).String() -// transaction := &modelpb.APMEvent{ -// Transaction: &modelpb.Transaction{Id: traceID}, -// } -// for pb.Next() { -// if err := sharded.WriteTraceEvent(traceID, traceID, transaction, wOpts); err != nil { -// b.Fatal(err) -// } -// } -// }) -//} -// -//func BenchmarkShardedWriteTransactionContended(b *testing.B) { -// db := newBadgerDB(b, badgerOptions) -// store := eventstorage.New(db, eventstorage.ProtobufCodec{}) -// sharded := store.NewShardedReadWriter() -// defer sharded.Close() -// wOpts := eventstorage.WriterOpts{ -// TTL: time.Minute, -// StorageLimitInBytes: 0, -// } -// -// // Use a single trace ID, causing all events to go through -// // the same sharded writer, contending for a single lock. -// traceID := uuid.Must(uuid.NewV4()).String() -// -// b.RunParallel(func(pb *testing.PB) { -// transactionID := uuid.Must(uuid.NewV4()).String() -// transaction := &modelpb.APMEvent{ -// Transaction: &modelpb.Transaction{Id: transactionID}, -// } -// for pb.Next() { -// if err := sharded.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { -// b.Fatal(err) -// } -// } -// }) -//} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index ed357285581..5c780b79f87 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -64,15 +64,6 @@ func New(db partitionedDB, codec Codec) *Storage { } } -// 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. // diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index c70fcc66bcb..1808d42f989 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -213,14 +213,13 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { func (sm *StorageManager) NewReadWriter() SplitReadWriter { return SplitReadWriter{ - // FIXME: use sharded? eventRW: sm.eventStorage.NewReadWriter(), decisionRW: sm.decisionStorage.NewReadWriter(), } } // NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, -// bypassing any wrapper e.g. ShardedReadWriter. +// bypassing any wrappers. // This should be used for testing only, useful to check if data is actually persisted to the DB. func (sm *StorageManager) NewBypassReadWriter() SplitReadWriter { return SplitReadWriter{ From 505b8c17a22b5e2a021f88b73dcd85850bee5658 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 12:35:03 +0000 Subject: [PATCH 092/184] Rename to PartitionReadWriter --- x-pack/apm-server/sampling/config_test.go | 2 +- .../sampling/eventstorage/partition_rw.go | 18 +++++++++--------- .../sampling/eventstorage/storage.go | 8 ++++---- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index b244329604a..4b04bdb0d02 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -74,7 +74,7 @@ func TestNewProcessorConfigInvalid(t *testing.T) { config.DB = &eventstorage.StorageManager{} assertInvalidConfigError("invalid storage config: Storage unspecified") - config.Storage = &eventstorage.ReadWriter{} + config.Storage = &eventstorage.PartitionReadWriter{} assertInvalidConfigError("invalid storage config: StorageDir unspecified") config.StorageDir = "tbs" diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 0f10a0c5f0c..f7d7e8cc7ab 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -10,9 +10,9 @@ import ( "github.com/elastic/apm-data/model/modelpb" ) -// ReadWriter provides a means of reading events from storage, and batched +// PartitionReadWriter provides a means of reading events from storage, and batched // writing of events to storage. -type ReadWriter struct { +type PartitionReadWriter struct { s *Storage } @@ -20,19 +20,19 @@ type ReadWriter struct { // // This must be called when the writer is no longer needed, in order to reclaim // resources. -func (rw *ReadWriter) Close() {} +func (rw *PartitionReadWriter) Close() {} // 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 { +func (rw *PartitionReadWriter) Flush() error { return nil } // WriteTraceSampled records the tail-sampling decision for the given trace ID. -func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { +func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { pid := rw.s.db.WritePartition().ID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) } @@ -40,7 +40,7 @@ func (rw *ReadWriter) WriteTraceSampled(traceID string, sampled bool, opts Write // 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) { +func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { // 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 @@ -63,13 +63,13 @@ func (rw *ReadWriter) IsTraceSampled(traceID string) (bool, error) { // WriteTraceEvent writes a trace event to storage. // // WriteTraceEvent may return before the write is committed to storage. -func (rw *ReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { +func (rw *PartitionReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { pid := rw.s.db.WritePartition().ID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) } // DeleteTraceEvent deletes the trace event from storage. -func (rw *ReadWriter) DeleteTraceEvent(traceID, id string) error { +func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // FIXME: use range delete var errs []error for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { @@ -82,7 +82,7 @@ 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 { +func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { var errs []error for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).ReadTraceEvents(traceID, out) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 5c780b79f87..b36edab1be1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -64,12 +64,12 @@ func New(db partitionedDB, codec Codec) *Storage { } } -// NewReadWriter returns a new ReadWriter for reading events from and +// NewReadWriter returns a new PartitionReadWriter 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 { - return &ReadWriter{ +// The returned PartitionReadWriter must be closed when it is no longer needed. +func (s *Storage) NewReadWriter() *PartitionReadWriter { + return &PartitionReadWriter{ s: s, } } From 05a6df417463e02a63f30901bb746181bb13d06b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 13:48:19 +0000 Subject: [PATCH 093/184] Add header --- x-pack/apm-server/sampling/eventstorage/partitioner_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 86c99e93a2a..60be111af8c 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -1,3 +1,7 @@ +// 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_test import ( From 2c850cfeab5cafb801473d22ab0aee5b0011df56 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 13:50:24 +0000 Subject: [PATCH 094/184] go mod tidy --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 8d3413fffac..39795b06ee7 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,6 @@ 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/v2 v2.0.2 github.com/dustin/go-humanize v1.0.1 github.com/elastic/apm-aggregation v1.2.0 @@ -63,6 +62,7 @@ require ( github.com/armon/go-radix v1.0.0 // indirect github.com/axiomhq/hyperloglog v0.2.0 // indirect github.com/beorn7/perks v1.0.1 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cockroachdb/crlib v0.0.0-20241015224233-894974b3ad94 // indirect github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect From e64642892a79275da473ed439fb35ba6f2b0e1c2 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 14:09:33 +0000 Subject: [PATCH 095/184] Add storage limit handling --- x-pack/apm-server/sampling/eventstorage/rw.go | 76 ++++++++++ .../sampling/eventstorage/storage_manager.go | 57 +++----- x-pack/apm-server/sampling/processor_test.go | 134 ++++++++---------- 3 files changed, 153 insertions(+), 114 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 8292d917255..863e29a7199 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -16,3 +16,79 @@ type RW interface { DeleteTraceEvent(traceID, id string) error Flush() error } + +type SplitReadWriter struct { + eventRW, decisionRW RW +} + +func (s SplitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + return s.eventRW.ReadTraceEvents(traceID, out) +} + +func (s SplitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { + return s.eventRW.WriteTraceEvent(traceID, id, event, opts) +} + +func (s SplitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + return s.decisionRW.WriteTraceSampled(traceID, sampled, opts) +} + +func (s SplitReadWriter) IsTraceSampled(traceID string) (bool, error) { + return s.decisionRW.IsTraceSampled(traceID) +} + +func (s SplitReadWriter) DeleteTraceEvent(traceID, id string) error { + return s.eventRW.DeleteTraceEvent(traceID, id) +} + +func (s SplitReadWriter) Flush() error { + return nil +} + +func (s SplitReadWriter) Close() error { + return nil +} + +type storageLimitChecker interface { + StorageLimitReached() bool +} + +type StorageLimitReadWriter struct { + checker storageLimitChecker + nextRW RW +} + +func (s StorageLimitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + return s.nextRW.ReadTraceEvents(traceID, out) +} + +func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { + if s.checker.StorageLimitReached() { + return ErrLimitReached + } + return s.nextRW.WriteTraceEvent(traceID, id, event, opts) +} + +func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { + if s.checker.StorageLimitReached() { + return ErrLimitReached + } + return s.nextRW.WriteTraceSampled(traceID, sampled, opts) +} + +func (s StorageLimitReadWriter) IsTraceSampled(traceID string) (bool, error) { + return s.nextRW.IsTraceSampled(traceID) +} + +func (s StorageLimitReadWriter) DeleteTraceEvent(traceID, id string) error { + // Technically DeleteTraceEvent writes, but it should have a net effect of reducing disk usage + return s.nextRW.DeleteTraceEvent(traceID, id) +} + +func (s StorageLimitReadWriter) Flush() error { + return s.nextRW.Flush() +} + +func (s StorageLimitReadWriter) Close() error { + 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 1808d42f989..79f05ac17ea 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -10,12 +10,12 @@ import ( "os" "path/filepath" "sync" + "sync/atomic" "time" "github.com/cockroachdb/pebble/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" ) @@ -81,6 +81,8 @@ type StorageManager struct { partitioner *Partitioner // FIXME: load the correct partition ID on restart + storageLimit atomic.Uint64 + codec Codec // subscriberPosMu protects the subscriber file from concurrent RW. @@ -129,11 +131,17 @@ func (sm *StorageManager) reset() error { return nil } -func (sm *StorageManager) Size() (lsm, vlog int64) { +func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it vlog // FIXME: separate WAL usage? return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 } +func (sm *StorageManager) StorageLimitReached() bool { + limit := sm.storageLimit.Load() + lsm, vlog := sm.Size() // FIXME: what's the overhead? + return limit != 0 && uint64(lsm+vlog) > limit +} + func (sm *StorageManager) Close() error { return sm.close() } @@ -163,6 +171,8 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration <-sm.runCh }() + sm.storageLimit.Store(storageLimit) + g := errgroup.Group{} g.Go(func() error { return sm.runTTLGCLoop(stopping, ttl) @@ -211,10 +221,13 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { return os.WriteFile(filepath.Join(sm.storageDir, subscriberPositionFile), data, 0644) } -func (sm *StorageManager) NewReadWriter() SplitReadWriter { - return SplitReadWriter{ - eventRW: sm.eventStorage.NewReadWriter(), - decisionRW: sm.decisionStorage.NewReadWriter(), +func (sm *StorageManager) NewReadWriter() StorageLimitReadWriter { + return StorageLimitReadWriter{ + checker: sm, + nextRW: SplitReadWriter{ + eventRW: sm.eventStorage.NewReadWriter(), + decisionRW: sm.decisionStorage.NewReadWriter(), + }, } } @@ -227,35 +240,3 @@ func (sm *StorageManager) NewBypassReadWriter() SplitReadWriter { decisionRW: sm.decisionStorage.NewReadWriter(), } } - -type SplitReadWriter struct { - eventRW, decisionRW RW -} - -func (s SplitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - return s.eventRW.ReadTraceEvents(traceID, out) -} - -func (s SplitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { - return s.eventRW.WriteTraceEvent(traceID, id, event, opts) -} - -func (s SplitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - return s.decisionRW.WriteTraceSampled(traceID, sampled, opts) -} - -func (s SplitReadWriter) IsTraceSampled(traceID string) (bool, error) { - return s.decisionRW.IsTraceSampled(traceID) -} - -func (s SplitReadWriter) DeleteTraceEvent(traceID, id string) error { - return s.eventRW.DeleteTraceEvent(traceID, id) -} - -func (s SplitReadWriter) Flush() error { - return nil -} - -func (s SplitReadWriter) Close() error { - return nil -} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 182305704bf..b907c5a6e96 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -646,82 +646,64 @@ 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. + 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) + config.TTL = time.Hour + // 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))) + }) + + err := config.DB.Reload() + assert.NoError(t, err) + + lsm, vlog := config.DB.Size() + assert.Greater(t, lsm+vlog, int64(10<<10)) + + config.StorageLimit = 10 << 10 // Set the storage limit to smaller than existing storage + + processor := writeBatch(1000, config, func(b modelpb.Batch) { + assert.Len(t, b, 1000) + }) + + failedWrites := collectProcessorMetrics(processor).Ints["sampling.events.failed_writes"] + t.Log(failedWrites) + // Ensure that there are some failed writes. + if failedWrites >= 1 { + return + } + + t.Fatal("storage limit error never thrown") +} func TestProcessRemoteTailSamplingPersistence(t *testing.T) { config := newTempdirConfig(t) From 1b993e1ba1b42c81c6b5604da6ec7a96598df0df Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 14:36:28 +0000 Subject: [PATCH 096/184] Stop exposing storage --- x-pack/apm-server/main.go | 16 +--------------- x-pack/apm-server/sampling/config.go | 9 ++------- x-pack/apm-server/sampling/config_test.go | 3 --- .../sampling/eventstorage/storage_manager.go | 4 ++++ x-pack/apm-server/sampling/processor.go | 7 ++++++- x-pack/apm-server/sampling/processor_test.go | 18 +++++------------- 6 files changed, 18 insertions(+), 39 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index de9c0bcd038..c53def6348c 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -44,9 +44,6 @@ var ( dbMu sync.Mutex db *eventstorage.StorageManager - storageMu sync.Mutex - storage eventstorage.RW - // samplerUUID is a UUID used to identify sampled trace ID documents // published by this process. samplerUUID = uuid.Must(uuid.NewV4()) @@ -119,9 +116,8 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er storageDir := paths.Resolve(paths.Data, tailSamplingStorageDir) db, err := getDB(storageDir) if err != nil { - return nil, fmt.Errorf("failed to get Badger database: %w", err) + return nil, fmt.Errorf("failed to get TBS database: %w", err) } - readWriter := getStorage(db) policies := make([]sampling.Policy, len(tailSamplingConfig.Policies)) for i, in := range tailSamplingConfig.Policies { @@ -156,7 +152,6 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er }, StorageConfig: sampling.StorageConfig{ DB: db, - Storage: readWriter, StorageDir: storageDir, StorageGCInterval: tailSamplingConfig.StorageGCInterval, StorageLimit: tailSamplingConfig.StorageLimitParsed, @@ -179,15 +174,6 @@ func getDB(storageDir string) (*eventstorage.StorageManager, error) { return db, nil } -func getStorage(sm *eventstorage.StorageManager) eventstorage.RW { - storageMu.Lock() - defer storageMu.Unlock() - if storage == nil { - storage = sm.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 cc879063aa5..853c0d56e66 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -100,10 +100,8 @@ type StorageConfig struct { // DB will not be closed when the processor is closed. DB *eventstorage.StorageManager - // Storage holds the read writers which provide access to storage. - // - // Storage lives outside processor lifecycle and will not be closed when processor - // is closed + // Storage overrides the default DB storage RW. + // For testing only. Storage eventstorage.RW // StorageDir holds the directory in which event storage will be maintained. @@ -242,9 +240,6 @@ func (config StorageConfig) validate() error { if config.DB == nil { return errors.New("DB unspecified") } - if config.Storage == nil { - return errors.New("Storage unspecified") - } if config.StorageDir == "" { return errors.New("StorageDir unspecified") } diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index 4b04bdb0d02..eadcd82d2e6 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -73,9 +73,6 @@ func TestNewProcessorConfigInvalid(t *testing.T) { assertInvalidConfigError("invalid storage config: DB unspecified") config.DB = &eventstorage.StorageManager{} - assertInvalidConfigError("invalid storage config: Storage unspecified") - config.Storage = &eventstorage.PartitionReadWriter{} - assertInvalidConfigError("invalid storage config: StorageDir unspecified") config.StorageDir = "tbs" diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 79f05ac17ea..cef13ccc8ae 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -142,6 +142,10 @@ func (sm *StorageManager) StorageLimitReached() bool { return limit != 0 && uint64(lsm+vlog) > limit } +func (sm *StorageManager) Flush() error { + return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush()) +} + func (sm *StorageManager) Close() error { return sm.close() } diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index a6c964e2a59..6bf755beae1 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -64,13 +64,18 @@ func NewProcessor(config Config) (*Processor, error) { return nil, errors.Wrap(err, "invalid tail-sampling config") } + rw := config.Storage + if rw == nil { + rw = config.DB.NewReadWriter() + } + logger := logp.NewLogger(logs.Sampling) p := &Processor{ config: config, logger: logger, rateLimitedLogger: logger.WithOptions(logs.WithRateLimit(loggerRateLimit)), groups: newTraceGroups(config.Policies, config.MaxDynamicServices, config.IngestRateDecayFactor), - eventStore: newWrappedRW(config.Storage, config.TTL, int64(config.StorageLimit)), + eventStore: newWrappedRW(rw, config.TTL, int64(config.StorageLimit)), eventMetrics: &eventMetrics{}, stopping: make(chan struct{}), stopped: make(chan struct{}), diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index b907c5a6e96..ce5c3fd2b1d 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -78,11 +78,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { assert.NoError(t, writer.Flush()) writer.Close() - // Badger transactions created globally before committing the above writes - // will not see them due to SSI (Serializable Snapshot Isolation). Flush - // the storage so that new transactions are created for the underlying - // writer shards that can list all the events committed so far. - require.NoError(t, config.Storage.Flush()) + require.NoError(t, config.DB.Flush()) processor, err := sampling.NewProcessor(config) require.NoError(t, err) @@ -140,7 +136,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { // Stop the processor and flush global storage so we can access the database. assert.NoError(t, processor.Stop(context.Background())) - assert.NoError(t, config.Storage.Flush()) + assert.NoError(t, config.DB.Flush()) reader := config.DB.NewBypassReadWriter() defer reader.Close() @@ -259,7 +255,7 @@ func TestProcessLocalTailSampling(t *testing.T) { // Stop the processor and flush global storage so we can access the database. assert.NoError(t, processor.Stop(context.Background())) - assert.NoError(t, config.Storage.Flush()) + assert.NoError(t, config.DB.Flush()) reader := config.DB.NewBypassReadWriter() defer reader.Close() @@ -323,7 +319,7 @@ func TestProcessLocalTailSamplingUnsampled(t *testing.T) { // Stop the processor so we can access the database. assert.NoError(t, processor.Stop(context.Background())) - assert.NoError(t, config.Storage.Flush()) + assert.NoError(t, config.DB.Flush()) reader := config.DB.NewBypassReadWriter() defer reader.Close() @@ -476,7 +472,7 @@ func TestProcessRemoteTailSampling(t *testing.T) { // Stop the processor and flush global storage so we can access the database. assert.NoError(t, processor.Stop(context.Background())) - assert.NoError(t, config.Storage.Flush()) + assert.NoError(t, config.DB.Flush()) assert.Empty(t, published) // remote decisions don't get republished expectedMonitoring := monitoring.MakeFlatSnapshot() @@ -757,7 +753,6 @@ func TestGracefulShutdown(t *testing.T) { assert.NoError(t, processor.ProcessBatch(context.Background(), &batch)) assert.Empty(t, batch) assert.NoError(t, processor.Stop(context.Background())) - assert.NoError(t, config.Storage.Flush()) reader := config.DB.NewBypassReadWriter() defer reader.Close() @@ -780,8 +775,6 @@ func newTempdirConfig(tb testing.TB) sampling.Config { require.NoError(tb, err) tb.Cleanup(func() { db.Close() }) - storage := db.NewReadWriter() - return sampling.Config{ BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), LocalSamplingConfig: sampling.LocalSamplingConfig{ @@ -803,7 +796,6 @@ func newTempdirConfig(tb testing.TB) sampling.Config { }, StorageConfig: sampling.StorageConfig{ DB: db, - Storage: storage, StorageDir: tempdir, StorageGCInterval: time.Second, TTL: 30 * time.Minute, From 8cf67b537b5980c36fa447fe7e684cbc415b9fdc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 14:41:58 +0000 Subject: [PATCH 097/184] Remove some badger mentions --- x-pack/apm-server/main.go | 8 ++++---- x-pack/apm-server/main_test.go | 2 +- x-pack/apm-server/sampling/processor.go | 2 +- x-pack/apm-server/sampling/processor_test.go | 5 ----- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index c53def6348c..f9c00675008 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -235,11 +235,11 @@ func wrapServer(args beater.ServerParams, runServer beater.RunServerFunc) (beate return args, wrappedRunServer, nil } -// closeBadger is called at process exit time to close the badger.DB opened +// closeDB is called at process exit time to close the StorageManager opened // by the tail-based sampling processor constructor, if any. This is never -// called concurrently with opening badger.DB/accessing the db global, +// called concurrently with opening DB/accessing the db global, // so it does not need to hold dbMu. -func closeBadger() error { +func closeDB() error { if db != nil { return db.Close() } @@ -247,7 +247,7 @@ func closeBadger() error { } func cleanup() error { - return closeBadger() + return closeDB() } func Main() error { diff --git a/x-pack/apm-server/main_test.go b/x-pack/apm-server/main_test.go index fad69edc008..e79f5c6dd8c 100644 --- a/x-pack/apm-server/main_test.go +++ b/x-pack/apm-server/main_test.go @@ -33,7 +33,7 @@ func TestMonitoring(t *testing.T) { home := t.TempDir() err := paths.InitPaths(&paths.Path{Home: home}) require.NoError(t, err) - defer closeBadger() // close badger.DB so data dir can be deleted on Windows + defer closeDB() // close DB so data dir can be deleted on Windows cfg := config.DefaultConfig() cfg.Sampling.Tail.Enabled = true diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 6bf755beae1..7d6956bceba 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -279,7 +279,7 @@ func (p *Processor) processSpan(event *modelpb.APMEvent) (report, stored bool, _ } // Stop stops the processor, flushing event storage. Note that the underlying -// badger.DB must be closed independently to ensure writes are synced to disk. +// StorageManager must be closed independently to ensure writes are synced to disk. func (p *Processor) Stop(ctx context.Context) error { p.stopMu.Lock() select { diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index ce5c3fd2b1d..9e4b9779398 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -630,12 +630,7 @@ func TestStorageMonitoring(t *testing.T) { assert.Empty(t, batch) } - // Stop the processor and create a new one, which will reopen storage - // and calculate the storage size. Otherwise we must wait for a minute - // (hard-coded in badger) for storage metrics to be updated. processor.Stop(context.Background()) - processor, err = sampling.NewProcessor(config) - require.NoError(t, err) metrics := collectProcessorMetrics(processor) assert.NotZero(t, metrics.Ints, "sampling.storage.lsm_size") From 1e09fa3ed776d7bd20bc9d881fc9b61d79b7e53d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:11:51 +0000 Subject: [PATCH 098/184] Pebble bug --- 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 9e4b9779398..53fed0e4f2e 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -749,6 +749,8 @@ func TestGracefulShutdown(t *testing.T) { assert.Empty(t, batch) assert.NoError(t, processor.Stop(context.Background())) + config.DB.Flush() // FIXME: pebble BUG? flush blocks indefinitely for totalTraces < 2000 + reader := config.DB.NewBypassReadWriter() defer reader.Close() From 9b75264308ad0d11f2e8b75eece1fa4165bad4dd Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:27:51 +0000 Subject: [PATCH 099/184] Flush on sm.Close --- 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 cef13ccc8ae..bee7692ae04 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -151,11 +151,10 @@ func (sm *StorageManager) Close() error { } func (sm *StorageManager) close() error { - return errors.Join(sm.eventDB.Close(), sm.decisionDB.Close()) + return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) } // Reload flushes out pending disk writes to disk by reloading the database. -// It does not flush uncommitted writes. // For testing only. func (sm *StorageManager) Reload() error { if err := sm.close(); err != nil { From 4c409d39c9622d0d3da13f16d2e7bc634e93157d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:37:07 +0000 Subject: [PATCH 100/184] Remove unused code --- x-pack/apm-server/sampling/processor_test.go | 30 -------------------- 1 file changed, 30 deletions(-) diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 53fed0e4f2e..60ea70340b1 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -11,8 +11,6 @@ import ( "os" "path" "path/filepath" - "sort" - "strings" "testing" "time" @@ -888,31 +886,3 @@ func waitFileModified(tb testing.TB, filename string, after time.Time) ([]byte, } } } - -func vlogFilenames(storageDir string) []string { - entries, _ := os.ReadDir(storageDir) - - var vlogs []string - for _, entry := range entries { - name := entry.Name() - if strings.HasSuffix(name, ".vlog") { - vlogs = append(vlogs, name) - } - } - sort.Strings(vlogs) - return vlogs -} - -func sstFilenames(storageDir string) []string { - entries, _ := os.ReadDir(storageDir) - - var ssts []string - for _, entry := range entries { - name := entry.Name() - if strings.HasSuffix(name, ".sst") { - ssts = append(ssts, name) - } - } - sort.Strings(ssts) - return ssts -} From e99da0390186731afaa607f685fdda456c3fd59e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:39:53 +0000 Subject: [PATCH 101/184] Remove badger mention --- x-pack/apm-server/sampling/config.go | 4 ++-- x-pack/apm-server/sampling/eventstorage/storage_bench_test.go | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 853c0d56e66..82365c40cb1 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -95,7 +95,7 @@ type DataStreamConfig struct { // StorageConfig holds Processor configuration related to event storage. type StorageConfig struct { - // DB holds the badger database in which event storage will be maintained. + // DB holds the StorageManager in which event storage will be maintained. // // DB will not be closed when the processor is closed. DB *eventstorage.StorageManager @@ -110,7 +110,7 @@ type StorageConfig struct { // StorageGCInterval holds the amount of time between storage garbage collections. StorageGCInterval time.Duration - // StorageLimit for the badger database, in bytes. + // StorageLimit for the TBS database, in bytes. StorageLimit uint64 // TTL holds the amount of time before events and sampling decisions diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index 260b54b51e4..2c07538f5e5 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -108,8 +108,6 @@ func BenchmarkReadEvents(b *testing.B) { } } - // NOTE(marclop) We want to check how badly the read performance is affected with - // by having uncommitted events in the badger TX. b.ResetTimer() var batch modelpb.Batch for i := 0; i < b.N; i++ { From 3abeccf7a34dcc5e6a60b5b4f2012cb7e7f3b0ad Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:51:45 +0000 Subject: [PATCH 102/184] Remove writerOpts and wrappedRW --- .../sampling/eventstorage/partition_rw.go | 8 +-- .../sampling/eventstorage/prefix.go | 4 +- .../sampling/eventstorage/prefix_test.go | 10 ++-- x-pack/apm-server/sampling/eventstorage/rw.go | 20 +++---- .../sampling/eventstorage/storage.go | 5 -- .../eventstorage/storage_bench_test.go | 22 ++----- .../eventstorage/storage_manager_test.go | 4 +- x-pack/apm-server/sampling/processor.go | 57 +------------------ x-pack/apm-server/sampling/processor_test.go | 11 ++-- 9 files changed, 34 insertions(+), 107 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index f7d7e8cc7ab..f05789b3006 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -32,9 +32,9 @@ func (rw *PartitionReadWriter) Flush() error { } // WriteTraceSampled records the tail-sampling decision for the given trace ID. -func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { +func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { pid := rw.s.db.WritePartition().ID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled, opts) + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -63,9 +63,9 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { // WriteTraceEvent writes a trace event to storage. // // WriteTraceEvent may return before the write is committed to storage. -func (rw *PartitionReadWriter) WriteTraceEvent(traceID string, id string, event *modelpb.APMEvent, opts WriterOpts) error { +func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { pid := rw.s.db.WritePartition().ID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event, opts) + return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) } // DeleteTraceEvent deletes the trace event from storage. diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index e47d5b407ec..9a512ebd990 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -61,7 +61,7 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e return nil } -func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { +func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { data, err := rw.codec.EncodeEvent(event) if err != nil { return err @@ -83,7 +83,7 @@ func (rw PrefixReadWriter) writeEntry(key, data []byte) error { return nil } -func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { +func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error { var b bytes.Buffer b.Grow(1 + len(traceID)) b.WriteByte(rw.prefix) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 25c781c7ced..4a1aa32d9f3 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -36,7 +36,7 @@ func TestPrefixReadWriter_WriteTraceEvent(t *testing.T) { txnID := "bar" txn := makeTransaction(txnID, traceID) rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + err := rw.WriteTraceEvent(traceID, txnID, txn) assert.NoError(t, err) item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) assert.NoError(t, err) @@ -55,7 +55,7 @@ func TestPrefixReadWriter_ReadTraceEvents(t *testing.T) { traceID := "foo" for _, txnID := range []string{"bar", "baz"} { txn := makeTransaction(txnID, traceID) - err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + err := rw.WriteTraceEvent(traceID, txnID, txn) require.NoError(t, err) } @@ -75,7 +75,7 @@ func TestPrefixReadWriter_DeleteTraceEvent(t *testing.T) { txnID := "bar" txn := makeTransaction(txnID, traceID) rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - err := rw.WriteTraceEvent(traceID, txnID, txn, eventstorage.WriterOpts{}) + err := rw.WriteTraceEvent(traceID, txnID, txn) require.NoError(t, err) key := append([]byte{1}, []byte("foo:bar")...) @@ -99,7 +99,7 @@ func TestPrefixReadWriter_WriteTraceSampled(t *testing.T) { db := newDecisionPebble(t) traceID := "foo" rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - err := rw.WriteTraceSampled(traceID, sampled, eventstorage.WriterOpts{}) + err := rw.WriteTraceSampled(traceID, sampled) assert.NoError(t, err) item, closer, err := db.Get(append([]byte{1}, []byte("foo")...)) assert.NoError(t, err) @@ -134,7 +134,7 @@ func TestPrefixReadWriter_IsTraceSampled(t *testing.T) { rw := eventstorage.NewPrefixReadWriter(db, 1, nopCodec{}) traceID := uuid.Must(uuid.NewV4()).String() if !tc.missing { - err := rw.WriteTraceSampled(traceID, tc.sampled, eventstorage.WriterOpts{}) + err := rw.WriteTraceSampled(traceID, tc.sampled) require.NoError(t, err) } sampled, err := rw.IsTraceSampled(traceID) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 863e29a7199..12738a8fcc8 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -10,8 +10,8 @@ import ( type RW interface { ReadTraceEvents(traceID string, out *modelpb.Batch) error - WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error - WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error + WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error + WriteTraceSampled(traceID string, sampled bool) error IsTraceSampled(traceID string) (bool, error) DeleteTraceEvent(traceID, id string) error Flush() error @@ -25,12 +25,12 @@ func (s SplitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) err return s.eventRW.ReadTraceEvents(traceID, out) } -func (s SplitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { - return s.eventRW.WriteTraceEvent(traceID, id, event, opts) +func (s SplitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { + return s.eventRW.WriteTraceEvent(traceID, id, event) } -func (s SplitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { - return s.decisionRW.WriteTraceSampled(traceID, sampled, opts) +func (s SplitReadWriter) WriteTraceSampled(traceID string, sampled bool) error { + return s.decisionRW.WriteTraceSampled(traceID, sampled) } func (s SplitReadWriter) IsTraceSampled(traceID string) (bool, error) { @@ -62,18 +62,18 @@ func (s StorageLimitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Bat return s.nextRW.ReadTraceEvents(traceID, out) } -func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts WriterOpts) error { +func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { if s.checker.StorageLimitReached() { return ErrLimitReached } - return s.nextRW.WriteTraceEvent(traceID, id, event, opts) + return s.nextRW.WriteTraceEvent(traceID, id, event) } -func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool, opts WriterOpts) error { +func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool) error { if s.checker.StorageLimitReached() { return ErrLimitReached } - return s.nextRW.WriteTraceSampled(traceID, sampled, opts) + return s.nextRW.WriteTraceSampled(traceID, sampled) } func (s StorageLimitReadWriter) IsTraceSampled(traceID string) (bool, error) { diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index b36edab1be1..1040fee5384 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -73,8 +73,3 @@ func (s *Storage) NewReadWriter() *PartitionReadWriter { s: s, } } - -// WriterOpts provides configuration options for writes to storage -type WriterOpts struct { - StorageLimitInBytes int64 -} diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index 2c07538f5e5..3c645da22c5 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -37,11 +37,8 @@ func BenchmarkWriteTransaction(b *testing.B) { b.ResetTimer() - wOpts := eventstorage.WriterOpts{ - StorageLimitInBytes: 0, - } for i := 0; i < b.N; i++ { - if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { + if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction); err != nil { b.Fatal(err) } } @@ -87,9 +84,6 @@ func BenchmarkReadEvents(b *testing.B) { sm := newStorageManager(b, eventstorage.WithCodec(codec)) readWriter := sm.NewBypassReadWriter() defer readWriter.Close() - wOpts := eventstorage.WriterOpts{ - StorageLimitInBytes: 0, - } for i := 0; i < count; i++ { transactionID := uuid.Must(uuid.NewV4()).String() @@ -103,7 +97,7 @@ func BenchmarkReadEvents(b *testing.B) { }, } } - if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { + if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction); err != nil { b.Fatal(err) } } @@ -166,9 +160,6 @@ func BenchmarkReadEventsHit(b *testing.B) { sm := newStorageManager(b) readWriter := sm.NewBypassReadWriter() defer readWriter.Close() - wOpts := eventstorage.WriterOpts{ - StorageLimitInBytes: 0, - } traceIDs := make([]string, b.N) @@ -187,7 +178,7 @@ func BenchmarkReadEventsHit(b *testing.B) { }, } } - if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction, wOpts); err != nil { + if err := readWriter.WriteTraceEvent(traceID, transactionID, transaction); err != nil { b.Fatal(err) } } @@ -246,14 +237,11 @@ func BenchmarkIsTraceSampled(b *testing.B) { sm := newStorageManager(b) readWriter := sm.NewBypassReadWriter() defer readWriter.Close() - wOpts := eventstorage.WriterOpts{ - StorageLimitInBytes: 0, - } - if err := readWriter.WriteTraceSampled(sampledTraceUUID.String(), true, wOpts); err != nil { + if err := readWriter.WriteTraceSampled(sampledTraceUUID.String(), true); err != nil { b.Fatal(err) } - if err := readWriter.WriteTraceSampled(unsampledTraceUUID.String(), false, wOpts); err != nil { + if err := readWriter.WriteTraceSampled(unsampledTraceUUID.String(), false); err != nil { b.Fatal(err) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 045c49d932c..17c59fb14bd 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -27,7 +27,7 @@ func TestStorageManager_samplingDecisionTTL(t *testing.T) { sm := newStorageManager(t) rw := sm.NewBypassReadWriter() traceID := uuid.Must(uuid.NewV4()).String() - err := rw.WriteTraceSampled(traceID, true, eventstorage.WriterOpts{}) + err := rw.WriteTraceSampled(traceID, true) assert.NoError(t, err) sampled, err := rw.IsTraceSampled(traceID) assert.NoError(t, err) @@ -62,7 +62,7 @@ func TestStorageManager_eventTTL(t *testing.T) { traceID := uuid.Must(uuid.NewV4()).String() txnID := uuid.Must(uuid.NewV4()).String() transaction := makeTransaction(txnID, traceID) - err := rw.WriteTraceEvent(traceID, txnID, transaction, eventstorage.WriterOpts{}) + err := rw.WriteTraceEvent(traceID, txnID, transaction) assert.NoError(t, err) var out modelpb.Batch diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 7d6956bceba..728db500f48 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -41,7 +41,7 @@ type Processor struct { rateLimitedLogger *logp.Logger groups *traceGroups - eventStore *wrappedRW + eventStore eventstorage.RW eventMetrics *eventMetrics // heap-allocated for 64-bit alignment stopMu sync.Mutex @@ -75,7 +75,7 @@ func NewProcessor(config Config) (*Processor, error) { logger: logger, rateLimitedLogger: logger.WithOptions(logs.WithRateLimit(loggerRateLimit)), groups: newTraceGroups(config.Policies, config.MaxDynamicServices, config.IngestRateDecayFactor), - eventStore: newWrappedRW(rw, config.TTL, int64(config.StorageLimit)), + eventStore: rw, eventMetrics: &eventMetrics{}, stopping: make(chan struct{}), stopped: make(chan struct{}), @@ -570,56 +570,3 @@ func sendTraceIDs(ctx context.Context, out chan<- string, traceIDs []string) err const ( storageLimitThreshold = 0.90 // Allow 90% of the quota to be used. ) - -// wrappedRW wraps configurable write options for global rw -type wrappedRW struct { - rw eventstorage.RW - writerOpts eventstorage.WriterOpts -} - -// Stored entries expire after ttl. -// The amount of storage that can be consumed can be limited by passing in a -// 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.RW, ttl time.Duration, limit int64) *wrappedRW { - if limit > 1 { - limit = int64(float64(limit) * storageLimitThreshold) - } - return &wrappedRW{ - rw: rw, - writerOpts: eventstorage.WriterOpts{ - StorageLimitInBytes: limit, - }, - } -} - -// ReadTraceEvents calls rw.ReadTraceEvents -func (s *wrappedRW) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - return s.rw.ReadTraceEvents(traceID, out) -} - -// WriteTraceEvent calls rw.WriteTraceEvent using the configured WriterOpts -func (s *wrappedRW) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { - return s.rw.WriteTraceEvent(traceID, id, event, s.writerOpts) -} - -// WriteTraceSampled calls rw.WriteTraceSampled using the configured WriterOpts -func (s *wrappedRW) WriteTraceSampled(traceID string, sampled bool) error { - return s.rw.WriteTraceSampled(traceID, sampled, s.writerOpts) -} - -// IsTraceSampled calls rw.IsTraceSampled -func (s *wrappedRW) IsTraceSampled(traceID string) (bool, error) { - return s.rw.IsTraceSampled(traceID) -} - -// DeleteTraceEvent calls rw.DeleteTraceEvent -func (s *wrappedRW) DeleteTraceEvent(traceID, id string) error { - return s.rw.DeleteTraceEvent(traceID, id) -} - -// Flush calls rw.Flush -func (s *wrappedRW) Flush() error { - return s.rw.Flush() -} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 60ea70340b1..f610663f127 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -60,10 +60,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { trace1 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f10"} trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} writer := config.DB.NewBypassReadWriter() - wOpts := eventstorage.WriterOpts{ - StorageLimitInBytes: 0, - } - assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true, wOpts)) + assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true)) assert.NoError(t, writer.Flush()) writer.Close() @@ -72,7 +69,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { assert.NoError(t, config.DB.IncrementPartition()) writer = config.DB.NewBypassReadWriter() - assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true, wOpts)) + assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true)) assert.NoError(t, writer.Flush()) writer.Close() @@ -514,11 +511,11 @@ func (m errorRW) ReadTraceEvents(traceID string, out *modelpb.Batch) error { return m.err } -func (m errorRW) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent, opts eventstorage.WriterOpts) error { +func (m errorRW) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { return m.err } -func (m errorRW) WriteTraceSampled(traceID string, sampled bool, opts eventstorage.WriterOpts) error { +func (m errorRW) WriteTraceSampled(traceID string, sampled bool) error { return m.err } From d39da8a53749b7141c0d4db18f7ba1dabc307b69 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:56:41 +0000 Subject: [PATCH 103/184] Add example --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index bee7692ae04..67498c60c10 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -28,6 +28,11 @@ const ( // partitionsPerTTL holds the number of partitions that events in 1 TTL should be stored over. // Increasing partitionsPerTTL increases read amplification, but decreases storage overhead, // as TTL GC can be performed sooner. + // + // For example, partitionPerTTL=1 means we need to keep 2 partitions active, + // such that the last entry in the previous partition is also kept for a full TTL. + // This means storage requirement is 2 * TTL, and it needs to read 2 keys per trace ID read. + // If partitionPerTTL=2, storage requirement is 1.5 * TTL at the expense of 3 reads per trace ID read. partitionsPerTTL = 1 ) From a41e3c9cc8a821c89732d9374f8b5a81ba276732 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 15:57:49 +0000 Subject: [PATCH 104/184] Add FIXME --- x-pack/apm-server/sampling/eventstorage/rw.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 12738a8fcc8..13c1553789a 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -64,14 +64,14 @@ func (s StorageLimitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Bat func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { if s.checker.StorageLimitReached() { - return ErrLimitReached + return ErrLimitReached // FIXME: return a more helpful error message } return s.nextRW.WriteTraceEvent(traceID, id, event) } func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool) error { if s.checker.StorageLimitReached() { - return ErrLimitReached + return ErrLimitReached // FIXME: return a more helpful error message } return s.nextRW.WriteTraceSampled(traceID, sampled) } From e4a0a0f8b84065cc54d2612dce47e9d2b687f99e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:02:18 +0000 Subject: [PATCH 105/184] Clean up comments --- x-pack/apm-server/sampling/eventstorage/partition_rw.go | 9 --------- x-pack/apm-server/sampling/eventstorage/storage.go | 8 +++----- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index f05789b3006..ce9bf3fa0b2 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -17,16 +17,9 @@ type PartitionReadWriter struct { } // 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 *PartitionReadWriter) Close() {} // 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 *PartitionReadWriter) Flush() error { return nil } @@ -61,8 +54,6 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { } // WriteTraceEvent writes a trace event to storage. -// -// WriteTraceEvent may return before the write is committed to storage. func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { pid := rw.s.db.WritePartition().ID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 1040fee5384..8dd7ec7fcc7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -21,12 +21,12 @@ const ( ) var ( - // ErrNotFound is returned by by the Storage.IsTraceSampled method, + // ErrNotFound is returned by the RW.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 is returned by RW methods when storage usage + // is greater than configured limit. ErrLimitReached = errors.New("configured storage limit reached") ) @@ -66,8 +66,6 @@ func New(db partitionedDB, codec Codec) *Storage { // NewReadWriter returns a new PartitionReadWriter for reading events from and // writing events to storage. -// -// The returned PartitionReadWriter must be closed when it is no longer needed. func (s *Storage) NewReadWriter() *PartitionReadWriter { return &PartitionReadWriter{ s: s, From 19bfccb5a625575212dab6560b12892a2a4e09a0 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:09:31 +0000 Subject: [PATCH 106/184] Add FIXME --- x-pack/apm-server/sampling/processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 728db500f48..7a65d6db895 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -107,7 +107,7 @@ func (p *Processor) CollectMonitoring(_ monitoring.Mode, V monitoring.Visitor) { 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.ReportInt(V, "value_log_size", int64(valueLogSize)) // FIXME: either report everything under lsm_size, or report with new fine-grained metrics, e.g. wal_size }) monitoring.ReportNamespace(V, "events", func() { monitoring.ReportInt(V, "processed", atomic.LoadInt64(&p.eventMetrics.processed)) From a1e444c20abeec98f90974ebfaa8b5cb70ee44a4 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:18:13 +0000 Subject: [PATCH 107/184] Remove storage gc interval --- internal/beater/config/config_test.go | 2 -- internal/beater/config/sampling.go | 2 -- x-pack/apm-server/main.go | 1 - x-pack/apm-server/sampling/config.go | 6 ------ x-pack/apm-server/sampling/config_test.go | 3 --- .../apm-server/sampling/eventstorage/storage_manager.go | 2 +- x-pack/apm-server/sampling/processor.go | 2 +- x-pack/apm-server/sampling/processor_test.go | 9 ++++----- 8 files changed, 6 insertions(+), 21 deletions(-) diff --git a/internal/beater/config/config_test.go b/internal/beater/config/config_test.go index 9b136f95b72..fbc8c08d3d3 100644 --- a/internal/beater/config/config_test.go +++ b/internal/beater/config/config_test.go @@ -362,7 +362,6 @@ func TestUnpackConfig(t *testing.T) { ESConfig: elasticsearch.DefaultConfig(), Interval: 1 * time.Minute, IngestRateDecayFactor: 0.25, - StorageGCInterval: 5 * time.Minute, StorageLimit: "3GB", StorageLimitParsed: 3000000000, TTL: 30 * time.Minute, @@ -494,7 +493,6 @@ func TestUnpackConfig(t *testing.T) { ESConfig: elasticsearch.DefaultConfig(), Interval: 2 * time.Minute, IngestRateDecayFactor: 1.0, - StorageGCInterval: 5 * time.Minute, StorageLimit: "1GB", StorageLimitParsed: 1000000000, TTL: 30 * time.Minute, diff --git a/internal/beater/config/sampling.go b/internal/beater/config/sampling.go index 0e00e1fd0c0..07bff0d1a23 100644 --- a/internal/beater/config/sampling.go +++ b/internal/beater/config/sampling.go @@ -48,7 +48,6 @@ type TailSamplingConfig struct { ESConfig *elasticsearch.Config `config:"elasticsearch"` Interval time.Duration `config:"interval" validate:"min=1s"` IngestRateDecayFactor float64 `config:"ingest_rate_decay" validate:"min=0, max=1"` - StorageGCInterval time.Duration `config:"storage_gc_interval" validate:"min=1s"` TTL time.Duration `config:"ttl" validate:"min=1s"` StorageLimit string `config:"storage_limit"` StorageLimitParsed uint64 @@ -151,7 +150,6 @@ func defaultTailSamplingConfig() TailSamplingConfig { ESConfig: elasticsearch.DefaultConfig(), Interval: 1 * time.Minute, IngestRateDecayFactor: 0.25, - StorageGCInterval: 5 * time.Minute, TTL: 30 * time.Minute, StorageLimit: "3GB", DiscardOnWriteFailure: false, diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index f9c00675008..0c6e58a5c70 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -153,7 +153,6 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er StorageConfig: sampling.StorageConfig{ DB: db, StorageDir: storageDir, - StorageGCInterval: tailSamplingConfig.StorageGCInterval, StorageLimit: tailSamplingConfig.StorageLimitParsed, TTL: tailSamplingConfig.TTL, DiscardOnWriteFailure: tailSamplingConfig.DiscardOnWriteFailure, diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 82365c40cb1..11157ef8a5d 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -107,9 +107,6 @@ type StorageConfig struct { // StorageDir holds the directory in which event storage will be maintained. StorageDir string - // StorageGCInterval holds the amount of time between storage garbage collections. - StorageGCInterval time.Duration - // StorageLimit for the TBS database, in bytes. StorageLimit uint64 @@ -243,9 +240,6 @@ func (config StorageConfig) validate() error { if config.StorageDir == "" { return errors.New("StorageDir unspecified") } - if config.StorageGCInterval <= 0 { - return errors.New("StorageGCInterval unspecified or negative") - } if config.TTL <= 0 { return errors.New("TTL unspecified or negative") } diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index eadcd82d2e6..55093813d47 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -76,9 +76,6 @@ func TestNewProcessorConfigInvalid(t *testing.T) { assertInvalidConfigError("invalid storage config: StorageDir unspecified") config.StorageDir = "tbs" - assertInvalidConfigError("invalid storage config: StorageGCInterval unspecified or negative") - config.StorageGCInterval = 1 - assertInvalidConfigError("invalid storage config: TTL unspecified or negative") config.TTL = 1 } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 67498c60c10..b43fe972400 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -169,7 +169,7 @@ func (sm *StorageManager) Reload() error { } // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. -func (sm *StorageManager) Run(stopping <-chan struct{}, gcInterval time.Duration, ttl time.Duration, storageLimit uint64, storageLimitThreshold float64) error { +func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, storageLimit uint64, storageLimitThreshold float64) error { select { case <-stopping: return nil diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 7a65d6db895..0ba09d1d145 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -378,7 +378,7 @@ func (p *Processor) Run() error { } }) g.Go(func() error { - return p.config.DB.Run(p.stopping, p.config.StorageGCInterval, p.config.TTL, p.config.StorageLimit, storageLimitThreshold) + return p.config.DB.Run(p.stopping, p.config.TTL, p.config.StorageLimit, storageLimitThreshold) }) g.Go(func() error { // Subscribe to remotely sampled trace IDs. This is cancelled immediately when diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index f610663f127..2816516da14 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -787,11 +787,10 @@ func newTempdirConfig(tb testing.TB) sampling.Config { UUID: "local-apm-server", }, StorageConfig: sampling.StorageConfig{ - DB: db, - StorageDir: tempdir, - StorageGCInterval: time.Second, - TTL: 30 * time.Minute, - StorageLimit: 0, // No storage limit. + DB: db, + StorageDir: tempdir, + TTL: 30 * time.Minute, + StorageLimit: 0, // No storage limit. }, } } From 0b027284eee7da5e32ee3c1677d18f00b14b58cb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:21:40 +0000 Subject: [PATCH 108/184] Rename to rotate partitions --- .../sampling/eventstorage/storage_manager.go | 7 ++++--- .../sampling/eventstorage/storage_manager_test.go | 12 ++++++------ x-pack/apm-server/sampling/processor_test.go | 4 ++-- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index b43fe972400..624b1b8f5b0 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -196,14 +196,15 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati case <-stopping: return nil case <-ticker.C: - if err := sm.IncrementPartition(); err != nil { - sm.logger.With(logp.Error(err)).Error("failed to increment partition") + if err := sm.RotatePartitions(); err != nil { + sm.logger.With(logp.Error(err)).Error("failed to rotate partition") } } } } -func (sm *StorageManager) IncrementPartition() error { +// RotatePartitions rotates the partitions to clean up TTL-expired entries. +func (sm *StorageManager) RotatePartitions() error { sm.partitioner.Rotate() // FIXME: potential race, wait for a bit before deleting? pidToDelete := sm.partitioner.Inactive().ID() diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 17c59fb14bd..f993dc8cd5f 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -34,7 +34,7 @@ func TestStorageManager_samplingDecisionTTL(t *testing.T) { assert.True(t, sampled) // after 1 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) sampled, err = rw.IsTraceSampled(traceID) @@ -42,14 +42,14 @@ func TestStorageManager_samplingDecisionTTL(t *testing.T) { assert.True(t, sampled) // after 2 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) _, err = rw.IsTraceSampled(traceID) assert.ErrorIs(t, err, eventstorage.ErrNotFound) // after 3 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) _, err = rw.IsTraceSampled(traceID) @@ -71,7 +71,7 @@ func TestStorageManager_eventTTL(t *testing.T) { assert.Len(t, out, 1) // after 1 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) out = nil @@ -80,7 +80,7 @@ func TestStorageManager_eventTTL(t *testing.T) { assert.Len(t, out, 1) // after 2 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) out = nil @@ -89,7 +89,7 @@ func TestStorageManager_eventTTL(t *testing.T) { assert.Len(t, out, 0) // after 3 TTL - err = sm.IncrementPartition() + err = sm.RotatePartitions() assert.NoError(t, err) out = nil diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 2816516da14..7b296c00caa 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -65,8 +65,8 @@ func TestProcessAlreadyTailSampled(t *testing.T) { writer.Close() // simulate 2 TTL - assert.NoError(t, config.DB.IncrementPartition()) - assert.NoError(t, config.DB.IncrementPartition()) + assert.NoError(t, config.DB.RotatePartitions()) + assert.NoError(t, config.DB.RotatePartitions()) writer = config.DB.NewBypassReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true)) From 2fe27f777cd0eac3cb3e505ba664dee709143ce9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:24:56 +0000 Subject: [PATCH 109/184] Use a stable format --- x-pack/apm-server/sampling/eventstorage/pebble.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 696fbff015d..83fefea40ea 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -26,8 +26,7 @@ func eventComparer() *pebble.Comparer { func OpenEventPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "event"), &pebble.Options{ - // FIXME: Specify FormatMajorVersion to use value blocks? - FormatMajorVersion: pebble.FormatNewest, + FormatMajorVersion: pebble.FormatColumnarBlocks, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 16 << 20, Levels: []pebble.LevelOptions{ @@ -44,8 +43,7 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "decision"), &pebble.Options{ - // FIXME: Specify FormatMajorVersion to use value blocks? - FormatMajorVersion: pebble.FormatNewest, + FormatMajorVersion: pebble.FormatColumnarBlocks, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 2 << 20, Levels: []pebble.LevelOptions{ From 917584e7173d80ca93ada89a772f8cdfafab53eb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:41:31 +0000 Subject: [PATCH 110/184] Add log --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 624b1b8f5b0..5b82e7c0d3b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -189,16 +189,19 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, stora } func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Duration) error { - ticker := time.NewTicker(ttl / partitionsPerTTL) + ttlGCInterval := ttl / partitionsPerTTL + ticker := time.NewTicker(ttlGCInterval) defer ticker.Stop() for { select { case <-stopping: return nil case <-ticker.C: + sm.logger.Info("running TTL GC to clean up expired entries") if err := sm.RotatePartitions(); err != nil { sm.logger.With(logp.Error(err)).Error("failed to rotate partition") } + sm.logger.Info("finished running TTL GC") } } } From 9c0df8c678463ab6705079b5923848f45e26ab12 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 16:52:06 +0000 Subject: [PATCH 111/184] Improve comment --- x-pack/apm-server/sampling/eventstorage/prefix.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 9a512ebd990..a9a686d40b1 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -43,7 +43,9 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e return err } defer iter.Close() - // SeekPrefixGE uses the prefix bloom filter, so that a miss will be much faster + // SeekPrefixGE uses prefix bloom filter for on disk tables. + // These bloom filters are cached in memory, and a "miss" on bloom filter avoids disk IO to check the actual table. + // Memtables still need to be scanned as pebble has no bloom filter on memtables. if valid := iter.SeekPrefixGE(lb.Bytes()); !valid { return nil } From 24297afec96f37755c3d1589ac9856efd5c3567a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 18:11:50 +0000 Subject: [PATCH 112/184] Improve log --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 5b82e7c0d3b..3d07da4b637 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -197,7 +197,7 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati case <-stopping: return nil case <-ticker.C: - sm.logger.Info("running TTL GC to clean up expired entries") + sm.logger.Info("running TTL GC to clear expired entries and reclaim disk space") if err := sm.RotatePartitions(); err != nil { sm.logger.With(logp.Error(err)).Error("failed to rotate partition") } From 80533b3bfc33bcc4f75ea8556548dd11264ab7c1 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 18:25:30 +0000 Subject: [PATCH 113/184] Add FIXME --- 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 83fefea40ea..74782a6dd13 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -38,6 +38,7 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { }, }, Comparer: eventComparer(), + // FIXME: configure merger }) } From 8c71c8dd9ebb26d769efef61136c41538aeb09df Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 19:59:00 +0000 Subject: [PATCH 114/184] No need for merger --- .../sampling/eventstorage/pebble.go | 1 - .../sampling/eventstorage/prefix_test.go | 54 ++++++++++++------- 2 files changed, 35 insertions(+), 20 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 74782a6dd13..83fefea40ea 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -38,7 +38,6 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { }, }, Comparer: eventComparer(), - // FIXME: configure merger }) } diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 4a1aa32d9f3..dad2814d20a 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -36,15 +36,23 @@ func TestPrefixReadWriter_WriteTraceEvent(t *testing.T) { txnID := "bar" txn := makeTransaction(txnID, traceID) rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - err := rw.WriteTraceEvent(traceID, txnID, txn) - assert.NoError(t, err) - item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) - assert.NoError(t, err) - defer closer.Close() - var actual modelpb.APMEvent - err = codec.DecodeEvent(item, &actual) - assert.NoError(t, err) - assert.Equal(t, *txn, actual) + + check := func() { + err := rw.WriteTraceEvent(traceID, txnID, txn) + assert.NoError(t, err) + item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) + assert.NoError(t, err) + defer closer.Close() + var actual modelpb.APMEvent + err = codec.DecodeEvent(item, &actual) + assert.NoError(t, err) + assert.Equal(t, *txn, actual) + } + + check() + + // Try writing to the same key again to simulate misbehaving agent / race condition + check() } func TestPrefixReadWriter_ReadTraceEvents(t *testing.T) { @@ -99,17 +107,25 @@ func TestPrefixReadWriter_WriteTraceSampled(t *testing.T) { db := newDecisionPebble(t) traceID := "foo" rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - err := rw.WriteTraceSampled(traceID, sampled) - assert.NoError(t, err) - item, closer, err := db.Get(append([]byte{1}, []byte("foo")...)) - assert.NoError(t, err) - defer closer.Close() - assert.NoError(t, err) - if sampled { - assert.Equal(t, []byte{'s'}, item) - } else { - assert.Equal(t, []byte{'u'}, item) + + check := func() { + err := rw.WriteTraceSampled(traceID, sampled) + assert.NoError(t, err) + item, closer, err := db.Get(append([]byte{1}, []byte("foo")...)) + assert.NoError(t, err) + defer closer.Close() + assert.NoError(t, err) + if sampled { + assert.Equal(t, []byte{'s'}, item) + } else { + assert.Equal(t, []byte{'u'}, item) + } } + + check() + + // Try writing to the same key again to simulate misbehaving agent / race condition + check() }) } } From 737f57be10467bf740e57cd413f82dfa7c254440 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 20:03:57 +0000 Subject: [PATCH 115/184] Add FIXME --- systemtest/benchtest/expvar/metrics.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/systemtest/benchtest/expvar/metrics.go b/systemtest/benchtest/expvar/metrics.go index c21bd0db051..3aec03785d1 100644 --- a/systemtest/benchtest/expvar/metrics.go +++ b/systemtest/benchtest/expvar/metrics.go @@ -167,7 +167,7 @@ func (c *Collector) accumulate(e expvar) { c.processMetric(HeapAlloc, int64(e.HeapAlloc)) c.processMetric(HeapObjects, int64(e.HeapObjects)) c.processMetric(TBSLsmSize, e.TBSLsmSize) - c.processMetric(TBSVlogSize, e.TBSVlogSize) + c.processMetric(TBSVlogSize, e.TBSVlogSize) // FIXME: update to use new metrics } func (c *Collector) processMetric(m Metric, val int64) { From 68ed084e5d65da82e80f180d474bf2909c7c2282 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 22:14:29 +0000 Subject: [PATCH 116/184] Resume from current partition on restart --- .../sampling/eventstorage/partitioner.go | 4 +++ .../sampling/eventstorage/storage_manager.go | 34 +++++++++++++++++++ .../eventstorage/storage_manager_test.go | 33 ++++++++++++++++-- 3 files changed, 68 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 49d18c6ff3c..311141b2348 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -25,6 +25,10 @@ func NewPartitioner(actives int) *Partitioner { return &Partitioner{total: actives + 1} // actives + 1 inactive } +func (p *Partitioner) SetCurrentID(current int) { + p.current.Store(int32(current)) +} + func (p *Partitioner) Rotate() { p.current.Store(int32((int(p.current.Load()) + 1) % p.total)) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 3d07da4b637..a5eafcddc28 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -114,6 +114,11 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora if err != nil { return nil, err } + + if err := sm.loadCurrentPartitionID(); err != nil { + return nil, err + } + return sm, nil } @@ -136,6 +141,30 @@ func (sm *StorageManager) reset() error { return nil } +const partitionIDKey = "!partition_id" + +// loadCurrentPartitionID loads the last saved partition ID from database, +// such that partitioner resumes from where it left off before an apm-server restart. +func (sm *StorageManager) loadCurrentPartitionID() error { + item, closer, err := sm.decisionDB.Get([]byte(partitionIDKey)) + if errors.Is(err, pebble.ErrNotFound) { + return nil + } else if err != nil { + return err + } + defer closer.Close() + sm.partitioner.SetCurrentID(int(item[0])) + return nil +} + +// saveCurrentPartitionID persists the current partition ID to disk for later. +func (sm *StorageManager) saveCurrentPartitionID() error { + // FIXME: Use either JSON or Protobuf + // FIXME: Add a timestamp to avoid using a very old database? + pid := sm.partitioner.Current().ID() + return sm.decisionDB.Set([]byte(partitionIDKey), []byte{byte(pid)}, pebble.NoSync) +} + func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it vlog // FIXME: separate WAL usage? return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 @@ -209,6 +238,11 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati // RotatePartitions rotates the partitions to clean up TTL-expired entries. func (sm *StorageManager) RotatePartitions() error { sm.partitioner.Rotate() + + if err := sm.saveCurrentPartitionID(); err != nil { + return err + } + // FIXME: potential race, wait for a bit before deleting? pidToDelete := sm.partitioner.Inactive().ID() lbPrefix := byte(pidToDelete) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index f993dc8cd5f..3236e1d87f3 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -15,11 +15,16 @@ import ( ) func newStorageManager(tb testing.TB, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { - sm, err := eventstorage.NewStorageManager(tb.TempDir(), opts...) + sm := newStorageManagerManual(tb, tb.TempDir(), opts...) + tb.Cleanup(func() { sm.Close() }) + return sm +} + +func newStorageManagerManual(tb testing.TB, path string, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { + sm, err := eventstorage.NewStorageManager(path, opts...) if err != nil { - panic(err) + tb.Fatal(err) } - tb.Cleanup(func() { sm.Close() }) return sm } @@ -97,3 +102,25 @@ func TestStorageManager_eventTTL(t *testing.T) { assert.NoError(t, err) assert.Len(t, out, 0) } + +func TestStorageManager_partitionID(t *testing.T) { + const traceID = "foo" + tmpDir := t.TempDir() + sm := newStorageManagerManual(t, tmpDir) + + // 0 -> 1 + assert.NoError(t, sm.RotatePartitions()) + + // write to partition 1 + err := sm.NewReadWriter().WriteTraceSampled(traceID, true) + assert.NoError(t, err) + + assert.NoError(t, sm.Close()) + + // it should read directly from partition 1 on startup instead of 0 + sm = newStorageManagerManual(t, tmpDir) + defer sm.Close() + sampled, err := sm.NewReadWriter().IsTraceSampled(traceID) + assert.NoError(t, err) + assert.True(t, sampled) +} From dac6f1ae456d8cb83dba7fb4f0227079f338e3fb Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 22:28:55 +0000 Subject: [PATCH 117/184] Use json --- .../sampling/eventstorage/storage_manager.go | 35 +++++++++++-------- 1 file changed, 20 insertions(+), 15 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index a5eafcddc28..175a4cfbd2c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -5,7 +5,9 @@ package eventstorage import ( + "encoding/json" "errors" + "fmt" "io" "os" "path/filepath" @@ -115,8 +117,10 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora return nil, err } - if err := sm.loadCurrentPartitionID(); err != nil { - return nil, err + if pid, err := sm.loadPartitionID(); err != nil { + sm.logger.With(logp.Error(err)).Warn("failed to load partition ID") + } else { + sm.partitioner.SetCurrentID(pid) } return sm, nil @@ -141,28 +145,29 @@ func (sm *StorageManager) reset() error { return nil } -const partitionIDKey = "!partition_id" +const partitionIDKey = "!partitioner" -// loadCurrentPartitionID loads the last saved partition ID from database, +// loadPartitionID loads the last saved partition ID from database, // such that partitioner resumes from where it left off before an apm-server restart. -func (sm *StorageManager) loadCurrentPartitionID() error { +func (sm *StorageManager) loadPartitionID() (int, error) { item, closer, err := sm.decisionDB.Get([]byte(partitionIDKey)) if errors.Is(err, pebble.ErrNotFound) { - return nil + return 0, nil } else if err != nil { - return err + return 0, err } defer closer.Close() - sm.partitioner.SetCurrentID(int(item[0])) - return nil + var pid struct { + ID int `json:"id"` + } + err = json.Unmarshal(item, &pid) + return pid.ID, err } -// saveCurrentPartitionID persists the current partition ID to disk for later. -func (sm *StorageManager) saveCurrentPartitionID() error { - // FIXME: Use either JSON or Protobuf +// savePartitionID saves the partition ID to database to be loaded by loadPartitionID later. +func (sm *StorageManager) savePartitionID(pid int) error { // FIXME: Add a timestamp to avoid using a very old database? - pid := sm.partitioner.Current().ID() - return sm.decisionDB.Set([]byte(partitionIDKey), []byte{byte(pid)}, pebble.NoSync) + return sm.decisionDB.Set([]byte(partitionIDKey), []byte(fmt.Sprintf(`{"id":%d}`, pid)), pebble.NoSync) } func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it vlog @@ -239,7 +244,7 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati func (sm *StorageManager) RotatePartitions() error { sm.partitioner.Rotate() - if err := sm.saveCurrentPartitionID(); err != nil { + if err := sm.savePartitionID(sm.partitioner.Current().ID()); err != nil { return err } From 84004832f6982a213e27bf96370beaf4952aca2e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 22:38:27 +0000 Subject: [PATCH 118/184] Better storage limit reached error message --- x-pack/apm-server/sampling/eventstorage/rw.go | 22 ++++++++++++++----- .../sampling/eventstorage/storage_manager.go | 10 +++++---- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 13c1553789a..3f0d6335306 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -5,6 +5,8 @@ package eventstorage import ( + "fmt" + "github.com/elastic/apm-data/model/modelpb" ) @@ -50,7 +52,8 @@ func (s SplitReadWriter) Close() error { } type storageLimitChecker interface { - StorageLimitReached() bool + DiskUsage() uint64 + StorageLimit() uint64 } type StorageLimitReadWriter struct { @@ -58,20 +61,29 @@ type StorageLimitReadWriter struct { nextRW RW } +func (s StorageLimitReadWriter) checkStorageLimit() error { + usage := s.checker.DiskUsage() + limit := s.checker.StorageLimit() + if limit != 0 && usage >= limit { + return fmt.Errorf("%w (current: %d, limit %d)", ErrLimitReached, usage, limit) + } + return nil +} + func (s StorageLimitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { return s.nextRW.ReadTraceEvents(traceID, out) } func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { - if s.checker.StorageLimitReached() { - return ErrLimitReached // FIXME: return a more helpful error message + if err := s.checkStorageLimit(); err != nil { + return err } return s.nextRW.WriteTraceEvent(traceID, id, event) } func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool) error { - if s.checker.StorageLimitReached() { - return ErrLimitReached // FIXME: return a more helpful error message + if err := s.checkStorageLimit(); err != nil { + return err } return s.nextRW.WriteTraceSampled(traceID, sampled) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 175a4cfbd2c..2924152a981 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -175,10 +175,12 @@ func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it v return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 } -func (sm *StorageManager) StorageLimitReached() bool { - limit := sm.storageLimit.Load() - lsm, vlog := sm.Size() // FIXME: what's the overhead? - return limit != 0 && uint64(lsm+vlog) > limit +func (sm *StorageManager) DiskUsage() uint64 { + return sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage() +} + +func (sm *StorageManager) StorageLimit() uint64 { + return sm.storageLimit.Load() } func (sm *StorageManager) Flush() error { From b9725a2d5dd21cdc04c0e5f7d91e6dd8fdace138 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 22:54:18 +0000 Subject: [PATCH 119/184] Test StorageLimitReadWriter --- x-pack/apm-server/sampling/eventstorage/rw.go | 7 ++ .../sampling/eventstorage/rw_test.go | 109 ++++++++++++++++++ .../sampling/eventstorage/storage_manager.go | 11 +- 3 files changed, 120 insertions(+), 7 deletions(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/rw_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 3f0d6335306..b11fe8cad99 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -61,6 +61,13 @@ type StorageLimitReadWriter struct { nextRW RW } +func NewStorageLimitReadWriter(checker storageLimitChecker, nextRW RW) StorageLimitReadWriter { + return StorageLimitReadWriter{ + checker: checker, + nextRW: nextRW, + } +} + func (s StorageLimitReadWriter) checkStorageLimit() error { usage := s.checker.DiskUsage() limit := s.checker.StorageLimit() diff --git a/x-pack/apm-server/sampling/eventstorage/rw_test.go b/x-pack/apm-server/sampling/eventstorage/rw_test.go new file mode 100644 index 00000000000..bcf6490c16d --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/rw_test.go @@ -0,0 +1,109 @@ +// 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_test + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/elastic/apm-data/model/modelpb" + "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" +) + +type mockChecker struct { + usage, limit uint64 +} + +func (m mockChecker) DiskUsage() uint64 { + return m.usage +} + +func (m mockChecker) StorageLimit() uint64 { + return m.limit +} + +type mockRW struct { + callback func() +} + +func (m mockRW) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + m.callback() + return nil +} + +func (m mockRW) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { + m.callback() + return nil +} + +func (m mockRW) WriteTraceSampled(traceID string, sampled bool) error { + m.callback() + return nil +} + +func (m mockRW) IsTraceSampled(traceID string) (bool, error) { + m.callback() + return false, nil +} + +func (m mockRW) DeleteTraceEvent(traceID, id string) error { + m.callback() + return nil +} + +func (m mockRW) Flush() error { + m.callback() + return nil +} + +func TestStorageLimitReadWriter(t *testing.T) { + for _, tt := range []struct { + limit, usage uint64 + wantCalled bool + }{ + { + limit: 0, // unlimited + usage: 1, + wantCalled: true, + }, + { + limit: 2, + usage: 3, + wantCalled: false, + }, + } { + t.Run(fmt.Sprintf("limit=%d,usage=%d", tt.limit, tt.usage), func(t *testing.T) { + checker := mockChecker{limit: tt.limit, usage: tt.usage} + var callCount int + rw := eventstorage.NewStorageLimitReadWriter(checker, mockRW{ + callback: func() { + callCount++ + }, + }) + assert.NoError(t, rw.ReadTraceEvents("foo", nil)) + _, err := rw.IsTraceSampled("foo") + assert.NoError(t, err) + assert.NoError(t, rw.DeleteTraceEvent("foo", "bar")) + + err = rw.WriteTraceEvent("foo", "bar", nil) + if tt.wantCalled { + assert.NoError(t, err) + assert.Equal(t, 4, callCount) + } else { + assert.Error(t, err) + } + err = rw.WriteTraceSampled("foo", true) + if tt.wantCalled { + assert.NoError(t, err) + assert.Equal(t, 5, callCount) + } else { + assert.Error(t, err) + } + }) + } + +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 2924152a981..4efb632fa9f 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -275,13 +275,10 @@ func (sm *StorageManager) WriteSubscriberPosition(data []byte) error { } func (sm *StorageManager) NewReadWriter() StorageLimitReadWriter { - return StorageLimitReadWriter{ - checker: sm, - nextRW: SplitReadWriter{ - eventRW: sm.eventStorage.NewReadWriter(), - decisionRW: sm.decisionStorage.NewReadWriter(), - }, - } + return NewStorageLimitReadWriter(sm, SplitReadWriter{ + eventRW: sm.eventStorage.NewReadWriter(), + decisionRW: sm.decisionStorage.NewReadWriter(), + }) } // NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, From e411486dba157b88d9b6db419ee81a496a3000f4 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 22:58:45 +0000 Subject: [PATCH 120/184] Refactor --- .../sampling/eventstorage/partition_rw.go | 6 +-- .../sampling/eventstorage/prefix.go | 14 ++++++ x-pack/apm-server/sampling/eventstorage/rw.go | 11 +++-- .../sampling/eventstorage/storage.go | 47 ++++++++++++------- .../sampling/eventstorage/storage_manager.go | 32 +------------ .../eventstorage/storage_manager_test.go | 8 ++-- 6 files changed, 56 insertions(+), 62 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index ce9bf3fa0b2..7d7341334d1 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -10,15 +10,11 @@ import ( "github.com/elastic/apm-data/model/modelpb" ) -// PartitionReadWriter provides a means of reading events from storage, and batched -// writing of events to storage. +// PartitionReadWriter reads from and writes to storage across partitions. type PartitionReadWriter struct { s *Storage } -// Close closes the writer. Any writes that have not been flushed may be lost. -func (rw *PartitionReadWriter) Close() {} - // Flush waits for preceding writes to be committed to storage. func (rw *PartitionReadWriter) Flush() error { return nil diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index a9a686d40b1..9b42a61e517 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -6,6 +6,7 @@ package eventstorage import ( "bytes" + "errors" "fmt" "github.com/cockroachdb/pebble/v2" @@ -13,6 +14,19 @@ import ( "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 byte = 's' + entryMetaTraceUnsampled byte = 'u' +) + +var ( + // ErrNotFound is returned by the RW.IsTraceSampled method, + // for non-existing trace IDs. + ErrNotFound = errors.New("key not found") +) + func NewPrefixReadWriter(db db, prefix byte, codec Codec) PrefixReadWriter { return PrefixReadWriter{db: db, prefix: prefix, codec: codec} } diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index b11fe8cad99..8e9a82d9e11 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -5,11 +5,18 @@ package eventstorage import ( + "errors" "fmt" "github.com/elastic/apm-data/model/modelpb" ) +var ( + // ErrLimitReached is returned by RW methods when storage usage + // is greater than configured limit. + ErrLimitReached = errors.New("configured storage limit reached") +) + type RW interface { ReadTraceEvents(traceID string, out *modelpb.Batch) error WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error @@ -107,7 +114,3 @@ func (s StorageLimitReadWriter) DeleteTraceEvent(traceID, id string) error { func (s StorageLimitReadWriter) Flush() error { return s.nextRW.Flush() } - -func (s StorageLimitReadWriter) Close() error { - return nil -} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 8dd7ec7fcc7..dd86a7c6569 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -5,7 +5,6 @@ package eventstorage import ( - "errors" "io" "github.com/cockroachdb/pebble/v2" @@ -13,23 +12,6 @@ import ( "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 byte = 's' - entryMetaTraceUnsampled byte = 'u' -) - -var ( - // ErrNotFound is returned by the RW.IsTraceSampled method, - // for non-existing trace IDs. - ErrNotFound = errors.New("key not found") - - // ErrLimitReached is returned by RW methods when storage usage - // is greater than configured limit. - ErrLimitReached = errors.New("configured storage limit reached") -) - type db interface { Get(key []byte) ([]byte, io.Closer, error) Set(key, value []byte, opts *pebble.WriteOptions) error @@ -43,6 +25,35 @@ type partitionedDB interface { WritePartition() PartitionIterator } +type wrappedDB struct { + partitioner *Partitioner + db *pebble.DB +} + +func (w *wrappedDB) Get(key []byte) ([]byte, io.Closer, error) { + return w.db.Get(key) +} + +func (w *wrappedDB) Set(key, value []byte, opts *pebble.WriteOptions) error { + return w.db.Set(key, value, opts) +} + +func (w *wrappedDB) Delete(key []byte, opts *pebble.WriteOptions) error { + return w.db.Delete(key, opts) +} + +func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { + return w.db.NewIter(o) +} + +func (w *wrappedDB) ReadPartitions() PartitionIterator { + return w.partitioner.Actives() +} + +func (w *wrappedDB) WritePartition() PartitionIterator { + return w.partitioner.Current() +} + // Storage provides storage for sampled transactions and spans, // and for recording trace sampling decisions. type Storage struct { diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 4efb632fa9f..e09ade28211 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -8,7 +8,6 @@ import ( "encoding/json" "errors" "fmt" - "io" "os" "path/filepath" "sync" @@ -38,35 +37,6 @@ const ( partitionsPerTTL = 1 ) -type wrappedDB struct { - partitioner *Partitioner - db *pebble.DB -} - -func (w *wrappedDB) Get(key []byte) ([]byte, io.Closer, error) { - return w.db.Get(key) -} - -func (w *wrappedDB) Set(key, value []byte, opts *pebble.WriteOptions) error { - return w.db.Set(key, value, opts) -} - -func (w *wrappedDB) Delete(key []byte, opts *pebble.WriteOptions) error { - return w.db.Delete(key, opts) -} - -func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { - return w.db.NewIter(o) -} - -func (w *wrappedDB) ReadPartitions() PartitionIterator { - return w.partitioner.Actives() -} - -func (w *wrappedDB) WritePartition() PartitionIterator { - return w.partitioner.Current() -} - type StorageManagerOptions func(*StorageManager) func WithCodec(codec Codec) StorageManagerOptions { @@ -283,7 +253,7 @@ func (sm *StorageManager) NewReadWriter() StorageLimitReadWriter { // NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, // bypassing any wrappers. -// This should be used for testing only, useful to check if data is actually persisted to the DB. +// For testing only. Useful for checking if data is actually persisted to the DB. func (sm *StorageManager) NewBypassReadWriter() SplitReadWriter { return SplitReadWriter{ eventRW: sm.eventStorage.NewReadWriter(), diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 3236e1d87f3..7205f39f295 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -15,12 +15,12 @@ import ( ) func newStorageManager(tb testing.TB, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { - sm := newStorageManagerManual(tb, tb.TempDir(), opts...) + sm := newStorageManagerNoCleanup(tb, tb.TempDir(), opts...) tb.Cleanup(func() { sm.Close() }) return sm } -func newStorageManagerManual(tb testing.TB, path string, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { +func newStorageManagerNoCleanup(tb testing.TB, path string, opts ...eventstorage.StorageManagerOptions) *eventstorage.StorageManager { sm, err := eventstorage.NewStorageManager(path, opts...) if err != nil { tb.Fatal(err) @@ -106,7 +106,7 @@ func TestStorageManager_eventTTL(t *testing.T) { func TestStorageManager_partitionID(t *testing.T) { const traceID = "foo" tmpDir := t.TempDir() - sm := newStorageManagerManual(t, tmpDir) + sm := newStorageManagerNoCleanup(t, tmpDir) // 0 -> 1 assert.NoError(t, sm.RotatePartitions()) @@ -118,7 +118,7 @@ func TestStorageManager_partitionID(t *testing.T) { assert.NoError(t, sm.Close()) // it should read directly from partition 1 on startup instead of 0 - sm = newStorageManagerManual(t, tmpDir) + sm = newStorageManagerNoCleanup(t, tmpDir) defer sm.Close() sampled, err := sm.NewReadWriter().IsTraceSampled(traceID) assert.NoError(t, err) From 64f0d78c18852efa0f2de516c34b99f3714fae6c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 23:24:14 +0000 Subject: [PATCH 121/184] Enable parallel writers --- 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 83fefea40ea..e1ecaf6a1e3 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -25,7 +25,7 @@ func eventComparer() *pebble.Comparer { } func OpenEventPebble(storageDir string) (*pebble.DB, error) { - return pebble.Open(filepath.Join(storageDir, "event"), &pebble.Options{ + opts := &pebble.Options{ FormatMajorVersion: pebble.FormatColumnarBlocks, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 16 << 20, @@ -38,7 +38,9 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { }, }, Comparer: eventComparer(), - }) + } + opts.Experimental.MaxWriterConcurrency = 1 // >0 enables parallel writers, the actual value doesn't matter + return pebble.Open(filepath.Join(storageDir, "event"), opts) } func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { From d5e10a493847264f052b0ab0d85eac4ffca28fc0 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 23:28:10 +0000 Subject: [PATCH 122/184] Remove storage limit threshold --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- x-pack/apm-server/sampling/processor.go | 6 +----- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index e09ade28211..fffa827dc83 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -175,7 +175,7 @@ func (sm *StorageManager) Reload() error { } // Run has the same lifecycle as the TBS processor as opposed to StorageManager to facilitate EA hot reload. -func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, storageLimit uint64, storageLimitThreshold float64) error { +func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, storageLimit uint64) error { select { case <-stopping: return nil diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 0ba09d1d145..ae291e5fc0e 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -378,7 +378,7 @@ func (p *Processor) Run() error { } }) g.Go(func() error { - return p.config.DB.Run(p.stopping, p.config.TTL, p.config.StorageLimit, storageLimitThreshold) + return p.config.DB.Run(p.stopping, p.config.TTL, p.config.StorageLimit) }) g.Go(func() error { // Subscribe to remotely sampled trace IDs. This is cancelled immediately when @@ -566,7 +566,3 @@ func sendTraceIDs(ctx context.Context, out chan<- string, traceIDs []string) err } return nil } - -const ( - storageLimitThreshold = 0.90 // Allow 90% of the quota to be used. -) From 9ff4008a26731ee467fea88d3e5af3e884f145f7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Thu, 23 Jan 2025 23:35:32 +0000 Subject: [PATCH 123/184] Change partitioner meta key --- .../sampling/eventstorage/storage_manager.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index fffa827dc83..89c208353aa 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -35,6 +35,9 @@ const ( // This means storage requirement is 2 * TTL, and it needs to read 2 keys per trace ID read. // If partitionPerTTL=2, storage requirement is 1.5 * TTL at the expense of 3 reads per trace ID read. partitionsPerTTL = 1 + + // partitionerMetaKey is the key used to store partitioner metadata, e.g. last partition ID, in decision DB. + partitionerMetaKey = "~partitioner" ) type StorageManagerOptions func(*StorageManager) @@ -56,7 +59,7 @@ type StorageManager struct { eventStorage *Storage decisionStorage *Storage - partitioner *Partitioner // FIXME: load the correct partition ID on restart + partitioner *Partitioner storageLimit atomic.Uint64 @@ -115,12 +118,10 @@ func (sm *StorageManager) reset() error { return nil } -const partitionIDKey = "!partitioner" - // loadPartitionID loads the last saved partition ID from database, // such that partitioner resumes from where it left off before an apm-server restart. func (sm *StorageManager) loadPartitionID() (int, error) { - item, closer, err := sm.decisionDB.Get([]byte(partitionIDKey)) + item, closer, err := sm.decisionDB.Get([]byte(partitionerMetaKey)) if errors.Is(err, pebble.ErrNotFound) { return 0, nil } else if err != nil { @@ -136,8 +137,7 @@ func (sm *StorageManager) loadPartitionID() (int, error) { // savePartitionID saves the partition ID to database to be loaded by loadPartitionID later. func (sm *StorageManager) savePartitionID(pid int) error { - // FIXME: Add a timestamp to avoid using a very old database? - return sm.decisionDB.Set([]byte(partitionIDKey), []byte(fmt.Sprintf(`{"id":%d}`, pid)), pebble.NoSync) + return sm.decisionDB.Set([]byte(partitionerMetaKey), []byte(fmt.Sprintf(`{"id":%d}`, pid)), pebble.NoSync) } func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it vlog From 9285fbce67fa932ae1776cce705b0ea201202758 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 00:03:01 +0000 Subject: [PATCH 124/184] Improve comments --- .../sampling/eventstorage/partitioner.go | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 311141b2348..61b85d249b9 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -11,9 +11,9 @@ import "sync/atomic" // `current` points at the rightmost active partition. // // Example for total=4: -// (A: active, I: inactive) +// (A: active, I: inactive, ^ points at the current active entry) // A-I-A-A -// ^ +// ^...... // current type Partitioner struct { total int // length of the ring @@ -25,14 +25,27 @@ func NewPartitioner(actives int) *Partitioner { return &Partitioner{total: actives + 1} // actives + 1 inactive } +// SetCurrentID sets the input partition ID as current partition. func (p *Partitioner) SetCurrentID(current int) { p.current.Store(int32(current)) } +// Rotate rotates partitions to the right by 1 position. +// +// Example for total=4: +// (A: active, I: inactive, ^ points at the current active entry) +// A-I-A-A +// ^...... +// +// After Rotate: +// A-A-I-A +// ..^.... func (p *Partitioner) Rotate() { p.current.Store(int32((int(p.current.Load()) + 1) % p.total)) } +// Actives returns a PartitionIterator containing all active partitions. +// It contains total - 1 partitions. func (p *Partitioner) Actives() PartitionIterator { return PartitionIterator{ id: int(p.current.Load()), @@ -41,6 +54,8 @@ func (p *Partitioner) Actives() PartitionIterator { } } +// Inactive returns a PartitionIterator pointing to the inactive partition. +// It contains only 1 partition. func (p *Partitioner) Inactive() PartitionIterator { return PartitionIterator{ id: (int(p.current.Load()) + 1) % p.total, @@ -49,6 +64,8 @@ func (p *Partitioner) Inactive() PartitionIterator { } } +// Current returns a PartitionIterator pointing to the current partition (rightmost active). +// It contains only 1 partition. func (p *Partitioner) Current() PartitionIterator { return PartitionIterator{ id: int(p.current.Load()), From 74109a180e615133364c4169298bf60e1059d583 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 10:50:40 +0000 Subject: [PATCH 125/184] Remove StorageDir config --- x-pack/apm-server/main.go | 1 - x-pack/apm-server/sampling/config.go | 6 -- x-pack/apm-server/sampling/config_test.go | 3 - .../sampling/processor_bench_test.go | 2 +- x-pack/apm-server/sampling/processor_test.go | 80 ++++++++++--------- 5 files changed, 45 insertions(+), 47 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 0c6e58a5c70..2e578a22b9d 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -152,7 +152,6 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er }, StorageConfig: sampling.StorageConfig{ DB: db, - StorageDir: storageDir, StorageLimit: tailSamplingConfig.StorageLimitParsed, TTL: tailSamplingConfig.TTL, DiscardOnWriteFailure: tailSamplingConfig.DiscardOnWriteFailure, diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index 11157ef8a5d..1e74a4c70f6 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -104,9 +104,6 @@ type StorageConfig struct { // For testing only. Storage eventstorage.RW - // StorageDir holds the directory in which event storage will be maintained. - StorageDir string - // StorageLimit for the TBS database, in bytes. StorageLimit uint64 @@ -237,9 +234,6 @@ func (config StorageConfig) validate() error { if config.DB == nil { return errors.New("DB unspecified") } - if config.StorageDir == "" { - return errors.New("StorageDir unspecified") - } if config.TTL <= 0 { return errors.New("TTL unspecified or negative") } diff --git a/x-pack/apm-server/sampling/config_test.go b/x-pack/apm-server/sampling/config_test.go index 55093813d47..b5575a939b3 100644 --- a/x-pack/apm-server/sampling/config_test.go +++ b/x-pack/apm-server/sampling/config_test.go @@ -73,9 +73,6 @@ func TestNewProcessorConfigInvalid(t *testing.T) { assertInvalidConfigError("invalid storage config: DB unspecified") config.DB = &eventstorage.StorageManager{} - assertInvalidConfigError("invalid storage config: StorageDir unspecified") - config.StorageDir = "tbs" - assertInvalidConfigError("invalid storage config: TTL unspecified or negative") config.TTL = 1 } diff --git a/x-pack/apm-server/sampling/processor_bench_test.go b/x-pack/apm-server/sampling/processor_bench_test.go index 196a62f0cd4..4ae725bcd2a 100644 --- a/x-pack/apm-server/sampling/processor_bench_test.go +++ b/x-pack/apm-server/sampling/processor_bench_test.go @@ -20,7 +20,7 @@ import ( ) func BenchmarkProcess(b *testing.B) { - processor, err := sampling.NewProcessor(newTempdirConfig(b)) + processor, err := sampling.NewProcessor(newTempdirConfig(b).Config) require.NoError(b, err) go processor.Run() b.Cleanup(func() { processor.Stop(context.Background()) }) diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 7b296c00caa..6a86de3167b 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -29,7 +29,7 @@ import ( ) func TestProcessUnsampled(t *testing.T) { - processor, err := sampling.NewProcessor(newTempdirConfig(t)) + processor, err := sampling.NewProcessor(newTempdirConfig(t).Config) require.NoError(t, err) go processor.Run() defer processor.Stop(context.Background()) @@ -53,7 +53,7 @@ func TestProcessUnsampled(t *testing.T) { } func TestProcessAlreadyTailSampled(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config // Seed event storage with a tail-sampling decisions, to show that // subsequent events in the trace will be reported immediately. @@ -158,7 +158,7 @@ func TestProcessLocalTailSampling(t *testing.T) { }, } { t.Run(fmt.Sprintf("%f", tc.sampleRate), func(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.Policies = []sampling.Policy{{SampleRate: tc.sampleRate}} config.FlushInterval = 10 * time.Millisecond published := make(chan string) @@ -280,7 +280,7 @@ func TestProcessLocalTailSampling(t *testing.T) { } func TestProcessLocalTailSamplingUnsampled(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.FlushInterval = time.Minute processor, err := sampling.NewProcessor(config) require.NoError(t, err) @@ -335,7 +335,7 @@ func TestProcessLocalTailSamplingUnsampled(t *testing.T) { } func TestProcessLocalTailSamplingPolicyOrder(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.Policies = []sampling.Policy{{ PolicyCriteria: sampling.PolicyCriteria{TraceName: "trace_name"}, SampleRate: 0.5, @@ -402,7 +402,7 @@ func TestProcessLocalTailSamplingPolicyOrder(t *testing.T) { } func TestProcessRemoteTailSampling(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.Policies = []sampling.Policy{{SampleRate: 0.5}} config.FlushInterval = 10 * time.Millisecond @@ -534,7 +534,7 @@ func (m errorRW) Flush() error { func TestProcessDiscardOnWriteFailure(t *testing.T) { for _, discard := range []bool{true, false} { t.Run(fmt.Sprintf("discard=%v", discard), func(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.DiscardOnWriteFailure = discard config.Storage = errorRW{err: errors.New("boom")} processor, err := sampling.NewProcessor(config) @@ -567,7 +567,7 @@ func TestProcessDiscardOnWriteFailure(t *testing.T) { } func TestGroupsMonitoring(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.MaxDynamicServices = 5 config.FlushInterval = time.Minute config.Policies[0].SampleRate = 0.99 @@ -603,7 +603,7 @@ func TestGroupsMonitoring(t *testing.T) { } func TestStorageMonitoring(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config processor, err := sampling.NewProcessor(config) require.NoError(t, err) @@ -661,7 +661,7 @@ func TestStorageLimit(t *testing.T) { return processor } - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config config.TTL = time.Hour // Write 5K span events and close the DB to persist to disk the storage // size and assert that none are reported immediately. @@ -692,7 +692,8 @@ func TestStorageLimit(t *testing.T) { } func TestProcessRemoteTailSamplingPersistence(t *testing.T) { - config := newTempdirConfig(t) + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.Policies = []sampling.Policy{{SampleRate: 0.5}} config.FlushInterval = 10 * time.Millisecond @@ -706,7 +707,7 @@ func TestProcessRemoteTailSamplingPersistence(t *testing.T) { defer processor.Stop(context.Background()) // Wait for subscriber_position.json to be written to the storage directory. - subscriberPositionFile := filepath.Join(config.StorageDir, "subscriber_position.json") + subscriberPositionFile := filepath.Join(tempdirConfig.tempDir, "subscriber_position.json") data, info := waitFileModified(t, subscriberPositionFile, time.Time{}) assert.Equal(t, "{}", string(data)) @@ -716,7 +717,7 @@ func TestProcessRemoteTailSamplingPersistence(t *testing.T) { } func TestGracefulShutdown(t *testing.T) { - config := newTempdirConfig(t) + config := newTempdirConfig(t).Config sampleRate := 0.5 config.Policies = []sampling.Policy{{SampleRate: sampleRate}} config.FlushInterval = time.Minute // disable finalize @@ -758,7 +759,12 @@ func TestGracefulShutdown(t *testing.T) { assert.Equal(t, int(sampleRate*float64(totalTraces)), count) } -func newTempdirConfig(tb testing.TB) sampling.Config { +type testConfig struct { + sampling.Config + tempDir string +} + +func newTempdirConfig(tb testing.TB) testConfig { tempdir, err := os.MkdirTemp("", "samplingtest") require.NoError(tb, err) tb.Cleanup(func() { os.RemoveAll(tempdir) }) @@ -767,30 +773,32 @@ func newTempdirConfig(tb testing.TB) sampling.Config { require.NoError(tb, err) tb.Cleanup(func() { db.Close() }) - return sampling.Config{ - BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), - LocalSamplingConfig: sampling.LocalSamplingConfig{ - FlushInterval: time.Second, - MaxDynamicServices: 1000, - IngestRateDecayFactor: 0.9, - Policies: []sampling.Policy{ - {SampleRate: 0.1}, + return testConfig{ + tempDir: tempdir, + Config: sampling.Config{ + BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), + LocalSamplingConfig: sampling.LocalSamplingConfig{ + FlushInterval: time.Second, + MaxDynamicServices: 1000, + IngestRateDecayFactor: 0.9, + Policies: []sampling.Policy{ + {SampleRate: 0.1}, + }, }, - }, - RemoteSamplingConfig: sampling.RemoteSamplingConfig{ - Elasticsearch: pubsubtest.Client(nil, nil), - SampledTracesDataStream: sampling.DataStreamConfig{ - Type: "traces", - Dataset: "sampled", - Namespace: "testing", + RemoteSamplingConfig: sampling.RemoteSamplingConfig{ + Elasticsearch: pubsubtest.Client(nil, nil), + SampledTracesDataStream: sampling.DataStreamConfig{ + Type: "traces", + Dataset: "sampled", + Namespace: "testing", + }, + UUID: "local-apm-server", + }, + StorageConfig: sampling.StorageConfig{ + DB: db, + TTL: 30 * time.Minute, + StorageLimit: 0, // No storage limit. }, - UUID: "local-apm-server", - }, - StorageConfig: sampling.StorageConfig{ - DB: db, - StorageDir: tempdir, - TTL: 30 * time.Minute, - StorageLimit: 0, // No storage limit. }, } } From 9b14a25394c1d47ba01cb266b5fa927e22b77ad2 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 10:51:46 +0000 Subject: [PATCH 126/184] Remove errgroup --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 89c208353aa..bfa93563389 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -15,7 +15,6 @@ import ( "time" "github.com/cockroachdb/pebble/v2" - "golang.org/x/sync/errgroup" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" @@ -187,11 +186,7 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, stora sm.storageLimit.Store(storageLimit) - g := errgroup.Group{} - g.Go(func() error { - return sm.runTTLGCLoop(stopping, ttl) - }) - return g.Wait() + return sm.runTTLGCLoop(stopping, ttl) } func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Duration) error { From c2d62ed74e34cf3aba4627610b4f1a644f02dfc7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 10:58:51 +0000 Subject: [PATCH 127/184] Change partitioner interface --- .../sampling/eventstorage/partition_rw.go | 4 ++-- .../sampling/eventstorage/partitioner.go | 20 +++++++++---------- .../sampling/eventstorage/storage.go | 4 ++-- .../sampling/eventstorage/storage_manager.go | 4 ++-- 4 files changed, 15 insertions(+), 17 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 7d7341334d1..52a93683309 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -22,7 +22,7 @@ func (rw *PartitionReadWriter) Flush() error { // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { - pid := rw.s.db.WritePartition().ID() + pid := rw.s.db.WritePartition() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) } @@ -51,7 +51,7 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { // WriteTraceEvent writes a trace event to storage. func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { - pid := rw.s.db.WritePartition().ID() + pid := rw.s.db.WritePartition() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 61b85d249b9..051622af7fa 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -30,7 +30,8 @@ func (p *Partitioner) SetCurrentID(current int) { p.current.Store(int32(current)) } -// Rotate rotates partitions to the right by 1 position. +// Rotate rotates partitions to the right by 1 position and +// returns the ID of the new current active entry. // // Example for total=4: // (A: active, I: inactive, ^ points at the current active entry) @@ -40,8 +41,10 @@ func (p *Partitioner) SetCurrentID(current int) { // After Rotate: // A-A-I-A // ..^.... -func (p *Partitioner) Rotate() { - p.current.Store(int32((int(p.current.Load()) + 1) % p.total)) +func (p *Partitioner) Rotate() int { + newCurrent := (int(p.current.Load()) + 1) % p.total + p.current.Store(int32(newCurrent)) + return newCurrent } // Actives returns a PartitionIterator containing all active partitions. @@ -64,14 +67,9 @@ func (p *Partitioner) Inactive() PartitionIterator { } } -// Current returns a PartitionIterator pointing to the current partition (rightmost active). -// It contains only 1 partition. -func (p *Partitioner) Current() PartitionIterator { - return PartitionIterator{ - id: int(p.current.Load()), - remaining: 0, - total: p.total, - } +// Current returns the ID of the current partition (rightmost active). +func (p *Partitioner) Current() int { + return int(p.current.Load()) } // PartitionIterator is for iterating on partition results. diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index dd86a7c6569..410e632e79c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -22,7 +22,7 @@ type db interface { type partitionedDB interface { db ReadPartitions() PartitionIterator - WritePartition() PartitionIterator + WritePartition() int } type wrappedDB struct { @@ -50,7 +50,7 @@ func (w *wrappedDB) ReadPartitions() PartitionIterator { return w.partitioner.Actives() } -func (w *wrappedDB) WritePartition() PartitionIterator { +func (w *wrappedDB) WritePartition() int { return w.partitioner.Current() } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index bfa93563389..a12048f4c63 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -209,9 +209,9 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati // RotatePartitions rotates the partitions to clean up TTL-expired entries. func (sm *StorageManager) RotatePartitions() error { - sm.partitioner.Rotate() + newCurrentPID := sm.partitioner.Rotate() - if err := sm.savePartitionID(sm.partitioner.Current().ID()); err != nil { + if err := sm.savePartitionID(newCurrentPID); err != nil { return err } From ed25987e303a6f0b4948498b9fca6e8092096c8d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 11:15:58 +0000 Subject: [PATCH 128/184] Remove writeEntry --- x-pack/apm-server/sampling/eventstorage/prefix.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 9b42a61e517..42cfe784625 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -89,14 +89,7 @@ func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.AP b.WriteByte(':') b.WriteString(id) key := b.Bytes() - return rw.writeEntry(key, data) -} - -func (rw PrefixReadWriter) writeEntry(key, data []byte) error { - if err := rw.db.Set(key, data, pebble.NoSync); err != nil { - return err - } - return nil + return rw.db.Set(key, data, pebble.NoSync) } func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error { From 7074ccf342c8b8292aa9d6b754ed082120964214 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 11:34:11 +0000 Subject: [PATCH 129/184] Add sanity check to total partitions --- .../sampling/eventstorage/partitioner.go | 15 ++++++++++++++- .../sampling/eventstorage/storage_manager.go | 5 ++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 051622af7fa..a08a766bc25 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -6,6 +6,15 @@ package eventstorage import "sync/atomic" +const ( + // maxTotalPartitions is the maximum number of total partitions. + // It is used for a sanity check specific to how we use it as a byte prefix in database keys. + // It MUST be less than 256 to be contained in a byte. + // It has an additional (arbitrary) limitation to be less than reservedKeyPrefix + // to avoid accidentally overwriting reserved keys down the line. + maxTotalPartitions = int(reservedKeyPrefix) - 1 +) + // Partitioner is a partitioned ring with `total` number of partitions. // 1 of them is inactive while all the others are active. // `current` points at the rightmost active partition. @@ -22,7 +31,11 @@ type Partitioner struct { // NewPartitioner returns a partitioner with `actives` number of active partitions. func NewPartitioner(actives int) *Partitioner { - return &Partitioner{total: actives + 1} // actives + 1 inactive + total := actives + 1 // actives + 1 inactive + if total >= maxTotalPartitions { + panic("too many partitions") + } + return &Partitioner{total: total} } // SetCurrentID sets the input partition ID as current partition. diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index a12048f4c63..71a517173d4 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -35,8 +35,11 @@ const ( // If partitionPerTTL=2, storage requirement is 1.5 * TTL at the expense of 3 reads per trace ID read. partitionsPerTTL = 1 + // reservedKeyPrefix is the prefix of internal keys used by StorageManager + reservedKeyPrefix byte = '~' + // partitionerMetaKey is the key used to store partitioner metadata, e.g. last partition ID, in decision DB. - partitionerMetaKey = "~partitioner" + partitionerMetaKey = string(reservedKeyPrefix) + "partitioner" ) type StorageManagerOptions func(*StorageManager) From 32213d668da54e37a6d6fb3c606c896160b46f61 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 14:01:59 +0000 Subject: [PATCH 130/184] Fix partitioner interface --- .../sampling/eventstorage/partitioner_test.go | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 60be111af8c..5538bfb2b53 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -16,8 +16,7 @@ func TestPartitioner(t *testing.T) { p := eventstorage.NewPartitioner(2) // partition id 0, 1, 2 cur := p.Current() - assert.True(t, cur.Valid()) - assert.Equal(t, 0, cur.ID()) + assert.Equal(t, 0, cur) inactive := p.Inactive() assert.True(t, inactive.Valid()) @@ -35,8 +34,7 @@ func TestPartitioner(t *testing.T) { p.Rotate() cur = p.Current() - assert.True(t, cur.Valid()) - assert.Equal(t, 1, cur.ID()) + assert.Equal(t, 1, cur) inactive = p.Inactive() assert.True(t, inactive.Valid()) @@ -54,8 +52,7 @@ func TestPartitioner(t *testing.T) { p.Rotate() cur = p.Current() - assert.True(t, cur.Valid()) - assert.Equal(t, 2, cur.ID()) + assert.Equal(t, 2, cur) inactive = p.Inactive() assert.True(t, inactive.Valid()) @@ -73,8 +70,7 @@ func TestPartitioner(t *testing.T) { p.Rotate() cur = p.Current() - assert.True(t, cur.Valid()) - assert.Equal(t, 0, cur.ID()) + assert.Equal(t, 0, cur) inactive = p.Inactive() assert.True(t, inactive.Valid()) From 25d31ed73f5e30c2792bb8e52639895684862eff Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 14:02:27 +0000 Subject: [PATCH 131/184] Bypass pebble Flush issue for now --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 4 +++- x-pack/apm-server/sampling/processor_test.go | 2 -- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 71a517173d4..6ca9c2c8f12 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -164,7 +164,9 @@ func (sm *StorageManager) Close() error { } func (sm *StorageManager) close() error { - return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) + return errors.Join(sm.eventDB.Close(), sm.decisionDB.Close()) + // FIXME: there seems to be an error with pebble Flush hanging indefinitely + //return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) } // Reload flushes out pending disk writes to disk by reloading the database. diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 6a86de3167b..c0999c7c7ba 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -745,8 +745,6 @@ func TestGracefulShutdown(t *testing.T) { assert.Empty(t, batch) assert.NoError(t, processor.Stop(context.Background())) - config.DB.Flush() // FIXME: pebble BUG? flush blocks indefinitely for totalTraces < 2000 - reader := config.DB.NewBypassReadWriter() defer reader.Close() From b9907929fd1b625cb025b1952a6ee6dce813510d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 14:13:15 +0000 Subject: [PATCH 132/184] Remove bounds when using SeekPrefixGE --- x-pack/apm-server/sampling/eventstorage/prefix.go | 13 ++++--------- .../apm-server/sampling/eventstorage/prefix_test.go | 9 +++++++-- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 42cfe784625..d7596236468 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -44,22 +44,17 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e lb.WriteString(traceID) lb.WriteByte(':') - var ub bytes.Buffer - ub.Grow(lb.Len()) - ub.Write(lb.Bytes()[:lb.Len()-1]) - ub.WriteByte(';') // This is a hack to stop before next ID - - iter, err := rw.db.NewIter(&pebble.IterOptions{ - LowerBound: lb.Bytes(), - UpperBound: ub.Bytes(), - }) + iter, err := rw.db.NewIter(&pebble.IterOptions{}) if err != nil { return err } defer iter.Close() + // SeekPrefixGE uses prefix bloom filter for on disk tables. // These bloom filters are cached in memory, and a "miss" on bloom filter avoids disk IO to check the actual table. // Memtables still need to be scanned as pebble has no bloom filter on memtables. + // + // SeekPrefixGE ensures the prefix is present and does not require lower bound and upper bound to be set on iterator. if valid := iter.SeekPrefixGE(lb.Bytes()); !valid { return nil } diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index dad2814d20a..0cd3d7e86e6 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -60,15 +60,20 @@ func TestPrefixReadWriter_ReadTraceEvents(t *testing.T) { db := newEventPebble(t) rw := eventstorage.NewPrefixReadWriter(db, 1, codec) - traceID := "foo" + traceID := "foo1" for _, txnID := range []string{"bar", "baz"} { txn := makeTransaction(txnID, traceID) err := rw.WriteTraceEvent(traceID, txnID, txn) require.NoError(t, err) } + // Create a transaction with a similar trace ID to ensure that iterator upper bound is enforced + txn := makeTransaction("bar", "foo2") + err := rw.WriteTraceEvent("foo2", "bar", txn) + require.NoError(t, err) + var out modelpb.Batch - err := rw.ReadTraceEvents(traceID, &out) + err = rw.ReadTraceEvents(traceID, &out) assert.NoError(t, err) assert.Equal(t, modelpb.Batch{ makeTransaction("bar", traceID), From 3bceb43b7ace1e26409a82589edaeac4ac57ce19 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 14:28:53 +0000 Subject: [PATCH 133/184] Remove RW Flush --- x-pack/apm-server/sampling/eventstorage/partition_rw.go | 5 ----- x-pack/apm-server/sampling/eventstorage/prefix.go | 4 ---- x-pack/apm-server/sampling/eventstorage/rw.go | 9 --------- .../sampling/eventstorage/storage_bench_test.go | 5 ----- x-pack/apm-server/sampling/processor.go | 8 ++++---- x-pack/apm-server/sampling/processor_test.go | 2 -- 6 files changed, 4 insertions(+), 29 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 52a93683309..0f7cdb377f6 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -15,11 +15,6 @@ type PartitionReadWriter struct { s *Storage } -// Flush waits for preceding writes to be committed to storage. -func (rw *PartitionReadWriter) Flush() error { - return nil -} - // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { pid := rw.s.db.WritePartition() diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index d7596236468..31d304b65fa 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -133,7 +133,3 @@ func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { } return nil } - -func (rw PrefixReadWriter) Flush() error { - return nil -} diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 8e9a82d9e11..7f25e4cf88b 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -23,7 +23,6 @@ type RW interface { WriteTraceSampled(traceID string, sampled bool) error IsTraceSampled(traceID string) (bool, error) DeleteTraceEvent(traceID, id string) error - Flush() error } type SplitReadWriter struct { @@ -50,10 +49,6 @@ func (s SplitReadWriter) DeleteTraceEvent(traceID, id string) error { return s.eventRW.DeleteTraceEvent(traceID, id) } -func (s SplitReadWriter) Flush() error { - return nil -} - func (s SplitReadWriter) Close() error { return nil } @@ -110,7 +105,3 @@ func (s StorageLimitReadWriter) DeleteTraceEvent(traceID, id string) error { // Technically DeleteTraceEvent writes, but it should have a net effect of reducing disk usage return s.nextRW.DeleteTraceEvent(traceID, id) } - -func (s StorageLimitReadWriter) Flush() error { - return s.nextRW.Flush() -} diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index 3c645da22c5..f03019a69b1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -10,7 +10,6 @@ import ( "testing" "github.com/gofrs/uuid/v5" - "github.com/stretchr/testify/assert" "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" @@ -42,7 +41,6 @@ func BenchmarkWriteTransaction(b *testing.B) { b.Fatal(err) } } - assert.NoError(b, readWriter.Flush()) } type testCase struct { @@ -183,9 +181,6 @@ func BenchmarkReadEventsHit(b *testing.B) { } } } - if err := readWriter.Flush(); err != nil { - b.Fatal(err) - } readWriter.Close() diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index ae291e5fc0e..457a6878ab9 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -278,8 +278,9 @@ func (p *Processor) processSpan(event *modelpb.APMEvent) (report, stored bool, _ return traceSampled, false, nil } -// Stop stops the processor, flushing event storage. Note that the underlying -// StorageManager must be closed independently to ensure writes are synced to disk. +// Stop stops the processor. +// Note that the underlying StorageManager must be closed independently +// to ensure writes are synced to disk. func (p *Processor) Stop(ctx context.Context) error { p.stopMu.Lock() select { @@ -298,8 +299,7 @@ func (p *Processor) Stop(ctx context.Context) error { case <-p.stopped: } - // Flush event store and the underlying read writers - return p.eventStore.Flush() + return nil } // Run runs the tail-sampling processor. This method is responsible for: diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index c0999c7c7ba..dbcb3c2e15e 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -61,7 +61,6 @@ func TestProcessAlreadyTailSampled(t *testing.T) { trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} writer := config.DB.NewBypassReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true)) - assert.NoError(t, writer.Flush()) writer.Close() // simulate 2 TTL @@ -70,7 +69,6 @@ func TestProcessAlreadyTailSampled(t *testing.T) { writer = config.DB.NewBypassReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true)) - assert.NoError(t, writer.Flush()) writer.Close() require.NoError(t, config.DB.Flush()) From 767128fcf60ed0a3247dc809cd2a1f7d0286cd87 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:03:21 +0000 Subject: [PATCH 134/184] Use iter.Seq, remove PartitionIterator --- .../sampling/eventstorage/partition_rw.go | 12 +-- .../sampling/eventstorage/partitioner.go | 61 ++++---------- .../sampling/eventstorage/partitioner_test.go | 83 +++++-------------- .../sampling/eventstorage/storage.go | 5 +- 4 files changed, 48 insertions(+), 113 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 0f7cdb377f6..8aa26f99255 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -30,8 +30,8 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { // 1. (pubsub) remote sampling decision // 2. (hot path) sampling decision not made yet var errs []error - for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { - sampled, err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).IsTraceSampled(traceID) + for pid := range rw.s.db.ReadPartitions() { + sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil } else if err != ErrNotFound { @@ -54,8 +54,8 @@ func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelp func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // FIXME: use range delete var errs []error - for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { - err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).DeleteTraceEvent(traceID, id) + for pid := range rw.s.db.ReadPartitions() { + err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) if err != nil { errs = append(errs, err) } @@ -66,8 +66,8 @@ func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { var errs []error - for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { - err := NewPrefixReadWriter(rw.s.db, byte(it.ID()), rw.s.codec).ReadTraceEvents(traceID, out) + for pid := range rw.s.db.ReadPartitions() { + err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).ReadTraceEvents(traceID, out) if err != nil { errs = append(errs, err) } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index a08a766bc25..c49093be0a1 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -4,7 +4,10 @@ package eventstorage -import "sync/atomic" +import ( + "iter" + "sync/atomic" +) const ( // maxTotalPartitions is the maximum number of total partitions. @@ -60,57 +63,25 @@ func (p *Partitioner) Rotate() int { return newCurrent } -// Actives returns a PartitionIterator containing all active partitions. +// Actives returns an iterator containing all active partitions. // It contains total - 1 partitions. -func (p *Partitioner) Actives() PartitionIterator { - return PartitionIterator{ - id: int(p.current.Load()), - remaining: p.total - 1 - 1, - total: p.total, +func (p *Partitioner) Actives() iter.Seq[int] { + cur := int(p.current.Load()) + return func(yield func(int) bool) { + for i := 0; i < p.total-1; i++ { + if !yield((cur + p.total - i) % p.total) { + return + } + } } } -// Inactive returns a PartitionIterator pointing to the inactive partition. -// It contains only 1 partition. -func (p *Partitioner) Inactive() PartitionIterator { - return PartitionIterator{ - id: (int(p.current.Load()) + 1) % p.total, - remaining: 0, - total: p.total, - } +// Inactive returns the ID of the inactive partition. +func (p *Partitioner) Inactive() int { + return (int(p.current.Load()) + 1) % p.total } // Current returns the ID of the current partition (rightmost active). func (p *Partitioner) Current() int { return int(p.current.Load()) } - -// PartitionIterator is for iterating on partition results. -// In theory Partitioner could have returned a slice of partition IDs, -// but returning an iterator should avoid allocs. -// -// Example usage: -// for it := rw.s.db.ReadPartitions(); it.Valid(); it = it.Prev() { -// // do something with it.ID() -// } -type PartitionIterator struct { - id int - total int // length of the ring - remaining int -} - -func (it PartitionIterator) Prev() PartitionIterator { - return PartitionIterator{ - id: (it.id + it.total - 1) % it.total, - remaining: it.remaining - 1, - total: it.total, - } -} - -func (it PartitionIterator) Valid() bool { - return it.remaining >= 0 -} - -func (it PartitionIterator) ID() int { - return it.id -} diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 5538bfb2b53..c1ad555d575 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -5,6 +5,7 @@ package eventstorage_test import ( + "iter" "testing" "github.com/stretchr/testify/assert" @@ -13,75 +14,37 @@ import ( ) func TestPartitioner(t *testing.T) { - p := eventstorage.NewPartitioner(2) // partition id 0, 1, 2 - - cur := p.Current() - assert.Equal(t, 0, cur) + iterToSlice := func(it iter.Seq[int]) (s []int) { + for i := range it { + s = append(s, i) + } + return + } - inactive := p.Inactive() - assert.True(t, inactive.Valid()) - assert.Equal(t, 1, inactive.ID()) + p := eventstorage.NewPartitioner(2) // partition id 0, 1, 2 - active := p.Actives() - assert.True(t, active.Valid()) - assert.Equal(t, 0, active.ID()) - active = active.Prev() - assert.True(t, active.Valid()) - assert.Equal(t, 2, active.ID()) - active = active.Prev() - assert.False(t, active.Valid()) + assert.Equal(t, 0, p.Current()) + assert.Equal(t, 1, p.Inactive()) + assert.Equal(t, []int{0, 2}, iterToSlice(p.Actives())) + // 0 -> 1 p.Rotate() - cur = p.Current() - assert.Equal(t, 1, cur) - - inactive = p.Inactive() - assert.True(t, inactive.Valid()) - assert.Equal(t, 2, inactive.ID()) - - active = p.Actives() - assert.True(t, active.Valid()) - assert.Equal(t, 1, active.ID()) - active = active.Prev() - assert.True(t, active.Valid()) - assert.Equal(t, 0, active.ID()) - active = active.Prev() - assert.False(t, active.Valid()) + assert.Equal(t, 1, p.Current()) + assert.Equal(t, 2, p.Inactive()) + assert.Equal(t, []int{1, 0}, iterToSlice(p.Actives())) + // 1 -> 2 p.Rotate() - cur = p.Current() - assert.Equal(t, 2, cur) - - inactive = p.Inactive() - assert.True(t, inactive.Valid()) - assert.Equal(t, 0, inactive.ID()) - - active = p.Actives() - assert.True(t, active.Valid()) - assert.Equal(t, 2, active.ID()) - active = active.Prev() - assert.True(t, active.Valid()) - assert.Equal(t, 1, active.ID()) - active = active.Prev() - assert.False(t, active.Valid()) + assert.Equal(t, 2, p.Current()) + assert.Equal(t, 0, p.Inactive()) + assert.Equal(t, []int{2, 1}, iterToSlice(p.Actives())) + // 2 -> 0 p.Rotate() - cur = p.Current() - assert.Equal(t, 0, cur) - - inactive = p.Inactive() - assert.True(t, inactive.Valid()) - assert.Equal(t, 1, inactive.ID()) - - active = p.Actives() - assert.True(t, active.Valid()) - assert.Equal(t, 0, active.ID()) - active = active.Prev() - assert.True(t, active.Valid()) - assert.Equal(t, 2, active.ID()) - active = active.Prev() - assert.False(t, active.Valid()) + assert.Equal(t, 0, p.Current()) + assert.Equal(t, 1, p.Inactive()) + assert.Equal(t, []int{0, 2}, iterToSlice(p.Actives())) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 410e632e79c..0c2ec3b4be7 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -6,6 +6,7 @@ package eventstorage import ( "io" + "iter" "github.com/cockroachdb/pebble/v2" @@ -21,7 +22,7 @@ type db interface { type partitionedDB interface { db - ReadPartitions() PartitionIterator + ReadPartitions() iter.Seq[int] WritePartition() int } @@ -46,7 +47,7 @@ func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { return w.db.NewIter(o) } -func (w *wrappedDB) ReadPartitions() PartitionIterator { +func (w *wrappedDB) ReadPartitions() iter.Seq[int] { return w.partitioner.Actives() } From ce25f01cb380abebe1d50a3013eb004fcfa0c914 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:28:15 +0000 Subject: [PATCH 135/184] Remove superfluous err checks --- x-pack/apm-server/sampling/eventstorage/prefix.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 31d304b65fa..472d717c7c3 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -97,11 +97,7 @@ func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error if sampled { meta = entryMetaTraceSampled } - err := rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) - if err != nil { - return err - } - return nil + return rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) } func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { @@ -127,9 +123,5 @@ func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { b.WriteString(id) key := b.Bytes() - err := rw.db.Delete(key, pebble.NoSync) - if err != nil { - return err - } - return nil + return rw.db.Delete(key, pebble.NoSync) } From 7d843955c9becc2785a5c3f41e070c4d48b3b077 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:29:12 +0000 Subject: [PATCH 136/184] Add missing err check --- x-pack/apm-server/sampling/eventstorage/prefix.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 472d717c7c3..2c7500e3c1b 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -109,6 +109,8 @@ func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { item, closer, err := rw.db.Get(b.Bytes()) if err == pebble.ErrNotFound { return false, ErrNotFound + } else if err != nil { + return false, err } defer closer.Close() return item[0] == entryMetaTraceSampled, nil From 153db275f40ec6e51a07625e47c876e6ebcb628c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:34:59 +0000 Subject: [PATCH 137/184] Rename lb->b --- x-pack/apm-server/sampling/eventstorage/prefix.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 2c7500e3c1b..3be58357c63 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -38,11 +38,11 @@ type PrefixReadWriter struct { } func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - var lb bytes.Buffer - lb.Grow(1 + len(traceID) + 1) - lb.WriteByte(rw.prefix) - lb.WriteString(traceID) - lb.WriteByte(':') + var b bytes.Buffer + b.Grow(1 + len(traceID) + 1) + b.WriteByte(rw.prefix) + b.WriteString(traceID) + b.WriteByte(':') iter, err := rw.db.NewIter(&pebble.IterOptions{}) if err != nil { @@ -55,7 +55,7 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e // Memtables still need to be scanned as pebble has no bloom filter on memtables. // // SeekPrefixGE ensures the prefix is present and does not require lower bound and upper bound to be set on iterator. - if valid := iter.SeekPrefixGE(lb.Bytes()); !valid { + if valid := iter.SeekPrefixGE(b.Bytes()); !valid { return nil } for ; iter.Valid(); iter.Next() { From a5fea7bbed0c9b1f9fd806b1aadeb9e9b7d0ac89 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:39:34 +0000 Subject: [PATCH 138/184] Add comment to ReadPartitions and WritePartition --- 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 0c2ec3b4be7..f9049c02c67 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -47,10 +47,14 @@ func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { return w.db.NewIter(o) } +// ReadPartitions returns ID of the partitions that all reads should read from. +// Reads should consider all active partitions as database entries may be written at +// any point of time in the past. func (w *wrappedDB) ReadPartitions() iter.Seq[int] { return w.partitioner.Actives() } +// WritePartition returns ID of the partition that current writes should write to. func (w *wrappedDB) WritePartition() int { return w.partitioner.Current() } From cbce23dcd5cbcce8ad65680c1e62d29ce503d5b5 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:43:24 +0000 Subject: [PATCH 139/184] Fix partitioner interface failing test --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 6ca9c2c8f12..3baa2ca7678 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -221,7 +221,7 @@ func (sm *StorageManager) RotatePartitions() error { } // FIXME: potential race, wait for a bit before deleting? - pidToDelete := sm.partitioner.Inactive().ID() + pidToDelete := sm.partitioner.Inactive() lbPrefix := byte(pidToDelete) ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb return errors.Join( From 8f367be5da3b66bf5c50d18476485358c3d0583f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 15:46:04 +0000 Subject: [PATCH 140/184] Refactor Size --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 5 +++-- 1 file changed, 3 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 3baa2ca7678..34058140406 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -142,9 +142,10 @@ func (sm *StorageManager) savePartitionID(pid int) error { return sm.decisionDB.Set([]byte(partitionerMetaKey), []byte(fmt.Sprintf(`{"id":%d}`, pid)), pebble.NoSync) } -func (sm *StorageManager) Size() (lsm, vlog int64) { // FIXME: stop calling it vlog +func (sm *StorageManager) Size() (lsm, vlog int64) { + // FIXME: stop calling it vlog // FIXME: separate WAL usage? - return int64(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()), 0 + return int64(sm.DiskUsage()), 0 } func (sm *StorageManager) DiskUsage() uint64 { From bb8d553e841fcbe5eb02880063f7f8393bff4aae Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 17:02:44 +0000 Subject: [PATCH 141/184] Add comment explaining partition buffer --- .../apm-server/sampling/eventstorage/storage_manager.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 34058140406..6e9d2a8b4e9 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -77,12 +77,18 @@ type StorageManager struct { // NewStorageManager returns a new StorageManager with pebble DB at storageDir. func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*StorageManager, error) { + // We need to keep an extra partition as buffer to respect the TTL, + // as the moving window needs to cover at least TTL at all times, + // where the moving window is defined as: + // all active partitions excluding current partition + duration since the start of current partition + activePartitions := partitionsPerTTL + 1 + sm := &StorageManager{ storageDir: storageDir, runCh: make(chan struct{}, 1), logger: logp.NewLogger(logs.Sampling), codec: ProtobufCodec{}, - partitioner: NewPartitioner(partitionsPerTTL + 1), + partitioner: NewPartitioner(activePartitions), } for _, opt := range opts { opt(sm) From 8ba4af0b2b9ecb73f2073baac8712448ff756c8d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 17:27:30 +0000 Subject: [PATCH 142/184] make fmt update --- NOTICE.txt | 538 ++++++++++++++++++++++++++++++++- cmd/intake-receiver/version.go | 17 -- 2 files changed, 533 insertions(+), 22 deletions(-) diff --git a/NOTICE.txt b/NOTICE.txt index d73bec9efc2..e76365c5d76 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -73,12 +73,12 @@ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- -Dependency : github.com/cockroachdb/pebble -Version: v1.1.2 +Dependency : github.com/cockroachdb/pebble/v2 +Version: v2.0.2 Licence type (autodetected): BSD-3-Clause -------------------------------------------------------------------------------- -Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/pebble@v1.1.2/LICENSE: +Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/pebble/v2@v2.0.2/LICENSE: Copyright (c) 2011 The LevelDB-Go Authors. All rights reserved. @@ -3325,6 +3325,37 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------------------- +Dependency : github.com/stretchr/testify +Version: v1.10.0 +Licence type (autodetected): MIT +-------------------------------------------------------------------------------- + +Contents of probable licence file $GOMODCACHE/github.com/stretchr/testify@v1.10.0/LICENSE: + +MIT License + +Copyright (c) 2012-2020 Mat Ryer, Tyler Bunnell and contributors. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + + -------------------------------------------------------------------------------- Dependency : go.elastic.co/apm/module/apmelasticsearch/v2 Version: v2.6.3 @@ -6175,6 +6206,217 @@ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. +-------------------------------------------------------------------------------- +Dependency : github.com/cockroachdb/crlib +Version: v0.0.0-20241015224233-894974b3ad94 +Licence type (autodetected): Apache-2.0 +-------------------------------------------------------------------------------- + +Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/crlib@v0.0.0-20241015224233-894974b3ad94/LICENSE: + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + + -------------------------------------------------------------------------------- Dependency : github.com/cockroachdb/errors Version: v1.11.3 @@ -6808,6 +7050,43 @@ Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/logtags@v0. limitations under the License. +-------------------------------------------------------------------------------- +Dependency : github.com/cockroachdb/pebble +Version: v1.1.2 +Licence type (autodetected): BSD-3-Clause +-------------------------------------------------------------------------------- + +Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/pebble@v1.1.2/LICENSE: + +Copyright (c) 2011 The LevelDB-Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + -------------------------------------------------------------------------------- Dependency : github.com/cockroachdb/redact Version: v1.1.5 @@ -7019,6 +7298,218 @@ Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/redact@v1.1 limitations under the License. +-------------------------------------------------------------------------------- +Dependency : github.com/cockroachdb/swiss +Version: v0.0.0-20240612210725-f4de07ae6964 +Licence type (autodetected): Apache-2.0 +-------------------------------------------------------------------------------- + +Contents of probable licence file $GOMODCACHE/github.com/cockroachdb/swiss@v0.0.0-20240612210725-f4de07ae6964/LICENSE: + +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + + + -------------------------------------------------------------------------------- Dependency : github.com/cockroachdb/tokenbucket Version: v0.0.0-20230807174530-cc333fc44b06 @@ -9558,11 +10049,11 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -------------------------------------------------------------------------------- Dependency : github.com/golang/snappy -Version: v0.0.4 +Version: v0.0.5-0.20231225225746-43d5d4cd4e0e Licence type (autodetected): BSD-3-Clause -------------------------------------------------------------------------------- -Contents of probable licence file $GOMODCACHE/github.com/golang/snappy@v0.0.4/LICENSE: +Contents of probable licence file $GOMODCACHE/github.com/golang/snappy@v0.0.5-0.20231225225746-43d5d4cd4e0e/LICENSE: Copyright (c) 2011 The Snappy-Go Authors. All rights reserved. @@ -13108,6 +13599,43 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------------------- +Dependency : github.com/pmezard/go-difflib +Version: v1.0.1-0.20181226105442-5d4384ee4fb2 +Licence type (autodetected): BSD-3-Clause +-------------------------------------------------------------------------------- + +Contents of probable licence file $GOMODCACHE/github.com/pmezard/go-difflib@v1.0.1-0.20181226105442-5d4384ee4fb2/LICENSE: + +Copyright (c) 2013, Patrick Mezard +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + The names of its contributors may not be used to endorse or promote +products derived from this software without specific prior written +permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS +IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED +TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A +PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + -------------------------------------------------------------------------------- Dependency : github.com/prometheus/client_golang Version: v1.20.5 diff --git a/cmd/intake-receiver/version.go b/cmd/intake-receiver/version.go index f5d3ff4c781..d3745dda969 100644 --- a/cmd/intake-receiver/version.go +++ b/cmd/intake-receiver/version.go @@ -1,20 +1,3 @@ -// Licensed to Elasticsearch B.V. under one or more contributor -// license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright -// ownership. Elasticsearch B.V. licenses this file to you under -// the Apache License, Version 2.0 (the "License"); you may -// not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - package main // version matches the APM Server's version From 5d97b1a2d7aee4d601564ce78a25046aff495508 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 17:28:19 +0000 Subject: [PATCH 143/184] make update --- cmd/intake-receiver/version.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/cmd/intake-receiver/version.go b/cmd/intake-receiver/version.go index d3745dda969..f5d3ff4c781 100644 --- a/cmd/intake-receiver/version.go +++ b/cmd/intake-receiver/version.go @@ -1,3 +1,20 @@ +// Licensed to Elasticsearch B.V. under one or more contributor +// license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright +// ownership. Elasticsearch B.V. licenses this file to you under +// the Apache License, Version 2.0 (the "License"); you may +// not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + package main // version matches the APM Server's version From 8264339a34f08074365132e079a23cccaa5880b3 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 18:16:35 +0000 Subject: [PATCH 144/184] Workaround pebble Flush hang --- x-pack/apm-server/sampling/eventstorage/pebble.go | 6 ++++-- x-pack/apm-server/sampling/eventstorage/prefix_test.go | 6 +++++- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 4 +--- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index e1ecaf6a1e3..147b7210c6a 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -26,7 +26,8 @@ func eventComparer() *pebble.Comparer { func OpenEventPebble(storageDir string) (*pebble.DB, error) { opts := &pebble.Options{ - FormatMajorVersion: pebble.FormatColumnarBlocks, + // TODO(carsonip): update version when upstream fixes issue https://github.com/cockroachdb/pebble/issues/4287 + FormatMajorVersion: pebble.FormatPrePebblev1MarkedCompacted, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 16 << 20, Levels: []pebble.LevelOptions{ @@ -45,7 +46,8 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "decision"), &pebble.Options{ - FormatMajorVersion: pebble.FormatColumnarBlocks, + // TODO(carsonip): update version when upstream fixes issue https://github.com/cockroachdb/pebble/issues/4287 + FormatMajorVersion: pebble.FormatPrePebblev1MarkedCompacted, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 2 << 20, Levels: []pebble.LevelOptions{ diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 0cd3d7e86e6..51fb2d527f2 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -67,11 +67,15 @@ func TestPrefixReadWriter_ReadTraceEvents(t *testing.T) { require.NoError(t, err) } - // Create a transaction with a similar trace ID to ensure that iterator upper bound is enforced + // Create transactions with similar trace IDs to ensure that iterator upper bound is enforced txn := makeTransaction("bar", "foo2") err := rw.WriteTraceEvent("foo2", "bar", txn) require.NoError(t, err) + txn = makeTransaction("bar", "foo12") + err = rw.WriteTraceEvent("foo12", "bar", txn) + require.NoError(t, err) + var out modelpb.Batch err = rw.ReadTraceEvents(traceID, &out) assert.NoError(t, err) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 6e9d2a8b4e9..f9254778b03 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -171,9 +171,7 @@ func (sm *StorageManager) Close() error { } func (sm *StorageManager) close() error { - return errors.Join(sm.eventDB.Close(), sm.decisionDB.Close()) - // FIXME: there seems to be an error with pebble Flush hanging indefinitely - //return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) + return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) } // Reload flushes out pending disk writes to disk by reloading the database. From 700ca2607f9a37c964936d87e7e2beca872573ae Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 18:17:41 +0000 Subject: [PATCH 145/184] Add missing pebble cleanup in test --- x-pack/apm-server/sampling/eventstorage/prefix_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 51fb2d527f2..b54696e6a3e 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -20,12 +20,18 @@ import ( func newEventPebble(t *testing.T) *pebble.DB { db, err := eventstorage.OpenEventPebble(t.TempDir()) require.NoError(t, err) + t.Cleanup(func() { + db.Close() + }) return db } func newDecisionPebble(t *testing.T) *pebble.DB { db, err := eventstorage.OpenDecisionPebble(t.TempDir()) require.NoError(t, err) + t.Cleanup(func() { + db.Close() + }) return db } From 449da8e5c31f2fd13db1c1915d493cee61ffb2f3 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 21:51:21 +0000 Subject: [PATCH 146/184] Fix invalid pebble comparer --- .../sampling/eventstorage/pebble.go | 10 +++++++- .../sampling/eventstorage/pebble_test.go | 23 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 x-pack/apm-server/sampling/eventstorage/pebble_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 147b7210c6a..68751b5c300 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -19,7 +19,15 @@ func eventComparer() *pebble.Comparer { comparer := *pebble.DefaultComparer // Required for prefix bloom filter comparer.Split = func(k []byte) int { - return bytes.IndexByte(k, ':') + return bytes.IndexByte(k, ':') + 1 + } + comparer.Compare = func(a, b []byte) int { + ap := comparer.Split(a) // a prefix length + bp := comparer.Split(b) // b prefix length + if prefixCmp := bytes.Compare(a[:ap], b[:bp]); prefixCmp != 0 { + return prefixCmp + } + return comparer.ComparePointSuffixes(a[ap:], b[bp:]) } return &comparer } diff --git a/x-pack/apm-server/sampling/eventstorage/pebble_test.go b/x-pack/apm-server/sampling/eventstorage/pebble_test.go new file mode 100644 index 00000000000..a657ba278bc --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/pebble_test.go @@ -0,0 +1,23 @@ +package eventstorage + +import ( + "testing" + + "github.com/cockroachdb/pebble/v2" + "github.com/stretchr/testify/assert" +) + +func TestEventComparer(t *testing.T) { + err := pebble.CheckComparer(eventComparer(), [][]byte{ + nil, + []byte("foo1:"), + []byte("foo12:"), + []byte("foo2:"), + }, [][]byte{ + nil, + []byte("bar1"), + []byte("bar12"), + []byte("bar2"), + }) + assert.NoError(t, err) +} From 5d1ed19e279169f14638722500db0c50702a7bcd Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 22:56:14 +0000 Subject: [PATCH 147/184] Change trace ID separator --- .../sampling/eventstorage/partitioner.go | 7 ++++--- .../apm-server/sampling/eventstorage/pebble.go | 9 ++++----- .../sampling/eventstorage/pebble_test.go | 10 +++++++--- .../apm-server/sampling/eventstorage/prefix.go | 10 +++++++--- .../sampling/eventstorage/prefix_test.go | 4 ++-- .../sampling/eventstorage/storage_manager.go | 17 ++++++++++++----- 6 files changed, 36 insertions(+), 21 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index c49093be0a1..e7677af10fe 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -13,9 +13,10 @@ const ( // maxTotalPartitions is the maximum number of total partitions. // It is used for a sanity check specific to how we use it as a byte prefix in database keys. // It MUST be less than 256 to be contained in a byte. - // It has an additional (arbitrary) limitation to be less than reservedKeyPrefix - // to avoid accidentally overwriting reserved keys down the line. - maxTotalPartitions = int(reservedKeyPrefix) - 1 + // It has additional (arbitrary) limitations: + // - MUST be less than reservedKeyPrefix to avoid accidentally overwriting reserved keys down the line. + // - MUST be less than traceIDSeparator to avoid being misinterpreted as the separator during pebble internal key comparisons + maxTotalPartitions = int(min(reservedKeyPrefix, traceIDSeparator)) - 1 ) // Partitioner is a partitioned ring with `total` number of partitions. diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 68751b5c300..420b2bb9720 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -19,7 +19,7 @@ func eventComparer() *pebble.Comparer { comparer := *pebble.DefaultComparer // Required for prefix bloom filter comparer.Split = func(k []byte) int { - return bytes.IndexByte(k, ':') + 1 + return bytes.IndexByte(k, traceIDSeparator) + 1 } comparer.Compare = func(a, b []byte) int { ap := comparer.Split(a) // a prefix length @@ -29,13 +29,13 @@ func eventComparer() *pebble.Comparer { } return comparer.ComparePointSuffixes(a[ap:], b[bp:]) } + comparer.Name = "apmserver.EventComparer" return &comparer } func OpenEventPebble(storageDir string) (*pebble.DB, error) { opts := &pebble.Options{ - // TODO(carsonip): update version when upstream fixes issue https://github.com/cockroachdb/pebble/issues/4287 - FormatMajorVersion: pebble.FormatPrePebblev1MarkedCompacted, + FormatMajorVersion: pebble.FormatColumnarBlocks, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 16 << 20, Levels: []pebble.LevelOptions{ @@ -54,8 +54,7 @@ func OpenEventPebble(storageDir string) (*pebble.DB, error) { func OpenDecisionPebble(storageDir string) (*pebble.DB, error) { return pebble.Open(filepath.Join(storageDir, "decision"), &pebble.Options{ - // TODO(carsonip): update version when upstream fixes issue https://github.com/cockroachdb/pebble/issues/4287 - FormatMajorVersion: pebble.FormatPrePebblev1MarkedCompacted, + FormatMajorVersion: pebble.FormatColumnarBlocks, Logger: logp.NewLogger(logs.Sampling), MemTableSize: 2 << 20, Levels: []pebble.LevelOptions{ diff --git a/x-pack/apm-server/sampling/eventstorage/pebble_test.go b/x-pack/apm-server/sampling/eventstorage/pebble_test.go index a657ba278bc..5010cb470ce 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble_test.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble_test.go @@ -10,11 +10,15 @@ import ( func TestEventComparer(t *testing.T) { err := pebble.CheckComparer(eventComparer(), [][]byte{ nil, - []byte("foo1:"), - []byte("foo12:"), - []byte("foo2:"), + []byte("12!"), + []byte("123!"), + []byte("foo1!"), + []byte("foo12!"), + []byte("foo2!"), }, [][]byte{ nil, + []byte("12"), + []byte("123"), []byte("bar1"), []byte("bar12"), []byte("bar2"), diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 3be58357c63..cbe0251122b 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -19,6 +19,10 @@ const ( // over time, to avoid misinterpreting historical data. entryMetaTraceSampled byte = 's' entryMetaTraceUnsampled byte = 'u' + + // traceIDSeparator is the separator between trace ID and transaction / span ID + // It has to be smaller than characters that can be part of trace ID for pebble DeleteRange to work effectively. + traceIDSeparator byte = '!' ) var ( @@ -42,7 +46,7 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e b.Grow(1 + len(traceID) + 1) b.WriteByte(rw.prefix) b.WriteString(traceID) - b.WriteByte(':') + b.WriteByte(traceIDSeparator) iter, err := rw.db.NewIter(&pebble.IterOptions{}) if err != nil { @@ -81,7 +85,7 @@ func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.AP b.Grow(1 + len(traceID) + 1 + len(id)) b.WriteByte(rw.prefix) b.WriteString(traceID) - b.WriteByte(':') + b.WriteByte(traceIDSeparator) b.WriteString(id) key := b.Bytes() return rw.db.Set(key, data, pebble.NoSync) @@ -121,7 +125,7 @@ func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { b.Grow(1 + len(traceID) + 1 + len(id)) b.WriteByte(rw.prefix) b.WriteString(traceID) - b.WriteByte(':') + b.WriteByte(traceIDSeparator) b.WriteString(id) key := b.Bytes() diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index b54696e6a3e..6229cdc1705 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -46,7 +46,7 @@ func TestPrefixReadWriter_WriteTraceEvent(t *testing.T) { check := func() { err := rw.WriteTraceEvent(traceID, txnID, txn) assert.NoError(t, err) - item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) + item, closer, err := db.Get(append([]byte{1}, []byte("foo!bar")...)) assert.NoError(t, err) defer closer.Close() var actual modelpb.APMEvent @@ -101,7 +101,7 @@ func TestPrefixReadWriter_DeleteTraceEvent(t *testing.T) { err := rw.WriteTraceEvent(traceID, txnID, txn) require.NoError(t, err) - key := append([]byte{1}, []byte("foo:bar")...) + key := append([]byte{1}, []byte("foo!bar")...) _, closer, err := db.Get(key) assert.NoError(t, err) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index f9254778b03..9f589562cd2 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -228,12 +228,19 @@ func (sm *StorageManager) RotatePartitions() error { // FIXME: potential race, wait for a bit before deleting? pidToDelete := sm.partitioner.Inactive() lbPrefix := byte(pidToDelete) - ubPrefix := lbPrefix + 1 // Do not use % here as it MUST BE greater than lb + + decisionLb := []byte{lbPrefix} + decisionUb := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb + + // Requires the trace ID separator due to how eventComparer works + eventLb := []byte{lbPrefix, traceIDSeparator} + eventUb := []byte{lbPrefix + 1, traceIDSeparator} // Do not use % here as ub MUST BE greater than lb + return errors.Join( - sm.eventDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), - sm.decisionDB.DeleteRange([]byte{lbPrefix}, []byte{ubPrefix}, pebble.NoSync), - sm.eventDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), - sm.decisionDB.Compact([]byte{lbPrefix}, []byte{ubPrefix}, false), + sm.eventDB.DeleteRange(eventLb, eventUb, pebble.NoSync), + sm.decisionDB.DeleteRange(decisionLb, decisionUb, pebble.NoSync), + sm.eventDB.Compact(eventLb, eventUb, false), + sm.decisionDB.Compact(decisionLb, decisionUb, false), ) } From e9fc73327b8002fd17063b159260cca868d2e02a Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Fri, 24 Jan 2025 23:12:45 +0000 Subject: [PATCH 148/184] make fmt --- x-pack/apm-server/sampling/eventstorage/pebble_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble_test.go b/x-pack/apm-server/sampling/eventstorage/pebble_test.go index 5010cb470ce..cea7f1f834c 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble_test.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble_test.go @@ -1,3 +1,7 @@ +// 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 ( From b83e172b35852847a2573730ffc36ab9e234109f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Sat, 25 Jan 2025 17:03:57 +0000 Subject: [PATCH 149/184] Ensure correctness of DeleteRange --- x-pack/apm-server/sampling/eventstorage/pebble_test.go | 10 +++++----- x-pack/apm-server/sampling/eventstorage/prefix.go | 3 +-- x-pack/apm-server/sampling/eventstorage/prefix_test.go | 4 ++-- .../sampling/eventstorage/storage_manager.go | 8 +++++--- 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble_test.go b/x-pack/apm-server/sampling/eventstorage/pebble_test.go index cea7f1f834c..fe51ce4aa76 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble_test.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble_test.go @@ -14,11 +14,11 @@ import ( func TestEventComparer(t *testing.T) { err := pebble.CheckComparer(eventComparer(), [][]byte{ nil, - []byte("12!"), - []byte("123!"), - []byte("foo1!"), - []byte("foo12!"), - []byte("foo2!"), + []byte("12:"), + []byte("123:"), + []byte("foo1:"), + []byte("foo12:"), + []byte("foo2:"), }, [][]byte{ nil, []byte("12"), diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index cbe0251122b..35df65e6159 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -21,8 +21,7 @@ const ( entryMetaTraceUnsampled byte = 'u' // traceIDSeparator is the separator between trace ID and transaction / span ID - // It has to be smaller than characters that can be part of trace ID for pebble DeleteRange to work effectively. - traceIDSeparator byte = '!' + traceIDSeparator byte = ':' ) var ( diff --git a/x-pack/apm-server/sampling/eventstorage/prefix_test.go b/x-pack/apm-server/sampling/eventstorage/prefix_test.go index 6229cdc1705..b54696e6a3e 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix_test.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix_test.go @@ -46,7 +46,7 @@ func TestPrefixReadWriter_WriteTraceEvent(t *testing.T) { check := func() { err := rw.WriteTraceEvent(traceID, txnID, txn) assert.NoError(t, err) - item, closer, err := db.Get(append([]byte{1}, []byte("foo!bar")...)) + item, closer, err := db.Get(append([]byte{1}, []byte("foo:bar")...)) assert.NoError(t, err) defer closer.Close() var actual modelpb.APMEvent @@ -101,7 +101,7 @@ func TestPrefixReadWriter_DeleteTraceEvent(t *testing.T) { err := rw.WriteTraceEvent(traceID, txnID, txn) require.NoError(t, err) - key := append([]byte{1}, []byte("foo!bar")...) + key := append([]byte{1}, []byte("foo:bar")...) _, closer, err := db.Get(key) assert.NoError(t, err) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 9f589562cd2..36b24f120db 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -232,9 +232,11 @@ func (sm *StorageManager) RotatePartitions() error { decisionLb := []byte{lbPrefix} decisionUb := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb - // Requires the trace ID separator due to how eventComparer works - eventLb := []byte{lbPrefix, traceIDSeparator} - eventUb := []byte{lbPrefix + 1, traceIDSeparator} // Do not use % here as ub MUST BE greater than lb + // Add a 0 between traceID separator and prefix, as it will be theoretically the smallest valid prefix for the partition ID. + // Counter example: Imagine an eventLb without the 0, and a trace ID that has the first byte smaller than trace ID separator, + // then that entry will be smaller than eventLb and will not be deleted by DeleteRange. + eventLb := []byte{lbPrefix, 0, traceIDSeparator} + eventUb := []byte{lbPrefix + 1, 0, traceIDSeparator} // Do not use % here as ub MUST BE greater than lb return errors.Join( sm.eventDB.DeleteRange(eventLb, eventUb, pebble.NoSync), From 0e4bfcf0148a125abc65d913b5f7b98943197969 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 11:09:24 +0000 Subject: [PATCH 150/184] Simplify DeleteRange calls and remove requirement to have separator --- .../apm-server/sampling/eventstorage/pebble.go | 7 ++++++- .../sampling/eventstorage/pebble_test.go | 2 -- .../sampling/eventstorage/storage_manager.go | 18 ++++++------------ 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/pebble.go b/x-pack/apm-server/sampling/eventstorage/pebble.go index 420b2bb9720..319836c622d 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble.go @@ -19,7 +19,12 @@ func eventComparer() *pebble.Comparer { comparer := *pebble.DefaultComparer // Required for prefix bloom filter comparer.Split = func(k []byte) int { - return bytes.IndexByte(k, traceIDSeparator) + 1 + if idx := bytes.IndexByte(k, traceIDSeparator); idx != -1 { + return idx + 1 + } + // If traceID separator does not exist, consider the entire key as prefix. + // This is required for deletes like DeleteRange([]byte{0}, []byte{1}) to work without specifying the separator. + return len(k) } comparer.Compare = func(a, b []byte) int { ap := comparer.Split(a) // a prefix length diff --git a/x-pack/apm-server/sampling/eventstorage/pebble_test.go b/x-pack/apm-server/sampling/eventstorage/pebble_test.go index fe51ce4aa76..e2ec93a914b 100644 --- a/x-pack/apm-server/sampling/eventstorage/pebble_test.go +++ b/x-pack/apm-server/sampling/eventstorage/pebble_test.go @@ -13,14 +13,12 @@ import ( func TestEventComparer(t *testing.T) { err := pebble.CheckComparer(eventComparer(), [][]byte{ - nil, []byte("12:"), []byte("123:"), []byte("foo1:"), []byte("foo12:"), []byte("foo2:"), }, [][]byte{ - nil, []byte("12"), []byte("123"), []byte("bar1"), diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 36b24f120db..037eaea2d1d 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -229,20 +229,14 @@ func (sm *StorageManager) RotatePartitions() error { pidToDelete := sm.partitioner.Inactive() lbPrefix := byte(pidToDelete) - decisionLb := []byte{lbPrefix} - decisionUb := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb - - // Add a 0 between traceID separator and prefix, as it will be theoretically the smallest valid prefix for the partition ID. - // Counter example: Imagine an eventLb without the 0, and a trace ID that has the first byte smaller than trace ID separator, - // then that entry will be smaller than eventLb and will not be deleted by DeleteRange. - eventLb := []byte{lbPrefix, 0, traceIDSeparator} - eventUb := []byte{lbPrefix + 1, 0, traceIDSeparator} // Do not use % here as ub MUST BE greater than lb + lb := []byte{lbPrefix} + ub := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb return errors.Join( - sm.eventDB.DeleteRange(eventLb, eventUb, pebble.NoSync), - sm.decisionDB.DeleteRange(decisionLb, decisionUb, pebble.NoSync), - sm.eventDB.Compact(eventLb, eventUb, false), - sm.decisionDB.Compact(decisionLb, decisionUb, false), + sm.eventDB.DeleteRange(lb, ub, pebble.NoSync), + sm.decisionDB.DeleteRange(lb, ub, pebble.NoSync), + sm.eventDB.Compact(lb, ub, false), + sm.decisionDB.Compact(lb, ub, false), ) } From 00b3c15fc72fffa315d554f59c46fc532127592c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 14:37:17 +0000 Subject: [PATCH 151/184] Remove BypassReadWriter --- .../eventstorage/storage_bench_test.go | 17 +++++---------- .../sampling/eventstorage/storage_manager.go | 10 --------- .../eventstorage/storage_manager_test.go | 4 ++-- x-pack/apm-server/sampling/processor_test.go | 21 +++++++------------ 4 files changed, 14 insertions(+), 38 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go index f03019a69b1..fa464df323b 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_bench_test.go @@ -18,8 +18,7 @@ import ( func BenchmarkWriteTransaction(b *testing.B) { test := func(b *testing.B, codec eventstorage.Codec, bigTX bool) { sm := newStorageManager(b, eventstorage.WithCodec(codec)) - readWriter := sm.NewBypassReadWriter() - defer readWriter.Close() + readWriter := sm.NewReadWriter() traceID := hex.EncodeToString([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}) transactionID := hex.EncodeToString([]byte{1, 2, 3, 4, 5, 6, 7, 8}) @@ -80,8 +79,7 @@ func BenchmarkReadEvents(b *testing.B) { for _, count := range counts { b.Run(fmt.Sprintf("%d events", count), func(b *testing.B) { sm := newStorageManager(b, eventstorage.WithCodec(codec)) - readWriter := sm.NewBypassReadWriter() - defer readWriter.Close() + readWriter := sm.NewReadWriter() for i := 0; i < count; i++ { transactionID := uuid.Must(uuid.NewV4()).String() @@ -156,8 +154,7 @@ func BenchmarkReadEventsHit(b *testing.B) { for _, hit := range []bool{false, true} { b.Run(fmt.Sprintf("hit=%v", hit), func(b *testing.B) { sm := newStorageManager(b) - readWriter := sm.NewBypassReadWriter() - defer readWriter.Close() + readWriter := sm.NewReadWriter() traceIDs := make([]string, b.N) @@ -182,16 +179,13 @@ func BenchmarkReadEventsHit(b *testing.B) { } } - readWriter.Close() - if reloadDB { if err := sm.Reload(); err != nil { b.Fatal(err) } } - readWriter = sm.NewBypassReadWriter() - defer readWriter.Close() + readWriter = sm.NewReadWriter() b.ResetTimer() var batch modelpb.Batch @@ -230,8 +224,7 @@ func BenchmarkIsTraceSampled(b *testing.B) { // Test with varying numbers of events in the trace. sm := newStorageManager(b) - readWriter := sm.NewBypassReadWriter() - defer readWriter.Close() + readWriter := sm.NewReadWriter() if err := readWriter.WriteTraceSampled(sampledTraceUUID.String(), true); err != nil { b.Fatal(err) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 037eaea2d1d..a1c44bc441c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -258,13 +258,3 @@ func (sm *StorageManager) NewReadWriter() StorageLimitReadWriter { decisionRW: sm.decisionStorage.NewReadWriter(), }) } - -// NewBypassReadWriter returns a SplitReadWriter directly reading and writing to the database, -// bypassing any wrappers. -// For testing only. Useful for checking if data is actually persisted to the DB. -func (sm *StorageManager) NewBypassReadWriter() SplitReadWriter { - return SplitReadWriter{ - eventRW: sm.eventStorage.NewReadWriter(), - decisionRW: sm.decisionStorage.NewReadWriter(), - } -} diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 7205f39f295..ac77d3c63bc 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -30,7 +30,7 @@ func newStorageManagerNoCleanup(tb testing.TB, path string, opts ...eventstorage func TestStorageManager_samplingDecisionTTL(t *testing.T) { sm := newStorageManager(t) - rw := sm.NewBypassReadWriter() + rw := sm.NewReadWriter() traceID := uuid.Must(uuid.NewV4()).String() err := rw.WriteTraceSampled(traceID, true) assert.NoError(t, err) @@ -63,7 +63,7 @@ func TestStorageManager_samplingDecisionTTL(t *testing.T) { func TestStorageManager_eventTTL(t *testing.T) { sm := newStorageManager(t) - rw := sm.NewBypassReadWriter() + rw := sm.NewReadWriter() traceID := uuid.Must(uuid.NewV4()).String() txnID := uuid.Must(uuid.NewV4()).String() transaction := makeTransaction(txnID, traceID) diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index dbcb3c2e15e..942a55fdd92 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -59,17 +59,15 @@ func TestProcessAlreadyTailSampled(t *testing.T) { // subsequent events in the trace will be reported immediately. trace1 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f10"} trace2 := modelpb.Trace{Id: "0102030405060708090a0b0c0d0e0f11"} - writer := config.DB.NewBypassReadWriter() + writer := config.DB.NewReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace2.Id, true)) - writer.Close() // simulate 2 TTL assert.NoError(t, config.DB.RotatePartitions()) assert.NoError(t, config.DB.RotatePartitions()) - writer = config.DB.NewBypassReadWriter() + writer = config.DB.NewReadWriter() assert.NoError(t, writer.WriteTraceSampled(trace1.Id, true)) - writer.Close() require.NoError(t, config.DB.Flush()) @@ -130,8 +128,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { // Stop the processor and flush global storage so we can access the database. assert.NoError(t, processor.Stop(context.Background())) assert.NoError(t, config.DB.Flush()) - reader := config.DB.NewBypassReadWriter() - defer reader.Close() + reader := config.DB.NewReadWriter() batch = nil err = reader.ReadTraceEvents(trace1.Id, &batch) @@ -249,8 +246,7 @@ func TestProcessLocalTailSampling(t *testing.T) { // Stop the processor and flush global storage so we can access the database. assert.NoError(t, processor.Stop(context.Background())) assert.NoError(t, config.DB.Flush()) - reader := config.DB.NewBypassReadWriter() - defer reader.Close() + reader := config.DB.NewReadWriter() sampled, err := reader.IsTraceSampled(sampledTraceID) assert.NoError(t, err) @@ -313,8 +309,7 @@ func TestProcessLocalTailSamplingUnsampled(t *testing.T) { // Stop the processor so we can access the database. assert.NoError(t, processor.Stop(context.Background())) assert.NoError(t, config.DB.Flush()) - reader := config.DB.NewBypassReadWriter() - defer reader.Close() + reader := config.DB.NewReadWriter() var anyUnsampled bool for _, traceID := range traceIDs { @@ -479,8 +474,7 @@ func TestProcessRemoteTailSampling(t *testing.T) { assert.Empty(t, cmp.Diff(trace1Events, events, protocmp.Transform())) - reader := config.DB.NewBypassReadWriter() - defer reader.Close() + reader := config.DB.NewReadWriter() sampled, err := reader.IsTraceSampled(traceID1) assert.NoError(t, err) @@ -743,8 +737,7 @@ func TestGracefulShutdown(t *testing.T) { assert.Empty(t, batch) assert.NoError(t, processor.Stop(context.Background())) - reader := config.DB.NewBypassReadWriter() - defer reader.Close() + reader := config.DB.NewReadWriter() var count int for i := 0; i < totalTraces; i++ { From 36c29d9e18cbd966e9e033fc2f85b6683ac7de3b Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 14:54:06 +0000 Subject: [PATCH 152/184] Update IsTraceSampled comment --- x-pack/apm-server/sampling/eventstorage/partition_rw.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 8aa26f99255..ea14fb3fa93 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -24,11 +24,12 @@ func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) e // IsTraceSampled reports whether traceID belongs to a trace that is sampled // or unsampled. If no sampling decision has been recorded, IsTraceSampled // returns ErrNotFound. +// +// The performance of IsTraceSampled is crucial since it is in the hot path. +// It is called +// 1. when a remote sampling decision is received from pubsub +// 2. (hot path) when a transaction / span comes in, check if a sampling decision has already been made func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { - // 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 var errs []error for pid := range rw.s.db.ReadPartitions() { sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) From bb9ee605c231e7f0da21153483499cb14e9e6a47 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 16:57:07 +0000 Subject: [PATCH 153/184] Use RWMutex instead of atomics --- .../sampling/eventstorage/partition_rw.go | 20 +++++-- .../sampling/eventstorage/partitioner.go | 30 +++++------ .../sampling/eventstorage/partitioner_test.go | 22 +++++--- .../sampling/eventstorage/storage.go | 52 +++---------------- .../sampling/eventstorage/storage_manager.go | 42 ++++++++------- 5 files changed, 74 insertions(+), 92 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index ea14fb3fa93..65c6ddbba8d 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -17,7 +17,9 @@ type PartitionReadWriter struct { // WriteTraceSampled records the tail-sampling decision for the given trace ID. func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { - pid := rw.s.db.WritePartition() + rw.s.partitioner.mu.RLock() + defer rw.s.partitioner.mu.RUnlock() + pid := rw.s.partitioner.Current() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) } @@ -30,8 +32,10 @@ func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) e // 1. when a remote sampling decision is received from pubsub // 2. (hot path) when a transaction / span comes in, check if a sampling decision has already been made func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { + rw.s.partitioner.mu.RLock() + defer rw.s.partitioner.mu.RUnlock() var errs []error - for pid := range rw.s.db.ReadPartitions() { + for pid := range rw.s.partitioner.Actives() { sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil @@ -47,15 +51,19 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { // WriteTraceEvent writes a trace event to storage. func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { - pid := rw.s.db.WritePartition() + rw.s.partitioner.mu.RLock() + defer rw.s.partitioner.mu.RUnlock() + pid := rw.s.partitioner.Current() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) } // DeleteTraceEvent deletes the trace event from storage. func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { + rw.s.partitioner.mu.RLock() + defer rw.s.partitioner.mu.RUnlock() // FIXME: use range delete var errs []error - for pid := range rw.s.db.ReadPartitions() { + for pid := range rw.s.partitioner.Actives() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) if err != nil { errs = append(errs, err) @@ -66,8 +74,10 @@ func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { + rw.s.partitioner.mu.RLock() + defer rw.s.partitioner.mu.RUnlock() var errs []error - for pid := range rw.s.db.ReadPartitions() { + for pid := range rw.s.partitioner.Actives() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).ReadTraceEvents(traceID, out) if err != nil { errs = append(errs, err) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index e7677af10fe..1e9e63bd149 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -6,7 +6,7 @@ package eventstorage import ( "iter" - "sync/atomic" + "sync" ) const ( @@ -30,21 +30,17 @@ const ( // current type Partitioner struct { total int // length of the ring - current atomic.Int32 + current int + mu sync.RWMutex } // NewPartitioner returns a partitioner with `actives` number of active partitions. -func NewPartitioner(actives int) *Partitioner { +func NewPartitioner(actives, currentID int) *Partitioner { total := actives + 1 // actives + 1 inactive if total >= maxTotalPartitions { panic("too many partitions") } - return &Partitioner{total: total} -} - -// SetCurrentID sets the input partition ID as current partition. -func (p *Partitioner) SetCurrentID(current int) { - p.current.Store(int32(current)) + return &Partitioner{total: total, current: currentID} } // Rotate rotates partitions to the right by 1 position and @@ -59,15 +55,17 @@ func (p *Partitioner) SetCurrentID(current int) { // A-A-I-A // ..^.... func (p *Partitioner) Rotate() int { - newCurrent := (int(p.current.Load()) + 1) % p.total - p.current.Store(int32(newCurrent)) - return newCurrent + p.mu.Lock() + defer p.mu.Unlock() + p.current = (p.current + 1) % p.total + return p.current } // Actives returns an iterator containing all active partitions. // It contains total - 1 partitions. +// Callers should obtain p.mu.RLock when using the returned PIDs. func (p *Partitioner) Actives() iter.Seq[int] { - cur := int(p.current.Load()) + cur := p.current return func(yield func(int) bool) { for i := 0; i < p.total-1; i++ { if !yield((cur + p.total - i) % p.total) { @@ -78,11 +76,13 @@ func (p *Partitioner) Actives() iter.Seq[int] { } // Inactive returns the ID of the inactive partition. +// Callers should obtain p.mu.RLock when using the returned PID. func (p *Partitioner) Inactive() int { - return (int(p.current.Load()) + 1) % p.total + return (p.current + 1) % p.total } // Current returns the ID of the current partition (rightmost active). +// Callers should obtain p.mu.RLock when using the returned PID. func (p *Partitioner) Current() int { - return int(p.current.Load()) + return p.current } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index c1ad555d575..4f302d47085 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -13,15 +13,15 @@ import ( "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) -func TestPartitioner(t *testing.T) { - iterToSlice := func(it iter.Seq[int]) (s []int) { - for i := range it { - s = append(s, i) - } - return +func iterToSlice[T any](it iter.Seq[T]) (s []T) { + for i := range it { + s = append(s, i) } + return +} - p := eventstorage.NewPartitioner(2) // partition id 0, 1, 2 +func TestPartitioner(t *testing.T) { + p := eventstorage.NewPartitioner(2, 0) // partition id 0, 1, 2 assert.Equal(t, 0, p.Current()) assert.Equal(t, 1, p.Inactive()) @@ -48,3 +48,11 @@ func TestPartitioner(t *testing.T) { assert.Equal(t, 1, p.Inactive()) assert.Equal(t, []int{0, 2}, iterToSlice(p.Actives())) } + +func TestPartitionerCurrentID(t *testing.T) { + p := eventstorage.NewPartitioner(2, 1) + + assert.Equal(t, 1, p.Current()) + assert.Equal(t, 2, p.Inactive()) + assert.Equal(t, []int{1, 0}, iterToSlice(p.Actives())) +} diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index f9049c02c67..47f0b6fb59e 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -6,7 +6,6 @@ package eventstorage import ( "io" - "iter" "github.com/cockroachdb/pebble/v2" @@ -20,50 +19,12 @@ type db interface { NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) } -type partitionedDB interface { - db - ReadPartitions() iter.Seq[int] - WritePartition() int -} - -type wrappedDB struct { - partitioner *Partitioner - db *pebble.DB -} - -func (w *wrappedDB) Get(key []byte) ([]byte, io.Closer, error) { - return w.db.Get(key) -} - -func (w *wrappedDB) Set(key, value []byte, opts *pebble.WriteOptions) error { - return w.db.Set(key, value, opts) -} - -func (w *wrappedDB) Delete(key []byte, opts *pebble.WriteOptions) error { - return w.db.Delete(key, opts) -} - -func (w *wrappedDB) NewIter(o *pebble.IterOptions) (*pebble.Iterator, error) { - return w.db.NewIter(o) -} - -// ReadPartitions returns ID of the partitions that all reads should read from. -// Reads should consider all active partitions as database entries may be written at -// any point of time in the past. -func (w *wrappedDB) ReadPartitions() iter.Seq[int] { - return w.partitioner.Actives() -} - -// WritePartition returns ID of the partition that current writes should write to. -func (w *wrappedDB) WritePartition() int { - return w.partitioner.Current() -} - // Storage provides storage for sampled transactions and spans, // and for recording trace sampling decisions. type Storage struct { - db partitionedDB - codec Codec + db db + partitioner *Partitioner + codec Codec } // Codec provides methods for encoding and decoding events. @@ -73,10 +34,11 @@ type Codec interface { } // New returns a new Storage using db and codec. -func New(db partitionedDB, codec Codec) *Storage { +func New(db db, partitioner *Partitioner, codec Codec) *Storage { return &Storage{ - db: db, - codec: codec, + db: db, + partitioner: partitioner, + codec: codec, } } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index a1c44bc441c..39ac8814ae1 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -77,18 +77,11 @@ type StorageManager struct { // NewStorageManager returns a new StorageManager with pebble DB at storageDir. func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*StorageManager, error) { - // We need to keep an extra partition as buffer to respect the TTL, - // as the moving window needs to cover at least TTL at all times, - // where the moving window is defined as: - // all active partitions excluding current partition + duration since the start of current partition - activePartitions := partitionsPerTTL + 1 - sm := &StorageManager{ - storageDir: storageDir, - runCh: make(chan struct{}, 1), - logger: logp.NewLogger(logs.Sampling), - codec: ProtobufCodec{}, - partitioner: NewPartitioner(activePartitions), + storageDir: storageDir, + runCh: make(chan struct{}, 1), + logger: logp.NewLogger(logs.Sampling), + codec: ProtobufCodec{}, } for _, opt := range opts { opt(sm) @@ -98,12 +91,6 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora return nil, err } - if pid, err := sm.loadPartitionID(); err != nil { - sm.logger.With(logp.Error(err)).Warn("failed to load partition ID") - } else { - sm.partitioner.SetCurrentID(pid) - } - return sm, nil } @@ -114,14 +101,29 @@ func (sm *StorageManager) reset() error { return err } sm.eventDB = eventDB - sm.eventStorage = New(&wrappedDB{partitioner: sm.partitioner, db: sm.eventDB}, sm.codec) decisionDB, err := OpenDecisionPebble(sm.storageDir) if err != nil { return err } sm.decisionDB = decisionDB - sm.decisionStorage = New(&wrappedDB{partitioner: sm.partitioner, db: sm.decisionDB}, sm.codec) + + // Only recreate partitioner on initial create + if sm.partitioner == nil { + var currentPID int + if currentPID, err = sm.loadPartitionID(); err != nil { + sm.logger.With(logp.Error(err)).Warn("failed to load partition ID") + } + // We need to keep an extra partition as buffer to respect the TTL, + // as the moving window needs to cover at least TTL at all times, + // where the moving window is defined as: + // all active partitions excluding current partition + duration since the start of current partition + activePartitions := partitionsPerTTL + 1 + sm.partitioner = NewPartitioner(activePartitions, currentPID) + } + + sm.eventStorage = New(sm.eventDB, sm.partitioner, sm.codec) + sm.decisionStorage = New(sm.decisionDB, sm.partitioner, sm.codec) return nil } @@ -225,7 +227,7 @@ func (sm *StorageManager) RotatePartitions() error { return err } - // FIXME: potential race, wait for a bit before deleting? + // No lock is needed here as the only writer to sm.partitioner is exactly this function. pidToDelete := sm.partitioner.Inactive() lbPrefix := byte(pidToDelete) From c84d04db78fa0126af8728f98f42b45b7d7a6468 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 17:05:41 +0000 Subject: [PATCH 154/184] Add comment --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 39ac8814ae1..fa1c43a8b16 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -201,6 +201,8 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, stora return sm.runTTLGCLoop(stopping, ttl) } +// runTTLGCLoop runs the TTL GC loop. +// The loop triggers a rotation on partitions at an interval based on ttl and partitionsPerTTL. func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Duration) error { ttlGCInterval := ttl / partitionsPerTTL ticker := time.NewTicker(ttlGCInterval) From ed023570e1e1944fc644584cb2d234c81f6c4418 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 17:15:20 +0000 Subject: [PATCH 155/184] Update comments --- x-pack/apm-server/sampling/eventstorage/rw.go | 1 + x-pack/apm-server/sampling/eventstorage/storage_manager.go | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 7f25e4cf88b..858474de367 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -73,6 +73,7 @@ func NewStorageLimitReadWriter(checker storageLimitChecker, nextRW RW) StorageLi func (s StorageLimitReadWriter) checkStorageLimit() error { usage := s.checker.DiskUsage() limit := s.checker.StorageLimit() + // FIXME: if limit is 0, skip DiskUsage call if limit != 0 && usage >= limit { return fmt.Errorf("%w (current: %d, limit %d)", ErrLimitReached, usage, limit) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index fa1c43a8b16..9285861b08d 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -151,12 +151,14 @@ func (sm *StorageManager) savePartitionID(pid int) error { } func (sm *StorageManager) Size() (lsm, vlog int64) { - // FIXME: stop calling it vlog - // FIXME: separate WAL usage? + // TODO(carsonip): this is reporting lsm and vlog for legacy reasons. + // vlog is always 0 because pebble does not have a vlog. + // Update this to report a more helpful size to monitoring. return int64(sm.DiskUsage()), 0 } func (sm *StorageManager) DiskUsage() uint64 { + // FIXME: measure overhead return sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage() } From 72c247f817ff7328f2f1645581481135abf341b8 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 17:26:13 +0000 Subject: [PATCH 156/184] Add TODO --- x-pack/apm-server/sampling/eventstorage/partition_rw.go | 1 - x-pack/apm-server/sampling/processor.go | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 65c6ddbba8d..67686725146 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -61,7 +61,6 @@ func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelp func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() - // FIXME: use range delete var errs []error for pid := range rw.s.partitioner.Actives() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 457a6878ab9..361dcc79787 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -501,6 +501,10 @@ func (p *Processor) Run() error { // deleted. We delete events from local storage so // we don't publish duplicates; delivery is therefore // at-most-once, not guaranteed. + // + // TODO(carsonip): pebble supports range deletes and may be better than + // deleting events separately, but as we do not use transactions, it is + // possible to race and delete something that is not read. for _, event := range events { switch event.Type() { case modelpb.TransactionEventType: From 2224b41994463cc9cc608ef51fde661b48d87efc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 17:48:50 +0000 Subject: [PATCH 157/184] Turn FIXME into TODO --- systemtest/benchtest/expvar/metrics.go | 2 +- .../sampling/eventstorage/storage_manager.go | 11 +++++++++-- x-pack/apm-server/sampling/processor.go | 2 +- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/systemtest/benchtest/expvar/metrics.go b/systemtest/benchtest/expvar/metrics.go index 0a50f617830..4bff808ce40 100644 --- a/systemtest/benchtest/expvar/metrics.go +++ b/systemtest/benchtest/expvar/metrics.go @@ -173,7 +173,7 @@ func (c *Collector) accumulate(e expvar) { c.processMetric(HeapAlloc, int64(e.HeapAlloc)) c.processMetric(HeapObjects, int64(e.HeapObjects)) c.processMetric(TBSLsmSize, e.TBSLsmSize) - c.processMetric(TBSVlogSize, e.TBSVlogSize) // FIXME: update to use new metrics + c.processMetric(TBSVlogSize, e.TBSVlogSize) } func (c *Collector) processMetric(m Metric, val int64) { diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 9285861b08d..bb2bd378bdd 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -151,9 +151,16 @@ func (sm *StorageManager) savePartitionID(pid int) error { } func (sm *StorageManager) Size() (lsm, vlog int64) { - // TODO(carsonip): this is reporting lsm and vlog for legacy reasons. + // This is reporting lsm and vlog for legacy reasons. // vlog is always 0 because pebble does not have a vlog. - // Update this to report a more helpful size to monitoring. + // Keeping this legacy structure such that the metrics are comparable across versions, + // and we don't need to update the tooling, e.g. kibana dashboards. + // + // TODO(carsonip): Update this to report a more helpful size to monitoring, + // maybe broken down into event DB vs decision DB, and LSM tree vs WAL vs misc. + // Also remember to update + // - x-pack/apm-server/sampling/processor.go:CollectMonitoring + // - systemtest/benchtest/expvar/metrics.go return int64(sm.DiskUsage()), 0 } diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 361dcc79787..a565851a856 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -107,7 +107,7 @@ func (p *Processor) CollectMonitoring(_ monitoring.Mode, V monitoring.Visitor) { 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)) // FIXME: either report everything under lsm_size, or report with new fine-grained metrics, e.g. wal_size + monitoring.ReportInt(V, "value_log_size", int64(valueLogSize)) }) monitoring.ReportNamespace(V, "events", func() { monitoring.ReportInt(V, "processed", atomic.LoadInt64(&p.eventMetrics.processed)) From d25aa45ee95b3d5ad0675330f2834f7c7806cf33 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 18:00:48 +0000 Subject: [PATCH 158/184] Update storage manager read test --- .../sampling/eventstorage/storage_manager_test.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index ac77d3c63bc..7e94dd4346c 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -65,9 +65,9 @@ func TestStorageManager_eventTTL(t *testing.T) { sm := newStorageManager(t) rw := sm.NewReadWriter() traceID := uuid.Must(uuid.NewV4()).String() - txnID := uuid.Must(uuid.NewV4()).String() - transaction := makeTransaction(txnID, traceID) - err := rw.WriteTraceEvent(traceID, txnID, transaction) + txnID1 := uuid.Must(uuid.NewV4()).String() + txn1 := makeTransaction(txnID1, traceID) + err := rw.WriteTraceEvent(traceID, txnID1, txn1) assert.NoError(t, err) var out modelpb.Batch @@ -79,10 +79,15 @@ func TestStorageManager_eventTTL(t *testing.T) { err = sm.RotatePartitions() assert.NoError(t, err) + txnID2 := uuid.Must(uuid.NewV4()).String() + txn2 := makeTransaction(txnID2, traceID) + err = rw.WriteTraceEvent(traceID, txnID2, txn2) + assert.NoError(t, err) + out = nil err = rw.ReadTraceEvents(traceID, &out) assert.NoError(t, err) - assert.Len(t, out, 1) + assert.Equal(t, modelpb.Batch{txn2, txn1}, out) // after 2 TTL err = sm.RotatePartitions() @@ -91,7 +96,7 @@ func TestStorageManager_eventTTL(t *testing.T) { out = nil err = rw.ReadTraceEvents(traceID, &out) assert.NoError(t, err) - assert.Len(t, out, 0) + assert.Equal(t, modelpb.Batch{txn2}, out) // after 3 TTL err = sm.RotatePartitions() From 7006e17539fceb876830859cf93259a9f329a012 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 20:47:41 +0000 Subject: [PATCH 159/184] Add DiskUsage bench test --- .../storage_manager_bench_test.go | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go new file mode 100644 index 00000000000..f4e16690b7a --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go @@ -0,0 +1,30 @@ +// 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_test + +import ( + "testing" + + "github.com/gofrs/uuid/v5" + "github.com/stretchr/testify/require" +) + +func BenchmarkStorageManager_DiskUsage(b *testing.B) { + sm := newStorageManager(b) + rw := sm.NewReadWriter() + for i := 0; i < 1000; i++ { + traceID := uuid.Must(uuid.NewV4()).String() + txnID := uuid.Must(uuid.NewV4()).String() + txn := makeTransaction(txnID, traceID) + err := rw.WriteTraceEvent(traceID, txnID, txn) + require.NoError(b, err) + err = rw.WriteTraceSampled(traceID, true) + require.NoError(b, err) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + _ = sm.DiskUsage() + } +} From 1e066d826e646280414ab1a76f320b640a87b734 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 23:38:55 +0000 Subject: [PATCH 160/184] Cache disk usage --- .../sampling/eventstorage/storage_manager.go | 42 +++++++++++++++++-- .../storage_manager_bench_test.go | 5 +++ .../eventstorage/storage_manager_test.go | 32 ++++++++++++++ 3 files changed, 76 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index bb2bd378bdd..840c619afc9 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/pebble/v2" + "golang.org/x/sync/errgroup" "github.com/elastic/apm-server/internal/logs" "github.com/elastic/elastic-agent-libs/logp" @@ -40,6 +41,9 @@ const ( // partitionerMetaKey is the key used to store partitioner metadata, e.g. last partition ID, in decision DB. partitionerMetaKey = string(reservedKeyPrefix) + "partitioner" + + // diskUsageFetchInterval is how often disk usage is fetched which is equivalent to how long disk usage is cached. + diskUsageFetchInterval = 1 * time.Second ) type StorageManagerOptions func(*StorageManager) @@ -70,6 +74,9 @@ type StorageManager struct { // subscriberPosMu protects the subscriber file from concurrent RW. subscriberPosMu sync.Mutex + // cachedDiskUsage is a cached result of DiskUsage + cachedDiskUsage atomic.Uint64 + // runCh acts as a mutex to ensure only 1 Run is actively running per StorageManager. // as it is possible that 2 separate Run are created by 2 TBS processors during a hot reload. runCh chan struct{} @@ -125,6 +132,8 @@ func (sm *StorageManager) reset() error { sm.eventStorage = New(sm.eventDB, sm.partitioner, sm.codec) sm.decisionStorage = New(sm.decisionDB, sm.partitioner, sm.codec) + sm.updateDiskUsage() + return nil } @@ -164,9 +173,28 @@ func (sm *StorageManager) Size() (lsm, vlog int64) { return int64(sm.DiskUsage()), 0 } +// DiskUsage returns the disk usage of databases in bytes. func (sm *StorageManager) DiskUsage() uint64 { - // FIXME: measure overhead - return sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage() + // pebble DiskSpaceUsage overhead is not high, but it adds up when performed per-event. + return sm.cachedDiskUsage.Load() +} + +func (sm *StorageManager) updateDiskUsage() { + sm.cachedDiskUsage.Store(sm.eventDB.Metrics().DiskSpaceUsage() + sm.decisionDB.Metrics().DiskSpaceUsage()) +} + +// runDiskUsageLoop runs a loop that updates cached disk usage regularly. +func (sm *StorageManager) runDiskUsageLoop(stopping <-chan struct{}) error { + ticker := time.NewTicker(diskUsageFetchInterval) + defer ticker.Stop() + for { + select { + case <-stopping: + return nil + case <-ticker.C: + sm.updateDiskUsage() + } + } } func (sm *StorageManager) StorageLimit() uint64 { @@ -207,7 +235,15 @@ func (sm *StorageManager) Run(stopping <-chan struct{}, ttl time.Duration, stora sm.storageLimit.Store(storageLimit) - return sm.runTTLGCLoop(stopping, ttl) + g := errgroup.Group{} + g.Go(func() error { + return sm.runTTLGCLoop(stopping, ttl) + }) + g.Go(func() error { + return sm.runDiskUsageLoop(stopping) + }) + + return g.Wait() } // runTTLGCLoop runs the TTL GC loop. diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go index f4e16690b7a..86ebc1d5430 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_bench_test.go @@ -6,13 +6,17 @@ package eventstorage_test import ( "testing" + "time" "github.com/gofrs/uuid/v5" "github.com/stretchr/testify/require" ) func BenchmarkStorageManager_DiskUsage(b *testing.B) { + stopping := make(chan struct{}) + defer close(stopping) sm := newStorageManager(b) + go sm.Run(stopping, time.Second, 0) rw := sm.NewReadWriter() for i := 0; i < 1000; i++ { traceID := uuid.Must(uuid.NewV4()).String() @@ -27,4 +31,5 @@ func BenchmarkStorageManager_DiskUsage(b *testing.B) { for i := 0; i < b.N; i++ { _ = sm.DiskUsage() } + b.StopTimer() } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 7e94dd4346c..4a8cb34f780 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -6,9 +6,11 @@ package eventstorage_test import ( "testing" + "time" "github.com/gofrs/uuid/v5" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/elastic/apm-data/model/modelpb" "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" @@ -129,3 +131,33 @@ func TestStorageManager_partitionID(t *testing.T) { assert.NoError(t, err) assert.True(t, sampled) } + +func TestStorageManager_DiskUsage(t *testing.T) { + stopping := make(chan struct{}) + defer close(stopping) + sm := newStorageManager(t) + go sm.Run(stopping, time.Second, 0) + old := sm.DiskUsage() + + err := sm.NewReadWriter().WriteTraceSampled("foo", true) + require.NoError(t, err) + + err = sm.Flush() + require.NoError(t, err) + + assert.Eventually(t, func() bool { + return sm.DiskUsage() > old + }, 10*time.Second, 100*time.Millisecond) + + old = sm.DiskUsage() + + err = sm.NewReadWriter().WriteTraceEvent("foo", "bar", makeTransaction("bar", "foo")) + require.NoError(t, err) + + err = sm.Flush() + require.NoError(t, err) + + assert.Eventually(t, func() bool { + return sm.DiskUsage() > old + }, 10*time.Second, 100*time.Millisecond) +} From 2535e6c932c0bde401da1734c74d4de1b91d01be Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 23:41:10 +0000 Subject: [PATCH 161/184] Skip check storage limit if there is no limit --- x-pack/apm-server/sampling/eventstorage/rw.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 858474de367..e42ef8ce345 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -71,11 +71,12 @@ func NewStorageLimitReadWriter(checker storageLimitChecker, nextRW RW) StorageLi } func (s StorageLimitReadWriter) checkStorageLimit() error { - usage := s.checker.DiskUsage() limit := s.checker.StorageLimit() - // FIXME: if limit is 0, skip DiskUsage call - if limit != 0 && usage >= limit { - return fmt.Errorf("%w (current: %d, limit %d)", ErrLimitReached, usage, limit) + if limit != 0 { + usage := s.checker.DiskUsage() + if usage >= limit { + return fmt.Errorf("%w (current: %d, limit %d)", ErrLimitReached, usage, limit) + } } return nil } From c27732f62757d220858c4b7a0e820a664066f42f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 23:44:18 +0000 Subject: [PATCH 162/184] Test storage manager run --- .../sampling/eventstorage/storage_manager_test.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go index 4a8cb34f780..1145a9d8fe4 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager_test.go @@ -161,3 +161,15 @@ func TestStorageManager_DiskUsage(t *testing.T) { return sm.DiskUsage() > old }, 10*time.Second, 100*time.Millisecond) } + +func TestStorageManager_Run(t *testing.T) { + done := make(chan struct{}) + stopping := make(chan struct{}) + sm := newStorageManager(t) + go func() { + assert.NoError(t, sm.Run(stopping, time.Second, 0)) + close(done) + }() + close(stopping) + <-done +} From 058d654e7524062013f0efe830efec7fb086f63f Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Mon, 27 Jan 2025 23:59:56 +0000 Subject: [PATCH 163/184] Better errors --- .../sampling/eventstorage/storage_manager.go | 40 +++++++++++++------ 1 file changed, 28 insertions(+), 12 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 840c619afc9..2d806a0bc84 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -93,9 +93,9 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora for _, opt := range opts { opt(sm) } - err := sm.reset() - if err != nil { - return nil, err + + if err := sm.reset(); err != nil { + return nil, fmt.Errorf("storage manager reset error: %w", err) } return sm, nil @@ -105,13 +105,13 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora func (sm *StorageManager) reset() error { eventDB, err := OpenEventPebble(sm.storageDir) if err != nil { - return err + return fmt.Errorf("open event db error: %w", err) } sm.eventDB = eventDB decisionDB, err := OpenDecisionPebble(sm.storageDir) if err != nil { - return err + return fmt.Errorf("open decision db error: %w", err) } sm.decisionDB = decisionDB @@ -119,7 +119,7 @@ func (sm *StorageManager) reset() error { if sm.partitioner == nil { var currentPID int if currentPID, err = sm.loadPartitionID(); err != nil { - sm.logger.With(logp.Error(err)).Warn("failed to load partition ID") + sm.logger.With(logp.Error(err)).Warn("failed to load partition ID, using 0 instead") } // We need to keep an extra partition as buffer to respect the TTL, // as the moving window needs to cover at least TTL at all times, @@ -202,7 +202,10 @@ func (sm *StorageManager) StorageLimit() uint64 { } func (sm *StorageManager) Flush() error { - return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush()) + return errors.Join( + wrapNonNilErr("event db flush error: %w", sm.eventDB.Flush()), + wrapNonNilErr("decision db flush error: %w", sm.decisionDB.Flush()), + ) } func (sm *StorageManager) Close() error { @@ -210,7 +213,12 @@ func (sm *StorageManager) Close() error { } func (sm *StorageManager) close() error { - return errors.Join(sm.eventDB.Flush(), sm.decisionDB.Flush(), sm.eventDB.Close(), sm.decisionDB.Close()) + return errors.Join( + wrapNonNilErr("event db flush error: %w", sm.eventDB.Flush()), + wrapNonNilErr("decision db flush error: %w", sm.decisionDB.Flush()), + wrapNonNilErr("event db close error: %w", sm.eventDB.Close()), + wrapNonNilErr("decision db close error: %w", sm.decisionDB.Close()), + ) } // Reload flushes out pending disk writes to disk by reloading the database. @@ -282,10 +290,10 @@ func (sm *StorageManager) RotatePartitions() error { ub := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb return errors.Join( - sm.eventDB.DeleteRange(lb, ub, pebble.NoSync), - sm.decisionDB.DeleteRange(lb, ub, pebble.NoSync), - sm.eventDB.Compact(lb, ub, false), - sm.decisionDB.Compact(lb, ub, false), + wrapNonNilErr("event db delete range error: %w", sm.eventDB.DeleteRange(lb, ub, pebble.NoSync)), + wrapNonNilErr("decision db delete range error: %w", sm.decisionDB.DeleteRange(lb, ub, pebble.NoSync)), + wrapNonNilErr("event db compact error: %w", sm.eventDB.Compact(lb, ub, false)), + wrapNonNilErr("decision db compact error: %w", sm.decisionDB.Compact(lb, ub, false)), ) } @@ -307,3 +315,11 @@ func (sm *StorageManager) NewReadWriter() StorageLimitReadWriter { decisionRW: sm.decisionStorage.NewReadWriter(), }) } + +// wrapNonNilErr only wraps an error with format if the error is not nil. +func wrapNonNilErr(format string, err error) error { + if err == nil { + return nil + } + return fmt.Errorf(format, err) +} From f691e0018f58fa400adb07389987151c4d417e05 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 00:30:04 +0000 Subject: [PATCH 164/184] Add doc.go --- x-pack/apm-server/sampling/eventstorage/doc.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 x-pack/apm-server/sampling/eventstorage/doc.go diff --git a/x-pack/apm-server/sampling/eventstorage/doc.go b/x-pack/apm-server/sampling/eventstorage/doc.go new file mode 100644 index 00000000000..4bd93791b94 --- /dev/null +++ b/x-pack/apm-server/sampling/eventstorage/doc.go @@ -0,0 +1,14 @@ +// 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 implements the storage layer for tail-based sampling event +// and sampling decision read-writes. +// +// The database of choice is Pebble, which does not have TTL handling built-in, +// and we implement our own TTL handling on top of the database: +// - TTL is divided up into N parts, where N is partitionsPerTTL. +// - A database holds N + 1 + 1 partitions. +// - Every TTL/N we will discard the oldest partition, so we keep a rolling window of N+1 partitions. +// - Writes will go to the most recent partition, and we'll read across N+1 partitions +package eventstorage From de566b09106266552dd249801f75784d13dca315 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 12:01:17 +0000 Subject: [PATCH 165/184] Rename partitioner methods --- .../sampling/eventstorage/partition_rw.go | 10 ++-- .../sampling/eventstorage/partitioner.go | 12 ++-- .../sampling/eventstorage/partitioner_test.go | 56 +++++++++---------- .../sampling/eventstorage/storage_manager.go | 2 +- 4 files changed, 37 insertions(+), 43 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index 67686725146..cb3bbbad9e8 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -19,7 +19,7 @@ type PartitionReadWriter struct { func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() - pid := rw.s.partitioner.Current() + pid := rw.s.partitioner.currentID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) } @@ -35,7 +35,7 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() var errs []error - for pid := range rw.s.partitioner.Actives() { + for pid := range rw.s.partitioner.activeIDs() { sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil @@ -53,7 +53,7 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() - pid := rw.s.partitioner.Current() + pid := rw.s.partitioner.currentID() return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) } @@ -62,7 +62,7 @@ func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() var errs []error - for pid := range rw.s.partitioner.Actives() { + for pid := range rw.s.partitioner.activeIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) if err != nil { errs = append(errs, err) @@ -76,7 +76,7 @@ func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batc rw.s.partitioner.mu.RLock() defer rw.s.partitioner.mu.RUnlock() var errs []error - for pid := range rw.s.partitioner.Actives() { + for pid := range rw.s.partitioner.activeIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).ReadTraceEvents(traceID, out) if err != nil { errs = append(errs, err) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 1e9e63bd149..919acde0c7e 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -61,10 +61,10 @@ func (p *Partitioner) Rotate() int { return p.current } -// Actives returns an iterator containing all active partitions. +// activeIDs returns an iterator containing all active partitions. // It contains total - 1 partitions. // Callers should obtain p.mu.RLock when using the returned PIDs. -func (p *Partitioner) Actives() iter.Seq[int] { +func (p *Partitioner) activeIDs() iter.Seq[int] { cur := p.current return func(yield func(int) bool) { for i := 0; i < p.total-1; i++ { @@ -75,14 +75,14 @@ func (p *Partitioner) Actives() iter.Seq[int] { } } -// Inactive returns the ID of the inactive partition. +// inactiveID returns the ID of the inactive partition. // Callers should obtain p.mu.RLock when using the returned PID. -func (p *Partitioner) Inactive() int { +func (p *Partitioner) inactiveID() int { return (p.current + 1) % p.total } -// Current returns the ID of the current partition (rightmost active). +// currentID returns the ID of the current partition (rightmost active). // Callers should obtain p.mu.RLock when using the returned PID. -func (p *Partitioner) Current() int { +func (p *Partitioner) currentID() int { return p.current } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 4f302d47085..eba9b93d60f 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -2,57 +2,51 @@ // 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_test +package eventstorage import ( - "iter" + "slices" "testing" "github.com/stretchr/testify/assert" - - "github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage" ) -func iterToSlice[T any](it iter.Seq[T]) (s []T) { - for i := range it { - s = append(s, i) - } - return -} - func TestPartitioner(t *testing.T) { - p := eventstorage.NewPartitioner(2, 0) // partition id 0, 1, 2 + p := NewPartitioner(2, 0) // partition id 0, 1, 2 - assert.Equal(t, 0, p.Current()) - assert.Equal(t, 1, p.Inactive()) - assert.Equal(t, []int{0, 2}, iterToSlice(p.Actives())) + assert.Equal(t, 0, p.currentID()) + assert.Equal(t, 1, p.inactiveID()) + assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) // 0 -> 1 - p.Rotate() + newCurrent := p.Rotate() - assert.Equal(t, 1, p.Current()) - assert.Equal(t, 2, p.Inactive()) - assert.Equal(t, []int{1, 0}, iterToSlice(p.Actives())) + assert.Equal(t, 1, newCurrent) + assert.Equal(t, 1, p.currentID()) + assert.Equal(t, 2, p.inactiveID()) + assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) // 1 -> 2 - p.Rotate() + newCurrent = p.Rotate() - assert.Equal(t, 2, p.Current()) - assert.Equal(t, 0, p.Inactive()) - assert.Equal(t, []int{2, 1}, iterToSlice(p.Actives())) + assert.Equal(t, 2, newCurrent) + assert.Equal(t, 2, p.currentID()) + assert.Equal(t, 0, p.inactiveID()) + assert.Equal(t, []int{2, 1}, slices.Collect(p.activeIDs())) // 2 -> 0 - p.Rotate() + newCurrent = p.Rotate() - assert.Equal(t, 0, p.Current()) - assert.Equal(t, 1, p.Inactive()) - assert.Equal(t, []int{0, 2}, iterToSlice(p.Actives())) + assert.Equal(t, 0, newCurrent) + assert.Equal(t, 0, p.currentID()) + assert.Equal(t, 1, p.inactiveID()) + assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) } func TestPartitionerCurrentID(t *testing.T) { - p := eventstorage.NewPartitioner(2, 1) + p := NewPartitioner(2, 1) - assert.Equal(t, 1, p.Current()) - assert.Equal(t, 2, p.Inactive()) - assert.Equal(t, []int{1, 0}, iterToSlice(p.Actives())) + assert.Equal(t, 1, p.currentID()) + assert.Equal(t, 2, p.inactiveID()) + assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 2d806a0bc84..7fd572279d4 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -283,7 +283,7 @@ func (sm *StorageManager) RotatePartitions() error { } // No lock is needed here as the only writer to sm.partitioner is exactly this function. - pidToDelete := sm.partitioner.Inactive() + pidToDelete := sm.partitioner.inactiveID() lbPrefix := byte(pidToDelete) lb := []byte{lbPrefix} From e57a2d3dbcaf41527162dbe5eada0019fb8c6d83 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 12:08:43 +0000 Subject: [PATCH 166/184] Return newInactive in Rotate --- .../sampling/eventstorage/partitioner.go | 10 ++-------- .../sampling/eventstorage/partitioner_test.go | 14 ++++++-------- .../sampling/eventstorage/storage_manager.go | 5 ++--- 3 files changed, 10 insertions(+), 19 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 919acde0c7e..46befc90abb 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -54,11 +54,11 @@ func NewPartitioner(actives, currentID int) *Partitioner { // After Rotate: // A-A-I-A // ..^.... -func (p *Partitioner) Rotate() int { +func (p *Partitioner) Rotate() (newCurrent, newInactive int) { p.mu.Lock() defer p.mu.Unlock() p.current = (p.current + 1) % p.total - return p.current + return p.current, (p.current + 1) % p.total } // activeIDs returns an iterator containing all active partitions. @@ -75,12 +75,6 @@ func (p *Partitioner) activeIDs() iter.Seq[int] { } } -// inactiveID returns the ID of the inactive partition. -// Callers should obtain p.mu.RLock when using the returned PID. -func (p *Partitioner) inactiveID() int { - return (p.current + 1) % p.total -} - // currentID returns the ID of the current partition (rightmost active). // Callers should obtain p.mu.RLock when using the returned PID. func (p *Partitioner) currentID() int { diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index eba9b93d60f..3ec5c34e382 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -15,31 +15,30 @@ func TestPartitioner(t *testing.T) { p := NewPartitioner(2, 0) // partition id 0, 1, 2 assert.Equal(t, 0, p.currentID()) - assert.Equal(t, 1, p.inactiveID()) assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) // 0 -> 1 - newCurrent := p.Rotate() + newCurrent, newInactive := p.Rotate() assert.Equal(t, 1, newCurrent) assert.Equal(t, 1, p.currentID()) - assert.Equal(t, 2, p.inactiveID()) + assert.Equal(t, 2, newInactive) assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) // 1 -> 2 - newCurrent = p.Rotate() + newCurrent, newInactive = p.Rotate() assert.Equal(t, 2, newCurrent) assert.Equal(t, 2, p.currentID()) - assert.Equal(t, 0, p.inactiveID()) + assert.Equal(t, 0, newInactive) assert.Equal(t, []int{2, 1}, slices.Collect(p.activeIDs())) // 2 -> 0 - newCurrent = p.Rotate() + newCurrent, newInactive = p.Rotate() assert.Equal(t, 0, newCurrent) assert.Equal(t, 0, p.currentID()) - assert.Equal(t, 1, p.inactiveID()) + assert.Equal(t, 1, newInactive) assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) } @@ -47,6 +46,5 @@ func TestPartitionerCurrentID(t *testing.T) { p := NewPartitioner(2, 1) assert.Equal(t, 1, p.currentID()) - assert.Equal(t, 2, p.inactiveID()) assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 7fd572279d4..f1ff730bd23 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -276,15 +276,14 @@ func (sm *StorageManager) runTTLGCLoop(stopping <-chan struct{}, ttl time.Durati // RotatePartitions rotates the partitions to clean up TTL-expired entries. func (sm *StorageManager) RotatePartitions() error { - newCurrentPID := sm.partitioner.Rotate() + newCurrentPID, newInactivePID := sm.partitioner.Rotate() if err := sm.savePartitionID(newCurrentPID); err != nil { return err } // No lock is needed here as the only writer to sm.partitioner is exactly this function. - pidToDelete := sm.partitioner.inactiveID() - lbPrefix := byte(pidToDelete) + lbPrefix := byte(newInactivePID) lb := []byte{lbPrefix} ub := []byte{lbPrefix + 1} // Do not use % here as ub MUST BE greater than lb From f68f7652b8c843b65232f9a4bdc87cc548d00104 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 12:12:35 +0000 Subject: [PATCH 167/184] activeIDs lock internally --- x-pack/apm-server/sampling/eventstorage/partition_rw.go | 6 ------ x-pack/apm-server/sampling/eventstorage/partitioner.go | 7 ++++--- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index cb3bbbad9e8..c4aed19bfaa 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -32,8 +32,6 @@ func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) e // 1. when a remote sampling decision is received from pubsub // 2. (hot path) when a transaction / span comes in, check if a sampling decision has already been made func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { - rw.s.partitioner.mu.RLock() - defer rw.s.partitioner.mu.RUnlock() var errs []error for pid := range rw.s.partitioner.activeIDs() { sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) @@ -59,8 +57,6 @@ func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelp // DeleteTraceEvent deletes the trace event from storage. func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { - rw.s.partitioner.mu.RLock() - defer rw.s.partitioner.mu.RUnlock() var errs []error for pid := range rw.s.partitioner.activeIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) @@ -73,8 +69,6 @@ func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { - rw.s.partitioner.mu.RLock() - defer rw.s.partitioner.mu.RUnlock() var errs []error for pid := range rw.s.partitioner.activeIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).ReadTraceEvents(traceID, out) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 46befc90abb..126a7b804ea 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -63,12 +63,13 @@ func (p *Partitioner) Rotate() (newCurrent, newInactive int) { // activeIDs returns an iterator containing all active partitions. // It contains total - 1 partitions. -// Callers should obtain p.mu.RLock when using the returned PIDs. +// Rotate should never be called within activeIDs as activeIDs holds RLock internally. func (p *Partitioner) activeIDs() iter.Seq[int] { - cur := p.current return func(yield func(int) bool) { + p.mu.RLock() + defer p.mu.RUnlock() for i := 0; i < p.total-1; i++ { - if !yield((cur + p.total - i) % p.total) { + if !yield((p.current + p.total - i) % p.total) { return } } From 34213e172da5039793487e5e49134afcfbb85125 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 12:33:02 +0000 Subject: [PATCH 168/184] Change currentIDs to CurrentIDFunc --- .../sampling/eventstorage/partition_rw.go | 26 ++++++++-------- .../sampling/eventstorage/partitioner.go | 22 +++++++++----- .../sampling/eventstorage/partitioner_test.go | 30 ++++++++++++------- 3 files changed, 48 insertions(+), 30 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/partition_rw.go b/x-pack/apm-server/sampling/eventstorage/partition_rw.go index c4aed19bfaa..c051e5ce069 100644 --- a/x-pack/apm-server/sampling/eventstorage/partition_rw.go +++ b/x-pack/apm-server/sampling/eventstorage/partition_rw.go @@ -16,11 +16,11 @@ type PartitionReadWriter struct { } // WriteTraceSampled records the tail-sampling decision for the given trace ID. -func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) error { - rw.s.partitioner.mu.RLock() - defer rw.s.partitioner.mu.RUnlock() - pid := rw.s.partitioner.currentID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) +func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) (err error) { + rw.s.partitioner.CurrentIDFunc(func(pid int) { + err = NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceSampled(traceID, sampled) + }) + return } // IsTraceSampled reports whether traceID belongs to a trace that is sampled @@ -33,7 +33,7 @@ func (rw *PartitionReadWriter) WriteTraceSampled(traceID string, sampled bool) e // 2. (hot path) when a transaction / span comes in, check if a sampling decision has already been made func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { var errs []error - for pid := range rw.s.partitioner.activeIDs() { + for pid := range rw.s.partitioner.ActiveIDs() { sampled, err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).IsTraceSampled(traceID) if err == nil { return sampled, nil @@ -48,17 +48,17 @@ func (rw *PartitionReadWriter) IsTraceSampled(traceID string) (bool, error) { } // WriteTraceEvent writes a trace event to storage. -func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { - rw.s.partitioner.mu.RLock() - defer rw.s.partitioner.mu.RUnlock() - pid := rw.s.partitioner.currentID() - return NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) +func (rw *PartitionReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) (err error) { + rw.s.partitioner.CurrentIDFunc(func(pid int) { + err = NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).WriteTraceEvent(traceID, id, event) + }) + return } // DeleteTraceEvent deletes the trace event from storage. func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { var errs []error - for pid := range rw.s.partitioner.activeIDs() { + for pid := range rw.s.partitioner.ActiveIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).DeleteTraceEvent(traceID, id) if err != nil { errs = append(errs, err) @@ -70,7 +70,7 @@ func (rw *PartitionReadWriter) DeleteTraceEvent(traceID, id string) error { // ReadTraceEvents reads trace events with the given trace ID from storage into out. func (rw *PartitionReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { var errs []error - for pid := range rw.s.partitioner.activeIDs() { + for pid := range rw.s.partitioner.ActiveIDs() { err := NewPrefixReadWriter(rw.s.db, byte(pid), rw.s.codec).ReadTraceEvents(traceID, out) if err != nil { errs = append(errs, err) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner.go b/x-pack/apm-server/sampling/eventstorage/partitioner.go index 126a7b804ea..512a4de1226 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner.go @@ -61,10 +61,13 @@ func (p *Partitioner) Rotate() (newCurrent, newInactive int) { return p.current, (p.current + 1) % p.total } -// activeIDs returns an iterator containing all active partitions. +// ActiveIDs returns an iterator containing all active partitions. // It contains total - 1 partitions. -// Rotate should never be called within activeIDs as activeIDs holds RLock internally. -func (p *Partitioner) activeIDs() iter.Seq[int] { +// +// As ActiveIDs holds RLock internally, +// - Rotate should never be called within ActiveIDs +// - the returned partition IDs may be outdated if used outside `range` +func (p *Partitioner) ActiveIDs() iter.Seq[int] { return func(yield func(int) bool) { p.mu.RLock() defer p.mu.RUnlock() @@ -76,8 +79,13 @@ func (p *Partitioner) activeIDs() iter.Seq[int] { } } -// currentID returns the ID of the current partition (rightmost active). -// Callers should obtain p.mu.RLock when using the returned PID. -func (p *Partitioner) currentID() int { - return p.current +// CurrentIDFunc calls function f which accepts the ID of the current partition (rightmost active). +// +// As CurrentIDFunc holds RLock internally, +// - Rotate should never be called within CurrentIDFunc +// - the returned partition ID may be outdated if used outside f +func (p *Partitioner) CurrentIDFunc(f func(int)) { + p.mu.RLock() + defer p.mu.RUnlock() + f(p.current) } diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index 3ec5c34e382..c1784faff03 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -14,37 +14,47 @@ import ( func TestPartitioner(t *testing.T) { p := NewPartitioner(2, 0) // partition id 0, 1, 2 - assert.Equal(t, 0, p.currentID()) - assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) + p.CurrentIDFunc(func(pid int) { + assert.Equal(t, 0, pid) + }) + assert.Equal(t, []int{0, 2}, slices.Collect(p.ActiveIDs())) // 0 -> 1 newCurrent, newInactive := p.Rotate() assert.Equal(t, 1, newCurrent) - assert.Equal(t, 1, p.currentID()) + p.CurrentIDFunc(func(pid int) { + assert.Equal(t, 1, pid) + }) assert.Equal(t, 2, newInactive) - assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) + assert.Equal(t, []int{1, 0}, slices.Collect(p.ActiveIDs())) // 1 -> 2 newCurrent, newInactive = p.Rotate() assert.Equal(t, 2, newCurrent) - assert.Equal(t, 2, p.currentID()) + p.CurrentIDFunc(func(pid int) { + assert.Equal(t, 2, pid) + }) assert.Equal(t, 0, newInactive) - assert.Equal(t, []int{2, 1}, slices.Collect(p.activeIDs())) + assert.Equal(t, []int{2, 1}, slices.Collect(p.ActiveIDs())) // 2 -> 0 newCurrent, newInactive = p.Rotate() assert.Equal(t, 0, newCurrent) - assert.Equal(t, 0, p.currentID()) + p.CurrentIDFunc(func(pid int) { + assert.Equal(t, 0, pid) + }) assert.Equal(t, 1, newInactive) - assert.Equal(t, []int{0, 2}, slices.Collect(p.activeIDs())) + assert.Equal(t, []int{0, 2}, slices.Collect(p.ActiveIDs())) } func TestPartitionerCurrentID(t *testing.T) { p := NewPartitioner(2, 1) - assert.Equal(t, 1, p.currentID()) - assert.Equal(t, []int{1, 0}, slices.Collect(p.activeIDs())) + p.CurrentIDFunc(func(pid int) { + assert.Equal(t, 1, pid) + }) + assert.Equal(t, []int{1, 0}, slices.Collect(p.ActiveIDs())) } From 7de54dd66a19dc3ec6b1d227dc7e90c8977f9ccc Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 23:18:21 +0000 Subject: [PATCH 169/184] Update error for consistency --- x-pack/apm-server/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index c56d8a2ecf7..491b2377000 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -116,7 +116,7 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er storageDir := paths.Resolve(paths.Data, tailSamplingStorageDir) db, err := getDB(storageDir) if err != nil { - return nil, fmt.Errorf("failed to get TBS database: %w", err) + return nil, fmt.Errorf("failed to get tail-sampling database: %w", err) } policies := make([]sampling.Policy, len(tailSamplingConfig.Policies)) From 4fa8670e6b1bbab43c0bc57ed2d281a67ee2ffc7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 23:34:57 +0000 Subject: [PATCH 170/184] make fmt --- x-pack/apm-server/main.go | 4 ++-- x-pack/apm-server/sampling/processor_test.go | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index ab4fdd438e8..df3db824a7a 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -37,8 +37,8 @@ const ( var ( // db holds the database to use when tail-based sampling is configured. - dbMu sync.Mutex - db *eventstorage.StorageManager + dbMu sync.Mutex + db *eventstorage.StorageManager dbMetricRegistration metric.Registration // samplerUUID is a UUID used to identify sampled trace ID documents diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index d7fb6afd6ff..1b57e821d40 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -640,7 +640,7 @@ func TestStorageLimit(t *testing.T) { }) // Ensure that there are some failed writes. - failedWrites := getSum(t, tempDirConfig.metricReader, "apm-server.sampling.tail.events.failed_writes") + failedWrites := getSum(t, tempDirConfig.metricReader, "apm-server.sampling.tail.events.failed_writes") t.Log(failedWrites) if failedWrites >= 1 { @@ -717,7 +717,7 @@ func TestGracefulShutdown(t *testing.T) { type testConfig struct { sampling.Config - tempDir string + tempDir string metricReader sdkmetric.Reader } @@ -738,7 +738,7 @@ func newTempdirConfig(tb testing.TB) testConfig { mp := sdkmetric.NewMeterProvider(sdkmetric.WithReader(reader)) return testConfig{ - tempDir: tempdir, + tempDir: tempdir, metricReader: reader, Config: sampling.Config{ BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), From 0adcd8cfb34661037452f4d0c1170eb95ced71e0 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Tue, 28 Jan 2025 23:56:40 +0000 Subject: [PATCH 171/184] Add concurrency test to partitioner --- .../sampling/eventstorage/partitioner_test.go | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go index c1784faff03..4715ed2c2d9 100644 --- a/x-pack/apm-server/sampling/eventstorage/partitioner_test.go +++ b/x-pack/apm-server/sampling/eventstorage/partitioner_test.go @@ -6,6 +6,7 @@ package eventstorage import ( "slices" + "sync" "testing" "github.com/stretchr/testify/assert" @@ -58,3 +59,46 @@ func TestPartitionerCurrentID(t *testing.T) { }) assert.Equal(t, []int{1, 0}, slices.Collect(p.ActiveIDs())) } + +func TestPartitioner_ActiveIDs_Concurrent(t *testing.T) { + wg := sync.WaitGroup{} + wg.Add(1) + p := NewPartitioner(2, 0) + rotate := make(chan struct{}) + go func() { + <-rotate + newCurrent, newInactive := p.Rotate() + assert.Equal(t, 1, newCurrent) + assert.Equal(t, 2, newInactive) + wg.Done() + }() + var activeIDs []int + for pid := range p.ActiveIDs() { + if rotate != nil { + rotate <- struct{}{} // blocks + } + rotate = nil + activeIDs = append(activeIDs, pid) + } + assert.Equal(t, []int{0, 2}, activeIDs) + wg.Wait() +} + +func TestPartitioner_CurrentIDFunc_Concurrent(t *testing.T) { + wg := sync.WaitGroup{} + wg.Add(1) + p := NewPartitioner(2, 0) + rotate := make(chan struct{}) + go func() { + <-rotate + newCurrent, newInactive := p.Rotate() + assert.Equal(t, 1, newCurrent) + assert.Equal(t, 2, newInactive) + wg.Done() + }() + p.CurrentIDFunc(func(pid int) { + rotate <- struct{}{} // blocks + assert.Equal(t, 0, pid) + }) + wg.Wait() +} From 35e5d0a5f1c751fa2f35c2fe37a3fcfa3a87c590 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:00:01 +0000 Subject: [PATCH 172/184] Reuse sampling decision byte slices --- x-pack/apm-server/sampling/eventstorage/prefix.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 35df65e6159..3e6dd95a6d3 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -28,6 +28,10 @@ var ( // ErrNotFound is returned by the RW.IsTraceSampled method, // for non-existing trace IDs. ErrNotFound = errors.New("key not found") + + // Reuse sampling decision value byte slices for performance + traceSampledVal = []byte{entryMetaTraceSampled} + traceUnsampledVal = []byte{entryMetaTraceUnsampled} ) func NewPrefixReadWriter(db db, prefix byte, codec Codec) PrefixReadWriter { @@ -96,11 +100,11 @@ func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error b.WriteByte(rw.prefix) b.WriteString(traceID) - meta := entryMetaTraceUnsampled + val := traceUnsampledVal if sampled { - meta = entryMetaTraceSampled + val = traceSampledVal } - return rw.db.Set(b.Bytes(), []byte{meta}, pebble.NoSync) + return rw.db.Set(b.Bytes(), val, pebble.NoSync) } func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { From 3d3602ba2324328a02a3c9ef0917cc3603acf2d9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:08:39 +0000 Subject: [PATCH 173/184] Add partitionerMeta struct --- .../sampling/eventstorage/storage_manager.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index f1ff730bd23..0382bceba3e 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -137,6 +137,10 @@ func (sm *StorageManager) reset() error { return nil } +type partitionerMeta struct { + ID int `json:"id"` +} + // loadPartitionID loads the last saved partition ID from database, // such that partitioner resumes from where it left off before an apm-server restart. func (sm *StorageManager) loadPartitionID() (int, error) { @@ -147,16 +151,18 @@ func (sm *StorageManager) loadPartitionID() (int, error) { return 0, err } defer closer.Close() - var pid struct { - ID int `json:"id"` - } + var pid partitionerMeta err = json.Unmarshal(item, &pid) return pid.ID, err } // savePartitionID saves the partition ID to database to be loaded by loadPartitionID later. func (sm *StorageManager) savePartitionID(pid int) error { - return sm.decisionDB.Set([]byte(partitionerMetaKey), []byte(fmt.Sprintf(`{"id":%d}`, pid)), pebble.NoSync) + b, err := json.Marshal(partitionerMeta{ID: pid}) + if err != nil { + return fmt.Errorf("error marshaling partition ID: %w", err) + } + return sm.decisionDB.Set([]byte(partitionerMetaKey), b, pebble.NoSync) } func (sm *StorageManager) Size() (lsm, vlog int64) { From e0857a52ae6ec3ebea81df0210b33629d9ce863d Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:16:18 +0000 Subject: [PATCH 174/184] Move back TestStorageMonitoring --- x-pack/apm-server/main_test.go | 108 ------------------- x-pack/apm-server/sampling/processor_test.go | 89 +++++++++++---- 2 files changed, 71 insertions(+), 126 deletions(-) diff --git a/x-pack/apm-server/main_test.go b/x-pack/apm-server/main_test.go index 0c7c14d0e7a..61cc31fac96 100644 --- a/x-pack/apm-server/main_test.go +++ b/x-pack/apm-server/main_test.go @@ -8,11 +8,8 @@ package main import ( "context" - "path/filepath" "testing" - "time" - "github.com/gofrs/uuid/v5" "github.com/pkg/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -28,8 +25,6 @@ import ( "github.com/elastic/apm-server/internal/beater/config" "github.com/elastic/apm-server/internal/beater/monitoringtest" "github.com/elastic/apm-server/internal/elasticsearch" - "github.com/elastic/apm-server/x-pack/apm-server/sampling" - "github.com/elastic/apm-server/x-pack/apm-server/sampling/pubsub/pubsubtest" ) func TestMonitoring(t *testing.T) { @@ -79,106 +74,3 @@ func TestMonitoring(t *testing.T) { }) } } - -func TestStorageMonitoring(t *testing.T) { - config, reader := newTempdirConfig(t) - - processor, err := sampling.NewProcessor(config) - require.NoError(t, err) - go processor.Run() - defer processor.Stop(context.Background()) - for i := 0; i < 100; i++ { - traceID := uuid.Must(uuid.NewV4()).String() - batch := modelpb.Batch{{ - Trace: &modelpb.Trace{Id: traceID}, - Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, - Transaction: &modelpb.Transaction{ - Type: "type", - Id: traceID, - Sampled: true, - }, - }} - err := processor.ProcessBatch(context.Background(), &batch) - require.NoError(t, err) - assert.Empty(t, batch) - } - - // Stop the processor, flushing pending writes, and reopen storage. - // Reopening storage is necessary to immediately recalculate the - // storage size, otherwise we must wait for a minute (hard-coded in - // badger) for storage metrics to be updated. - err = processor.Stop(context.Background()) - require.NoError(t, err) - require.NoError(t, closeBadger()) - badgerDB, err = getBadgerDB(config.StorageDir, config.MeterProvider) - require.NoError(t, err) - - lsmSize := getGauge(t, reader, "apm-server.sampling.tail.storage.lsm_size") - assert.NotZero(t, lsmSize) - vlogSize := getGauge(t, reader, "apm-server.sampling.tail.storage.value_log_size") - assert.NotZero(t, vlogSize) -} - -func newTempdirConfig(tb testing.TB) (sampling.Config, sdkmetric.Reader) { - tempdir := filepath.Join(tb.TempDir(), "samplingtest") - reader := sdkmetric.NewManualReader(sdkmetric.WithTemporalitySelector( - func(ik sdkmetric.InstrumentKind) metricdata.Temporality { - return metricdata.DeltaTemporality - }, - )) - mp := sdkmetric.NewMeterProvider(sdkmetric.WithReader(reader)) - - closeBadger() - badgerDB, err := getBadgerDB(tempdir, mp) - require.NoError(tb, err) - tb.Cleanup(func() { closeBadger() }) - - storage := badgerDB.NewReadWriter() - - return sampling.Config{ - BatchProcessor: modelpb.ProcessBatchFunc(func(context.Context, *modelpb.Batch) error { return nil }), - MeterProvider: mp, - LocalSamplingConfig: sampling.LocalSamplingConfig{ - FlushInterval: time.Second, - MaxDynamicServices: 1000, - IngestRateDecayFactor: 0.9, - Policies: []sampling.Policy{ - {SampleRate: 0.1}, - }, - }, - RemoteSamplingConfig: sampling.RemoteSamplingConfig{ - Elasticsearch: pubsubtest.Client(nil, nil), - SampledTracesDataStream: sampling.DataStreamConfig{ - Type: "traces", - Dataset: "sampled", - Namespace: "testing", - }, - UUID: "local-apm-server", - }, - StorageConfig: sampling.StorageConfig{ - DB: badgerDB, - Storage: storage, - StorageDir: tempdir, - StorageGCInterval: time.Second, - TTL: 30 * time.Minute, - StorageLimit: 0, // No storage limit. - }, - }, reader -} - -func getGauge(t testing.TB, reader sdkmetric.Reader, name string) int64 { - var rm metricdata.ResourceMetrics - assert.NoError(t, reader.Collect(context.Background(), &rm)) - - assert.NotEqual(t, 0, len(rm.ScopeMetrics)) - - for _, sm := range rm.ScopeMetrics { - for _, m := range sm.Metrics { - if m.Name == name { - return m.Data.(metricdata.Gauge[int64]).DataPoints[0].Value - } - } - } - - return 0 -} diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 1b57e821d40..271580217e3 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -54,8 +54,8 @@ func TestProcessUnsampled(t *testing.T) { } func TestProcessAlreadyTailSampled(t *testing.T) { - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config // Seed event storage with a tail-sampling decisions, to show that // subsequent events in the trace will be reported immediately. @@ -118,7 +118,7 @@ func TestProcessAlreadyTailSampled(t *testing.T) { // they were received after the trace sampling entry expired. assert.Equal(t, modelpb.Batch{&transaction1, &span1}, batch) - monitoringtest.ExpectContainOtelMetrics(t, tempDirConfig.metricReader, map[string]any{ + monitoringtest.ExpectContainOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.events.processed": 4, "apm-server.sampling.tail.events.stored": 2, "apm-server.sampling.tail.events.sampled": 2, @@ -152,8 +152,8 @@ func TestProcessLocalTailSampling(t *testing.T) { }, } { t.Run(fmt.Sprintf("%f", tc.sampleRate), func(t *testing.T) { - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.Policies = []sampling.Policy{{SampleRate: tc.sampleRate}} config.FlushInterval = 10 * time.Millisecond published := make(chan string) @@ -234,7 +234,7 @@ func TestProcessLocalTailSampling(t *testing.T) { sampledTraceEvents = trace2Events } - monitoringtest.ExpectContainOtelMetrics(t, tempDirConfig.metricReader, map[string]any{ + monitoringtest.ExpectContainOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.events.processed": 4, "apm-server.sampling.tail.events.stored": 4, "apm-server.sampling.tail.events.sampled": 2, @@ -271,8 +271,8 @@ func TestProcessLocalTailSampling(t *testing.T) { } func TestProcessLocalTailSamplingUnsampled(t *testing.T) { - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.FlushInterval = time.Minute processor, err := sampling.NewProcessor(config) require.NoError(t, err) @@ -298,7 +298,7 @@ func TestProcessLocalTailSamplingUnsampled(t *testing.T) { assert.Empty(t, batch) // break out of the loop as soon as the first one is dropped. - droppedEvents := getSum(t, tempDirConfig.metricReader, "apm-server.sampling.events.dropped") + droppedEvents := getSum(t, tempdirConfig.metricReader, "apm-server.sampling.events.dropped") if droppedEvents != 0 { break } @@ -393,8 +393,8 @@ func TestProcessLocalTailSamplingPolicyOrder(t *testing.T) { } func TestProcessRemoteTailSampling(t *testing.T) { - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.Policies = []sampling.Policy{{SampleRate: 0.5}} config.FlushInterval = 10 * time.Millisecond @@ -462,7 +462,7 @@ func TestProcessRemoteTailSampling(t *testing.T) { assert.NoError(t, config.DB.Flush()) assert.Empty(t, published) // remote decisions don't get republished - monitoringtest.ExpectOtelMetrics(t, tempDirConfig.metricReader, map[string]any{ + monitoringtest.ExpectOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.events.processed": 1, "apm-server.sampling.tail.events.stored": 1, "apm-server.sampling.tail.events.sampled": 1, @@ -555,8 +555,8 @@ func TestProcessDiscardOnWriteFailure(t *testing.T) { } func TestGroupsMonitoring(t *testing.T) { - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.MaxDynamicServices = 5 config.FlushInterval = time.Minute config.Policies[0].SampleRate = 0.99 @@ -580,7 +580,7 @@ func TestGroupsMonitoring(t *testing.T) { require.NoError(t, err) } - monitoringtest.ExpectOtelMetrics(t, tempDirConfig.metricReader, map[string]any{ + monitoringtest.ExpectOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.dynamic_service_groups": config.MaxDynamicServices, "apm-server.sampling.tail.events.processed": config.MaxDynamicServices + 2, "apm-server.sampling.tail.events.stored": config.MaxDynamicServices, @@ -589,6 +589,59 @@ func TestGroupsMonitoring(t *testing.T) { }) } +func getGauge(t testing.TB, reader sdkmetric.Reader, name string) int64 { + var rm metricdata.ResourceMetrics + assert.NoError(t, reader.Collect(context.Background(), &rm)) + + assert.NotEqual(t, 0, len(rm.ScopeMetrics)) + + for _, sm := range rm.ScopeMetrics { + for _, m := range sm.Metrics { + if m.Name == name { + return m.Data.(metricdata.Gauge[int64]).DataPoints[0].Value + } + } + } + + return 0 +} + +func TestStorageMonitoring(t *testing.T) { + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config + + processor, err := sampling.NewProcessor(config) + require.NoError(t, err) + go processor.Run() + defer processor.Stop(context.Background()) + for i := 0; i < 100; i++ { + traceID := uuid.Must(uuid.NewV4()).String() + batch := modelpb.Batch{{ + Trace: &modelpb.Trace{Id: traceID}, + Event: &modelpb.Event{Duration: uint64(123 * time.Millisecond)}, + Transaction: &modelpb.Transaction{ + Type: "type", + Id: traceID, + Sampled: true, + }, + }} + err := processor.ProcessBatch(context.Background(), &batch) + require.NoError(t, err) + assert.Empty(t, batch) + } + + // Stop the processor, flushing pending writes, and reopen storage. + err = processor.Stop(context.Background()) + require.NoError(t, err) + + require.NoError(t, config.DB.Reload()) + + lsmSize := getGauge(t, tempdirConfig.metricReader, "apm-server.sampling.tail.storage.lsm_size") + assert.NotZero(t, lsmSize) + vlogSize := getGauge(t, tempdirConfig.metricReader, "apm-server.sampling.tail.storage.value_log_size") + assert.NotZero(t, vlogSize) +} + 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. @@ -618,8 +671,8 @@ func TestStorageLimit(t *testing.T) { return processor } - tempDirConfig := newTempdirConfig(t) - config := tempDirConfig.Config + tempdirConfig := newTempdirConfig(t) + config := tempdirConfig.Config config.TTL = time.Hour // Write 5K span events and close the DB to persist to disk the storage // size and assert that none are reported immediately. @@ -640,7 +693,7 @@ func TestStorageLimit(t *testing.T) { }) // Ensure that there are some failed writes. - failedWrites := getSum(t, tempDirConfig.metricReader, "apm-server.sampling.tail.events.failed_writes") + failedWrites := getSum(t, tempdirConfig.metricReader, "apm-server.sampling.tail.events.failed_writes") t.Log(failedWrites) if failedWrites >= 1 { From 6f40324c3eaca4456a045c6df05518fb1250a222 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:48:15 +0000 Subject: [PATCH 175/184] Move meter provider to storage manager --- .../beater/monitoringtest/opentelemetry.go | 32 ++++++++++++++----- x-pack/apm-server/main.go | 21 +++--------- x-pack/apm-server/main_test.go | 6 ++-- .../sampling/eventstorage/storage_manager.go | 31 ++++++++++++++++++ x-pack/apm-server/sampling/processor_test.go | 19 ++++++----- 5 files changed, 72 insertions(+), 37 deletions(-) diff --git a/internal/beater/monitoringtest/opentelemetry.go b/internal/beater/monitoringtest/opentelemetry.go index 4e6ff2eabf9..33a60c4f135 100644 --- a/internal/beater/monitoringtest/opentelemetry.go +++ b/internal/beater/monitoringtest/opentelemetry.go @@ -26,29 +26,39 @@ import ( "go.opentelemetry.io/otel/sdk/metric/metricdata" ) -func ExpectOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]interface{}) { - assertOtelMetrics(t, reader, expectedMetrics, true) +func ExpectOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]any) { + assertOtelMetrics(t, reader, expectedMetrics, true, true) } -func ExpectContainOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]interface{}) { - assertOtelMetrics(t, reader, expectedMetrics, false) +func ExpectContainOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]any) { + assertOtelMetrics(t, reader, expectedMetrics, false, true) } -func assertOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]interface{}, match bool) { +func ExpectContainOtelMetricsKeys(t *testing.T, reader sdkmetric.Reader, expectedMetricsKeys []string) { + expectedMetrics := make(map[string]any) + for _, metricKey := range expectedMetricsKeys { + expectedMetrics[metricKey] = nil + } + assertOtelMetrics(t, reader, expectedMetrics, false, false) +} + +func assertOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics map[string]any, match, matchVal bool) { t.Helper() var rm metricdata.ResourceMetrics assert.NoError(t, reader.Collect(context.Background(), &rm)) assert.NotEqual(t, 0, len(rm.ScopeMetrics)) - foundMetrics := []string{} + var foundMetrics []string for _, sm := range rm.ScopeMetrics { - for _, m := range sm.Metrics { switch d := m.Data.(type) { case metricdata.Gauge[int64]: assert.Equal(t, 1, len(d.DataPoints)) foundMetrics = append(foundMetrics, m.Name) + if !matchVal { + continue + } if v, ok := expectedMetrics[m.Name]; ok { if dp, ok := v.(int); ok { @@ -62,6 +72,9 @@ func assertOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics ma case metricdata.Sum[int64]: assert.Equal(t, 1, len(d.DataPoints)) foundMetrics = append(foundMetrics, m.Name) + if !matchVal { + continue + } if v, ok := expectedMetrics[m.Name]; ok { if dp, ok := v.(int); ok { @@ -75,6 +88,9 @@ func assertOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics ma case metricdata.Histogram[int64]: assert.Equal(t, 1, len(d.DataPoints)) foundMetrics = append(foundMetrics, m.Name) + if !matchVal { + continue + } if v, ok := expectedMetrics[m.Name]; ok { if dp, ok := v.(int); ok { @@ -89,7 +105,7 @@ func assertOtelMetrics(t *testing.T, reader sdkmetric.Reader, expectedMetrics ma } } - expectedMetricsKeys := []string{} + var expectedMetricsKeys []string for k := range expectedMetrics { expectedMetricsKeys = append(expectedMetricsKeys, k) } diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index df3db824a7a..ff0aee247e8 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -159,22 +159,15 @@ func getDB(storageDir string, mp metric.MeterProvider) (*eventstorage.StorageMan dbMu.Lock() defer dbMu.Unlock() if db == nil { - sm, err := eventstorage.NewStorageManager(storageDir) + var opts []eventstorage.StorageManagerOptions + if mp != nil { + opts = append(opts, eventstorage.WithMeterProvider(mp)) + } + sm, err := eventstorage.NewStorageManager(storageDir, opts...) if err != nil { return nil, err } db = sm - - meter := mp.Meter("github.com/elastic/apm-server/x-pack/apm-server") - lsmSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.lsm_size") - valueLogSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.value_log_size") - - dbMetricRegistration, _ = meter.RegisterCallback(func(ctx context.Context, o metric.Observer) error { - lsmSize, valueLogSize := sm.Size() - o.ObserveInt64(lsmSizeGauge, lsmSize) - o.ObserveInt64(valueLogSizeGauge, valueLogSize) - return nil - }, lsmSizeGauge, valueLogSizeGauge) } return db, nil } @@ -245,10 +238,6 @@ func wrapServer(args beater.ServerParams, runServer beater.RunServerFunc) (beate // called concurrently with opening DB/accessing the db global, // so it does not need to hold dbMu. func closeDB() error { - if dbMetricRegistration != nil { - dbMetricRegistration.Unregister() - dbMetricRegistration = nil - } if db != nil { return db.Close() } diff --git a/x-pack/apm-server/main_test.go b/x-pack/apm-server/main_test.go index 61cc31fac96..ff8de5e4969 100644 --- a/x-pack/apm-server/main_test.go +++ b/x-pack/apm-server/main_test.go @@ -68,9 +68,9 @@ func TestMonitoring(t *testing.T) { err = runServer(context.Background(), serverParams) assert.Equal(t, runServerError, err) - monitoringtest.ExpectOtelMetrics(t, reader, map[string]any{ - "apm-server.sampling.tail.storage.lsm_size": 0, - "apm-server.sampling.tail.storage.value_log_size": 0, + monitoringtest.ExpectContainOtelMetricsKeys(t, reader, []string{ + "apm-server.sampling.tail.storage.lsm_size", + "apm-server.sampling.tail.storage.value_log_size", }) } } diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 0382bceba3e..c41be072883 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -5,6 +5,7 @@ package eventstorage import ( + "context" "encoding/json" "errors" "fmt" @@ -15,6 +16,7 @@ import ( "time" "github.com/cockroachdb/pebble/v2" + "go.opentelemetry.io/otel/metric" "golang.org/x/sync/errgroup" "github.com/elastic/apm-server/internal/logs" @@ -54,6 +56,12 @@ func WithCodec(codec Codec) StorageManagerOptions { } } +func WithMeterProvider(mp metric.MeterProvider) StorageManagerOptions { + return func(sm *StorageManager) { + sm.meterProvider = mp + } +} + // StorageManager encapsulates pebble.DB. // It assumes exclusive access to pebble DB at storageDir. type StorageManager struct { @@ -80,6 +88,11 @@ type StorageManager struct { // runCh acts as a mutex to ensure only 1 Run is actively running per StorageManager. // as it is possible that 2 separate Run are created by 2 TBS processors during a hot reload. runCh chan struct{} + + // meterProvider is the OTel meter provider + meterProvider metric.MeterProvider + + metricRegistration metric.Registration } // NewStorageManager returns a new StorageManager with pebble DB at storageDir. @@ -94,6 +107,19 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora opt(sm) } + if sm.meterProvider != nil { + meter := sm.meterProvider.Meter("github.com/elastic/apm-server/x-pack/apm-server") + lsmSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.lsm_size") + valueLogSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.value_log_size") + + sm.metricRegistration, _ = meter.RegisterCallback(func(ctx context.Context, o metric.Observer) error { + lsmSize, valueLogSize := sm.Size() + o.ObserveInt64(lsmSizeGauge, lsmSize) + o.ObserveInt64(valueLogSizeGauge, valueLogSize) + return nil + }, lsmSizeGauge, valueLogSizeGauge) + } + if err := sm.reset(); err != nil { return nil, fmt.Errorf("storage manager reset error: %w", err) } @@ -219,6 +245,11 @@ func (sm *StorageManager) Close() error { } func (sm *StorageManager) close() error { + if sm.metricRegistration != nil { + if err := sm.metricRegistration.Unregister(); err != nil { + sm.logger.With(logp.Error(err)).Error("failed to unregister metric") + } + } return errors.Join( wrapNonNilErr("event db flush error: %w", sm.eventDB.Flush()), wrapNonNilErr("decision db flush error: %w", sm.decisionDB.Flush()), diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 271580217e3..5dd9c61d267 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -462,7 +462,7 @@ func TestProcessRemoteTailSampling(t *testing.T) { assert.NoError(t, config.DB.Flush()) assert.Empty(t, published) // remote decisions don't get republished - monitoringtest.ExpectOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ + monitoringtest.ExpectContainOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.events.processed": 1, "apm-server.sampling.tail.events.stored": 1, "apm-server.sampling.tail.events.sampled": 1, @@ -580,7 +580,7 @@ func TestGroupsMonitoring(t *testing.T) { require.NoError(t, err) } - monitoringtest.ExpectOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ + monitoringtest.ExpectContainOtelMetrics(t, tempdirConfig.metricReader, map[string]any{ "apm-server.sampling.tail.dynamic_service_groups": config.MaxDynamicServices, "apm-server.sampling.tail.events.processed": config.MaxDynamicServices + 2, "apm-server.sampling.tail.events.stored": config.MaxDynamicServices, @@ -613,7 +613,6 @@ func TestStorageMonitoring(t *testing.T) { processor, err := sampling.NewProcessor(config) require.NoError(t, err) go processor.Run() - defer processor.Stop(context.Background()) for i := 0; i < 100; i++ { traceID := uuid.Must(uuid.NewV4()).String() batch := modelpb.Batch{{ @@ -630,16 +629,16 @@ func TestStorageMonitoring(t *testing.T) { assert.Empty(t, batch) } - // Stop the processor, flushing pending writes, and reopen storage. + // Stop the processor, flushing pending writes. err = processor.Stop(context.Background()) require.NoError(t, err) - require.NoError(t, config.DB.Reload()) + require.NoError(t, config.DB.Flush()) lsmSize := getGauge(t, tempdirConfig.metricReader, "apm-server.sampling.tail.storage.lsm_size") assert.NotZero(t, lsmSize) vlogSize := getGauge(t, tempdirConfig.metricReader, "apm-server.sampling.tail.storage.value_log_size") - assert.NotZero(t, vlogSize) + assert.Zero(t, vlogSize) } func TestStorageLimit(t *testing.T) { @@ -779,10 +778,6 @@ func newTempdirConfig(tb testing.TB) testConfig { require.NoError(tb, err) tb.Cleanup(func() { os.RemoveAll(tempdir) }) - db, err := eventstorage.NewStorageManager(tempdir) - require.NoError(tb, err) - tb.Cleanup(func() { db.Close() }) - reader := sdkmetric.NewManualReader(sdkmetric.WithTemporalitySelector( func(ik sdkmetric.InstrumentKind) metricdata.Temporality { return metricdata.DeltaTemporality @@ -790,6 +785,10 @@ func newTempdirConfig(tb testing.TB) testConfig { )) mp := sdkmetric.NewMeterProvider(sdkmetric.WithReader(reader)) + db, err := eventstorage.NewStorageManager(tempdir, eventstorage.WithMeterProvider(mp)) + require.NoError(tb, err) + tb.Cleanup(func() { db.Close() }) + return testConfig{ tempDir: tempdir, metricReader: reader, From f4822dff5938f33b0a55bad459a146322324f12e Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:56:26 +0000 Subject: [PATCH 176/184] Add comment to prefix rw --- x-pack/apm-server/sampling/eventstorage/prefix.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 3e6dd95a6d3..43240b9a09e 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -38,12 +38,14 @@ func NewPrefixReadWriter(db db, prefix byte, codec Codec) PrefixReadWriter { return PrefixReadWriter{db: db, prefix: prefix, codec: codec} } +// PrefixReadWriter is a helper read writer that reads from and writes to db with a prefix byte in key. type PrefixReadWriter struct { db db prefix byte codec Codec } +// ReadTraceEvents reads rw.db using a key consisting of rw.prefix and traceID, and appends results to out. func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { var b bytes.Buffer b.Grow(1 + len(traceID) + 1) @@ -79,6 +81,7 @@ func (rw PrefixReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) e return nil } +// WriteTraceEvent writes encoded event as value to rw.db with key consisting of rw.prefix, traceID and id. func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { data, err := rw.codec.EncodeEvent(event) if err != nil { @@ -94,6 +97,7 @@ func (rw PrefixReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.AP return rw.db.Set(key, data, pebble.NoSync) } +// WriteTraceSampled writes sampling decision sampled to rw.db with key consisting of rw.prefix and traceID. func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error { var b bytes.Buffer b.Grow(1 + len(traceID)) @@ -107,6 +111,8 @@ func (rw PrefixReadWriter) WriteTraceSampled(traceID string, sampled bool) error return rw.db.Set(b.Bytes(), val, pebble.NoSync) } +// IsTraceSampled reads sampling decision from rw.db using a key consisting of rw.prefix and traceID. +// Returns ErrNotFound if trace sampling decision is not found. func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { var b bytes.Buffer b.Grow(1 + len(traceID)) @@ -123,6 +129,7 @@ func (rw PrefixReadWriter) IsTraceSampled(traceID string) (bool, error) { return item[0] == entryMetaTraceSampled, nil } +// DeleteTraceEvent deletes event associated with key consisting of rw.prefix, traceID and id from rw.db. func (rw PrefixReadWriter) DeleteTraceEvent(traceID, id string) error { var b bytes.Buffer b.Grow(1 + len(traceID) + 1 + len(id)) From 84f1a78e83037394ac189be656f411ae21065a48 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 00:57:10 +0000 Subject: [PATCH 177/184] Correct meter name --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index c41be072883..983fa319dfa 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -108,7 +108,7 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora } if sm.meterProvider != nil { - meter := sm.meterProvider.Meter("github.com/elastic/apm-server/x-pack/apm-server") + meter := sm.meterProvider.Meter("github.com/elastic/apm-server/sampling/eventstorage") lsmSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.lsm_size") valueLogSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.value_log_size") From 81cd61024679b78cc6845a9f7d060e8c56ceaf28 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 01:04:47 +0000 Subject: [PATCH 178/184] Add RW comment --- x-pack/apm-server/sampling/eventstorage/prefix.go | 2 +- x-pack/apm-server/sampling/eventstorage/rw.go | 12 +++++++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/x-pack/apm-server/sampling/eventstorage/prefix.go b/x-pack/apm-server/sampling/eventstorage/prefix.go index 43240b9a09e..99f18d02f04 100644 --- a/x-pack/apm-server/sampling/eventstorage/prefix.go +++ b/x-pack/apm-server/sampling/eventstorage/prefix.go @@ -38,7 +38,7 @@ func NewPrefixReadWriter(db db, prefix byte, codec Codec) PrefixReadWriter { return PrefixReadWriter{db: db, prefix: prefix, codec: codec} } -// PrefixReadWriter is a helper read writer that reads from and writes to db with a prefix byte in key. +// PrefixReadWriter is a helper RW that reads from and writes to db with a prefix byte in key. type PrefixReadWriter struct { db db prefix byte diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index e42ef8ce345..985f9e54ac2 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -25,6 +25,9 @@ type RW interface { DeleteTraceEvent(traceID, id string) error } +// SplitReadWriter is a RW that splits method calls to eventRW and decisionRW. +// - *TraceEvent* method calls are passed through to eventRW. +// - *TraceSampled method calls are passed through to decisionRW. type SplitReadWriter struct { eventRW, decisionRW RW } @@ -58,6 +61,8 @@ type storageLimitChecker interface { StorageLimit() uint64 } +// StorageLimitReadWriter is a RW that forbids Write* method calls based on disk usage and limit from storageLimitChecker. +// If there is no limit or limit is not reached, method calls are passed through to nextRW. type StorageLimitReadWriter struct { checker storageLimitChecker nextRW RW @@ -72,7 +77,7 @@ func NewStorageLimitReadWriter(checker storageLimitChecker, nextRW RW) StorageLi func (s StorageLimitReadWriter) checkStorageLimit() error { limit := s.checker.StorageLimit() - if limit != 0 { + if limit != 0 { // unlimited storage usage := s.checker.DiskUsage() if usage >= limit { return fmt.Errorf("%w (current: %d, limit %d)", ErrLimitReached, usage, limit) @@ -81,10 +86,12 @@ func (s StorageLimitReadWriter) checkStorageLimit() error { return nil } +// ReadTraceEvents passes through to s.nextRW.ReadTraceEvents. func (s StorageLimitReadWriter) ReadTraceEvents(traceID string, out *modelpb.Batch) error { return s.nextRW.ReadTraceEvents(traceID, out) } +// WriteTraceEvent passes through to s.nextRW.WriteTraceEvent only if storage limit is not reached. func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error { if err := s.checkStorageLimit(); err != nil { return err @@ -92,6 +99,7 @@ func (s StorageLimitReadWriter) WriteTraceEvent(traceID, id string, event *model return s.nextRW.WriteTraceEvent(traceID, id, event) } +// WriteTraceSampled passes through to s.nextRW.WriteTraceSampled only if storage limit is not reached. func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool) error { if err := s.checkStorageLimit(); err != nil { return err @@ -99,10 +107,12 @@ func (s StorageLimitReadWriter) WriteTraceSampled(traceID string, sampled bool) return s.nextRW.WriteTraceSampled(traceID, sampled) } +// IsTraceSampled passes through to s.nextRW.IsTraceSampled. func (s StorageLimitReadWriter) IsTraceSampled(traceID string) (bool, error) { return s.nextRW.IsTraceSampled(traceID) } +// DeleteTraceEvent passes through to s.nextRW.DeleteTraceEvent. func (s StorageLimitReadWriter) DeleteTraceEvent(traceID, id string) error { // Technically DeleteTraceEvent writes, but it should have a net effect of reducing disk usage return s.nextRW.DeleteTraceEvent(traceID, id) From d8d0b5529b671fae0a380c7070a3ea1244a6d321 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 01:07:44 +0000 Subject: [PATCH 179/184] Remove unused var --- x-pack/apm-server/main.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index ff0aee247e8..fb85aae1121 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -37,9 +37,8 @@ const ( var ( // db holds the database to use when tail-based sampling is configured. - dbMu sync.Mutex - db *eventstorage.StorageManager - dbMetricRegistration metric.Registration + dbMu sync.Mutex + db *eventstorage.StorageManager // samplerUUID is a UUID used to identify sampled trace ID documents // published by this process. From 545a0e2b124b4ed7e3c031136a47b43a73e4d3b7 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 09:00:18 +0000 Subject: [PATCH 180/184] Fix meter name again --- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 983fa319dfa..00a0cce0718 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -108,7 +108,7 @@ func NewStorageManager(storageDir string, opts ...StorageManagerOptions) (*Stora } if sm.meterProvider != nil { - meter := sm.meterProvider.Meter("github.com/elastic/apm-server/sampling/eventstorage") + meter := sm.meterProvider.Meter("github.com/elastic/apm-server/x-pack/apm-server/sampling/eventstorage") lsmSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.lsm_size") valueLogSizeGauge, _ := meter.Int64ObservableGauge("apm-server.sampling.tail.storage.value_log_size") From 31577e1e103159d48e9d5271eadd414d73f4c236 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 11:07:01 +0000 Subject: [PATCH 181/184] Add comment --- x-pack/apm-server/sampling/eventstorage/rw.go | 1 + x-pack/apm-server/sampling/eventstorage/storage.go | 1 + 2 files changed, 2 insertions(+) diff --git a/x-pack/apm-server/sampling/eventstorage/rw.go b/x-pack/apm-server/sampling/eventstorage/rw.go index 985f9e54ac2..be208cb3f0a 100644 --- a/x-pack/apm-server/sampling/eventstorage/rw.go +++ b/x-pack/apm-server/sampling/eventstorage/rw.go @@ -17,6 +17,7 @@ var ( ErrLimitReached = errors.New("configured storage limit reached") ) +// RW is a read writer interface that has methods to read and write trace event and sampling decisions. type RW interface { ReadTraceEvents(traceID string, out *modelpb.Batch) error WriteTraceEvent(traceID, id string, event *modelpb.APMEvent) error diff --git a/x-pack/apm-server/sampling/eventstorage/storage.go b/x-pack/apm-server/sampling/eventstorage/storage.go index 47f0b6fb59e..41b5a63302a 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage.go +++ b/x-pack/apm-server/sampling/eventstorage/storage.go @@ -12,6 +12,7 @@ import ( "github.com/elastic/apm-data/model/modelpb" ) +// db is a key value database interface. type db interface { Get(key []byte) ([]byte, io.Closer, error) Set(key, value []byte, opts *pebble.WriteOptions) error From afd5994633f86e57ba1d766607ca03a623c4cbc9 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 11:18:52 +0000 Subject: [PATCH 182/184] Use config.Storage not only for testing --- x-pack/apm-server/main.go | 1 + x-pack/apm-server/sampling/config.go | 3 +-- x-pack/apm-server/sampling/eventstorage/storage_manager.go | 1 + x-pack/apm-server/sampling/processor.go | 7 +------ x-pack/apm-server/sampling/processor_test.go | 3 +++ 5 files changed, 7 insertions(+), 8 deletions(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index fb85aae1121..07a1a0cdade 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -147,6 +147,7 @@ func newTailSamplingProcessor(args beater.ServerParams) (*sampling.Processor, er }, StorageConfig: sampling.StorageConfig{ DB: db, + Storage: db.NewReadWriter(), StorageLimit: tailSamplingConfig.StorageLimitParsed, TTL: tailSamplingConfig.TTL, DiscardOnWriteFailure: tailSamplingConfig.DiscardOnWriteFailure, diff --git a/x-pack/apm-server/sampling/config.go b/x-pack/apm-server/sampling/config.go index b5821167594..41d85265690 100644 --- a/x-pack/apm-server/sampling/config.go +++ b/x-pack/apm-server/sampling/config.go @@ -105,8 +105,7 @@ type StorageConfig struct { // DB will not be closed when the processor is closed. DB *eventstorage.StorageManager - // Storage overrides the default DB storage RW. - // For testing only. + // Storage is the read writer to DB. Storage eventstorage.RW // StorageLimit for the TBS database, in bytes. diff --git a/x-pack/apm-server/sampling/eventstorage/storage_manager.go b/x-pack/apm-server/sampling/eventstorage/storage_manager.go index 00a0cce0718..9abec2117f8 100644 --- a/x-pack/apm-server/sampling/eventstorage/storage_manager.go +++ b/x-pack/apm-server/sampling/eventstorage/storage_manager.go @@ -260,6 +260,7 @@ func (sm *StorageManager) close() error { // Reload flushes out pending disk writes to disk by reloading the database. // For testing only. +// Read writers created prior to Reload cannot be used and will need to be recreated via NewReadWriter. func (sm *StorageManager) Reload() error { if err := sm.close(); err != nil { return err diff --git a/x-pack/apm-server/sampling/processor.go b/x-pack/apm-server/sampling/processor.go index 32225b8aa4d..7a149ef2cea 100644 --- a/x-pack/apm-server/sampling/processor.go +++ b/x-pack/apm-server/sampling/processor.go @@ -65,18 +65,13 @@ func NewProcessor(config Config) (*Processor, error) { meter := config.MeterProvider.Meter("github.com/elastic/apm-server/x-pack/apm-server/sampling") - rw := config.Storage - if rw == nil { - rw = config.DB.NewReadWriter() - } - logger := logp.NewLogger(logs.Sampling) p := &Processor{ config: config, logger: logger, rateLimitedLogger: logger.WithOptions(logs.WithRateLimit(loggerRateLimit)), groups: newTraceGroups(meter, config.Policies, config.MaxDynamicServices, config.IngestRateDecayFactor), - eventStore: rw, + eventStore: config.Storage, stopping: make(chan struct{}), stopped: make(chan struct{}), } diff --git a/x-pack/apm-server/sampling/processor_test.go b/x-pack/apm-server/sampling/processor_test.go index 5dd9c61d267..fdc1e0d578b 100644 --- a/x-pack/apm-server/sampling/processor_test.go +++ b/x-pack/apm-server/sampling/processor_test.go @@ -682,6 +682,8 @@ func TestStorageLimit(t *testing.T) { err := config.DB.Reload() assert.NoError(t, err) + config.Storage = config.DB.NewReadWriter() + lsm, vlog := config.DB.Size() assert.Greater(t, lsm+vlog, int64(10<<10)) @@ -814,6 +816,7 @@ func newTempdirConfig(tb testing.TB) testConfig { }, StorageConfig: sampling.StorageConfig{ DB: db, + Storage: db.NewReadWriter(), TTL: 30 * time.Minute, StorageLimit: 0, // No storage limit. }, From 287fc4c3242070185244ec80249ff1da20ab4919 Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 11:20:01 +0000 Subject: [PATCH 183/184] Close db once in main --- x-pack/apm-server/main.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/apm-server/main.go b/x-pack/apm-server/main.go index 07a1a0cdade..3ffb9dc6757 100644 --- a/x-pack/apm-server/main.go +++ b/x-pack/apm-server/main.go @@ -239,7 +239,9 @@ func wrapServer(args beater.ServerParams, runServer beater.RunServerFunc) (beate // so it does not need to hold dbMu. func closeDB() error { if db != nil { - return db.Close() + err := db.Close() + db = nil + return err } return nil } From 5cb24d7fc36a9f7822dbd028701db68fd6d0ff5c Mon Sep 17 00:00:00 2001 From: Carson Ip Date: Wed, 29 Jan 2025 11:36:03 +0000 Subject: [PATCH 184/184] Add changelog --- changelogs/head.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/changelogs/head.asciidoc b/changelogs/head.asciidoc index c30c9d98f47..46aa533582f 100644 --- a/changelogs/head.asciidoc +++ b/changelogs/head.asciidoc @@ -18,3 +18,4 @@ https://github.com/elastic/apm-server/compare/8.16\...8.x[View commits] [float] ==== Added +- Tail-based sampling: Storage layer is rewritten to use Pebble database instead of BadgerDB. The new implementation offers a substantial throughput increase while consuming significantly less memory. Disk usage is lower and more stable. See PR for benchmark details. {pull}15235[15235]