From d762528921061725f47eb7086b8dabccd0e935c6 Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Sun, 18 Jun 2023 15:41:39 -0700 Subject: [PATCH 1/6] It compiles --- temporal-kotlin/build.gradle | 1 + ...otKotlinWorkflowInboundCallsInterceptor.kt | 61 ++ .../async/DynamicKotlinWorkflowDefinition.kt | 82 +++ .../internal/async/KotlinActivityStub.kt | 56 ++ .../internal/async/KotlinActivityStubImpl.kt | 56 ++ .../temporal/internal/async/KotlinWorkflow.kt | 138 ++++ .../internal/async/KotlinWorkflowContext.kt | 395 +++++++++++ .../async/KotlinWorkflowDefinition.kt | 13 + .../async/KotlinWorkflowExecutionHandler.kt | 161 +++++ .../KotlinWorkflowImplementationFactory.kt | 375 +++++++++++ .../internal/async/KotlinWorkflowInfoImpl.kt | 59 ++ .../internal/async/KotlinWorkflowInternal.kt | 619 ++++++++++++++++++ .../async/TemporalCoroutineDispatcher.kt | 109 +++ .../kotlin/interceptors/WorkerInterceptor.kt | 97 +++ .../interceptors/WorkerInterceptorBase.kt | 32 + .../WorkflowInboundCallsInterceptor.kt | 100 +++ .../WorkflowInboundCallsInterceptorBase.kt | 48 ++ .../WorkflowOutboundCallsInterceptor.kt | 175 +++++ .../WorkflowOutboundCallsInterceptorBase.kt | 127 ++++ .../kotlin/workflow/KotlinDynamicWorkflow.kt | 51 ++ .../temporal/workflow/KotlinWorkflowInfo.kt | 137 ++++ 21 files changed, 2892 insertions(+) create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt diff --git a/temporal-kotlin/build.gradle b/temporal-kotlin/build.gradle index 7b7359254..99527e076 100644 --- a/temporal-kotlin/build.gradle +++ b/temporal-kotlin/build.gradle @@ -22,6 +22,7 @@ dependencies { compileOnly project(':temporal-sdk') implementation "org.jetbrains.kotlin:kotlin-reflect" + implementation("org.jetbrains.kotlinx:kotlinx-coroutines-core:1.7.1") implementation "com.fasterxml.jackson.datatype:jackson-datatype-jsr310" implementation "com.fasterxml.jackson.datatype:jackson-datatype-jdk8" diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt new file mode 100644 index 000000000..5c1a749fa --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor + +/** + * Provides core functionality for a root WorkflowInboundCallsInterceptor that is reused by specific + * root RootWorkflowInboundCallsInterceptor implementations inside [ ] and [POJOWorkflowImplementationFactory] + * + * + * Root `WorkflowInboundCallsInterceptor` is an interceptor that should be at the end of + * the [WorkflowInboundCallsInterceptor] interceptors chain and which encapsulates calls into + * Temporal internals while providing a WorkflowInboundCallsInterceptor interface for chaining on + * top of it. + */ +abstract class BaseRootKotlinWorkflowInboundCallsInterceptor(protected val workflowContext: KotlinWorkflowContext) : + WorkflowInboundCallsInterceptor { + override suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) { + workflowContext.initHeadOutboundCallsInterceptor(outboundCalls) + } + + override suspend fun handleSignal(input: WorkflowInboundCallsInterceptor.SignalInput) { + TODO("Not yet implemented") + // workflowContext.handleInterceptedSignal(input) + } + + + override fun handleQuery(input: WorkflowInboundCallsInterceptor.QueryInput): WorkflowInboundCallsInterceptor.QueryOutput { + TODO("Implement") +// return workflowContext.handleInterceptedQuery(input) + } + + override fun validateUpdate(input: WorkflowInboundCallsInterceptor.UpdateInput) { + TODO("Implement") +// workflowContext.handleInterceptedValidateUpdate(input) + } + + override suspend fun executeUpdate(input: WorkflowInboundCallsInterceptor.UpdateInput): WorkflowInboundCallsInterceptor.UpdateOutput { + TODO("Implement") +// return workflowContext.handleInterceptedExecuteUpdate(input) + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt new file mode 100644 index 000000000..a22c401a9 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.api.common.v1.Payloads +import io.temporal.common.converter.DataConverter +import io.temporal.common.converter.EncodedValues +import io.temporal.common.converter.Values +import io.temporal.common.interceptors.Header +import io.temporal.internal.sync.WorkflowInternal +import io.temporal.kotlin.interceptors.WorkerInterceptor +import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import io.temporal.kotlin.workflow.KotlinDynamicWorkflow +import io.temporal.workflow.DynamicWorkflow +import io.temporal.workflow.Functions.Func + +internal class DynamicKotlinWorkflowDefinition( + private val factory: Func, + private val workerInterceptors: Array, + private val dataConverter: DataConverter +) : KotlinWorkflowDefinition { + private var workflowInvoker: WorkflowInboundCallsInterceptor? = null + + override suspend fun initialize() { + val workflowContext: KotlinWorkflowContext = KotlinWorkflowInternal.rootWorkflowContext + workflowInvoker = RootWorkflowInboundCallsInterceptor(workflowContext) + for (workerInterceptor in workerInterceptors) { + workflowInvoker = workerInterceptor.interceptWorkflow(workflowInvoker!!) + } + workflowContext.initHeadInboundCallsInterceptor(workflowInvoker!!) + workflowInvoker!!.init(workflowContext) + } + + override suspend fun execute(header: Header?, input: Payloads?): Payloads? { + val args: Values = EncodedValues(input, dataConverter) + val result = workflowInvoker!!.execute( + WorkflowInboundCallsInterceptor.WorkflowInput(header!!, arrayOf(args)) + ) + return dataConverter.toPayloads(result!!.result).orElse(null) + } + + internal inner class RootWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext?) : + BaseRootKotlinWorkflowInboundCallsInterceptor( + workflowContext!! + ) { + private var workflow: KotlinDynamicWorkflow? = null + + override suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) { + super.init(outboundCalls!!) + newInstance() + WorkflowInternal.registerListener(workflow) + } + + override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { + val result = workflow!!.execute(input.arguments[0] as EncodedValues) + return WorkflowInboundCallsInterceptor.WorkflowOutput(result) + } + + private fun newInstance() { + check(workflow == null) { "Already called" } + workflow = factory.apply() + } + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt new file mode 100644 index 000000000..3e953dd00 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.workflow.Promise +import java.lang.reflect.Type + +/** + * KotlinActivityStub is used to call an activity without referencing an interface it implements. This is + * useful to call activities when their type is not known at compile time or to execute activities + * implemented in other languages. Created through [Workflow.newActivityStub]. + */ +interface KotlinActivityStub { + /** + * Executes an activity by its type name and arguments. Blocks until the activity completion. + * + * @param activityName name of an activity type to execute. + * @param resultClass the expected return type of the activity. Use Void.class for activities that + * return void type. + * @param args arguments of the activity. + * @param return type. + * @return an activity result. + */ + suspend fun execute(activityName: String, resultClass: Class, vararg args: Any): R? + + /** + * Executes an activity by its type name and arguments. Blocks until the activity completion. + * + * @param activityName name of an activity type to execute. + * @param resultClass the expected return class of the activity. Use Void.class for activities + * that return void type. + * @param resultType the expected return type of the activity. Differs from resultClass for + * generic types. + * @param args arguments of the activity. + * @param return type. + * @return an activity result. + */ + suspend fun execute(activityName: String, resultClass: Class, resultType: Type, vararg args: Any): R? +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt new file mode 100644 index 000000000..70db08f08 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.activity.ActivityOptions +import io.temporal.common.interceptors.Header +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import java.lang.reflect.Type + +internal class KotlinActivityStubImpl( + options: ActivityOptions?, + private val activityExecutor: WorkflowOutboundCallsInterceptor +) : KotlinActivityStub { + + private val options: ActivityOptions = ActivityOptions.newBuilder(options).validateAndBuildWithDefaults(); + + override suspend fun execute(activityName: String, resultClass: Class, vararg args: Any): R? { + return activityExecutor + .executeActivity( + WorkflowOutboundCallsInterceptor.ActivityInput( + activityName, resultClass, resultClass, args, options, Header.empty() + ) + ).result + } + + override suspend fun execute( + activityName: String, + resultClass: Class, + resultType: Type, + vararg args: Any + ): R? { + return activityExecutor + .executeActivity( + WorkflowOutboundCallsInterceptor.ActivityInput( + activityName, resultClass, resultType, args, options, Header.empty() + ) + ).result + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt new file mode 100644 index 000000000..b6fbf7572 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt @@ -0,0 +1,138 @@ +package io.temporal.internal.async + +import io.temporal.api.common.v1.Payloads +import io.temporal.api.common.v1.WorkflowExecution +import io.temporal.api.enums.v1.EventType +import io.temporal.api.history.v1.HistoryEvent +import io.temporal.api.query.v1.WorkflowQuery +import io.temporal.client.WorkflowClient +import io.temporal.common.context.ContextPropagator +import io.temporal.common.converter.DataConverter +import io.temporal.common.converter.DefaultDataConverter +import io.temporal.internal.replay.ReplayWorkflow +import io.temporal.internal.replay.ReplayWorkflowContext +import io.temporal.internal.replay.WorkflowContext +import io.temporal.internal.statemachines.UpdateProtocolCallback +import io.temporal.internal.sync.* +import io.temporal.internal.worker.WorkflowExecutorCache +import io.temporal.worker.WorkflowImplementationOptions +import kotlinx.coroutines.async +import kotlinx.coroutines.cancel +import kotlinx.coroutines.launch +import org.slf4j.LoggerFactory +import java.util.* + +class KotlinWorkflow( + private val namespace: String, + private val workflowExecution: WorkflowExecution, + private val workflow: KotlinWorkflowDefinition, + workflowImplementationOptions: WorkflowImplementationOptions?, + private val dataConverter: DataConverter, + private val cache: WorkflowExecutorCache, + private val contextPropagators: List?, + private val defaultDeadlockDetectionTimeout: Long +) : ReplayWorkflow { + + private val log = LoggerFactory.getLogger(KotlinWorkflow::class.java) + + private val workflowImplementationOptions = workflowImplementationOptions + ?: WorkflowImplementationOptions.getDefaultInstance() + + private val workflowContext = + KotlinWorkflowContext( + namespace, + workflowExecution, + this.workflowImplementationOptions, + dataConverter, + contextPropagators + ) + + private val dispatcher = TemporalCoroutineDispatcher() + private val coroutineDispatcher = TemporalCallbackCoroutineDispatcher(dispatcher) + private val scope = TemporalScope() + + private var executionHandler: KotlinWorkflowExecutionHandler? = null + + override fun start(event: HistoryEvent, context: ReplayWorkflowContext) { + require( + !(event.eventType != EventType.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED + || !event.hasWorkflowExecutionStartedEventAttributes()) + ) { "first event is not WorkflowExecutionStarted, but " + event.eventType } + val startEvent = event.workflowExecutionStartedEventAttributes + val workflowType = startEvent.workflowType + requireNotNull(workflow) { "Unknown workflow type: $workflowType" } + workflowContext!!.setReplayContext(context) + + executionHandler = KotlinWorkflowExecutionHandler( + workflowContext, workflow, startEvent, workflowImplementationOptions!! + ) + // The following order is ensured by this code and DeterministicRunner implementation: + // 1. workflow.initialize + // 2. signal handler (if signalWithStart was called) + // 3. main workflow method + scope.launch(dispatcher) { + workflow.initialize() + async { + executionHandler!!.runWorkflowMethod() + } + } + } + + override fun handleSignal(signalName: String, input: Optional?, eventId: Long) { + scope.launch(coroutineDispatcher) { + executionHandler!!.handleSignal(signalName, input, eventId) + } + } + + override fun handleUpdate( + updateName: String?, + input: Optional?, + eventId: Long, + callbacks: UpdateProtocolCallback? + ) { + TODO("Not yet implemented") + } + + override fun eventLoop(): Boolean { + if (executionHandler == null) { + return false + } + dispatcher!!.eventLoop(defaultDeadlockDetectionTimeout) + return dispatcher!!.isDone() || executionHandler!!.isDone // Do not wait for all other threads. + } + + override fun getOutput(): Optional { + return Optional.ofNullable(executionHandler!!.output) + } + + override fun cancel(reason: String?) { + TODO("Implement cancellation") +// runner!!.cancel(reason) + } + + override fun close() { + if (executionHandler != null) { + //TODO: Validate that cancel is the right operation to call here + dispatcher!!.cancel() + } + } + + override fun query(query: WorkflowQuery): Optional { + if (WorkflowClient.QUERY_TYPE_REPLAY_ONLY == query.queryType) { + return Optional.empty() + } + if (WorkflowClient.QUERY_TYPE_STACK_TRACE == query.queryType) { + // stack trace query result should be readable for UI even if user specifies a custom data + // converter + TODO("Implement stack trace if possible") +// return DefaultDataConverter.STANDARD_INSTANCE.toPayloads(runner!!.stackTrace()) + } + val args = if (query.hasQueryArgs()) Optional.of(query.queryArgs) else Optional.empty() + TODO("Implement query") +// return executionHandler!!.handleQuery(query.queryType, args) + } + + override fun getWorkflowContext(): WorkflowContext? { + return workflowContext + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt new file mode 100644 index 000000000..ac23125c1 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt @@ -0,0 +1,395 @@ +package io.temporal.internal.async + +import com.google.common.base.MoreObjects +import com.uber.m3.tally.Scope +import io.temporal.activity.ActivityOptions +import io.temporal.activity.LocalActivityOptions +import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes +import io.temporal.api.common.v1.ActivityType +import io.temporal.api.common.v1.Payload +import io.temporal.api.common.v1.Payloads +import io.temporal.api.common.v1.WorkflowExecution +import io.temporal.api.failure.v1.Failure +import io.temporal.api.taskqueue.v1.TaskQueue +import io.temporal.common.SearchAttributeUpdate +import io.temporal.common.context.ContextPropagator +import io.temporal.common.converter.DataConverter +import io.temporal.common.interceptors.Header +import io.temporal.failure.CanceledFailure +import io.temporal.internal.common.ActivityOptionUtils +import io.temporal.internal.common.HeaderUtils +import io.temporal.internal.common.ProtobufTimeUtils +import io.temporal.internal.common.SerializerUtils +import io.temporal.internal.replay.ReplayWorkflowContext +import io.temporal.internal.replay.WorkflowContext +import io.temporal.internal.statemachines.ExecuteActivityParameters +import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import io.temporal.payload.context.ActivitySerializationContext +import io.temporal.payload.context.WorkflowSerializationContext +import io.temporal.worker.WorkflowImplementationOptions +import io.temporal.workflow.Functions +import kotlinx.coroutines.suspendCancellableCoroutine +import org.slf4j.LoggerFactory +import java.lang.reflect.Type +import java.time.Duration +import java.util.* +import java.util.function.BiPredicate +import java.util.function.Supplier +import kotlin.coroutines.resumeWithException + +class KotlinWorkflowContext( + private val namespace: String, + private val workflowExecution: WorkflowExecution, + private var workflowImplementationOptions: WorkflowImplementationOptions?, + val dataConverter: DataConverter, + private val contextPropagators: List? +) : WorkflowContext, WorkflowOutboundCallsInterceptor { + + private val log = LoggerFactory.getLogger(KotlinWorkflowContext::class.java) + + private var headInboundInterceptor: WorkflowInboundCallsInterceptor? = null + private var headOutboundInterceptor: WorkflowOutboundCallsInterceptor? = null + + var defaultActivityOptions: ActivityOptions? = null + var defaultLocalActivityOptions: LocalActivityOptions? = null + + private var activityOptionsMap: Map? = null + private var localActivityOptionsMap: Map? = null + + init { + if (workflowImplementationOptions != null) { + defaultActivityOptions = workflowImplementationOptions!!.defaultActivityOptions + this.activityOptionsMap = HashMap(workflowImplementationOptions!!.activityOptions) + this.defaultLocalActivityOptions = workflowImplementationOptions!!.defaultLocalActivityOptions + this.localActivityOptionsMap = + HashMap(workflowImplementationOptions!!.localActivityOptions) + workflowImplementationOptions = WorkflowImplementationOptions.getDefaultInstance() + } + // initial values for headInboundInterceptor and headOutboundInterceptor until they initialized + // with actual interceptors through #initHeadInboundCallsInterceptor and + // #initHeadOutboundCallsInterceptor during initialization phase. + // See workflow.initialize() performed inside the workflow root thread inside + // SyncWorkflow#start(HistoryEvent, ReplayWorkflowContext) + // initial values for headInboundInterceptor and headOutboundInterceptor until they initialized + // with actual interceptors through #initHeadInboundCallsInterceptor and + // #initHeadOutboundCallsInterceptor during initialization phase. + // See workflow.initialize() performed inside the workflow root thread inside + // SyncWorkflow#start(HistoryEvent, ReplayWorkflowContext) + headInboundInterceptor = InitialWorkflowInboundCallsInterceptor(this) + headOutboundInterceptor = this + + } + + private val dataConverterWithCurrentWorkflowContext = dataConverter.withContext( + WorkflowSerializationContext(namespace, workflowExecution.getWorkflowId()) + ) + + fun setReplayContext(context: ReplayWorkflowContext) { + replayContext = context + } + + override fun getReplayContext(): ReplayWorkflowContext { + return replayContext + } + + fun getWorkflowOutboundInterceptor(): WorkflowOutboundCallsInterceptor { + return headOutboundInterceptor!! + } + + fun getWorkflowInboundInterceptor(): WorkflowInboundCallsInterceptor? { + return headInboundInterceptor + } + + fun initHeadOutboundCallsInterceptor(head: WorkflowOutboundCallsInterceptor) { + headOutboundInterceptor = head + } + + fun initHeadInboundCallsInterceptor(head: WorkflowInboundCallsInterceptor) { + headInboundInterceptor = head + //TODO: signal, query, update dispatchers +// signalDispatcher.setInboundCallsInterceptor(head) +// queryDispatcher.setInboundCallsInterceptor(head) +// updateDispatcher.setInboundCallsInterceptor(head) + } + + override fun mapWorkflowExceptionToFailure(failure: Throwable): Failure { + return dataConverterWithCurrentWorkflowContext.exceptionToFailure(failure) + } + + + override fun getWorkflowImplementationOptions(): WorkflowImplementationOptions { + return workflowImplementationOptions!! + } + + fun applyActivityOptions(activityTypeToOption: Map) { + if (activityOptionsMap == null) { + activityOptionsMap = HashMap(activityTypeToOption) + return + } + ActivityOptionUtils.mergePredefinedActivityOptions(activityOptionsMap!!, activityTypeToOption) + } + + fun applyLocalActivityOptions(activityTypeToOption: Map) { + if (localActivityOptionsMap == null) { + localActivityOptionsMap = HashMap(activityTypeToOption) + return + } + ActivityOptionUtils.mergePredefinedLocalActivityOptions( + localActivityOptionsMap!!, activityTypeToOption + ) + } + + override fun getLastCompletionResult(resultClass: Class, resultType: Type): R? { + return dataConverter.fromPayloads( + 0, Optional.ofNullable(replayContext.lastCompletionResult), resultClass, resultType + ) + } + + override fun getContextPropagators(): List? { + return contextPropagators + } + + override fun getPropagatedContexts(): MutableMap { + if (contextPropagators == null || contextPropagators.isEmpty()) { + return HashMap() + } + + val headerData: Map = HashMap(replayContext.header) + val contextData: MutableMap = HashMap() + for (propagator in contextPropagators) { + contextData[propagator.name] = propagator.deserializeContext(headerData) + } + + return contextData + } + + override suspend fun executeActivity(input: WorkflowOutboundCallsInterceptor.ActivityInput): WorkflowOutboundCallsInterceptor.ActivityOutput { + val serializationContext = ActivitySerializationContext( + replayContext.namespace, + replayContext.workflowId, + replayContext.workflowType.name, + input.activityName, + // input.getOptions().getTaskQueue() may be not specified, workflow task queue is used + // by the Server in this case + if (input.options != null && input.options.taskQueue != null) input.options.taskQueue else replayContext.taskQueue, + false + ) + val dataConverterWithActivityContext = dataConverter.withContext(serializationContext) + val args = dataConverterWithActivityContext.toPayloads(*input.args) + try { + val output = executeActivityOnce(input.activityName, input.options, input.header, args) + val result = if (input.resultType !== Void.TYPE) dataConverterWithActivityContext.fromPayloads( + 0, Optional.of(output.result), input.resultClass, input.resultType + ) else null + return WorkflowOutboundCallsInterceptor.ActivityOutput(output.activityId, result) + } catch (e: FailureWrapperException) { + throw dataConverterWithActivityContext.failureToException(e.failure) + } + } + + override suspend fun executeLocalActivity(input: WorkflowOutboundCallsInterceptor.LocalActivityInput): WorkflowOutboundCallsInterceptor.LocalActivityOutput { + TODO("Not yet implemented") + } + + override suspend fun executeChildWorkflow(input: WorkflowOutboundCallsInterceptor.ChildWorkflowInput): WorkflowOutboundCallsInterceptor.ChildWorkflowOutput { + TODO("Not yet implemented") + } + + override fun newRandom(): Random = replayContext.newRandom() + + + override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput): WorkflowOutboundCallsInterceptor.SignalExternalOutput { + TODO("Not yet implemented") + } + + override fun cancelWorkflow(input: WorkflowOutboundCallsInterceptor.CancelWorkflowInput): WorkflowOutboundCallsInterceptor.CancelWorkflowOutput { + TODO("Not yet implemented") + } + + override suspend fun sleep(duration: Duration) { + TODO("Not yet implemented") + } + + override suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean { + TODO("Not yet implemented") + } + + override suspend fun await(reason: String?, unblockCondition: Supplier) { + TODO("Not yet implemented") + } + + override fun sideEffect(resultClass: Class, resultType: Type, func: Functions.Func): R? { + TODO("Not yet implemented") + } + + override fun mutableSideEffect( + id: String, + resultClass: Class, + resultType: Type, + updated: BiPredicate, + func: Functions.Func + ): R? { + TODO("Not yet implemented") + } + + override fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int { + TODO("Not yet implemented") + } + + override fun continueAsNew(input: WorkflowOutboundCallsInterceptor.ContinueAsNewInput) { + TODO("Not yet implemented") + } + + override fun registerQuery(input: WorkflowOutboundCallsInterceptor.RegisterQueryInput) { + TODO("Not yet implemented") + } + + override fun registerSignalHandlers(input: WorkflowOutboundCallsInterceptor.RegisterSignalHandlersInput) { + TODO("Not yet implemented") + } + + override fun registerUpdateHandlers(input: WorkflowOutboundCallsInterceptor.RegisterUpdateHandlersInput) { + TODO("Not yet implemented") + } + + override fun registerDynamicSignalHandler(handler: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput) { + TODO("Not yet implemented") + } + + override fun registerDynamicQueryHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicQueryHandlerInput) { + TODO("Not yet implemented") + } + + override fun registerDynamicUpdateHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicUpdateHandlerInput) { + TODO("Not yet implemented") + } + + override fun randomUUID(): UUID { + TODO("Not yet implemented") + } + + override fun upsertSearchAttributes(searchAttributes: Map) { + TODO("Not yet implemented") + } + + override fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>) { + TODO("Not yet implemented") + } + + override fun currentTimeMillis(): Long { + TODO("Not yet implemented") + } + + val metricScope: Scope + get() = replayContext.metricsScope + + private suspend fun executeActivityOnce( + activityTypeName: String, options: ActivityOptions, header: Header, input: Optional + ): ActivityOutput { + val params: ExecuteActivityParameters = constructExecuteActivityParameters(activityTypeName, options, header, input) + + return suspendCancellableCoroutine { continuation -> + var activityId: String? = null + val activityOutput = replayContext.scheduleActivityTask( + params + ) { output: Optional, failure: Failure? -> + if (failure == null) { + continuation.resume(ActivityOutput(activityId!!, output.get()), onCancellation = null); + } else { + continuation.resumeWithException(FailureWrapperException(failure)) + } + } + activityId = activityOutput.activityId + // Handle coroutine cancellation + continuation.invokeOnCancellation { reason: Throwable? -> + activityOutput.cancellationHandle.apply(CanceledFailure(reason.toString())) + } + } + } + + // TODO: this is copy of the similar method in SyncWorkflowContext. Extract to common class. + private fun constructExecuteActivityParameters( + name: String, options: ActivityOptions, header: Header, input: Optional + ): ExecuteActivityParameters { + var taskQueue = options.taskQueue + if (taskQueue == null) { + taskQueue = replayContext.taskQueue + } + val attributes = ScheduleActivityTaskCommandAttributes.newBuilder() + .setActivityType(ActivityType.newBuilder().setName(name)) + .setTaskQueue(TaskQueue.newBuilder().setName(taskQueue)) + .setScheduleToStartTimeout( + ProtobufTimeUtils.toProtoDuration(options.scheduleToStartTimeout) + ) + .setStartToCloseTimeout( + ProtobufTimeUtils.toProtoDuration(options.startToCloseTimeout) + ) + .setScheduleToCloseTimeout( + ProtobufTimeUtils.toProtoDuration(options.scheduleToCloseTimeout) + ) + .setHeartbeatTimeout(ProtobufTimeUtils.toProtoDuration(options.heartbeatTimeout)) + .setRequestEagerExecution( + !options.isEagerExecutionDisabled && (taskQueue == replayContext.taskQueue) + ) + input.ifPresent { value: Payloads? -> + attributes.input = value + } + val retryOptions = options.retryOptions + if (retryOptions != null) { + attributes.setRetryPolicy(SerializerUtils.toRetryPolicy(retryOptions)) + } + + // Set the context value. Use the context propagators from the ActivityOptions + // if present, otherwise use the ones configured on the WorkflowContext + var propagators = options.contextPropagators + if (propagators == null) { + propagators = contextPropagators + } + val grpcHeader = HeaderUtils.toHeaderGrpc(header, extractContextsAndConvertToBytes(propagators)) + attributes.header = grpcHeader + if (options.versioningIntent != null) { + attributes.useCompatibleVersion = options + .versioningIntent + .determineUseCompatibleFlag(replayContext.taskQueue == options.taskQueue) + } + return ExecuteActivityParameters(attributes, options.cancellationType) + } + + // TODO: this is copy of the similar method in SyncWorkflowContext. Extract to common class. + private fun extractContextsAndConvertToBytes( + contextPropagators: List? + ): Header? { + if (contextPropagators == null) { + return null + } + val result: MutableMap = HashMap() + for (propagator in contextPropagators) { + result.putAll(propagator.serializeContext(propagator.currentContext)) + } + return Header(result) + } + + // TODO: this is copy of the similar method in SyncWorkflowContext. Extract to common class. + private class FailureWrapperException(val failure: Failure) : + RuntimeException() + + class ActivityOutput(val activityId: String, val result: R) + + /** + * This WorkflowInboundCallsInterceptor is used during creation of the initial root workflow + * thread and should be replaced with another specific implementation during initialization stage + * `workflow.initialize()` performed inside the workflow root thread. + * + * @see SyncWorkflow.start + */ + private class InitialWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext) : + BaseRootKotlinWorkflowInboundCallsInterceptor(workflowContext) { + override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { + throw UnsupportedOperationException( + "SyncWorkflowContext should be initialized with a non-initial WorkflowInboundCallsInterceptor " + + "before #execute can be called" + ) + } + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt new file mode 100644 index 000000000..3278840e8 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt @@ -0,0 +1,13 @@ +package io.temporal.internal.async + +import io.temporal.api.common.v1.Payloads +import io.temporal.common.interceptors.Header +import java.util.* + +interface KotlinWorkflowDefinition { + + /** Always called first. */ + suspend fun initialize() + + suspend fun execute(header: Header?, input: Payloads?): Payloads? +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt new file mode 100644 index 000000000..2ed4011cd --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt @@ -0,0 +1,161 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.api.common.v1.Payloads +import io.temporal.api.history.v1.WorkflowExecutionStartedEventAttributes +import io.temporal.common.interceptors.Header +import io.temporal.failure.CanceledFailure +import io.temporal.failure.TemporalFailure +import io.temporal.internal.sync.DestroyWorkflowThreadError +import io.temporal.internal.sync.WorkflowInternal +import io.temporal.internal.worker.WorkflowExecutionException +import io.temporal.serviceclient.CheckedExceptionWrapper +import io.temporal.worker.WorkflowImplementationOptions +import io.temporal.workflow.Workflow +import org.slf4j.LoggerFactory +import java.util.* + +internal class KotlinWorkflowExecutionHandler( + private val context: KotlinWorkflowContext, + private val workflow: KotlinWorkflowDefinition, + private val attributes: WorkflowExecutionStartedEventAttributes, + private val implementationOptions: WorkflowImplementationOptions +) { + var output: Payloads? = null + var isDone = false + + suspend fun runWorkflowMethod() { + try { + val input = if (attributes.hasInput()) attributes.input else null + output = workflow.execute(Header(attributes.header), input) + } catch (e: Throwable) { + applyWorkflowFailurePolicyAndRethrow(e) + } finally { + isDone = true + } + } + +// fun cancel(reason: String?) {} +// fun close() {} + + suspend fun handleSignal(signalName: String?, input: Optional?, eventId: Long) { + try { + TODO("Not yet implemented") +// context.handleSignal(signalName, input, eventId) + } catch (e: Throwable) { + applyWorkflowFailurePolicyAndRethrow(e) + } + } + + // +// fun handleQuery(type: String?, args: Optional?): Optional { +// return context.handleQuery(type, args) +// } +// +// fun handleValidateUpdate(updateName: String?, input: Optional?, eventId: Long) { +// try { +// context.handleValidateUpdate(updateName, input, eventId) +// } catch (e: Throwable) { +// applyWorkflowFailurePolicyAndRethrow(e) +// } +// } +// +// fun handleExecuteUpdate( +// updateName: String?, input: Optional?, eventId: Long +// ): Optional { +// try { +// return context.handleExecuteUpdate(updateName, input, eventId) +// } catch (e: Throwable) { +// applyWorkflowFailurePolicyAndRethrow(e) +// } +// return Optional.empty() +// } +// + private fun applyWorkflowFailurePolicyAndRethrow(e: Throwable) { + if (e is DestroyWorkflowThreadError) { + throw e + } + val exception = WorkflowInternal.unwrap(e) + val failTypes = implementationOptions.failWorkflowExceptionTypes + (exception as? TemporalFailure)?.let { throwAndFailWorkflowExecution(it) } + for (failType in failTypes) { + if (failType.isAssignableFrom(exception.javaClass)) { + throwAndFailWorkflowExecution(exception) + } + } + throw CheckedExceptionWrapper.wrap(exception) + } + + private fun throwAndFailWorkflowExecution(exception: Throwable) { + val replayWorkflowContext = context.getReplayContext() + val fullReplayDirectQueryName = replayWorkflowContext.fullReplayDirectQueryName + val info = Workflow.getInfo() + if (fullReplayDirectQueryName != null) { + if (log.isDebugEnabled + && !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) + ) { + log.debug( + "Replayed workflow execution failure WorkflowId='{}', RunId={}, WorkflowType='{}' for direct query QueryType='{}'", + info.workflowId, + info.runId, + info.workflowType, + fullReplayDirectQueryName, + exception + ) + } + } else { + if (log.isWarnEnabled + && !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) + ) { + log.warn( + "Workflow execution failure WorkflowId='{}', RunId={}, WorkflowType='{}'", + info.workflowId, + info.runId, + info.workflowType, + exception + ) + } + } + throw WorkflowExecutionException(context.mapWorkflowExceptionToFailure(exception)) + } + + /** + * @return true if both workflow cancellation is requested and the exception contains a + * cancellation exception in the chain + */ + private fun requestedCancellation(cancelRequested: Boolean, exception: Throwable): Boolean { + return cancelRequested && isCanceledCause(exception) + } + + companion object { + private val log = LoggerFactory.getLogger(KotlinWorkflowExecutionHandler::class.java) + private fun isCanceledCause(exception: Throwable): Boolean { + var exception: Throwable? = exception + while (exception != null) { + if (exception is CanceledFailure) { + return true + } + exception = exception.cause + } + return false + } + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt new file mode 100644 index 000000000..8541cbfca --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt @@ -0,0 +1,375 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import com.google.common.base.Preconditions +import com.google.common.collect.ImmutableSet +import io.temporal.api.common.v1.Payloads +import io.temporal.api.common.v1.WorkflowExecution +import io.temporal.api.common.v1.WorkflowType +import io.temporal.common.context.ContextPropagator +import io.temporal.common.converter.DataConverter +import io.temporal.common.interceptors.Header +import io.temporal.common.metadata.POJOWorkflowImplMetadata +import io.temporal.common.metadata.POJOWorkflowInterfaceMetadata +import io.temporal.common.metadata.WorkflowMethodType +import io.temporal.failure.CanceledFailure +import io.temporal.internal.common.env.ReflectionUtils +import io.temporal.internal.replay.ReplayWorkflow +import io.temporal.internal.replay.ReplayWorkflowFactory +import io.temporal.internal.sync.* +import io.temporal.internal.worker.SingleWorkerOptions +import io.temporal.internal.worker.WorkflowExecutionException +import io.temporal.internal.worker.WorkflowExecutorCache +import io.temporal.kotlin.interceptors.WorkerInterceptor +import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import io.temporal.kotlin.workflow.KotlinDynamicWorkflow +import io.temporal.payload.context.WorkflowSerializationContext +import io.temporal.serviceclient.CheckedExceptionWrapper +import io.temporal.worker.TypeAlreadyRegisteredException +import io.temporal.worker.WorkflowImplementationOptions +import io.temporal.workflow.DynamicWorkflow +import io.temporal.workflow.Functions.Func +import io.temporal.workflow.Functions.Func1 +import org.slf4j.LoggerFactory +import java.lang.reflect.InvocationTargetException +import java.lang.reflect.Method +import java.util.* + +class KotlinWorkflowImplementationFactory( + singleWorkerOptions: SingleWorkerOptions, + workerInterceptors: Array, + cache: WorkflowExecutorCache, + namespace: String +) : ReplayWorkflowFactory { + private val workerInterceptors: Array + private val dataConverter: DataConverter + private val contextPropagators: List + private val defaultDeadlockDetectionTimeout: Long + + /** Key: workflow type name, Value: function that creates SyncWorkflowDefinition instance. */ + private val workflowDefinitions = + Collections.synchronizedMap(HashMap>()) + + /** Factories providing instances of workflow classes. */ + private val workflowInstanceFactories = Collections.synchronizedMap(HashMap, Func<*>>()) + + /** If present then it is called for any unknown workflow type. */ + private var dynamicWorkflowImplementationFactory: Func? = null + private val implementationOptions = Collections.synchronizedMap(HashMap()) + private val cache: WorkflowExecutorCache + private val namespace: String + + init { + Objects.requireNonNull(singleWorkerOptions) + dataConverter = singleWorkerOptions.dataConverter + this.workerInterceptors = Objects.requireNonNull(workerInterceptors) + this.cache = cache + contextPropagators = singleWorkerOptions.contextPropagators + defaultDeadlockDetectionTimeout = singleWorkerOptions.defaultDeadlockDetectionTimeout + this.namespace = namespace + } + + fun registerWorkflowImplementationTypes( + options: WorkflowImplementationOptions, workflowImplementationTypes: Array> + ) { + for (type in workflowImplementationTypes) { + registerWorkflowImplementationType(options, type) + } + } + + /** + * @param clazz has to be a workflow interface class. The only exception is if it's a + * DynamicWorkflow class. + */ + fun addWorkflowImplementationFactory( + options: WorkflowImplementationOptions, clazz: Class, factory: Func + ) { + if (DynamicWorkflow::class.java.isAssignableFrom(clazz)) { + if (dynamicWorkflowImplementationFactory != null) { + throw TypeAlreadyRegisteredException( + "KotlinDynamicWorkflow", + "An implementation of KotlinDynamicWorkflow or its factory is already registered with the worker" + ) + } + dynamicWorkflowImplementationFactory = factory as Func + return + } + workflowInstanceFactories[clazz] = factory + val workflowMetadata = POJOWorkflowInterfaceMetadata.newInstance(clazz) + require(workflowMetadata.workflowMethod.isPresent) { "Workflow interface doesn't contain a method annotated with @WorkflowMethod: $clazz" } + val methodsMetadata = workflowMetadata.methodsMetadata + for (methodMetadata in methodsMetadata) { + when (methodMetadata.type) { + WorkflowMethodType.WORKFLOW -> { + val typeName = methodMetadata.name + if (workflowDefinitions.containsKey(typeName)) { + throw TypeAlreadyRegisteredException( + typeName, + "\"$typeName\" workflow type is already registered with the worker" + ) + } + workflowDefinitions[typeName] = Func1 { execution: WorkflowExecution -> + KotlinWorkflowImplementation( + clazz, + methodMetadata.workflowMethod, + dataConverter.withContext( + WorkflowSerializationContext(namespace, execution.workflowId) + ) + ) + } + implementationOptions[typeName] = options + } + WorkflowMethodType.SIGNAL -> {} + } + } + } + + private fun registerWorkflowImplementationType( + options: WorkflowImplementationOptions, workflowImplementationClass: Class + ) { +// if (KotlinDynamicWorkflow::class.java.isAssignableFrom(workflowImplementationClass)) { +// addWorkflowImplementationFactory( +// options, +// workflowImplementationClass, +// Func { +// try { +// val newInstance: T? = workflowImplementationClass.getDeclaredConstructor().newInstance() +// return@Func newInstance +// } catch (e: NoSuchMethodException) { +// // Error to fail workflow task as this can be fixed by a new deployment. +// throw Error( +// "Failure instantiating workflow implementation class " +// + workflowImplementationClass.name, +// e +// ) +// } catch (e: InstantiationException) { +// throw Error( +// "Failure instantiating workflow implementation class " +// + workflowImplementationClass.name, +// e +// ) +// } catch (e: IllegalAccessException) { +// throw Error( +// "Failure instantiating workflow implementation class " +// + workflowImplementationClass.name, +// e +// ) +// } catch (e: InvocationTargetException) { +// throw Error( +// "Failure instantiating workflow implementation class " +// + workflowImplementationClass.name, +// e +// ) +// } +// }) +// return +// } + val workflowMetadata = POJOWorkflowImplMetadata.newInstance(workflowImplementationClass) + val workflowMethods = workflowMetadata.workflowMethods + require(!workflowMethods.isEmpty()) { + ("Workflow implementation doesn't implement any interface " + + "with a workflow method annotated with @WorkflowMethod: " + + workflowImplementationClass) + } + for (workflowMethod in workflowMethods) { + val workflowName = workflowMethod.name + val method = workflowMethod.workflowMethod + val definition = Func1 { execution: WorkflowExecution -> + KotlinWorkflowImplementation( + workflowImplementationClass, + method, + dataConverter.withContext( + WorkflowSerializationContext(namespace, execution.workflowId) + ) + ) + } + check(!workflowDefinitions.containsKey(workflowName)) { "$workflowName workflow type is already registered with the worker" } + workflowDefinitions[workflowName] = definition + implementationOptions[workflowName] = options + } + } + + private fun getWorkflowDefinition( + workflowType: WorkflowType, workflowExecution: WorkflowExecution + ): KotlinWorkflowDefinition { + val factory = workflowDefinitions[workflowType.name] + if (factory == null) { + if (dynamicWorkflowImplementationFactory != null) { + return DynamicKotlinWorkflowDefinition( + dynamicWorkflowImplementationFactory!!, workerInterceptors, dataConverter + ) + } + throw Error( + "Unknown workflow type \"" + + workflowType.name + + "\". Known types are " + + workflowDefinitions.keys + ) + } + return try { + factory.apply(workflowExecution) + } catch (e: Exception) { + throw Error(e) + } + } + + override fun getWorkflow( + workflowType: WorkflowType, workflowExecution: WorkflowExecution + ): ReplayWorkflow { + val workflow = getWorkflowDefinition(workflowType, workflowExecution) + val workflowImplementationOptions = implementationOptions[workflowType.name] + val dataConverterWithWorkflowContext = dataConverter.withContext( + WorkflowSerializationContext(namespace, workflowExecution.workflowId) + ) + return KotlinWorkflow( + namespace, + workflowExecution, + workflow, +// SignalDispatcher(dataConverterWithWorkflowContext), +// QueryDispatcher(dataConverterWithWorkflowContext), +// UpdateDispatcher(dataConverterWithWorkflowContext), + workflowImplementationOptions, + dataConverter, + cache, + contextPropagators, + defaultDeadlockDetectionTimeout + ) + } + + override fun isAnyTypeSupported(): Boolean { + return !workflowDefinitions.isEmpty() || dynamicWorkflowImplementationFactory != null + } + + private inner class KotlinWorkflowImplementation( + private val workflowImplementationClass: Class<*>, + private val workflowMethod: Method, + // don't pass it down to other classes, it's a "cached" instance for internal usage only + private val dataConverterWithWorkflowContext: DataConverter + ) : KotlinWorkflowDefinition { + private var workflowInvoker: WorkflowInboundCallsInterceptor? = null + override suspend fun initialize() { + val workflowContext = KotlinWorkflowInternal.rootWorkflowContext + workflowInvoker = RootWorkflowInboundCallsInterceptor(workflowContext) + for (workerInterceptor in workerInterceptors) { + workflowInvoker = workerInterceptor.interceptWorkflow(workflowInvoker!!) + } + workflowContext.initHeadInboundCallsInterceptor(workflowInvoker!!) + workflowInvoker!!.init(workflowContext) + } + + @Throws(CanceledFailure::class, WorkflowExecutionException::class) + override suspend fun execute(header: Header?, input: Payloads?): Payloads? { + val args = dataConverterWithWorkflowContext.fromPayloads( + Optional.ofNullable(input), workflowMethod.parameterTypes, workflowMethod.genericParameterTypes + ) + Preconditions.checkNotNull(workflowInvoker, "initialize not called") + val result = workflowInvoker!!.execute(WorkflowInboundCallsInterceptor.WorkflowInput(header, args)) + return if (workflowMethod.returnType == Void.TYPE) { + null + } else dataConverterWithWorkflowContext.toPayloads( + result.result + ).orElse(null) + } + + private inner class RootWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext) : + BaseRootKotlinWorkflowInboundCallsInterceptor(workflowContext) { + private var workflow: Any? = null + + override suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) { + super.init(outboundCalls) + newInstance() + WorkflowInternal.registerListener(workflow) + } + + override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { + return try { + val result = workflowMethod.invoke(workflow, *input.arguments) + WorkflowInboundCallsInterceptor.WorkflowOutput(result) + } catch (e: IllegalAccessException) { + throw CheckedExceptionWrapper.wrap(e) + } catch (e: InvocationTargetException) { + val target = e.targetException + throw CheckedExceptionWrapper.wrap(target) + } + } + + protected fun newInstance() { + val factory = workflowInstanceFactories[workflowImplementationClass] + workflow = if (factory != null) { + factory.apply() + } else { + try { + workflowImplementationClass.getDeclaredConstructor().newInstance() + } catch (e: NoSuchMethodException) { + // Error to fail workflow task as this can be fixed by a new deployment. + throw Error( + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, + e + ) + } catch (e: InstantiationException) { + throw Error( + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, + e + ) + } catch (e: IllegalAccessException) { + throw Error( + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, + e + ) + } catch (e: InvocationTargetException) { + throw Error( + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, + e + ) + } + } + } + } + } + + override fun toString(): String { + return ("POJOWorkflowImplementationFactory{" + + "registeredWorkflowTypes=" + + workflowDefinitions.keys + + '}') + } + + companion object { + private val log = LoggerFactory.getLogger(KotlinWorkflowImplementationFactory::class.java) + val WORKFLOW_HANDLER_STACKTRACE_CUTOFF = ImmutableSet.builder() // POJO + .add( + ReflectionUtils.getMethodNameForStackTraceCutoff( + KotlinWorkflowImplementation::class.java, "execute", Header::class.java, Optional::class.java + ) + ) // Dynamic + .add( + ReflectionUtils.getMethodNameForStackTraceCutoff( + DynamicKotlinWorkflowDefinition::class.java, "execute", Header::class.java, Optional::class.java + ) + ) + .build() + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt new file mode 100644 index 000000000..23ab07b1b --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import io.temporal.api.common.v1.SearchAttributes +import io.temporal.internal.replay.ReplayWorkflowContext +import io.temporal.workflow.KotlinWorkflowInfo +import java.time.Duration + +internal class KotlinWorkflowInfoImpl( + private val context: ReplayWorkflowContext, +) : KotlinWorkflowInfo { + override val namespace: String = context.namespace + override val workflowId: String = context.workflowId + override val workflowType: String = context.workflowType.name + override val runId: String = context.runId + override val firstExecutionRunId: String = context.firstExecutionRunId + override val continuedExecutionRunId: String? = context.continuedExecutionRunId.orElseGet(null) + override val originalExecutionRunId: String = context.originalExecutionRunId + override val taskQueue: String = context.taskQueue + override val workflowRunTimeout: Duration? = context.workflowRunTimeout + override val workflowExecutionTimeout: Duration? = context.workflowExecutionTimeout + override val runStartedTimestampMillis: Long = context.runStartedTimestampMillis + override val searchAttributes: SearchAttributes? = context.searchAttributes + override val parentWorkflowId: String? = + if (context.parentWorkflowExecution != null) context.parentWorkflowExecution.workflowId else null + override val parentRunId: String? = + if (context.parentWorkflowExecution != null) context.parentWorkflowExecution.runId else null + override val attempt: Int = context.attempt + override val cronSchedule: String? = context.cronSchedule + override val historyLength: Long = context.currentWorkflowTaskStartedEventId + + override fun toString(): String { + return "KotlinWorkflowInfoImpl(context=$context, namespace='$namespace', workflowId='$workflowId', " + + "workflowType='$workflowType', runId='$runId', firstExecutionRunId='$firstExecutionRunId', " + + "continuedExecutionRunId=$continuedExecutionRunId, originalExecutionRunId='$originalExecutionRunId', " + + "taskQueue='$taskQueue', workflowRunTimeout=$workflowRunTimeout, " + + "workflowExecutionTimeout=$workflowExecutionTimeout, runStartedTimestampMillis=$runStartedTimestampMillis, " + + "searchAttributes=$searchAttributes, parentWorkflowId=$parentWorkflowId, parentRunId=$parentRunId, " + + "attempt=$attempt, cronSchedule=$cronSchedule, historyLength=$historyLength)" + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt new file mode 100644 index 000000000..1ed71d369 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt @@ -0,0 +1,619 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.internal.async + +import com.uber.m3.tally.Scope +import io.temporal.activity.ActivityOptions +import io.temporal.activity.LocalActivityOptions +import io.temporal.api.failure.v1.Failure +import io.temporal.common.converter.DataConverter +import io.temporal.common.metadata.POJOWorkflowInterfaceMetadata +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import io.temporal.workflow.KotlinWorkflowInfo +import io.temporal.workflow.WorkflowMethod +import kotlinx.coroutines.GlobalScope.coroutineContext +import java.util.* + +/** + * Never reference directly. It is public only because Java doesn't have internal package support. + */ +object KotlinWorkflowInternal { + const val DEFAULT_VERSION = -1 + + /** + * Register query or queries implementation object. There is no need to register top level + * workflow implementation object as it is done implicitly. Only methods annotated with @[ ] are registered. TODO(quinn) LIES! + */ + fun registerListener(implementation: Any) { + TODO("Implement") +// if (implementation is DynamicSignalHandler) { +// workflowOutboundInterceptor +// .registerDynamicSignalHandler( +// WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput( +// implementation +// ) +// ) +// return +// } +// if (implementation is DynamicQueryHandler) { +// workflowOutboundInterceptor +// .registerDynamicQueryHandler( +// WorkflowOutboundCallsInterceptor.RegisterDynamicQueryHandlerInput( +// implementation +// ) +// ) +// return +// } +// if (implementation is DynamicUpdateHandler) { +// workflowOutboundInterceptor +// .registerDynamicUpdateHandler( +// WorkflowOutboundCallsInterceptor.RegisterDynamicUpdateHandlerInput( +// implementation +// ) +// ) +// return +// } +// val cls: Class<*> = implementation.javaClass +// val workflowMetadata = POJOWorkflowImplMetadata.newListenerInstance(cls) +// for (methodMetadata in workflowMetadata.queryMethods) { +// val method = methodMetadata.workflowMethod +// workflowOutboundInterceptor +// .registerQuery( +// WorkflowOutboundCallsInterceptor.RegisterQueryInput( +// methodMetadata.name, +// method.parameterTypes, +// method.genericParameterTypes, label@ +// Func1 { args: Array -> +// try { +// return@label method.invoke(implementation, *args) +// } catch (e: Throwable) { +// throw CheckedExceptionWrapper.wrap(e) +// } +// }) +// ) +// } +// val requests: MutableList = ArrayList() +// for (methodMetadata in workflowMetadata.signalMethods) { +// val method = methodMetadata.workflowMethod +// requests.add( +// WorkflowOutboundCallsInterceptor.SignalRegistrationRequest( +// methodMetadata.name, +// method.parameterTypes, +// method.genericParameterTypes +// ) { args: Array -> +// try { +// method.invoke(implementation, *args) +// } catch (e: Throwable) { +// throw CheckedExceptionWrapper.wrap(e) +// } +// }) +// } +// if (!requests.isEmpty()) { +// workflowOutboundInterceptor +// .registerSignalHandlers( +// WorkflowOutboundCallsInterceptor.RegisterSignalHandlersInput(requests) +// ) +// } +// +// // Get all validators and lazily assign them to update handlers as we see them. +// val validators: MutableMap = +// HashMap(workflowMetadata.updateValidatorMethods.size) +// for (methodMetadata in workflowMetadata.updateValidatorMethods) { +// val method = methodMetadata.workflowMethod +// val updateValidatorMethod = method.getAnnotation( +// UpdateValidatorMethod::class.java +// ) +// require(!validators.containsKey(updateValidatorMethod.updateName())) { "Duplicate validator for update handle " + updateValidatorMethod.updateName() } +// validators[updateValidatorMethod.updateName()] = methodMetadata +// } +// val updateRequests: MutableList = ArrayList() +// for (methodMetadata in workflowMetadata.updateMethods) { +// val method = methodMetadata.workflowMethod +// val updateMethod = method.getAnnotation(UpdateMethod::class.java) +// var updateMethodName: String = updateMethod.name() +// if (updateMethodName.isEmpty()) { +// updateMethodName = method.name +// } +// // Check if any validators claim they are the validator for this update +// val validatorMethodMetadata = validators.remove(updateMethodName) +// var validatorMethod: Method? +// if (validatorMethodMetadata != null) { +// validatorMethod = validatorMethodMetadata.workflowMethod +// require(Arrays.equals(validatorMethod.parameterTypes, method.parameterTypes)) { +// ("Validator for: " +// + updateMethodName +// + " type parameters do not match the update handle") +// } +// } else { +// validatorMethod = null +// } +// updateRequests.add( +// WorkflowOutboundCallsInterceptor.UpdateRegistrationRequest( +// methodMetadata.name, +// method.parameterTypes, +// method.genericParameterTypes, +// { args: Array -> +// try { +// validatorMethod?.invoke?.invoke() +// } catch (e: Throwable) { +// throw CheckedExceptionWrapper.wrap(e) +// } +// }, label@ +// Func1, Any> { args: Array -> +// try { +// return@label method.invoke(implementation, *args) +// } catch (e: Throwable) { +// throw CheckedExceptionWrapper.wrap(e) +// } +// }) +// ) +// } +// if (!updateRequests.isEmpty()) { +// workflowOutboundInterceptor +// .registerUpdateHandlers( +// WorkflowOutboundCallsInterceptor.RegisterUpdateHandlersInput(updateRequests) +// ) +// } +// require(validators.isEmpty()) { +// ("Missing update methods for update validator(s): " +// + Joiner.on(", ").join(validators.keys)) +// } + } + + /** Should be used to get current time instead of [System.currentTimeMillis] */ + fun currentTimeMillis(): Long { + return workflowOutboundInterceptor.currentTimeMillis() + } + + fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { + rootWorkflowContext.defaultActivityOptions = activityOptions + } + + fun applyActivityOptions(activityTypeToOptions: Map) { + rootWorkflowContext.applyActivityOptions(activityTypeToOptions) + } + + fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { + rootWorkflowContext.defaultLocalActivityOptions = localActivityOptions + } + + fun applyLocalActivityOptions( + activityTypeToOptions: Map + ) { + rootWorkflowContext.applyLocalActivityOptions(activityTypeToOptions) + } + +// /** +// * Creates client stub to activities that implement given interface. +// * +// * @param activityInterface interface type implemented by activities +// * @param options options that together with the properties of [ ] specify the activity invocation parameters +// * @param activityMethodOptions activity method-specific invocation parameters +// */ +//// fun newActivityStub( +//// activityInterface: Class?, +//// options: ActivityOptions?, +//// activityMethodOptions: Map? +//// ): T { +//// // Merge the activity options we may have received from the workflow with the options we may +//// // have received in WorkflowImplementationOptions. +//// var options = options +//// val context = rootWorkflowContext +//// options = options ?: context.getDefaultActivityOptions() +//// val mergedActivityOptionsMap: Map +//// @Nonnull val predefinedActivityOptions = context.getActivityOptions() +//// if (activityMethodOptions != null && !activityMethodOptions.isEmpty() +//// && predefinedActivityOptions.isEmpty() +//// ) { +//// // we need to merge only in this case +//// mergedActivityOptionsMap = HashMap(predefinedActivityOptions) +//// ActivityOptionUtils.mergePredefinedActivityOptions( +//// mergedActivityOptionsMap, activityMethodOptions +//// ) +//// } else { +//// mergedActivityOptionsMap = MoreObjects.firstNonNull( +//// activityMethodOptions, +//// MoreObjects.firstNonNull(predefinedActivityOptions, emptyMap()) +//// ) +//// } +//// val invocationHandler = ActivityInvocationHandler.newInstance( +//// activityInterface, +//// options, +//// mergedActivityOptionsMap, +//// context.getWorkflowOutboundInterceptor() +//// ) +//// return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) +//// } +//// +// /** +// * Creates client stub to local activities that implement given interface. +// * +// * @param activityInterface interface type implemented by activities +// * @param options options that together with the properties of [ ] specify the activity invocation parameters +// * @param activityMethodOptions activity method-specific invocation parameters +// */ +// fun newLocalActivityStub( +// activityInterface: Class?, +// options: LocalActivityOptions?, +// activityMethodOptions: Map? +// ): T { +// // Merge the activity options we may have received from the workflow with the options we may +// // have received in WorkflowImplementationOptions. +// var options = options +// val context = rootWorkflowContext +// options = options ?: context.getDefaultLocalActivityOptions() +// val mergedLocalActivityOptionsMap: Map +// @Nonnull val predefinedLocalActivityOptions = context.getLocalActivityOptions() +// if (activityMethodOptions != null && !activityMethodOptions.isEmpty() +// && predefinedLocalActivityOptions.isEmpty() +// ) { +// // we need to merge only in this case +// mergedLocalActivityOptionsMap = HashMap(predefinedLocalActivityOptions) +// ActivityOptionUtils.mergePredefinedLocalActivityOptions( +// mergedLocalActivityOptionsMap, activityMethodOptions +// ) +// } else { +// mergedLocalActivityOptionsMap = MoreObjects.firstNonNull( +// activityMethodOptions, +// MoreObjects.firstNonNull(predefinedLocalActivityOptions, emptyMap()) +// ) +// } +// val invocationHandler = LocalActivityInvocationHandler.newInstance( +// activityInterface, +// options, +// mergedLocalActivityOptionsMap, +// workflowOutboundInterceptor +// ) +// return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) +// } + + fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + return KotlinActivityStubImpl(options, workflowOutboundInterceptor) + } + +// fun newUntypedLocalActivityStub(options: LocalActivityOptions?): KotlinActivityStub { +// return LocalActivityStubImpl.newInstance(options, workflowOutboundInterceptor) +// } +// +// fun newChildWorkflowStub( +// workflowInterface: Class, options: ChildWorkflowOptions? +// ): T { +// return Proxy.newProxyInstance( +// workflowInterface.classLoader, arrayOf(workflowInterface, StubMarker::class.java, AsyncMarker::class.java), +// ChildWorkflowInvocationHandler( +// workflowInterface, options, workflowOutboundInterceptor +// ) +// ) as T +// } +// +// fun newExternalWorkflowStub( +// workflowInterface: Class, execution: WorkflowExecution? +// ): T { +// return Proxy.newProxyInstance( +// workflowInterface.classLoader, arrayOf(workflowInterface, StubMarker::class.java, AsyncMarker::class.java), +// ExternalWorkflowInvocationHandler( +// workflowInterface, execution, workflowOutboundInterceptor +// ) +// ) as T +// } +// +// fun getWorkflowExecution(workflowStub: Any): Promise { +// if (workflowStub is StubMarker) { +// val stub = workflowStub.__getUntypedStub() +// return (stub as ChildWorkflowStub).execution +// } +// throw IllegalArgumentException( +// "Not a workflow stub created through Workflow.newChildWorkflowStub: $workflowStub" +// ) +// } +// +// fun newUntypedChildWorkflowStub( +// workflowType: String?, options: ChildWorkflowOptions? +// ): ChildWorkflowStub { +// return ChildWorkflowStubImpl(workflowType, options, workflowOutboundInterceptor) +// } +// +// fun newUntypedExternalWorkflowStub(execution: WorkflowExecution?): ExternalWorkflowStub { +// return ExternalWorkflowStubImpl(execution, workflowOutboundInterceptor) +// } +// +// /** +// * Creates client stub that can be used to continue this workflow as new. +// * +// * @param workflowInterface interface type implemented by the next generation of workflow +// */ +// fun newContinueAsNewStub( +// workflowInterface: Class, options: ContinueAsNewOptions? +// ): T { +// return Proxy.newProxyInstance( +// workflowInterface.classLoader, arrayOf>(workflowInterface), +// ContinueAsNewWorkflowInvocationHandler( +// workflowInterface, options, workflowOutboundInterceptor +// ) +// ) as T +// } +// +// /** +// * Execute activity by name. +// * +// * @param name name of the activity +// * @param resultClass activity return type +// * @param args list of activity arguments +// * @param activity return type +// * @return activity result +// */ +// fun executeActivity( +// name: String?, options: ActivityOptions?, resultClass: Class?, resultType: Type?, vararg args: Any? +// ): R? { +// val result = workflowOutboundInterceptor +// .executeActivity( +// WorkflowOutboundCallsInterceptor.ActivityInput( +// name, resultClass, resultType, args, options, Header.empty() +// ) +// ) +// .result +// if (AsyncInternal.isAsync()) { +// AsyncInternal.setAsyncResult(result) +// return null // ignored +// } +// return result.get() +// } +// +// @Throws(DestroyWorkflowThreadError::class) +// fun await(reason: String?, unblockCondition: Supplier?) { +// workflowOutboundInterceptor.await(reason, unblockCondition) +// } +// +// @Throws(DestroyWorkflowThreadError::class) +// fun await(timeout: Duration?, reason: String?, unblockCondition: Supplier?): Boolean { +// return workflowOutboundInterceptor.await(timeout, reason, unblockCondition) +// } +// +// fun sideEffect(resultClass: Class?, resultType: Type?, func: Func?): R { +// return workflowOutboundInterceptor.sideEffect(resultClass, resultType, func) +// } +// +// fun mutableSideEffect( +// id: String?, resultClass: Class?, resultType: Type?, updated: BiPredicate?, func: Func? +// ): R { +// return workflowOutboundInterceptor +// .mutableSideEffect(id, resultClass, resultType, updated, func) +// } +// +// fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int { +// return workflowOutboundInterceptor.getVersion(changeId, minSupported, maxSupported) +// } +// +// fun promiseAllOf(promises: Iterable?>?): Promise { +// return AllOfPromise(promises) +// } +// +// fun promiseAllOf(vararg promises: Promise<*>?): Promise { +// return AllOfPromise(promises) +// } +// +// fun promiseAnyOf(promises: Iterable?>?): Promise { +// return CompletablePromiseImpl.promiseAnyOf(promises) +// } +// +// fun promiseAnyOf(vararg promises: Promise<*>?): Promise { +// return CompletablePromiseImpl.promiseAnyOf(promises) +// } +// +// fun newCancellationScope(detached: Boolean, runnable: Runnable?): CancellationScope { +// return CancellationScopeImpl(detached, runnable) +// } +// +// fun newCancellationScope( +// detached: Boolean, proc: Proc1? +// ): CancellationScope { +// return CancellationScopeImpl(detached, proc) +// } +// +// fun currentCancellationScope(): CancellationScopeImpl { +// return CancellationScopeImpl.current() +// } +// +// fun wrap(e: Throwable?): RuntimeException { +// return CheckedExceptionWrapper.wrap(e) +// } +// +// fun unwrap(e: Throwable?): Throwable { +// return CheckedExceptionWrapper.unwrap(e) +// } +// +// /** Returns false if not under workflow code. */ +// val isReplaying: Boolean +// get() { +// val thread = DeterministicRunnerImpl.currentThreadInternalIfPresent() +// return thread.isPresent && rootWorkflowContext.isReplaying() +// } +// +// fun getMemo(key: String?, valueClass: Class?, genericType: Type?): T? { +// val memo = rootWorkflowContext.getReplayContext().getMemo(key) ?: return null +// return dataConverter.fromPayload(memo, valueClass, genericType) +// } +// +// fun retry( +// options: RetryOptions, expiration: Optional?, fn: Func? +// ): R { +// return WorkflowRetryerInternal.retry( +// options.toBuilder().validateBuildWithDefaults(), expiration, fn +// ) +// } +// +// fun continueAsNew( +// workflowType: String?, options: ContinueAsNewOptions?, args: Array? +// ) { +// workflowOutboundInterceptor +// .continueAsNew( +// WorkflowOutboundCallsInterceptor.ContinueAsNewInput( +// workflowType, options, args, Header.empty() +// ) +// ) +// } +// +// fun continueAsNew( +// workflowType: String?, +// options: ContinueAsNewOptions?, +// args: Array?, +// outboundCallsInterceptor: WorkflowOutboundCallsInterceptor +// ) { +// outboundCallsInterceptor.continueAsNew( +// WorkflowOutboundCallsInterceptor.ContinueAsNewInput( +// workflowType, options, args, Header.empty() +// ) +// ) +// } +// +// fun cancelWorkflow(execution: WorkflowExecution?): Promise { +// return workflowOutboundInterceptor +// .cancelWorkflow(WorkflowOutboundCallsInterceptor.CancelWorkflowInput(execution)) +// .result +// } +// +// fun sleep(duration: Duration?) { +// workflowOutboundInterceptor.sleep(duration) +// } +// +// val isWorkflowThread: Boolean +// get() = WorkflowThreadMarker.isWorkflowThread() +// +// fun deadlockDetectorOff(func: Func): T { +// if (isWorkflowThread) { +// getWorkflowThread().lockDeadlockDetector().use { ignored -> return func.apply() } +// } else { +// return func.apply() +// } +// } + + val workflowInfo: KotlinWorkflowInfo + get() = KotlinWorkflowInfoImpl(rootWorkflowContext.getReplayContext()) + + val metricsScope: Scope + get() = rootWorkflowContext.metricScope + + + fun randomUUID(): UUID { + return rootWorkflowContext.randomUUID() + } + + fun newRandom(): Random { + return rootWorkflowContext.newRandom() + } + +// private val isLoggingEnabledInReplay: Boolean +// private get() = rootWorkflowContext.isLoggingEnabledInReplay() +// fun getLogger(clazz: Class<*>?): Logger { +// val logger = LoggerFactory.getLogger(clazz) +// return ReplayAwareLogger( +// logger, +// ReplayAware { obj: KotlinWorkflowInternal? -> isReplaying }, +// Supplier { obj: KotlinWorkflowInternal? -> isLoggingEnabledInReplay }) +// } +// +// fun getLogger(name: String?): Logger { +// val logger = LoggerFactory.getLogger(name) +// return ReplayAwareLogger( +// logger, +// ReplayAware { obj: KotlinWorkflowInternal? -> isReplaying }, +// Supplier { obj: KotlinWorkflowInternal? -> isLoggingEnabledInReplay }) +// } + +// fun getLastCompletionResult(resultClass: Class?, resultType: Type?): R? { +// return rootWorkflowContext.getLastCompletionResult(resultClass, resultType) +// } +// +// fun getSearchAttribute(name: String?): T? { +// val list = getSearchAttributeValues(name) ?: return null +// Preconditions.checkState(list.size > 0) +// Preconditions.checkState( +// list.size == 1, +// "search attribute with name '%s' contains a list '%s' of values instead of a single value", +// name, +// list +// ) +// return list[0] +// } +// +// fun getSearchAttributeValues(name: String?): List? { +// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes ?: return null +// val decoded = SearchAttributesUtil.decode(searchAttributes, name!!) +// return if (decoded != null) Collections.unmodifiableList(decoded) else null +// } +// +// val searchAttributes: Map> +// get() { +// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes +// ?: return emptyMap() +// return Collections.unmodifiableMap(SearchAttributesUtil.decode(searchAttributes)) +// } +// +// val typedSearchAttributes: SearchAttributes +// get() { +// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes +// return SearchAttributesUtil.decodeTyped(searchAttributes) +// } +// +// fun upsertSearchAttributes(searchAttributes: Map?) { +// workflowOutboundInterceptor.upsertSearchAttributes(searchAttributes) +// } +// +// fun upsertTypedSearchAttributes( +// vararg searchAttributeUpdates: SearchAttributeUpdate<*>? +// ) { +// workflowOutboundInterceptor.upsertTypedSearchAttributes(*searchAttributeUpdates) +// } + + val dataConverter: DataConverter + get() = rootWorkflowContext.dataConverter + + /** + * Name of the workflow type the interface defines. It is either the interface short name * or + * value of [WorkflowMethod.name] parameter. + * + * @param workflowInterfaceClass interface annotated with @WorkflowInterface + */ + fun getWorkflowType(workflowInterfaceClass: Class<*>?): String { + val metadata = POJOWorkflowInterfaceMetadata.newInstance(workflowInterfaceClass) + return metadata.workflowType.get() + } + + // Temporal Failure Values are additional user payload and serialized using user data + // converter + val previousRunFailure: Optional + get() = Optional.ofNullable(rootWorkflowContext.getReplayContext().previousRunFailure) // Temporal Failure Values are additional user payload and serialized using user data + // converter + .map { f: Failure? -> + dataConverter.failureToException( + f!! + ) + } + + private val workflowOutboundInterceptor: WorkflowOutboundCallsInterceptor + private get() = rootWorkflowContext.getWorkflowOutboundInterceptor() + + val rootWorkflowContext: KotlinWorkflowContext + get() { + val temporalCoroutineContext = coroutineContext[TemporalCoroutineContext] + ?: throw Error("Called from non workflow thread or coroutine") + return temporalCoroutineContext.workflowContext + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt new file mode 100644 index 000000000..59dad84e6 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt @@ -0,0 +1,109 @@ +package io.temporal.internal.async + +import kotlinx.coroutines.* +import java.lang.Runnable +import java.util.* +import java.util.concurrent.DelayQueue +import java.util.concurrent.Delayed +import java.util.concurrent.TimeUnit +import kotlin.coroutines.AbstractCoroutineContextElement +import kotlin.coroutines.CoroutineContext + +@OptIn(InternalCoroutinesApi::class) +class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { + + private val queue: Queue = LinkedList() + private val callbackQueue: Queue = LinkedList() + private val delayQueue: DelayQueue = DelayQueue() + + override fun dispatch(context: CoroutineContext, block: Runnable) { + queue.add(block) + } + + fun dispatchCallback(context: CoroutineContext, block: Runnable) { + callbackQueue.add(block) + } + + //TODO: deadlock detector + fun eventLoop(defaultDeadlockDetectionTimeout: Long): Boolean { +// println("eventLoop begin") + if (isDone()) { + println("eventLoop completed") + return false + } + + while (callbackQueue.isNotEmpty()) { + val block = callbackQueue.poll() + block.run() + } + + while (queue.isNotEmpty()) { + val block = queue.poll() + block.run() + } + + while (true) { +// println("delayedContinuation while begin count=" + delayQueue.size) + + val delayedContinuation = delayQueue.poll() ?: break + println("delayedContinuation returned") + with(delayedContinuation.continuation) { resumeUndispatched(Unit) } + } + + return true + } + + fun isDone() = queue.isEmpty() && callbackQueue.isEmpty() && delayQueue.isEmpty() + + override fun scheduleResumeAfterDelay(timeMillis: Long, continuation: CancellableContinuation) { + println("scheduleResumeAfterDelay delay=$timeMillis") + delayQueue.add(DelayedContinuation(timeMillis, continuation)) + } + + private class DelayedContinuation( + private val delayTime: Long, + val continuation: CancellableContinuation + ) : Delayed { + private val startTime = System.currentTimeMillis() + delayTime + + override fun compareTo(other: Delayed): Int { + return (getDelay(TimeUnit.MILLISECONDS) - other.getDelay(TimeUnit.MILLISECONDS)).toInt() + } + + override fun getDelay(unit: TimeUnit): Long { + val diff = startTime - System.currentTimeMillis() + return unit.convert(diff, TimeUnit.MILLISECONDS) + } + } +} + +/** + * Dispatcher used to schedule callback coroutines which should run before any other coroutines. + * This is to avoid signal loss due to UnhandledCommand. + */ +@OptIn(InternalCoroutinesApi::class) +class TemporalCallbackCoroutineDispatcher(val dispatcher: TemporalCoroutineDispatcher) : CoroutineDispatcher(), Delay { + + override fun scheduleResumeAfterDelay(timeMillis: Long, continuation: CancellableContinuation) { + dispatcher.scheduleResumeAfterDelay(timeMillis, continuation) + } + + override fun dispatch(context: CoroutineContext, block: Runnable) { + dispatcher.dispatchCallback(context, block) + } +} + +internal class TemporalScope(private val workflowContext: KotlinWorkflowContext) : CoroutineScope { + //TODO: Add argument to the Temporal context. + override val coroutineContext: CoroutineContext = TemporalCoroutineContext(workflowContext) + + // CoroutineScope is used intentionally for user-friendly representation + override fun toString(): String = "CoroutineScope(coroutineContext=$coroutineContext)" +} + +class TemporalCoroutineContext(val workflowContext: KotlinWorkflowContext) : + AbstractCoroutineContextElement(TemporalCoroutineContext) { + override val key = Key + + companion object Key : CoroutineContext.Key +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt new file mode 100644 index 000000000..5545ccca9 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt @@ -0,0 +1,97 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +import io.temporal.common.Experimental +import io.temporal.common.interceptors.ActivityInboundCallsInterceptor + +/** + * Intercepts workflow and activity executions. + * + * + * Prefer extending [WorkerInterceptorBase] and overriding only the methods you need + * instead of implementing this interface directly. [WorkerInterceptorBase] provides correct + * default implementations to all the methods of this interface. + * + * + * You may want to start your implementation with this initial structure: + * + *
`
+ * public class CustomWorkerInterceptor extends WorkerInterceptorBase {
+ * // remove if you don't need to have a custom WorkflowInboundCallsInterceptor or
+ * // WorkflowOutboundCallsInterceptor
+ * @Override
+ * public WorkflowInboundCallsInterceptor interceptWorkflow(WorkflowInboundCallsInterceptor next) {
+ * return new CustomWorkflowInboundCallsInterceptor(next) {
+ * // remove if you don't need to have a custom WorkflowOutboundCallsInterceptor
+ * @Override
+ * public void init(WorkflowOutboundCallsInterceptor outboundCalls) {
+ * next.init(new CustomWorkflowOutboundCallsInterceptor(outboundCalls));
+ * }
+ * };
+ * }
+ *
+ * // remove if you don't need to have a custom ActivityInboundCallsInterceptor
+ * @Override
+ * public ActivityInboundCallsInterceptor interceptActivity(ActivityInboundCallsInterceptor next) {
+ * return new CustomActivityInboundCallsInterceptor(next);
+ * }
+ *
+ * private static class CustomWorkflowInboundCallsInterceptor
+ * extends WorkflowInboundCallsInterceptorBase {
+ * public CustomWorkflowInboundCallsInterceptor(WorkflowInboundCallsInterceptor next) {
+ * super(next);
+ * }
+ *
+ * // override only the methods you need
+ * }
+ *
+ * private static class CustomWorkflowOutboundCallsInterceptor
+ * extends WorkflowOutboundCallsInterceptorBase {
+ * public CustomWorkflowOutboundCallsInterceptor(WorkflowOutboundCallsInterceptor next) {
+ * super(next);
+ * }
+ *
+ * // override only the methods you need
+ * }
+ *
+ * private static class CustomActivityInboundCallsInterceptor
+ * extends ActivityInboundCallsInterceptorBase {
+ * public CustomActivityInboundCallsInterceptor(ActivityInboundCallsInterceptor next) {
+ * super(next);
+ * }
+ *
+ * // override only the methods you need
+ * }
+ * }
+`
* + */ +@Experimental +interface WorkerInterceptor { + /** + * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `next` [WorkflowInboundCallsInterceptor], but it may change the input parameters. + * + * @param next an existing interceptor instance to be proxied by the interceptor created inside + * this method + * @return an interceptor that passes all the calls to `next` + */ + fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor + fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt new file mode 100644 index 000000000..f96908eac --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt @@ -0,0 +1,32 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +import io.temporal.common.interceptors.ActivityInboundCallsInterceptor + +class WorkerInterceptorBase : WorkerInterceptor { + override fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor { + return next + } + + override fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor { + return next + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt new file mode 100644 index 000000000..f4a8ee400 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +import io.temporal.common.Experimental +import io.temporal.common.interceptors.Header + +/** + * Intercepts inbound calls to the workflow execution on the worker side. + * + * + * An instance should be created in [ ][WorkerInterceptor.interceptWorkflow]. + * + * + * The calls to this interceptor are executed under workflow context, all the rules and + * restrictions on the workflow code apply. See [io.temporal.workflow]. + * + * + * Prefer extending [WorkflowInboundCallsInterceptorBase] and overriding only the methods + * you need instead of implementing this interface directly. [ ] provides correct default implementations to all the methods + * of this interface. + * + * + * The implementation must forward all the calls to `next`, but it may change the input + * parameters. + * + * @see WorkerInterceptor.interceptWorkflow + */ +@Experimental +interface WorkflowInboundCallsInterceptor { + class WorkflowInput(val header: Header?, val arguments: Array) + class WorkflowOutput(val result: Any?) + class SignalInput(val signalName: String, val arguments: Array, val eventId: Long) + class QueryInput(val queryName: String, val arguments: Array) + class QueryOutput(val result: Any) + + @Experimental + class UpdateInput(val updateName: String, val arguments: Array) + + @Experimental + class UpdateOutput(val result: Any) + + /** + * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `outboundCalls`, but it may change the input parameters. + * + * + * The instance should be passed into the {next.init(newWorkflowOutboundCallsInterceptor)}. + * + * @param outboundCalls an existing interceptor instance to be proxied by the interceptor created + * inside this method + * @see WorkerInterceptor.interceptWorkflow for the definition of "next" {@link + * * WorkflowInboundCallsInterceptor} + */ + suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) + + /** + * Called when workflow main method is called. + * + * @return result of the workflow execution. + */ + suspend fun execute(input: WorkflowInput): WorkflowOutput + + /** Called when signal is delivered to a workflow execution. */ + suspend fun handleSignal(input: SignalInput) + + /** Called when a workflow is queried. */ + fun handleQuery(input: QueryInput): QueryOutput + + /** + * Called when update workflow execution request is delivered to a workflow execution, before the + * update is executed. + */ + @Experimental + fun validateUpdate(input: UpdateInput) + + /** + * Called when update workflow execution request is delivered to a workflow execution, after + * passing the validator. + */ + @Experimental + suspend fun executeUpdate(input: UpdateInput): UpdateOutput + +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt new file mode 100644 index 000000000..e2880580f --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +/** Convenience base class for WorkflowInboundCallsInterceptor implementations. */ +class WorkflowInboundCallsInterceptorBase(private val next: WorkflowInboundCallsInterceptor) : + WorkflowInboundCallsInterceptor { + override suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) { + next.init(outboundCalls) + } + + override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { + return next.execute(input) + } + + override suspend fun handleSignal(input: WorkflowInboundCallsInterceptor.SignalInput) { + next.handleSignal(input) + } + + override fun handleQuery(input: WorkflowInboundCallsInterceptor.QueryInput): WorkflowInboundCallsInterceptor.QueryOutput { + return next.handleQuery(input) + } + + override fun validateUpdate(input: WorkflowInboundCallsInterceptor.UpdateInput) { + next.validateUpdate(input) + } + + override suspend fun executeUpdate(input: WorkflowInboundCallsInterceptor.UpdateInput): WorkflowInboundCallsInterceptor.UpdateOutput { + return next.executeUpdate(input) + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt new file mode 100644 index 000000000..058a1be1b --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -0,0 +1,175 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +import io.temporal.activity.ActivityOptions +import io.temporal.activity.LocalActivityOptions +import io.temporal.api.common.v1.WorkflowExecution +import io.temporal.common.Experimental +import io.temporal.common.SearchAttributeUpdate +import io.temporal.common.interceptors.Header +import io.temporal.workflow.* +import io.temporal.workflow.Functions.* +import java.lang.reflect.Type +import java.time.Duration +import java.util.* +import java.util.function.BiPredicate +import java.util.function.Supplier + +/** + * Can be used to intercept calls from to workflow code into the Temporal APIs. + * + * + * The calls to the interceptor are executed in the context of a workflow and must follow the + * same rules all the other workflow code follows. + * + * + * Prefer extending [WorkflowOutboundCallsInterceptorBase] and overriding only the methods + * you need instead of implementing this interface directly. [ ] provides correct default implementations to all the methods + * of this interface. + * + * + * An instance may be created in [ ][WorkflowInboundCallsInterceptor.init] and set by passing it + * into `init` method of the `next` [WorkflowInboundCallsInterceptor] The + * implementation must forward all the calls to the outbound interceptor passed as a `outboundCalls` parameter to the `init` call. + * + * @see WorkerInterceptor.interceptWorkflow for the definition of "next" {@link + * * WorkflowInboundCallsInterceptor}. + */ +@Experimental +interface WorkflowOutboundCallsInterceptor { + class ActivityInput( + val activityName: String, + val resultClass: Class, + val resultType: Type, + val args: Array, + val options: ActivityOptions, + val header: Header + ) + + class ActivityOutput(val activityId: String, val result: R?) + + class LocalActivityInput( + val activityName: String, + val resultClass: Class, + val resultType: Type, + val args: Array, + val options: LocalActivityOptions, + val header: Header + ) + + class LocalActivityOutput(val result: R?) + class ChildWorkflowInput( + val workflowId: String, + val workflowType: String, + val resultClass: Class, + val resultType: Type, + val args: Array, + val options: ChildWorkflowOptions, + val header: Header + ) + + class ChildWorkflowOutput(val result: Promise, val workflowExecution: Promise) + class SignalExternalInput(val execution: WorkflowExecution, val signalName: String, val args: Array) + class SignalExternalOutput(val result: Promise) + class CancelWorkflowInput(val execution: WorkflowExecution) + class CancelWorkflowOutput(val result: Promise) + class ContinueAsNewInput( + /** + * @return workflowType for the continue-as-new workflow run. null if continue-as-new should + * inherit the type of the original workflow run. + */ + val workflowType: String?, + /** + * @return options for the continue-as-new workflow run. Can be null, in that case the values + * will be taken from the original workflow run. + */ + val options: ContinueAsNewOptions?, + val args: Array, + val header: Header + ) + + class SignalRegistrationRequest( + val signalType: String, + val argTypes: Array>, + val genericArgTypes: Array, + val callback: Proc1> + ) + + class RegisterSignalHandlersInput(val requests: List) + + @Experimental + class UpdateRegistrationRequest( + val updateName: String, + val argTypes: Array>, + val genericArgTypes: Array, + val validateCallback: Proc1>, + val executeCallback: Func1, Any> + ) + + @Experimental + class RegisterUpdateHandlersInput(val requests: List) + class RegisterQueryInput( + val queryType: String, + val argTypes: Array>, + val genericArgTypes: Array, + val callback: Func1, Any> + ) + + class RegisterDynamicQueryHandlerInput(val handler: DynamicQueryHandler) + class RegisterDynamicSignalHandlerInput(val handler: DynamicSignalHandler) + + @Experimental + class RegisterDynamicUpdateHandlerInput(val handler: DynamicUpdateHandler) + + suspend fun executeActivity(input: ActivityInput): ActivityOutput + suspend fun executeLocalActivity(input: LocalActivityInput): LocalActivityOutput + suspend fun executeChildWorkflow(input: ChildWorkflowInput): ChildWorkflowOutput + fun newRandom(): Random + suspend fun signalExternalWorkflow(input: SignalExternalInput): SignalExternalOutput + fun cancelWorkflow(input: CancelWorkflowInput): CancelWorkflowOutput + + //TODO: Consider removing sleep and keep only built in delay + suspend fun sleep(duration: Duration) + suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean + suspend fun await(reason: String?, unblockCondition: Supplier) + fun sideEffect(resultClass: Class, resultType: Type, func: Func): R? + fun mutableSideEffect( + id: String, resultClass: Class, resultType: Type, updated: BiPredicate, func: Func + ): R? + + fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int + fun continueAsNew(input: ContinueAsNewInput) + fun registerQuery(input: RegisterQueryInput) + fun registerSignalHandlers(input: RegisterSignalHandlersInput) + + @Experimental + fun registerUpdateHandlers(input: RegisterUpdateHandlersInput) + fun registerDynamicSignalHandler(handler: RegisterDynamicSignalHandlerInput) + fun registerDynamicQueryHandler(input: RegisterDynamicQueryHandlerInput) + + @Experimental + fun registerDynamicUpdateHandler(input: RegisterDynamicUpdateHandlerInput) + fun randomUUID(): UUID + fun upsertSearchAttributes(searchAttributes: Map) + fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>) + + fun currentTimeMillis(): Long +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt new file mode 100644 index 000000000..03b34f71e --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.interceptors + +import io.temporal.common.SearchAttributeUpdate +import io.temporal.workflow.Functions.Func +import io.temporal.workflow.Promise +import java.lang.reflect.Type +import java.time.Duration +import java.util.* +import java.util.function.BiPredicate +import java.util.function.Supplier + +/** Convenience base class for WorkflowOutboundCallsInterceptor implementations. */ +class WorkflowOutboundCallsInterceptorBase(private val next: WorkflowOutboundCallsInterceptor) : + WorkflowOutboundCallsInterceptor { + override suspend fun executeActivity(input: WorkflowOutboundCallsInterceptor.ActivityInput?): WorkflowOutboundCallsInterceptor.ActivityOutput? { + return next.executeActivity(input) + } + + override suspend fun executeLocalActivity(input: WorkflowOutboundCallsInterceptor.LocalActivityInput?): WorkflowOutboundCallsInterceptor.LocalActivityOutput? { + return next.executeLocalActivity(input) + } + + override suspend fun executeChildWorkflow(input: WorkflowOutboundCallsInterceptor.ChildWorkflowInput?): WorkflowOutboundCallsInterceptor.ChildWorkflowOutput? { + return next.executeChildWorkflow(input) + } + + override fun newRandom(): Random? { + return next.newRandom() + } + + override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput?): WorkflowOutboundCallsInterceptor.SignalExternalOutput? { + return next.signalExternalWorkflow(input) + } + + override fun cancelWorkflow(input: WorkflowOutboundCallsInterceptor.CancelWorkflowInput?): WorkflowOutboundCallsInterceptor.CancelWorkflowOutput? { + return next.cancelWorkflow(input) + } + + override suspend fun sleep(duration: Duration?) { + next.sleep(duration) + } + + override suspend fun await(timeout: Duration?, reason: String?, unblockCondition: Supplier?): Boolean { + return next.await(timeout, reason, unblockCondition) + } + + override suspend fun await(reason: String?, unblockCondition: Supplier?) { + next.await(reason, unblockCondition) + } + + override fun sideEffect(resultClass: Class?, resultType: Type?, func: Func?): R? { + return next.sideEffect(resultClass, resultType, func) + } + + override fun mutableSideEffect( + id: String?, resultClass: Class?, resultType: Type?, updated: BiPredicate?, func: Func? + ): R? { + return next.mutableSideEffect(id, resultClass, resultType, updated, func) + } + + override fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int { + return next.getVersion(changeId, minSupported, maxSupported) + } + + override fun continueAsNew(input: WorkflowOutboundCallsInterceptor.ContinueAsNewInput?) { + next.continueAsNew(input) + } + + override fun registerQuery(input: WorkflowOutboundCallsInterceptor.RegisterQueryInput?) { + next.registerQuery(input) + } + + override fun registerSignalHandlers(input: WorkflowOutboundCallsInterceptor.RegisterSignalHandlersInput?) { + next.registerSignalHandlers(input) + } + + override fun registerUpdateHandlers(input: WorkflowOutboundCallsInterceptor.RegisterUpdateHandlersInput?) { + next.registerUpdateHandlers(input) + } + + override fun registerDynamicSignalHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput?) { + next.registerDynamicSignalHandler(input) + } + + override fun registerDynamicQueryHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicQueryHandlerInput?) { + next.registerDynamicQueryHandler(input) + } + + override fun registerDynamicUpdateHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicUpdateHandlerInput?) { + next.registerDynamicUpdateHandler(input) + } + + override fun randomUUID(): UUID? { + return next.randomUUID() + } + + override fun upsertSearchAttributes(searchAttributes: Map?) { + next.upsertSearchAttributes(searchAttributes) + } + + override fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>?) { + next.upsertTypedSearchAttributes(*searchAttributeUpdates) + } + + override fun currentTimeMillis(): Long { + return next.currentTimeMillis() + } +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt new file mode 100644 index 000000000..18ddc10aa --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.kotlin.workflow + +import io.temporal.common.converter.EncodedValues + +/** + * Use DynamicWorkflow to implement any number of workflow types dynamically. When a workflow + * implementation type that extends DynamicWorkflow is registered, it is used to implement any + * workflow type that is not implicitly registered with the [io.temporal.worker.Worker]. Only + * one type that implements DynamicWorkflow per worker is allowed. + * + * + * The main use case for DynamicWorkflow is an implementation of custom Domain Specific Languages + * (DSLs). A single implementation can implement a workflow type which definition is dynamically + * loaded from some external source. + * + * + * Use [Workflow.getInfo] to query information about the workflow type that should be + * implemented dynamically. + * + * + * Use [Workflow.registerListener] to register signal and query listeners. Consider + * using [DynamicSignalHandler] and [DynamicQueryHandler] to implement handlers that can + * support any signal or query type dynamically. + * + * + * All the determinism rules still apply to workflows that implement this interface. + * + * @see io.temporal.activity.DynamicActivity + */ +interface KotlinDynamicWorkflow { + suspend fun execute(args: EncodedValues): Any? +} \ No newline at end of file diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt new file mode 100644 index 000000000..73e35eab6 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.workflow + +import io.temporal.api.common.v1.SearchAttributes +import java.time.Duration +import java.util.* + +/** + * Provides information about the current Workflow Execution and Run. Also provides access to + * immutable information about connected entities like Parent Workflow Execution or a previous Run. + */ +interface KotlinWorkflowInfo { + /** + * @return Workflow Namespace + */ + val namespace: String + + /** + * @return Workflow ID + */ + val workflowId: String + + /** + * @return Workflow Type + */ + val workflowType: String + + /** + * Note: RunId is unique identifier of one workflow code execution. Reset changes RunId. + * + * @return Workflow Run ID that is handled by the current workflow code execution. + * @see .getOriginalExecutionRunId + * @see .getFirstExecutionRunId + */ + val runId: String + + /** + * @return The very first original RunId of the current Workflow Execution preserved along the + * chain of ContinueAsNew, Retry, Cron and Reset. Identifies the whole Runs chain of Workflow + * Execution. + */ + val firstExecutionRunId: String + + /** + * @return Run ID of the previous Workflow Run which continued-as-new or retried or cron-scheduled + * into the current Workflow Run. + */ + val continuedExecutionRunId: String? + + /** + * Note: This value is NOT preserved by continue-as-new, retries or cron Runs. They are separate + * Runs of one Workflow Execution Chain. + * + * @return original RunId of the current Workflow Run. This value is preserved during Reset which + * changes RunID. + * @see .getFirstExecutionRunId + */ + val originalExecutionRunId: String + + /** + * @return Workflow Task Queue name + */ + val taskQueue: String + + /** + * @return Timeout for a Workflow Run specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowRunTimeout] + */ + val workflowRunTimeout: Duration? + + /** + * @return Timeout for the Workflow Execution specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowExecutionTimeout] + */ + val workflowExecutionTimeout: Duration? + + /** + * The time workflow run has started. Note that this time can be different from the time workflow + * function started actual execution. + */ + val runStartedTimestampMillis: Long + + /** + * This method is used to get raw proto serialized Search Attributes. + * + * + * Consider using more user-friendly methods on [Workflow] class, including [ ][Workflow.getSearchAttributes], [Workflow.getSearchAttribute] or [ ][Workflow.getSearchAttributeValues] instead of this method to access deserialized search + * attributes. + * + * @return raw Search Attributes Protobuf entity, null if empty + */ + @get:Deprecated("use {@link Workflow#getTypedSearchAttributes()} instead.") + val searchAttributes: SearchAttributes? + + /** + * @return Workflow ID of the parent Workflow + */ + val parentWorkflowId: String? + + /** + * @return Run ID of the parent Workflow + */ + val parentRunId: String? + + /** + * @return Workflow retry attempt handled by this Workflow code execution. Starts on "1". + */ + val attempt: Int + + /** + * @return Workflow cron schedule + */ + val cronSchedule: String? + + /** + * @return length of Workflow history up until the current moment of execution. This value changes + * during the lifetime of a Workflow Execution. You may use this information to decide when to + * call [Workflow.continueAsNew]. + */ + val historyLength: Long +} \ No newline at end of file From 5a9db4bee8a28d301ec9502e7125bc579c944c77 Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Mon, 19 Jun 2023 13:38:57 -0700 Subject: [PATCH 2/6] Refactored worker factory to support KotlinWorkerFactory --- .../async/DynamicKotlinWorkflowDefinition.kt | 5 +- .../KotlinWorkflowImplementationFactory.kt | 52 ++- ...erceptor.kt => KotlinWorkerInterceptor.kt} | 2 +- .../interceptors/WorkerInterceptorBase.kt | 2 +- .../WorkflowInboundCallsInterceptor.kt | 6 +- .../WorkflowOutboundCallsInterceptor.kt | 2 +- .../io/temporal/worker/KotlinWorkerFactory.kt | 29 ++ .../worker/KotlinWorkerFactoryOptions.kt | 160 +++++++++ .../client/WorkflowClientInternal.java | 5 +- .../replay/ReplayWorkflowFactory.java | 8 + .../POJOWorkflowImplementationFactory.java | 22 +- .../internal/worker/ActivityTaskHandler.java | 2 + .../internal/worker/SyncActivityWorker.java | 2 +- .../internal/worker/SyncWorkflowWorker.java | 15 +- .../io/temporal/worker/BaseWorkerFactory.java | 322 ++++++++++++++++++ .../main/java/io/temporal/worker/Worker.java | 13 +- .../io/temporal/worker/WorkerFactory.java | 311 +---------------- 17 files changed, 598 insertions(+), 360 deletions(-) rename temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/{WorkerInterceptor.kt => KotlinWorkerInterceptor.kt} (99%) create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt create mode 100644 temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt index a22c401a9..dd65ec9c5 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt @@ -25,16 +25,15 @@ import io.temporal.common.converter.EncodedValues import io.temporal.common.converter.Values import io.temporal.common.interceptors.Header import io.temporal.internal.sync.WorkflowInternal -import io.temporal.kotlin.interceptors.WorkerInterceptor +import io.temporal.kotlin.interceptors.KotlinWorkerInterceptor import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor import io.temporal.kotlin.workflow.KotlinDynamicWorkflow -import io.temporal.workflow.DynamicWorkflow import io.temporal.workflow.Functions.Func internal class DynamicKotlinWorkflowDefinition( private val factory: Func, - private val workerInterceptors: Array, + private val workerInterceptors: Array, private val dataConverter: DataConverter ) : KotlinWorkflowDefinition { private var workflowInvoker: WorkflowInboundCallsInterceptor? = null diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt index 8541cbfca..c600451e3 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet import io.temporal.api.common.v1.Payloads import io.temporal.api.common.v1.WorkflowExecution import io.temporal.api.common.v1.WorkflowType +import io.temporal.client.WorkflowClientOptions import io.temporal.common.context.ContextPropagator import io.temporal.common.converter.DataConverter import io.temporal.common.interceptors.Header @@ -34,18 +35,16 @@ import io.temporal.failure.CanceledFailure import io.temporal.internal.common.env.ReflectionUtils import io.temporal.internal.replay.ReplayWorkflow import io.temporal.internal.replay.ReplayWorkflowFactory -import io.temporal.internal.sync.* -import io.temporal.internal.worker.SingleWorkerOptions +import io.temporal.internal.sync.WorkflowInternal import io.temporal.internal.worker.WorkflowExecutionException import io.temporal.internal.worker.WorkflowExecutorCache -import io.temporal.kotlin.interceptors.WorkerInterceptor +import io.temporal.kotlin.interceptors.KotlinWorkerInterceptor import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor import io.temporal.kotlin.workflow.KotlinDynamicWorkflow import io.temporal.payload.context.WorkflowSerializationContext import io.temporal.serviceclient.CheckedExceptionWrapper -import io.temporal.worker.TypeAlreadyRegisteredException -import io.temporal.worker.WorkflowImplementationOptions +import io.temporal.worker.* import io.temporal.workflow.DynamicWorkflow import io.temporal.workflow.Functions.Func import io.temporal.workflow.Functions.Func1 @@ -55,15 +54,26 @@ import java.lang.reflect.Method import java.util.* class KotlinWorkflowImplementationFactory( - singleWorkerOptions: SingleWorkerOptions, - workerInterceptors: Array, - cache: WorkflowExecutorCache, - namespace: String + clientOptions: WorkflowClientOptions, + workerOptions: WorkerOptions, + cache: WorkflowExecutorCache ) : ReplayWorkflowFactory { - private val workerInterceptors: Array - private val dataConverter: DataConverter - private val contextPropagators: List - private val defaultDeadlockDetectionTimeout: Long + +// dataConverter = clientOptions.dataConverter +// workflowThreadExecutor = Objects.requireNonNull(workflowThreadExecutor) +// workerInterceptors = Objects.requireNonNull(factoryOptions.workerInterceptors) +// this.cache = cache!! +// contextPropagators = clientOptions.contextPropagators +// defaultDeadlockDetectionTimeout = workerOptions.defaultDeadlockDetectionTimeout +// namespace = clientOptions.namespace + + //TODO: Kotlin specific interceptors. + private var workerInterceptors: Array = emptyArray() //factoryOptions.workerInterceptors + private var dataConverter: DataConverter = clientOptions.dataConverter + private var contextPropagators: List = clientOptions.contextPropagators + private var defaultDeadlockDetectionTimeout: Long = workerOptions.defaultDeadlockDetectionTimeout + private val cache: WorkflowExecutorCache = cache + private var namespace: String = clientOptions.namespace /** Key: workflow type name, Value: function that creates SyncWorkflowDefinition instance. */ private val workflowDefinitions = @@ -75,20 +85,8 @@ class KotlinWorkflowImplementationFactory( /** If present then it is called for any unknown workflow type. */ private var dynamicWorkflowImplementationFactory: Func? = null private val implementationOptions = Collections.synchronizedMap(HashMap()) - private val cache: WorkflowExecutorCache - private val namespace: String - - init { - Objects.requireNonNull(singleWorkerOptions) - dataConverter = singleWorkerOptions.dataConverter - this.workerInterceptors = Objects.requireNonNull(workerInterceptors) - this.cache = cache - contextPropagators = singleWorkerOptions.contextPropagators - defaultDeadlockDetectionTimeout = singleWorkerOptions.defaultDeadlockDetectionTimeout - this.namespace = namespace - } - fun registerWorkflowImplementationTypes( + override fun registerWorkflowImplementationTypes( options: WorkflowImplementationOptions, workflowImplementationTypes: Array> ) { for (type in workflowImplementationTypes) { @@ -100,7 +98,7 @@ class KotlinWorkflowImplementationFactory( * @param clazz has to be a workflow interface class. The only exception is if it's a * DynamicWorkflow class. */ - fun addWorkflowImplementationFactory( + override fun addWorkflowImplementationFactory( options: WorkflowImplementationOptions, clazz: Class, factory: Func ) { if (DynamicWorkflow::class.java.isAssignableFrom(clazz)) { diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt similarity index 99% rename from temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt rename to temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt index 5545ccca9..21a5bc70f 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt @@ -84,7 +84,7 @@ import io.temporal.common.interceptors.ActivityInboundCallsInterceptor ` * */ @Experimental -interface WorkerInterceptor { +interface KotlinWorkerInterceptor { /** * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `next` [WorkflowInboundCallsInterceptor], but it may change the input parameters. * diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt index f96908eac..4a9edb9b1 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt @@ -21,7 +21,7 @@ package io.temporal.kotlin.interceptors import io.temporal.common.interceptors.ActivityInboundCallsInterceptor -class WorkerInterceptorBase : WorkerInterceptor { +class WorkerInterceptorBase : KotlinWorkerInterceptor { override fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor { return next } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt index f4a8ee400..0770075ee 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt @@ -26,7 +26,7 @@ import io.temporal.common.interceptors.Header * Intercepts inbound calls to the workflow execution on the worker side. * * - * An instance should be created in [ ][WorkerInterceptor.interceptWorkflow]. + * An instance should be created in [ ][KotlinWorkerInterceptor.interceptWorkflow]. * * * The calls to this interceptor are executed under workflow context, all the rules and @@ -41,7 +41,7 @@ import io.temporal.common.interceptors.Header * The implementation must forward all the calls to `next`, but it may change the input * parameters. * - * @see WorkerInterceptor.interceptWorkflow + * @see KotlinWorkerInterceptor.interceptWorkflow */ @Experimental interface WorkflowInboundCallsInterceptor { @@ -65,7 +65,7 @@ interface WorkflowInboundCallsInterceptor { * * @param outboundCalls an existing interceptor instance to be proxied by the interceptor created * inside this method - * @see WorkerInterceptor.interceptWorkflow for the definition of "next" {@link + * @see KotlinWorkerInterceptor.interceptWorkflow for the definition of "next" {@link * * WorkflowInboundCallsInterceptor} */ suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt index 058a1be1b..59a64400d 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -50,7 +50,7 @@ import java.util.function.Supplier * into `init` method of the `next` [WorkflowInboundCallsInterceptor] The * implementation must forward all the calls to the outbound interceptor passed as a `outboundCalls` parameter to the `init` call. * - * @see WorkerInterceptor.interceptWorkflow for the definition of "next" {@link + * @see KotlinWorkerInterceptor.interceptWorkflow for the definition of "next" {@link * * WorkflowInboundCallsInterceptor}. */ @Experimental diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt new file mode 100644 index 000000000..981f3101b --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt @@ -0,0 +1,29 @@ +package io.temporal.worker + +import io.temporal.client.WorkflowClient +import io.temporal.client.WorkflowClientOptions +import io.temporal.internal.async.KotlinWorkflowImplementationFactory +import io.temporal.internal.replay.ReplayWorkflowFactory +import io.temporal.internal.worker.WorkflowExecutorCache + +class KotlinWorkerFactory(workflowClient: WorkflowClient, factoryOptions: KotlinWorkerFactoryOptions?) : + BaseWorkerFactory(workflowClient, toFactoryOptions(factoryOptions)) { + + override fun newReplayWorkflowFactory( + workerOptions: WorkerOptions, + clientOptions: WorkflowClientOptions, + cache: WorkflowExecutorCache, + ): ReplayWorkflowFactory { + return KotlinWorkflowImplementationFactory(clientOptions, workerOptions, cache); + } +} + +// TODO(maxim): This is temporary hack until WorkerFactoryOptions are removed from base. +fun toFactoryOptions(factoryOptions: KotlinWorkerFactoryOptions?): WorkerFactoryOptions? { + val o = KotlinWorkerFactoryOptions.newBuilder(factoryOptions).validateAndBuildWithDefaults() + + return WorkerFactoryOptions.newBuilder() + .setEnableLoggingInReplay(o.isEnableLoggingInReplay) + .setWorkflowCacheSize(o.workflowCacheSize) + .build(); +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt new file mode 100644 index 000000000..e3c15e1f6 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt @@ -0,0 +1,160 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.temporal.worker + +import com.google.common.base.Preconditions +import io.temporal.kotlin.interceptors.KotlinWorkerInterceptor +import java.time.Duration + +class KotlinWorkerFactoryOptions private constructor( + workflowCacheSize: Int, + workflowHostLocalTaskQueueScheduleToStartTimeout: Duration?, + workerInterceptors: Array, + enableLoggingInReplay: Boolean, + validate: Boolean +) { + class Builder { + private var workflowHostLocalTaskQueueScheduleToStartTimeout: Duration? = null + private var workflowCacheSize = 0 + private var workerInterceptors: Array = emptyArray() + private var enableLoggingInReplay = false + + internal constructor() {} + internal constructor(options: KotlinWorkerFactoryOptions?) { + if (options == null) { + return + } + workflowHostLocalTaskQueueScheduleToStartTimeout = options.workflowHostLocalTaskQueueScheduleToStartTimeout + workflowCacheSize = options.workflowCacheSize + workerInterceptors = options.workerInterceptors + enableLoggingInReplay = options.isEnableLoggingInReplay + } + + /** + * To avoid constant replay of code the workflow objects are cached on a worker. This cache is + * shared by all workers created by the Factory. Note that in the majority of situations the + * number of cached workflows is limited not by this value, but by the number of the threads + * defined through [.setMaxWorkflowThreadCount]. + * + * + * Default value is 600 + */ + fun setWorkflowCacheSize(workflowCacheSize: Int): Builder { + this.workflowCacheSize = workflowCacheSize + return this + } + + /** + * Timeout for a workflow task routed to the "sticky worker" - host that has the workflow + * instance cached in memory. Once it times out, then it can be picked up by any worker. + * + * + * Default value is 5 seconds. + * + */ + @Deprecated( + """use {@link WorkerOptions.Builder#setStickyQueueScheduleToStartTimeout(Duration)} + to specify this value per-worker instead""" + ) + fun setWorkflowHostLocalTaskQueueScheduleToStartTimeout(timeout: Duration?): Builder { + workflowHostLocalTaskQueueScheduleToStartTimeout = timeout + return this + } + + fun setWorkerInterceptors(vararg workerInterceptors: KotlinWorkerInterceptor): Builder { + this.workerInterceptors = workerInterceptors + return this + } + + fun setEnableLoggingInReplay(enableLoggingInReplay: Boolean): Builder { + this.enableLoggingInReplay = enableLoggingInReplay + return this + } + + @Deprecated("not used anymore by JavaSDK, this value doesn't have any effect") + fun setWorkflowHostLocalPollThreadCount(workflowHostLocalPollThreadCount: Int): Builder { + return this + } + + fun build(): KotlinWorkerFactoryOptions { + return KotlinWorkerFactoryOptions( + workflowCacheSize, + workflowHostLocalTaskQueueScheduleToStartTimeout, + workerInterceptors, + enableLoggingInReplay, + false + ) + } + + fun validateAndBuildWithDefaults(): KotlinWorkerFactoryOptions { + return KotlinWorkerFactoryOptions( + workflowCacheSize, + workflowHostLocalTaskQueueScheduleToStartTimeout, + workerInterceptors, + enableLoggingInReplay, + true + ) + } + } + + val workflowCacheSize: Int + val workflowHostLocalTaskQueueScheduleToStartTimeout: Duration? + val workerInterceptors: Array + val isEnableLoggingInReplay: Boolean + + init { + var workflowCacheSize = workflowCacheSize + var workerInterceptors = workerInterceptors + if (validate) { + Preconditions.checkState(workflowCacheSize >= 0, "negative workflowCacheSize") + if (workflowCacheSize <= 0) { + workflowCacheSize = DEFAULT_WORKFLOW_CACHE_SIZE + } + if (workflowHostLocalTaskQueueScheduleToStartTimeout != null) { + Preconditions.checkState( + !workflowHostLocalTaskQueueScheduleToStartTimeout.isNegative, + "negative workflowHostLocalTaskQueueScheduleToStartTimeoutSeconds" + ) + } + } + this.workflowCacheSize = workflowCacheSize + this.workflowHostLocalTaskQueueScheduleToStartTimeout = workflowHostLocalTaskQueueScheduleToStartTimeout + this.workerInterceptors = workerInterceptors + isEnableLoggingInReplay = enableLoggingInReplay + } + + fun toBuilder(): Builder { + return Builder(this) + } + + companion object { + fun newBuilder(): Builder { + return Builder() + } + + fun newBuilder(options: KotlinWorkerFactoryOptions?): Builder { + return Builder(options) + } + + private const val DEFAULT_WORKFLOW_CACHE_SIZE = 600 + private const val DEFAULT_MAX_WORKFLOW_THREAD_COUNT = 600 + val defaultInstance: KotlinWorkerFactoryOptions = newBuilder().build() + } +} \ No newline at end of file diff --git a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java index 83f27b8bb..b8349826e 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java @@ -21,6 +21,7 @@ package io.temporal.internal.client; import io.temporal.client.WorkflowClient; +import io.temporal.worker.BaseWorkerFactory; import io.temporal.worker.WorkerFactory; /** @@ -32,7 +33,7 @@ * {@link WorkflowClient#getInternal()} is used only for internal functionality. */ public interface WorkflowClientInternal { - void registerWorkerFactory(WorkerFactory workerFactory); + void registerWorkerFactory(BaseWorkerFactory workerFactory); - void deregisterWorkerFactory(WorkerFactory workerFactory); + void deregisterWorkerFactory(BaseWorkerFactory workerFactory); } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/replay/ReplayWorkflowFactory.java b/temporal-sdk/src/main/java/io/temporal/internal/replay/ReplayWorkflowFactory.java index 76703481f..2098cec6e 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/replay/ReplayWorkflowFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/replay/ReplayWorkflowFactory.java @@ -22,10 +22,18 @@ import io.temporal.api.common.v1.WorkflowExecution; import io.temporal.api.common.v1.WorkflowType; +import io.temporal.worker.WorkflowImplementationOptions; +import io.temporal.workflow.Functions; public interface ReplayWorkflowFactory { ReplayWorkflow getWorkflow(WorkflowType workflowType, WorkflowExecution workflowExecution) throws Exception; boolean isAnyTypeSupported(); + + void registerWorkflowImplementationTypes( + WorkflowImplementationOptions options, Class[] workflowImplementationTypes); + + void addWorkflowImplementationFactory( + WorkflowImplementationOptions options, Class clazz, Functions.Func factory); } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java b/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java index bdf209cc1..5f4371926 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java @@ -27,6 +27,7 @@ import io.temporal.api.common.v1.Payloads; import io.temporal.api.common.v1.WorkflowExecution; import io.temporal.api.common.v1.WorkflowType; +import io.temporal.client.WorkflowClientOptions; import io.temporal.common.context.ContextPropagator; import io.temporal.common.converter.DataConverter; import io.temporal.common.interceptors.Header; @@ -45,6 +46,8 @@ import io.temporal.internal.worker.WorkflowExecutorCache; import io.temporal.payload.context.WorkflowSerializationContext; import io.temporal.worker.TypeAlreadyRegisteredException; +import io.temporal.worker.WorkerFactoryOptions; +import io.temporal.worker.WorkerOptions; import io.temporal.worker.WorkflowImplementationOptions; import io.temporal.workflow.DynamicWorkflow; import io.temporal.workflow.Functions; @@ -100,19 +103,18 @@ public final class POJOWorkflowImplementationFactory implements ReplayWorkflowFa private final String namespace; public POJOWorkflowImplementationFactory( - SingleWorkerOptions singleWorkerOptions, + WorkerFactoryOptions factoryOptions, + WorkflowClientOptions clientOptions, + WorkerOptions workerOptions, WorkflowThreadExecutor workflowThreadExecutor, - WorkerInterceptor[] workerInterceptors, - WorkflowExecutorCache cache, - @Nonnull String namespace) { - Objects.requireNonNull(singleWorkerOptions); - this.dataConverter = singleWorkerOptions.getDataConverter(); + WorkflowExecutorCache cache) { + this.dataConverter = clientOptions.getDataConverter(); this.workflowThreadExecutor = Objects.requireNonNull(workflowThreadExecutor); - this.workerInterceptors = Objects.requireNonNull(workerInterceptors); + this.workerInterceptors = Objects.requireNonNull(factoryOptions.getWorkerInterceptors()); this.cache = cache; - this.contextPropagators = singleWorkerOptions.getContextPropagators(); - this.defaultDeadlockDetectionTimeout = singleWorkerOptions.getDefaultDeadlockDetectionTimeout(); - this.namespace = namespace; + this.contextPropagators = clientOptions.getContextPropagators(); + this.defaultDeadlockDetectionTimeout = workerOptions.getDefaultDeadlockDetectionTimeout(); + this.namespace = clientOptions.getNamespace(); } public void registerWorkflowImplementationTypes( diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityTaskHandler.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityTaskHandler.java index 24128639f..fef14879d 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityTaskHandler.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/ActivityTaskHandler.java @@ -113,6 +113,8 @@ public boolean isManualCompletion() { } } + void registerActivityImplementations(Object[] activitiesImplementation); + /** * The implementation should be called when a polling activity worker receives a new activity * task. This method shouldn't throw any Throwables unless there is a need to not reply to the diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java index 0634e0a5a..53073fdb2 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncActivityWorker.java @@ -41,7 +41,7 @@ public class SyncActivityWorker implements SuspendableWorker { private final String taskQueue; private final ScheduledExecutorService heartbeatExecutor; - private final ActivityTaskHandlerImpl taskHandler; + private final ActivityTaskHandler taskHandler; private final ActivityWorker worker; public SyncActivityWorker( diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java index a1ece312f..79bcdee58 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java @@ -28,6 +28,7 @@ import io.temporal.internal.activity.ActivityExecutionContextFactory; import io.temporal.internal.activity.ActivityTaskHandlerImpl; import io.temporal.internal.activity.LocalActivityExecutionContextFactoryImpl; +import io.temporal.internal.replay.ReplayWorkflowFactory; import io.temporal.internal.replay.ReplayWorkflowTaskHandler; import io.temporal.internal.sync.POJOWorkflowImplementationFactory; import io.temporal.internal.sync.WorkflowThreadExecutor; @@ -66,7 +67,7 @@ public class SyncWorkflowWorker implements SuspendableWorker { private final WorkflowWorker workflowWorker; private final QueryReplayHelper queryReplayHelper; private final LocalActivityWorker laWorker; - private final POJOWorkflowImplementationFactory factory; + private final ReplayWorkflowFactory factory; private final DataConverter dataConverter; private final ActivityTaskHandlerImpl laTaskHandler; @@ -79,20 +80,14 @@ public SyncWorkflowWorker( @Nonnull WorkflowRunLockManager runLocks, @Nonnull WorkflowExecutorCache cache, String stickyTaskQueueName, - @Nonnull WorkflowThreadExecutor workflowThreadExecutor, - @Nonnull EagerActivityDispatcher eagerActivityDispatcher) { + @Nonnull EagerActivityDispatcher eagerActivityDispatcher, + ReplayWorkflowFactory factory) { this.identity = singleWorkerOptions.getIdentity(); this.namespace = namespace; this.taskQueue = taskQueue; this.dataConverter = singleWorkerOptions.getDataConverter(); - factory = - new POJOWorkflowImplementationFactory( - singleWorkerOptions, - Objects.requireNonNull(workflowThreadExecutor), - singleWorkerOptions.getWorkerInterceptors(), - cache, - namespace); + this.factory = factory; ActivityExecutionContextFactory laActivityExecutionContextFactory = new LocalActivityExecutionContextFactoryImpl(); diff --git a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java new file mode 100644 index 000000000..7d2d67d50 --- /dev/null +++ b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java @@ -0,0 +1,322 @@ +package io.temporal.worker; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.uber.m3.tally.Scope; +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowClientOptions; +import io.temporal.common.converter.DataConverter; +import io.temporal.internal.client.WorkflowClientInternal; +import io.temporal.internal.replay.ReplayWorkflowFactory; +import io.temporal.internal.worker.ShutdownManager; +import io.temporal.internal.worker.WorkflowExecutorCache; +import io.temporal.internal.worker.WorkflowRunLockManager; +import io.temporal.serviceclient.MetricsTag; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public abstract class BaseWorkerFactory { + private static final Logger log = LoggerFactory.getLogger(WorkerFactory.class); + private final WorkflowRunLockManager runLocks = new WorkflowRunLockManager(); + protected final Scope metricsScope; + private final WorkflowClient workflowClient; + // TODO(maxim): Move factory options down into WorkerFactory. + // This requires moving ActivityTaskHandler creation there as well. + private final WorkerFactoryOptions factoryOptions; + private final @Nonnull WorkflowExecutorCache cache; + private final Map workers = new HashMap<>(); + private final String statusErrorMessage = + "attempted to %s while in %s state. Acceptable States: %s"; + private State state = State.Initial; + + protected BaseWorkerFactory(WorkflowClient workflowClient, WorkerFactoryOptions factoryOptions) { + this.workflowClient = Objects.requireNonNull(workflowClient); + this.factoryOptions = + WorkerFactoryOptions.newBuilder(factoryOptions).validateAndBuildWithDefaults(); + WorkflowClientOptions workflowClientOptions = workflowClient.getOptions(); + + String namespace = workflowClientOptions.getNamespace(); + this.metricsScope = + this.workflowClient + .getWorkflowServiceStubs() + .getOptions() + .getMetricsScope() + .tagged(MetricsTag.defaultTags(namespace)); + this.cache = + new WorkflowExecutorCache( + this.factoryOptions.getWorkflowCacheSize(), runLocks, metricsScope); + } + + /** + * Creates worker that connects to an instance of the Temporal Service. It uses the namespace + * configured at the Factory level. New workers cannot be created after the start() has been + * called + * + * @param taskQueue task queue name worker uses to poll. It uses this name for both workflow and + * activity task queue polls. + * @return Worker + */ + public Worker newWorker(String taskQueue) { + return newWorker(taskQueue, null); + } + + /** + * Creates worker that connects to an instance of the Temporal Service. It uses the namespace + * configured at the Factory level. New workers cannot be created after the start() has been + * called + * + * @param taskQueue task queue name worker uses to poll. It uses this name for both workflow and + * activity task queue polls. + * @param workerOptions Options (like {@link DataConverter} override) for configuring worker. + * @return Worker + */ + public synchronized Worker newWorker(String taskQueue, WorkerOptions workerOptions) { + Preconditions.checkArgument( + !Strings.isNullOrEmpty(taskQueue), "taskQueue should not be an empty string"); + Preconditions.checkState( + state == State.Initial, + String.format(statusErrorMessage, "create new worker", state.name(), State.Initial.name())); + + // Only one worker can exist for a task queue + Worker existingWorker = workers.get(taskQueue); + if (existingWorker == null) { + WorkflowClientOptions clientOptions = workflowClient.getOptions(); + ReplayWorkflowFactory workflowFactory = + newReplayWorkflowFactory(workerOptions, clientOptions, cache); + + Worker worker = + new Worker( + workflowClient, + taskQueue, + factoryOptions, + workerOptions, + metricsScope, + runLocks, + cache, + true, + clientOptions.getContextPropagators(), + workflowFactory); + workers.put(taskQueue, worker); + return worker; + } else { + log.warn( + "Only one worker can be registered for a task queue, " + + "subsequent calls to WorkerFactory#newWorker with the same task queue are ignored and " + + "initially created worker is returned"); + return existingWorker; + } + } + + protected abstract ReplayWorkflowFactory newReplayWorkflowFactory( + WorkerOptions workerOptions, + WorkflowClientOptions clientOptions, + WorkflowExecutorCache cache1); + + /** + * @param taskQueue task queue name to lookup an existing worker for + * @return a worker created previously through {@link #newWorker(String)} for the given task + * queue. + * @throws IllegalStateException if the worker has not been registered for the given task queue. + */ + public synchronized Worker getWorker(String taskQueue) { + Worker result = workers.get(taskQueue); + if (result == null) { + throw new IllegalArgumentException("No worker for taskQueue: " + taskQueue); + } + return result; + } + + /** + * @param taskQueue task queue name to lookup an existing worker for + * @return a worker created previously through {@link #newWorker(String)} for the given task queue + * or null. + */ + @Nullable + public synchronized Worker tryGetWorker(@Nonnull String taskQueue) { + return workers.get(taskQueue); + } + + /** Starts all the workers created by this factory. */ + public synchronized void start() { + Preconditions.checkState( + state == State.Initial || state == State.Started, + String.format( + statusErrorMessage, + "start WorkerFactory", + state.name(), + String.format("%s, %s", State.Initial.name(), State.Initial.name()))); + if (state == State.Started) { + return; + } + + // Workers check and require that Temporal Server is available during start to fail-fast in case + // of configuration issues. + workflowClient.getWorkflowServiceStubs().connect(null); + + for (Worker worker : workers.values()) { + worker.start(); + } + + state = State.Started; + ((WorkflowClientInternal) workflowClient.getInternal()).registerWorkerFactory(this); + } + + /** Was {@link #start()} called. */ + public synchronized boolean isStarted() { + return state != State.Initial; + } + + /** Was {@link #shutdown()} or {@link #shutdownNow()} called. */ + public synchronized boolean isShutdown() { + return state == State.Shutdown; + } + + /** + * Returns true if all tasks have completed following shut down. Note that isTerminated is never + * true unless either shutdown or shutdownNow was called first. + */ + public synchronized boolean isTerminated() { + if (state != State.Shutdown) { + return false; + } + for (Worker worker : workers.values()) { + if (!worker.isTerminated()) { + return false; + } + } + return true; + } + + /** + * @return instance of the Temporal client that this worker factory uses. + */ + public WorkflowClient getWorkflowClient() { + return workflowClient; + } + + /** + * Initiates an orderly shutdown in which polls are stopped and already received workflow and + * activity tasks are executed.
+ * After the shutdown, calls to {@link + * io.temporal.activity.ActivityExecutionContext#heartbeat(Object)} start throwing {@link + * io.temporal.client.ActivityWorkerShutdownException}.
+ * This method does not wait for the shutdown to complete. Use {@link #awaitTermination(long, + * TimeUnit)} to do that.
+ * Invocation has no additional effect if already shut down. + */ + public synchronized void shutdown() { + log.info("shutdown: {}", this); + shutdownInternal(false); + } + + /** + * Initiates an orderly shutdown in which polls are stopped and already received workflow and + * activity tasks are attempted to be stopped.
+ * This implementation cancels tasks via Thread.interrupt(), so any task that fails to respond to + * interrupts may never terminate.
+ * After the shutdownNow calls to {@link + * io.temporal.activity.ActivityExecutionContext#heartbeat(Object)} start throwing {@link + * io.temporal.client.ActivityWorkerShutdownException}.
+ * This method does not wait for the shutdown to complete. Use {@link #awaitTermination(long, + * TimeUnit)} to do that.
+ * Invocation has no additional effect if already shut down. + */ + public synchronized void shutdownNow() { + log.info("shutdownNow: {}", this); + shutdownInternal(true); + } + + private void shutdownInternal(boolean interruptUserTasks) { + state = State.Shutdown; + ((WorkflowClientInternal) workflowClient.getInternal()).deregisterWorkerFactory(this); + ShutdownManager shutdownManager = new ShutdownManager(); + CompletableFuture.allOf( + workers.values().stream() + .map(worker -> worker.shutdown(shutdownManager, interruptUserTasks)) + .toArray(CompletableFuture[]::new)) + .thenApply( + r -> { + cache.invalidateAll(); + handleShutdown(); + return null; + }) + .whenComplete( + (r, e) -> { + if (e != null) { + log.error("[BUG] Unexpected exception during shutdown", e); + } + shutdownManager.close(); + }); + } + + /** Override to clean resources upon shutdown request. */ + protected void handleShutdown() {} + + /** + * Blocks until all tasks have completed execution after a shutdown request, or the timeout + * occurs. + */ + public void awaitTermination(long timeout, TimeUnit unit) { + log.info("awaitTermination begin: {}", this); + long timeoutMillis = unit.toMillis(timeout); + for (Worker worker : workers.values()) { + long t = timeoutMillis; // closure needs immutable value + timeoutMillis = + ShutdownManager.runAndGetRemainingTimeoutMs( + t, () -> worker.awaitTermination(t, TimeUnit.MILLISECONDS)); + } + log.info("awaitTermination done: {}", this); + } + + // TODO we should hide an actual implementation of WorkerFactory under WorkerFactory interface and + // expose this method on the implementation only + @VisibleForTesting + WorkflowExecutorCache getCache() { + return this.cache; + } + + public synchronized void suspendPolling() { + if (state != State.Started) { + return; + } + + log.info("suspendPolling: {}", this); + state = State.Suspended; + for (Worker worker : workers.values()) { + worker.suspendPolling(); + } + } + + public synchronized void resumePolling() { + if (state != State.Suspended) { + return; + } + + log.info("resumePolling: {}", this); + state = State.Started; + for (Worker worker : workers.values()) { + worker.resumePolling(); + } + } + + @Override + public String toString() { + return String.format("WorkerFactory{identity=%s}", workflowClient.getOptions().getIdentity()); + } + + enum State { + Initial, + Started, + Suspended, + Shutdown + } +} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java index 4255368a5..dbda35ee9 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java @@ -31,6 +31,8 @@ import io.temporal.common.context.ContextPropagator; import io.temporal.common.converter.DataConverter; import io.temporal.failure.TemporalFailure; +import io.temporal.internal.replay.ReplayWorkflowFactory; +import io.temporal.internal.sync.POJOWorkflowImplementationFactory; import io.temporal.internal.sync.WorkflowInternal; import io.temporal.internal.sync.WorkflowThreadExecutor; import io.temporal.internal.worker.*; @@ -75,7 +77,7 @@ public final class Worker { * activity task queue polls. * @param options Options (like {@link DataConverter} override) for configuring worker. * @param useStickyTaskQueue if sticky task queue should be used - * @param workflowThreadExecutor workflow methods thread executor + * @param workflowFactory factory that creates workflow implementations */ Worker( WorkflowClient client, @@ -86,15 +88,14 @@ public final class Worker { @Nonnull WorkflowRunLockManager runLocks, @Nonnull WorkflowExecutorCache cache, boolean useStickyTaskQueue, - WorkflowThreadExecutor workflowThreadExecutor, - List contextPropagators) { + List contextPropagators, + ReplayWorkflowFactory workflowFactory) { Objects.requireNonNull(client, "client should not be null"); Preconditions.checkArgument( !Strings.isNullOrEmpty(taskQueue), "taskQueue should not be an empty string"); this.taskQueue = taskQueue; this.options = WorkerOptions.newBuilder(options).validateAndBuildWithDefaults(); - factoryOptions = WorkerFactoryOptions.newBuilder(factoryOptions).validateAndBuildWithDefaults(); WorkflowServiceStubs service = client.getWorkflowServiceStubs(); WorkflowClientOptions clientOptions = client.getOptions(); String namespace = clientOptions.getNamespace(); @@ -142,8 +143,8 @@ public final class Worker { runLocks, cache, useStickyTaskQueue ? getStickyTaskQueueName(client.getOptions().getIdentity()) : null, - workflowThreadExecutor, - eagerActivityDispatcher); + eagerActivityDispatcher, + workflowFactory); } /** diff --git a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java index 4de582ac0..1c71cc557 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java @@ -20,52 +20,25 @@ package io.temporal.worker; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; -import com.uber.m3.tally.Scope; import io.temporal.client.WorkflowClient; import io.temporal.client.WorkflowClientOptions; -import io.temporal.common.converter.DataConverter; -import io.temporal.internal.client.WorkflowClientInternal; +import io.temporal.internal.replay.ReplayWorkflowFactory; +import io.temporal.internal.sync.POJOWorkflowImplementationFactory; import io.temporal.internal.sync.WorkflowThreadExecutor; -import io.temporal.internal.worker.*; import io.temporal.internal.worker.WorkflowExecutorCache; -import io.temporal.serviceclient.MetricsTag; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; + import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Maintains worker creation and lifecycle. */ -public final class WorkerFactory { - private static final Logger log = LoggerFactory.getLogger(WorkerFactory.class); - - private final WorkflowRunLockManager runLocks = new WorkflowRunLockManager(); - - private final Scope metricsScope; +public final class WorkerFactory extends BaseWorkerFactory { - private final Map workers = new HashMap<>(); - private final WorkflowClient workflowClient; + private final WorkerFactoryOptions factoryOptions; private final ThreadPoolExecutor workflowThreadPool; private final WorkflowThreadExecutor workflowThreadExecutor; private final AtomicInteger workflowThreadCounter = new AtomicInteger(); - private final WorkerFactoryOptions factoryOptions; - - private final @Nonnull WorkflowExecutorCache cache; - - private State state = State.Initial; - - private final String statusErrorMessage = - "attempted to %s while in %s state. Acceptable States: %s"; public static WorkerFactory newInstance(WorkflowClient workflowClient) { return WorkerFactory.newInstance(workflowClient, WorkerFactoryOptions.getDefaultInstance()); @@ -84,24 +57,14 @@ public static WorkerFactory newInstance( * @param factoryOptions Options used to configure factory settings */ private WorkerFactory(WorkflowClient workflowClient, WorkerFactoryOptions factoryOptions) { - this.workflowClient = Objects.requireNonNull(workflowClient); - WorkflowClientOptions workflowClientOptions = workflowClient.getOptions(); - String namespace = workflowClientOptions.getNamespace(); - + super(workflowClient, factoryOptions); this.factoryOptions = WorkerFactoryOptions.newBuilder(factoryOptions).validateAndBuildWithDefaults(); - this.metricsScope = - this.workflowClient - .getWorkflowServiceStubs() - .getOptions() - .getMetricsScope() - .tagged(MetricsTag.defaultTags(namespace)); - this.workflowThreadPool = new ThreadPoolExecutor( 0, - this.factoryOptions.getMaxWorkflowThreadCount(), + factoryOptions.getMaxWorkflowThreadCount(), 1, TimeUnit.MINUTES, new SynchronousQueue<>()); @@ -109,261 +72,19 @@ private WorkerFactory(WorkflowClient workflowClient, WorkerFactoryOptions factor r -> new Thread(r, "workflow-thread-" + workflowThreadCounter.incrementAndGet())); this.workflowThreadExecutor = new ActiveThreadReportingExecutor(this.workflowThreadPool, this.metricsScope); - - this.cache = - new WorkflowExecutorCache( - this.factoryOptions.getWorkflowCacheSize(), runLocks, metricsScope); - } - - /** - * Creates worker that connects to an instance of the Temporal Service. It uses the namespace - * configured at the Factory level. New workers cannot be created after the start() has been - * called - * - * @param taskQueue task queue name worker uses to poll. It uses this name for both workflow and - * activity task queue polls. - * @return Worker - */ - public Worker newWorker(String taskQueue) { - return newWorker(taskQueue, null); - } - - /** - * Creates worker that connects to an instance of the Temporal Service. It uses the namespace - * configured at the Factory level. New workers cannot be created after the start() has been - * called - * - * @param taskQueue task queue name worker uses to poll. It uses this name for both workflow and - * activity task queue polls. - * @param options Options (like {@link DataConverter} override) for configuring worker. - * @return Worker - */ - public synchronized Worker newWorker(String taskQueue, WorkerOptions options) { - Preconditions.checkArgument( - !Strings.isNullOrEmpty(taskQueue), "taskQueue should not be an empty string"); - Preconditions.checkState( - state == State.Initial, - String.format(statusErrorMessage, "create new worker", state.name(), State.Initial.name())); - - // Only one worker can exist for a task queue - Worker existingWorker = workers.get(taskQueue); - if (existingWorker == null) { - Worker worker = - new Worker( - workflowClient, - taskQueue, - factoryOptions, - options, - metricsScope, - runLocks, - cache, - true, - workflowThreadExecutor, - workflowClient.getOptions().getContextPropagators()); - workers.put(taskQueue, worker); - return worker; - } else { - log.warn( - "Only one worker can be registered for a task queue, " - + "subsequent calls to WorkerFactory#newWorker with the same task queue are ignored and " - + "initially created worker is returned"); - return existingWorker; - } - } - - /** - * @param taskQueue task queue name to lookup an existing worker for - * @return a worker created previously through {@link #newWorker(String)} for the given task - * queue. - * @throws IllegalStateException if the worker has not been registered for the given task queue. - */ - public synchronized Worker getWorker(String taskQueue) { - Worker result = workers.get(taskQueue); - if (result == null) { - throw new IllegalArgumentException("No worker for taskQueue: " + taskQueue); - } - return result; - } - - /** - * @param taskQueue task queue name to lookup an existing worker for - * @return a worker created previously through {@link #newWorker(String)} for the given task queue - * or null. - */ - @Nullable - public synchronized Worker tryGetWorker(@Nonnull String taskQueue) { - return workers.get(taskQueue); - } - - /** Starts all the workers created by this factory. */ - public synchronized void start() { - Preconditions.checkState( - state == State.Initial || state == State.Started, - String.format( - statusErrorMessage, - "start WorkerFactory", - state.name(), - String.format("%s, %s", State.Initial.name(), State.Initial.name()))); - if (state == State.Started) { - return; - } - - // Workers check and require that Temporal Server is available during start to fail-fast in case - // of configuration issues. - workflowClient.getWorkflowServiceStubs().connect(null); - - for (Worker worker : workers.values()) { - worker.start(); - } - - state = State.Started; - ((WorkflowClientInternal) workflowClient.getInternal()).registerWorkerFactory(this); - } - - /** Was {@link #start()} called. */ - public synchronized boolean isStarted() { - return state != State.Initial; - } - - /** Was {@link #shutdown()} or {@link #shutdownNow()} called. */ - public synchronized boolean isShutdown() { - return state == State.Shutdown; - } - - /** - * Returns true if all tasks have completed following shut down. Note that isTerminated is never - * true unless either shutdown or shutdownNow was called first. - */ - public synchronized boolean isTerminated() { - if (state != State.Shutdown) { - return false; - } - for (Worker worker : workers.values()) { - if (!worker.isTerminated()) { - return false; - } - } - return true; - } - - /** - * @return instance of the Temporal client that this worker factory uses. - */ - public WorkflowClient getWorkflowClient() { - return workflowClient; - } - - /** - * Initiates an orderly shutdown in which polls are stopped and already received workflow and - * activity tasks are executed.
- * After the shutdown, calls to {@link - * io.temporal.activity.ActivityExecutionContext#heartbeat(Object)} start throwing {@link - * io.temporal.client.ActivityWorkerShutdownException}.
- * This method does not wait for the shutdown to complete. Use {@link #awaitTermination(long, - * TimeUnit)} to do that.
- * Invocation has no additional effect if already shut down. - */ - public synchronized void shutdown() { - log.info("shutdown: {}", this); - shutdownInternal(false); - } - - /** - * Initiates an orderly shutdown in which polls are stopped and already received workflow and - * activity tasks are attempted to be stopped.
- * This implementation cancels tasks via Thread.interrupt(), so any task that fails to respond to - * interrupts may never terminate.
- * After the shutdownNow calls to {@link - * io.temporal.activity.ActivityExecutionContext#heartbeat(Object)} start throwing {@link - * io.temporal.client.ActivityWorkerShutdownException}.
- * This method does not wait for the shutdown to complete. Use {@link #awaitTermination(long, - * TimeUnit)} to do that.
- * Invocation has no additional effect if already shut down. - */ - public synchronized void shutdownNow() { - log.info("shutdownNow: {}", this); - shutdownInternal(true); - } - - private void shutdownInternal(boolean interruptUserTasks) { - state = State.Shutdown; - ((WorkflowClientInternal) workflowClient.getInternal()).deregisterWorkerFactory(this); - ShutdownManager shutdownManager = new ShutdownManager(); - CompletableFuture.allOf( - workers.values().stream() - .map(worker -> worker.shutdown(shutdownManager, interruptUserTasks)) - .toArray(CompletableFuture[]::new)) - .thenApply( - r -> { - cache.invalidateAll(); - workflowThreadPool.shutdownNow(); - return null; - }) - .whenComplete( - (r, e) -> { - if (e != null) { - log.error("[BUG] Unexpected exception during shutdown", e); - } - shutdownManager.close(); - }); - } - - /** - * Blocks until all tasks have completed execution after a shutdown request, or the timeout - * occurs. - */ - public void awaitTermination(long timeout, TimeUnit unit) { - log.info("awaitTermination begin: {}", this); - long timeoutMillis = unit.toMillis(timeout); - for (Worker worker : workers.values()) { - long t = timeoutMillis; // closure needs immutable value - timeoutMillis = - ShutdownManager.runAndGetRemainingTimeoutMs( - t, () -> worker.awaitTermination(t, TimeUnit.MILLISECONDS)); - } - log.info("awaitTermination done: {}", this); - } - - // TODO we should hide an actual implementation of WorkerFactory under WorkerFactory interface and - // expose this method on the implementation only - @VisibleForTesting - WorkflowExecutorCache getCache() { - return this.cache; - } - - public synchronized void suspendPolling() { - if (state != State.Started) { - return; - } - - log.info("suspendPolling: {}", this); - state = State.Suspended; - for (Worker worker : workers.values()) { - worker.suspendPolling(); - } - } - - public synchronized void resumePolling() { - if (state != State.Suspended) { - return; - } - - log.info("resumePolling: {}", this); - state = State.Started; - for (Worker worker : workers.values()) { - worker.resumePolling(); - } } @Override - public String toString() { - return String.format("WorkerFactory{identity=%s}", workflowClient.getOptions().getIdentity()); + protected ReplayWorkflowFactory newReplayWorkflowFactory( + WorkerOptions workerOptions, + WorkflowClientOptions clientOptions, + WorkflowExecutorCache cache) { + return new POJOWorkflowImplementationFactory( + factoryOptions, clientOptions, workerOptions, workflowThreadExecutor, cache); } - enum State { - Initial, - Started, - Suspended, - Shutdown + @Override + protected void handleShutdown() { + workflowThreadPool.shutdownNow(); } } From 2e558cdf3649a6b39437693143f6e55e1092c1a7 Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Mon, 19 Jun 2023 14:38:26 -0700 Subject: [PATCH 3/6] Fixed compilation and linter errors --- build.gradle | 2 +- temporal-kotlin/build.gradle | 2 +- ...otKotlinWorkflowInboundCallsInterceptor.kt | 3 +- .../async/DynamicKotlinWorkflowDefinition.kt | 6 +- .../internal/async/KotlinActivityStub.kt | 51 ++--- .../internal/async/KotlinActivityStubImpl.kt | 18 +- .../temporal/internal/async/KotlinWorkflow.kt | 29 +-- .../internal/async/KotlinWorkflowContext.kt | 55 +++-- .../async/KotlinWorkflowDefinition.kt | 9 +- .../async/KotlinWorkflowExecutionHandler.kt | 10 +- .../KotlinWorkflowImplementationFactory.kt | 108 +++++---- .../internal/async/KotlinWorkflowInfoImpl.kt | 4 +- .../internal/async/KotlinWorkflowInternal.kt | 216 +++++++++--------- .../async/TemporalCoroutineDispatcher.kt | 14 +- .../interceptors/KotlinWorkerInterceptor.kt | 20 +- .../interceptors/WorkerInterceptorBase.kt | 14 +- .../WorkflowInboundCallsInterceptor.kt | 89 ++++---- .../WorkflowInboundCallsInterceptorBase.kt | 2 +- .../WorkflowOutboundCallsInterceptor.kt | 27 ++- .../WorkflowOutboundCallsInterceptorBase.kt | 195 ++++++++-------- .../kotlin/workflow/KotlinDynamicWorkflow.kt | 4 +- .../io/temporal/worker/KotlinWorkerFactory.kt | 6 +- .../worker/KotlinWorkerFactoryOptions.kt | 2 +- .../io/temporal/workflow/KotlinWorkflow.kt | 18 ++ .../temporal/workflow/KotlinWorkflowInfo.kt | 214 ++++++++--------- .../workflow/CoroutineWorkflowTest.kt | 11 + .../client/WorkflowClientInternalImpl.java | 6 +- .../client/EagerWorkflowTaskDispatcher.java | 4 +- .../client/WorkerFactoryRegistry.java | 13 +- .../client/WorkflowClientInternal.java | 1 - .../POJOWorkflowImplementationFactory.java | 2 - .../internal/worker/SyncWorkflowWorker.java | 3 - .../io/temporal/worker/BaseWorkerFactory.java | 9 +- .../main/java/io/temporal/worker/Worker.java | 2 - .../io/temporal/worker/WorkerFactory.java | 1 - 35 files changed, 632 insertions(+), 538 deletions(-) create mode 100644 temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt create mode 100644 temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt diff --git a/build.gradle b/build.gradle index c39cd786c..8c62322c2 100644 --- a/build.gradle +++ b/build.gradle @@ -1,7 +1,7 @@ buildscript { ext { palantirGitVersionVersion = "${JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_11) ? '0.15.0' : '0.13.0'}" - kotlinVersion = "${project.hasProperty("edgeDepsTest") ? '1.8.20' : (JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_16) ? '1.5.32' : '1.4.32')}" + kotlinVersion = '1.8.20' //"${project.hasProperty("edgeDepsTest") ? '1.8.20' : (JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_16) ? '1.5.32' : '1.4.32')}" } } diff --git a/temporal-kotlin/build.gradle b/temporal-kotlin/build.gradle index 99527e076..4c063ab3e 100644 --- a/temporal-kotlin/build.gradle +++ b/temporal-kotlin/build.gradle @@ -21,7 +21,7 @@ dependencies { // this module shouldn't carry temporal-sdk with it, especially for situations when users may be using a shaded artifact compileOnly project(':temporal-sdk') - implementation "org.jetbrains.kotlin:kotlin-reflect" + implementation "org.jetbrains.kotlin:kotlin-reflect:1.8.20" implementation("org.jetbrains.kotlinx:kotlinx-coroutines-core:1.7.1") implementation "com.fasterxml.jackson.datatype:jackson-datatype-jsr310" diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt index 5c1a749fa..bc70e1ce5 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt @@ -43,7 +43,6 @@ abstract class BaseRootKotlinWorkflowInboundCallsInterceptor(protected val workf // workflowContext.handleInterceptedSignal(input) } - override fun handleQuery(input: WorkflowInboundCallsInterceptor.QueryInput): WorkflowInboundCallsInterceptor.QueryOutput { TODO("Implement") // return workflowContext.handleInterceptedQuery(input) @@ -58,4 +57,4 @@ abstract class BaseRootKotlinWorkflowInboundCallsInterceptor(protected val workf TODO("Implement") // return workflowContext.handleInterceptedExecuteUpdate(input) } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt index dd65ec9c5..22bb34cbb 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt @@ -53,7 +53,7 @@ internal class DynamicKotlinWorkflowDefinition( val result = workflowInvoker!!.execute( WorkflowInboundCallsInterceptor.WorkflowInput(header!!, arrayOf(args)) ) - return dataConverter.toPayloads(result!!.result).orElse(null) + return dataConverter.toPayloads(result.result).orElse(null) } internal inner class RootWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext?) : @@ -63,7 +63,7 @@ internal class DynamicKotlinWorkflowDefinition( private var workflow: KotlinDynamicWorkflow? = null override suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) { - super.init(outboundCalls!!) + super.init(outboundCalls) newInstance() WorkflowInternal.registerListener(workflow) } @@ -78,4 +78,4 @@ internal class DynamicKotlinWorkflowDefinition( workflow = factory.apply() } } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt index 3e953dd00..d18c34cbd 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt @@ -19,7 +19,6 @@ */ package io.temporal.internal.async -import io.temporal.workflow.Promise import java.lang.reflect.Type /** @@ -28,29 +27,29 @@ import java.lang.reflect.Type * implemented in other languages. Created through [Workflow.newActivityStub]. */ interface KotlinActivityStub { - /** - * Executes an activity by its type name and arguments. Blocks until the activity completion. - * - * @param activityName name of an activity type to execute. - * @param resultClass the expected return type of the activity. Use Void.class for activities that - * return void type. - * @param args arguments of the activity. - * @param return type. - * @return an activity result. - */ - suspend fun execute(activityName: String, resultClass: Class, vararg args: Any): R? + /** + * Executes an activity by its type name and arguments. Blocks until the activity completion. + * + * @param activityName name of an activity type to execute. + * @param resultClass the expected return type of the activity. Use Void.class for activities that + * return void type. + * @param args arguments of the activity. + * @param return type. + * @return an activity result. + */ + suspend fun execute(activityName: String, resultClass: Class, vararg args: Any): R? - /** - * Executes an activity by its type name and arguments. Blocks until the activity completion. - * - * @param activityName name of an activity type to execute. - * @param resultClass the expected return class of the activity. Use Void.class for activities - * that return void type. - * @param resultType the expected return type of the activity. Differs from resultClass for - * generic types. - * @param args arguments of the activity. - * @param return type. - * @return an activity result. - */ - suspend fun execute(activityName: String, resultClass: Class, resultType: Type, vararg args: Any): R? -} \ No newline at end of file + /** + * Executes an activity by its type name and arguments. Blocks until the activity completion. + * + * @param activityName name of an activity type to execute. + * @param resultClass the expected return class of the activity. Use Void.class for activities + * that return void type. + * @param resultType the expected return type of the activity. Differs from resultClass for + * generic types. + * @param args arguments of the activity. + * @param return type. + * @return an activity result. + */ + suspend fun execute(activityName: String, resultClass: Class, resultType: Type, vararg args: Any): R? +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt index 70db08f08..f61a3a602 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt @@ -29,13 +29,18 @@ internal class KotlinActivityStubImpl( private val activityExecutor: WorkflowOutboundCallsInterceptor ) : KotlinActivityStub { - private val options: ActivityOptions = ActivityOptions.newBuilder(options).validateAndBuildWithDefaults(); + private val options: ActivityOptions = ActivityOptions.newBuilder(options).validateAndBuildWithDefaults() override suspend fun execute(activityName: String, resultClass: Class, vararg args: Any): R? { return activityExecutor .executeActivity( WorkflowOutboundCallsInterceptor.ActivityInput( - activityName, resultClass, resultClass, args, options, Header.empty() + activityName, + resultClass, + resultClass, + args, + options, + Header.empty() ) ).result } @@ -49,8 +54,13 @@ internal class KotlinActivityStubImpl( return activityExecutor .executeActivity( WorkflowOutboundCallsInterceptor.ActivityInput( - activityName, resultClass, resultType, args, options, Header.empty() + activityName, + resultClass, + resultType, + args, + options, + Header.empty() ) ).result } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt index b6fbf7572..6e6456438 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt @@ -8,12 +8,10 @@ import io.temporal.api.query.v1.WorkflowQuery import io.temporal.client.WorkflowClient import io.temporal.common.context.ContextPropagator import io.temporal.common.converter.DataConverter -import io.temporal.common.converter.DefaultDataConverter import io.temporal.internal.replay.ReplayWorkflow import io.temporal.internal.replay.ReplayWorkflowContext import io.temporal.internal.replay.WorkflowContext import io.temporal.internal.statemachines.UpdateProtocolCallback -import io.temporal.internal.sync.* import io.temporal.internal.worker.WorkflowExecutorCache import io.temporal.worker.WorkflowImplementationOptions import kotlinx.coroutines.async @@ -49,22 +47,27 @@ class KotlinWorkflow( private val dispatcher = TemporalCoroutineDispatcher() private val coroutineDispatcher = TemporalCallbackCoroutineDispatcher(dispatcher) - private val scope = TemporalScope() + private val scope = TemporalScope(workflowContext) private var executionHandler: KotlinWorkflowExecutionHandler? = null override fun start(event: HistoryEvent, context: ReplayWorkflowContext) { require( - !(event.eventType != EventType.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED - || !event.hasWorkflowExecutionStartedEventAttributes()) + !( + event.eventType != EventType.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED || + !event.hasWorkflowExecutionStartedEventAttributes() + ) ) { "first event is not WorkflowExecutionStarted, but " + event.eventType } val startEvent = event.workflowExecutionStartedEventAttributes val workflowType = startEvent.workflowType requireNotNull(workflow) { "Unknown workflow type: $workflowType" } - workflowContext!!.setReplayContext(context) + workflowContext.setReplayContext(context) executionHandler = KotlinWorkflowExecutionHandler( - workflowContext, workflow, startEvent, workflowImplementationOptions!! + workflowContext, + workflow, + startEvent, + workflowImplementationOptions!! ) // The following order is ensured by this code and DeterministicRunner implementation: // 1. workflow.initialize @@ -97,8 +100,8 @@ class KotlinWorkflow( if (executionHandler == null) { return false } - dispatcher!!.eventLoop(defaultDeadlockDetectionTimeout) - return dispatcher!!.isDone() || executionHandler!!.isDone // Do not wait for all other threads. + dispatcher.eventLoop(defaultDeadlockDetectionTimeout) + return dispatcher.isDone() || executionHandler!!.isDone // Do not wait for all other threads. } override fun getOutput(): Optional { @@ -112,8 +115,8 @@ class KotlinWorkflow( override fun close() { if (executionHandler != null) { - //TODO: Validate that cancel is the right operation to call here - dispatcher!!.cancel() + // TODO: Validate that cancel is the right operation to call here + dispatcher.cancel() } } @@ -127,7 +130,7 @@ class KotlinWorkflow( TODO("Implement stack trace if possible") // return DefaultDataConverter.STANDARD_INSTANCE.toPayloads(runner!!.stackTrace()) } - val args = if (query.hasQueryArgs()) Optional.of(query.queryArgs) else Optional.empty() +// val args = if (query.hasQueryArgs()) Optional.of(query.queryArgs) else Optional.empty() TODO("Implement query") // return executionHandler!!.handleQuery(query.queryType, args) } @@ -135,4 +138,4 @@ class KotlinWorkflow( override fun getWorkflowContext(): WorkflowContext? { return workflowContext } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt index ac23125c1..a81e98891 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt @@ -1,6 +1,5 @@ package io.temporal.internal.async -import com.google.common.base.MoreObjects import com.uber.m3.tally.Scope import io.temporal.activity.ActivityOptions import io.temporal.activity.LocalActivityOptions @@ -29,6 +28,7 @@ import io.temporal.payload.context.ActivitySerializationContext import io.temporal.payload.context.WorkflowSerializationContext import io.temporal.worker.WorkflowImplementationOptions import io.temporal.workflow.Functions +import kotlinx.coroutines.ExperimentalCoroutinesApi import kotlinx.coroutines.suspendCancellableCoroutine import org.slf4j.LoggerFactory import java.lang.reflect.Type @@ -78,7 +78,6 @@ class KotlinWorkflowContext( // SyncWorkflow#start(HistoryEvent, ReplayWorkflowContext) headInboundInterceptor = InitialWorkflowInboundCallsInterceptor(this) headOutboundInterceptor = this - } private val dataConverterWithCurrentWorkflowContext = dataConverter.withContext( @@ -107,7 +106,7 @@ class KotlinWorkflowContext( fun initHeadInboundCallsInterceptor(head: WorkflowInboundCallsInterceptor) { headInboundInterceptor = head - //TODO: signal, query, update dispatchers + // TODO: signal, query, update dispatchers // signalDispatcher.setInboundCallsInterceptor(head) // queryDispatcher.setInboundCallsInterceptor(head) // updateDispatcher.setInboundCallsInterceptor(head) @@ -117,7 +116,6 @@ class KotlinWorkflowContext( return dataConverterWithCurrentWorkflowContext.exceptionToFailure(failure) } - override fun getWorkflowImplementationOptions(): WorkflowImplementationOptions { return workflowImplementationOptions!! } @@ -136,13 +134,17 @@ class KotlinWorkflowContext( return } ActivityOptionUtils.mergePredefinedLocalActivityOptions( - localActivityOptionsMap!!, activityTypeToOption + localActivityOptionsMap!!, + activityTypeToOption ) } override fun getLastCompletionResult(resultClass: Class, resultType: Type): R? { return dataConverter.fromPayloads( - 0, Optional.ofNullable(replayContext.lastCompletionResult), resultClass, resultType + 0, + Optional.ofNullable(replayContext.lastCompletionResult), + resultClass, + resultType ) } @@ -172,16 +174,23 @@ class KotlinWorkflowContext( input.activityName, // input.getOptions().getTaskQueue() may be not specified, workflow task queue is used // by the Server in this case - if (input.options != null && input.options.taskQueue != null) input.options.taskQueue else replayContext.taskQueue, + if (input.options.taskQueue != null) input.options.taskQueue else replayContext.taskQueue, false ) val dataConverterWithActivityContext = dataConverter.withContext(serializationContext) val args = dataConverterWithActivityContext.toPayloads(*input.args) try { val output = executeActivityOnce(input.activityName, input.options, input.header, args) - val result = if (input.resultType !== Void.TYPE) dataConverterWithActivityContext.fromPayloads( - 0, Optional.of(output.result), input.resultClass, input.resultType - ) else null + val result = if (input.resultType !== Void.TYPE) { + dataConverterWithActivityContext.fromPayloads( + 0, + Optional.of(output.result), + input.resultClass, + input.resultType + ) + } else { + null + } return WorkflowOutboundCallsInterceptor.ActivityOutput(output.activityId, result) } catch (e: FailureWrapperException) { throw dataConverterWithActivityContext.failureToException(e.failure) @@ -198,7 +207,6 @@ class KotlinWorkflowContext( override fun newRandom(): Random = replayContext.newRandom() - override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput): WorkflowOutboundCallsInterceptor.SignalExternalOutput { TODO("Not yet implemented") } @@ -233,7 +241,7 @@ class KotlinWorkflowContext( TODO("Not yet implemented") } - override fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int { + override fun getVersion(changeId: String, minSupported: Int, maxSupported: Int): Int { TODO("Not yet implemented") } @@ -253,7 +261,7 @@ class KotlinWorkflowContext( TODO("Not yet implemented") } - override fun registerDynamicSignalHandler(handler: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput) { + override fun registerDynamicSignalHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput) { TODO("Not yet implemented") } @@ -273,7 +281,7 @@ class KotlinWorkflowContext( TODO("Not yet implemented") } - override fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>) { + override fun upsertTypedSearchAttributes(searchAttributeUpdates: List>) { TODO("Not yet implemented") } @@ -284,8 +292,12 @@ class KotlinWorkflowContext( val metricScope: Scope get() = replayContext.metricsScope + @OptIn(ExperimentalCoroutinesApi::class) private suspend fun executeActivityOnce( - activityTypeName: String, options: ActivityOptions, header: Header, input: Optional + activityTypeName: String, + options: ActivityOptions, + header: Header, + input: Optional ): ActivityOutput { val params: ExecuteActivityParameters = constructExecuteActivityParameters(activityTypeName, options, header, input) @@ -295,7 +307,7 @@ class KotlinWorkflowContext( params ) { output: Optional, failure: Failure? -> if (failure == null) { - continuation.resume(ActivityOutput(activityId!!, output.get()), onCancellation = null); + continuation.resume(ActivityOutput(activityId!!, output.get()), onCancellation = null) } else { continuation.resumeWithException(FailureWrapperException(failure)) } @@ -310,7 +322,10 @@ class KotlinWorkflowContext( // TODO: this is copy of the similar method in SyncWorkflowContext. Extract to common class. private fun constructExecuteActivityParameters( - name: String, options: ActivityOptions, header: Header, input: Optional + name: String, + options: ActivityOptions, + header: Header, + input: Optional ): ExecuteActivityParameters { var taskQueue = options.taskQueue if (taskQueue == null) { @@ -387,9 +402,9 @@ class KotlinWorkflowContext( BaseRootKotlinWorkflowInboundCallsInterceptor(workflowContext) { override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { throw UnsupportedOperationException( - "SyncWorkflowContext should be initialized with a non-initial WorkflowInboundCallsInterceptor " - + "before #execute can be called" + "SyncWorkflowContext should be initialized with a non-initial WorkflowInboundCallsInterceptor " + + "before #execute can be called" ) } } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt index 3278840e8..b0a083cb1 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt @@ -2,12 +2,11 @@ package io.temporal.internal.async import io.temporal.api.common.v1.Payloads import io.temporal.common.interceptors.Header -import java.util.* interface KotlinWorkflowDefinition { - /** Always called first. */ - suspend fun initialize() + /** Always called first. */ + suspend fun initialize() - suspend fun execute(header: Header?, input: Payloads?): Payloads? -} \ No newline at end of file + suspend fun execute(header: Header?, input: Payloads?): Payloads? +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt index 2ed4011cd..0f14809dd 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt @@ -109,8 +109,8 @@ internal class KotlinWorkflowExecutionHandler( val fullReplayDirectQueryName = replayWorkflowContext.fullReplayDirectQueryName val info = Workflow.getInfo() if (fullReplayDirectQueryName != null) { - if (log.isDebugEnabled - && !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) + if (log.isDebugEnabled && + !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) ) { log.debug( "Replayed workflow execution failure WorkflowId='{}', RunId={}, WorkflowType='{}' for direct query QueryType='{}'", @@ -122,8 +122,8 @@ internal class KotlinWorkflowExecutionHandler( ) } } else { - if (log.isWarnEnabled - && !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) + if (log.isWarnEnabled && + !requestedCancellation(replayWorkflowContext.isCancelRequested, exception) ) { log.warn( "Workflow execution failure WorkflowId='{}', RunId={}, WorkflowType='{}'", @@ -158,4 +158,4 @@ internal class KotlinWorkflowExecutionHandler( return false } } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt index c600451e3..0aa81b62e 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt @@ -44,7 +44,9 @@ import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor import io.temporal.kotlin.workflow.KotlinDynamicWorkflow import io.temporal.payload.context.WorkflowSerializationContext import io.temporal.serviceclient.CheckedExceptionWrapper -import io.temporal.worker.* +import io.temporal.worker.TypeAlreadyRegisteredException +import io.temporal.worker.WorkerOptions +import io.temporal.worker.WorkflowImplementationOptions import io.temporal.workflow.DynamicWorkflow import io.temporal.workflow.Functions.Func import io.temporal.workflow.Functions.Func1 @@ -67,8 +69,8 @@ class KotlinWorkflowImplementationFactory( // defaultDeadlockDetectionTimeout = workerOptions.defaultDeadlockDetectionTimeout // namespace = clientOptions.namespace - //TODO: Kotlin specific interceptors. - private var workerInterceptors: Array = emptyArray() //factoryOptions.workerInterceptors + // TODO: Kotlin specific interceptors. + private var workerInterceptors: Array = emptyArray() // factoryOptions.workerInterceptors private var dataConverter: DataConverter = clientOptions.dataConverter private var contextPropagators: List = clientOptions.contextPropagators private var defaultDeadlockDetectionTimeout: Long = workerOptions.defaultDeadlockDetectionTimeout @@ -87,7 +89,8 @@ class KotlinWorkflowImplementationFactory( private val implementationOptions = Collections.synchronizedMap(HashMap()) override fun registerWorkflowImplementationTypes( - options: WorkflowImplementationOptions, workflowImplementationTypes: Array> + options: WorkflowImplementationOptions, + workflowImplementationTypes: Array> ) { for (type in workflowImplementationTypes) { registerWorkflowImplementationType(options, type) @@ -98,8 +101,11 @@ class KotlinWorkflowImplementationFactory( * @param clazz has to be a workflow interface class. The only exception is if it's a * DynamicWorkflow class. */ + override fun addWorkflowImplementationFactory( - options: WorkflowImplementationOptions, clazz: Class, factory: Func + options: WorkflowImplementationOptions, + clazz: Class, + factory: Func ) { if (DynamicWorkflow::class.java.isAssignableFrom(clazz)) { if (dynamicWorkflowImplementationFactory != null) { @@ -108,6 +114,7 @@ class KotlinWorkflowImplementationFactory( "An implementation of KotlinDynamicWorkflow or its factory is already registered with the worker" ) } + @Suppress("UNCHECKED_CAST") dynamicWorkflowImplementationFactory = factory as Func return } @@ -116,7 +123,7 @@ class KotlinWorkflowImplementationFactory( require(workflowMetadata.workflowMethod.isPresent) { "Workflow interface doesn't contain a method annotated with @WorkflowMethod: $clazz" } val methodsMetadata = workflowMetadata.methodsMetadata for (methodMetadata in methodsMetadata) { - when (methodMetadata.type) { + when (methodMetadata.type!!) { WorkflowMethodType.WORKFLOW -> { val typeName = methodMetadata.name if (workflowDefinitions.containsKey(typeName)) { @@ -137,12 +144,17 @@ class KotlinWorkflowImplementationFactory( implementationOptions[typeName] = options } WorkflowMethodType.SIGNAL -> {} + WorkflowMethodType.NONE -> TODO() + WorkflowMethodType.QUERY -> TODO() + WorkflowMethodType.UPDATE -> TODO() + WorkflowMethodType.UPDATE_VALIDATOR -> TODO() } } } private fun registerWorkflowImplementationType( - options: WorkflowImplementationOptions, workflowImplementationClass: Class + options: WorkflowImplementationOptions, + workflowImplementationClass: Class ) { // if (KotlinDynamicWorkflow::class.java.isAssignableFrom(workflowImplementationClass)) { // addWorkflowImplementationFactory( @@ -184,9 +196,11 @@ class KotlinWorkflowImplementationFactory( val workflowMetadata = POJOWorkflowImplMetadata.newInstance(workflowImplementationClass) val workflowMethods = workflowMetadata.workflowMethods require(!workflowMethods.isEmpty()) { - ("Workflow implementation doesn't implement any interface " - + "with a workflow method annotated with @WorkflowMethod: " - + workflowImplementationClass) + ( + "Workflow implementation doesn't implement any interface " + + "with a workflow method annotated with @WorkflowMethod: " + + workflowImplementationClass + ) } for (workflowMethod in workflowMethods) { val workflowName = workflowMethod.name @@ -207,20 +221,23 @@ class KotlinWorkflowImplementationFactory( } private fun getWorkflowDefinition( - workflowType: WorkflowType, workflowExecution: WorkflowExecution + workflowType: WorkflowType, + workflowExecution: WorkflowExecution ): KotlinWorkflowDefinition { val factory = workflowDefinitions[workflowType.name] if (factory == null) { if (dynamicWorkflowImplementationFactory != null) { return DynamicKotlinWorkflowDefinition( - dynamicWorkflowImplementationFactory!!, workerInterceptors, dataConverter + dynamicWorkflowImplementationFactory!!, + workerInterceptors, + dataConverter ) } throw Error( - "Unknown workflow type \"" - + workflowType.name - + "\". Known types are " - + workflowDefinitions.keys + "Unknown workflow type \"" + + workflowType.name + + "\". Known types are " + + workflowDefinitions.keys ) } return try { @@ -231,13 +248,14 @@ class KotlinWorkflowImplementationFactory( } override fun getWorkflow( - workflowType: WorkflowType, workflowExecution: WorkflowExecution + workflowType: WorkflowType, + workflowExecution: WorkflowExecution ): ReplayWorkflow { val workflow = getWorkflowDefinition(workflowType, workflowExecution) val workflowImplementationOptions = implementationOptions[workflowType.name] - val dataConverterWithWorkflowContext = dataConverter.withContext( - WorkflowSerializationContext(namespace, workflowExecution.workflowId) - ) +// val dataConverterWithWorkflowContext = dataConverter.withContext( + WorkflowSerializationContext(namespace, workflowExecution.workflowId) + return KotlinWorkflow( namespace, workflowExecution, @@ -277,15 +295,19 @@ class KotlinWorkflowImplementationFactory( @Throws(CanceledFailure::class, WorkflowExecutionException::class) override suspend fun execute(header: Header?, input: Payloads?): Payloads? { val args = dataConverterWithWorkflowContext.fromPayloads( - Optional.ofNullable(input), workflowMethod.parameterTypes, workflowMethod.genericParameterTypes + Optional.ofNullable(input), + workflowMethod.parameterTypes, + workflowMethod.genericParameterTypes ) Preconditions.checkNotNull(workflowInvoker, "initialize not called") val result = workflowInvoker!!.execute(WorkflowInboundCallsInterceptor.WorkflowInput(header, args)) return if (workflowMethod.returnType == Void.TYPE) { null - } else dataConverterWithWorkflowContext.toPayloads( - result.result - ).orElse(null) + } else { + dataConverterWithWorkflowContext.toPayloads( + result.result + ).orElse(null) + } } private inner class RootWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext) : @@ -320,26 +342,26 @@ class KotlinWorkflowImplementationFactory( } catch (e: NoSuchMethodException) { // Error to fail workflow task as this can be fixed by a new deployment. throw Error( - "Failure instantiating workflow implementation class " - + workflowImplementationClass.name, + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, e ) } catch (e: InstantiationException) { throw Error( - "Failure instantiating workflow implementation class " - + workflowImplementationClass.name, + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, e ) } catch (e: IllegalAccessException) { throw Error( - "Failure instantiating workflow implementation class " - + workflowImplementationClass.name, + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, e ) } catch (e: InvocationTargetException) { throw Error( - "Failure instantiating workflow implementation class " - + workflowImplementationClass.name, + "Failure instantiating workflow implementation class " + + workflowImplementationClass.name, e ) } @@ -349,10 +371,12 @@ class KotlinWorkflowImplementationFactory( } override fun toString(): String { - return ("POJOWorkflowImplementationFactory{" - + "registeredWorkflowTypes=" - + workflowDefinitions.keys - + '}') + return ( + "POJOWorkflowImplementationFactory{" + + "registeredWorkflowTypes=" + + workflowDefinitions.keys + + '}' + ) } companion object { @@ -360,14 +384,20 @@ class KotlinWorkflowImplementationFactory( val WORKFLOW_HANDLER_STACKTRACE_CUTOFF = ImmutableSet.builder() // POJO .add( ReflectionUtils.getMethodNameForStackTraceCutoff( - KotlinWorkflowImplementation::class.java, "execute", Header::class.java, Optional::class.java + KotlinWorkflowImplementation::class.java, + "execute", + Header::class.java, + Optional::class.java ) ) // Dynamic .add( ReflectionUtils.getMethodNameForStackTraceCutoff( - DynamicKotlinWorkflowDefinition::class.java, "execute", Header::class.java, Optional::class.java + DynamicKotlinWorkflowDefinition::class.java, + "execute", + Header::class.java, + Optional::class.java ) ) .build() } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt index 23ab07b1b..0e4b99e28 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt @@ -25,7 +25,7 @@ import io.temporal.workflow.KotlinWorkflowInfo import java.time.Duration internal class KotlinWorkflowInfoImpl( - private val context: ReplayWorkflowContext, + private val context: ReplayWorkflowContext ) : KotlinWorkflowInfo { override val namespace: String = context.namespace override val workflowId: String = context.workflowId @@ -56,4 +56,4 @@ internal class KotlinWorkflowInfoImpl( "searchAttributes=$searchAttributes, parentWorkflowId=$parentWorkflowId, parentRunId=$parentRunId, " + "attempt=$attempt, cronSchedule=$cronSchedule, historyLength=$historyLength)" } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt index 1ed71d369..aa43b3ac3 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt @@ -34,15 +34,17 @@ import java.util.* /** * Never reference directly. It is public only because Java doesn't have internal package support. */ -object KotlinWorkflowInternal { - const val DEFAULT_VERSION = -1 +class KotlinWorkflowInternal { - /** - * Register query or queries implementation object. There is no need to register top level - * workflow implementation object as it is done implicitly. Only methods annotated with @[ ] are registered. TODO(quinn) LIES! - */ - fun registerListener(implementation: Any) { - TODO("Implement") + companion object { + const val DEFAULT_VERSION = -1 + + /** + * Register query or queries implementation object. There is no need to register top level + * workflow implementation object as it is done implicitly. Only methods annotated with @[ ] are registered. TODO(quinn) LIES! + */ + fun registerListener(implementation: Any) { + TODO("Implement") // if (implementation is DynamicSignalHandler) { // workflowOutboundInterceptor // .registerDynamicSignalHandler( @@ -175,30 +177,30 @@ object KotlinWorkflowInternal { // ("Missing update methods for update validator(s): " // + Joiner.on(", ").join(validators.keys)) // } - } + } - /** Should be used to get current time instead of [System.currentTimeMillis] */ - fun currentTimeMillis(): Long { - return workflowOutboundInterceptor.currentTimeMillis() - } + /** Should be used to get current time instead of [System.currentTimeMillis] */ + fun currentTimeMillis(): Long { + return workflowOutboundInterceptor.currentTimeMillis() + } - fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { - rootWorkflowContext.defaultActivityOptions = activityOptions - } + fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { + rootWorkflowContext.defaultActivityOptions = activityOptions + } - fun applyActivityOptions(activityTypeToOptions: Map) { - rootWorkflowContext.applyActivityOptions(activityTypeToOptions) - } + fun applyActivityOptions(activityTypeToOptions: Map) { + rootWorkflowContext.applyActivityOptions(activityTypeToOptions) + } - fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { - rootWorkflowContext.defaultLocalActivityOptions = localActivityOptions - } + fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { + rootWorkflowContext.defaultLocalActivityOptions = localActivityOptions + } - fun applyLocalActivityOptions( - activityTypeToOptions: Map - ) { - rootWorkflowContext.applyLocalActivityOptions(activityTypeToOptions) - } + fun applyLocalActivityOptions( + activityTypeToOptions: Map + ) { + rootWorkflowContext.applyLocalActivityOptions(activityTypeToOptions) + } // /** // * Creates client stub to activities that implement given interface. @@ -207,41 +209,41 @@ object KotlinWorkflowInternal { // * @param options options that together with the properties of [ ] specify the activity invocation parameters // * @param activityMethodOptions activity method-specific invocation parameters // */ -//// fun newActivityStub( -//// activityInterface: Class?, -//// options: ActivityOptions?, -//// activityMethodOptions: Map? -//// ): T { -//// // Merge the activity options we may have received from the workflow with the options we may -//// // have received in WorkflowImplementationOptions. -//// var options = options -//// val context = rootWorkflowContext -//// options = options ?: context.getDefaultActivityOptions() -//// val mergedActivityOptionsMap: Map -//// @Nonnull val predefinedActivityOptions = context.getActivityOptions() -//// if (activityMethodOptions != null && !activityMethodOptions.isEmpty() -//// && predefinedActivityOptions.isEmpty() -//// ) { -//// // we need to merge only in this case -//// mergedActivityOptionsMap = HashMap(predefinedActivityOptions) -//// ActivityOptionUtils.mergePredefinedActivityOptions( -//// mergedActivityOptionsMap, activityMethodOptions -//// ) -//// } else { -//// mergedActivityOptionsMap = MoreObjects.firstNonNull( -//// activityMethodOptions, -//// MoreObjects.firstNonNull(predefinedActivityOptions, emptyMap()) -//// ) -//// } -//// val invocationHandler = ActivityInvocationHandler.newInstance( -//// activityInterface, -//// options, -//// mergedActivityOptionsMap, -//// context.getWorkflowOutboundInterceptor() -//// ) -//// return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) -//// } -//// +// // fun newActivityStub( +// // activityInterface: Class?, +// // options: ActivityOptions?, +// // activityMethodOptions: Map? +// // ): T { +// // // Merge the activity options we may have received from the workflow with the options we may +// // // have received in WorkflowImplementationOptions. +// // var options = options +// // val context = rootWorkflowContext +// // options = options ?: context.getDefaultActivityOptions() +// // val mergedActivityOptionsMap: Map +// // @Nonnull val predefinedActivityOptions = context.getActivityOptions() +// // if (activityMethodOptions != null && !activityMethodOptions.isEmpty() +// // && predefinedActivityOptions.isEmpty() +// // ) { +// // // we need to merge only in this case +// // mergedActivityOptionsMap = HashMap(predefinedActivityOptions) +// // ActivityOptionUtils.mergePredefinedActivityOptions( +// // mergedActivityOptionsMap, activityMethodOptions +// // ) +// // } else { +// // mergedActivityOptionsMap = MoreObjects.firstNonNull( +// // activityMethodOptions, +// // MoreObjects.firstNonNull(predefinedActivityOptions, emptyMap()) +// // ) +// // } +// // val invocationHandler = ActivityInvocationHandler.newInstance( +// // activityInterface, +// // options, +// // mergedActivityOptionsMap, +// // context.getWorkflowOutboundInterceptor() +// // ) +// // return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) +// // } +// // // /** // * Creates client stub to local activities that implement given interface. // * @@ -284,9 +286,9 @@ object KotlinWorkflowInternal { // return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) // } - fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { - return KotlinActivityStubImpl(options, workflowOutboundInterceptor) - } + fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + return KotlinActivityStubImpl(options, workflowOutboundInterceptor) + } // fun newUntypedLocalActivityStub(options: LocalActivityOptions?): KotlinActivityStub { // return LocalActivityStubImpl.newInstance(options, workflowOutboundInterceptor) @@ -504,20 +506,19 @@ object KotlinWorkflowInternal { // } // } - val workflowInfo: KotlinWorkflowInfo - get() = KotlinWorkflowInfoImpl(rootWorkflowContext.getReplayContext()) + val workflowInfo: KotlinWorkflowInfo + get() = KotlinWorkflowInfoImpl(rootWorkflowContext.getReplayContext()) - val metricsScope: Scope - get() = rootWorkflowContext.metricScope + val metricsScope: Scope + get() = rootWorkflowContext.metricScope + fun randomUUID(): UUID { + return rootWorkflowContext.randomUUID() + } - fun randomUUID(): UUID { - return rootWorkflowContext.randomUUID() - } - - fun newRandom(): Random { - return rootWorkflowContext.newRandom() - } + fun newRandom(): Random { + return rootWorkflowContext.newRandom() + } // private val isLoggingEnabledInReplay: Boolean // private get() = rootWorkflowContext.isLoggingEnabledInReplay() @@ -582,38 +583,39 @@ object KotlinWorkflowInternal { // workflowOutboundInterceptor.upsertTypedSearchAttributes(*searchAttributeUpdates) // } - val dataConverter: DataConverter - get() = rootWorkflowContext.dataConverter + val dataConverter: DataConverter + get() = rootWorkflowContext.dataConverter - /** - * Name of the workflow type the interface defines. It is either the interface short name * or - * value of [WorkflowMethod.name] parameter. - * - * @param workflowInterfaceClass interface annotated with @WorkflowInterface - */ - fun getWorkflowType(workflowInterfaceClass: Class<*>?): String { - val metadata = POJOWorkflowInterfaceMetadata.newInstance(workflowInterfaceClass) - return metadata.workflowType.get() - } + /** + * Name of the workflow type the interface defines. It is either the interface short name * or + * value of [WorkflowMethod.name] parameter. + * + * @param workflowInterfaceClass interface annotated with @WorkflowInterface + */ + fun getWorkflowType(workflowInterfaceClass: Class<*>?): String { + val metadata = POJOWorkflowInterfaceMetadata.newInstance(workflowInterfaceClass) + return metadata.workflowType.get() + } - // Temporal Failure Values are additional user payload and serialized using user data - // converter - val previousRunFailure: Optional - get() = Optional.ofNullable(rootWorkflowContext.getReplayContext().previousRunFailure) // Temporal Failure Values are additional user payload and serialized using user data - // converter - .map { f: Failure? -> - dataConverter.failureToException( - f!! - ) - } + // Temporal Failure Values are additional user payload and serialized using user data + // converter + val previousRunFailure: Optional + get() = Optional.ofNullable(rootWorkflowContext.getReplayContext().previousRunFailure) // Temporal Failure Values are additional user payload and serialized using user data + // converter + .map { f: Failure? -> + dataConverter.failureToException( + f!! + ) + } - private val workflowOutboundInterceptor: WorkflowOutboundCallsInterceptor - private get() = rootWorkflowContext.getWorkflowOutboundInterceptor() + private val workflowOutboundInterceptor: WorkflowOutboundCallsInterceptor + private get() = rootWorkflowContext.getWorkflowOutboundInterceptor() - val rootWorkflowContext: KotlinWorkflowContext - get() { - val temporalCoroutineContext = coroutineContext[TemporalCoroutineContext] - ?: throw Error("Called from non workflow thread or coroutine") - return temporalCoroutineContext.workflowContext - } -} \ No newline at end of file + val rootWorkflowContext: KotlinWorkflowContext + get() { + val temporalCoroutineContext = coroutineContext[TemporalCoroutineContext] + ?: throw Error("Called from non workflow thread or coroutine") + return temporalCoroutineContext.workflowContext + } + } +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt index 59dad84e6..d18bdc948 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt @@ -1,7 +1,10 @@ package io.temporal.internal.async -import kotlinx.coroutines.* -import java.lang.Runnable +import kotlinx.coroutines.CancellableContinuation +import kotlinx.coroutines.CoroutineDispatcher +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Delay +import kotlinx.coroutines.InternalCoroutinesApi import java.util.* import java.util.concurrent.DelayQueue import java.util.concurrent.Delayed @@ -9,10 +12,11 @@ import java.util.concurrent.TimeUnit import kotlin.coroutines.AbstractCoroutineContextElement import kotlin.coroutines.CoroutineContext +@Suppress("UNUSED_PARAMETER") @OptIn(InternalCoroutinesApi::class) class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { - private val queue: Queue = LinkedList() + private val queue: java.util.Queue = LinkedList() private val callbackQueue: Queue = LinkedList() private val delayQueue: DelayQueue = DelayQueue() @@ -24,7 +28,7 @@ class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { callbackQueue.add(block) } - //TODO: deadlock detector + // TODO: deadlock detector fun eventLoop(defaultDeadlockDetectionTimeout: Long): Boolean { // println("eventLoop begin") if (isDone()) { @@ -94,7 +98,7 @@ class TemporalCallbackCoroutineDispatcher(val dispatcher: TemporalCoroutineDispa } internal class TemporalScope(private val workflowContext: KotlinWorkflowContext) : CoroutineScope { - //TODO: Add argument to the Temporal context. + // TODO: Add argument to the Temporal context. override val coroutineContext: CoroutineContext = TemporalCoroutineContext(workflowContext) // CoroutineScope is used intentionally for user-friendly representation diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt index 21a5bc70f..0a874863b 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt @@ -85,13 +85,13 @@ import io.temporal.common.interceptors.ActivityInboundCallsInterceptor */ @Experimental interface KotlinWorkerInterceptor { - /** - * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `next` [WorkflowInboundCallsInterceptor], but it may change the input parameters. - * - * @param next an existing interceptor instance to be proxied by the interceptor created inside - * this method - * @return an interceptor that passes all the calls to `next` - */ - fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor - fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor -} \ No newline at end of file + /** + * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `next` [WorkflowInboundCallsInterceptor], but it may change the input parameters. + * + * @param next an existing interceptor instance to be proxied by the interceptor created inside + * this method + * @return an interceptor that passes all the calls to `next` + */ + fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor + fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt index 4a9edb9b1..0a05dfbff 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt @@ -22,11 +22,11 @@ package io.temporal.kotlin.interceptors import io.temporal.common.interceptors.ActivityInboundCallsInterceptor class WorkerInterceptorBase : KotlinWorkerInterceptor { - override fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor { - return next - } + override fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor { + return next + } - override fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor { - return next - } -} \ No newline at end of file + override fun interceptActivity(next: ActivityInboundCallsInterceptor): ActivityInboundCallsInterceptor { + return next + } +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt index 0770075ee..a5874de7a 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt @@ -45,56 +45,55 @@ import io.temporal.common.interceptors.Header */ @Experimental interface WorkflowInboundCallsInterceptor { - class WorkflowInput(val header: Header?, val arguments: Array) - class WorkflowOutput(val result: Any?) - class SignalInput(val signalName: String, val arguments: Array, val eventId: Long) - class QueryInput(val queryName: String, val arguments: Array) - class QueryOutput(val result: Any) + class WorkflowInput(val header: Header?, val arguments: Array) + class WorkflowOutput(val result: Any?) + class SignalInput(val signalName: String, val arguments: Array, val eventId: Long) + class QueryInput(val queryName: String, val arguments: Array) + class QueryOutput(val result: Any) - @Experimental - class UpdateInput(val updateName: String, val arguments: Array) + @Experimental + class UpdateInput(val updateName: String, val arguments: Array) - @Experimental - class UpdateOutput(val result: Any) + @Experimental + class UpdateOutput(val result: Any) - /** - * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `outboundCalls`, but it may change the input parameters. - * - * - * The instance should be passed into the {next.init(newWorkflowOutboundCallsInterceptor)}. - * - * @param outboundCalls an existing interceptor instance to be proxied by the interceptor created - * inside this method - * @see KotlinWorkerInterceptor.interceptWorkflow for the definition of "next" {@link - * * WorkflowInboundCallsInterceptor} - */ - suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) + /** + * Called when workflow class is instantiated. May create a [ ] instance. The instance must forward all the calls to `outboundCalls`, but it may change the input parameters. + * + * + * The instance should be passed into the {next.init(newWorkflowOutboundCallsInterceptor)}. + * + * @param outboundCalls an existing interceptor instance to be proxied by the interceptor created + * inside this method + * @see KotlinWorkerInterceptor.interceptWorkflow for the definition of "next" {@link + * * WorkflowInboundCallsInterceptor} + */ + suspend fun init(outboundCalls: WorkflowOutboundCallsInterceptor) - /** - * Called when workflow main method is called. - * - * @return result of the workflow execution. - */ - suspend fun execute(input: WorkflowInput): WorkflowOutput + /** + * Called when workflow main method is called. + * + * @return result of the workflow execution. + */ + suspend fun execute(input: WorkflowInput): WorkflowOutput - /** Called when signal is delivered to a workflow execution. */ - suspend fun handleSignal(input: SignalInput) + /** Called when signal is delivered to a workflow execution. */ + suspend fun handleSignal(input: SignalInput) - /** Called when a workflow is queried. */ - fun handleQuery(input: QueryInput): QueryOutput + /** Called when a workflow is queried. */ + fun handleQuery(input: QueryInput): QueryOutput - /** - * Called when update workflow execution request is delivered to a workflow execution, before the - * update is executed. - */ - @Experimental - fun validateUpdate(input: UpdateInput) + /** + * Called when update workflow execution request is delivered to a workflow execution, before the + * update is executed. + */ + @Experimental + fun validateUpdate(input: UpdateInput) - /** - * Called when update workflow execution request is delivered to a workflow execution, after - * passing the validator. - */ - @Experimental - suspend fun executeUpdate(input: UpdateInput): UpdateOutput - -} \ No newline at end of file + /** + * Called when update workflow execution request is delivered to a workflow execution, after + * passing the validator. + */ + @Experimental + suspend fun executeUpdate(input: UpdateInput): UpdateOutput +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt index e2880580f..a4d849d5c 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt @@ -45,4 +45,4 @@ class WorkflowInboundCallsInterceptorBase(private val next: WorkflowInboundCalls override suspend fun executeUpdate(input: WorkflowInboundCallsInterceptor.UpdateInput): WorkflowInboundCallsInterceptor.UpdateOutput { return next.executeUpdate(input) } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt index 59a64400d..e9daf8c70 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -25,8 +25,15 @@ import io.temporal.api.common.v1.WorkflowExecution import io.temporal.common.Experimental import io.temporal.common.SearchAttributeUpdate import io.temporal.common.interceptors.Header -import io.temporal.workflow.* -import io.temporal.workflow.Functions.* +import io.temporal.workflow.ChildWorkflowOptions +import io.temporal.workflow.ContinueAsNewOptions +import io.temporal.workflow.DynamicQueryHandler +import io.temporal.workflow.DynamicSignalHandler +import io.temporal.workflow.DynamicUpdateHandler +import io.temporal.workflow.Functions.Func +import io.temporal.workflow.Functions.Func1 +import io.temporal.workflow.Functions.Proc1 +import io.temporal.workflow.Promise import java.lang.reflect.Type import java.time.Duration import java.util.* @@ -146,30 +153,34 @@ interface WorkflowOutboundCallsInterceptor { suspend fun signalExternalWorkflow(input: SignalExternalInput): SignalExternalOutput fun cancelWorkflow(input: CancelWorkflowInput): CancelWorkflowOutput - //TODO: Consider removing sleep and keep only built in delay + // TODO: Consider removing sleep and keep only built in delay suspend fun sleep(duration: Duration) suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean suspend fun await(reason: String?, unblockCondition: Supplier) fun sideEffect(resultClass: Class, resultType: Type, func: Func): R? fun mutableSideEffect( - id: String, resultClass: Class, resultType: Type, updated: BiPredicate, func: Func + id: String, + resultClass: Class, + resultType: Type, + updated: BiPredicate, + func: Func ): R? - fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int + fun getVersion(changeId: String, minSupported: Int, maxSupported: Int): Int fun continueAsNew(input: ContinueAsNewInput) fun registerQuery(input: RegisterQueryInput) fun registerSignalHandlers(input: RegisterSignalHandlersInput) @Experimental fun registerUpdateHandlers(input: RegisterUpdateHandlersInput) - fun registerDynamicSignalHandler(handler: RegisterDynamicSignalHandlerInput) + fun registerDynamicSignalHandler(input: RegisterDynamicSignalHandlerInput) fun registerDynamicQueryHandler(input: RegisterDynamicQueryHandlerInput) @Experimental fun registerDynamicUpdateHandler(input: RegisterDynamicUpdateHandlerInput) fun randomUUID(): UUID fun upsertSearchAttributes(searchAttributes: Map) - fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>) + fun upsertTypedSearchAttributes(searchAttributeUpdates: List>) fun currentTimeMillis(): Long -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt index 03b34f71e..6d8f5705b 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt @@ -21,7 +21,6 @@ package io.temporal.kotlin.interceptors import io.temporal.common.SearchAttributeUpdate import io.temporal.workflow.Functions.Func -import io.temporal.workflow.Promise import java.lang.reflect.Type import java.time.Duration import java.util.* @@ -30,98 +29,102 @@ import java.util.function.Supplier /** Convenience base class for WorkflowOutboundCallsInterceptor implementations. */ class WorkflowOutboundCallsInterceptorBase(private val next: WorkflowOutboundCallsInterceptor) : - WorkflowOutboundCallsInterceptor { - override suspend fun executeActivity(input: WorkflowOutboundCallsInterceptor.ActivityInput?): WorkflowOutboundCallsInterceptor.ActivityOutput? { - return next.executeActivity(input) - } - - override suspend fun executeLocalActivity(input: WorkflowOutboundCallsInterceptor.LocalActivityInput?): WorkflowOutboundCallsInterceptor.LocalActivityOutput? { - return next.executeLocalActivity(input) - } - - override suspend fun executeChildWorkflow(input: WorkflowOutboundCallsInterceptor.ChildWorkflowInput?): WorkflowOutboundCallsInterceptor.ChildWorkflowOutput? { - return next.executeChildWorkflow(input) - } - - override fun newRandom(): Random? { - return next.newRandom() - } - - override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput?): WorkflowOutboundCallsInterceptor.SignalExternalOutput? { - return next.signalExternalWorkflow(input) - } - - override fun cancelWorkflow(input: WorkflowOutboundCallsInterceptor.CancelWorkflowInput?): WorkflowOutboundCallsInterceptor.CancelWorkflowOutput? { - return next.cancelWorkflow(input) - } - - override suspend fun sleep(duration: Duration?) { - next.sleep(duration) - } - - override suspend fun await(timeout: Duration?, reason: String?, unblockCondition: Supplier?): Boolean { - return next.await(timeout, reason, unblockCondition) - } - - override suspend fun await(reason: String?, unblockCondition: Supplier?) { - next.await(reason, unblockCondition) - } - - override fun sideEffect(resultClass: Class?, resultType: Type?, func: Func?): R? { - return next.sideEffect(resultClass, resultType, func) - } - - override fun mutableSideEffect( - id: String?, resultClass: Class?, resultType: Type?, updated: BiPredicate?, func: Func? - ): R? { - return next.mutableSideEffect(id, resultClass, resultType, updated, func) - } - - override fun getVersion(changeId: String?, minSupported: Int, maxSupported: Int): Int { - return next.getVersion(changeId, minSupported, maxSupported) - } - - override fun continueAsNew(input: WorkflowOutboundCallsInterceptor.ContinueAsNewInput?) { - next.continueAsNew(input) - } - - override fun registerQuery(input: WorkflowOutboundCallsInterceptor.RegisterQueryInput?) { - next.registerQuery(input) - } - - override fun registerSignalHandlers(input: WorkflowOutboundCallsInterceptor.RegisterSignalHandlersInput?) { - next.registerSignalHandlers(input) - } - - override fun registerUpdateHandlers(input: WorkflowOutboundCallsInterceptor.RegisterUpdateHandlersInput?) { - next.registerUpdateHandlers(input) - } - - override fun registerDynamicSignalHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput?) { - next.registerDynamicSignalHandler(input) - } - - override fun registerDynamicQueryHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicQueryHandlerInput?) { - next.registerDynamicQueryHandler(input) - } - - override fun registerDynamicUpdateHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicUpdateHandlerInput?) { - next.registerDynamicUpdateHandler(input) - } - - override fun randomUUID(): UUID? { - return next.randomUUID() - } - - override fun upsertSearchAttributes(searchAttributes: Map?) { - next.upsertSearchAttributes(searchAttributes) - } - - override fun upsertTypedSearchAttributes(vararg searchAttributeUpdates: SearchAttributeUpdate<*>?) { - next.upsertTypedSearchAttributes(*searchAttributeUpdates) - } - - override fun currentTimeMillis(): Long { - return next.currentTimeMillis() - } -} \ No newline at end of file + WorkflowOutboundCallsInterceptor { + override suspend fun executeActivity(input: WorkflowOutboundCallsInterceptor.ActivityInput): WorkflowOutboundCallsInterceptor.ActivityOutput { + return next.executeActivity(input) + } + + override suspend fun executeLocalActivity(input: WorkflowOutboundCallsInterceptor.LocalActivityInput): WorkflowOutboundCallsInterceptor.LocalActivityOutput { + return next.executeLocalActivity(input) + } + + override suspend fun executeChildWorkflow(input: WorkflowOutboundCallsInterceptor.ChildWorkflowInput): WorkflowOutboundCallsInterceptor.ChildWorkflowOutput { + return next.executeChildWorkflow(input) + } + + override fun newRandom(): Random { + return next.newRandom() + } + + override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput): WorkflowOutboundCallsInterceptor.SignalExternalOutput { + return next.signalExternalWorkflow(input) + } + + override fun cancelWorkflow(input: WorkflowOutboundCallsInterceptor.CancelWorkflowInput): WorkflowOutboundCallsInterceptor.CancelWorkflowOutput { + return next.cancelWorkflow(input) + } + + override suspend fun sleep(duration: Duration) { + return next.sleep(duration) + } + + override suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean { + return next.await(timeout, reason, unblockCondition) + } + + override suspend fun await(reason: String?, unblockCondition: Supplier) { + return next.await(reason, unblockCondition) + } + + override fun sideEffect(resultClass: Class, resultType: Type, func: Func): R? { + return next.sideEffect(resultClass, resultType, func) + } + + override fun mutableSideEffect( + id: String, + resultClass: Class, + resultType: Type, + updated: BiPredicate, + func: Func + ): R? { + return next.mutableSideEffect(id, resultClass, resultType, updated, func) + } + + override fun getVersion(changeId: String, minSupported: Int, maxSupported: Int): Int { + return next.getVersion(changeId, minSupported, maxSupported) + } + + override fun continueAsNew(input: WorkflowOutboundCallsInterceptor.ContinueAsNewInput) { + return next.continueAsNew(input) + } + + override fun registerQuery(input: WorkflowOutboundCallsInterceptor.RegisterQueryInput) { + return next.registerQuery(input) + } + + override fun registerSignalHandlers(input: WorkflowOutboundCallsInterceptor.RegisterSignalHandlersInput) { + return next.registerSignalHandlers(input) + } + + override fun registerUpdateHandlers(input: WorkflowOutboundCallsInterceptor.RegisterUpdateHandlersInput) { + return next.registerUpdateHandlers(input) + } + + override fun registerDynamicSignalHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicSignalHandlerInput) { + return next.registerDynamicSignalHandler(input) + } + + override fun registerDynamicQueryHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicQueryHandlerInput) { + return next.registerDynamicQueryHandler(input) + } + + override fun registerDynamicUpdateHandler(input: WorkflowOutboundCallsInterceptor.RegisterDynamicUpdateHandlerInput) { + return next.registerDynamicUpdateHandler(input) + } + + override fun randomUUID(): UUID { + return next.randomUUID() + } + + override fun upsertSearchAttributes(searchAttributes: Map) { + return next.upsertSearchAttributes(searchAttributes) + } + + override fun upsertTypedSearchAttributes(searchAttributeUpdates: List>) { + return next.upsertTypedSearchAttributes(searchAttributeUpdates) + } + + override fun currentTimeMillis(): Long { + return next.currentTimeMillis() + } +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt index 18ddc10aa..697f107ab 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt @@ -47,5 +47,5 @@ import io.temporal.common.converter.EncodedValues * @see io.temporal.activity.DynamicActivity */ interface KotlinDynamicWorkflow { - suspend fun execute(args: EncodedValues): Any? -} \ No newline at end of file + suspend fun execute(args: EncodedValues): Any? +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt index 981f3101b..2756cf172 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt @@ -12,9 +12,9 @@ class KotlinWorkerFactory(workflowClient: WorkflowClient, factoryOptions: Kotlin override fun newReplayWorkflowFactory( workerOptions: WorkerOptions, clientOptions: WorkflowClientOptions, - cache: WorkflowExecutorCache, + cache: WorkflowExecutorCache ): ReplayWorkflowFactory { - return KotlinWorkflowImplementationFactory(clientOptions, workerOptions, cache); + return KotlinWorkflowImplementationFactory(clientOptions, workerOptions, cache) } } @@ -25,5 +25,5 @@ fun toFactoryOptions(factoryOptions: KotlinWorkerFactoryOptions?): WorkerFactory return WorkerFactoryOptions.newBuilder() .setEnableLoggingInReplay(o.isEnableLoggingInReplay) .setWorkflowCacheSize(o.workflowCacheSize) - .build(); + .build() } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt index e3c15e1f6..66114fb48 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt @@ -157,4 +157,4 @@ class KotlinWorkerFactoryOptions private constructor( private const val DEFAULT_MAX_WORKFLOW_THREAD_COUNT = 600 val defaultInstance: KotlinWorkerFactoryOptions = newBuilder().build() } -} \ No newline at end of file +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt new file mode 100644 index 000000000..67384f435 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt @@ -0,0 +1,18 @@ +package io.temporal.workflow + +import io.temporal.activity.ActivityOptions +import io.temporal.internal.async.KotlinActivityStub +import io.temporal.internal.async.KotlinWorkflowInternal + +class KotlinWorkflow { + companion object { + /** + * Creates non typed client stub to activities. Allows executing activities by their string name. + * + * @param options specify the activity invocation parameters. + */ + fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + return KotlinWorkflowInternal.newUntypedActivityStub(options) + } + } +} diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt index 73e35eab6..f57846834 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt @@ -28,110 +28,110 @@ import java.util.* * immutable information about connected entities like Parent Workflow Execution or a previous Run. */ interface KotlinWorkflowInfo { - /** - * @return Workflow Namespace - */ - val namespace: String - - /** - * @return Workflow ID - */ - val workflowId: String - - /** - * @return Workflow Type - */ - val workflowType: String - - /** - * Note: RunId is unique identifier of one workflow code execution. Reset changes RunId. - * - * @return Workflow Run ID that is handled by the current workflow code execution. - * @see .getOriginalExecutionRunId - * @see .getFirstExecutionRunId - */ - val runId: String - - /** - * @return The very first original RunId of the current Workflow Execution preserved along the - * chain of ContinueAsNew, Retry, Cron and Reset. Identifies the whole Runs chain of Workflow - * Execution. - */ - val firstExecutionRunId: String - - /** - * @return Run ID of the previous Workflow Run which continued-as-new or retried or cron-scheduled - * into the current Workflow Run. - */ - val continuedExecutionRunId: String? - - /** - * Note: This value is NOT preserved by continue-as-new, retries or cron Runs. They are separate - * Runs of one Workflow Execution Chain. - * - * @return original RunId of the current Workflow Run. This value is preserved during Reset which - * changes RunID. - * @see .getFirstExecutionRunId - */ - val originalExecutionRunId: String - - /** - * @return Workflow Task Queue name - */ - val taskQueue: String - - /** - * @return Timeout for a Workflow Run specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowRunTimeout] - */ - val workflowRunTimeout: Duration? - - /** - * @return Timeout for the Workflow Execution specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowExecutionTimeout] - */ - val workflowExecutionTimeout: Duration? - - /** - * The time workflow run has started. Note that this time can be different from the time workflow - * function started actual execution. - */ - val runStartedTimestampMillis: Long - - /** - * This method is used to get raw proto serialized Search Attributes. - * - * - * Consider using more user-friendly methods on [Workflow] class, including [ ][Workflow.getSearchAttributes], [Workflow.getSearchAttribute] or [ ][Workflow.getSearchAttributeValues] instead of this method to access deserialized search - * attributes. - * - * @return raw Search Attributes Protobuf entity, null if empty - */ - @get:Deprecated("use {@link Workflow#getTypedSearchAttributes()} instead.") - val searchAttributes: SearchAttributes? - - /** - * @return Workflow ID of the parent Workflow - */ - val parentWorkflowId: String? - - /** - * @return Run ID of the parent Workflow - */ - val parentRunId: String? - - /** - * @return Workflow retry attempt handled by this Workflow code execution. Starts on "1". - */ - val attempt: Int - - /** - * @return Workflow cron schedule - */ - val cronSchedule: String? - - /** - * @return length of Workflow history up until the current moment of execution. This value changes - * during the lifetime of a Workflow Execution. You may use this information to decide when to - * call [Workflow.continueAsNew]. - */ - val historyLength: Long -} \ No newline at end of file + /** + * @return Workflow Namespace + */ + val namespace: String + + /** + * @return Workflow ID + */ + val workflowId: String + + /** + * @return Workflow Type + */ + val workflowType: String + + /** + * Note: RunId is unique identifier of one workflow code execution. Reset changes RunId. + * + * @return Workflow Run ID that is handled by the current workflow code execution. + * @see .getOriginalExecutionRunId + * @see .getFirstExecutionRunId + */ + val runId: String + + /** + * @return The very first original RunId of the current Workflow Execution preserved along the + * chain of ContinueAsNew, Retry, Cron and Reset. Identifies the whole Runs chain of Workflow + * Execution. + */ + val firstExecutionRunId: String + + /** + * @return Run ID of the previous Workflow Run which continued-as-new or retried or cron-scheduled + * into the current Workflow Run. + */ + val continuedExecutionRunId: String? + + /** + * Note: This value is NOT preserved by continue-as-new, retries or cron Runs. They are separate + * Runs of one Workflow Execution Chain. + * + * @return original RunId of the current Workflow Run. This value is preserved during Reset which + * changes RunID. + * @see .getFirstExecutionRunId + */ + val originalExecutionRunId: String + + /** + * @return Workflow Task Queue name + */ + val taskQueue: String + + /** + * @return Timeout for a Workflow Run specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowRunTimeout] + */ + val workflowRunTimeout: Duration? + + /** + * @return Timeout for the Workflow Execution specified during Workflow start in [ ][io.temporal.client.WorkflowOptions.Builder.setWorkflowExecutionTimeout] + */ + val workflowExecutionTimeout: Duration? + + /** + * The time workflow run has started. Note that this time can be different from the time workflow + * function started actual execution. + */ + val runStartedTimestampMillis: Long + + /** + * This method is used to get raw proto serialized Search Attributes. + * + * + * Consider using more user-friendly methods on [Workflow] class, including [ ][Workflow.getSearchAttributes], [Workflow.getSearchAttribute] or [ ][Workflow.getSearchAttributeValues] instead of this method to access deserialized search + * attributes. + * + * @return raw Search Attributes Protobuf entity, null if empty + */ + @get:Deprecated("use {@link Workflow#getTypedSearchAttributes()} instead.") + val searchAttributes: SearchAttributes? + + /** + * @return Workflow ID of the parent Workflow + */ + val parentWorkflowId: String? + + /** + * @return Run ID of the parent Workflow + */ + val parentRunId: String? + + /** + * @return Workflow retry attempt handled by this Workflow code execution. Starts on "1". + */ + val attempt: Int + + /** + * @return Workflow cron schedule + */ + val cronSchedule: String? + + /** + * @return length of Workflow history up until the current moment of execution. This value changes + * during the lifetime of a Workflow Execution. You may use this information to decide when to + * call [Workflow.continueAsNew]. + */ + val historyLength: Long +} diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt new file mode 100644 index 000000000..7e016f647 --- /dev/null +++ b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt @@ -0,0 +1,11 @@ +package io.temporal.workflow + +import org.junit.Assert.assertTrue +import org.junit.Test + +public class CoroutineWorkflowTest { + @Test + fun asyncLambdaTest() { + assertTrue(false) + } +} diff --git a/temporal-sdk/src/main/java/io/temporal/client/WorkflowClientInternalImpl.java b/temporal-sdk/src/main/java/io/temporal/client/WorkflowClientInternalImpl.java index 51627901e..4ed736990 100644 --- a/temporal-sdk/src/main/java/io/temporal/client/WorkflowClientInternalImpl.java +++ b/temporal-sdk/src/main/java/io/temporal/client/WorkflowClientInternalImpl.java @@ -47,7 +47,7 @@ import io.temporal.internal.sync.StubMarker; import io.temporal.serviceclient.MetricsTag; import io.temporal.serviceclient.WorkflowServiceStubs; -import io.temporal.worker.WorkerFactory; +import io.temporal.worker.BaseWorkerFactory; import io.temporal.workflow.Functions; import io.temporal.workflow.QueryMethod; import io.temporal.workflow.SignalMethod; @@ -542,12 +542,12 @@ public Object getInternal() { } @Override - public void registerWorkerFactory(WorkerFactory workerFactory) { + public void registerWorkerFactory(BaseWorkerFactory workerFactory) { workerFactoryRegistry.register(workerFactory); } @Override - public void deregisterWorkerFactory(WorkerFactory workerFactory) { + public void deregisterWorkerFactory(BaseWorkerFactory workerFactory) { workerFactoryRegistry.deregister(workerFactory); } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/client/EagerWorkflowTaskDispatcher.java b/temporal-sdk/src/main/java/io/temporal/internal/client/EagerWorkflowTaskDispatcher.java index 7573c99ab..60476bf29 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/client/EagerWorkflowTaskDispatcher.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/client/EagerWorkflowTaskDispatcher.java @@ -21,8 +21,8 @@ package io.temporal.internal.client; import io.temporal.common.interceptors.WorkflowClientCallsInterceptor; +import io.temporal.worker.BaseWorkerFactory; import io.temporal.worker.Worker; -import io.temporal.worker.WorkerFactory; import io.temporal.worker.WorkflowTaskDispatchHandle; import javax.annotation.Nullable; @@ -36,7 +36,7 @@ public EagerWorkflowTaskDispatcher(WorkerFactoryRegistry workerFactories) { @Nullable public WorkflowTaskDispatchHandle tryGetLocalDispatchHandler( WorkflowClientCallsInterceptor.WorkflowStartInput workflowStartInput) { - for (WorkerFactory workerFactory : workerFactories.workerFactoriesRandomOrder()) { + for (BaseWorkerFactory workerFactory : workerFactories.workerFactoriesRandomOrder()) { Worker worker = workerFactory.tryGetWorker(workflowStartInput.getOptions().getTaskQueue()); if (worker != null) { WorkflowTaskDispatchHandle workflowTaskDispatchHandle = worker.reserveWorkflowExecutor(); diff --git a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkerFactoryRegistry.java b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkerFactoryRegistry.java index 670a7619c..9c37ebe2a 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkerFactoryRegistry.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkerFactoryRegistry.java @@ -20,18 +20,19 @@ package io.temporal.internal.client; -import io.temporal.worker.WorkerFactory; +import io.temporal.worker.BaseWorkerFactory; import java.util.ArrayList; import java.util.Collections; import java.util.concurrent.CopyOnWriteArrayList; public class WorkerFactoryRegistry { - private final CopyOnWriteArrayList workerFactories = new CopyOnWriteArrayList<>(); + private final CopyOnWriteArrayList workerFactories = + new CopyOnWriteArrayList<>(); - public Iterable workerFactoriesRandomOrder() { + public Iterable workerFactoriesRandomOrder() { int count = workerFactories.size(); if (count > 1) { - ArrayList result = new ArrayList<>(workerFactories); + ArrayList result = new ArrayList<>(workerFactories); Collections.shuffle(result); return result; } else { @@ -39,11 +40,11 @@ public Iterable workerFactoriesRandomOrder() { } } - public void register(WorkerFactory workerFactory) { + public void register(BaseWorkerFactory workerFactory) { workerFactories.addIfAbsent(workerFactory); } - public void deregister(WorkerFactory workerFactory) { + public void deregister(BaseWorkerFactory workerFactory) { workerFactories.remove(workerFactory); } } diff --git a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java index b8349826e..89009671a 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/client/WorkflowClientInternal.java @@ -22,7 +22,6 @@ import io.temporal.client.WorkflowClient; import io.temporal.worker.BaseWorkerFactory; -import io.temporal.worker.WorkerFactory; /** * From OOP point of view, there is no reason for this interface not to extend {@link diff --git a/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java b/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java index 5f4371926..700eb81a0 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/sync/POJOWorkflowImplementationFactory.java @@ -41,7 +41,6 @@ import io.temporal.internal.common.env.ReflectionUtils; import io.temporal.internal.replay.ReplayWorkflow; import io.temporal.internal.replay.ReplayWorkflowFactory; -import io.temporal.internal.worker.SingleWorkerOptions; import io.temporal.internal.worker.WorkflowExecutionException; import io.temporal.internal.worker.WorkflowExecutorCache; import io.temporal.payload.context.WorkflowSerializationContext; @@ -60,7 +59,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import javax.annotation.Nonnull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java index 79bcdee58..9058a2fd6 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/worker/SyncWorkflowWorker.java @@ -30,15 +30,12 @@ import io.temporal.internal.activity.LocalActivityExecutionContextFactoryImpl; import io.temporal.internal.replay.ReplayWorkflowFactory; import io.temporal.internal.replay.ReplayWorkflowTaskHandler; -import io.temporal.internal.sync.POJOWorkflowImplementationFactory; -import io.temporal.internal.sync.WorkflowThreadExecutor; import io.temporal.serviceclient.WorkflowServiceStubs; import io.temporal.worker.WorkflowImplementationOptions; import io.temporal.worker.WorkflowTaskDispatchHandle; import io.temporal.workflow.Functions.Func; import java.lang.reflect.Type; import java.time.Duration; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.*; import javax.annotation.Nonnull; diff --git a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java index 7d2d67d50..98c87c9f5 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java @@ -13,16 +13,15 @@ import io.temporal.internal.worker.WorkflowExecutorCache; import io.temporal.internal.worker.WorkflowRunLockManager; import io.temporal.serviceclient.MetricsTag; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.util.HashMap; import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class BaseWorkerFactory { private static final Logger log = LoggerFactory.getLogger(WorkerFactory.class); diff --git a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java index dbda35ee9..9fbdbd2d9 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java @@ -32,9 +32,7 @@ import io.temporal.common.converter.DataConverter; import io.temporal.failure.TemporalFailure; import io.temporal.internal.replay.ReplayWorkflowFactory; -import io.temporal.internal.sync.POJOWorkflowImplementationFactory; import io.temporal.internal.sync.WorkflowInternal; -import io.temporal.internal.sync.WorkflowThreadExecutor; import io.temporal.internal.worker.*; import io.temporal.internal.worker.SyncActivityWorker; import io.temporal.internal.worker.SyncWorkflowWorker; diff --git a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java index 1c71cc557..689dd32aa 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java @@ -26,7 +26,6 @@ import io.temporal.internal.sync.POJOWorkflowImplementationFactory; import io.temporal.internal.sync.WorkflowThreadExecutor; import io.temporal.internal.worker.WorkflowExecutorCache; - import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; From a40e5698526a4ded98ab55be9cc0101012212b0c Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Mon, 19 Jun 2023 17:51:42 -0700 Subject: [PATCH 4/6] Fixed options initialization bug introduced during refactoring --- .../io/temporal/workflow/HelloActivity.kt | 181 ++++++++++++++++++ .../io/temporal/worker/BaseWorkerFactory.java | 1 + .../main/java/io/temporal/worker/Worker.java | 2 +- .../io/temporal/worker/WorkerFactory.java | 2 +- .../client/WorkerFactoryRegistryTest.java | 12 +- 5 files changed, 191 insertions(+), 7 deletions(-) create mode 100644 temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt new file mode 100644 index 000000000..cec6d3917 --- /dev/null +++ b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt @@ -0,0 +1,181 @@ +/* + * Copyright (c) 2020 Temporal Technologies, Inc. All Rights Reserved + * + * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"). You may not + * use this file except in compliance with the License. A copy of the License is + * located at + * + * http://aws.amazon.com/apache2.0 + * + * or in the "license" file accompanying this file. This file is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ +package io.temporal.workflow + +import io.temporal.activity.ActivityInterface +import io.temporal.activity.ActivityMethod +import io.temporal.activity.ActivityOptions +import io.temporal.client.WorkflowClient +import io.temporal.client.WorkflowOptions +import io.temporal.serviceclient.WorkflowServiceStubs +import io.temporal.worker.WorkerFactory +import io.temporal.workflow.HelloActivity.GreetingActivitiesImpl +import org.slf4j.LoggerFactory +import java.time.Duration + +/** Sample Temporal Workflow Definition that executes a single Activity. */ +object HelloActivity { + // Define the task queue name + const val TASK_QUEUE = "HelloActivityTaskQueue" + + // Define our workflow unique id + const val WORKFLOW_ID = "HelloActivityWorkflow" + + /** + * With our Workflow and Activities defined, we can now start execution. The main method starts + * the worker and then the workflow. + */ + @JvmStatic + fun main(args: Array) { + // Get a Workflow service stub. + val service = WorkflowServiceStubs.newLocalServiceStubs() + + /* + * Get a Workflow service client which can be used to start, Signal, and Query Workflow Executions. + */ + val client = WorkflowClient.newInstance(service) + + /* + * Define the workflow factory. It is used to create workflow workers for a specific task queue. + */ + val factory = WorkerFactory.newInstance(client) + + /* + * Define the workflow worker. Workflow workers listen to a defined task queue and process + * workflows and activities. + */ + val worker = factory.newWorker(TASK_QUEUE) + + /* + * Register our workflow implementation with the worker. + * Workflow implementations must be known to the worker at runtime in + * order to dispatch workflow tasks. + */worker.registerWorkflowImplementationTypes(GreetingWorkflowImpl::class.java) + /** + * Register our Activity Types with the Worker. Since Activities are stateless and thread-safe, + * the Activity Type is a shared instance. + */ + worker.registerActivitiesImplementations(GreetingActivitiesImpl()) + + /* + * Start all the workers registered for a specific task queue. + * The started workers then start polling for workflows and activities. + */factory.start() + + // Create the workflow client stub. It is used to start our workflow execution. + val workflow = client.newWorkflowStub( + GreetingWorkflow::class.java, + WorkflowOptions.newBuilder() + .setWorkflowId(WORKFLOW_ID) + .setTaskQueue(TASK_QUEUE) + .build() + ) + + /* + * Execute our workflow and wait for it to complete. The call to our getGreeting method is + * synchronous. + * + * See {@link io.temporal.samples.hello.HelloSignal} for an example of starting workflow + * without waiting synchronously for its result. + */ + val greeting = workflow.getGreeting("World") + + // Display workflow execution results + println(greeting) + System.exit(0) + } + + /** + * The Workflow Definition's Interface must contain one method annotated with @WorkflowMethod. + * + * + * Workflow Definitions should not contain any heavyweight computations, non-deterministic + * code, network calls, database operations, etc. Those things should be handled by the + * Activities. + * + * @see io.temporal.workflow.WorkflowInterface + * + * @see io.temporal.workflow.WorkflowMethod + */ + @WorkflowInterface + interface GreetingWorkflow { + /** + * This is the method that is executed when the Workflow Execution is started. The Workflow + * Execution completes when this method finishes execution. + */ + @WorkflowMethod + fun getGreeting(name: String?): String + } + + /** + * This is the Activity Definition's Interface. Activities are building blocks of any Temporal + * Workflow and contain any business logic that could perform long running computation, network + * calls, etc. + * + * + * Annotating Activity Definition methods with @ActivityMethod is optional. + * + * @see io.temporal.activity.ActivityInterface + * + * @see io.temporal.activity.ActivityMethod + */ + @ActivityInterface + interface GreetingActivities { + // Define your activity method which can be called during workflow execution + @ActivityMethod(name = "greet") + fun composeGreeting(greeting: String?, name: String?): String + } + + // Define the workflow implementation which implements our getGreeting workflow method. + class GreetingWorkflowImpl : GreetingWorkflow { + /** + * Define the GreetingActivities stub. Activity stubs are proxies for activity invocations that + * are executed outside of the workflow thread on the activity worker, that can be on a + * different host. Temporal is going to dispatch the activity results back to the workflow and + * unblock the stub as soon as activity is completed on the activity worker. + * + * + * In the [ActivityOptions] definition the "setStartToCloseTimeout" option sets the + * overall timeout that our workflow is willing to wait for activity to complete. For this + * example it is set to 2 seconds. + */ + private val activities = Workflow.newActivityStub( + GreetingActivities::class.java, + ActivityOptions.newBuilder().setStartToCloseTimeout(Duration.ofSeconds(2)).build() + ) + + override fun getGreeting(name: String?): String { + // This is a blocking call that returns only after the activity has completed. + return activities.composeGreeting("Hello", name) + } + } + + /** Simple activity implementation, that concatenates two strings. */ + internal class GreetingActivitiesImpl : GreetingActivities { + override fun composeGreeting(greeting: String?, name: String?): String { + log.info("Composing greeting...") +// throw ApplicationFailure.newNonRetryableFailure("simulated", "illegal-argument") + return greeting + " " + name + "!" + } + + companion object { + private val log = LoggerFactory.getLogger(GreetingActivitiesImpl::class.java) + } + } +} diff --git a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java index 98c87c9f5..eec6e155f 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java @@ -85,6 +85,7 @@ public synchronized Worker newWorker(String taskQueue, WorkerOptions workerOptio state == State.Initial, String.format(statusErrorMessage, "create new worker", state.name(), State.Initial.name())); + workerOptions = WorkerOptions.newBuilder(workerOptions).validateAndBuildWithDefaults(); // Only one worker can exist for a task queue Worker existingWorker = workers.get(taskQueue); if (existingWorker == null) { diff --git a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java index 9fbdbd2d9..aed055ff1 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java @@ -93,7 +93,7 @@ public final class Worker { Preconditions.checkArgument( !Strings.isNullOrEmpty(taskQueue), "taskQueue should not be an empty string"); this.taskQueue = taskQueue; - this.options = WorkerOptions.newBuilder(options).validateAndBuildWithDefaults(); + this.options = options; WorkflowServiceStubs service = client.getWorkflowServiceStubs(); WorkflowClientOptions clientOptions = client.getOptions(); String namespace = clientOptions.getNamespace(); diff --git a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java index 689dd32aa..53aae1adc 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/WorkerFactory.java @@ -63,7 +63,7 @@ private WorkerFactory(WorkflowClient workflowClient, WorkerFactoryOptions factor this.workflowThreadPool = new ThreadPoolExecutor( 0, - factoryOptions.getMaxWorkflowThreadCount(), + this.factoryOptions.getMaxWorkflowThreadCount(), 1, TimeUnit.MINUTES, new SynchronousQueue<>()); diff --git a/temporal-sdk/src/test/java/io/temporal/internal/client/WorkerFactoryRegistryTest.java b/temporal-sdk/src/test/java/io/temporal/internal/client/WorkerFactoryRegistryTest.java index f2c9470dd..abe84b5e5 100644 --- a/temporal-sdk/src/test/java/io/temporal/internal/client/WorkerFactoryRegistryTest.java +++ b/temporal-sdk/src/test/java/io/temporal/internal/client/WorkerFactoryRegistryTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock; import io.temporal.client.WorkflowClient; +import io.temporal.worker.BaseWorkerFactory; import io.temporal.worker.WorkerFactory; import java.util.Iterator; import org.junit.Test; @@ -53,11 +54,12 @@ public void testRandomOrder() { int thirdFactoryFirst = 0; for (int i = 0; i < TOTAL_COUNT; i++) { - Iterable workerFactories = workerFactoryRegistry.workerFactoriesRandomOrder(); - Iterator iterator = workerFactories.iterator(); - WorkerFactory first = iterator.next(); - WorkerFactory second = iterator.next(); - WorkerFactory third = iterator.next(); + Iterable workerFactories = + workerFactoryRegistry.workerFactoriesRandomOrder(); + Iterator iterator = workerFactories.iterator(); + BaseWorkerFactory first = iterator.next(); + BaseWorkerFactory second = iterator.next(); + BaseWorkerFactory third = iterator.next(); assertFalse(iterator.hasNext()); assertNotEquals(first, second); From 4ed872355bff4154521dc6e757982833b0ccc72e Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Mon, 19 Jun 2023 19:57:22 -0700 Subject: [PATCH 5/6] HelloKotlinWorkflow works --- ...otKotlinWorkflowInboundCallsInterceptor.kt | 1 + .../async/DynamicKotlinWorkflowDefinition.kt | 3 +- .../internal/async/KotlinActivityStub.kt | 1 + .../internal/async/KotlinActivityStubImpl.kt | 1 + .../temporal/internal/async/KotlinWorkflow.kt | 22 ++++- .../internal/async/KotlinWorkflowContext.kt | 50 ++++++++--- .../async/KotlinWorkflowDefinition.kt | 20 +++++ .../async/KotlinWorkflowExecutionHandler.kt | 3 +- .../KotlinWorkflowImplementationFactory.kt | 55 ++++++------ .../internal/async/KotlinWorkflowInfoImpl.kt | 1 + .../internal/async/KotlinWorkflowInternal.kt | 85 +++++++++---------- .../async/TemporalCoroutineDispatcher.kt | 20 +++++ .../interceptors/KotlinWorkerInterceptor.kt | 1 + .../interceptors/WorkerInterceptorBase.kt | 1 + .../WorkflowInboundCallsInterceptor.kt | 1 + .../WorkflowInboundCallsInterceptorBase.kt | 1 + .../WorkflowOutboundCallsInterceptor.kt | 1 + .../WorkflowOutboundCallsInterceptorBase.kt | 1 + .../kotlin/workflow/KotlinDynamicWorkflow.kt | 1 + .../io/temporal/worker/KotlinWorkerFactory.kt | 20 +++++ .../worker/KotlinWorkerFactoryOptions.kt | 1 + .../io/temporal/workflow/KotlinWorkflow.kt | 22 ++++- .../temporal/workflow/KotlinWorkflowInfo.kt | 1 + .../workflow/CoroutineWorkflowTest.kt | 11 --- ...elloActivity.kt => HelloKotlinWorkflow.kt} | 77 ++++++++--------- .../io/temporal/worker/BaseWorkerFactory.java | 20 +++++ 26 files changed, 280 insertions(+), 141 deletions(-) delete mode 100644 temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt rename temporal-kotlin/src/test/kotlin/io/temporal/workflow/{HelloActivity.kt => HelloKotlinWorkflow.kt} (64%) diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt index bc70e1ce5..126330360 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/BaseRootKotlinWorkflowInboundCallsInterceptor.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt index 22bb34cbb..0d2b8d336 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/DynamicKotlinWorkflowDefinition.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import io.temporal.api.common.v1.Payloads @@ -39,7 +40,7 @@ internal class DynamicKotlinWorkflowDefinition( private var workflowInvoker: WorkflowInboundCallsInterceptor? = null override suspend fun initialize() { - val workflowContext: KotlinWorkflowContext = KotlinWorkflowInternal.rootWorkflowContext + val workflowContext: KotlinWorkflowContext = KotlinWorkflowInternal.getRootWorkflowContext() workflowInvoker = RootWorkflowInboundCallsInterceptor(workflowContext) for (workerInterceptor in workerInterceptors) { workflowInvoker = workerInterceptor.interceptWorkflow(workflowInvoker!!) diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt index d18c34cbd..9b9e36d00 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStub.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import java.lang.reflect.Type diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt index f61a3a602..1c5806e8b 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import io.temporal.activity.ActivityOptions diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt index 6e6456438..d629fc625 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.internal.async import io.temporal.api.common.v1.Payloads @@ -101,7 +121,7 @@ class KotlinWorkflow( return false } dispatcher.eventLoop(defaultDeadlockDetectionTimeout) - return dispatcher.isDone() || executionHandler!!.isDone // Do not wait for all other threads. + return /*dispatcher.isDone() ||*/ executionHandler!!.isDone // Do not wait for all other threads. } override fun getOutput(): Optional { diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt index a81e98891..7877ec30b 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.internal.async import com.uber.m3.tally.Scope @@ -57,6 +77,8 @@ class KotlinWorkflowContext( private var activityOptionsMap: Map? = null private var localActivityOptionsMap: Map? = null + private var replayContext: ReplayWorkflowContext? = null + init { if (workflowImplementationOptions != null) { defaultActivityOptions = workflowImplementationOptions!!.defaultActivityOptions @@ -88,7 +110,7 @@ class KotlinWorkflowContext( replayContext = context } - override fun getReplayContext(): ReplayWorkflowContext { + override fun getReplayContext(): ReplayWorkflowContext? { return replayContext } @@ -142,7 +164,7 @@ class KotlinWorkflowContext( override fun getLastCompletionResult(resultClass: Class, resultType: Type): R? { return dataConverter.fromPayloads( 0, - Optional.ofNullable(replayContext.lastCompletionResult), + Optional.ofNullable(replayContext!!.lastCompletionResult), resultClass, resultType ) @@ -157,7 +179,7 @@ class KotlinWorkflowContext( return HashMap() } - val headerData: Map = HashMap(replayContext.header) + val headerData: Map = HashMap(replayContext!!.header) val contextData: MutableMap = HashMap() for (propagator in contextPropagators) { contextData[propagator.name] = propagator.deserializeContext(headerData) @@ -168,13 +190,13 @@ class KotlinWorkflowContext( override suspend fun executeActivity(input: WorkflowOutboundCallsInterceptor.ActivityInput): WorkflowOutboundCallsInterceptor.ActivityOutput { val serializationContext = ActivitySerializationContext( - replayContext.namespace, - replayContext.workflowId, - replayContext.workflowType.name, + replayContext!!.namespace, + replayContext!!.workflowId, + replayContext!!.workflowType.name, input.activityName, // input.getOptions().getTaskQueue() may be not specified, workflow task queue is used // by the Server in this case - if (input.options.taskQueue != null) input.options.taskQueue else replayContext.taskQueue, + if (input.options.taskQueue != null) input.options.taskQueue else replayContext!!.taskQueue, false ) val dataConverterWithActivityContext = dataConverter.withContext(serializationContext) @@ -205,7 +227,7 @@ class KotlinWorkflowContext( TODO("Not yet implemented") } - override fun newRandom(): Random = replayContext.newRandom() + override fun newRandom(): Random = replayContext!!.newRandom() override suspend fun signalExternalWorkflow(input: WorkflowOutboundCallsInterceptor.SignalExternalInput): WorkflowOutboundCallsInterceptor.SignalExternalOutput { TODO("Not yet implemented") @@ -290,7 +312,7 @@ class KotlinWorkflowContext( } val metricScope: Scope - get() = replayContext.metricsScope + get() = replayContext!!.metricsScope @OptIn(ExperimentalCoroutinesApi::class) private suspend fun executeActivityOnce( @@ -303,7 +325,7 @@ class KotlinWorkflowContext( return suspendCancellableCoroutine { continuation -> var activityId: String? = null - val activityOutput = replayContext.scheduleActivityTask( + val activityOutput = replayContext!!.scheduleActivityTask( params ) { output: Optional, failure: Failure? -> if (failure == null) { @@ -329,7 +351,7 @@ class KotlinWorkflowContext( ): ExecuteActivityParameters { var taskQueue = options.taskQueue if (taskQueue == null) { - taskQueue = replayContext.taskQueue + taskQueue = replayContext!!.taskQueue } val attributes = ScheduleActivityTaskCommandAttributes.newBuilder() .setActivityType(ActivityType.newBuilder().setName(name)) @@ -345,7 +367,7 @@ class KotlinWorkflowContext( ) .setHeartbeatTimeout(ProtobufTimeUtils.toProtoDuration(options.heartbeatTimeout)) .setRequestEagerExecution( - !options.isEagerExecutionDisabled && (taskQueue == replayContext.taskQueue) + !options.isEagerExecutionDisabled && (taskQueue == replayContext!!.taskQueue) ) input.ifPresent { value: Payloads? -> attributes.input = value @@ -366,7 +388,7 @@ class KotlinWorkflowContext( if (options.versioningIntent != null) { attributes.useCompatibleVersion = options .versioningIntent - .determineUseCompatibleFlag(replayContext.taskQueue == options.taskQueue) + .determineUseCompatibleFlag(replayContext!!.taskQueue == options.taskQueue) } return ExecuteActivityParameters(attributes, options.cancellationType) } @@ -396,7 +418,7 @@ class KotlinWorkflowContext( * thread and should be replaced with another specific implementation during initialization stage * `workflow.initialize()` performed inside the workflow root thread. * - * @see SyncWorkflow.start + * @see KotlinWorkflow.start */ private class InitialWorkflowInboundCallsInterceptor(workflowContext: KotlinWorkflowContext) : BaseRootKotlinWorkflowInboundCallsInterceptor(workflowContext) { diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt index b0a083cb1..c7e5244fa 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.internal.async import io.temporal.api.common.v1.Payloads diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt index 0f14809dd..0b689bb78 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import io.temporal.api.common.v1.Payloads @@ -106,7 +107,7 @@ internal class KotlinWorkflowExecutionHandler( private fun throwAndFailWorkflowExecution(exception: Throwable) { val replayWorkflowContext = context.getReplayContext() - val fullReplayDirectQueryName = replayWorkflowContext.fullReplayDirectQueryName + val fullReplayDirectQueryName = replayWorkflowContext!!.fullReplayDirectQueryName val info = Workflow.getInfo() if (fullReplayDirectQueryName != null) { if (log.isDebugEnabled && diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt index 0aa81b62e..7d3fccebe 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt @@ -17,10 +17,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import com.google.common.base.Preconditions -import com.google.common.collect.ImmutableSet import io.temporal.api.common.v1.Payloads import io.temporal.api.common.v1.WorkflowExecution import io.temporal.api.common.v1.WorkflowType @@ -32,7 +32,6 @@ import io.temporal.common.metadata.POJOWorkflowImplMetadata import io.temporal.common.metadata.POJOWorkflowInterfaceMetadata import io.temporal.common.metadata.WorkflowMethodType import io.temporal.failure.CanceledFailure -import io.temporal.internal.common.env.ReflectionUtils import io.temporal.internal.replay.ReplayWorkflow import io.temporal.internal.replay.ReplayWorkflowFactory import io.temporal.internal.sync.WorkflowInternal @@ -54,6 +53,8 @@ import org.slf4j.LoggerFactory import java.lang.reflect.InvocationTargetException import java.lang.reflect.Method import java.util.* +import kotlin.reflect.full.callSuspend +import kotlin.reflect.jvm.kotlinFunction class KotlinWorkflowImplementationFactory( clientOptions: WorkflowClientOptions, @@ -281,9 +282,11 @@ class KotlinWorkflowImplementationFactory( // don't pass it down to other classes, it's a "cached" instance for internal usage only private val dataConverterWithWorkflowContext: DataConverter ) : KotlinWorkflowDefinition { + private var workflowInvoker: WorkflowInboundCallsInterceptor? = null + override suspend fun initialize() { - val workflowContext = KotlinWorkflowInternal.rootWorkflowContext + val workflowContext = KotlinWorkflowInternal.getRootWorkflowContext() workflowInvoker = RootWorkflowInboundCallsInterceptor(workflowContext) for (workerInterceptor in workerInterceptors) { workflowInvoker = workerInterceptor.interceptWorkflow(workflowInvoker!!) @@ -296,8 +299,9 @@ class KotlinWorkflowImplementationFactory( override suspend fun execute(header: Header?, input: Payloads?): Payloads? { val args = dataConverterWithWorkflowContext.fromPayloads( Optional.ofNullable(input), - workflowMethod.parameterTypes, - workflowMethod.genericParameterTypes + // TODO(maxim): Validate that the last element is coroutine continuation + workflowMethod.parameterTypes.dropLast(1).toTypedArray(), + workflowMethod.genericParameterTypes.dropLast(1).toTypedArray() ) Preconditions.checkNotNull(workflowInvoker, "initialize not called") val result = workflowInvoker!!.execute(WorkflowInboundCallsInterceptor.WorkflowInput(header, args)) @@ -322,7 +326,8 @@ class KotlinWorkflowImplementationFactory( override suspend fun execute(input: WorkflowInboundCallsInterceptor.WorkflowInput): WorkflowInboundCallsInterceptor.WorkflowOutput { return try { - val result = workflowMethod.invoke(workflow, *input.arguments) + val kMethod = workflowMethod.kotlinFunction + val result = kMethod!!.callSuspend(workflow, *input.arguments) WorkflowInboundCallsInterceptor.WorkflowOutput(result) } catch (e: IllegalAccessException) { throw CheckedExceptionWrapper.wrap(e) @@ -381,23 +386,25 @@ class KotlinWorkflowImplementationFactory( companion object { private val log = LoggerFactory.getLogger(KotlinWorkflowImplementationFactory::class.java) - val WORKFLOW_HANDLER_STACKTRACE_CUTOFF = ImmutableSet.builder() // POJO - .add( - ReflectionUtils.getMethodNameForStackTraceCutoff( - KotlinWorkflowImplementation::class.java, - "execute", - Header::class.java, - Optional::class.java - ) - ) // Dynamic - .add( - ReflectionUtils.getMethodNameForStackTraceCutoff( - DynamicKotlinWorkflowDefinition::class.java, - "execute", - Header::class.java, - Optional::class.java - ) - ) - .build() + + // TODO(maxim): See if this is needed for Kotlin + val WORKFLOW_HANDLER_STACKTRACE_CUTOFF = 0 // ImmutableSet.builder() // POJO +// .add( +// ReflectionUtils.getMethodNameForStackTraceCutoff( +// KotlinWorkflowImplementation::class.java, +// "execute", +// Header::class.java, +// Payloads::class.java +// ) +// ) // Dynamic +// .add( +// ReflectionUtils.getMethodNameForStackTraceCutoff( +// DynamicKotlinWorkflowDefinition::class.java, +// "execute", +// Header::class.java, +// Payloads::class.java +// ) +// ) +// .build() } } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt index 0e4b99e28..c10f49f09 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import io.temporal.api.common.v1.SearchAttributes diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt index aa43b3ac3..a9d99a878 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.internal.async import com.uber.m3.tally.Scope @@ -25,10 +26,9 @@ import io.temporal.activity.LocalActivityOptions import io.temporal.api.failure.v1.Failure import io.temporal.common.converter.DataConverter import io.temporal.common.metadata.POJOWorkflowInterfaceMetadata -import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor import io.temporal.workflow.KotlinWorkflowInfo import io.temporal.workflow.WorkflowMethod -import kotlinx.coroutines.GlobalScope.coroutineContext +import kotlinx.coroutines.currentCoroutineContext import java.util.* /** @@ -180,26 +180,26 @@ class KotlinWorkflowInternal { } /** Should be used to get current time instead of [System.currentTimeMillis] */ - fun currentTimeMillis(): Long { - return workflowOutboundInterceptor.currentTimeMillis() + suspend fun currentTimeMillis(): Long { + return getWorkflowOutboundCallsInterceptor().currentTimeMillis() } - fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { - rootWorkflowContext.defaultActivityOptions = activityOptions + suspend fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { + getRootWorkflowContext().defaultActivityOptions = activityOptions } - fun applyActivityOptions(activityTypeToOptions: Map) { - rootWorkflowContext.applyActivityOptions(activityTypeToOptions) + suspend fun applyActivityOptions(activityTypeToOptions: Map) { + getRootWorkflowContext().applyActivityOptions(activityTypeToOptions) } - fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { - rootWorkflowContext.defaultLocalActivityOptions = localActivityOptions + suspend fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { + getRootWorkflowContext().defaultLocalActivityOptions = localActivityOptions } - fun applyLocalActivityOptions( + suspend fun applyLocalActivityOptions( activityTypeToOptions: Map ) { - rootWorkflowContext.applyLocalActivityOptions(activityTypeToOptions) + getRootWorkflowContext().applyLocalActivityOptions(activityTypeToOptions) } // /** @@ -286,8 +286,8 @@ class KotlinWorkflowInternal { // return ActivityInvocationHandlerBase.newProxy(activityInterface, invocationHandler) // } - fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { - return KotlinActivityStubImpl(options, workflowOutboundInterceptor) + suspend fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + return KotlinActivityStubImpl(options, getWorkflowOutboundCallsInterceptor()) } // fun newUntypedLocalActivityStub(options: LocalActivityOptions?): KotlinActivityStub { @@ -445,11 +445,11 @@ class KotlinWorkflowInternal { // val isReplaying: Boolean // get() { // val thread = DeterministicRunnerImpl.currentThreadInternalIfPresent() -// return thread.isPresent && rootWorkflowContext.isReplaying() +// return thread.isPresent && getRootWorkflowContext().isReplaying() // } // // fun getMemo(key: String?, valueClass: Class?, genericType: Type?): T? { -// val memo = rootWorkflowContext.getReplayContext().getMemo(key) ?: return null +// val memo = getRootWorkflowContext().getReplayContext().getMemo(key) ?: return null // return dataConverter.fromPayload(memo, valueClass, genericType) // } // @@ -506,22 +506,16 @@ class KotlinWorkflowInternal { // } // } - val workflowInfo: KotlinWorkflowInfo - get() = KotlinWorkflowInfoImpl(rootWorkflowContext.getReplayContext()) + suspend fun getWorkflowInfo(): KotlinWorkflowInfo = KotlinWorkflowInfoImpl(getRootWorkflowContext().replayContext!!) - val metricsScope: Scope - get() = rootWorkflowContext.metricScope + suspend fun getMetricsScope(): Scope = getRootWorkflowContext().metricScope - fun randomUUID(): UUID { - return rootWorkflowContext.randomUUID() - } + suspend fun randomUUID(): UUID = getRootWorkflowContext().randomUUID() - fun newRandom(): Random { - return rootWorkflowContext.newRandom() - } + suspend fun newRandom(): Random = getRootWorkflowContext().newRandom() // private val isLoggingEnabledInReplay: Boolean -// private get() = rootWorkflowContext.isLoggingEnabledInReplay() +// private get() = getRootWorkflowContext().isLoggingEnabledInReplay() // fun getLogger(clazz: Class<*>?): Logger { // val logger = LoggerFactory.getLogger(clazz) // return ReplayAwareLogger( @@ -539,7 +533,7 @@ class KotlinWorkflowInternal { // } // fun getLastCompletionResult(resultClass: Class?, resultType: Type?): R? { -// return rootWorkflowContext.getLastCompletionResult(resultClass, resultType) +// return getRootWorkflowContext().getLastCompletionResult(resultClass, resultType) // } // // fun getSearchAttribute(name: String?): T? { @@ -555,21 +549,21 @@ class KotlinWorkflowInternal { // } // // fun getSearchAttributeValues(name: String?): List? { -// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes ?: return null +// val searchAttributes = getRootWorkflowContext().getReplayContext().searchAttributes ?: return null // val decoded = SearchAttributesUtil.decode(searchAttributes, name!!) // return if (decoded != null) Collections.unmodifiableList(decoded) else null // } // // val searchAttributes: Map> // get() { -// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes +// val searchAttributes = getRootWorkflowContext().getReplayContext().searchAttributes // ?: return emptyMap() // return Collections.unmodifiableMap(SearchAttributesUtil.decode(searchAttributes)) // } // // val typedSearchAttributes: SearchAttributes // get() { -// val searchAttributes = rootWorkflowContext.getReplayContext().searchAttributes +// val searchAttributes = getRootWorkflowContext().getReplayContext().searchAttributes // return SearchAttributesUtil.decodeTyped(searchAttributes) // } // @@ -583,8 +577,7 @@ class KotlinWorkflowInternal { // workflowOutboundInterceptor.upsertTypedSearchAttributes(*searchAttributeUpdates) // } - val dataConverter: DataConverter - get() = rootWorkflowContext.dataConverter + suspend fun getDataConverter(): DataConverter = getRootWorkflowContext().dataConverter /** * Name of the workflow type the interface defines. It is either the interface short name * or @@ -599,23 +592,23 @@ class KotlinWorkflowInternal { // Temporal Failure Values are additional user payload and serialized using user data // converter - val previousRunFailure: Optional - get() = Optional.ofNullable(rootWorkflowContext.getReplayContext().previousRunFailure) // Temporal Failure Values are additional user payload and serialized using user data - // converter + suspend fun getPreviousRunFailure(): Optional { + // Temporal Failure Values are additional user payload and serialized using user data converter + val dataConverter = getDataConverter() + return Optional.ofNullable(getRootWorkflowContext().replayContext!!.previousRunFailure) .map { f: Failure? -> - dataConverter.failureToException( - f!! - ) + dataConverter.failureToException(f!!) } + } - private val workflowOutboundInterceptor: WorkflowOutboundCallsInterceptor - private get() = rootWorkflowContext.getWorkflowOutboundInterceptor() + suspend fun getWorkflowOutboundCallsInterceptor() = getRootWorkflowContext().getWorkflowOutboundInterceptor() - val rootWorkflowContext: KotlinWorkflowContext - get() { - val temporalCoroutineContext = coroutineContext[TemporalCoroutineContext] - ?: throw Error("Called from non workflow thread or coroutine") - return temporalCoroutineContext.workflowContext + suspend fun getRootWorkflowContext(): KotlinWorkflowContext { + val temporalCoroutineContext = currentCoroutineContext()[TemporalCoroutineContext] + if (temporalCoroutineContext == null) { + throw Error("Called from non workflow thread or coroutine") } + return temporalCoroutineContext.workflowContext + } } } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt index d18bdc948..11d5db5a4 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.internal.async import kotlinx.coroutines.CancellableContinuation diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt index 0a874863b..87b63df88 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors import io.temporal.common.Experimental diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt index 0a05dfbff..e381bf1fb 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors import io.temporal.common.interceptors.ActivityInboundCallsInterceptor diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt index a5874de7a..fec0f5996 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptor.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors import io.temporal.common.Experimental diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt index a4d849d5c..22b4f1254 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors /** Convenience base class for WorkflowInboundCallsInterceptor implementations. */ diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt index e9daf8c70..71e185c78 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors import io.temporal.activity.ActivityOptions diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt index 6d8f5705b..24dfe788c 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.interceptors import io.temporal.common.SearchAttributeUpdate diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt index 697f107ab..5061125a1 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.kotlin.workflow import io.temporal.common.converter.EncodedValues diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt index 2756cf172..80ab007d3 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.worker import io.temporal.client.WorkflowClient diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt index 66114fb48..8befbd95e 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.worker import com.google.common.base.Preconditions diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt index 67384f435..f803a4b88 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.workflow import io.temporal.activity.ActivityOptions @@ -11,7 +31,7 @@ class KotlinWorkflow { * * @param options specify the activity invocation parameters. */ - fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + suspend fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { return KotlinWorkflowInternal.newUntypedActivityStub(options) } } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt index f57846834..b3e4ae938 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt @@ -17,6 +17,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.temporal.workflow import io.temporal.api.common.v1.SearchAttributes diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt deleted file mode 100644 index 7e016f647..000000000 --- a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/CoroutineWorkflowTest.kt +++ /dev/null @@ -1,11 +0,0 @@ -package io.temporal.workflow - -import org.junit.Assert.assertTrue -import org.junit.Test - -public class CoroutineWorkflowTest { - @Test - fun asyncLambdaTest() { - assertTrue(false) - } -} diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt similarity index 64% rename from temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt rename to temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt index cec6d3917..7f3db83f6 100644 --- a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloActivity.kt +++ b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt @@ -1,21 +1,23 @@ /* - * Copyright (c) 2020 Temporal Technologies, Inc. All Rights Reserved + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. * - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Modifications copyright (C) 2017 Uber Technologies, Inc. + * Modifications copyright (C) 2017 Uber Technologies, Inc. * - * Licensed under the Apache License, Version 2.0 (the "License"). You may not - * use this file except in compliance with the License. A copy of the License is - * located at + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at * - * http://aws.amazon.com/apache2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * or in the "license" file accompanying this file. This file is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ + package io.temporal.workflow import io.temporal.activity.ActivityInterface @@ -23,19 +25,21 @@ import io.temporal.activity.ActivityMethod import io.temporal.activity.ActivityOptions import io.temporal.client.WorkflowClient import io.temporal.client.WorkflowOptions +import io.temporal.client.getResult import io.temporal.serviceclient.WorkflowServiceStubs -import io.temporal.worker.WorkerFactory -import io.temporal.workflow.HelloActivity.GreetingActivitiesImpl +import io.temporal.worker.KotlinWorkerFactory +import kotlinx.coroutines.async +import kotlinx.coroutines.coroutineScope import org.slf4j.LoggerFactory import java.time.Duration -/** Sample Temporal Workflow Definition that executes a single Activity. */ -object HelloActivity { +/** Sample Kotlin Temporal Workflow Definition that executes a couple activities in parallel using coroutines */ +object HelloKotlinWorkflow { // Define the task queue name - const val TASK_QUEUE = "HelloActivityTaskQueue" + const val TASK_QUEUE = "HelloKotlinTaskQueue" // Define our workflow unique id - const val WORKFLOW_ID = "HelloActivityWorkflow" + const val WORKFLOW_ID = "HelloKotlinWorkflow" /** * With our Workflow and Activities defined, we can now start execution. The main method starts @@ -54,7 +58,7 @@ object HelloActivity { /* * Define the workflow factory. It is used to create workflow workers for a specific task queue. */ - val factory = WorkerFactory.newInstance(client) + val factory = KotlinWorkerFactory(client, null) /* * Define the workflow worker. Workflow workers listen to a defined task queue and process @@ -79,8 +83,8 @@ object HelloActivity { */factory.start() // Create the workflow client stub. It is used to start our workflow execution. - val workflow = client.newWorkflowStub( - GreetingWorkflow::class.java, + val workflow = client.newUntypedWorkflowStub( + "GreetingWorkflow", WorkflowOptions.newBuilder() .setWorkflowId(WORKFLOW_ID) .setTaskQueue(TASK_QUEUE) @@ -94,8 +98,8 @@ object HelloActivity { * See {@link io.temporal.samples.hello.HelloSignal} for an example of starting workflow * without waiting synchronously for its result. */ - val greeting = workflow.getGreeting("World") - + workflow.start("Kotlin") + val greeting = workflow.getResult() // Display workflow execution results println(greeting) System.exit(0) @@ -120,7 +124,7 @@ object HelloActivity { * Execution completes when this method finishes execution. */ @WorkflowMethod - fun getGreeting(name: String?): String + suspend fun getGreeting(name: String?): String } /** @@ -144,25 +148,15 @@ object HelloActivity { // Define the workflow implementation which implements our getGreeting workflow method. class GreetingWorkflowImpl : GreetingWorkflow { - /** - * Define the GreetingActivities stub. Activity stubs are proxies for activity invocations that - * are executed outside of the workflow thread on the activity worker, that can be on a - * different host. Temporal is going to dispatch the activity results back to the workflow and - * unblock the stub as soon as activity is completed on the activity worker. - * - * - * In the [ActivityOptions] definition the "setStartToCloseTimeout" option sets the - * overall timeout that our workflow is willing to wait for activity to complete. For this - * example it is set to 2 seconds. - */ - private val activities = Workflow.newActivityStub( - GreetingActivities::class.java, - ActivityOptions.newBuilder().setStartToCloseTimeout(Duration.ofSeconds(2)).build() - ) - override fun getGreeting(name: String?): String { - // This is a blocking call that returns only after the activity has completed. - return activities.composeGreeting("Hello", name) + override suspend fun getGreeting(name: String?): String = coroutineScope { + val activities = KotlinWorkflow.newUntypedActivityStub( + ActivityOptions.newBuilder().setStartToCloseTimeout(Duration.ofSeconds(2)).build() + ) + + val result1 = async { activities.execute("greet", String::class.java, "Hello", name!!) } + val result2 = async { activities.execute("greet", String::class.java, "Bye", name!!) } + return@coroutineScope result1.await()!! + "\n" + result2.await()!! } } @@ -170,7 +164,6 @@ object HelloActivity { internal class GreetingActivitiesImpl : GreetingActivities { override fun composeGreeting(greeting: String?, name: String?): String { log.info("Composing greeting...") -// throw ApplicationFailure.newNonRetryableFailure("simulated", "illegal-argument") return greeting + " " + name + "!" } diff --git a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java index eec6e155f..7205ab11a 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java @@ -1,3 +1,23 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.temporal.worker; import com.google.common.annotations.VisibleForTesting; From a61af8bf23cd2c604cc1c17d25a52f0647262570 Mon Sep 17 00:00:00 2001 From: Maxim Fateev Date: Tue, 20 Jun 2023 08:05:40 -0700 Subject: [PATCH 6/6] Added support for delay --- .../temporal/internal/async/KotlinWorkflow.kt | 2 +- .../internal/async/KotlinWorkflowContext.kt | 4 -- .../async/TemporalCoroutineDispatcher.kt | 56 ++++--------------- .../WorkflowOutboundCallsInterceptor.kt | 1 - .../WorkflowOutboundCallsInterceptorBase.kt | 4 -- .../temporal/workflow/HelloKotlinWorkflow.kt | 6 +- 6 files changed, 18 insertions(+), 55 deletions(-) diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt index d629fc625..b18386c14 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.kt @@ -65,7 +65,7 @@ class KotlinWorkflow( contextPropagators ) - private val dispatcher = TemporalCoroutineDispatcher() + private val dispatcher = TemporalCoroutineDispatcher(workflowContext) private val coroutineDispatcher = TemporalCallbackCoroutineDispatcher(dispatcher) private val scope = TemporalScope(workflowContext) diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt index 7877ec30b..9d14d915c 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt @@ -237,10 +237,6 @@ class KotlinWorkflowContext( TODO("Not yet implemented") } - override suspend fun sleep(duration: Duration) { - TODO("Not yet implemented") - } - override suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean { TODO("Not yet implemented") } diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt index 11d5db5a4..f788051c3 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt @@ -25,20 +25,18 @@ import kotlinx.coroutines.CoroutineDispatcher import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.Delay import kotlinx.coroutines.InternalCoroutinesApi +import java.lang.RuntimeException +import java.time.Duration import java.util.* -import java.util.concurrent.DelayQueue -import java.util.concurrent.Delayed -import java.util.concurrent.TimeUnit import kotlin.coroutines.AbstractCoroutineContextElement import kotlin.coroutines.CoroutineContext @Suppress("UNUSED_PARAMETER") @OptIn(InternalCoroutinesApi::class) -class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { +class TemporalCoroutineDispatcher(val workflowContext: KotlinWorkflowContext) : CoroutineDispatcher(), Delay { private val queue: java.util.Queue = LinkedList() private val callbackQueue: Queue = LinkedList() - private val delayQueue: DelayQueue = DelayQueue() override fun dispatch(context: CoroutineContext, block: Runnable) { queue.add(block) @@ -49,13 +47,7 @@ class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { } // TODO: deadlock detector - fun eventLoop(defaultDeadlockDetectionTimeout: Long): Boolean { -// println("eventLoop begin") - if (isDone()) { - println("eventLoop completed") - return false - } - + fun eventLoop(defaultDeadlockDetectionTimeout: Long) { while (callbackQueue.isNotEmpty()) { val block = callbackQueue.poll() block.run() @@ -65,39 +57,16 @@ class TemporalCoroutineDispatcher : CoroutineDispatcher(), Delay { val block = queue.poll() block.run() } - - while (true) { -// println("delayedContinuation while begin count=" + delayQueue.size) - - val delayedContinuation = delayQueue.poll() ?: break - println("delayedContinuation returned") - with(delayedContinuation.continuation) { resumeUndispatched(Unit) } - } - - return true } - fun isDone() = queue.isEmpty() && callbackQueue.isEmpty() && delayQueue.isEmpty() - override fun scheduleResumeAfterDelay(timeMillis: Long, continuation: CancellableContinuation) { - println("scheduleResumeAfterDelay delay=$timeMillis") - delayQueue.add(DelayedContinuation(timeMillis, continuation)) - } - - private class DelayedContinuation( - private val delayTime: Long, - val continuation: CancellableContinuation - ) : Delayed { - private val startTime = System.currentTimeMillis() + delayTime - - override fun compareTo(other: Delayed): Int { - return (getDelay(TimeUnit.MILLISECONDS) - other.getDelay(TimeUnit.MILLISECONDS)).toInt() - } - - override fun getDelay(unit: TimeUnit): Long { - val diff = startTime - System.currentTimeMillis() - return unit.convert(diff, TimeUnit.MILLISECONDS) - } + val cancellationHandler = + workflowContext.replayContext!!.newTimer(Duration.ofMillis(timeMillis)) { cancellationRequest -> + cancellationRequest ?: callbackQueue.add { + with(continuation) { resumeUndispatched(Unit) } + } + } + continuation.invokeOnCancellation { cause -> cancellationHandler.apply(cause as RuntimeException?) } } } @@ -118,11 +87,10 @@ class TemporalCallbackCoroutineDispatcher(val dispatcher: TemporalCoroutineDispa } internal class TemporalScope(private val workflowContext: KotlinWorkflowContext) : CoroutineScope { - // TODO: Add argument to the Temporal context. override val coroutineContext: CoroutineContext = TemporalCoroutineContext(workflowContext) // CoroutineScope is used intentionally for user-friendly representation - override fun toString(): String = "CoroutineScope(coroutineContext=$coroutineContext)" + override fun toString(): String = "TemporalScope(coroutineContext=$coroutineContext)" } class TemporalCoroutineContext(val workflowContext: KotlinWorkflowContext) : diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt index 71e185c78..19d3c288e 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -155,7 +155,6 @@ interface WorkflowOutboundCallsInterceptor { fun cancelWorkflow(input: CancelWorkflowInput): CancelWorkflowOutput // TODO: Consider removing sleep and keep only built in delay - suspend fun sleep(duration: Duration) suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean suspend fun await(reason: String?, unblockCondition: Supplier) fun sideEffect(resultClass: Class, resultType: Type, func: Func): R? diff --git a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt index 24dfe788c..c689555c0 100644 --- a/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptorBase.kt @@ -55,10 +55,6 @@ class WorkflowOutboundCallsInterceptorBase(private val next: WorkflowOutboundCal return next.cancelWorkflow(input) } - override suspend fun sleep(duration: Duration) { - return next.sleep(duration) - } - override suspend fun await(timeout: Duration, reason: String?, unblockCondition: Supplier): Boolean { return next.await(timeout, reason, unblockCondition) } diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt index 7f3db83f6..6877af641 100644 --- a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt +++ b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt @@ -30,6 +30,7 @@ import io.temporal.serviceclient.WorkflowServiceStubs import io.temporal.worker.KotlinWorkerFactory import kotlinx.coroutines.async import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.delay import org.slf4j.LoggerFactory import java.time.Duration @@ -155,7 +156,10 @@ object HelloKotlinWorkflow { ) val result1 = async { activities.execute("greet", String::class.java, "Hello", name!!) } - val result2 = async { activities.execute("greet", String::class.java, "Bye", name!!) } + val result2 = async { + delay(1000) + activities.execute("greet", String::class.java, "Bye", name!!) + } return@coroutineScope result1.await()!! + "\n" + result2.await()!! } }