Skip to content

Commit

Permalink
Add HistoryService.AddTasks API
Browse files Browse the repository at this point in the history
  • Loading branch information
MichaelSnowden committed Oct 12, 2023
1 parent 7f21d05 commit ad6e47a
Show file tree
Hide file tree
Showing 22 changed files with 2,143 additions and 501 deletions.
1,509 changes: 1,120 additions & 389 deletions api/historyservice/v1/request_response.pb.go

Large diffs are not rendered by default.

235 changes: 144 additions & 91 deletions api/historyservice/v1/service.pb.go

Large diffs are not rendered by default.

35 changes: 35 additions & 0 deletions api/historyservicemock/v1/service.pb.mock.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

20 changes: 20 additions & 0 deletions client/history/client_gen.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

14 changes: 14 additions & 0 deletions client/history/metric_client_gen.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

15 changes: 15 additions & 0 deletions client/history/retryable_client_gen.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions common/rpc/interceptor/namespace_test.go
Expand Up @@ -72,6 +72,7 @@ var (
"GetReplicationStatus": {},
"GetDLQTasks": {},
"DeleteDLQTasks": {},
"AddTasks": {},
}
)

Expand Down
Expand Up @@ -794,3 +794,22 @@ message DeleteDLQTasksRequest {
}

message DeleteDLQTasksResponse {}

message AddTasksRequest {
// Even though we can obtain the shard ID from the tasks, we still need the shard_id in the request for routing. If
// not, it would be possible to include tasks for shards that belong to different hosts, and we'd need to fan-out the
// request, which would be more complicated.
int32 shard_id = 1;

message Task {
// category is needed to deserialize the tasks. Examples include "transfer", "timer", etc. See the history/tasks
// package for a definitive list. Warning: this is not the same as the stringified value of a TaskCategory enum.
string category = 1;
// blob is the serialized task.
temporal.api.common.v1.DataBlob blob = 2;
}

repeated Task tasks = 2;
}

message AddTasksResponse {}
10 changes: 10 additions & 0 deletions proto/internal/temporal/server/api/historyservice/v1/service.proto
Expand Up @@ -323,4 +323,14 @@ service HistoryService {

rpc DeleteDLQTasks (DeleteDLQTasksRequest) returns (DeleteDLQTasksResponse) {
}
// The AddTasks API is used to add history tasks to a shard. The first use-case for this API is the DLQ. When we are
// unable to process history tasks, we add them to a DLQ. When they need to be retried, we take them out of the DLQ
// and add them back using this API. We expose this via an API instead of doing this in the history engine because
// replication tasks, which are DLQ'd on the target cluster need to be added back to the queue on the source
// cluster, so there is already a network boundary. There is a maximum of 1000 tasks per request. There must be at
// least one task per request. If any task in the list does not have the same shard ID as the request, the request
// will fail with an InvalidArgument error. It is ok to have tasks for different workflow runs as long as they are
// in the same shard. Calls to the persistence API will be batched by workflow run.
rpc AddTasks (AddTasksRequest) returns (AddTasksResponse) {
}
}
152 changes: 152 additions & 0 deletions service/history/api/addtasks/api.go
@@ -0,0 +1,152 @@
// The MIT License
//
// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved.
//
// Copyright (c) 2020 Uber Technologies, Inc.
//
// 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 addtasks

import (
"context"
"fmt"

commonpb "go.temporal.io/api/common/v1"
"go.temporal.io/api/serviceerror"

"go.temporal.io/server/api/historyservice/v1"
"go.temporal.io/server/common/definition"
"go.temporal.io/server/common/persistence"
"go.temporal.io/server/service/history/shard"
"go.temporal.io/server/service/history/tasks"
)

type (
// TaskDeserializer is a trimmed version of [go.temporal.io/server/common/persistence/serialization.Serializer] that
// requires only the DeserializeTask method.
TaskDeserializer interface {
DeserializeTask(category tasks.Category, blob commonpb.DataBlob) (tasks.Task, error)
}
)

const (
// maxTasksPerRequest is the maximum number of tasks that can be added in a single AddTasks API call. We set this to
// prevent the history service from OOMing when a client sends a request with a large number of tasks because we
// will deserialize all tasks in memory before adding them to the queue.
maxTasksPerRequest = 1000
)

// Invoke is the implementation of the history service's AddTasks API. This exposes the [shard.Context.AddTasks] API via
// the history service. This method works by batching tasks by workflow run, and then invoking the relevant shard's
// AddTasks API for each task batch. See [historyservice.HistoryServiceClient.AddTasks] for more details. We don't do
// any validation on the shard ID because that must have been done by whoever provided the shard.Context to this method.
func Invoke(
ctx context.Context,
shardContext shard.Context,
deserializer TaskDeserializer,
numShards int,
req *historyservice.AddTasksRequest,
) (*historyservice.AddTasksResponse, error) {
if len(req.Tasks) > maxTasksPerRequest {
return nil, serviceerror.NewInvalidArgument(fmt.Sprintf(
"Too many tasks in request: %d > %d",
len(req.Tasks),
maxTasksPerRequest,
))
}

if len(req.Tasks) == 0 {
return nil, serviceerror.NewInvalidArgument("No tasks in request")
}

taskBatches := make(map[definition.WorkflowKey]map[tasks.Category][]tasks.Task)

for i, task := range req.Tasks {
if task == nil {
return nil, serviceerror.NewInvalidArgument(fmt.Sprintf("Nil task at index: %d", i))
}

category, ok := getCategoryByName(task.Category)
if !ok {
return nil, serviceerror.NewInvalidArgument(fmt.Sprintf(
"Invalid task category: %s",
task.Category,
))
}

if task.Blob == nil {
return nil, serviceerror.NewInvalidArgument(fmt.Sprintf(
"Task blob is nil at index: %d",
i,
))
}

deserializedTask, err := deserializer.DeserializeTask(category, *task.Blob)
if err != nil {
return nil, err
}

shardID := tasks.GetShardIDForTask(deserializedTask, numShards)
if shardID != int(req.ShardId) {
return nil, serviceerror.NewInvalidArgument(fmt.Sprintf(
"Task is for wrong shard: index = %d, task shard = %d, request shard = %d",
i, shardID, req.ShardId,
))
}

workflowKey := definition.NewWorkflowKey(
deserializedTask.GetNamespaceID(),
deserializedTask.GetWorkflowID(),
deserializedTask.GetRunID(),
)
if _, ok := taskBatches[workflowKey]; !ok {
taskBatches[workflowKey] = make(map[tasks.Category][]tasks.Task, 1)
}

taskBatches[workflowKey][category] = append(taskBatches[workflowKey][category], deserializedTask)
}

for workflowKey, taskBatch := range taskBatches {
err := shardContext.AddTasks(ctx, &persistence.AddHistoryTasksRequest{
ShardID: shardContext.GetShardID(),
RangeID: shardContext.GetRangeID(),
NamespaceID: workflowKey.NamespaceID,
WorkflowID: workflowKey.WorkflowID,
RunID: workflowKey.RunID,
Tasks: taskBatch,
})
if err != nil {
return nil, err
}
}

return &historyservice.AddTasksResponse{}, nil
}

func getCategoryByName(categoryName string) (tasks.Category, bool) {
categories := tasks.GetCategories()
for _, category := range categories {
if category.Name() == categoryName {
return category, true
}
}

return tasks.Category{}, false
}

0 comments on commit ad6e47a

Please sign in to comment.