Skip to content

Commit

Permalink
Add eager workflow start (#1164)
Browse files Browse the repository at this point in the history
Add eager workflow start
  • Loading branch information
Quinn-With-Two-Ns authored Jul 26, 2023
1 parent a059d88 commit 746bcf2
Show file tree
Hide file tree
Showing 14 changed files with 544 additions and 46 deletions.
2 changes: 2 additions & 0 deletions .github/workflows/docker/dynamic-config-custom.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -6,4 +6,6 @@ system.enableActivityEagerExecution:
frontend.enableUpdateWorkflowExecution:
- value: true
frontend.enableUpdateWorkflowExecutionAsyncAccepted:
- value: true
system.enableEagerWorkflowStart:
- value: true
8 changes: 8 additions & 0 deletions internal/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -596,6 +596,11 @@ type (
// supported when Temporal server is using ElasticSearch). The key and value type must be registered on Temporal server side.
// Use GetSearchAttributes API to get valid key and corresponding value type.
SearchAttributes map[string]interface{}

// EnableEagerStart - request eager execution for this workflow, if a local worker is available.
//
// NOTE: Experimental
EnableEagerStart bool
}

// RetryPolicy defines the retry policy.
Expand Down Expand Up @@ -813,6 +818,9 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien
contextPropagators: options.ContextPropagators,
workerInterceptors: workerInterceptors,
excludeInternalFromRetry: options.ConnectionOptions.excludeInternalFromRetry,
eagerDispatcher: &eagerWorkflowDispatcher{
workersByTaskQueue: make(map[string][]eagerWorker),
},
}

// Create outbound interceptor by wrapping backwards through chain
Expand Down
35 changes: 35 additions & 0 deletions internal/internal_eager.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
// The MIT License
//
// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved.
//
// 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.

package internal

// eagerWorker is the minimal worker interface needed for eager activities and workflows
type eagerWorker interface {
// tryReserveSlot tries to reserver a task slot on the worker without blocking
// caller is expected to release the slot with releaseSlot
tryReserveSlot() bool
// releaseSlot release a task slot acquired by tryReserveSlot
releaseSlot()
// processTaskAsync process a new task on the worker asynchronously and
// call callback once complete
processTaskAsync(task interface{}, callback func())
}
42 changes: 12 additions & 30 deletions internal/internal_eager_activity.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import (
type eagerActivityExecutor struct {
eagerActivityExecutorOptions

activityWorker *activityWorker
activityWorker eagerWorker
heldSlotCount int
countLock sync.Mutex
}
Expand Down Expand Up @@ -97,11 +97,8 @@ func (e *eagerActivityExecutor) reserveOnePendingSlot() bool {
// No more room
return false
}
// Reserve a spot for our request via a non-blocking attempt to take a poller
// request entry which essentially reserves a spot
select {
case <-e.activityWorker.worker.pollerRequestCh:
default:
// Reserve a spot for our request via a non-blocking attempt
if !e.activityWorker.tryReserveSlot() {
return false
}

Expand Down Expand Up @@ -131,35 +128,20 @@ func (e *eagerActivityExecutor) handleResponse(

// Put every unfulfilled slot back on the poller channel
for i := 0; i < unfulfilledSlots; i++ {
// Like other parts that push onto this channel, we assume there is room
// because we took it, so we do a blocking send
e.activityWorker.worker.pollerRequestCh <- struct{}{}
e.activityWorker.releaseSlot()
}

// Start each activity asynchronously
for _, activity := range resp.GetActivityTasks() {
// Before starting the goroutine we have to increase the wait group counter
// that the poller would have otherwise increased
e.activityWorker.worker.stopWG.Add(1)
// Asynchronously execute
task := &activityTask{activity}
go func() {
// Mark completed when complete
defer func() {
// Like other sends to this channel, we assume there is room because we
// reserved it, so we make a blocking send. The processTask does not do
// this itself because our task is *activityTask, not *polledTask.
e.activityWorker.worker.pollerRequestCh <- struct{}{}
// Decrement executing count
e.countLock.Lock()
e.heldSlotCount--
e.countLock.Unlock()
}()

// Process the task synchronously. We call the processor on the base
// worker instead of a higher level so we can get the benefits of metrics,
// stop wait group update, etc.
e.activityWorker.worker.processTask(task)
}()
e.activityWorker.processTaskAsync(task, func() {
// The processTaskAsync does not do this itself because our task is *activityTask, not *polledTask.
e.activityWorker.releaseSlot()
// Decrement executing count
e.countLock.Lock()
e.heldSlotCount--
e.countLock.Unlock()
})
}
}
26 changes: 17 additions & 9 deletions internal/internal_eager_activity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import (
func TestEagerActivityDisabled(t *testing.T) {
exec := newEagerActivityExecutor(eagerActivityExecutorOptions{disabled: true, taskQueue: "task-queue1"})
exec.activityWorker = newActivityWorker(nil,
workerExecutionParameters{TaskQueue: "task-queue1"}, nil, newRegistry(), nil)
workerExecutionParameters{TaskQueue: "task-queue1"}, nil, newRegistry(), nil).worker

// Turns requests to false when disabled
var req workflowservice.RespondWorkflowTaskCompletedRequest
Expand All @@ -59,11 +59,13 @@ func TestEagerActivityNoActivityWorker(t *testing.T) {

func TestEagerActivityWrongTaskQueue(t *testing.T) {
exec := newEagerActivityExecutor(eagerActivityExecutorOptions{taskQueue: "task-queue1"})
exec.activityWorker = newActivityWorker(nil,
workerExecutionParameters{TaskQueue: "task-queue1", ConcurrentActivityExecutionSize: 10}, nil, newRegistry(), nil)
activityWorker := newActivityWorker(nil, workerExecutionParameters{TaskQueue: "task-queue1", ConcurrentActivityExecutionSize: 10}, nil, newRegistry(), nil)
activityWorker.worker.isWorkerStarted = true

exec.activityWorker = activityWorker.worker
// Fill up the poller request channel
for i := 0; i < 10; i++ {
exec.activityWorker.worker.pollerRequestCh <- struct{}{}
activityWorker.worker.pollerRequestCh <- struct{}{}
}

// Turns requests to false when wrong task queue
Expand All @@ -77,11 +79,14 @@ func TestEagerActivityWrongTaskQueue(t *testing.T) {

func TestEagerActivityMaxPerTask(t *testing.T) {
exec := newEagerActivityExecutor(eagerActivityExecutorOptions{taskQueue: "task-queue1"})
exec.activityWorker = newActivityWorker(nil,
activityWorker := newActivityWorker(nil,
workerExecutionParameters{TaskQueue: "task-queue1", ConcurrentActivityExecutionSize: 10}, nil, newRegistry(), nil)
activityWorker.worker.isWorkerStarted = true

exec.activityWorker = activityWorker.worker
// Fill up the poller request channel
for i := 0; i < 10; i++ {
exec.activityWorker.worker.pollerRequestCh <- struct{}{}
activityWorker.worker.pollerRequestCh <- struct{}{}
}

// Add 8, but it limits to only the first 3
Expand All @@ -99,16 +104,19 @@ func TestEagerActivityCounts(t *testing.T) {
// We'll create an eager activity executor with 3 max eager concurrent and 5
// max concurrent
exec := newEagerActivityExecutor(eagerActivityExecutorOptions{taskQueue: "task-queue1", maxConcurrent: 3})
exec.activityWorker = newActivityWorker(nil,
activityWorker := newActivityWorker(nil,
workerExecutionParameters{TaskQueue: "task-queue1", ConcurrentActivityExecutionSize: 5}, nil, newRegistry(), nil)
activityWorker.worker.isWorkerStarted = true

exec.activityWorker = activityWorker.worker
// Fill up the poller request channel
slotsCh := exec.activityWorker.worker.pollerRequestCh
slotsCh := activityWorker.worker.pollerRequestCh
for i := 0; i < 5; i++ {
slotsCh <- struct{}{}
}
// Replace task processor
taskProcessor := newWaitingTaskProcessor()
exec.activityWorker.worker.options.taskWorker = taskProcessor
activityWorker.worker.options.taskWorker = taskProcessor

// Request 2 commands on wrong task queue then 5 commands on proper task queue
// but have 2nd request disabled
Expand Down
99 changes: 99 additions & 0 deletions internal/internal_eager_workflow.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
// The MIT License
//
// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved.
//
// 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.

package internal

import (
"math/rand"
"sync"
"sync/atomic"

"go.temporal.io/api/workflowservice/v1"
)

// eagerWorkflowDispatcher is responsible for finding an available worker for an eager workflow task.
type eagerWorkflowDispatcher struct {
lock sync.RWMutex
workersByTaskQueue map[string][]eagerWorker
}

// registerWorker registers a worker that can be used for eager workflow dispatch
func (e *eagerWorkflowDispatcher) registerWorker(worker *workflowWorker) {
e.lock.Lock()
defer e.lock.Unlock()
e.workersByTaskQueue[worker.executionParameters.TaskQueue] = append(e.workersByTaskQueue[worker.executionParameters.TaskQueue], worker.worker)
}

// applyToRequest updates request if eager workflow dispatch is possible and returns the eagerWorkflowExecutor to use
func (e *eagerWorkflowDispatcher) applyToRequest(request *workflowservice.StartWorkflowExecutionRequest) *eagerWorkflowExecutor {
// Try every worker that is assigned to the desired task queue.
e.lock.RLock()
workers := e.workersByTaskQueue[request.GetTaskQueue().Name]
randWorkers := make([]eagerWorker, len(workers))
// Copy the slice so we can release the lock.
copy(randWorkers, workers)
e.lock.RUnlock()
rand.Shuffle(len(randWorkers), func(i, j int) { randWorkers[i], randWorkers[j] = randWorkers[j], randWorkers[i] })
for _, worker := range randWorkers {
if worker.tryReserveSlot() {
request.RequestEagerExecution = true
return &eagerWorkflowExecutor{
worker: worker,
}
}
}
return nil
}

// eagerWorkflowExecutor is a worker-scoped executor for an eager workflow task.
type eagerWorkflowExecutor struct {
handledResponse atomic.Bool
worker eagerWorker
}

// handleResponse of an eager workflow task from a StartWorkflowExecution request.
func (e *eagerWorkflowExecutor) handleResponse(response *workflowservice.PollWorkflowTaskQueueResponse) {
if !e.handledResponse.CompareAndSwap(false, true) {
panic("eagerWorkflowExecutor trying to handle multiple responses")
}
// Asynchronously execute the task
task := &eagerWorkflowTask{
task: response,
}
e.worker.processTaskAsync(task, func() {
// The processTaskAsync does not do this itself because our task is *eagerWorkflowTask, not *polledTask.
e.worker.releaseSlot()
})
}

// release the executor task slot this eagerWorkflowExecutor was holding.
// If it is currently handling a responses or has already released the task slot
// then do nothing.
func (e *eagerWorkflowExecutor) release() {
if e.handledResponse.CompareAndSwap(false, true) {
// Assume there is room because it is reserved on creation, so we make a blocking send.
// The processTask does not do this itself because our task is not *polledTask.
e.worker.releaseSlot()
} else {
panic("trying to release an eagerWorkflowExecutor that has already been released")
}
}
Loading

0 comments on commit 746bcf2

Please sign in to comment.