go.temporal.io/server@v1.23.0/common/persistence/visibility/store/elasticsearch/processor.go (about) 1 // The MIT License 2 // 3 // Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. 4 // 5 // Copyright (c) 2020 Uber Technologies, Inc. 6 // 7 // Permission is hereby granted, free of charge, to any person obtaining a copy 8 // of this software and associated documentation files (the "Software"), to deal 9 // in the Software without restriction, including without limitation the rights 10 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 11 // copies of the Software, and to permit persons to whom the Software is 12 // furnished to do so, subject to the following conditions: 13 // 14 // The above copyright notice and this permission notice shall be included in 15 // all copies or substantial portions of the Software. 16 // 17 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 18 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 19 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 20 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 21 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 22 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 23 // THE SOFTWARE. 24 25 // TODO: enable this after https://github.com/golang/mock/issues/621 26 // mockgen -copyright_file ../../../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination processor_mock.go 27 28 package elasticsearch 29 30 import ( 31 "context" 32 "encoding/json" 33 "errors" 34 "fmt" 35 "strings" 36 "sync" 37 "sync/atomic" 38 "time" 39 40 "github.com/dgryski/go-farm" 41 "github.com/olivere/elastic/v7" 42 43 "go.temporal.io/server/common" 44 "go.temporal.io/server/common/collection" 45 "go.temporal.io/server/common/dynamicconfig" 46 "go.temporal.io/server/common/future" 47 "go.temporal.io/server/common/log" 48 "go.temporal.io/server/common/log/tag" 49 "go.temporal.io/server/common/metrics" 50 "go.temporal.io/server/common/persistence/visibility/store/elasticsearch/client" 51 "go.temporal.io/server/common/searchattribute" 52 ) 53 54 type ( 55 // Processor is interface for Elasticsearch bulk processor 56 Processor interface { 57 // Add request to bulk processor. 58 Add(request *client.BulkableRequest, visibilityTaskKey string) *future.FutureImpl[bool] 59 Start() 60 Stop() 61 } 62 63 // processorImpl implements Processor, it's an agent of elastic.BulkProcessor 64 processorImpl struct { 65 status int32 66 bulkProcessor client.BulkProcessor 67 bulkProcessorParameters *client.BulkProcessorParameters 68 client client.Client 69 mapToAckFuture collection.ConcurrentTxMap // used to map ES request to ack channel 70 logger log.Logger 71 metricsHandler metrics.Handler 72 indexerConcurrency uint32 73 shutdownLock sync.RWMutex 74 } 75 76 // ProcessorConfig contains all configs for processor 77 ProcessorConfig struct { 78 IndexerConcurrency dynamicconfig.IntPropertyFn 79 // TODO: remove ESProcessor prefix 80 ESProcessorNumOfWorkers dynamicconfig.IntPropertyFn 81 ESProcessorBulkActions dynamicconfig.IntPropertyFn // max number of requests in bulk 82 ESProcessorBulkSize dynamicconfig.IntPropertyFn // max total size of bytes in bulk 83 ESProcessorFlushInterval dynamicconfig.DurationPropertyFn 84 85 ESProcessorAckTimeout dynamicconfig.DurationPropertyFn 86 } 87 88 ackFuture struct { // value of processorImpl.mapToAckFuture 89 future *future.FutureImpl[bool] 90 createdAt time.Time // Time when request was created (used to report metrics). 91 addedAt atomic.Value // of time.Time // Time when request was added to bulk processor (used to report metrics). 92 startedAt time.Time // Time when request was sent to Elasticsearch by bulk processor (used to report metrics). 93 } 94 ) 95 96 var _ Processor = (*processorImpl)(nil) 97 98 const ( 99 visibilityProcessorName = "visibility-processor" 100 ) 101 102 var ( 103 errVisibilityShutdown = errors.New("visiblity processor was shut down") 104 ) 105 106 // NewProcessor create new processorImpl 107 func NewProcessor( 108 cfg *ProcessorConfig, 109 esClient client.Client, 110 logger log.Logger, 111 metricsHandler metrics.Handler, 112 ) *processorImpl { 113 114 p := &processorImpl{ 115 status: common.DaemonStatusInitialized, 116 client: esClient, 117 logger: log.With(logger, tag.ComponentIndexerESProcessor), 118 metricsHandler: metricsHandler.WithTags(metrics.OperationTag(metrics.ElasticsearchBulkProcessor)), 119 indexerConcurrency: uint32(cfg.IndexerConcurrency()), 120 bulkProcessorParameters: &client.BulkProcessorParameters{ 121 Name: visibilityProcessorName, 122 NumOfWorkers: cfg.ESProcessorNumOfWorkers(), 123 BulkActions: cfg.ESProcessorBulkActions(), 124 BulkSize: cfg.ESProcessorBulkSize(), 125 FlushInterval: cfg.ESProcessorFlushInterval(), 126 }, 127 } 128 p.bulkProcessorParameters.AfterFunc = p.bulkAfterAction 129 p.bulkProcessorParameters.BeforeFunc = p.bulkBeforeAction 130 return p 131 } 132 133 func (p *processorImpl) Start() { 134 if !atomic.CompareAndSwapInt32( 135 &p.status, 136 common.DaemonStatusInitialized, 137 common.DaemonStatusStarted, 138 ) { 139 return 140 } 141 142 var err error 143 p.mapToAckFuture = collection.NewShardedConcurrentTxMap(1024, p.hashFn) 144 p.bulkProcessor, err = p.client.RunBulkProcessor(context.Background(), p.bulkProcessorParameters) 145 if err != nil { 146 p.logger.Fatal("Unable to start Elasticsearch processor.", tag.LifeCycleStartFailed, tag.Error(err)) 147 } 148 } 149 150 func (p *processorImpl) Stop() { 151 if !atomic.CompareAndSwapInt32( 152 &p.status, 153 common.DaemonStatusStarted, 154 common.DaemonStatusStopped, 155 ) { 156 return 157 } 158 159 p.shutdownLock.Lock() 160 defer p.shutdownLock.Unlock() 161 162 err := p.bulkProcessor.Stop() 163 if err != nil { 164 // This could happen if ES is down when we're trying to shut down the server. 165 p.logger.Error("Unable to stop Elasticsearch processor.", tag.LifeCycleStopFailed, tag.Error(err)) 166 return 167 } 168 } 169 170 func (p *processorImpl) hashFn(key interface{}) uint32 { 171 id, ok := key.(string) 172 if !ok { 173 return 0 174 } 175 idBytes := []byte(id) 176 hash := farm.Hash32(idBytes) 177 return hash % p.indexerConcurrency 178 } 179 180 // Add request to the bulk and return a future object which will receive ack signal when request is processed. 181 func (p *processorImpl) Add(request *client.BulkableRequest, visibilityTaskKey string) *future.FutureImpl[bool] { 182 newFuture := newAckFuture() // Create future first to measure impact of following RWLock on latency 183 184 p.shutdownLock.RLock() 185 defer p.shutdownLock.RUnlock() 186 187 if atomic.LoadInt32(&p.status) == common.DaemonStatusStopped { 188 p.logger.Warn("Rejecting ES request for visibility task key because processor has been shut down.", tag.Key(visibilityTaskKey), tag.ESDocID(request.ID), tag.Value(request.Doc)) 189 newFuture.future.Set(false, errVisibilityShutdown) 190 return newFuture.future 191 } 192 193 _, isDup, _ := p.mapToAckFuture.PutOrDo(visibilityTaskKey, newFuture, func(key interface{}, value interface{}) error { 194 existingFuture, ok := value.(*ackFuture) 195 if !ok { 196 p.logger.Fatal(fmt.Sprintf("mapToAckFuture has item of a wrong type %T (%T expected).", value, &ackFuture{}), tag.Value(key)) 197 } 198 199 p.logger.Warn("Skipping duplicate ES request for visibility task key.", tag.Key(visibilityTaskKey), tag.ESDocID(request.ID), tag.Value(request.Doc), tag.NewDurationTag("interval-between-duplicates", newFuture.createdAt.Sub(existingFuture.createdAt))) 200 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorDuplicateRequest.Name()).Record(1) 201 newFuture = existingFuture 202 return nil 203 }) 204 if !isDup { 205 p.bulkProcessor.Add(request) 206 newFuture.recordAdd(p.metricsHandler) 207 } 208 return newFuture.future 209 } 210 211 // bulkBeforeAction is triggered before bulk processor commit 212 func (p *processorImpl) bulkBeforeAction(_ int64, requests []elastic.BulkableRequest) { 213 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorRequests.Name()).Record(int64(len(requests))) 214 p.metricsHandler.Histogram(metrics.ElasticsearchBulkProcessorBulkSize.Name(), metrics.ElasticsearchBulkProcessorBulkSize.Unit()). 215 Record(int64(len(requests))) 216 217 for _, request := range requests { 218 visibilityTaskKey := p.extractVisibilityTaskKey(request) 219 if visibilityTaskKey == "" { 220 continue 221 } 222 _, _, _ = p.mapToAckFuture.GetAndDo(visibilityTaskKey, func(key interface{}, value interface{}) error { 223 ackF, ok := value.(*ackFuture) 224 if !ok { 225 p.logger.Fatal(fmt.Sprintf("mapToAckFuture has item of a wrong type %T (%T expected).", value, &ackFuture{}), tag.Value(key)) 226 } 227 ackF.recordStart(p.metricsHandler) 228 return nil 229 }) 230 } 231 } 232 233 // bulkAfterAction is triggered after bulk processor commit 234 func (p *processorImpl) bulkAfterAction(_ int64, requests []elastic.BulkableRequest, response *elastic.BulkResponse, err error) { 235 if err != nil { 236 const logFirstNRequests = 5 237 var httpStatus int 238 var esErr *elastic.Error 239 if errors.As(err, &esErr) { 240 httpStatus = esErr.Status 241 } 242 243 var logRequests strings.Builder 244 for i, request := range requests { 245 if i < logFirstNRequests { 246 logRequests.WriteString(request.String()) 247 logRequests.WriteRune('\n') 248 } 249 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorFailures.Name()).Record(1, metrics.HttpStatusTag(httpStatus)) 250 visibilityTaskKey := p.extractVisibilityTaskKey(request) 251 if visibilityTaskKey == "" { 252 continue 253 } 254 p.notifyResult(visibilityTaskKey, false) 255 } 256 p.logger.Error("Unable to commit bulk ES request.", tag.Error(err), tag.RequestCount(len(requests)), tag.ESRequest(logRequests.String())) 257 return 258 } 259 260 // Record how long the Elasticsearch took to process the bulk request. 261 p.metricsHandler.Timer(metrics.ElasticsearchBulkProcessorBulkResquestTookLatency.Name()). 262 Record(time.Duration(response.Took) * time.Millisecond) 263 264 responseIndex := p.buildResponseIndex(response) 265 for i, request := range requests { 266 visibilityTaskKey := p.extractVisibilityTaskKey(request) 267 if visibilityTaskKey == "" { 268 continue 269 } 270 271 docID := p.extractDocID(request) 272 responseItem, ok := responseIndex[docID] 273 if !ok { 274 p.logger.Error("ES request failed. Request item doesn't have corresponding response item.", 275 tag.Value(i), 276 tag.Key(visibilityTaskKey), 277 tag.ESDocID(docID), 278 tag.ESRequest(request.String())) 279 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 280 p.notifyResult(visibilityTaskKey, false) 281 continue 282 } 283 284 if !isSuccess(responseItem) { 285 p.logger.Error("ES request failed.", 286 tag.ESResponseStatus(responseItem.Status), 287 tag.ESResponseError(extractErrorReason(responseItem)), 288 tag.Key(visibilityTaskKey), 289 tag.ESDocID(docID), 290 tag.ESRequest(request.String())) 291 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorFailures.Name()).Record(1, metrics.HttpStatusTag(responseItem.Status)) 292 p.notifyResult(visibilityTaskKey, false) 293 continue 294 } 295 296 p.notifyResult(visibilityTaskKey, true) 297 } 298 299 // Record how many documents are waiting to be flushed to Elasticsearch after this bulk is committed. 300 p.metricsHandler.Histogram(metrics.ElasticsearchBulkProcessorQueuedRequests.Name(), metrics.ElasticsearchBulkProcessorBulkSize.Unit()). 301 Record(int64(p.mapToAckFuture.Len())) 302 } 303 304 func (p *processorImpl) buildResponseIndex(response *elastic.BulkResponse) map[string]*elastic.BulkResponseItem { 305 result := make(map[string]*elastic.BulkResponseItem) 306 for _, operationResponseItemMap := range response.Items { 307 for _, responseItem := range operationResponseItemMap { 308 existingResponseItem, duplicateID := result[responseItem.Id] 309 // In some rare cases there might be duplicate document Ids in the same bulk. 310 // (for example, if two sequential upsert search attributes operation for the same workflow run end up being in the same bulk request) 311 // In this case, item with greater status code (error) will overwrite existing item with smaller status code. 312 if !duplicateID || existingResponseItem.Status < responseItem.Status { 313 result[responseItem.Id] = responseItem 314 } 315 } 316 } 317 return result 318 } 319 320 func (p *processorImpl) notifyResult(visibilityTaskKey string, ack bool) { 321 // Use RemoveIf here to prevent race condition with de-dup logic in Add method. 322 _ = p.mapToAckFuture.RemoveIf(visibilityTaskKey, func(key interface{}, value interface{}) bool { 323 ackF, ok := value.(*ackFuture) 324 if !ok { 325 p.logger.Fatal(fmt.Sprintf("mapToAckFuture has item of a wrong type %T (%T expected).", value, &ackFuture{}), tag.ESKey(visibilityTaskKey)) 326 } 327 328 ackF.done(ack, p.metricsHandler) 329 return true 330 }) 331 } 332 333 func (p *processorImpl) extractVisibilityTaskKey(request elastic.BulkableRequest) string { 334 req, err := request.Source() 335 if err != nil { 336 p.logger.Error("Unable to get ES request source.", tag.Error(err), tag.ESRequest(request.String())) 337 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 338 return "" 339 } 340 341 if len(req) == 2 { // index or update requests 342 var body map[string]interface{} 343 if err = json.Unmarshal([]byte(req[1]), &body); err != nil { 344 p.logger.Error("Unable to unmarshal ES request body.", tag.Error(err)) 345 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 346 return "" 347 } 348 349 k, ok := body[searchattribute.VisibilityTaskKey] 350 if !ok { 351 p.logger.Error("Unable to extract VisibilityTaskKey from ES request.", tag.ESRequest(request.String())) 352 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 353 return "" 354 } 355 return k.(string) 356 } else { // delete requests 357 return p.extractDocID(request) 358 } 359 } 360 361 func (p *processorImpl) extractDocID(request elastic.BulkableRequest) string { 362 req, err := request.Source() 363 if err != nil { 364 p.logger.Error("Unable to get ES request source.", tag.Error(err), tag.ESRequest(request.String())) 365 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 366 367 return "" 368 } 369 370 var body map[string]map[string]interface{} 371 if err = json.Unmarshal([]byte(req[0]), &body); err != nil { 372 p.logger.Error("Unable to unmarshal ES request body.", tag.Error(err), tag.ESRequest(request.String())) 373 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 374 return "" 375 } 376 377 // There should be only one operation "index" or "delete". 378 for _, opMap := range body { 379 _id, ok := opMap["_id"] 380 if ok { 381 return _id.(string) 382 } 383 } 384 385 p.logger.Error("Unable to extract _id from ES request.", tag.ESRequest(request.String())) 386 p.metricsHandler.Counter(metrics.ElasticsearchBulkProcessorCorruptedData.Name()).Record(1) 387 return "" 388 } 389 390 func isSuccess(item *elastic.BulkResponseItem) bool { 391 if item.Status >= 200 && item.Status < 300 { 392 return true 393 } 394 395 // Ignore version conflict. 396 if item.Status == 409 { 397 return true 398 } 399 400 if item.Status == 404 { 401 if item.Error != nil && item.Error.Type == "index_not_found_exception" { 402 return false 403 } 404 405 // Ignore document not found during delete operation. 406 return true 407 } 408 409 return false 410 } 411 412 func extractErrorReason(resp *elastic.BulkResponseItem) string { 413 if resp.Error != nil { 414 return resp.Error.Reason 415 } 416 return "" 417 } 418 419 func newAckFuture() *ackFuture { 420 var addedAt atomic.Value 421 addedAt.Store(time.Time{}) 422 return &ackFuture{ 423 future: future.NewFuture[bool](), 424 createdAt: time.Now().UTC(), 425 addedAt: addedAt, 426 } 427 } 428 429 func (a *ackFuture) recordAdd(metricsHandler metrics.Handler) { 430 addedAt := time.Now().UTC() 431 a.addedAt.Store(addedAt) 432 metricsHandler.Timer(metrics.ElasticsearchBulkProcessorWaitAddLatency.Name()).Record(addedAt.Sub(a.createdAt)) 433 } 434 435 func (a *ackFuture) recordStart(metricsHandler metrics.Handler) { 436 a.startedAt = time.Now().UTC() 437 addedAt := a.addedAt.Load().(time.Time) 438 if !addedAt.IsZero() { 439 metricsHandler.Timer(metrics.ElasticsearchBulkProcessorWaitStartLatency.Name()).Record(a.startedAt.Sub(addedAt)) 440 } 441 } 442 443 func (a *ackFuture) done(ack bool, metricsHandler metrics.Handler) { 444 a.future.Set(ack, nil) 445 doneAt := time.Now().UTC() 446 if !a.createdAt.IsZero() { 447 metricsHandler.Timer(metrics.ElasticsearchBulkProcessorRequestLatency.Name()).Record(doneAt.Sub(a.createdAt)) 448 } 449 if !a.startedAt.IsZero() { 450 metricsHandler.Timer(metrics.ElasticsearchBulkProcessorCommitLatency.Name()).Record(doneAt.Sub(a.startedAt)) 451 } 452 }