Skip to content

Commit a7240b4

Browse files
authored
Fix ES bulk processor commit timeout (#3696)
1 parent 23a2c4e commit a7240b4

File tree

6 files changed

+54
-40
lines changed

6 files changed

+54
-40
lines changed

common/persistence/visibility/store/elasticsearch/processor.go

+8-6
Original file line numberDiff line numberDiff line change
@@ -199,20 +199,18 @@ func (p *processorImpl) bulkBeforeAction(_ int64, requests []elastic.BulkableReq
199199
p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorRequests.GetMetricName()).Record(int64(len(requests)))
200200
p.metricsHandler.Histogram(metrics.ElasticsearchBulkProcessorBulkSize.GetMetricName(), metrics.ElasticsearchBulkProcessorBulkSize.GetMetricUnit()).
201201
Record(int64(len(requests)))
202-
p.metricsHandler.Histogram(metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricName(), metrics.ElasticsearchBulkProcessorBulkSize.GetMetricUnit()).
203-
Record(int64(p.mapToAckFuture.Len() - len(requests)))
204202

205203
for _, request := range requests {
206204
visibilityTaskKey := p.extractVisibilityTaskKey(request)
207205
if visibilityTaskKey == "" {
208206
continue
209207
}
210208
_, _, _ = p.mapToAckFuture.GetAndDo(visibilityTaskKey, func(key interface{}, value interface{}) error {
211-
future, ok := value.(*ackFuture)
209+
ackF, ok := value.(*ackFuture)
212210
if !ok {
213211
p.logger.Fatal(fmt.Sprintf("mapToAckFuture has item of a wrong type %T (%T expected).", value, &ackFuture{}), tag.Value(key))
214212
}
215-
future.recordStart(p.metricsHandler)
213+
ackF.recordStart(p.metricsHandler)
216214
return nil
217215
})
218216
}
@@ -286,6 +284,10 @@ func (p *processorImpl) bulkAfterAction(_ int64, requests []elastic.BulkableRequ
286284
p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorRetries.GetMetricName()).Record(1, metrics.HttpStatusTag(responseItem.Status))
287285
}
288286
}
287+
288+
// Record how many documents are waiting to be flushed to Elasticsearch after this bulk is committed.
289+
p.metricsHandler.Histogram(metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricName(), metrics.ElasticsearchBulkProcessorBulkSize.GetMetricUnit()).
290+
Record(int64(p.mapToAckFuture.Len()))
289291
}
290292

291293
func (p *processorImpl) buildResponseIndex(response *elastic.BulkResponse) map[string]*elastic.BulkResponseItem {
@@ -307,12 +309,12 @@ func (p *processorImpl) buildResponseIndex(response *elastic.BulkResponse) map[s
307309
func (p *processorImpl) notifyResult(visibilityTaskKey string, ack bool) {
308310
// Use RemoveIf here to prevent race condition with de-dup logic in Add method.
309311
_ = p.mapToAckFuture.RemoveIf(visibilityTaskKey, func(key interface{}, value interface{}) bool {
310-
future, ok := value.(*ackFuture)
312+
ackF, ok := value.(*ackFuture)
311313
if !ok {
312314
p.logger.Fatal(fmt.Sprintf("mapToAckFuture has item of a wrong type %T (%T expected).", value, &ackFuture{}), tag.ESKey(visibilityTaskKey))
313315
}
314316

315-
future.done(ack, p.metricsHandler)
317+
ackF.done(ack, p.metricsHandler)
316318
return true
317319
})
318320
}

common/persistence/visibility/store/elasticsearch/processor_test.go

+13-7
Original file line numberDiff line numberDiff line change
@@ -244,6 +244,12 @@ func (s *processorSuite) TestBulkAfterAction_Ack() {
244244
Items: []map[string]*elastic.BulkResponseItem{mSuccess},
245245
}
246246

247+
queuedRequestHistogram := metrics.NewMockHistogramIface(s.controller)
248+
s.mockMetricHandler.EXPECT().Histogram(
249+
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricName(),
250+
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricUnit(),
251+
).Return(queuedRequestHistogram)
252+
queuedRequestHistogram.EXPECT().Record(int64(0))
247253
s.mockMetricHandler.EXPECT().Timer(metrics.ElasticsearchBulkProcessorRequestLatency.GetMetricName()).Return(metrics.NoopTimerMetricFunc)
248254
mapVal := newAckFuture()
249255
s.esProcessor.mapToAckFuture.Put(testKey, mapVal)
@@ -287,6 +293,12 @@ func (s *processorSuite) TestBulkAfterAction_Nack() {
287293
Items: []map[string]*elastic.BulkResponseItem{mFailed},
288294
}
289295

296+
queuedRequestHistogram := metrics.NewMockHistogramIface(s.controller)
297+
s.mockMetricHandler.EXPECT().Histogram(
298+
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricName(),
299+
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricUnit(),
300+
).Return(queuedRequestHistogram)
301+
queuedRequestHistogram.EXPECT().Record(int64(0))
290302
s.mockMetricHandler.EXPECT().Timer(metrics.ElasticsearchBulkProcessorRequestLatency.GetMetricName()).Return(metrics.NoopTimerMetricFunc)
291303
mapVal := newAckFuture()
292304
s.esProcessor.mapToAckFuture.Put(testKey, mapVal)
@@ -352,12 +364,6 @@ func (s *processorSuite) TestBulkBeforeAction() {
352364
metrics.ElasticsearchBulkProcessorBulkSize.GetMetricUnit(),
353365
).Return(bulkSizeHistogram)
354366
bulkSizeHistogram.EXPECT().Record(int64(1))
355-
queuedRequestHistorgram := metrics.NewMockHistogramIface(s.controller)
356-
s.mockMetricHandler.EXPECT().Histogram(
357-
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricName(),
358-
metrics.ElasticsearchBulkProcessorQueuedRequests.GetMetricUnit(),
359-
).Return(queuedRequestHistorgram)
360-
queuedRequestHistorgram.EXPECT().Record(int64(0))
361367
s.mockMetricHandler.EXPECT().Timer(metrics.ElasticsearchBulkProcessorWaitAddLatency.GetMetricName()).Return(metrics.NoopTimerMetricFunc)
362368
s.mockMetricHandler.EXPECT().Timer(metrics.ElasticsearchBulkProcessorWaitStartLatency.GetMetricName()).Return(metrics.NoopTimerMetricFunc)
363369
mapVal := newAckFuture()
@@ -484,7 +490,7 @@ func (s *processorSuite) TestErrorReasonFromResponse() {
484490
func (s *processorSuite) Test_End2End() {
485491
docsCount := 1000
486492
parallelFactor := 10
487-
version := int64(2208) //random
493+
version := int64(2208) // random
488494

489495
request := &client.BulkableRequest{}
490496
bulkIndexRequests := make([]elastic.BulkableRequest, docsCount)

common/persistence/visibility/store/elasticsearch/visibility_store.go

+18-21
Original file line numberDiff line numberDiff line change
@@ -49,11 +49,10 @@ import (
4949
"go.temporal.io/server/common/persistence/visibility/store/elasticsearch/client"
5050
"go.temporal.io/server/common/persistence/visibility/store/query"
5151
"go.temporal.io/server/common/searchattribute"
52-
"go.temporal.io/server/common/util"
5352
)
5453

5554
const (
56-
persistenceName = "elasticsearch"
55+
PersistenceName = "elasticsearch"
5756

5857
delimiter = "~"
5958
pointInTimeKeepAliveInterval = "1m"
@@ -128,7 +127,7 @@ func (s *visibilityStore) Close() {
128127
}
129128

130129
func (s *visibilityStore) GetName() string {
131-
return persistenceName
130+
return PersistenceName
132131
}
133132

134133
func (s *visibilityStore) RecordWorkflowExecutionStarted(
@@ -232,32 +231,30 @@ func (s *visibilityStore) addBulkIndexRequestAndWait(
232231
}
233232

234233
func (s *visibilityStore) addBulkRequestAndWait(
235-
ctx context.Context,
234+
_ context.Context,
236235
bulkRequest *client.BulkableRequest,
237236
visibilityTaskKey string,
238237
) error {
239238
s.checkProcessor()
240239

241240
// Add method is blocking. If bulk processor is busy flushing previous bulk, request will wait here.
242-
// Therefore, ackTimeoutTimer in fact wait for request to be committed after it was added to bulk processor.
243-
// TODO: this also means ctx is not respected if bulk processor is busy. Shall we make Add non-blocking or
244-
// respecting the context?
245-
future := s.processor.Add(bulkRequest, visibilityTaskKey)
246-
247-
ackTimeout := s.processorAckTimeout()
248-
if deadline, ok := ctx.Deadline(); ok {
249-
ackTimeout = util.Min(ackTimeout, time.Until(deadline))
250-
}
251-
subCtx, subCtxCancelFn := context.WithTimeout(context.Background(), ackTimeout)
252-
defer subCtxCancelFn()
253-
254-
ack, err := future.Get(subCtx)
241+
ackF := s.processor.Add(bulkRequest, visibilityTaskKey)
255242

256-
if errors.Is(err, context.DeadlineExceeded) {
257-
return &persistence.TimeoutError{Msg: fmt.Sprintf("visibility task %s timedout waiting for ACK after %v", visibilityTaskKey, s.processorAckTimeout())}
258-
}
243+
// processorAckTimeout is a maximum duration for bulk processor to commit the bulk and unblock the `ackF`.
244+
// Default value is 30s and this timeout should never have happened,
245+
// because Elasticsearch must process a bulk within 30s.
246+
// Parent context is not respected here because it has shorter timeout (3s),
247+
// which might already expired here due to wait at Add method above.
248+
ctx, cancel := context.WithTimeout(context.Background(), s.processorAckTimeout())
249+
defer cancel()
250+
ack, err := ackF.Get(ctx)
259251

260252
if err != nil {
253+
if errors.Is(err, context.DeadlineExceeded) {
254+
return &persistence.TimeoutError{Msg: fmt.Sprintf("visibility task %s timed out waiting for ACK after %v", visibilityTaskKey, s.processorAckTimeout())}
255+
}
256+
// Returns non-retryable Internal error here because these errors are unexpected.
257+
// Visibility task processor retries all errors though, therefore new request will be generated for the same task.
261258
return serviceerror.NewInternal(fmt.Sprintf("visibility task %s received error %v", visibilityTaskKey, err))
262259
}
263260

@@ -900,7 +897,7 @@ func (s *visibilityStore) parseESDoc(docID string, docSource json.RawMessage, sa
900897
// Very important line. See finishParseJSONValue bellow.
901898
d.UseNumber()
902899
if err := d.Decode(&sourceMap); err != nil {
903-
s.metricsHandler.Counter(metrics.ElasticsearchDocumentParseFailuresCount.GetMetricName()) //.Record(1)
900+
s.metricsHandler.Counter(metrics.ElasticsearchDocumentParseFailuresCount.GetMetricName()).Record(1)
904901
return nil, serviceerror.NewInternal(fmt.Sprintf("Unable to unmarshal JSON from Elasticsearch document(%s): %v", docID, err))
905902
}
906903

service/history/configs/config.go

+2-2
Original file line numberDiff line numberDiff line change
@@ -498,14 +498,14 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int32, isAdvancedVis
498498
SearchAttributesTotalSizeLimit: dc.GetIntPropertyFilteredByNamespace(dynamicconfig.SearchAttributesTotalSizeLimit, 40*1024),
499499
IndexerConcurrency: dc.GetIntProperty(dynamicconfig.WorkerIndexerConcurrency, 100),
500500
ESProcessorNumOfWorkers: dc.GetIntProperty(dynamicconfig.WorkerESProcessorNumOfWorkers, 1),
501-
// Should be not greater than NumberOfShards(512)/NumberOfHistoryNodes(4) * VisibilityTaskWorkerCount(10)/ESProcessorNumOfWorkers(1) divided by workflow distribution factor (2 at least).
501+
// Should not be greater than number of visibility task queue workers VisibilityProcessorSchedulerWorkerCount (default 512)
502502
// Otherwise, visibility queue processors won't be able to fill up bulk with documents (even under heavy load) and bulk will flush due to interval, not number of actions.
503503
ESProcessorBulkActions: dc.GetIntProperty(dynamicconfig.WorkerESProcessorBulkActions, 500),
504504
// 16MB - just a sanity check. With ES document size ~1Kb it should never be reached.
505505
ESProcessorBulkSize: dc.GetIntProperty(dynamicconfig.WorkerESProcessorBulkSize, 16*1024*1024),
506506
// Bulk processor will flush every this interval regardless of last flush due to bulk actions.
507507
ESProcessorFlushInterval: dc.GetDurationProperty(dynamicconfig.WorkerESProcessorFlushInterval, 1*time.Second),
508-
ESProcessorAckTimeout: dc.GetDurationProperty(dynamicconfig.WorkerESProcessorAckTimeout, 1*time.Minute),
508+
ESProcessorAckTimeout: dc.GetDurationProperty(dynamicconfig.WorkerESProcessorAckTimeout, 30*time.Second),
509509

510510
EnableCrossNamespaceCommands: dc.GetBoolProperty(dynamicconfig.EnableCrossNamespaceCommands, true),
511511
EnableActivityEagerExecution: dc.GetBoolPropertyFnWithNamespaceFilter(dynamicconfig.EnableActivityEagerExecution, false),

service/history/visibilityQueueTaskExecutor.go

+11-3
Original file line numberDiff line numberDiff line change
@@ -319,10 +319,10 @@ func (t *visibilityQueueTaskExecutor) upsertExecution(
319319
}
320320

321321
func (t *visibilityQueueTaskExecutor) processCloseExecution(
322-
ctx context.Context,
322+
parentCtx context.Context,
323323
task *tasks.CloseExecutionVisibilityTask,
324324
) (retError error) {
325-
ctx, cancel := context.WithTimeout(ctx, taskTimeout)
325+
ctx, cancel := context.WithTimeout(parentCtx, taskTimeout)
326326
defer cancel()
327327

328328
namespaceEntry, err := t.shard.GetNamespaceRegistry().GetNamespaceByID(namespace.ID(task.GetNamespaceID()))
@@ -396,8 +396,13 @@ func (t *visibilityQueueTaskExecutor) processCloseExecution(
396396
return err
397397
}
398398

399+
// Elasticsearch bulk processor doesn't respect context timeout
400+
// because under heavy load bulk flush might take longer than taskTimeout.
401+
// Therefore, ctx timeout might be already expired
402+
// and parentCtx (which doesn't have timeout) must be used everywhere bellow.
403+
399404
if t.enableCloseWorkflowCleanup(namespaceEntry.Name().String()) {
400-
return t.cleanupExecutionInfo(ctx, task)
405+
return t.cleanupExecutionInfo(parentCtx, task)
401406
}
402407
return nil
403408
}
@@ -500,6 +505,9 @@ func (t *visibilityQueueTaskExecutor) cleanupExecutionInfo(
500505
ctx context.Context,
501506
task *tasks.CloseExecutionVisibilityTask,
502507
) (retError error) {
508+
ctx, cancel := context.WithTimeout(ctx, taskTimeout)
509+
defer cancel()
510+
503511
weContext, release, err := getWorkflowExecutionContextForTask(ctx, t.cache, task)
504512
if err != nil {
505513
return err

service/worker/deletenamespace/reclaimresources/activities.go

+2-1
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@ import (
3737
"go.temporal.io/server/common/namespace"
3838
"go.temporal.io/server/common/persistence"
3939
"go.temporal.io/server/common/persistence/visibility/manager"
40+
"go.temporal.io/server/common/persistence/visibility/store/elasticsearch"
4041
"go.temporal.io/server/service/worker/deletenamespace/errors"
4142
)
4243

@@ -63,7 +64,7 @@ func NewActivities(
6364
}
6465
}
6566
func (a *Activities) IsAdvancedVisibilityActivity(_ context.Context) (bool, error) {
66-
return strings.Contains(a.visibilityManager.GetName(), "elasticsearch"), nil
67+
return strings.Contains(a.visibilityManager.GetName(), elasticsearch.PersistenceName), nil
6768
}
6869

6970
func (a *Activities) CountExecutionsAdvVisibilityActivity(ctx context.Context, nsID namespace.ID, nsName namespace.Name) (int64, error) {

0 commit comments

Comments
 (0)