Skip to content

Commit 8f1d12f

Browse files
feat: Use WAL Manager (#13491)
1 parent 08615bf commit 8f1d12f

File tree

4 files changed

+93
-102
lines changed

4 files changed

+93
-102
lines changed

pkg/ingester-rf1/flush.go

+22-17
Original file line numberDiff line numberDiff line change
@@ -4,15 +4,16 @@ import (
44
"crypto/rand"
55
"fmt"
66
"net/http"
7+
"strconv"
78
"time"
89

10+
"github.com/dustin/go-humanize"
911
"github.com/go-kit/log"
1012
"github.com/go-kit/log/level"
1113
"github.com/grafana/dskit/backoff"
1214
"github.com/grafana/dskit/ring"
1315
"github.com/grafana/dskit/runutil"
1416
"github.com/oklog/ulid"
15-
"github.com/prometheus/common/model"
1617
"golang.org/x/net/context"
1718

1819
"github.com/grafana/loki/v3/pkg/storage/wal"
@@ -77,18 +78,16 @@ func (i *Ingester) FlushHandler(w http.ResponseWriter, _ *http.Request) {
7778
}
7879

7980
type flushOp struct {
80-
from model.Time
81-
userID string
82-
fp model.Fingerprint
83-
immediate bool
81+
it *wal.PendingItem
82+
num int64
8483
}
8584

8685
func (o *flushOp) Key() string {
87-
return fmt.Sprintf("%s-%s-%v", o.userID, o.fp, o.immediate)
86+
return strconv.Itoa(int(o.num))
8887
}
8988

9089
func (o *flushOp) Priority() int64 {
91-
return -int64(o.from)
90+
return -o.num
9291
}
9392

9493
func (i *Ingester) flushLoop(j int) {
@@ -103,29 +102,35 @@ func (i *Ingester) flushLoop(j int) {
103102
if o == nil {
104103
return
105104
}
106-
op := o.(*flushCtx)
105+
op := o.(*flushOp)
106+
107+
start := time.Now()
108+
109+
// We'll use this to log the size of the segment that was flushed.
110+
n := humanize.Bytes(uint64(op.it.Writer.InputSize()))
107111

108112
err := i.flushOp(l, op)
113+
d := time.Since(start)
109114
if err != nil {
110-
level.Error(l).Log("msg", "failed to flush", "err", err)
111-
// Immediately re-queue another attempt at flushing this segment.
112-
// TODO: Add some backoff or something?
113-
i.flushQueues[j].Enqueue(op)
115+
level.Error(l).Log("msg", "failed to flush", "size", n, "duration", d, "err", err)
114116
} else {
115-
// Close the channel and trigger all waiting listeners to return
116-
// TODO: Figure out how to return an error if we want to?
117-
close(op.flushDone)
117+
level.Debug(l).Log("msg", "flushed", "size", n, "duration", d)
118+
}
119+
120+
op.it.Result.SetDone(err)
121+
if err = i.wal.Put(op.it); err != nil {
122+
level.Error(l).Log("msg", "failed to put back in WAL Manager", "err", err)
118123
}
119124
}
120125
}
121126

122-
func (i *Ingester) flushOp(l log.Logger, flushCtx *flushCtx) error {
127+
func (i *Ingester) flushOp(l log.Logger, op *flushOp) error {
123128
ctx, cancelFunc := context.WithCancel(context.Background())
124129
defer cancelFunc()
125130

126131
b := backoff.New(ctx, i.cfg.FlushOpBackoff)
127132
for b.Ongoing() {
128-
err := i.flushSegment(ctx, flushCtx.segmentWriter)
133+
err := i.flushSegment(ctx, op.it.Writer)
129134
if err == nil {
130135
break
131136
}

pkg/ingester-rf1/ingester.go

+25-70
Original file line numberDiff line numberDiff line change
@@ -174,22 +174,6 @@ type Interface interface {
174174
PrepareShutdown(w http.ResponseWriter, r *http.Request)
175175
}
176176

177-
type flushCtx struct {
178-
lock *sync.RWMutex
179-
flushDone chan struct{}
180-
newCtxAvailable chan struct{}
181-
segmentWriter *wal.SegmentWriter
182-
creationTime time.Time
183-
}
184-
185-
func (o *flushCtx) Key() string {
186-
return fmt.Sprintf("%d", o.creationTime.UnixNano())
187-
}
188-
189-
func (o *flushCtx) Priority() int64 {
190-
return -o.creationTime.UnixNano()
191-
}
192-
193177
// Ingester builds chunks for incoming log streams.
194178
type Ingester struct {
195179
services.Service
@@ -217,10 +201,11 @@ type Ingester struct {
217201

218202
// One queue per flush thread. Fingerprint is used to
219203
// pick a queue.
204+
numOps int64
220205
flushQueues []*util.PriorityQueue
221206
flushQueuesDone sync.WaitGroup
222207

223-
flushCtx *flushCtx
208+
wal *wal.Manager
224209

225210
limiter *Limiter
226211

@@ -268,7 +253,11 @@ func New(cfg Config, clientConfig client.Config,
268253
targetSizeStats.Set(int64(cfg.TargetChunkSize))
269254
metrics := newIngesterMetrics(registerer, metricsNamespace)
270255

271-
segmentWriter, err := wal.NewWalSegmentWriter()
256+
walManager, err := wal.NewManager(wal.Config{
257+
MaxAge: wal.DefaultMaxAge,
258+
MaxSegments: wal.DefaultMaxSegments,
259+
MaxSegmentSize: wal.DefaultMaxSegmentSize,
260+
})
272261
if err != nil {
273262
return nil, err
274263
}
@@ -291,12 +280,7 @@ func New(cfg Config, clientConfig client.Config,
291280
writeLogManager: writefailures.NewManager(logger, registerer, writeFailuresCfg, configs, "ingester_rf1"),
292281
customStreamsTracker: customStreamsTracker,
293282
readRing: readRing,
294-
flushCtx: &flushCtx{
295-
lock: &sync.RWMutex{},
296-
flushDone: make(chan struct{}),
297-
newCtxAvailable: make(chan struct{}),
298-
segmentWriter: segmentWriter,
299-
},
283+
wal: walManager,
300284
}
301285

302286
// TODO: change flush on shutdown
@@ -477,7 +461,6 @@ func (i *Ingester) running(ctx context.Context) error {
477461
func (i *Ingester) stopping(_ error) error {
478462
i.stopIncomingRequests()
479463
var errs util.MultiError
480-
// errs.Add(i.wal.Stop())
481464

482465
//if i.flushOnShutdownSwitch.Get() {
483466
// i.lifecycler.SetFlushOnShutdown(true)
@@ -567,30 +550,18 @@ func (i *Ingester) loop() {
567550
}
568551

569552
func (i *Ingester) doFlushTick() {
570-
i.flushCtx.lock.Lock()
571-
572-
// i.logger.Log("msg", "starting periodic flush")
573-
// Stop new chunks being written while we swap destinations - we'll never unlock as this flushctx can no longer be used.
574-
currentFlushCtx := i.flushCtx
575-
576-
// APIs become unblocked after resetting flushCtx
577-
segmentWriter, err := wal.NewWalSegmentWriter()
578-
if err != nil {
579-
// TODO: handle this properly
580-
panic(err)
581-
}
582-
i.flushCtx = &flushCtx{
583-
lock: &sync.RWMutex{},
584-
flushDone: make(chan struct{}),
585-
newCtxAvailable: make(chan struct{}),
586-
segmentWriter: segmentWriter,
587-
}
588-
close(currentFlushCtx.newCtxAvailable) // Broadcast to all waiters that they can now fetch a new flushCtx. Small chance of a race but if they re-fetch the old one, they'll just check again immediately.
589-
// Flush the finished context in the background & then notify watching API requests
590-
// TODO: use multiple flush queues if required
591-
// Don't write empty segments if there is nothing to write.
592-
if currentFlushCtx.segmentWriter.InputSize() > 0 {
593-
i.flushQueues[0].Enqueue(currentFlushCtx)
553+
for {
554+
// Keep adding ops to the queue until there are no more.
555+
it, _ := i.wal.NextPending()
556+
if it == nil {
557+
break
558+
}
559+
i.numOps++
560+
flushQueueIndex := i.numOps % int64(i.cfg.ConcurrentFlushes)
561+
i.flushQueues[flushQueueIndex].Enqueue(&flushOp{
562+
num: i.numOps,
563+
it: it,
564+
})
594565
}
595566
}
596567

@@ -796,27 +767,11 @@ func (i *Ingester) Push(ctx context.Context, req *logproto.PushRequest) (*logpro
796767
return &logproto.PushResponse{}, err
797768
}
798769

799-
// Fetch a flush context and try to acquire the RLock
800-
// The only time the Write Lock is held is when this context is no longer usable and a new one is being created.
801-
// In this case, we need to re-read i.flushCtx in order to fetch the new one as soon as it's available.
802-
// The newCtxAvailable chan is closed as soon as the new one is available to avoid a busy loop.
803-
currentFlushCtx := i.flushCtx
804-
for !currentFlushCtx.lock.TryRLock() {
805-
select {
806-
case <-currentFlushCtx.newCtxAvailable:
807-
case <-ctx.Done():
808-
return &logproto.PushResponse{}, ctx.Err()
809-
}
810-
currentFlushCtx = i.flushCtx
811-
}
812-
err = instance.Push(ctx, req, currentFlushCtx)
813-
currentFlushCtx.lock.RUnlock()
814-
select {
815-
case <-ctx.Done():
816-
return &logproto.PushResponse{}, ctx.Err()
817-
case <-currentFlushCtx.flushDone:
818-
return &logproto.PushResponse{}, err
770+
if err = instance.Push(ctx, i.wal, req); err != nil {
771+
return nil, err
819772
}
773+
774+
return &logproto.PushResponse{}, nil
820775
}
821776

822777
// GetStreamRates returns a response containing all streams and their current rate
@@ -851,7 +806,7 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { /
851806
inst, ok = i.instances[instanceID]
852807
if !ok {
853808
var err error
854-
inst, err = newInstance(&i.cfg, i.periodicConfigs, instanceID, i.limiter, i.tenantConfigs, i.metrics, i.streamRateCalculator, i.writeLogManager, i.customStreamsTracker)
809+
inst, err = newInstance(&i.cfg, i.periodicConfigs, instanceID, i.limiter, i.tenantConfigs, i.metrics, i.streamRateCalculator, i.writeLogManager, i.customStreamsTracker, i.logger)
855810
if err != nil {
856811
return nil, err
857812
}

pkg/ingester-rf1/instance.go

+25-7
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ import (
77
"net/http"
88
"sync"
99

10+
"github.com/go-kit/log"
1011
"github.com/go-kit/log/level"
1112
"github.com/grafana/dskit/httpgrpc"
1213
"github.com/prometheus/client_golang/prometheus"
@@ -23,6 +24,7 @@ import (
2324
"github.com/grafana/loki/v3/pkg/logql/syntax"
2425
"github.com/grafana/loki/v3/pkg/runtime"
2526
"github.com/grafana/loki/v3/pkg/storage/config"
27+
"github.com/grafana/loki/v3/pkg/storage/wal"
2628
"github.com/grafana/loki/v3/pkg/util/constants"
2729
util_log "github.com/grafana/loki/v3/pkg/util/log"
2830
"github.com/grafana/loki/v3/pkg/validation"
@@ -70,6 +72,7 @@ type instance struct {
7072
// tailers map[uint32]*tailer
7173
tailerMtx sync.RWMutex
7274

75+
logger log.Logger
7376
limiter *Limiter
7477
streamCountLimiter *streamCountLimiter
7578
ownedStreamsSvc *ownedStreamService
@@ -87,10 +90,10 @@ type instance struct {
8790
customStreamsTracker push.UsageTracker
8891
}
8992

90-
func (i *instance) Push(ctx context.Context, req *logproto.PushRequest, flushCtx *flushCtx) error {
93+
func (i *instance) Push(ctx context.Context, w *wal.Manager, req *logproto.PushRequest) error {
9194
rateLimitWholeStream := i.limiter.limits.ShardStreams(i.instanceID).Enabled
9295

93-
var appendErr error
96+
results := make([]*wal.AppendResult, 0, len(req.Streams))
9497
for _, reqStream := range req.Streams {
9598
s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels,
9699
func() (*stream, error) {
@@ -102,13 +105,27 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest, flushCtx
102105
},
103106
)
104107
if err != nil {
105-
appendErr = err
106-
continue
108+
return err
107109
}
110+
_, res, err := s.Push(ctx, w, reqStream.Entries, rateLimitWholeStream, i.customStreamsTracker)
111+
if err != nil {
112+
return err
113+
}
114+
results = append(results, res)
115+
}
108116

109-
_, appendErr = s.Push(ctx, reqStream.Entries, rateLimitWholeStream, i.customStreamsTracker, flushCtx)
117+
for _, result := range results {
118+
select {
119+
case <-ctx.Done():
120+
return ctx.Err()
121+
case <-result.Done():
122+
if err := result.Err(); err != nil {
123+
return err
124+
}
125+
}
110126
}
111-
return appendErr
127+
128+
return nil
112129
}
113130

114131
func newInstance(
@@ -121,8 +138,8 @@ func newInstance(
121138
streamRateCalculator *StreamRateCalculator,
122139
writeFailures *writefailures.Manager,
123140
customStreamsTracker push.UsageTracker,
141+
logger log.Logger,
124142
) (*instance, error) {
125-
fmt.Println("new instance for", instanceID)
126143
invertedIndex, err := index.NewMultiInvertedIndex(periodConfigs, uint32(cfg.IndexShards))
127144
if err != nil {
128145
return nil, err
@@ -141,6 +158,7 @@ func newInstance(
141158
streamsRemovedTotal: streamsRemovedTotal.WithLabelValues(instanceID),
142159
//
143160
//tailers: map[uint32]*tailer{},
161+
logger: logger,
144162
limiter: limiter,
145163
streamCountLimiter: newStreamCountLimiter(instanceID, streams.Len, limiter, ownedStreamsSvc),
146164
ownedStreamsSvc: ownedStreamsSvc,

pkg/ingester-rf1/stream.go

+21-8
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/grafana/loki/v3/pkg/distributor/writefailures"
1818
"github.com/grafana/loki/v3/pkg/loghttp/push"
1919
"github.com/grafana/loki/v3/pkg/logproto"
20+
"github.com/grafana/loki/v3/pkg/storage/wal"
2021
"github.com/grafana/loki/v3/pkg/util/flagext"
2122
"github.com/grafana/loki/v3/pkg/validation"
2223
)
@@ -130,21 +131,24 @@ func (s *stream) consumeChunk(_ context.Context, _ *logproto.Chunk) error {
130131

131132
func (s *stream) Push(
132133
ctx context.Context,
134+
wal *wal.Manager,
133135
entries []logproto.Entry,
134136
// Whether nor not to ingest all at once or not. It is a per-tenant configuration.
135137
rateLimitWholeStream bool,
136138

137139
usageTracker push.UsageTracker,
138-
flushCtx *flushCtx,
139-
) (int, error) {
140+
) (int, *wal.AppendResult, error) {
140141
toStore, invalid := s.validateEntries(ctx, entries, rateLimitWholeStream, usageTracker)
141142
if rateLimitWholeStream && hasRateLimitErr(invalid) {
142-
return 0, errorForFailedEntries(s, invalid, len(entries))
143+
return 0, nil, errorForFailedEntries(s, invalid, len(entries))
143144
}
144145

145-
bytesAdded := s.storeEntries(ctx, toStore, usageTracker, flushCtx)
146+
bytesAdded, res, err := s.storeEntries(ctx, wal, toStore, usageTracker)
147+
if err != nil {
148+
return 0, nil, err
149+
}
146150

147-
return bytesAdded, errorForFailedEntries(s, invalid, len(entries))
151+
return bytesAdded, res, errorForFailedEntries(s, invalid, len(entries))
148152
}
149153

150154
func errorForFailedEntries(s *stream, failedEntriesWithError []entryWithError, totalEntries int) error {
@@ -195,7 +199,7 @@ func hasRateLimitErr(errs []entryWithError) bool {
195199
return ok
196200
}
197201

198-
func (s *stream) storeEntries(ctx context.Context, entries []*logproto.Entry, usageTracker push.UsageTracker, flushCtx *flushCtx) int {
202+
func (s *stream) storeEntries(ctx context.Context, w *wal.Manager, entries []*logproto.Entry, usageTracker push.UsageTracker) (int, *wal.AppendResult, error) {
199203
if sp := opentracing.SpanFromContext(ctx); sp != nil {
200204
sp.LogKV("event", "stream started to store entries", "labels", s.labelsString)
201205
defer sp.LogKV("event", "stream finished to store entries")
@@ -213,9 +217,18 @@ func (s *stream) storeEntries(ctx context.Context, entries []*logproto.Entry, us
213217

214218
bytesAdded += len(entries[i].Line)
215219
}
216-
flushCtx.segmentWriter.Append(s.tenant, s.labels.String(), s.labels, entries)
220+
221+
res, err := w.Append(wal.AppendRequest{
222+
TenantID: s.tenant,
223+
Labels: s.labels,
224+
LabelsStr: s.labels.String(),
225+
Entries: entries,
226+
})
227+
if err != nil {
228+
return 0, nil, err
229+
}
217230
s.reportMetrics(ctx, outOfOrderSamples, outOfOrderBytes, 0, 0, usageTracker)
218-
return bytesAdded
231+
return bytesAdded, res, nil
219232
}
220233

221234
func (s *stream) validateEntries(ctx context.Context, entries []logproto.Entry, rateLimitWholeStream bool, usageTracker push.UsageTracker) ([]*logproto.Entry, []entryWithError) {

0 commit comments

Comments
 (0)