diff --git a/build.gradle b/build.gradle index c39cd786cc..8c62322c29 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 7b73592547..4c063ab3e4 100644 --- a/temporal-kotlin/build.gradle +++ b/temporal-kotlin/build.gradle @@ -21,7 +21,8 @@ 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" 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 0000000000..126330360f --- /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) + } +} 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 0000000000..0d2b8d3362 --- /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.KotlinWorkerInterceptor +import io.temporal.kotlin.interceptors.WorkflowInboundCallsInterceptor +import io.temporal.kotlin.interceptors.WorkflowOutboundCallsInterceptor +import io.temporal.kotlin.workflow.KotlinDynamicWorkflow +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.getRootWorkflowContext() + 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() + } + } +} 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 0000000000..9b9e36d00c --- /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 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? +} 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 0000000000..1c5806e8b1 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinActivityStubImpl.kt @@ -0,0 +1,67 @@ +/* + * 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 + } +} 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 0000000000..b18386c143 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflow.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.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.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.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(workflowContext) + private val coroutineDispatcher = TemporalCallbackCoroutineDispatcher(dispatcher) + 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() + ) + ) { "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 + } +} 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 0000000000..9d14d915c0 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowContext.kt @@ -0,0 +1,428 @@ +/* + * 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.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.ExperimentalCoroutinesApi +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 + + private var replayContext: ReplayWorkflowContext? = 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.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 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(input: 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(searchAttributeUpdates: List>) { + TODO("Not yet implemented") + } + + override fun currentTimeMillis(): Long { + TODO("Not yet implemented") + } + + val metricScope: Scope + get() = replayContext!!.metricsScope + + @OptIn(ExperimentalCoroutinesApi::class) + 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 KotlinWorkflow.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" + ) + } + } +} 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 0000000000..c7e5244fa7 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowDefinition.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.internal.async + +import io.temporal.api.common.v1.Payloads +import io.temporal.common.interceptors.Header + +interface KotlinWorkflowDefinition { + + /** Always called first. */ + suspend fun initialize() + + 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 new file mode 100644 index 0000000000..0b689bb78b --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowExecutionHandler.kt @@ -0,0 +1,162 @@ +/* + * 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 + } + } +} 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 0000000000..7d3fccebe1 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowImplementationFactory.kt @@ -0,0 +1,410 @@ +/* + * 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 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 +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.replay.ReplayWorkflow +import io.temporal.internal.replay.ReplayWorkflowFactory +import io.temporal.internal.sync.WorkflowInternal +import io.temporal.internal.worker.WorkflowExecutionException +import io.temporal.internal.worker.WorkflowExecutorCache +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.WorkerOptions +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.* +import kotlin.reflect.full.callSuspend +import kotlin.reflect.jvm.kotlinFunction + +class KotlinWorkflowImplementationFactory( + clientOptions: WorkflowClientOptions, + workerOptions: WorkerOptions, + cache: WorkflowExecutorCache +) : ReplayWorkflowFactory { + +// 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 = + 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()) + + override 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. + */ + + override 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" + ) + } + @Suppress("UNCHECKED_CAST") + 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 -> {} + WorkflowMethodType.NONE -> TODO() + WorkflowMethodType.QUERY -> TODO() + WorkflowMethodType.UPDATE -> TODO() + WorkflowMethodType.UPDATE_VALIDATOR -> TODO() + } + } + } + + 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.getRootWorkflowContext() + 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), + // 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)) + 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 kMethod = workflowMethod.kotlinFunction + val result = kMethod!!.callSuspend(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) + + // 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 new file mode 100644 index 0000000000..c10f49f093 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInfoImpl.kt @@ -0,0 +1,60 @@ +/* + * 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)" + } +} 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 0000000000..a9d99a878c --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/KotlinWorkflowInternal.kt @@ -0,0 +1,614 @@ +/* + * 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.workflow.KotlinWorkflowInfo +import io.temporal.workflow.WorkflowMethod +import kotlinx.coroutines.currentCoroutineContext +import java.util.* + +/** + * Never reference directly. It is public only because Java doesn't have internal package support. + */ +class KotlinWorkflowInternal { + + 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( +// 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] */ + suspend fun currentTimeMillis(): Long { + return getWorkflowOutboundCallsInterceptor().currentTimeMillis() + } + + suspend fun setDefaultActivityOptions(activityOptions: ActivityOptions?) { + getRootWorkflowContext().defaultActivityOptions = activityOptions + } + + suspend fun applyActivityOptions(activityTypeToOptions: Map) { + getRootWorkflowContext().applyActivityOptions(activityTypeToOptions) + } + + suspend fun setDefaultLocalActivityOptions(localActivityOptions: LocalActivityOptions?) { + getRootWorkflowContext().defaultLocalActivityOptions = localActivityOptions + } + + suspend fun applyLocalActivityOptions( + activityTypeToOptions: Map + ) { + getRootWorkflowContext().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) +// } + + suspend fun newUntypedActivityStub(options: ActivityOptions?): KotlinActivityStub { + return KotlinActivityStubImpl(options, getWorkflowOutboundCallsInterceptor()) + } + +// 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 && getRootWorkflowContext().isReplaying() +// } +// +// fun getMemo(key: String?, valueClass: Class?, genericType: Type?): T? { +// val memo = getRootWorkflowContext().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() +// } +// } + + suspend fun getWorkflowInfo(): KotlinWorkflowInfo = KotlinWorkflowInfoImpl(getRootWorkflowContext().replayContext!!) + + suspend fun getMetricsScope(): Scope = getRootWorkflowContext().metricScope + + suspend fun randomUUID(): UUID = getRootWorkflowContext().randomUUID() + + suspend fun newRandom(): Random = getRootWorkflowContext().newRandom() + +// private val isLoggingEnabledInReplay: Boolean +// private get() = getRootWorkflowContext().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 getRootWorkflowContext().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 = 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 = getRootWorkflowContext().getReplayContext().searchAttributes +// ?: return emptyMap() +// return Collections.unmodifiableMap(SearchAttributesUtil.decode(searchAttributes)) +// } +// +// val typedSearchAttributes: SearchAttributes +// get() { +// val searchAttributes = getRootWorkflowContext().getReplayContext().searchAttributes +// return SearchAttributesUtil.decodeTyped(searchAttributes) +// } +// +// fun upsertSearchAttributes(searchAttributes: Map?) { +// workflowOutboundInterceptor.upsertSearchAttributes(searchAttributes) +// } +// +// fun upsertTypedSearchAttributes( +// vararg searchAttributeUpdates: SearchAttributeUpdate<*>? +// ) { +// workflowOutboundInterceptor.upsertTypedSearchAttributes(*searchAttributeUpdates) +// } + + suspend fun getDataConverter(): DataConverter = getRootWorkflowContext().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 + 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!!) + } + } + + suspend fun getWorkflowOutboundCallsInterceptor() = getRootWorkflowContext().getWorkflowOutboundInterceptor() + + 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 new file mode 100644 index 0000000000..f788051c32 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/internal/async/TemporalCoroutineDispatcher.kt @@ -0,0 +1,101 @@ +/* + * 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 +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 kotlin.coroutines.AbstractCoroutineContextElement +import kotlin.coroutines.CoroutineContext + +@Suppress("UNUSED_PARAMETER") +@OptIn(InternalCoroutinesApi::class) +class TemporalCoroutineDispatcher(val workflowContext: KotlinWorkflowContext) : CoroutineDispatcher(), Delay { + + private val queue: java.util.Queue = LinkedList() + private val callbackQueue: Queue = LinkedList() + + 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) { + while (callbackQueue.isNotEmpty()) { + val block = callbackQueue.poll() + block.run() + } + + while (queue.isNotEmpty()) { + val block = queue.poll() + block.run() + } + } + + override fun scheduleResumeAfterDelay(timeMillis: Long, continuation: CancellableContinuation) { + val cancellationHandler = + workflowContext.replayContext!!.newTimer(Duration.ofMillis(timeMillis)) { cancellationRequest -> + cancellationRequest ?: callbackQueue.add { + with(continuation) { resumeUndispatched(Unit) } + } + } + continuation.invokeOnCancellation { cause -> cancellationHandler.apply(cause as RuntimeException?) } + } +} + +/** + * 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 { + override val coroutineContext: CoroutineContext = TemporalCoroutineContext(workflowContext) + + // CoroutineScope is used intentionally for user-friendly representation + override fun toString(): String = "TemporalScope(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/KotlinWorkerInterceptor.kt b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt new file mode 100644 index 0000000000..87b63df885 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/KotlinWorkerInterceptor.kt @@ -0,0 +1,98 @@ +/* + * 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 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 +} 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 0000000000..e381bf1fb7 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkerInterceptorBase.kt @@ -0,0 +1,33 @@ +/* + * 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 : KotlinWorkerInterceptor { + override fun interceptWorkflow(next: WorkflowInboundCallsInterceptor): WorkflowInboundCallsInterceptor { + return next + } + + 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 new file mode 100644 index 0000000000..fec0f59960 --- /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 [ ][KotlinWorkerInterceptor.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 KotlinWorkerInterceptor.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 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 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 +} 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 0000000000..22b4f1254f --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowInboundCallsInterceptorBase.kt @@ -0,0 +1,49 @@ +/* + * 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) + } +} 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 0000000000..19d3c288e9 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/interceptors/WorkflowOutboundCallsInterceptor.kt @@ -0,0 +1,186 @@ +/* + * 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.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.* +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 KotlinWorkerInterceptor.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 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(input: RegisterDynamicSignalHandlerInput) + fun registerDynamicQueryHandler(input: RegisterDynamicQueryHandlerInput) + + @Experimental + fun registerDynamicUpdateHandler(input: RegisterDynamicUpdateHandlerInput) + fun randomUUID(): UUID + fun upsertSearchAttributes(searchAttributes: Map) + fun upsertTypedSearchAttributes(searchAttributeUpdates: List>) + + fun currentTimeMillis(): Long +} 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 0000000000..c689555c0b --- /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 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 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 new file mode 100644 index 0000000000..5061125a15 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/kotlin/workflow/KotlinDynamicWorkflow.kt @@ -0,0 +1,52 @@ +/* + * 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? +} 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 0000000000..80ab007d32 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactory.kt @@ -0,0 +1,49 @@ +/* + * 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 +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 0000000000..8befbd95e7 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/worker/KotlinWorkerFactoryOptions.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.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() + } +} 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 0000000000..f803a4b889 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflow.kt @@ -0,0 +1,38 @@ +/* + * 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 +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. + */ + 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 new file mode 100644 index 0000000000..b3e4ae9384 --- /dev/null +++ b/temporal-kotlin/src/main/kotlin/io/temporal/workflow/KotlinWorkflowInfo.kt @@ -0,0 +1,138 @@ +/* + * 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 +} diff --git a/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt new file mode 100644 index 0000000000..6877af6410 --- /dev/null +++ b/temporal-kotlin/src/test/kotlin/io/temporal/workflow/HelloKotlinWorkflow.kt @@ -0,0 +1,178 @@ +/* + * 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.ActivityInterface +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.KotlinWorkerFactory +import kotlinx.coroutines.async +import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.delay +import org.slf4j.LoggerFactory +import java.time.Duration + +/** 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 = "HelloKotlinTaskQueue" + + // Define our workflow unique id + const val WORKFLOW_ID = "HelloKotlinWorkflow" + + /** + * 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 = KotlinWorkerFactory(client, null) + + /* + * 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.newUntypedWorkflowStub( + "GreetingWorkflow", + 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. + */ + workflow.start("Kotlin") + val greeting = workflow.getResult() + // 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 + suspend 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 { + + 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 { + delay(1000) + activities.execute("greet", String::class.java, "Bye", name!!) + } + return@coroutineScope result1.await()!! + "\n" + result2.await()!! + } + } + + /** Simple activity implementation, that concatenates two strings. */ + internal class GreetingActivitiesImpl : GreetingActivities { + override fun composeGreeting(greeting: String?, name: String?): String { + log.info("Composing greeting...") + return greeting + " " + name + "!" + } + + companion object { + private val log = LoggerFactory.getLogger(GreetingActivitiesImpl::class.java) + } + } +} 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 51627901e3..4ed736990c 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 7573c99ab7..60476bf296 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 670a7619cf..9c37ebe2aa 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 83f27b8bb2..89009671a3 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,7 +21,7 @@ package io.temporal.internal.client; import io.temporal.client.WorkflowClient; -import io.temporal.worker.WorkerFactory; +import io.temporal.worker.BaseWorkerFactory; /** * From OOP point of view, there is no reason for this interface not to extend {@link @@ -32,7 +32,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 76703481ff..2098cec6e4 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 bdf209cc1b..700eb81a00 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; @@ -40,11 +41,12 @@ 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; 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; @@ -57,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; @@ -100,19 +101,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 24128639fe..fef14879de 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 0634e0a5a1..53073fdb25 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 a1ece312f5..9058a2fd6c 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,16 +28,14 @@ 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; 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; @@ -66,7 +64,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 +77,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 0000000000..7205ab11a1 --- /dev/null +++ b/temporal-sdk/src/main/java/io/temporal/worker/BaseWorkerFactory.java @@ -0,0 +1,342 @@ +/* + * 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; +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 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); + 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())); + + workerOptions = WorkerOptions.newBuilder(workerOptions).validateAndBuildWithDefaults(); + // 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 4255368a55..aed055ff11 100644 --- a/temporal-sdk/src/main/java/io/temporal/worker/Worker.java +++ b/temporal-sdk/src/main/java/io/temporal/worker/Worker.java @@ -31,8 +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.WorkflowInternal; -import io.temporal.internal.sync.WorkflowThreadExecutor; import io.temporal.internal.worker.*; import io.temporal.internal.worker.SyncActivityWorker; import io.temporal.internal.worker.SyncWorkflowWorker; @@ -75,7 +75,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 +86,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(); + this.options = options; WorkflowServiceStubs service = client.getWorkflowServiceStubs(); WorkflowClientOptions clientOptions = client.getOptions(); String namespace = clientOptions.getNamespace(); @@ -142,8 +141,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 4de582ac05..53aae1adc6 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,24 @@ 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); +public final class WorkerFactory extends BaseWorkerFactory { - private final WorkflowRunLockManager runLocks = new WorkflowRunLockManager(); - - private final Scope metricsScope; - - 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,20 +56,10 @@ 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, @@ -109,261 +71,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(); } } 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 f2c9470ddd..abe84b5e5a 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);