github.com/Jeffail/benthos/v3@v3.65.0/lib/processor/workflow.go (about) 1 package processor 2 3 import ( 4 "fmt" 5 "sort" 6 "sync" 7 "time" 8 9 "github.com/Jeffail/benthos/v3/internal/docs" 10 "github.com/Jeffail/benthos/v3/internal/tracing" 11 "github.com/Jeffail/benthos/v3/lib/log" 12 "github.com/Jeffail/benthos/v3/lib/metrics" 13 "github.com/Jeffail/benthos/v3/lib/types" 14 "github.com/Jeffail/gabs/v2" 15 ) 16 17 //------------------------------------------------------------------------------ 18 19 func init() { 20 Constructors[TypeWorkflow] = TypeSpec{ 21 constructor: NewWorkflow, 22 Categories: []Category{ 23 CategoryComposition, 24 }, 25 Status: docs.StatusStable, 26 Summary: ` 27 Executes a topology of ` + "[`branch` processors][processors.branch]" + `, 28 performing them in parallel where possible.`, 29 Description: ` 30 ## Why Use a Workflow 31 32 ### Performance 33 34 Most of the time the best way to compose processors is also the simplest, just configure them in series. This is because processors are often CPU bound, low-latency, and you can gain vertical scaling by increasing the number of processor pipeline threads, allowing Benthos to process [multiple messages in parallel][configuration.pipelines]. 35 36 However, some processors such as ` + "[`http`][processors.http], [`lambda`][processors.lambda] or [`cache`][processors.cache]" + ` interact with external services and therefore spend most of their time waiting for a response. These processors tend to be high-latency and low CPU activity, which causes messages to process slowly. 37 38 When a processing pipeline contains multiple network processors that aren't dependent on each other we can benefit from performing these processors in parallel for each individual message, reducing the overall message processing latency. 39 40 ### Simplifying Processor Topology 41 42 A workflow is often expressed as a [DAG][dag_wiki] of processing stages, where each stage can result in N possible next stages, until finally the flow ends at an exit node. 43 44 For example, if we had processing stages A, B, C and D, where stage A could result in either stage B or C being next, always followed by D, it might look something like this: 45 46 ` + "```text" + ` 47 /--> B --\ 48 A --| |--> D 49 \--> C --/ 50 ` + "```" + ` 51 52 This flow would be easy to express in a standard Benthos config, we could simply use a ` + "[`switch` processor][processors.switch]" + ` to route to either B or C depending on a condition on the result of A. However, this method of flow control quickly becomes unfeasible as the DAG gets more complicated, imagine expressing this flow using switch processors: 53 54 ` + "```text" + ` 55 /--> B -------------|--> D 56 / / 57 A --| /--> E --| 58 \--> C --| \ 59 \----------|--> F 60 ` + "```" + ` 61 62 And imagine doing so knowing that the diagram is subject to change over time. Yikes! Instead, with a workflow we can either trust it to automatically resolve the DAG or express it manually as simply as ` + "`order: [ [ A ], [ B, C ], [ E ], [ D, F ] ]`" + `, and the conditional logic for determining if a stage is executed is defined as part of the branch itself.`, 63 Footnotes: ` 64 ## Structured Metadata 65 66 When the field ` + "`meta_path`" + ` is non-empty the workflow processor creates an object describing which workflows were successful, skipped or failed for each message and stores the object within the message at the end. 67 68 The object is of the following form: 69 70 ` + "```json" + ` 71 { 72 "succeeded": [ "foo" ], 73 "skipped": [ "bar" ], 74 "failed": { 75 "baz": "the error message from the branch" 76 } 77 } 78 ` + "```" + ` 79 80 If a message already has a meta object at the given path when it is processed then the object is used in order to determine which branches have already been performed on the message (or skipped) and can therefore be skipped on this run. 81 82 This is a useful pattern when replaying messages that have failed some branches previously. For example, given the above example object the branches foo and bar would automatically be skipped, and baz would be reattempted. 83 84 The previous meta object will also be preserved in the field ` + "`<meta_path>.previous`" + ` when the new meta object is written, preserving a full record of all workflow executions. 85 86 If a field ` + "`<meta_path>.apply`" + ` exists in the meta object for a message and is an array then it will be used as an explicit list of stages to apply, all other stages will be skipped. 87 88 ## Resources 89 90 It's common to configure processors (and other components) [as resources][configuration.resources] in order to keep the pipeline configuration cleaner. With the workflow processor you can include branch processors configured as resources within your workflow either by specifying them by name in the field ` + "`order`" + `, if Benthos doesn't find a branch within the workflow configuration of that name it'll refer to the resources. 91 92 Alternatively, if you do not wish to have an explicit ordering, you can add resource names to the field ` + "`branch_resources`" + ` and they will be included in the workflow with automatic DAG resolution along with any branches configured in the ` + "`branches`" + ` field. 93 94 ### Resource Error Conditions 95 96 There are two error conditions that could potentially occur when resources included in your workflow are mutated, and if you are planning to mutate resources in your workflow it is important that you understand them. 97 98 The first error case is that a resource in the workflow is removed and not replaced, when this happens the workflow will still be executed but the individual branch will fail. This should only happen if you explicitly delete a branch resource, as any mutation operation will create the new resource before removing the old one. 99 100 The second error case is when automatic DAG resolution is being used and a resource in the workflow is changed in a way that breaks the DAG (circular dependencies, etc). When this happens it is impossible to execute the workflow and therefore the processor will fail, which is possible to capture and handle using [standard error handling patterns][configuration.error-handling]. 101 102 ## Error Handling 103 104 The recommended approach to handle failures within a workflow is to query against the [structured metadata](#structured-metadata) it provides, as it provides granular information about exactly which branches failed and which ones succeeded and therefore aren't necessary to perform again. 105 106 For example, if our meta object is stored at the path ` + "`meta.workflow`" + ` and we wanted to check whether a message has failed for any branch we can do that using a [Bloblang query][guides.bloblang] like ` + "`this.meta.workflow.failed.length() | 0 > 0`" + `, or to check whether a specific branch failed we can use ` + "`this.exists(\"meta.workflow.failed.foo\")`" + `. 107 108 However, if structured metadata is disabled by setting the field ` + "`meta_path`" + ` to empty then the workflow processor instead adds a general error flag to messages when any executed branch fails. In this case it's possible to handle failures using [standard error handling patterns][configuration.error-handling]. 109 110 [dag_wiki]: https://en.wikipedia.org/wiki/Directed_acyclic_graph 111 [processors.switch]: /docs/components/processors/switch 112 [processors.http]: /docs/components/processors/http 113 [processors.lambda]: /docs/components/processors/lambda 114 [processors.cache]: /docs/components/processors/cache 115 [processors.branch]: /docs/components/processors/branch 116 [guides.bloblang]: /docs/guides/bloblang/about 117 [configuration.pipelines]: /docs/configuration/processing_pipelines 118 [configuration.error-handling]: /docs/configuration/error_handling 119 [configuration.resources]: /docs/configuration/resources 120 `, 121 Examples: []docs.AnnotatedExample{ 122 { 123 Title: "Automatic Ordering", 124 Summary: ` 125 When the field ` + "`order`" + ` is omitted a best attempt is made to determine a dependency tree between branches based on their request and result mappings. In the following example the branches foo and bar will be executed first in parallel, and afterwards the branch baz will be executed.`, 126 Config: ` 127 pipeline: 128 processors: 129 - workflow: 130 meta_path: meta.workflow 131 branches: 132 foo: 133 request_map: 'root = ""' 134 processors: 135 - http: 136 url: TODO 137 result_map: 'root.foo = this' 138 139 bar: 140 request_map: 'root = this.body' 141 processors: 142 - aws_lambda: 143 function: TODO 144 result_map: 'root.bar = this' 145 146 baz: 147 request_map: | 148 root.fooid = this.foo.id 149 root.barstuff = this.bar.content 150 processors: 151 - cache: 152 resource: TODO 153 operator: set 154 key: ${! json("fooid") } 155 value: ${! json("barstuff") } 156 `, 157 }, 158 { 159 Title: "Conditional Branches", 160 Summary: ` 161 Branches of a workflow are skipped when the ` + "`request_map`" + ` assigns ` + "`deleted()`" + ` to the root. In this example the branch A is executed when the document type is "foo", and branch B otherwise. Branch C is executed afterwards and is skipped unless either A or B successfully provided a result at ` + "`tmp.result`" + `.`, 162 Config: ` 163 pipeline: 164 processors: 165 - workflow: 166 branches: 167 A: 168 request_map: | 169 root = if this.document.type != "foo" { 170 deleted() 171 } 172 processors: 173 - http: 174 url: TODO 175 result_map: 'root.tmp.result = this' 176 177 B: 178 request_map: | 179 root = if this.document.type == "foo" { 180 deleted() 181 } 182 processors: 183 - aws_lambda: 184 function: TODO 185 result_map: 'root.tmp.result = this' 186 187 C: 188 request_map: | 189 root = if this.tmp.result != null { 190 deleted() 191 } 192 processors: 193 - http: 194 url: TODO_SOMEWHERE_ELSE 195 result_map: 'root.tmp.result = this' 196 `, 197 }, 198 { 199 Title: "Resources", 200 Summary: ` 201 The ` + "`order`" + ` field can be used in order to refer to [branch processor resources](#resources), this can sometimes make your pipeline configuration cleaner, as well as allowing you to reuse branch configurations in order places. It's also possible to mix and match branches configured within the workflow and configured as resources.`, 202 Config: ` 203 pipeline: 204 processors: 205 - workflow: 206 order: [ [ foo, bar ], [ baz ] ] 207 branches: 208 bar: 209 request_map: 'root = this.body' 210 processors: 211 - aws_lambda: 212 function: TODO 213 result_map: 'root.bar = this' 214 215 processor_resources: 216 - label: foo 217 branch: 218 request_map: 'root = ""' 219 processors: 220 - http: 221 url: TODO 222 result_map: 'root.foo = this' 223 224 - label: baz 225 branch: 226 request_map: | 227 root.fooid = this.foo.id 228 root.barstuff = this.bar.content 229 processors: 230 - cache: 231 resource: TODO 232 operator: set 233 key: ${! json("fooid") } 234 value: ${! json("barstuff") } 235 `, 236 }, 237 }, 238 FieldSpecs: docs.FieldSpecs{ 239 docs.FieldString("meta_path", "A [dot path](/docs/configuration/field_paths) indicating where to store and reference [structured metadata](#structured-metadata) about the workflow execution."), 240 docs.FieldDeprecated("stages").Map(), 241 docs.FieldString( 242 "order", 243 "An explicit declaration of branch ordered tiers, which describes the order in which parallel tiers of branches should be executed. Branches should be identified by the name as they are configured in the field `branches`. It's also possible to specify branch processors configured [as a resource](#resources).", 244 [][]string{{"foo", "bar"}, {"baz"}}, 245 [][]string{{"foo"}, {"bar"}, {"baz"}}, 246 ).ArrayOfArrays(), 247 docs.FieldString( 248 "branch_resources", 249 "An optional list of [`branch` processor](/docs/components/processors/branch) names that are configured as [resources](#resources). These resources will be included in the workflow with any branches configured inline within the [`branches`](#branches) field. The order and parallelism in which branches are executed is automatically resolved based on the mappings of each branch. When using resources with an explicit order it is not necessary to list resources in this field.", 250 ).AtVersion("3.38.0").Advanced().Array(), 251 docs.FieldCommon( 252 "branches", 253 "An object of named [`branch` processors](/docs/components/processors/branch) that make up the workflow. The order and parallelism in which branches are executed can either be made explicit with the field `order`, or if omitted an attempt is made to automatically resolve an ordering based on the mappings of each branch.", 254 ).Map().WithChildren(branchFields...), 255 }, 256 } 257 } 258 259 //------------------------------------------------------------------------------ 260 261 // WorkflowConfig is a config struct containing fields for the Workflow 262 // processor. 263 type WorkflowConfig struct { 264 MetaPath string `json:"meta_path" yaml:"meta_path"` 265 Order [][]string `json:"order" yaml:"order"` 266 BranchResources []string `json:"branch_resources" yaml:"branch_resources"` 267 Branches map[string]BranchConfig `json:"branches" yaml:"branches"` 268 Stages map[string]DepProcessMapConfig `json:"stages" yaml:"stages"` 269 } 270 271 // NewWorkflowConfig returns a default WorkflowConfig. 272 func NewWorkflowConfig() WorkflowConfig { 273 return WorkflowConfig{ 274 MetaPath: "meta.workflow", 275 Order: [][]string{}, 276 BranchResources: []string{}, 277 Branches: map[string]BranchConfig{}, 278 Stages: map[string]DepProcessMapConfig{}, 279 } 280 } 281 282 //------------------------------------------------------------------------------ 283 284 // Workflow is a processor that applies a list of child processors to a new 285 // payload mapped from the original, and after processing attempts to overlay 286 // the results back onto the original payloads according to more mappings. 287 type Workflow struct { 288 log log.Modular 289 stats metrics.Type 290 291 children *workflowBranchMap 292 allStages map[string]struct{} 293 metaPath []string 294 295 mCount metrics.StatCounter 296 mSent metrics.StatCounter 297 mSentParts metrics.StatCounter 298 mSkippedNoStages metrics.StatCounter 299 mErr metrics.StatCounter 300 mErrJSON metrics.StatCounter 301 mErrMeta metrics.StatCounter 302 mErrOverlay metrics.StatCounter 303 mErrStages map[string]metrics.StatCounter 304 mSuccStages map[string]metrics.StatCounter 305 metricsMut sync.RWMutex 306 } 307 308 // NewWorkflow returns a new workflow processor. 309 func NewWorkflow( 310 conf Config, mgr types.Manager, log log.Modular, stats metrics.Type, 311 ) (Type, error) { 312 if len(conf.Workflow.Stages) > 0 { 313 if len(conf.Workflow.Branches) > 0 { 314 return nil, fmt.Errorf("cannot combine both workflow branches and stages in the same processor") 315 } 316 if len(conf.Workflow.Order) > 0 { 317 return nil, fmt.Errorf("cannot combine both manual ordering and stages in the same processor") 318 } 319 return newWorkflowDeprecated(conf, mgr, log, stats) 320 } 321 322 w := &Workflow{ 323 log: log, 324 stats: stats, 325 mErrStages: map[string]metrics.StatCounter{}, 326 mSuccStages: map[string]metrics.StatCounter{}, 327 metaPath: nil, 328 allStages: map[string]struct{}{}, 329 } 330 if len(conf.Workflow.MetaPath) > 0 { 331 w.metaPath = gabs.DotPathToSlice(conf.Workflow.MetaPath) 332 } 333 334 var err error 335 if w.children, err = newWorkflowBranchMap(conf.Workflow, mgr, log, stats); err != nil { 336 return nil, err 337 } 338 for k := range w.children.dynamicBranches { 339 w.allStages[k] = struct{}{} 340 } 341 342 w.mCount = stats.GetCounter("count") 343 w.mSent = stats.GetCounter("sent") 344 w.mSentParts = stats.GetCounter("parts.sent") 345 w.mSkippedNoStages = stats.GetCounter("skipped.no_stages") 346 w.mErr = stats.GetCounter("error") 347 w.mErrJSON = stats.GetCounter("error.json_parse") 348 w.mErrMeta = stats.GetCounter("error.meta_set") 349 w.mErrOverlay = stats.GetCounter("error.overlay") 350 351 return w, nil 352 } 353 354 //------------------------------------------------------------------------------ 355 356 func (w *Workflow) incrStageErr(id string) { 357 w.metricsMut.RLock() 358 ctr, exists := w.mErrStages[id] 359 w.metricsMut.RUnlock() 360 if exists { 361 ctr.Incr(1) 362 return 363 } 364 365 w.metricsMut.Lock() 366 defer w.metricsMut.Unlock() 367 368 ctr = w.stats.GetCounter(fmt.Sprintf("%v.error", id)) 369 ctr.Incr(1) 370 w.mErrStages[id] = ctr 371 } 372 373 func (w *Workflow) incrStageSucc(id string) { 374 w.metricsMut.RLock() 375 ctr, exists := w.mSuccStages[id] 376 w.metricsMut.RUnlock() 377 if exists { 378 ctr.Incr(1) 379 return 380 } 381 382 w.metricsMut.Lock() 383 defer w.metricsMut.Unlock() 384 385 ctr = w.stats.GetCounter(fmt.Sprintf("%v.success", id)) 386 ctr.Incr(1) 387 w.mSuccStages[id] = ctr 388 } 389 390 //------------------------------------------------------------------------------ 391 392 type resultTracker struct { 393 succeeded map[string]struct{} 394 skipped map[string]struct{} 395 failed map[string]string 396 sync.Mutex 397 } 398 399 func trackerFromTree(tree [][]string) *resultTracker { 400 r := &resultTracker{ 401 succeeded: map[string]struct{}{}, 402 skipped: map[string]struct{}{}, 403 failed: map[string]string{}, 404 } 405 for _, layer := range tree { 406 for _, k := range layer { 407 r.succeeded[k] = struct{}{} 408 } 409 } 410 return r 411 } 412 413 func (r *resultTracker) Skipped(k string) { 414 r.Lock() 415 delete(r.succeeded, k) 416 417 r.skipped[k] = struct{}{} 418 r.Unlock() 419 } 420 421 func (r *resultTracker) Failed(k, why string) { 422 r.Lock() 423 delete(r.succeeded, k) 424 delete(r.skipped, k) 425 426 r.failed[k] = why 427 r.Unlock() 428 } 429 430 func (r *resultTracker) ToObject() map[string]interface{} { 431 succeeded := make([]interface{}, 0, len(r.succeeded)) 432 skipped := make([]interface{}, 0, len(r.skipped)) 433 failed := make(map[string]interface{}, len(r.failed)) 434 435 for k := range r.succeeded { 436 succeeded = append(succeeded, k) 437 } 438 sort.Slice(succeeded, func(i, j int) bool { 439 return succeeded[i].(string) < succeeded[j].(string) 440 }) 441 for k := range r.skipped { 442 skipped = append(skipped, k) 443 } 444 sort.Slice(skipped, func(i, j int) bool { 445 return skipped[i].(string) < skipped[j].(string) 446 }) 447 for k, v := range r.failed { 448 failed[k] = v 449 } 450 451 m := map[string]interface{}{} 452 if len(succeeded) > 0 { 453 m["succeeded"] = succeeded 454 } 455 if len(skipped) > 0 { 456 m["skipped"] = skipped 457 } 458 if len(failed) > 0 { 459 m["failed"] = failed 460 } 461 return m 462 } 463 464 // Returns a map of enrichment IDs that should be skipped for this payload. 465 func (w *Workflow) skipFromMeta(root interface{}) map[string]struct{} { 466 skipList := map[string]struct{}{} 467 if len(w.metaPath) == 0 { 468 return skipList 469 } 470 471 gObj := gabs.Wrap(root) 472 473 // If a whitelist is provided for this flow then skip stages that aren't 474 // within it. 475 if apply, ok := gObj.S(append(w.metaPath, "apply")...).Data().([]interface{}); ok { 476 if len(apply) > 0 { 477 for k := range w.allStages { 478 skipList[k] = struct{}{} 479 } 480 for _, id := range apply { 481 if idStr, isString := id.(string); isString { 482 delete(skipList, idStr) 483 } 484 } 485 } 486 } 487 488 // Skip stages that already succeeded in a previous run of this workflow. 489 if succeeded, ok := gObj.S(append(w.metaPath, "succeeded")...).Data().([]interface{}); ok { 490 for _, id := range succeeded { 491 if idStr, isString := id.(string); isString { 492 if _, exists := w.allStages[idStr]; exists { 493 skipList[idStr] = struct{}{} 494 } 495 } 496 } 497 } 498 499 // Skip stages that were already skipped in a previous run of this workflow. 500 if skipped, ok := gObj.S(append(w.metaPath, "skipped")...).Data().([]interface{}); ok { 501 for _, id := range skipped { 502 if idStr, isString := id.(string); isString { 503 if _, exists := w.allStages[idStr]; exists { 504 skipList[idStr] = struct{}{} 505 } 506 } 507 } 508 } 509 510 return skipList 511 } 512 513 // ProcessMessage applies workflow stages to each part of a message type. 514 func (w *Workflow) ProcessMessage(msg types.Message) ([]types.Message, types.Response) { 515 w.mCount.Incr(1) 516 517 payload := msg.DeepCopy() 518 519 // Prevent resourced branches from being updated mid-flow. 520 dag, children, unlock, err := w.children.Lock() 521 if err != nil { 522 w.mErr.Incr(1) 523 w.log.Errorf("Failed to establish workflow: %v\n", err) 524 525 payload.Iter(func(i int, p types.Part) error { 526 FlagErr(p, err) 527 return nil 528 }) 529 w.mSentParts.Incr(int64(payload.Len())) 530 w.mSent.Incr(1) 531 return []types.Message{payload}, nil 532 } 533 defer unlock() 534 535 skipOnMeta := make([]map[string]struct{}, msg.Len()) 536 payload.Iter(func(i int, p types.Part) error { 537 p.Get() 538 p.Metadata() 539 if jObj, err := p.JSON(); err == nil { 540 skipOnMeta[i] = w.skipFromMeta(jObj) 541 } else { 542 skipOnMeta[i] = map[string]struct{}{} 543 } 544 return nil 545 }) 546 547 propMsg, _ := tracing.WithChildSpans("workflow", payload) 548 549 records := make([]*resultTracker, payload.Len()) 550 for i := range records { 551 records[i] = trackerFromTree(dag) 552 } 553 554 for _, layer := range dag { 555 results := make([][]types.Part, len(layer)) 556 errors := make([]error, len(layer)) 557 558 wg := sync.WaitGroup{} 559 wg.Add(len(layer)) 560 for i, eid := range layer { 561 go func(id string, index int) { 562 branchMsg, branchSpans := tracing.WithChildSpans(id, propMsg.Copy()) 563 564 branchParts := make([]types.Part, branchMsg.Len()) 565 branchMsg.Iter(func(partIndex int, part types.Part) error { 566 // Remove errors so that they aren't propagated into the 567 // branch. 568 ClearFail(part) 569 if _, exists := skipOnMeta[partIndex][id]; !exists { 570 branchParts[partIndex] = part 571 } 572 return nil 573 }) 574 575 var mapErrs []branchMapError 576 results[index], mapErrs, errors[index] = children[id].createResult(branchParts, propMsg) 577 for _, s := range branchSpans { 578 s.Finish() 579 } 580 for j, p := range results[index] { 581 if p == nil { 582 records[j].Skipped(id) 583 } 584 } 585 for _, e := range mapErrs { 586 records[e.index].Failed(id, e.err.Error()) 587 } 588 wg.Done() 589 }(eid, i) 590 } 591 wg.Wait() 592 593 for i, id := range layer { 594 var failed []branchMapError 595 err := errors[i] 596 if err == nil { 597 if failed, err = children[id].overlayResult(payload, results[i]); err != nil { 598 w.mErrOverlay.Incr(1) 599 } 600 } 601 if err != nil { 602 w.incrStageErr(id) 603 w.mErr.Incr(1) 604 w.log.Errorf("Failed to perform enrichment '%v': %v\n", id, err) 605 for j := range records { 606 records[j].Failed(id, err.Error()) 607 } 608 continue 609 } 610 for _, e := range failed { 611 records[e.index].Failed(id, e.err.Error()) 612 } 613 w.incrStageSucc(id) 614 } 615 } 616 617 // Finally, set the meta records of each document. 618 if len(w.metaPath) > 0 { 619 payload.Iter(func(i int, p types.Part) error { 620 pJSON, err := p.JSON() 621 if err != nil { 622 w.mErr.Incr(1) 623 w.mErrMeta.Incr(1) 624 w.log.Errorf("Failed to parse message for meta update: %v\n", err) 625 FlagErr(p, err) 626 return nil 627 } 628 629 gObj := gabs.Wrap(pJSON) 630 previous := gObj.S(w.metaPath...).Data() 631 current := records[i].ToObject() 632 if previous != nil { 633 current["previous"] = previous 634 } 635 gObj.Set(current, w.metaPath...) 636 637 p.SetJSON(gObj.Data()) 638 return nil 639 }) 640 } else { 641 payload.Iter(func(i int, p types.Part) error { 642 if lf := len(records[i].failed); lf > 0 { 643 failed := make([]string, 0, lf) 644 for k := range records[i].failed { 645 failed = append(failed, k) 646 } 647 sort.Strings(failed) 648 FlagErr(p, fmt.Errorf("workflow branches failed: %v", failed)) 649 } 650 return nil 651 }) 652 } 653 654 tracing.FinishSpans(propMsg) 655 656 w.mSentParts.Incr(int64(payload.Len())) 657 w.mSent.Incr(1) 658 msgs := [1]types.Message{payload} 659 return msgs[:], nil 660 } 661 662 // CloseAsync shuts down the processor and stops processing requests. 663 func (w *Workflow) CloseAsync() { 664 w.children.CloseAsync() 665 } 666 667 // WaitForClose blocks until the processor has closed down. 668 func (w *Workflow) WaitForClose(timeout time.Duration) error { 669 return w.children.WaitForClose(timeout) 670 } 671 672 //------------------------------------------------------------------------------