getResultAsync(
* Request cancellation of a workflow execution.
*
* Cancellation cancels {@link io.temporal.workflow.CancellationScope} that wraps the main
- * workflow method. Note that workflow can take long time to get cancelled or even completely
+ * workflow method. Note that workflow can take long time to get canceled or even completely
* ignore the cancellation request.
*/
void cancel();
diff --git a/src/main/java/io/temporal/common/CronSchedule.java b/src/main/java/io/temporal/common/CronSchedule.java
index c4410988a..0cae81ca1 100644
--- a/src/main/java/io/temporal/common/CronSchedule.java
+++ b/src/main/java/io/temporal/common/CronSchedule.java
@@ -31,7 +31,7 @@
* supplied, and the workflow failed or timeout, the workflow will be retried based on the retry
* policy. While the workflow is retrying, it won't schedule its next run. If next schedule is due
* while workflow is running (or retrying), then it will skip that schedule. Cron workflow will not
- * stop until it is terminated or cancelled.
+ * stop until it is terminated or canceled.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
diff --git a/src/main/java/io/temporal/common/converter/EncodedValues.java b/src/main/java/io/temporal/common/converter/EncodedValues.java
index 9a11234f5..5c8e51d0c 100644
--- a/src/main/java/io/temporal/common/converter/EncodedValues.java
+++ b/src/main/java/io/temporal/common/converter/EncodedValues.java
@@ -27,7 +27,7 @@
public final class EncodedValues implements Values {
private Optional payloads;
private DataConverter converter;
- private Object[] values;
+ private final Object[] values;
public EncodedValues(Optional payloads, DataConverter converter) {
this.payloads = Objects.requireNonNull(payloads);
diff --git a/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptor.java b/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptor.java
index 01335163d..98da51652 100644
--- a/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptor.java
+++ b/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptor.java
@@ -129,4 +129,6 @@ void registerSignal(
void upsertSearchAttributes(Map searchAttributes);
Object newThread(Runnable runnable, boolean detached, String name);
+
+ long currentTimeMillis();
}
diff --git a/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptorBase.java b/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptorBase.java
index bdedd8517..67a9ff42b 100644
--- a/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptorBase.java
+++ b/src/main/java/io/temporal/common/interceptors/WorkflowOutboundCallsInterceptorBase.java
@@ -166,4 +166,9 @@ public void upsertSearchAttributes(Map searchAttributes) {
public Object newThread(Runnable runnable, boolean detached, String name) {
return next.newThread(runnable, detached, name);
}
+
+ @Override
+ public long currentTimeMillis() {
+ return next.currentTimeMillis();
+ }
}
diff --git a/src/main/java/io/temporal/failure/ApplicationFailure.java b/src/main/java/io/temporal/failure/ApplicationFailure.java
index e375dc5f9..62633fea5 100644
--- a/src/main/java/io/temporal/failure/ApplicationFailure.java
+++ b/src/main/java/io/temporal/failure/ApplicationFailure.java
@@ -120,7 +120,7 @@ public void setDataConverter(DataConverter converter) {
}
private static String getMessage(String message, String type, boolean nonRetryable) {
- return (Strings.isNullOrEmpty(message) ? "" : "message='" + message + "\', ")
+ return (Strings.isNullOrEmpty(message) ? "" : "message='" + message + "', ")
+ "type='"
+ type
+ '\''
diff --git a/src/main/java/io/temporal/failure/CanceledFailure.java b/src/main/java/io/temporal/failure/CanceledFailure.java
index 82b6d8058..e298033fa 100644
--- a/src/main/java/io/temporal/failure/CanceledFailure.java
+++ b/src/main/java/io/temporal/failure/CanceledFailure.java
@@ -22,13 +22,14 @@
import io.temporal.common.converter.DataConverter;
import io.temporal.common.converter.EncodedValues;
import io.temporal.common.converter.Values;
+import java.util.Objects;
public final class CanceledFailure extends TemporalFailure {
private final Values details;
public CanceledFailure(String message, Values details, Throwable cause) {
super(message, message, cause);
- this.details = details;
+ this.details = Objects.requireNonNull(details);
}
public CanceledFailure(String message, Object details) {
diff --git a/src/main/java/io/temporal/failure/FailureConverter.java b/src/main/java/io/temporal/failure/FailureConverter.java
index 3a33111a8..85596bfc2 100644
--- a/src/main/java/io/temporal/failure/FailureConverter.java
+++ b/src/main/java/io/temporal/failure/FailureConverter.java
@@ -33,7 +33,7 @@
import io.temporal.api.failure.v1.ServerFailureInfo;
import io.temporal.api.failure.v1.TerminatedFailureInfo;
import io.temporal.api.failure.v1.TimeoutFailureInfo;
-import io.temporal.client.ActivityCancelledException;
+import io.temporal.client.ActivityCanceledException;
import io.temporal.common.converter.DataConverter;
import io.temporal.common.converter.EncodedValues;
import io.temporal.internal.common.CheckedExceptionWrapper;
@@ -66,11 +66,11 @@ public class FailureConverter {
private static final Pattern TRACE_ELEMENT_PATTERN = Pattern.compile(TRACE_ELEMENT_REGEXP);
- public static Exception failureToException(Failure failure, DataConverter dataConverter) {
+ public static RuntimeException failureToException(Failure failure, DataConverter dataConverter) {
if (failure == null) {
return null;
}
- Exception result = failureToExceptionImpl(failure, dataConverter);
+ RuntimeException result = failureToExceptionImpl(failure, dataConverter);
if (result instanceof TemporalFailure) {
((TemporalFailure) result).setFailure(failure);
}
@@ -81,8 +81,9 @@ public static Exception failureToException(Failure failure, DataConverter dataCo
return result;
}
- private static Exception failureToExceptionImpl(Failure failure, DataConverter dataConverter) {
- Exception cause =
+ private static RuntimeException failureToExceptionImpl(
+ Failure failure, DataConverter dataConverter) {
+ RuntimeException cause =
failure.hasCause() ? failureToException(failure.getCause(), dataConverter) : null;
switch (failure.getFailureInfoCase()) {
case APPLICATION_FAILURE_INFO:
@@ -257,7 +258,7 @@ public static Failure exceptionToFailureNoUnwrapping(Throwable e) {
.setWorkflowType(WorkflowType.newBuilder().setName(ce.getWorkflowType()))
.setWorkflowExecution(ce.getExecution());
failure.setChildWorkflowExecutionFailureInfo(info);
- } else if (e instanceof ActivityCancelledException) {
+ } else if (e instanceof ActivityCanceledException) {
CanceledFailureInfo.Builder info = CanceledFailureInfo.newBuilder();
failure.setCanceledFailureInfo(info);
} else {
diff --git a/src/main/java/io/temporal/internal/common/LocalActivityMarkerData.java b/src/main/java/io/temporal/internal/common/LocalActivityMarkerData.java
deleted file mode 100644
index 859652c8c..000000000
--- a/src/main/java/io/temporal/internal/common/LocalActivityMarkerData.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.common;
-
-import io.temporal.api.common.v1.ActivityType;
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.enums.v1.EventType;
-import io.temporal.api.failure.v1.CanceledFailureInfo;
-import io.temporal.api.failure.v1.Failure;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.MarkerRecordedEventAttributes;
-import io.temporal.api.workflowservice.v1.RespondActivityTaskCanceledRequest;
-import io.temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest;
-import io.temporal.common.converter.DataConverter;
-import io.temporal.internal.replay.ReplayClockContext;
-import java.time.Duration;
-import java.util.Optional;
-
-public final class LocalActivityMarkerData {
- static final String MARKER_RESULT_KEY = "result";
- static final String MARKER_DATA_KEY = "data";
-
- public static final class Builder {
- private String activityId;
- private String activityType;
- private Optional failure = Optional.empty();
- private Optional result = Optional.empty();
- private long replayTimeMillis;
- private int attempt;
- private Duration backoff;
-
- public Builder setActivityId(String activityId) {
- this.activityId = activityId;
- return this;
- }
-
- public Builder setActivityType(ActivityType activityType) {
- this.activityType = activityType.getName();
- return this;
- }
-
- public Builder setTaskFailedRequest(RespondActivityTaskFailedRequest request) {
- this.failure = Optional.of(request.getFailure());
- return this;
- }
-
- public Builder setTaskCancelledRequest(RespondActivityTaskCanceledRequest request) {
- CanceledFailureInfo.Builder failureInfo = CanceledFailureInfo.newBuilder();
- if (request.hasDetails()) {
- failureInfo.setDetails(request.getDetails());
- }
- this.failure = Optional.of(Failure.newBuilder().setCanceledFailureInfo(failureInfo).build());
- this.result = Optional.empty();
- return this;
- }
-
- public Builder setResult(Payloads result) {
- this.result = Optional.of(result);
- return this;
- }
-
- public Builder setFailure(Failure failure) {
- this.failure = Optional.of(failure);
- return this;
- }
-
- public Builder setReplayTimeMillis(long replayTimeMillis) {
- this.replayTimeMillis = replayTimeMillis;
- return this;
- }
-
- public Builder setAttempt(int attempt) {
- this.attempt = attempt;
- return this;
- }
-
- public Builder setBackoff(Duration backoff) {
- this.backoff = backoff;
- return this;
- }
-
- public LocalActivityMarkerData build() {
- return new LocalActivityMarkerData(
- activityId, activityType, replayTimeMillis, result, failure, attempt, backoff);
- }
- }
-
- private static class DataValue {
- private String activityId;
- private String activityType;
- private long replayTimeMillis;
- private int attempt;
- private long backoffMillis;
-
- // Needed by Jackson deserializer
- DataValue() {}
-
- DataValue(
- String activityId,
- String activityType,
- long replayTimeMillis,
- int attempt,
- Duration backoff) {
- this.activityId = activityId;
- this.activityType = activityType;
- this.replayTimeMillis = replayTimeMillis;
- this.attempt = attempt;
- this.backoffMillis = backoff == null ? 0 : backoff.toMillis();
- }
- }
-
- private final DataValue data;
- private final Optional result;
- private final Optional failure;
-
- private LocalActivityMarkerData(
- String activityId,
- String activityType,
- long replayTimeMillis,
- Optional result,
- Optional failure,
- int attempt,
- Duration backoff) {
- this.data = new DataValue(activityId, activityType, replayTimeMillis, attempt, backoff);
- this.result = result;
- this.failure = failure;
- }
-
- private LocalActivityMarkerData(
- DataValue data, Optional result, Optional failure) {
- this.data = data;
- this.result = result;
- this.failure = failure;
- }
-
- public String getActivityId() {
- return data.activityId;
- }
-
- public String getActivityType() {
- return data.activityType;
- }
-
- public Optional getFailure() {
- return failure;
- }
-
- public Optional getResult() {
- return result;
- }
-
- public long getReplayTimeMillis() {
- return data.replayTimeMillis;
- }
-
- public int getAttempt() {
- return data.attempt;
- }
-
- public Duration getBackoff() {
- return Duration.ofMillis(data.backoffMillis);
- }
-
- public HistoryEvent toEvent(DataConverter converter) {
- Payloads data = converter.toPayloads(this.data).get();
- MarkerRecordedEventAttributes.Builder attributes =
- MarkerRecordedEventAttributes.newBuilder()
- .setMarkerName(ReplayClockContext.LOCAL_ACTIVITY_MARKER_NAME)
- .putDetails(MARKER_DATA_KEY, data);
- if (result.isPresent()) {
- attributes.putDetails(MARKER_RESULT_KEY, result.get());
- }
- if (failure.isPresent()) {
- attributes.setFailure(failure.get());
- }
- return HistoryEvent.newBuilder()
- .setEventType(EventType.EVENT_TYPE_MARKER_RECORDED)
- .setMarkerRecordedEventAttributes(attributes)
- .build();
- }
-
- public static LocalActivityMarkerData fromEventAttributes(
- MarkerRecordedEventAttributes attributes, DataConverter converter) {
- Payloads data = attributes.getDetailsOrThrow(MARKER_DATA_KEY);
- DataValue laHeader =
- converter.fromPayloads(0, Optional.of(data), DataValue.class, DataValue.class);
- Optional result =
- attributes.containsDetails(MARKER_RESULT_KEY)
- ? Optional.of(attributes.getDetailsOrThrow(MARKER_RESULT_KEY))
- : Optional.empty();
- Optional failure =
- attributes.hasFailure() ? Optional.of(attributes.getFailure()) : Optional.empty();
- return new LocalActivityMarkerData(laHeader, result, failure);
- }
-}
diff --git a/src/main/java/io/temporal/internal/common/ProtobufTimeUtils.java b/src/main/java/io/temporal/internal/common/ProtobufTimeUtils.java
index c9915d4db..00d83e92d 100644
--- a/src/main/java/io/temporal/internal/common/ProtobufTimeUtils.java
+++ b/src/main/java/io/temporal/internal/common/ProtobufTimeUtils.java
@@ -25,7 +25,7 @@
import java.time.Duration;
public class ProtobufTimeUtils {
- public static Duration ToJavaDuration(com.google.protobuf.Duration d) {
+ public static Duration toJavaDuration(com.google.protobuf.Duration d) {
if (d == null) {
return Duration.ZERO;
}
@@ -33,7 +33,7 @@ public static Duration ToJavaDuration(com.google.protobuf.Duration d) {
return Duration.ofMillis(Durations.toMillis(d));
}
- public static com.google.protobuf.Duration ToProtoDuration(Duration d) {
+ public static com.google.protobuf.Duration toProtoDuration(Duration d) {
if (d == null) {
return Durations.ZERO;
}
@@ -41,15 +41,15 @@ public static com.google.protobuf.Duration ToProtoDuration(Duration d) {
return Durations.fromMillis(d.toMillis());
}
- public static com.google.protobuf.Timestamp GetCurrentProtoTime() {
+ public static com.google.protobuf.Timestamp getCurrentProtoTime() {
return Timestamps.fromMillis(System.currentTimeMillis());
}
- public static com.uber.m3.util.Duration ToM3Duration(Timestamp to, Timestamp from) {
+ public static com.uber.m3.util.Duration toM3Duration(Timestamp to, Timestamp from) {
return com.uber.m3.util.Duration.ofMillis(Timestamps.toMillis(to) - Timestamps.toMillis(from));
}
- public static com.uber.m3.util.Duration ToM3DurationSinceNow(Timestamp t) {
+ public static com.uber.m3.util.Duration toM3DurationSinceNow(Timestamp t) {
return com.uber.m3.util.Duration.ofMillis(System.currentTimeMillis() - Timestamps.toMillis(t));
}
}
diff --git a/src/main/java/io/temporal/internal/common/WorkflowExecutionUtils.java b/src/main/java/io/temporal/internal/common/WorkflowExecutionUtils.java
index 6b4d3c3ba..958a016a1 100644
--- a/src/main/java/io/temporal/internal/common/WorkflowExecutionUtils.java
+++ b/src/main/java/io/temporal/internal/common/WorkflowExecutionUtils.java
@@ -96,7 +96,7 @@ public class WorkflowExecutionUtils {
private static final Logger log = LoggerFactory.getLogger(WorkflowExecutionUtils.class);
- private static RpcRetryOptions retryParameters =
+ private static final RpcRetryOptions retryParameters =
RpcRetryOptions.newBuilder()
.setBackoffCoefficient(2)
.setInitialInterval(Duration.ofMillis(500))
@@ -113,7 +113,7 @@ public class WorkflowExecutionUtils {
* @param workflowType is optional.
* @param metricsScope metrics with NAMESPACE tag populated
* @throws TimeoutException if workflow didn't complete within specified timeout
- * @throws CanceledFailure if workflow was cancelled
+ * @throws CanceledFailure if workflow was canceled
* @throws WorkflowExecutionFailedException if workflow execution failed
*/
public static Optional getWorkflowExecutionResult(
@@ -168,7 +168,12 @@ private static Optional getResultFromCloseEvent(
closeEvent.getWorkflowExecutionCanceledEventAttributes();
Optional details =
attributes.hasDetails() ? Optional.of(attributes.getDetails()) : Optional.empty();
- throw new CanceledFailure("Workflow canceled", new EncodedValues(details, converter), null);
+ throw new WorkflowFailedException(
+ workflowExecution,
+ workflowType.orElse(null),
+ 0,
+ RetryState.RETRY_STATE_NON_RETRYABLE_FAILURE,
+ new CanceledFailure("Workflow canceled", new EncodedValues(details, converter), null));
case EVENT_TYPE_WORKFLOW_EXECUTION_FAILED:
WorkflowExecutionFailedEventAttributes failed =
closeEvent.getWorkflowExecutionFailedEventAttributes();
@@ -813,6 +818,7 @@ public static boolean isCommandEvent(HistoryEvent event) {
return result;
}
+ /** Returns event that corresponds to a command. */
public static EventType getEventTypeForCommand(CommandType commandType) {
switch (commandType) {
case COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK:
@@ -830,7 +836,7 @@ public static EventType getEventTypeForCommand(CommandType commandType) {
case COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION:
return EventType.EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED;
case COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION:
- return EventType.EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED;
+ return EventType.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED;
case COMMAND_TYPE_RECORD_MARKER:
return EventType.EVENT_TYPE_MARKER_RECORDED;
case COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION:
diff --git a/src/main/java/io/temporal/internal/context/ContextThreadLocal.java b/src/main/java/io/temporal/internal/context/ContextThreadLocal.java
index d9851d8f7..39044b773 100644
--- a/src/main/java/io/temporal/internal/context/ContextThreadLocal.java
+++ b/src/main/java/io/temporal/internal/context/ContextThreadLocal.java
@@ -30,7 +30,7 @@
/** This class holds the current set of context propagators */
public class ContextThreadLocal {
- private static WorkflowThreadLocal> contextPropagators =
+ private static final WorkflowThreadLocal> contextPropagators =
WorkflowThreadLocal.withInitial(
new Supplier>() {
@Override
diff --git a/src/main/java/io/temporal/internal/external/ManualActivityCompletionClientImpl.java b/src/main/java/io/temporal/internal/external/ManualActivityCompletionClientImpl.java
index 40607a192..e096e6ae0 100644
--- a/src/main/java/io/temporal/internal/external/ManualActivityCompletionClientImpl.java
+++ b/src/main/java/io/temporal/internal/external/ManualActivityCompletionClientImpl.java
@@ -37,7 +37,7 @@
import io.temporal.api.workflowservice.v1.RespondActivityTaskCompletedRequest;
import io.temporal.api.workflowservice.v1.RespondActivityTaskFailedByIdRequest;
import io.temporal.api.workflowservice.v1.RespondActivityTaskFailedRequest;
-import io.temporal.client.ActivityCancelledException;
+import io.temporal.client.ActivityCanceledException;
import io.temporal.client.ActivityCompletionFailureException;
import io.temporal.client.ActivityNotExistsException;
import io.temporal.common.converter.DataConverter;
@@ -228,7 +228,7 @@ public void recordHeartbeat(Object details) throws CanceledFailure {
.withOption(METRICS_TAGS_CALL_OPTIONS_KEY, metricsScope)
.recordActivityTaskHeartbeat(request.build());
if (status.getCancelRequested()) {
- throw new ActivityCancelledException();
+ throw new ActivityCanceledException();
}
} catch (StatusRuntimeException e) {
if (e.getStatus().getCode() == Status.Code.NOT_FOUND) {
@@ -256,7 +256,7 @@ public void recordHeartbeat(Object details) throws CanceledFailure {
.withOption(METRICS_TAGS_CALL_OPTIONS_KEY, metricsScope)
.recordActivityTaskHeartbeatById(request.build());
if (status.getCancelRequested()) {
- throw new ActivityCancelledException();
+ throw new ActivityCanceledException();
}
} catch (StatusRuntimeException e) {
if (e.getStatus().getCode() == Status.Code.NOT_FOUND) {
diff --git a/src/main/java/io/temporal/internal/grpc/GrpcDeadlineInterceptor.java b/src/main/java/io/temporal/internal/grpc/GrpcDeadlineInterceptor.java
index 3afee9a90..de2f018b3 100644
--- a/src/main/java/io/temporal/internal/grpc/GrpcDeadlineInterceptor.java
+++ b/src/main/java/io/temporal/internal/grpc/GrpcDeadlineInterceptor.java
@@ -49,7 +49,7 @@ public ClientCall interceptCall(
long duration;
if (LongPollUtil.isLongPoll(method, callOptions)) {
- duration = options.getRpcLongPollTimeoutMillis();
+ duration = options.getRpcLongPollTimeout().toMillis();
if (deadline != null) {
duration = Math.min(duration, deadline.timeRemaining(TimeUnit.MILLISECONDS));
}
@@ -57,9 +57,9 @@ public ClientCall interceptCall(
duration = deadline.timeRemaining(TimeUnit.MILLISECONDS);
} else {
if (method == WorkflowServiceGrpc.getQueryWorkflowMethod()) {
- duration = options.getRpcQueryTimeoutMillis();
+ duration = options.getRpcQueryTimeout().toMillis();
} else {
- duration = options.getRpcTimeoutMillis();
+ duration = options.getRpcTimeout().toMillis();
}
}
if (log.isTraceEnabled()) {
diff --git a/src/main/java/io/temporal/internal/grpc/GrpcTracingInterceptor.java b/src/main/java/io/temporal/internal/grpc/GrpcTracingInterceptor.java
index 6e14c9ad4..7cbe98b3e 100644
--- a/src/main/java/io/temporal/internal/grpc/GrpcTracingInterceptor.java
+++ b/src/main/java/io/temporal/internal/grpc/GrpcTracingInterceptor.java
@@ -49,7 +49,7 @@ public ClientCall interceptCall(
next.newCall(method, callOptions)) {
@Override
public void sendMessage(ReqT message) {
- log.trace("Invoking " + method.getFullMethodName() + "with input: " + message);
+ log.trace("Invoking \"" + method.getFullMethodName() + "\" with input: " + message);
super.sendMessage(message);
}
diff --git a/src/main/java/io/temporal/internal/grpc/LongPollUtil.java b/src/main/java/io/temporal/internal/grpc/LongPollUtil.java
index 2479a9927..c9f12fe39 100644
--- a/src/main/java/io/temporal/internal/grpc/LongPollUtil.java
+++ b/src/main/java/io/temporal/internal/grpc/LongPollUtil.java
@@ -34,9 +34,7 @@ static boolean isLongPoll(
}
if (method == WorkflowServiceGrpc.getGetWorkflowExecutionHistoryMethod()) {
Boolean longPoll = callOptions.getOption(MetricsTag.HISTORY_LONG_POLL_CALL_OPTIONS_KEY);
- if (longPoll != null && longPoll.booleanValue()) {
- return true;
- }
+ return longPoll != null && longPoll.booleanValue();
}
return false;
}
diff --git a/src/main/java/io/temporal/internal/grpc/WorkflowServiceStubsImpl.java b/src/main/java/io/temporal/internal/grpc/WorkflowServiceStubsImpl.java
index eeb5f54a6..807853bbb 100644
--- a/src/main/java/io/temporal/internal/grpc/WorkflowServiceStubsImpl.java
+++ b/src/main/java/io/temporal/internal/grpc/WorkflowServiceStubsImpl.java
@@ -44,6 +44,8 @@ public final class WorkflowServiceStubsImpl implements WorkflowServiceStubs {
private static final Logger log = LoggerFactory.getLogger(WorkflowServiceStubsImpl.class);
+ private static final int MAX_INBOUND_MESSAGE_SIZE = 25_000_000;
+
/** refers to the name of the gRPC header that contains the client library version */
private static final Metadata.Key LIBRARY_VERSION_HEADER_KEY =
Metadata.Key.of("temporal-client-version", Metadata.ASCII_STRING_MARSHALLER);
@@ -106,7 +108,8 @@ public WorkflowServiceStubsImpl(
} else {
NettyChannelBuilder builder =
NettyChannelBuilder.forTarget(options.getTarget())
- .defaultLoadBalancingPolicy("round_robin");
+ .defaultLoadBalancingPolicy("round_robin")
+ .maxInboundMessageSize(MAX_INBOUND_MESSAGE_SIZE);
if (options.getSslContext() == null && !options.getEnableHttps()) {
builder.usePlaintext();
diff --git a/src/main/java/io/temporal/internal/metrics/MetricsType.java b/src/main/java/io/temporal/internal/metrics/MetricsType.java
index 93ea0b77c..fe700a064 100644
--- a/src/main/java/io/temporal/internal/metrics/MetricsType.java
+++ b/src/main/java/io/temporal/internal/metrics/MetricsType.java
@@ -23,14 +23,15 @@ public class MetricsType {
public static final String TEMPORAL_METRICS_PREFIX = "temporal_";
public static final String WORKFLOW_COMPLETED_COUNTER =
TEMPORAL_METRICS_PREFIX + "workflow_completed";
- public static final String WORKFLOW_CANCELLED_COUNTER =
+ public static final String WORKFLOW_CANCELED_COUNTER =
TEMPORAL_METRICS_PREFIX + "workflow_canceled";
public static final String WORKFLOW_FAILED_COUNTER = TEMPORAL_METRICS_PREFIX + "workflow_failed";
public static final String WORKFLOW_CONTINUE_AS_NEW_COUNTER =
TEMPORAL_METRICS_PREFIX + "workflow_continue_as_new";
- // measure workflow execution from start to close
+ /** measure workflow execution from start to close */
public static final String WORKFLOW_E2E_LATENCY =
TEMPORAL_METRICS_PREFIX + "workflow_endtoend_latency";
+
public static final String WORKFLOW_TASK_REPLAY_LATENCY =
TEMPORAL_METRICS_PREFIX + "workflow_task_replay_latency";
@@ -42,6 +43,9 @@ public class MetricsType {
TEMPORAL_METRICS_PREFIX + "workflow_task_schedule_to_start_latency";
public static final String WORKFLOW_TASK_EXECUTION_LATENCY =
TEMPORAL_METRICS_PREFIX + "workflow_task_execution_latency";
+ /** Total latency of a workflow task which can include multiple forced decision tasks */
+ public static final String WORKFLOW_TASK_EXECUTION_TOTAL_LATENCY =
+ TEMPORAL_METRICS_PREFIX + "workflow_task_execution_total_latency";
/** Workflow task failed, possibly failing workflow or reporting failure to the service. */
public static final String WORKFLOW_TASK_EXECUTION_FAILURE_COUNTER =
TEMPORAL_METRICS_PREFIX + "workflow_task_execution_failed";
@@ -49,6 +53,9 @@ public class MetricsType {
public static final String WORKFLOW_TASK_NO_COMPLETION_COUNTER =
TEMPORAL_METRICS_PREFIX + "workflow_task_no_completion";
+ public static final String WORKFLOW_TASK_HEARTBEAT_COUNTER =
+ TEMPORAL_METRICS_PREFIX + "workflow_task_heartbeat";
+
public static final String ACTIVITY_POLL_NO_TASK_COUNTER =
TEMPORAL_METRICS_PREFIX + "activity_poll_no_task";
public static final String ACTIVITY_SCHEDULE_TO_START_LATENCY =
diff --git a/src/main/java/io/temporal/internal/replay/ActivityCommandStateMachine.java b/src/main/java/io/temporal/internal/replay/ActivityCommandStateMachine.java
deleted file mode 100644
index 1f7ae58fe..000000000
--- a/src/main/java/io/temporal/internal/replay/ActivityCommandStateMachine.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.command.v1.RequestCancelActivityTaskCommandAttributes;
-import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes;
-import io.temporal.api.enums.v1.CommandType;
-import io.temporal.api.history.v1.HistoryEvent;
-
-final class ActivityCommandStateMachine extends CommandStateMachineBase {
-
- private ScheduleActivityTaskCommandAttributes scheduleAttributes;
- private long scheduledEventId;
-
- public ActivityCommandStateMachine(
- CommandId id,
- ScheduleActivityTaskCommandAttributes scheduleAttributes,
- long scheduledEventId) {
- super(id);
- this.scheduleAttributes = scheduleAttributes;
- this.scheduledEventId = scheduledEventId;
- }
-
- /** Used for unit testing */
- ActivityCommandStateMachine(
- CommandId id,
- ScheduleActivityTaskCommandAttributes scheduleAttributes,
- CommandState state,
- long scheduledEventId) {
- super(id, state);
- this.scheduleAttributes = scheduleAttributes;
- this.scheduledEventId = scheduledEventId;
- }
-
- @Override
- public Command getCommand() {
- switch (state) {
- case CREATED:
- return createScheduleActivityTaskCommand();
- case CANCELED_AFTER_INITIATED:
- return createRequestCancelActivityTaskCommand();
- default:
- return null;
- }
- }
-
- @Override
- public void handleWorkflowTaskStartedEvent() {
- switch (state) {
- case CANCELED_AFTER_INITIATED:
- stateHistory.add("handleWorkflowTaskStartedEvent");
- state = CommandState.CANCELLATION_COMMAND_SENT;
- stateHistory.add(state.toString());
- break;
- default:
- super.handleWorkflowTaskStartedEvent();
- }
- }
-
- @Override
- public void handleCancellationFailureEvent(HistoryEvent event) {
- switch (state) {
- case CANCELLATION_COMMAND_SENT:
- stateHistory.add("handleCancellationFailureEvent");
- state = CommandState.INITIATED;
- stateHistory.add(state.toString());
- break;
- default:
- super.handleCancellationFailureEvent(event);
- }
- }
-
- private Command createRequestCancelActivityTaskCommand() {
- return Command.newBuilder()
- .setRequestCancelActivityTaskCommandAttributes(
- RequestCancelActivityTaskCommandAttributes.newBuilder()
- .setScheduledEventId(scheduledEventId))
- .setCommandType(CommandType.COMMAND_TYPE_REQUEST_CANCEL_ACTIVITY_TASK)
- .build();
- }
-
- private Command createScheduleActivityTaskCommand() {
- scheduledEventId = getId().getCommandEventId();
- return Command.newBuilder()
- .setScheduleActivityTaskCommandAttributes(scheduleAttributes)
- .setCommandType(CommandType.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK)
- .build();
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/ChildWorkflowCommandStateMachine.java b/src/main/java/io/temporal/internal/replay/ChildWorkflowCommandStateMachine.java
deleted file mode 100644
index 5c4e950e8..000000000
--- a/src/main/java/io/temporal/internal/replay/ChildWorkflowCommandStateMachine.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes;
-import io.temporal.api.enums.v1.CommandType;
-import io.temporal.api.history.v1.HistoryEvent;
-
-final class ChildWorkflowCommandStateMachine extends CommandStateMachineBase {
-
- private StartChildWorkflowExecutionCommandAttributes startAttributes;
-
- private String runId;
-
- public ChildWorkflowCommandStateMachine(
- CommandId id, StartChildWorkflowExecutionCommandAttributes startAttributes) {
- super(id);
- this.startAttributes = startAttributes;
- }
-
- /** Used for unit testing */
- ChildWorkflowCommandStateMachine(
- CommandId id,
- StartChildWorkflowExecutionCommandAttributes startAttributes,
- CommandState state) {
- super(id, state);
- this.startAttributes = startAttributes;
- }
-
- @Override
- public Command getCommand() {
- switch (state) {
- case CREATED:
- return createStartChildWorkflowExecutionCommand();
- case CANCELED_AFTER_STARTED:
- return createRequestCancelExternalWorkflowExecutionCommand();
- default:
- return null;
- }
- }
-
- @Override
- public void handleWorkflowTaskStartedEvent() {
- switch (state) {
- case CANCELED_AFTER_STARTED:
- state = CommandState.CANCELLATION_COMMAND_SENT;
- break;
- default:
- super.handleWorkflowTaskStartedEvent();
- }
- }
-
- @Override
- public void handleStartedEvent(HistoryEvent event) {
- stateHistory.add("handleStartedEvent");
- switch (state) {
- case INITIATED:
- state = CommandState.STARTED;
- break;
- case CANCELED_AFTER_INITIATED:
- state = CommandState.CANCELED_AFTER_STARTED;
- break;
- default:
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleCancellationFailureEvent(HistoryEvent event) {
- switch (state) {
- case CANCELLATION_COMMAND_SENT:
- stateHistory.add("handleCancellationFailureEvent");
- state = CommandState.STARTED;
- stateHistory.add(state.toString());
- break;
- default:
- super.handleCancellationFailureEvent(event);
- }
- }
-
- @Override
- public boolean cancel(Runnable immediateCancellationCallback) {
- switch (state) {
- case STARTED:
- stateHistory.add("cancel");
- state = CommandState.CANCELED_AFTER_STARTED;
- stateHistory.add(state.toString());
- return true;
- default:
- return super.cancel(immediateCancellationCallback);
- }
- }
-
- @Override
- public void handleCancellationEvent() {
- switch (state) {
- case STARTED:
- stateHistory.add("handleCancellationEvent");
- state = CommandState.COMPLETED;
- stateHistory.add(state.toString());
- break;
- default:
- super.handleCancellationEvent();
- }
- }
-
- @Override
- public void handleCompletionEvent() {
- switch (state) {
- case STARTED:
- case CANCELED_AFTER_STARTED:
- stateHistory.add("handleCompletionEvent");
- state = CommandState.COMPLETED;
- stateHistory.add(state.toString());
- break;
- default:
- super.handleCompletionEvent();
- }
- }
-
- private Command createRequestCancelExternalWorkflowExecutionCommand() {
- return Command.newBuilder()
- .setRequestCancelExternalWorkflowExecutionCommandAttributes(
- RequestCancelExternalWorkflowExecutionCommandAttributes.newBuilder()
- .setWorkflowId(startAttributes.getWorkflowId())
- .setRunId(runId))
- .setCommandType(CommandType.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION)
- .build();
- }
-
- private Command createStartChildWorkflowExecutionCommand() {
- return Command.newBuilder()
- .setStartChildWorkflowExecutionCommandAttributes(startAttributes)
- .setCommandType(CommandType.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION)
- .build();
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/CommandHelper.java b/src/main/java/io/temporal/internal/replay/CommandHelper.java
deleted file mode 100644
index 57cae3a60..000000000
--- a/src/main/java/io/temporal/internal/replay/CommandHelper.java
+++ /dev/null
@@ -1,711 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import com.google.protobuf.util.Durations;
-import io.temporal.api.command.v1.CancelWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.command.v1.CompleteWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.FailWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.RecordMarkerCommandAttributes;
-import io.temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes;
-import io.temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.StartTimerCommandAttributes;
-import io.temporal.api.command.v1.UpsertWorkflowSearchAttributesCommandAttributes;
-import io.temporal.api.common.v1.Header;
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.common.v1.SearchAttributes;
-import io.temporal.api.enums.v1.CommandType;
-import io.temporal.api.enums.v1.EventType;
-import io.temporal.api.failure.v1.Failure;
-import io.temporal.api.history.v1.ActivityTaskCancelRequestedEventAttributes;
-import io.temporal.api.history.v1.ActivityTaskCanceledEventAttributes;
-import io.temporal.api.history.v1.ActivityTaskStartedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes;
-import io.temporal.api.history.v1.ExternalWorkflowExecutionCancelRequestedEventAttributes;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.MarkerRecordedEventAttributes;
-import io.temporal.api.history.v1.RequestCancelExternalWorkflowExecutionFailedEventAttributes;
-import io.temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes;
-import io.temporal.api.history.v1.TimerCanceledEventAttributes;
-import io.temporal.api.history.v1.TimerFiredEventAttributes;
-import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse;
-import io.temporal.common.converter.DataConverter;
-import io.temporal.internal.common.WorkflowExecutionUtils;
-import io.temporal.internal.replay.HistoryHelper.WorkflowTaskEvents;
-import io.temporal.internal.worker.WorkflowExecutionException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-
-final class CommandHelper {
-
- // private static final Logger log = LoggerFactory.getLogger(CommandHelper.class);
-
- /**
- * TODO: Update constant once Temporal introduces the limit of command per completion. Or remove
- * code path if Temporal deals with this problem differently like paginating through commands.
- */
- private static final int MAXIMUM_COMMANDS_PER_COMPLETION = 10000;
-
- static final String FORCE_IMMEDIATE_WORKFLOW_TASK_TIMER = "FORCE_IMMEDIATE_WORKFLOW_TASK";
-
- private static final String NON_DETERMINISTIC_MESSAGE =
- "The possible causes are a nondeterministic workflow definition code or an incompatible "
- + "change in the workflow definition.";
-
- private final PollWorkflowTaskQueueResponse.Builder task;
-
- /**
- * When workflow task completes the commands are converted to events that follow the command task
- * completion event. The nextCommandEventId is the id of an event that corresponds to the next
- * command to be added.
- */
- private long nextCommandEventId;
-
- private long lastStartedEventId;
-
- private long idCounter;
-
- private WorkflowTaskEvents workflowTaskEvents;
-
- /** Use access-order to ensure that commands are emitted in order of their creation */
- private final Map commands =
- new LinkedHashMap<>(100, 0.75f, true);
-
- // TODO: removal of completed activities
- private final Map activityIdToScheduledEventId = new HashMap<>();
-
- CommandHelper(PollWorkflowTaskQueueResponse.Builder task) {
- this.task = task;
- }
-
- long getNextCommandEventId() {
- return nextCommandEventId;
- }
-
- public long getLastStartedEventId() {
- return lastStartedEventId;
- }
-
- long scheduleActivityTask(ScheduleActivityTaskCommandAttributes schedule) {
- addAllMissingVersionMarker();
-
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.ACTIVITY, nextCommandEventId);
- activityIdToScheduledEventId.put(schedule.getActivityId(), nextCommandEventId);
- addCommand(commandId, new ActivityCommandStateMachine(commandId, schedule, nextCommandEventId));
- return nextCommandEventId;
- }
-
- /**
- * Returns true if cancellation already happened as schedule event was found in the new commands
- * list
- */
- boolean requestCancelActivityTask(long scheduledEventId, Runnable immediateCancellationCallback) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, scheduledEventId));
- if (command.cancel(immediateCancellationCallback)) {
- nextCommandEventId++;
- }
- return command.isDone();
- }
-
- void handleActivityTaskStarted(HistoryEvent event) {
- ActivityTaskStartedEventAttributes attributes = event.getActivityTaskStartedEventAttributes();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, attributes.getScheduledEventId()));
- command.handleStartedEvent(event);
- }
-
- void handleActivityTaskScheduled(HistoryEvent event) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, event.getEventId()));
- command.handleInitiatedEvent(event);
- }
-
- boolean handleActivityTaskClosed(long scheduledEventId) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, scheduledEventId));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleActivityTaskCancelRequested(HistoryEvent event) {
- ActivityTaskCancelRequestedEventAttributes attributes =
- event.getActivityTaskCancelRequestedEventAttributes();
- long scheduledEventId = attributes.getScheduledEventId();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, scheduledEventId));
- command.handleCancellationInitiatedEvent();
- return command.isDone();
- }
-
- private long getActivityScheduledEventId(String activityId) {
- Long scheduledEventId = activityIdToScheduledEventId.get(activityId);
- if (scheduledEventId == null) {
- throw new Error("Unknown activityId: " + activityId);
- }
- return scheduledEventId;
- }
-
- boolean handleActivityTaskCanceled(HistoryEvent event) {
- ActivityTaskCanceledEventAttributes attributes = event.getActivityTaskCanceledEventAttributes();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.ACTIVITY, attributes.getScheduledEventId()));
- command.handleCancellationEvent();
- return command.isDone();
- }
-
- long startChildWorkflowExecution(StartChildWorkflowExecutionCommandAttributes childWorkflow) {
- addAllMissingVersionMarker();
-
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.CHILD_WORKFLOW, nextCommandEventId);
- addCommand(commandId, new ChildWorkflowCommandStateMachine(commandId, childWorkflow));
- return nextCommandEventId;
- }
-
- void handleStartChildWorkflowExecutionInitiated(HistoryEvent event) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, event.getEventId()));
- command.handleInitiatedEvent(event);
- }
-
- boolean handleStartChildWorkflowExecutionFailed(HistoryEvent event) {
- StartChildWorkflowExecutionFailedEventAttributes attributes =
- event.getStartChildWorkflowExecutionFailedEventAttributes();
- long initiatedEventId = attributes.getInitiatedEventId();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, initiatedEventId));
- command.handleInitiationFailedEvent(event);
- return command.isDone();
- }
-
- /**
- * @return true if cancellation already happened as schedule event was found in the new commands
- * list
- */
- long requestCancelExternalWorkflowExecution(
- RequestCancelExternalWorkflowExecutionCommandAttributes schedule) {
- addAllMissingVersionMarker();
-
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.CANCEL_EXTERNAL_WORKFLOW, nextCommandEventId);
- addCommand(commandId, new ExternalWorkflowCancellationCommandStateMachine(commandId, schedule));
- return nextCommandEventId;
- }
-
- void handleRequestCancelExternalWorkflowExecutionInitiated(HistoryEvent event) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CANCEL_EXTERNAL_WORKFLOW, event.getEventId()));
- command.handleInitiatedEvent(event);
- }
-
- void handleExternalWorkflowExecutionCancelRequested(HistoryEvent event) {
- ExternalWorkflowExecutionCancelRequestedEventAttributes attributes =
- event.getExternalWorkflowExecutionCancelRequestedEventAttributes();
- CommandStateMachine command =
- getCommand(
- new CommandId(
- CommandTarget.CANCEL_EXTERNAL_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- }
-
- void handleRequestCancelExternalWorkflowExecutionFailed(HistoryEvent event) {
- RequestCancelExternalWorkflowExecutionFailedEventAttributes attributes =
- event.getRequestCancelExternalWorkflowExecutionFailedEventAttributes();
- CommandStateMachine command =
- getCommand(
- new CommandId(
- CommandTarget.CANCEL_EXTERNAL_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- }
-
- long signalExternalWorkflowExecution(SignalExternalWorkflowExecutionCommandAttributes signal) {
- addAllMissingVersionMarker();
-
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.SIGNAL_EXTERNAL_WORKFLOW, nextCommandEventId);
- addCommand(commandId, new SignalCommandStateMachine(commandId, signal));
- return nextCommandEventId;
- }
-
- void cancelSignalExternalWorkflowExecution(
- long initiatedEventId, Runnable immediateCancellationCallback) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.SIGNAL_EXTERNAL_WORKFLOW, initiatedEventId));
- if (command.cancel(immediateCancellationCallback)) {
- nextCommandEventId++;
- }
- }
-
- boolean handleSignalExternalWorkflowExecutionFailed(long initiatedEventId) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.SIGNAL_EXTERNAL_WORKFLOW, initiatedEventId));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleExternalWorkflowExecutionSignaled(long initiatedEventId) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.SIGNAL_EXTERNAL_WORKFLOW, initiatedEventId));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- long startTimer(StartTimerCommandAttributes request) {
- addAllMissingVersionMarker();
-
- long startEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.TIMER, startEventId);
- addCommand(commandId, new TimerCommandStateMachine(commandId, request));
- return startEventId;
- }
-
- boolean cancelTimer(long startEventId, Runnable immediateCancellationCallback) {
- CommandStateMachine command = getCommand(new CommandId(CommandTarget.TIMER, startEventId));
- if (command.isDone()) {
- // Cancellation callbacks are not deregistered and might be invoked after timer firing
- return true;
- }
- if (command.cancel(immediateCancellationCallback)) {
- nextCommandEventId++;
- }
- return command.isDone();
- }
-
- void handleChildWorkflowExecutionStarted(HistoryEvent event) {
- ChildWorkflowExecutionStartedEventAttributes attributes =
- event.getChildWorkflowExecutionStartedEventAttributes();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleStartedEvent(event);
- }
-
- boolean handleChildWorkflowExecutionCompleted(
- ChildWorkflowExecutionCompletedEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleChildWorkflowExecutionTimedOut(
- ChildWorkflowExecutionTimedOutEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleChildWorkflowExecutionTerminated(
- ChildWorkflowExecutionTerminatedEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleChildWorkflowExecutionFailed(
- ChildWorkflowExecutionFailedEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleChildWorkflowExecutionCanceled(
- ChildWorkflowExecutionCanceledEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.CHILD_WORKFLOW, attributes.getInitiatedEventId()));
- command.handleCancellationEvent();
- return command.isDone();
- }
-
- void handleSignalExternalWorkflowExecutionInitiated(HistoryEvent event) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.SIGNAL_EXTERNAL_WORKFLOW, event.getEventId()));
- command.handleInitiatedEvent(event);
- }
-
- boolean handleTimerClosed(TimerFiredEventAttributes attributes) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.TIMER, attributes.getStartedEventId()));
- command.handleCompletionEvent();
- return command.isDone();
- }
-
- boolean handleTimerCanceled(HistoryEvent event) {
- TimerCanceledEventAttributes attributes = event.getTimerCanceledEventAttributes();
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.TIMER, attributes.getStartedEventId()));
- command.handleCancellationEvent();
- return command.isDone();
- }
-
- void handleTimerStarted(HistoryEvent event) {
- CommandStateMachine command =
- getCommand(new CommandId(CommandTarget.TIMER, event.getEventId()));
- // Timer started event is indeed initiation event for the timer as
- // it doesn't have a separate event for started as an activity does.
- command.handleInitiatedEvent(event);
- }
-
- /** This happens during strongly consistent query processing for completed workflows */
- public void handleWorkflowExecutionCompleted(HistoryEvent event) {
- CommandId commandId = new CommandId(CommandTarget.SELF, 0);
- CommandStateMachine command = getCommand(commandId);
- if (!(command instanceof CompleteWorkflowStateMachine)) {
- throw new IllegalStateException("Unexpected command: " + command);
- }
- commands.clear();
- }
-
- void completeWorkflowExecution(Optional output) {
- addAllMissingVersionMarker();
-
- CompleteWorkflowExecutionCommandAttributes.Builder attributes =
- CompleteWorkflowExecutionCommandAttributes.newBuilder();
- if (output.isPresent()) {
- attributes.setResult(output.get());
- }
- Command command =
- Command.newBuilder()
- .setCompleteWorkflowExecutionCommandAttributes(attributes)
- .setCommandType(CommandType.COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION)
- .build();
- CommandId commandId = new CommandId(CommandTarget.SELF, 0);
- addCommand(commandId, new CompleteWorkflowStateMachine(commandId, command));
- }
-
- void continueAsNewWorkflowExecution(ContinueAsNewWorkflowExecutionCommandAttributes attributes) {
- addAllMissingVersionMarker();
-
- HistoryEvent firstEvent = task.getHistory().getEvents(0);
- if (!firstEvent.hasWorkflowExecutionStartedEventAttributes()) {
- throw new IllegalStateException(
- "The first event is not WorkflowExecutionStarted: " + firstEvent);
- }
-
- Command command =
- Command.newBuilder()
- .setCommandType(CommandType.COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION)
- .setContinueAsNewWorkflowExecutionCommandAttributes(attributes)
- .build();
-
- CommandId commandId = new CommandId(CommandTarget.SELF, 0);
- addCommand(commandId, new CompleteWorkflowStateMachine(commandId, command));
- }
-
- void failWorkflowExecution(WorkflowExecutionException exception) {
- addAllMissingVersionMarker();
-
- FailWorkflowExecutionCommandAttributes.Builder attributes =
- FailWorkflowExecutionCommandAttributes.newBuilder().setFailure(exception.getFailure());
- Command command =
- Command.newBuilder()
- .setFailWorkflowExecutionCommandAttributes(attributes)
- .setCommandType(CommandType.COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION)
- .build();
- CommandId commandId = new CommandId(CommandTarget.SELF, 0);
- addCommand(commandId, new CompleteWorkflowStateMachine(commandId, command));
- }
-
- /**
- * @return false
means that cancel failed, true
that
- * CancelWorkflowExecution was created.
- */
- void cancelWorkflowExecution() {
- addAllMissingVersionMarker();
-
- Command command =
- Command.newBuilder()
- .setCancelWorkflowExecutionCommandAttributes(
- CancelWorkflowExecutionCommandAttributes.getDefaultInstance())
- .setCommandType(CommandType.COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION)
- .build();
- CommandId commandId = new CommandId(CommandTarget.SELF, 0);
- addCommand(commandId, new CompleteWorkflowStateMachine(commandId, command));
- }
-
- void recordMarker(
- String markerName,
- Optional header,
- Map details,
- Optional failure) {
- // no need to call addAllMissingVersionMarker here as all the callers are already doing it.
-
- RecordMarkerCommandAttributes.Builder marker =
- RecordMarkerCommandAttributes.newBuilder().setMarkerName(markerName);
- marker.putAllDetails(details);
- if (header.isPresent()) {
- marker.setHeader(header.get());
- }
- if (failure.isPresent()) {
- marker.setFailure(failure.get());
- }
- Command command =
- Command.newBuilder()
- .setCommandType(CommandType.COMMAND_TYPE_RECORD_MARKER)
- .setRecordMarkerCommandAttributes(marker)
- .build();
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.MARKER, nextCommandEventId);
- addCommand(commandId, new MarkerCommandStateMachine(commandId, command));
- }
-
- void upsertSearchAttributes(SearchAttributes searchAttributes) {
- Command command =
- Command.newBuilder()
- .setCommandType(CommandType.COMMAND_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES)
- .setUpsertWorkflowSearchAttributesCommandAttributes(
- UpsertWorkflowSearchAttributesCommandAttributes.newBuilder()
- .setSearchAttributes(searchAttributes))
- .build();
- long nextCommandEventId = getNextCommandEventId();
- CommandId commandId = new CommandId(CommandTarget.UPSERT_SEARCH_ATTRIBUTES, nextCommandEventId);
- addCommand(commandId, new UpsertSearchAttributesCommandStateMachine(commandId, command));
- }
-
- List getCommands() {
- List result = new ArrayList<>(MAXIMUM_COMMANDS_PER_COMPLETION + 1);
- for (CommandStateMachine commandStateMachine : commands.values()) {
- Command command = commandStateMachine.getCommand();
- if (command != null) {
- result.add(command);
- }
- }
- // Include FORCE_IMMEDIATE_WORKFLOW_TASK_TIMER only if there are more then
- // MAXIMUM_COMMANDS_PER_COMPLETION events
- int size = result.size();
- if (size > MAXIMUM_COMMANDS_PER_COMPLETION
- && !isCompletionEvent(result.get(MAXIMUM_COMMANDS_PER_COMPLETION - 2))) {
- result = result.subList(0, MAXIMUM_COMMANDS_PER_COMPLETION - 1);
- Command d =
- Command.newBuilder()
- .setStartTimerCommandAttributes(
- StartTimerCommandAttributes.newBuilder()
- .setStartToFireTimeout(Durations.ZERO)
- .setTimerId(FORCE_IMMEDIATE_WORKFLOW_TASK_TIMER))
- .setCommandType(CommandType.COMMAND_TYPE_START_TIMER)
- .build();
- result.add(d);
- }
-
- return result;
- }
-
- private boolean isCompletionEvent(Command command) {
- CommandType type = command.getCommandType();
- switch (type) {
- case COMMAND_TYPE_CANCEL_WORKFLOW_EXECUTION:
- case COMMAND_TYPE_COMPLETE_WORKFLOW_EXECUTION:
- case COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION:
- case COMMAND_TYPE_CONTINUE_AS_NEW_WORKFLOW_EXECUTION:
- return true;
- default:
- return false;
- }
- }
-
- public void handleWorkflowTaskStartedEvent(WorkflowTaskEvents command) {
- this.workflowTaskEvents = command;
- this.nextCommandEventId = command.getNextCommandEventId();
- // Account for WorkflowTaskCompleted
- this.lastStartedEventId = command.getNextCommandEventId() - 2;
- }
-
- void notifyCommandSent() {
- int count = 0;
- Iterator iterator = commands.values().iterator();
- CommandStateMachine next = null;
-
- CommandStateMachine commandStateMachine = getNextCommand(iterator);
- while (commandStateMachine != null) {
- next = getNextCommand(iterator);
- if (++count == MAXIMUM_COMMANDS_PER_COMPLETION
- && next != null
- && !isCompletionEvent(next.getCommand())) {
- break;
- }
- commandStateMachine.handleWorkflowTaskStartedEvent();
- commandStateMachine = next;
- }
- if (next != null && count < MAXIMUM_COMMANDS_PER_COMPLETION) {
- next.handleWorkflowTaskStartedEvent();
- }
- }
-
- private CommandStateMachine getNextCommand(Iterator iterator) {
- CommandStateMachine result = null;
- while (result == null && iterator.hasNext()) {
- result = iterator.next();
- if (result.getCommand() == null) {
- result = null;
- }
- }
- return result;
- }
-
- @Override
- public String toString() {
- return WorkflowExecutionUtils.prettyPrintCommands(getCommands());
- }
-
- PollWorkflowTaskQueueResponse.Builder getTask() {
- return task;
- }
-
- // addAllMissingVersionMarker should always be called before addCommand. In non-replay mode,
- // addAllMissingVersionMarker is a no-op. In replay mode, it tries to insert back missing
- // version marker commands, as we allow user to remove getVersion and not breaking their code.
- // Be careful that addAllMissingVersionMarker can add command and hence change
- // nextCommandEventId, so any call to determine the event ID for the next command should happen
- // after that.
- private void addCommand(CommandId commandId, CommandStateMachine command) {
- Objects.requireNonNull(commandId);
- commands.put(commandId, command);
- nextCommandEventId++;
- }
-
- void addAllMissingVersionMarker() {
- addAllMissingVersionMarker(Optional.empty(), Optional.empty());
- }
-
- Optional getVersionMakerEvent(long eventId) {
- Optional optionalEvent = getCommandEvent(eventId);
- if (!optionalEvent.isPresent()) {
- return Optional.empty();
- }
-
- HistoryEvent event = optionalEvent.get();
- if (event.getEventType() != EventType.EVENT_TYPE_MARKER_RECORDED) {
- return Optional.empty();
- }
-
- if (!event
- .getMarkerRecordedEventAttributes()
- .getMarkerName()
- .equals(ReplayClockContext.VERSION_MARKER_NAME)) {
- return Optional.empty();
- }
- return Optional.of(event);
- }
-
- /**
- * As getVersion calls can be added and removed any time this method inserts missing command
- * events that correspond to removed getVersion calls.
- *
- * @param changeId optional getVersion change id to compare
- * @param converter must be present if changeId is present
- */
- void addAllMissingVersionMarker(Optional changeId, Optional converter) {
- Optional markerEvent = getVersionMakerEvent(nextCommandEventId);
-
- if (!markerEvent.isPresent()) {
- return;
- }
-
- // Look ahead to see if there is a marker with changeId following current version marker
- // If it is the case then all the markers that precede it should be added as commands
- // as their correspondent getVersion calls were removed.
- long changeIdMarkerEventId = -1;
- if (changeId.isPresent()) {
- String id = changeId.get();
- long eventId = nextCommandEventId;
- while (true) {
- MarkerRecordedEventAttributes eventAttributes =
- markerEvent.get().getMarkerRecordedEventAttributes();
- MarkerHandler.MarkerData markerData =
- MarkerHandler.MarkerData.fromEventAttributes(eventAttributes, converter.get());
-
- if (id.equals(markerData.getId())) {
- changeIdMarkerEventId = eventId;
- break;
- }
- eventId++;
- markerEvent = getVersionMakerEvent(eventId);
- if (!markerEvent.isPresent()) {
- break;
- }
- }
- // There are no version markers preceding a marker with the changeId
- if (changeIdMarkerEventId < 0 || changeIdMarkerEventId == nextCommandEventId) {
- return;
- }
- }
- do {
- MarkerRecordedEventAttributes eventAttributes =
- markerEvent.get().getMarkerRecordedEventAttributes();
- // If we have a version marker in history event but not in commands, let's add one.
- RecordMarkerCommandAttributes.Builder attributes =
- RecordMarkerCommandAttributes.newBuilder()
- .setMarkerName(ReplayClockContext.VERSION_MARKER_NAME);
- if (eventAttributes.hasHeader()) {
- attributes.setHeader(eventAttributes.getHeader());
- }
- if (eventAttributes.hasFailure()) {
- attributes.setFailure(eventAttributes.getFailure());
- }
- attributes.putAllDetails(eventAttributes.getDetailsMap());
- Command markerCommand =
- Command.newBuilder()
- .setCommandType(CommandType.COMMAND_TYPE_RECORD_MARKER)
- .setRecordMarkerCommandAttributes(attributes)
- .build();
- CommandId markerCommandId = new CommandId(CommandTarget.MARKER, nextCommandEventId);
- commands.put(markerCommandId, new MarkerCommandStateMachine(markerCommandId, markerCommand));
- nextCommandEventId++;
- markerEvent = getVersionMakerEvent(nextCommandEventId);
- } while (markerEvent.isPresent()
- && (changeIdMarkerEventId < 0 || nextCommandEventId < changeIdMarkerEventId));
- }
-
- private CommandStateMachine getCommand(CommandId commandId) {
- CommandStateMachine result = commands.get(commandId);
- if (result == null) {
- throw new NonDeterminisicWorkflowError(
- "Unknown " + commandId + ". " + NON_DETERMINISTIC_MESSAGE);
- }
- return result;
- }
-
- String getAndIncrementNextId() {
- return String.valueOf(idCounter++);
- }
-
- Optional getCommandEvent(long eventId) {
- return workflowTaskEvents.getCommandEvent(eventId);
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/CommandId.java b/src/main/java/io/temporal/internal/replay/CommandId.java
deleted file mode 100644
index 8678a5d8d..000000000
--- a/src/main/java/io/temporal/internal/replay/CommandId.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import java.util.Objects;
-
-class CommandId {
-
- private final CommandTarget commandTarget;
-
- private final long commandEventId;
-
- CommandId(CommandTarget commandTarget, long commandEventId) {
- this.commandEventId = commandEventId;
- this.commandTarget = Objects.requireNonNull(commandTarget);
- }
-
- CommandTarget getCommandTarget() {
- return commandTarget;
- }
-
- long getCommandEventId() {
- return commandEventId;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || !(o instanceof CommandId)) {
- return false;
- }
-
- CommandId that = (CommandId) o;
-
- if (commandEventId != that.commandEventId) {
- return false;
- }
- return commandTarget == that.commandTarget;
- }
-
- @Override
- public int hashCode() {
- int result = commandTarget.hashCode();
- result = 31 * result + (int) (commandEventId ^ (commandEventId >>> 32));
- return result;
- }
-
- @Override
- public String toString() {
- return "CommandId{"
- + "commandTarget="
- + commandTarget
- + ", commandEventId="
- + commandEventId
- + '}';
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/CommandStateMachine.java b/src/main/java/io/temporal/internal/replay/CommandStateMachine.java
deleted file mode 100644
index c33564098..000000000
--- a/src/main/java/io/temporal/internal/replay/CommandStateMachine.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.history.v1.HistoryEvent;
-
-interface CommandStateMachine {
-
- Command getCommand();
-
- /** @return true if produced a command */
- boolean cancel(Runnable immediateCancellationCallback);
-
- void handleStartedEvent(HistoryEvent event);
-
- void handleCancellationInitiatedEvent();
-
- void handleCancellationEvent();
-
- void handleCancellationFailureEvent(HistoryEvent event);
-
- void handleCompletionEvent();
-
- void handleInitiationFailedEvent(HistoryEvent event);
-
- void handleInitiatedEvent(HistoryEvent event);
-
- void handleWorkflowTaskStartedEvent();
-
- CommandState getState();
-
- boolean isDone();
-
- CommandId getId();
-}
diff --git a/src/main/java/io/temporal/internal/replay/CommandStateMachineBase.java b/src/main/java/io/temporal/internal/replay/CommandStateMachineBase.java
deleted file mode 100644
index 695ad338f..000000000
--- a/src/main/java/io/temporal/internal/replay/CommandStateMachineBase.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.history.v1.HistoryEvent;
-import java.util.ArrayList;
-import java.util.List;
-
-abstract class CommandStateMachineBase implements CommandStateMachine {
-
- protected CommandState state = CommandState.CREATED;
-
- protected List stateHistory = new ArrayList();
-
- private final CommandId id;
-
- public CommandStateMachineBase(CommandId id) {
- this.id = id;
- stateHistory.add(state.toString());
- }
-
- /** Used for unit testing. */
- protected CommandStateMachineBase(CommandId id, CommandState state) {
- this.id = id;
- this.state = state;
- stateHistory.add(state.toString());
- }
-
- @Override
- public CommandState getState() {
- return state;
- }
-
- @Override
- public CommandId getId() {
- return id;
- }
-
- @Override
- public boolean isDone() {
- return state == CommandState.COMPLETED
- || state == CommandState.COMPLETED_AFTER_CANCELLATION_COMMAND_SENT;
- }
-
- @Override
- public void handleWorkflowTaskStartedEvent() {
- switch (state) {
- case CREATED:
- stateHistory.add("handleWorkflowTaskStartedEvent");
- state = CommandState.COMMAND_SENT;
- stateHistory.add(state.toString());
- break;
- default:
- }
- }
-
- @Override
- public boolean cancel(Runnable immediateCancellationCallback) {
- stateHistory.add("cancel");
- boolean result = false;
- switch (state) {
- case CREATED:
- state = CommandState.COMPLETED;
- if (immediateCancellationCallback != null) {
- immediateCancellationCallback.run();
- }
- break;
- case COMMAND_SENT:
- state = CommandState.CANCELED_BEFORE_INITIATED;
- result = true;
- break;
- case INITIATED:
- state = CommandState.CANCELED_AFTER_INITIATED;
- result = true;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- return result;
- }
-
- @Override
- public void handleInitiatedEvent(HistoryEvent event) {
- stateHistory.add("handleInitiatedEvent");
- switch (state) {
- case COMMAND_SENT:
- state = CommandState.INITIATED;
- break;
- case CANCELED_BEFORE_INITIATED:
- state = CommandState.CANCELED_AFTER_INITIATED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleInitiationFailedEvent(HistoryEvent event) {
- stateHistory.add("handleInitiationFailedEvent");
- switch (state) {
- case INITIATED:
- case COMMAND_SENT:
- case CANCELED_BEFORE_INITIATED:
- state = CommandState.COMPLETED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleStartedEvent(HistoryEvent event) {
- stateHistory.add("handleStartedEvent");
- }
-
- @Override
- public void handleCompletionEvent() {
- stateHistory.add("handleCompletionEvent");
- switch (state) {
- case CANCELED_AFTER_INITIATED:
- case INITIATED:
- state = CommandState.COMPLETED;
- break;
- case CANCELLATION_COMMAND_SENT:
- state = CommandState.COMPLETED_AFTER_CANCELLATION_COMMAND_SENT;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleCancellationInitiatedEvent() {
- stateHistory.add("handleCancellationInitiatedEvent");
- switch (state) {
- case CANCELLATION_COMMAND_SENT:
- // No state change
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleCancellationFailureEvent(HistoryEvent event) {
- stateHistory.add("handleCancellationFailureEvent");
- switch (state) {
- case COMPLETED_AFTER_CANCELLATION_COMMAND_SENT:
- state = CommandState.COMPLETED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleCancellationEvent() {
- stateHistory.add("handleCancellationEvent");
- switch (state) {
- case CANCELLATION_COMMAND_SENT:
- state = CommandState.COMPLETED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public String toString() {
- return "CommandStateMachineBase [id="
- + id
- + ", state="
- + state
- + ", isDone="
- + isDone()
- + ", stateHistory="
- + stateHistory
- + "]";
- }
-
- protected void failStateTransition() {
- throw new IllegalStateException("id=" + id + ", transitions=" + stateHistory);
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/CompleteWorkflowStateMachine.java b/src/main/java/io/temporal/internal/replay/CompleteWorkflowStateMachine.java
deleted file mode 100644
index cc5f7dc15..000000000
--- a/src/main/java/io/temporal/internal/replay/CompleteWorkflowStateMachine.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.history.v1.HistoryEvent;
-
-final class CompleteWorkflowStateMachine implements CommandStateMachine {
-
- private Command command;
- private final CommandId id;
-
- public CompleteWorkflowStateMachine(CommandId id, Command command) {
- this.id = id;
- this.command = command;
- }
-
- @Override
- public CommandId getId() {
- return id;
- }
-
- @Override
- public Command getCommand() {
- return command;
- }
-
- @Override
- public void handleInitiationFailedEvent(HistoryEvent event) {
- command = null;
- }
-
- @Override
- public boolean cancel(Runnable immediateCancellationCallback) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleStartedEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCancellationEvent() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCancellationFailureEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCompletionEvent() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleInitiatedEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public CommandState getState() {
- return CommandState.CREATED;
- }
-
- @Override
- public void handleCancellationInitiatedEvent() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean isDone() {
- return command != null;
- }
-
- @Override
- public void handleWorkflowTaskStartedEvent() {}
-
- @Override
- public String toString() {
- return "CompleteWorkflowStateMachine [command=" + command + ", id=" + id + "]";
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/ExecuteActivityParameters.java b/src/main/java/io/temporal/internal/replay/ExecuteActivityParameters.java
index 9ae5c693d..04c08f829 100644
--- a/src/main/java/io/temporal/internal/replay/ExecuteActivityParameters.java
+++ b/src/main/java/io/temporal/internal/replay/ExecuteActivityParameters.java
@@ -21,6 +21,7 @@
import io.temporal.activity.ActivityCancellationType;
import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes;
+import java.util.Objects;
public class ExecuteActivityParameters implements Cloneable {
@@ -30,8 +31,8 @@ public class ExecuteActivityParameters implements Cloneable {
public ExecuteActivityParameters(
ScheduleActivityTaskCommandAttributes.Builder attributes,
ActivityCancellationType cancellationType) {
- this.attributes = attributes;
- this.cancellationType = cancellationType;
+ this.attributes = Objects.requireNonNull(attributes);
+ this.cancellationType = Objects.requireNonNull(cancellationType);
}
public ScheduleActivityTaskCommandAttributes.Builder getAttributes() {
diff --git a/src/main/java/io/temporal/internal/replay/ExecuteLocalActivityParameters.java b/src/main/java/io/temporal/internal/replay/ExecuteLocalActivityParameters.java
index 4989fcb41..10f2280d2 100644
--- a/src/main/java/io/temporal/internal/replay/ExecuteLocalActivityParameters.java
+++ b/src/main/java/io/temporal/internal/replay/ExecuteLocalActivityParameters.java
@@ -20,23 +20,34 @@
package io.temporal.internal.replay;
import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse;
+import java.time.Duration;
public class ExecuteLocalActivityParameters {
private final PollActivityTaskQueueResponse.Builder activityTask;
- private final long elapsedTime;
+ private final Duration localRetryThreshold;
public ExecuteLocalActivityParameters(
- PollActivityTaskQueueResponse.Builder activityTask, long elapsedTime) {
+ PollActivityTaskQueueResponse.Builder activityTask, Duration localRetryThreshold) {
this.activityTask = activityTask;
- this.elapsedTime = elapsedTime;
+ this.localRetryThreshold = localRetryThreshold;
}
public PollActivityTaskQueueResponse.Builder getActivityTask() {
return activityTask;
}
- public long getElapsedTime() {
- return elapsedTime;
+ public Duration getLocalRetryThreshold() {
+ return localRetryThreshold;
+ }
+
+ @Override
+ public String toString() {
+ return "ExecuteLocalActivityParameters{"
+ + "activityTask="
+ + activityTask
+ + ", localRetryThreshold="
+ + localRetryThreshold
+ + '}';
}
}
diff --git a/src/main/java/io/temporal/internal/replay/ExternalWorkflowCancellationCommandStateMachine.java b/src/main/java/io/temporal/internal/replay/ExternalWorkflowCancellationCommandStateMachine.java
deleted file mode 100644
index 37eed765c..000000000
--- a/src/main/java/io/temporal/internal/replay/ExternalWorkflowCancellationCommandStateMachine.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-import io.temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.enums.v1.CommandType;
-import io.temporal.api.history.v1.HistoryEvent;
-
-final class ExternalWorkflowCancellationCommandStateMachine extends CommandStateMachineBase {
-
- private RequestCancelExternalWorkflowExecutionCommandAttributes attributes;
-
- ExternalWorkflowCancellationCommandStateMachine(
- CommandId commandId, RequestCancelExternalWorkflowExecutionCommandAttributes attributes) {
- super(commandId);
- this.attributes = attributes;
- }
-
- @Override
- public Command getCommand() {
- switch (state) {
- case CREATED:
- return createRequestCancelExternalWorkflowExecutionCommand();
- default:
- return null;
- }
- }
-
- @Override
- public boolean cancel(Runnable immediateCancellationCallback) {
- stateHistory.add("cancel");
- failStateTransition();
- return false;
- }
-
- @Override
- public void handleInitiatedEvent(HistoryEvent event) {
- stateHistory.add("handleInitiatedEvent");
- switch (state) {
- case COMMAND_SENT:
- state = CommandState.INITIATED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleInitiationFailedEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleStartedEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCompletionEvent() {
- stateHistory.add("handleCompletionEvent");
- switch (state) {
- case COMMAND_SENT:
- case INITIATED:
- state = CommandState.COMPLETED;
- break;
- default:
- failStateTransition();
- }
- stateHistory.add(state.toString());
- }
-
- @Override
- public void handleCancellationInitiatedEvent() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCancellationFailureEvent(HistoryEvent event) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void handleCancellationEvent() {
- throw new UnsupportedOperationException();
- }
-
- private Command createRequestCancelExternalWorkflowExecutionCommand() {
- return Command.newBuilder()
- .setRequestCancelExternalWorkflowExecutionCommandAttributes(attributes)
- .setCommandType(CommandType.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION)
- .build();
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/HistoryEventHandler.java b/src/main/java/io/temporal/internal/replay/HistoryEventHandler.java
deleted file mode 100644
index efc0f64e2..000000000
--- a/src/main/java/io/temporal/internal/replay/HistoryEventHandler.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.TimerFiredEventAttributes;
-
-interface HistoryEventHandler {
-
- void handleActivityTaskCanceled(HistoryEvent event);
-
- void handleActivityTaskCompleted(HistoryEvent event);
-
- void handleActivityTaskFailed(HistoryEvent event);
-
- void handleActivityTaskTimedOut(HistoryEvent event);
-
- void handleChildWorkflowExecutionCancelRequested(HistoryEvent event);
-
- void handleChildWorkflowExecutionCanceled(HistoryEvent event);
-
- void handleChildWorkflowExecutionStarted(HistoryEvent event);
-
- void handleChildWorkflowExecutionTimedOut(HistoryEvent event);
-
- void handleChildWorkflowExecutionTerminated(HistoryEvent event);
-
- void handleStartChildWorkflowExecutionFailed(HistoryEvent event);
-
- void handleChildWorkflowExecutionFailed(HistoryEvent event);
-
- void handleChildWorkflowExecutionCompleted(HistoryEvent event);
-
- void handleTimerFired(TimerFiredEventAttributes attributes);
-
- void handleTimerCanceled(HistoryEvent event);
-
- void handleExternalWorkflowExecutionSignaled(HistoryEvent event);
-
- void handleMarkerRecorded(HistoryEvent event);
-
- void handleUpsertSearchAttributes(HistoryEvent event);
-}
diff --git a/src/main/java/io/temporal/internal/replay/HistoryHelper.java b/src/main/java/io/temporal/internal/replay/HistoryHelper.java
deleted file mode 100644
index 2217617d9..000000000
--- a/src/main/java/io/temporal/internal/replay/HistoryHelper.java
+++ /dev/null
@@ -1,277 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import com.google.common.collect.PeekingIterator;
-import com.google.protobuf.util.Timestamps;
-import io.temporal.api.enums.v1.EventType;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponseOrBuilder;
-import io.temporal.internal.common.WorkflowExecutionUtils;
-import io.temporal.internal.worker.WorkflowTaskWithHistoryIterator;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.Optional;
-
-class HistoryHelper {
-
- /**
- * Events of a single workflow task. It includes all new events in the history since the last
- * workflow task as events. It doesn't include events that are events that correspond to commands
- * of the previous workflow task.
- */
- static final class WorkflowTaskEvents {
-
- private final List events;
- private final List commandEvents;
- private final List markers = new ArrayList<>();
- private final boolean replay;
- private final long replayCurrentTimeMilliseconds;
- private final long nextCommandEventId;
-
- WorkflowTaskEvents(
- List events,
- List commandEvents,
- boolean replay,
- long replayCurrentTimeMilliseconds,
- long nextCommandEventId) {
- if (nextCommandEventId <= 0) {
- throw new Error("nextCommandEventId is not set");
- }
- this.events = events;
- this.commandEvents = commandEvents;
- for (HistoryEvent event : commandEvents) {
- if (event.getEventType() == EventType.EVENT_TYPE_MARKER_RECORDED) {
- markers.add(event);
- }
- }
- this.replay = replay;
- this.replayCurrentTimeMilliseconds = replayCurrentTimeMilliseconds;
- this.nextCommandEventId = nextCommandEventId;
- }
-
- public List getEvents() {
- return events;
- }
-
- List getCommandEvents() {
- return commandEvents;
- }
-
- /**
- * Returns event that was generated from a command.
- *
- * @return Optional#empty if event at that eventId is not a command originated event.
- */
- Optional getCommandEvent(long eventId) {
- int index = (int) (eventId - nextCommandEventId);
- if (index < 0 || index >= commandEvents.size()) {
- return Optional.empty();
- }
- return Optional.of(commandEvents.get(index));
- }
-
- public List getMarkers() {
- return markers;
- }
-
- public boolean isReplay() {
- return replay;
- }
-
- public long getReplayCurrentTimeMilliseconds() {
- return replayCurrentTimeMilliseconds;
- }
-
- public long getNextCommandEventId() {
- return nextCommandEventId;
- }
-
- @Override
- public String toString() {
- return "WorkflowTaskEvents{"
- + "events="
- + events
- + ", commandEvents="
- + commandEvents
- + ", markers="
- + markers
- + ", replay="
- + replay
- + ", replayCurrentTimeMilliseconds="
- + replayCurrentTimeMilliseconds
- + ", nextCommandEventId="
- + nextCommandEventId
- + '}';
- }
- }
-
- /** Allows peeking for the next event. */
- private static final class EventsIterator implements PeekingIterator {
-
- private Iterator events;
- private HistoryEvent next;
-
- EventsIterator(Iterator events) {
- this.events = events;
- if (events.hasNext()) {
- next = events.next();
- }
- }
-
- @Override
- public HistoryEvent peek() {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
- return next;
- }
-
- @Override
- public boolean hasNext() {
- return next != null;
- }
-
- @Override
- public HistoryEvent next() {
- HistoryEvent result = next;
- if (events.hasNext()) {
- next = events.next();
- } else {
- next = null;
- }
- return result;
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException("not implemented");
- }
- }
-
- /**
- * Iterates through workflow tasks in the history and returns a WorkflowTaskEvents instance per
- * WorkflowTaskStarted event.
- */
- private static class WorkflowTaskEventsIterator implements Iterator {
-
- private EventsIterator events;
- private long replayCurrentTimeMilliseconds;
-
- WorkflowTaskEventsIterator(
- WorkflowTaskWithHistoryIterator workflowTaskWithHistoryIterator,
- long replayCurrentTimeMilliseconds) {
- this.events = new EventsIterator(workflowTaskWithHistoryIterator.getHistory());
- this.replayCurrentTimeMilliseconds = replayCurrentTimeMilliseconds;
- }
-
- @Override
- public boolean hasNext() {
- return events.hasNext();
- }
-
- @Override
- public WorkflowTaskEvents next() {
- // Events generated from commands.
- //
- // For example: ScheduleActivityTaskCommand -> ActivityTaskScheduledEvent
- List commandEvents = new ArrayList<>();
- List newEvents = new ArrayList<>();
- boolean replay = true;
- long nextCommandEventId = -1;
- while (events.hasNext()) {
- HistoryEvent event = events.next();
- EventType eventType = event.getEventType();
-
- // Sticky workers receive an event history that starts with WorkflowTaskCompleted
- if (eventType == EventType.EVENT_TYPE_WORKFLOW_TASK_COMPLETED && nextCommandEventId == -1) {
- nextCommandEventId = event.getEventId() + 1;
- break;
- }
-
- if (eventType == EventType.EVENT_TYPE_WORKFLOW_TASK_STARTED || !events.hasNext()) {
- replayCurrentTimeMilliseconds = Timestamps.toMillis(event.getEventTime());
- if (!events.hasNext()) {
- replay = false;
- nextCommandEventId =
- event.getEventId() + 2; // +1 for next, +1 for WorkflowTaskCompleted
- break;
- }
- HistoryEvent peeked = events.peek();
- EventType peekedType = peeked.getEventType();
- if (peekedType == EventType.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT
- || peekedType == EventType.EVENT_TYPE_WORKFLOW_TASK_FAILED) {
- continue;
- } else if (peekedType == EventType.EVENT_TYPE_WORKFLOW_TASK_COMPLETED) {
- events.next(); // consume WorkflowTaskCompleted
- nextCommandEventId = peeked.getEventId() + 1; // +1 for next and skip over completed
- break;
- } else {
- throw new Error(
- "Unexpected event after WorkflowTaskStarted: "
- + peeked
- + " WorkflowTaskStarted Event: "
- + event);
- }
- }
- newEvents.add(event);
- }
- while (events.hasNext()) {
- if (!WorkflowExecutionUtils.isCommandEvent(events.peek())) {
- break;
- }
- commandEvents.add(events.next());
- }
- WorkflowTaskEvents result =
- new WorkflowTaskEvents(
- newEvents, commandEvents, replay, replayCurrentTimeMilliseconds, nextCommandEventId);
- return result;
- }
- }
-
- private final WorkflowTaskWithHistoryIterator workflowTaskWithHistoryIterator;
- private final Iterator iterator;
-
- HistoryHelper(WorkflowTaskWithHistoryIterator workflowTasks, long replayCurrentTimeMilliseconds) {
- this.workflowTaskWithHistoryIterator = workflowTasks;
- this.iterator = new WorkflowTaskEventsIterator(workflowTasks, replayCurrentTimeMilliseconds);
- }
-
- public Iterator getIterator() {
- return iterator;
- }
-
- public PollWorkflowTaskQueueResponseOrBuilder getWorkflowTask() {
- return workflowTaskWithHistoryIterator.getWorkflowTask();
- }
-
- @Override
- public String toString() {
- return WorkflowExecutionUtils.prettyPrintHistory(
- workflowTaskWithHistoryIterator.getWorkflowTask().getHistory().getEventsList().iterator(),
- true);
- }
-
- long getPreviousStartedEventId() {
- return getWorkflowTask().getPreviousStartedEventId();
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/MarkerCommandStateMachine.java b/src/main/java/io/temporal/internal/replay/MarkerCommandStateMachine.java
deleted file mode 100644
index d827c0a76..000000000
--- a/src/main/java/io/temporal/internal/replay/MarkerCommandStateMachine.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.Command;
-
-final class MarkerCommandStateMachine extends CommandStateMachineBase {
-
- private final Command command;
-
- MarkerCommandStateMachine(CommandId id, Command command) {
- super(id);
- this.command = command;
- }
-
- @Override
- public Command getCommand() {
- if (state == CommandState.CREATED) {
- return command;
- }
- return null;
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/MarkerHandler.java b/src/main/java/io/temporal/internal/replay/MarkerHandler.java
deleted file mode 100644
index f6f37cd78..000000000
--- a/src/main/java/io/temporal/internal/replay/MarkerHandler.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.enums.v1.EventType;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.MarkerRecordedEventAttributes;
-import io.temporal.common.converter.DataConverter;
-import io.temporal.internal.sync.WorkflowInternal;
-import io.temporal.workflow.Functions.Func1;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-
-class MarkerHandler {
- // Including mutable side effect and version marker.
- static final String MUTABLE_MARKER_HEADER_KEY = "header";
- static final String MUTABLE_MARKER_DATA_KEY = "data";
-
- private static final class MarkerResult {
-
- private final Optional data;
-
- /**
- * Count of how many times handle was called since the last marker recorded. It is used to
- * ensure that an updated value is returned after the same exact number of times during a
- * replay.
- */
- private int accessCount;
-
- private MarkerResult(Optional data) {
- this.data = data;
- }
-
- public Optional getData() {
- accessCount++;
- return data;
- }
-
- int getAccessCount() {
- return accessCount;
- }
- }
-
- static final class MarkerData {
-
- private static final class MarkerHeader {
- private String id;
- private long eventId;
- private int accessCount;
-
- // Needed for Jackson deserialization
- MarkerHeader() {}
-
- MarkerHeader(String id, long eventId, int accessCount) {
- this.id = id;
- this.eventId = eventId;
- this.accessCount = accessCount;
- }
- }
-
- private final MarkerHeader header;
- private final Optional data;
-
- static MarkerData fromEventAttributes(
- MarkerRecordedEventAttributes attributes, DataConverter converter) {
- Optional details =
- attributes.containsDetails(MUTABLE_MARKER_DATA_KEY)
- ? Optional.of(attributes.getDetailsOrThrow(MUTABLE_MARKER_DATA_KEY))
- : Optional.empty();
- MarkerData.MarkerHeader header =
- converter.fromPayloads(
- 0,
- Optional.of(attributes.getDetailsOrThrow(MUTABLE_MARKER_HEADER_KEY)),
- MarkerData.MarkerHeader.class,
- MarkerData.MarkerHeader.class);
-
- return new MarkerData(header, details);
- }
-
- MarkerData(String id, long eventId, Optional data, int accessCount) {
- this.header = new MarkerHeader(id, eventId, accessCount);
- this.data = Objects.requireNonNull(data);
- }
-
- MarkerData(MarkerHeader header, Optional data) {
- this.header = header;
- this.data = Objects.requireNonNull(data);
- }
-
- public MarkerHeader getHeader() {
- return header;
- }
-
- public String getId() {
- return header.id;
- }
-
- public long getEventId() {
- return header.eventId;
- }
-
- public int getAccessCount() {
- return header.accessCount;
- }
-
- public Optional getData() {
- return data;
- }
- }
-
- private final CommandHelper commandHelper;
- private final String markerName;
- private final ReplayAware replayContext;
-
- // Key is marker id
- private final Map mutableMarkerResults = new HashMap<>();
-
- MarkerHandler(CommandHelper commandHelper, String markerName, ReplayAware replayContext) {
- this.commandHelper = commandHelper;
- this.markerName = markerName;
- this.replayContext = replayContext;
- }
-
- /**
- * @param id marker id
- * @param func given the value from the last marker returns value to store. If result is empty
- * nothing is recorded into the history.
- * @return the latest value returned by func
- */
- Optional handle(
- String id, DataConverter converter, Func1, Optional> func) {
- MarkerResult result = mutableMarkerResults.get(id);
- Optional stored;
- if (result == null) {
- stored = Optional.empty();
- } else {
- stored = result.getData();
- }
- long eventId = commandHelper.getNextCommandEventId();
- int accessCount = result == null ? 0 : result.getAccessCount();
-
- if (replayContext.isReplaying()) {
- Optional data = getMarkerDataFromHistory(eventId, id, accessCount, converter);
- if (data.isPresent()) {
- // Need to insert marker to ensure that eventId is incremented
- recordMutableMarker(id, eventId, data, accessCount, converter);
- return data;
- }
-
- // TODO(maxim): Verify why this is necessary.
- if (!stored.isPresent()) {
- mutableMarkerResults.put(
- id, new MarkerResult(converter.toPayloads(WorkflowInternal.DEFAULT_VERSION)));
- }
-
- return stored;
- }
- Optional toStore = func.apply(stored);
- if (toStore.isPresent()) {
- recordMutableMarker(id, eventId, toStore, accessCount, converter);
- return toStore;
- }
- return stored;
- }
-
- private Optional getMarkerDataFromHistory(
- long eventId, String markerId, int expectedAcccessCount, DataConverter converter) {
- Optional event = commandHelper.getCommandEvent(eventId);
- if (!event.isPresent() || event.get().getEventType() != EventType.EVENT_TYPE_MARKER_RECORDED) {
- return Optional.empty();
- }
-
- MarkerRecordedEventAttributes attributes = event.get().getMarkerRecordedEventAttributes();
- String name = attributes.getMarkerName();
- if (!markerName.equals(name)) {
- return Optional.empty();
- }
-
- MarkerData markerData = MarkerData.fromEventAttributes(attributes, converter);
- // access count is used to not return data from the marker before the recorded number of calls
- if (!markerId.equals(markerData.getId())
- || markerData.getAccessCount() > expectedAcccessCount) {
- return Optional.empty();
- }
- return markerData.getData();
- }
-
- private void recordMutableMarker(
- String id, long eventId, Optional data, int accessCount, DataConverter converter) {
- MarkerData marker = new MarkerData(id, eventId, data, accessCount);
- mutableMarkerResults.put(id, new MarkerResult(data));
- Map details = new HashMap<>();
- if (data.isPresent()) {
- details.put(MUTABLE_MARKER_DATA_KEY, data.get());
- }
- details.put(MUTABLE_MARKER_HEADER_KEY, converter.toPayloads(marker.getHeader()).get());
- commandHelper.recordMarker(markerName, Optional.empty(), details, Optional.empty());
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/NonDeterminisicWorkflowError.java b/src/main/java/io/temporal/internal/replay/NonDeterministicWorkflowError.java
similarity index 82%
rename from src/main/java/io/temporal/internal/replay/NonDeterminisicWorkflowError.java
rename to src/main/java/io/temporal/internal/replay/NonDeterministicWorkflowError.java
index 59b9684bc..9829abbef 100644
--- a/src/main/java/io/temporal/internal/replay/NonDeterminisicWorkflowError.java
+++ b/src/main/java/io/temporal/internal/replay/NonDeterministicWorkflowError.java
@@ -19,9 +19,9 @@
package io.temporal.internal.replay;
-final class NonDeterminisicWorkflowError extends Error {
+public final class NonDeterministicWorkflowError extends Error {
- NonDeterminisicWorkflowError(String message) {
- super(message);
+ public NonDeterministicWorkflowError(String message, Throwable cause) {
+ super(message, cause);
}
}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayActivityContext.java b/src/main/java/io/temporal/internal/replay/ReplayActivityContext.java
deleted file mode 100644
index 92cfd7237..000000000
--- a/src/main/java/io/temporal/internal/replay/ReplayActivityContext.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import static io.temporal.failure.FailureConverter.JAVA_SDK;
-
-import io.temporal.activity.ActivityCancellationType;
-import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes;
-import io.temporal.api.common.v1.ActivityType;
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.failure.v1.CanceledFailureInfo;
-import io.temporal.api.failure.v1.Failure;
-import io.temporal.api.history.v1.ActivityTaskCanceledEventAttributes;
-import io.temporal.api.history.v1.ActivityTaskCompletedEventAttributes;
-import io.temporal.api.history.v1.ActivityTaskFailedEventAttributes;
-import io.temporal.api.history.v1.ActivityTaskTimedOutEventAttributes;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.failure.CanceledFailure;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-
-final class ReplayActivityContext {
-
- private final class ActivityCancellationHandler implements Consumer {
-
- private final long scheduledEventId;
-
- private final String activityId;
-
- private final BiConsumer, Exception> callback;
-
- private final ActivityCancellationType cancellationType;
-
- private ActivityCancellationHandler(
- long scheduledEventId,
- String activityId,
- BiConsumer, Exception> callaback,
- ActivityCancellationType cancellationType) {
- this.scheduledEventId = scheduledEventId;
- this.activityId = activityId;
- this.callback = callaback;
- this.cancellationType = cancellationType;
- }
-
- @Override
- public void accept(Exception cause) {
- if (!scheduledActivities.containsKey(scheduledEventId)) {
- // Cancellation handlers are not deregistered. So they fire after an activity completion.
- return;
- }
- Runnable immediateCancellationCallback =
- () -> {
- OpenRequestInfo, OpenActivityInfo> scheduled =
- scheduledActivities.remove(scheduledEventId);
- if (scheduled == null) {
- throw new IllegalArgumentException(
- String.format(
- "Activity with activityId=%s and scheduledEventId=%d wasn't found",
- activityId, scheduledEventId));
- }
- callback.accept(null, new CanceledFailure("Cancelled by request"));
- };
- if (cancellationType != ActivityCancellationType.WAIT_CANCELLATION_COMPLETED) {
- immediateCancellationCallback.run();
- immediateCancellationCallback = () -> {};
- }
- if (cancellationType != ActivityCancellationType.ABANDON) {
- commandHelper.requestCancelActivityTask(scheduledEventId, immediateCancellationCallback);
- }
- }
- }
-
- private final CommandHelper commandHelper;
-
- private static class OpenActivityInfo {
- private final ActivityType activityType;
- private final String activityId;
- private final long scheduledEventId;
- private long startedEventId;
-
- private OpenActivityInfo(ActivityType activityType, String activityId, long scheduledEventId) {
- this.activityType = activityType;
- this.activityId = activityId;
- this.scheduledEventId = scheduledEventId;
- }
-
- public ActivityType getActivityType() {
- return activityType;
- }
-
- public String getActivityId() {
- return activityId;
- }
-
- public long getScheduledEventId() {
- return scheduledEventId;
- }
-
- public long getStartedEventId() {
- return startedEventId;
- }
-
- public void setStartedEventId(long startedEventId) {
- this.startedEventId = startedEventId;
- }
- }
-
- // key is scheduledEventId
- private final Map, OpenActivityInfo>>
- scheduledActivities = new HashMap<>();
-
- ReplayActivityContext(CommandHelper commandHelper) {
- this.commandHelper = commandHelper;
- }
-
- Consumer scheduleActivityTask(
- ExecuteActivityParameters parameters, BiConsumer, Exception> callback) {
- final ScheduleActivityTaskCommandAttributes.Builder attributes = parameters.getAttributes();
-
- if (attributes.getActivityId().isEmpty()) {
- attributes.setActivityId(commandHelper.getAndIncrementNextId());
- }
-
- long scheduledEventId = commandHelper.scheduleActivityTask(attributes.build());
- final OpenRequestInfo, OpenActivityInfo> context =
- new OpenRequestInfo<>(
- new OpenActivityInfo(
- attributes.getActivityType(), attributes.getActivityId(), scheduledEventId));
- context.setCompletionHandle(callback);
- scheduledActivities.put(scheduledEventId, context);
- return new ReplayActivityContext.ActivityCancellationHandler(
- scheduledEventId, attributes.getActivityId(), callback, parameters.getCancellationType());
- }
-
- void handleActivityTaskCanceled(HistoryEvent event) {
- ActivityTaskCanceledEventAttributes attributes = event.getActivityTaskCanceledEventAttributes();
- if (commandHelper.handleActivityTaskCanceled(event)) {
- Failure failure =
- Failure.newBuilder()
- .setSource(JAVA_SDK)
- .setCanceledFailureInfo(
- CanceledFailureInfo.newBuilder().setDetails(attributes.getDetails()))
- .build();
- FailureWrapperException e = new FailureWrapperException(failure);
- OpenRequestInfo, OpenActivityInfo> scheduled =
- scheduledActivities.remove(attributes.getScheduledEventId());
- if (scheduled != null) {
- BiConsumer, Exception> completionHandle =
- scheduled.getCompletionCallback();
- // It is OK to fail with subclass of CanceledException when cancellation requested.
- // It allows passing information about cancellation (details in this case) to the
- // surrounding doCatch block
- completionHandle.accept(Optional.empty(), e);
- }
- }
- }
-
- void handleActivityTaskCompleted(HistoryEvent event) {
- ActivityTaskCompletedEventAttributes attributes =
- event.getActivityTaskCompletedEventAttributes();
- if (commandHelper.handleActivityTaskClosed(attributes.getScheduledEventId())) {
- OpenRequestInfo, OpenActivityInfo> scheduled =
- scheduledActivities.remove(attributes.getScheduledEventId());
- if (scheduled != null) {
- Optional result =
- attributes.hasResult() ? Optional.of(attributes.getResult()) : Optional.empty();
- BiConsumer, Exception> completionHandle =
- scheduled.getCompletionCallback();
- completionHandle.accept(result, null);
- } else {
- throw new NonDeterminisicWorkflowError(
- "Trying to complete activity event "
- + attributes.getScheduledEventId()
- + " that is not in scheduledActivities");
- }
- }
- }
-
- void handleActivityTaskFailed(HistoryEvent event) {
- ActivityTaskFailedEventAttributes attributes = event.getActivityTaskFailedEventAttributes();
- if (commandHelper.handleActivityTaskClosed(attributes.getScheduledEventId())) {
- OpenRequestInfo, OpenActivityInfo> scheduled =
- scheduledActivities.remove(attributes.getScheduledEventId());
- if (scheduled != null) {
- OpenActivityInfo context = scheduled.getUserContext();
- ActivityTaskFailedException failure =
- new ActivityTaskFailedException(
- event.getEventId(),
- attributes.getScheduledEventId(),
- attributes.getStartedEventId(),
- context.getActivityType(),
- context.getActivityId(),
- attributes.getFailure());
- BiConsumer, Exception> completionHandle =
- scheduled.getCompletionCallback();
- completionHandle.accept(Optional.empty(), failure);
- }
- }
- }
-
- void handleActivityTaskTimedOut(HistoryEvent event) {
- ActivityTaskTimedOutEventAttributes attributes = event.getActivityTaskTimedOutEventAttributes();
- if (commandHelper.handleActivityTaskClosed(attributes.getScheduledEventId())) {
- OpenRequestInfo, OpenActivityInfo> scheduled =
- scheduledActivities.remove(attributes.getScheduledEventId());
- if (scheduled != null) {
- Failure failure = attributes.getFailure();
- OpenActivityInfo context = scheduled.getUserContext();
- ActivityTaskTimeoutException timeoutException =
- new ActivityTaskTimeoutException(
- event.getEventId(),
- context.getScheduledEventId(),
- context.getStartedEventId(),
- context.getActivityType(),
- context.getActivityId(),
- attributes.getRetryState(),
- failure);
- BiConsumer, Exception> completionHandle =
- scheduled.getCompletionCallback();
- completionHandle.accept(Optional.empty(), timeoutException);
- }
- }
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayChildWorkflowContext.java b/src/main/java/io/temporal/internal/replay/ReplayChildWorkflowContext.java
deleted file mode 100644
index e2d4e8e26..000000000
--- a/src/main/java/io/temporal/internal/replay/ReplayChildWorkflowContext.java
+++ /dev/null
@@ -1,383 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import io.temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes;
-import io.temporal.api.command.v1.StartChildWorkflowExecutionCommandAttributes;
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.common.v1.WorkflowExecution;
-import io.temporal.api.enums.v1.RetryState;
-import io.temporal.api.enums.v1.TimeoutType;
-import io.temporal.api.history.v1.ChildWorkflowExecutionCanceledEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionCompletedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionFailedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionStartedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionTerminatedEventAttributes;
-import io.temporal.api.history.v1.ChildWorkflowExecutionTimedOutEventAttributes;
-import io.temporal.api.history.v1.ExternalWorkflowExecutionCancelRequestedEventAttributes;
-import io.temporal.api.history.v1.ExternalWorkflowExecutionSignaledEventAttributes;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.SignalExternalWorkflowExecutionFailedEventAttributes;
-import io.temporal.api.history.v1.StartChildWorkflowExecutionFailedEventAttributes;
-import io.temporal.client.WorkflowExecutionAlreadyStarted;
-import io.temporal.common.converter.EncodedValues;
-import io.temporal.failure.CanceledFailure;
-import io.temporal.failure.ChildWorkflowFailure;
-import io.temporal.failure.TerminatedFailure;
-import io.temporal.failure.TimeoutFailure;
-import io.temporal.workflow.ChildWorkflowCancellationType;
-import io.temporal.workflow.SignalExternalWorkflowException;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Random;
-import java.util.UUID;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
-
-final class ReplayChildWorkflowContext {
-
- private final class ChildWorkflowCancellationHandler implements Consumer {
-
- private final long initiatedEventId;
- private final String workflowId;
- private final ChildWorkflowCancellationType cancellationType;
-
- private ChildWorkflowCancellationHandler(
- long initiatedEventId, String workflowId, ChildWorkflowCancellationType cancellationType) {
- this.initiatedEventId = initiatedEventId;
- this.workflowId = Objects.requireNonNull(workflowId);
- this.cancellationType = cancellationType;
- }
-
- @Override
- public void accept(Exception cause) {
- OpenChildWorkflowRequestInfo scheduled = scheduledExternalWorkflows.get(initiatedEventId);
-
- if (scheduled == null) {
- // Cancellation handlers are not deregistered. So they fire after a child completion.
- return;
- }
- switch (cancellationType) {
- case WAIT_CANCELLATION_REQUESTED:
- case WAIT_CANCELLATION_COMPLETED:
- case TRY_CANCEL:
- RequestCancelExternalWorkflowExecutionCommandAttributes cancelAttributes =
- RequestCancelExternalWorkflowExecutionCommandAttributes.newBuilder()
- .setWorkflowId(workflowId)
- .setChildWorkflowOnly(true)
- .build();
- long cancellationInitiatedEventId =
- commandHelper.requestCancelExternalWorkflowExecution(cancelAttributes);
- scheduledExternalCancellations.put(cancellationInitiatedEventId, initiatedEventId);
- }
- switch (cancellationType) {
- case ABANDON:
- case TRY_CANCEL:
- scheduledExternalWorkflows.remove(initiatedEventId);
- CanceledFailure e = new CanceledFailure("Canceled without waiting", null);
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), e);
- }
- }
- }
-
- private final CommandHelper commandHelper;
-
- private final WorkflowContext workflowContext;
-
- // key is initiatedEventId
- private final Map scheduledExternalWorkflows =
- new HashMap<>();
- /** Maps cancellationInitiatedEventId to child initiatedEventId */
- private final Map scheduledExternalCancellations = new HashMap<>();
-
- // key is initiatedEventId
- private final Map> scheduledSignals = new HashMap<>();
-
- ReplayChildWorkflowContext(CommandHelper commandHelper, WorkflowContext workflowContext) {
- this.commandHelper = commandHelper;
- this.workflowContext = workflowContext;
- }
-
- Consumer startChildWorkflow(
- StartChildWorkflowExecutionParameters parameters,
- Consumer executionCallback,
- BiConsumer, Exception> callback) {
- final StartChildWorkflowExecutionCommandAttributes.Builder attributes = parameters.getRequest();
- long initiatedEventId = commandHelper.startChildWorkflowExecution(attributes.build());
- final OpenChildWorkflowRequestInfo context =
- new OpenChildWorkflowRequestInfo(parameters.getCancellationType(), executionCallback);
- context.setCompletionHandle(callback);
- scheduledExternalWorkflows.put(initiatedEventId, context);
- return new ChildWorkflowCancellationHandler(
- initiatedEventId, attributes.getWorkflowId(), parameters.getCancellationType());
- }
-
- Consumer signalWorkflowExecution(
- SignalExternalWorkflowExecutionCommandAttributes.Builder attributes,
- BiConsumer callback) {
- OpenRequestInfo context = new OpenRequestInfo<>();
- attributes.setControl(commandHelper.getAndIncrementNextId());
- long finalSignalId = commandHelper.signalExternalWorkflowExecution(attributes.build());
- context.setCompletionHandle(callback);
- scheduledSignals.put(finalSignalId, context);
- return (e) -> {
- if (!scheduledSignals.containsKey(finalSignalId)) {
- // Cancellation handlers are not deregistered. So they fire after a signal completion.
- return;
- }
- commandHelper.cancelSignalExternalWorkflowExecution(finalSignalId, null);
- OpenRequestInfo scheduled = scheduledSignals.remove(finalSignalId);
- if (scheduled == null) {
- throw new IllegalArgumentException("Signal \"" + finalSignalId + "\" wasn't scheduled");
- }
- callback.accept(null, e);
- };
- }
-
- void requestCancelWorkflowExecution(WorkflowExecution execution) {
- RequestCancelExternalWorkflowExecutionCommandAttributes attributes =
- RequestCancelExternalWorkflowExecutionCommandAttributes.newBuilder()
- .setWorkflowId(execution.getWorkflowId())
- .setRunId(execution.getRunId())
- .build();
- commandHelper.requestCancelExternalWorkflowExecution(attributes);
- }
-
- void continueAsNewOnCompletion(ContinueAsNewWorkflowExecutionCommandAttributes attributes) {
- // TODO: add validation to check if continueAsNew is not set
- workflowContext.setContinueAsNewOnCompletion(attributes);
- }
-
- /** Replay safe UUID */
- UUID randomUUID() {
- String runId = workflowContext.getCurrentRunId();
- if (runId == null) {
- throw new Error("null currentRunId");
- }
- String id = runId + ":" + commandHelper.getAndIncrementNextId();
- byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
- return UUID.nameUUIDFromBytes(bytes);
- }
-
- Random newRandom() {
- return new Random(randomUUID().getLeastSignificantBits());
- }
-
- void handleChildWorkflowExecutionCancelRequested(HistoryEvent event) {
- ExternalWorkflowExecutionCancelRequestedEventAttributes attributes =
- event.getExternalWorkflowExecutionCancelRequestedEventAttributes();
- commandHelper.handleExternalWorkflowExecutionCancelRequested(event);
- Long initiatedEventId = scheduledExternalCancellations.remove(attributes.getInitiatedEventId());
- if (initiatedEventId == null) {
- return;
- }
- OpenChildWorkflowRequestInfo scheduled = scheduledExternalWorkflows.get(initiatedEventId);
- if (scheduled == null) {
- return;
- }
- if (scheduled.getCancellationType()
- == ChildWorkflowCancellationType.WAIT_CANCELLATION_REQUESTED) {
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- CanceledFailure e = new CanceledFailure("Child workflow cancellation requested");
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), e);
- }
- }
-
- void handleChildWorkflowExecutionCanceled(HistoryEvent event) {
- ChildWorkflowExecutionCanceledEventAttributes attributes =
- event.getChildWorkflowExecutionCanceledEventAttributes();
- if (commandHelper.handleChildWorkflowExecutionCanceled(attributes)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- // TODO(maxim): Add support for passing details without using converter here
- CanceledFailure e =
- new CanceledFailure("Child canceled", new EncodedValues(attributes.getDetails()), null);
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), e);
- }
- }
- }
-
- void handleChildWorkflowExecutionStarted(HistoryEvent event) {
- ChildWorkflowExecutionStartedEventAttributes attributes =
- event.getChildWorkflowExecutionStartedEventAttributes();
- commandHelper.handleChildWorkflowExecutionStarted(event);
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.get(attributes.getInitiatedEventId());
- if (scheduled != null) {
- scheduled.getExecutionCallback().accept(attributes.getWorkflowExecution());
- }
- }
-
- void handleChildWorkflowExecutionTimedOut(HistoryEvent event) {
- ChildWorkflowExecutionTimedOutEventAttributes attributes =
- event.getChildWorkflowExecutionTimedOutEventAttributes();
- if (commandHelper.handleChildWorkflowExecutionTimedOut(attributes)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- TimeoutFailure timeoutFailure =
- new TimeoutFailure(null, null, TimeoutType.TIMEOUT_TYPE_START_TO_CLOSE);
- timeoutFailure.setStackTrace(new StackTraceElement[0]);
- RuntimeException failure =
- new ChildWorkflowFailure(
- attributes.getInitiatedEventId(),
- attributes.getStartedEventId(),
- attributes.getWorkflowType().getName(),
- attributes.getWorkflowExecution(),
- attributes.getNamespace(),
- attributes.getRetryState(),
- timeoutFailure);
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), failure);
- }
- }
- }
-
- void handleChildWorkflowExecutionTerminated(HistoryEvent event) {
- ChildWorkflowExecutionTerminatedEventAttributes attributes =
- event.getChildWorkflowExecutionTerminatedEventAttributes();
- WorkflowExecution execution = attributes.getWorkflowExecution();
- if (commandHelper.handleChildWorkflowExecutionTerminated(attributes)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- RuntimeException failure =
- new ChildWorkflowFailure(
- attributes.getInitiatedEventId(),
- attributes.getStartedEventId(),
- attributes.getWorkflowType().getName(),
- attributes.getWorkflowExecution(),
- attributes.getNamespace(),
- RetryState.RETRY_STATE_NON_RETRYABLE_FAILURE,
- new TerminatedFailure(null, null));
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), failure);
- }
- }
- }
-
- void handleStartChildWorkflowExecutionFailed(HistoryEvent event) {
- StartChildWorkflowExecutionFailedEventAttributes attributes =
- event.getStartChildWorkflowExecutionFailedEventAttributes();
- if (commandHelper.handleStartChildWorkflowExecutionFailed(event)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- Exception failure =
- new ChildWorkflowTaskFailedException(
- event.getEventId(),
- WorkflowExecution.newBuilder().setWorkflowId(attributes.getWorkflowId()).build(),
- attributes.getWorkflowType(),
- RetryState.RETRY_STATE_NON_RETRYABLE_FAILURE,
- null);
- failure.initCause(
- new WorkflowExecutionAlreadyStarted(
- WorkflowExecution.newBuilder().setWorkflowId(attributes.getWorkflowId()).build(),
- attributes.getWorkflowType().getName(),
- null));
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), failure);
- }
- }
- }
-
- void handleChildWorkflowExecutionFailed(HistoryEvent event) {
- ChildWorkflowExecutionFailedEventAttributes attributes =
- event.getChildWorkflowExecutionFailedEventAttributes();
- if (commandHelper.handleChildWorkflowExecutionFailed(attributes)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- RuntimeException failure =
- new ChildWorkflowTaskFailedException(
- event.getEventId(),
- attributes.getWorkflowExecution(),
- attributes.getWorkflowType(),
- attributes.getRetryState(),
- attributes.getFailure());
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- completionCallback.accept(Optional.empty(), failure);
- }
- }
- }
-
- void handleChildWorkflowExecutionCompleted(HistoryEvent event) {
- ChildWorkflowExecutionCompletedEventAttributes attributes =
- event.getChildWorkflowExecutionCompletedEventAttributes();
- if (commandHelper.handleChildWorkflowExecutionCompleted(attributes)) {
- OpenChildWorkflowRequestInfo scheduled =
- scheduledExternalWorkflows.remove(attributes.getInitiatedEventId());
- if (scheduled != null) {
- BiConsumer, Exception> completionCallback =
- scheduled.getCompletionCallback();
- Optional result =
- attributes.hasResult() ? Optional.of(attributes.getResult()) : Optional.empty();
- completionCallback.accept(result, null);
- }
- }
- }
-
- void handleSignalExternalWorkflowExecutionFailed(HistoryEvent event) {
- SignalExternalWorkflowExecutionFailedEventAttributes attributes =
- event.getSignalExternalWorkflowExecutionFailedEventAttributes();
- long initiatedEventId = attributes.getInitiatedEventId();
- if (commandHelper.handleSignalExternalWorkflowExecutionFailed(initiatedEventId)) {
- OpenRequestInfo signalContextAndResult =
- scheduledSignals.remove(initiatedEventId);
- if (signalContextAndResult != null) {
- WorkflowExecution signaledExecution =
- WorkflowExecution.newBuilder()
- .setWorkflowId(attributes.getWorkflowExecution().getWorkflowId())
- .setRunId(attributes.getWorkflowExecution().getRunId())
- .build();
- RuntimeException failure = new SignalExternalWorkflowException(signaledExecution, null);
- signalContextAndResult.getCompletionCallback().accept(null, failure);
- }
- }
- }
-
- void handleExternalWorkflowExecutionSignaled(HistoryEvent event) {
- ExternalWorkflowExecutionSignaledEventAttributes attributes =
- event.getExternalWorkflowExecutionSignaledEventAttributes();
- long initiatedEventId = attributes.getInitiatedEventId();
- if (commandHelper.handleExternalWorkflowExecutionSignaled(initiatedEventId)) {
- OpenRequestInfo signalCtxAndResult = scheduledSignals.remove(initiatedEventId);
- if (signalCtxAndResult != null) {
- signalCtxAndResult.getCompletionCallback().accept(null, null);
- }
- }
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayClockContext.java b/src/main/java/io/temporal/internal/replay/ReplayClockContext.java
deleted file mode 100644
index b80564989..000000000
--- a/src/main/java/io/temporal/internal/replay/ReplayClockContext.java
+++ /dev/null
@@ -1,399 +0,0 @@
-/*
- * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
- *
- * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
- *
- * Modifications copyright (C) 2017 Uber Technologies, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"). You may not
- * use this file except in compliance with the License. A copy of the License is
- * located at
- *
- * http://aws.amazon.com/apache2.0
- *
- * or in the "license" file accompanying this file. This file is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
- * express or implied. See the License for the specific language governing
- * permissions and limitations under the License.
- */
-
-package io.temporal.internal.replay;
-
-import static io.temporal.internal.replay.MarkerHandler.MUTABLE_MARKER_DATA_KEY;
-
-import io.temporal.api.command.v1.StartTimerCommandAttributes;
-import io.temporal.api.common.v1.ActivityType;
-import io.temporal.api.common.v1.Header;
-import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.common.v1.SearchAttributes;
-import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.MarkerRecordedEventAttributes;
-import io.temporal.api.history.v1.TimerCanceledEventAttributes;
-import io.temporal.api.history.v1.TimerFiredEventAttributes;
-import io.temporal.api.workflowservice.v1.PollActivityTaskQueueResponse;
-import io.temporal.common.converter.DataConverter;
-import io.temporal.failure.CanceledFailure;
-import io.temporal.internal.common.LocalActivityMarkerData;
-import io.temporal.internal.common.ProtobufTimeUtils;
-import io.temporal.internal.sync.WorkflowInternal;
-import io.temporal.internal.worker.LocalActivityWorker;
-import io.temporal.workflow.Functions.Func;
-import io.temporal.workflow.Functions.Func1;
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.locks.Condition;
-import java.util.function.BiConsumer;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Clock that must be used inside workflow definition code to ensure replay determinism. */
-public final class ReplayClockContext {
-
- private static final String SIDE_EFFECT_MARKER_NAME = "SideEffect";
- private static final String MUTABLE_SIDE_EFFECT_MARKER_NAME = "MutableSideEffect";
- public static final String VERSION_MARKER_NAME = "Version";
- public static final String LOCAL_ACTIVITY_MARKER_NAME = "LocalActivity";
-
- private static final Logger log = LoggerFactory.getLogger(ReplayClockContext.class);
-
- private final class TimerCancellationHandler implements Consumer {
-
- private final long startEventId;
-
- TimerCancellationHandler(long timerId) {
- this.startEventId = timerId;
- }
-
- @Override
- public void accept(Exception reason) {
- commandHelper.cancelTimer(startEventId, () -> timerCancelled(startEventId, reason));
- }
- }
-
- private final CommandHelper commandHelper;
- // key is startedEventId
- private final Map> scheduledTimers = new HashMap<>();
- private long replayCurrentTimeMilliseconds = -1;
- // Local time when replayCurrentTimeMilliseconds was updated.
- private long replayTimeUpdatedAtMillis = -1;
- private boolean replaying = true;
- // Key is side effect marker eventId
- private final Map> sideEffectResults = new HashMap<>();
- private final MarkerHandler mutableSideEffectHandler;
- private final MarkerHandler versionHandler;
- private final BiFunction laTaskPoller;
- private final Map, ActivityType>> pendingLaTasks =
- new HashMap<>();
- private final Map unstartedLaTasks = new HashMap<>();
- private final ReplayWorkflowExecutor workflowExecutor;
- private final DataConverter dataConverter;
- private final Condition taskCondition;
- private boolean taskCompleted = false;
-
- ReplayClockContext(
- CommandHelper commandHelper,
- BiFunction laTaskPoller,
- ReplayWorkflowExecutor workflowExecutor,
- DataConverter dataConverter) {
- this.commandHelper = commandHelper;
- this.taskCondition = workflowExecutor.getLock().newCondition();
- mutableSideEffectHandler =
- new MarkerHandler(commandHelper, MUTABLE_SIDE_EFFECT_MARKER_NAME, () -> replaying);
- versionHandler = new MarkerHandler(commandHelper, VERSION_MARKER_NAME, () -> replaying);
- this.laTaskPoller = laTaskPoller;
- this.workflowExecutor = workflowExecutor;
- this.dataConverter = dataConverter;
- }
-
- public long currentTimeMillis() {
- return replayCurrentTimeMilliseconds;
- }
-
- private long replayTimeUpdatedAtMillis() {
- return replayTimeUpdatedAtMillis;
- }
-
- void setReplayCurrentTimeMilliseconds(long replayCurrentTimeMilliseconds) {
- if (this.replayCurrentTimeMilliseconds < replayCurrentTimeMilliseconds) {
- this.replayCurrentTimeMilliseconds = replayCurrentTimeMilliseconds;
- this.replayTimeUpdatedAtMillis = System.currentTimeMillis();
- }
- }
-
- boolean isReplaying() {
- return replaying;
- }
-
- Consumer createTimer(Duration delay, Consumer callback) {
- if (delay.isNegative()) {
- throw new IllegalArgumentException("Negative delay: " + delay);
- }
- if (delay.isZero()) {
- callback.accept(null);
- return null;
- }
- long firingTime = currentTimeMillis() + delay.toMillis();
- final OpenRequestInfo, Long> context = new OpenRequestInfo<>(firingTime);
- final StartTimerCommandAttributes timer =
- StartTimerCommandAttributes.newBuilder()
- .setStartToFireTimeout(ProtobufTimeUtils.ToProtoDuration(delay))
- .setTimerId(String.valueOf(commandHelper.getAndIncrementNextId()))
- .build();
- long startEventId = commandHelper.startTimer(timer);
- context.setCompletionHandle((ctx, e) -> callback.accept(e));
- scheduledTimers.put(startEventId, context);
- return new TimerCancellationHandler(startEventId);
- }
-
- void setReplaying(boolean replaying) {
- this.replaying = replaying;
- }
-
- void handleTimerFired(TimerFiredEventAttributes attributes) {
- long startedEventId = attributes.getStartedEventId();
- if (commandHelper.handleTimerClosed(attributes)) {
- OpenRequestInfo, Long> scheduled = scheduledTimers.remove(startedEventId);
- if (scheduled != null) {
- // Server doesn't guarantee that the timer fire timestamp is larger or equal of the
- // expected fire time. So fix the time or timer firing will be ignored.
- long firingTime = scheduled.getUserContext();
- if (replayCurrentTimeMilliseconds < firingTime) {
- setReplayCurrentTimeMilliseconds(firingTime);
- }
- BiConsumer, Exception> completionCallback = scheduled.getCompletionCallback();
- completionCallback.accept(null, null);
- }
- }
- }
-
- void handleTimerCanceled(HistoryEvent event) {
- TimerCanceledEventAttributes attributes = event.getTimerCanceledEventAttributes();
- long startedEventId = attributes.getStartedEventId();
- if (commandHelper.handleTimerCanceled(event)) {
- timerCancelled(startedEventId, null);
- }
- }
-
- private void timerCancelled(long startEventId, Exception reason) {
- OpenRequestInfo, ?> scheduled = scheduledTimers.remove(startEventId);
- if (scheduled == null) {
- return;
- }
- BiConsumer, Exception> context = scheduled.getCompletionCallback();
- CanceledFailure exception = new CanceledFailure("Cancelled by request");
- if (reason != null) {
- exception.initCause(reason);
- }
- context.accept(null, exception);
- }
-
- Optional sideEffect(Func> func) {
- commandHelper.addAllMissingVersionMarker();
- long sideEffectEventId = commandHelper.getNextCommandEventId();
- Optional result;
- if (replaying) {
- result = sideEffectResults.get(sideEffectEventId);
- if (result == null) {
- throw new Error("No cached result found for SideEffect EventId=" + sideEffectEventId);
- }
- } else {
- try {
- result = func.apply();
- } catch (Error e) {
- throw e;
- } catch (Exception e) {
- throw new Error("sideEffect function failed", e);
- }
- }
- Map details = new HashMap<>();
- if (result.isPresent()) {
- details.put(MUTABLE_MARKER_DATA_KEY, result.get());
- }
- commandHelper.recordMarker(
- SIDE_EFFECT_MARKER_NAME, Optional.empty(), details, Optional.empty());
- return result;
- }
-
- /**
- * @param id mutable side effect id
- * @param func given the value from the last marker returns value to store. If result is empty
- * nothing is recorded into the history.
- * @return the latest value returned by func
- */
- Optional mutableSideEffect(
- String id, DataConverter converter, Func1, Optional> func) {
- commandHelper.addAllMissingVersionMarker();
- return mutableSideEffectHandler.handle(id, converter, func);
- }
-
- void upsertSearchAttributes(SearchAttributes searchAttributes) {
- commandHelper.upsertSearchAttributes(searchAttributes);
- }
-
- void handleMarkerRecorded(HistoryEvent event) {
- MarkerRecordedEventAttributes attributes = event.getMarkerRecordedEventAttributes();
- String name = attributes.getMarkerName();
- if (SIDE_EFFECT_MARKER_NAME.equals(name)) {
- Optional details =
- attributes.containsDetails(MUTABLE_MARKER_DATA_KEY)
- ? Optional.of(attributes.getDetailsOrThrow(MUTABLE_MARKER_DATA_KEY))
- : Optional.empty();
- sideEffectResults.put(event.getEventId(), details);
- } else if (LOCAL_ACTIVITY_MARKER_NAME.equals(name)) {
- handleLocalActivityMarker(event.getEventId(), attributes);
- } else if (!MUTABLE_SIDE_EFFECT_MARKER_NAME.equals(name) && !VERSION_MARKER_NAME.equals(name)) {
- if (log.isWarnEnabled()) {
- log.warn("Unexpected marker: " + event);
- }
- }
- }
-
- private void handleLocalActivityMarker(long eventId, MarkerRecordedEventAttributes attributes) {
- LocalActivityMarkerData marker =
- LocalActivityMarkerData.fromEventAttributes(attributes, dataConverter);
- if (pendingLaTasks.containsKey(marker.getActivityId())) {
- if (log.isDebugEnabled()) {
- log.debug("Handle LocalActivityMarker for activity " + marker.getActivityId());
- }
- Map details = attributes.getDetailsMap();
- Optional header =
- attributes.hasHeader() ? Optional.of(attributes.getHeader()) : Optional.empty();
- commandHelper.recordMarker(LOCAL_ACTIVITY_MARKER_NAME, header, details, marker.getFailure());
-
- OpenRequestInfo, ActivityType> scheduled =
- pendingLaTasks.remove(marker.getActivityId());
- unstartedLaTasks.remove(marker.getActivityId());
-
- Exception failure = null;
- if (marker.getFailure().isPresent()) {
- failure =
- new ActivityTaskFailedException(
- eventId,
- 0,
- 0,
- ActivityType.newBuilder().setName(marker.getActivityType()).build(),
- marker.getActivityId(),
- marker.getFailure().get());
- }
-
- BiConsumer, Exception> completionHandle =
- scheduled.getCompletionCallback();
- completionHandle.accept(marker.getResult(), failure);
- setReplayCurrentTimeMilliseconds(marker.getReplayTimeMillis());
-
- taskCompleted = true;
- // This method is already called under the lock.
- taskCondition.signal();
- } else {
- log.warn(
- "Local Activity completion ignored for eventId="
- + eventId
- + ", activityId="
- + marker.getActivityId()
- + ", activityType="
- + marker.getActivityType());
- }
- }
-
- /**
- * During replay getVersion should account for the following situations at the current eventId.
- *
- *
- * - There is correspondent Marker with the same changeId: return version from the marker.
- *
- There is no Marker with the same changeId: return DEFAULT_VERSION,
- *
- There is marker with a different changeId (possibly more than one) and the marker with
- * matching changeId follows them: add fake commands for all the version markers that
- * precede the matching one as the correspondent getVersion calls were removed
- *
- There is marker with a different changeId (possibly more than one) and no marker with
- * matching changeId follows them: return DEFAULT_VERSION as it looks like the getVersion
- * was added after that part of code has executed
- *
- Another case is when there is no call to getVersion and there is a version marker: insert
- * fake commands for all version markers up to the event that caused the lookup.
- *
- */
- int getVersion(String changeId, DataConverter converter, int minSupported, int maxSupported) {
- commandHelper.addAllMissingVersionMarker(Optional.of(changeId), Optional.of(converter));
-
- Optional result =
- versionHandler.handle(
- changeId,
- converter,
- (stored) -> {
- if (stored.isPresent()) {
- return Optional.empty();
- }
- return converter.toPayloads(maxSupported);
- });
-
- if (!result.isPresent()) {
- return WorkflowInternal.DEFAULT_VERSION;
- }
- int version = converter.fromPayloads(0, result, Integer.class, Integer.class);
- validateVersion(changeId, version, minSupported, maxSupported);
- return version;
- }
-
- private void validateVersion(String changeId, int version, int minSupported, int maxSupported) {
- if ((version < minSupported || version > maxSupported)
- && version != WorkflowInternal.DEFAULT_VERSION) {
- throw new Error(
- String.format(
- "Version %d of changeId %s is not supported. Supported version is between %d and %d.",
- version, changeId, minSupported, maxSupported));
- }
- }
-
- Consumer scheduleLocalActivityTask(
- ExecuteLocalActivityParameters params, BiConsumer, Exception> callback) {
- PollActivityTaskQueueResponse.Builder activityTask = params.getActivityTask();
- final OpenRequestInfo, ActivityType> context =
- new OpenRequestInfo<>(activityTask.getActivityType());
- context.setCompletionHandle(callback);
- String activityId = activityTask.getActivityId();
- if (activityId.isEmpty()) {
- activityId = commandHelper.getAndIncrementNextId();
- activityTask.setActivityId(activityId);
- }
- pendingLaTasks.put(activityId, context);
- unstartedLaTasks.put(activityId, params);
- return null;
- }
-
- boolean startUnstartedLaTasks(Duration maxWaitAllowed) {
- long startTime = System.currentTimeMillis();
- for (ExecuteLocalActivityParameters params : unstartedLaTasks.values()) {
- long currTime = System.currentTimeMillis();
- maxWaitAllowed = maxWaitAllowed.minus(Duration.ofMillis(currTime - startTime));
- boolean applied =
- laTaskPoller.apply(
- new LocalActivityWorker.Task(
- params,
- workflowExecutor.getLocalActivityCompletionSink(),
- workflowExecutor.getWorkflowTaskTimeout(),
- this::currentTimeMillis,
- this::replayTimeUpdatedAtMillis),
- maxWaitAllowed);
- if (!applied) {
- return false;
- }
- }
- unstartedLaTasks.clear();
- return true;
- }
-
- int numPendingLaTasks() {
- return pendingLaTasks.size();
- }
-
- void awaitTaskCompletion(Duration duration) throws InterruptedException {
- while (!taskCompleted) {
- // This call is called from already locked object
- taskCondition.awaitNanos(duration.toNanos());
- }
- taskCompleted = false;
- }
-}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayWorkflow.java b/src/main/java/io/temporal/internal/replay/ReplayWorkflow.java
index 3a78cb7f4..f1bfddb2f 100644
--- a/src/main/java/io/temporal/internal/replay/ReplayWorkflow.java
+++ b/src/main/java/io/temporal/internal/replay/ReplayWorkflow.java
@@ -42,12 +42,6 @@ public interface ReplayWorkflow {
void close();
- /**
- * @return time at which workflow can make progress. For example when {@link
- * io.temporal.workflow.Workflow#sleep(long)} expires.
- */
- long getNextWakeUpTime();
-
/**
* Called after all history is replayed and workflow cannot make any progress if workflow task is
* a query.
@@ -67,7 +61,7 @@ public interface ReplayWorkflow {
*/
WorkflowExecutionException mapUnexpectedException(Throwable failure);
- WorkflowExecutionException mapError(Error failure);
+ WorkflowExecutionException mapError(Throwable failure);
WorkflowImplementationOptions getWorkflowImplementationOptions();
}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayWorkflowContext.java b/src/main/java/io/temporal/internal/replay/ReplayWorkflowContext.java
index 23ce942aa..0dc856495 100644
--- a/src/main/java/io/temporal/internal/replay/ReplayWorkflowContext.java
+++ b/src/main/java/io/temporal/internal/replay/ReplayWorkflowContext.java
@@ -26,24 +26,24 @@
import io.temporal.api.common.v1.SearchAttributes;
import io.temporal.api.common.v1.WorkflowExecution;
import io.temporal.api.common.v1.WorkflowType;
+import io.temporal.api.failure.v1.Failure;
import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse;
import io.temporal.common.context.ContextPropagator;
-import io.temporal.common.converter.DataConverter;
+import io.temporal.workflow.Functions;
import io.temporal.workflow.Functions.Func;
import io.temporal.workflow.Functions.Func1;
-import io.temporal.workflow.Promise;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
-import java.util.function.BiConsumer;
-import java.util.function.Consumer;
/**
* Represents the context for workflow. Should only be used within the scope of workflow definition
* code, meaning any code which is not part of activity implementations.
+ *
+ * TODO(maxim): Get rid of any Exceptions in the callbacks. They should only return Failure.
*/
public interface ReplayWorkflowContext extends ReplayAware {
@@ -53,16 +53,28 @@ public interface ReplayWorkflowContext extends ReplayAware {
WorkflowType getWorkflowType();
+ /** Is cancel of the workflow requested. */
boolean isCancelRequested();
+ /**
+ * When these attributes are present upon completion of the workflow code the ContinueAsNew
+ * command is emitted instead of the workflow completion.
+ */
ContinueAsNewWorkflowExecutionCommandAttributes getContinueAsNewOnCompletion();
+ /** Set the attributes to request continue as new upon workflow code completion. */
void setContinueAsNewOnCompletion(ContinueAsNewWorkflowExecutionCommandAttributes attributes);
+ /**
+ * RunId of the first run in the continue as new chain. Empty if this workflow never called
+ * continue as new.
+ */
Optional getContinuedExecutionRunId();
+ /** Workflow task queue name. */
String getTaskQueue();
+ /** Workflow namespace. */
String getNamespace();
String getWorkflowId();
@@ -75,6 +87,9 @@ public interface ReplayWorkflowContext extends ReplayAware {
long getRunStartedTimestampMillis();
+ /**
+ * The time that is calculated as the start time of the first run plus workflow execution timeout.
+ */
long getWorkflowExecutionExpirationTimestampMillis();
Duration getWorkflowTaskTimeout();
@@ -99,19 +114,20 @@ public interface ReplayWorkflowContext extends ReplayAware {
List getContextPropagators();
/**
- * Used to dynamically schedule an activity for execution
+ * Requests an activity execution.
*
* @param parameters An object which encapsulates all the information required to schedule an
* activity for execution
* @param callback Callback that is called upon activity completion or failure.
- * @return cancellation handle. Invoke {@link Consumer#accept(Object)} to cancel activity task.
+ * @return cancellation handle. Invoke {@link io.temporal.workflow.Functions.Proc1#apply(Object)}
+ * to cancel activity task.
*/
- Consumer scheduleActivityTask(
- ExecuteActivityParameters parameters, BiConsumer, Exception> callback);
+ Functions.Proc1 scheduleActivityTask(
+ ExecuteActivityParameters parameters, Functions.Proc2, Failure> callback);
- Consumer scheduleLocalActivityTask(
+ Functions.Proc scheduleLocalActivityTask(
ExecuteLocalActivityParameters parameters,
- BiConsumer, Exception> callback);
+ Functions.Proc2, Failure> callback);
/**
* Start child workflow.
@@ -119,24 +135,36 @@ Consumer scheduleLocalActivityTask(
* @param parameters An object which encapsulates all the information required to schedule a child
* workflow for execution
* @param callback Callback that is called upon child workflow completion or failure.
- * @return cancellation handle. Invoke {@link Consumer#accept(Object)} to cancel activity task.
+ * @return cancellation handle. Invoke {@link io.temporal.workflow.Functions.Proc1#apply(Object)}
+ * to cancel activity task.
*/
- Consumer startChildWorkflow(
+ Functions.Proc1 startChildWorkflow(
StartChildWorkflowExecutionParameters parameters,
- Consumer executionCallback,
- BiConsumer, Exception> callback);
+ Functions.Proc1 executionCallback,
+ Functions.Proc2, Exception> callback);
- Consumer signalWorkflowExecution(
+ /**
+ * Signal a workflow execution by WorkflowId and optionally RunId.
+ *
+ * @param attributes signal information
+ * @param callback callback notified about the operation result
+ * @return cancellation handler that should be calle to cancel the operation.
+ */
+ Functions.Proc1 signalExternalWorkflowExecution(
SignalExternalWorkflowExecutionCommandAttributes.Builder attributes,
- BiConsumer callback);
+ Functions.Proc2 callback);
- Promise requestCancelWorkflowExecution(WorkflowExecution execution);
+ /**
+ * Request cancellation of a workflow execution by WorkflowId and optionally RunId.
+ *
+ * @param execution contains WorkflowId and optional RunId of the workflow to send request to.
+ * @param callback callback notified about the operation result
+ */
+ void requestCancelExternalWorkflowExecution(
+ WorkflowExecution execution, Functions.Proc2 callback);
void continueAsNewOnCompletion(ContinueAsNewWorkflowExecutionCommandAttributes attributes);
- Optional mutableSideEffect(
- String id, DataConverter dataConverter, Func1, Optional> func);
-
/**
* @return time of the {@link PollWorkflowTaskQueueResponse} start event of the workflow task
* being processed or replayed.
@@ -149,9 +177,11 @@ Optional mutableSideEffect(
* @param delay time-interval after which the Value becomes ready.
* @param callback Callback that is called with null parameter after the specified delay.
* CanceledException is passed as a parameter in case of a cancellation.
- * @return cancellation handle. Invoke {@link Consumer#accept(Object)} to cancel timer.
+ * @return cancellation handle. Invoke {@link io.temporal.workflow.Functions.Proc1#apply(Object)}
+ * to cancel timer.
*/
- Consumer createTimer(Duration delay, Consumer callback);
+ Functions.Proc1 newTimer(
+ Duration delay, Functions.Proc1 callback);
/**
* Executes the provided function once, records its result into the workflow history. The recorded
@@ -160,12 +190,45 @@ Optional mutableSideEffect(
* in replay. Common use case is to run some short non-deterministic code in workflow, like
* getting random number or new UUID. The only way to fail SideEffect is to throw {@link Error}
* which causes workflow task failure. The workflow task after timeout is rescheduled and
- * re-executed giving SideEffect another chance to succeed.
+ * re-executed giving SideEffect another chance to succeed. Use {@link
+ * #scheduleLocalActivityTask(ExecuteLocalActivityParameters, Functions.Proc2)} for executing
+ * operations that rely on non global dependencies and can fail.
*
* @param func function that is called once to return a value.
- * @return value of the side effect.
+ * @param callback function that accepts the result of the side effect.
+ */
+ void sideEffect(Func> func, Functions.Proc1> callback);
+
+ /**
+ * {@code mutableSideEffect} is similar to {@code sideEffect} in allowing calls of
+ * non-deterministic functions from workflow code.
+ *
+ * The difference between {@code mutableSideEffect} and {@code sideEffect} is that every new
+ * {@code sideEffect} call in non-replay mode results in a new marker event recorded into the
+ * history. However, {@code mutableSideEffect} only records a new marker if a value has changed.
+ * During the replay, {@code mutableSideEffect} will not execute the function again, but it will
+ * return the exact same value as it was returning during the non-replay run.
+ *
+ *
One good use case of {@code mutableSideEffect} is to access a dynamically changing config
+ * without breaking determinism. Even if called very frequently the config value is recorded only
+ * when it changes not causing any performance degradation due to a large history size.
+ *
+ *
Caution: do not use {@code mutableSideEffect} function to modify any workflow state. Only
+ * use the mutableSideEffect's return value.
+ *
+ * @param id id of the side effect call. It links multiple calls together. Calls with different
+ * ids are completely independent.
+ * @param func function that gets as input a result of a previous {@code mutableSideEffect} call.
+ * The function executes its business logic (like checking config value) and if value didn't
+ * change returns {@link Optional#empty()}. If value has changed and needs to be recorded in
+ * the history then it is returned instead.
+ * @param callback function that accepts the result of the mutable side effect which is current or
+ * cached result of the func.
*/
- Optional sideEffect(Func> func);
+ void mutableSideEffect(
+ String id,
+ Func1, Optional> func,
+ Functions.Proc1> callback);
/**
* GetVersion is used to safely perform backwards incompatible changes to workflow definitions. It
@@ -180,10 +243,12 @@ Optional mutableSideEffect(
* @param changeId identifier of a particular change
* @param minSupported min version supported for the change
* @param maxSupported max version supported for the change
- * @return version
+ * @param callback used to return version
*/
- int getVersion(String changeId, DataConverter dataConverter, int minSupported, int maxSupported);
+ void getVersion(
+ String changeId, int minSupported, int maxSupported, Functions.Proc1 callback);
+ /** Replay safe random. */
Random newRandom();
/** @return scope to be used for metrics reporting. */
@@ -195,5 +260,6 @@ Optional mutableSideEffect(
/** @return replay safe UUID */
UUID randomUUID();
+ /** Updates or inserts search attributes used to index workflows. */
void upsertSearchAttributes(SearchAttributes searchAttributes);
}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayWorkflowContextImpl.java b/src/main/java/io/temporal/internal/replay/ReplayWorkflowContextImpl.java
index 21dae0bab..5372ff74d 100644
--- a/src/main/java/io/temporal/internal/replay/ReplayWorkflowContextImpl.java
+++ b/src/main/java/io/temporal/internal/replay/ReplayWorkflowContextImpl.java
@@ -21,72 +21,65 @@
import com.uber.m3.tally.Scope;
import io.temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes;
+import io.temporal.api.command.v1.RequestCancelExternalWorkflowExecutionCommandAttributes;
+import io.temporal.api.command.v1.ScheduleActivityTaskCommandAttributes;
import io.temporal.api.command.v1.SignalExternalWorkflowExecutionCommandAttributes;
+import io.temporal.api.command.v1.StartTimerCommandAttributes;
import io.temporal.api.common.v1.Payloads;
import io.temporal.api.common.v1.SearchAttributes;
import io.temporal.api.common.v1.WorkflowExecution;
import io.temporal.api.common.v1.WorkflowType;
import io.temporal.api.enums.v1.WorkflowTaskFailedCause;
+import io.temporal.api.failure.v1.Failure;
import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.TimerFiredEventAttributes;
-import io.temporal.api.history.v1.UpsertWorkflowSearchAttributesEventAttributes;
import io.temporal.api.history.v1.WorkflowExecutionStartedEventAttributes;
import io.temporal.api.history.v1.WorkflowTaskFailedEventAttributes;
import io.temporal.common.context.ContextPropagator;
-import io.temporal.common.converter.DataConverter;
+import io.temporal.failure.CanceledFailure;
+import io.temporal.internal.common.ProtobufTimeUtils;
import io.temporal.internal.metrics.ReplayAwareScope;
-import io.temporal.internal.worker.LocalActivityWorker;
+import io.temporal.internal.statemachines.WorkflowStateMachines;
import io.temporal.internal.worker.SingleWorkerOptions;
+import io.temporal.workflow.Functions;
import io.temporal.workflow.Functions.Func;
import io.temporal.workflow.Functions.Func1;
-import io.temporal.workflow.Promise;
-import io.temporal.workflow.Workflow;
import java.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Random;
import java.util.UUID;
-import java.util.function.BiConsumer;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-final class ReplayWorkflowContextImpl implements ReplayWorkflowContext, HistoryEventHandler {
-
- private static final Logger log = LoggerFactory.getLogger(ReplayWorkflowContextImpl.class);
+/**
+ * TODO(maxim): callbacks usage is non consistent. It accepts Optional and Exception which can be
+ * null. Either switch both to Optional or both to nullable.
+ */
+final class ReplayWorkflowContextImpl implements ReplayWorkflowContext {
- private final ReplayActivityContext activityClient;
- private final ReplayChildWorkflowContext workflowClient;
- private final ReplayClockContext workflowClock;
private final WorkflowContext workflowContext;
private final Scope metricsScope;
private final boolean enableLoggingInReplay;
+ private final WorkflowStateMachines workflowStateMachines;
ReplayWorkflowContextImpl(
- CommandHelper commandHelper,
+ WorkflowStateMachines workflowStateMachines,
String namespace,
WorkflowExecutionStartedEventAttributes startedAttributes,
+ WorkflowExecution workflowExecution,
long runStartedTimestampMillis,
SingleWorkerOptions options,
- Scope metricsScope,
- BiFunction laTaskPoller,
- ReplayWorkflowExecutor workflowExecutor) {
- this.activityClient = new ReplayActivityContext(commandHelper);
+ Scope metricsScope) {
+ this.workflowStateMachines = workflowStateMachines;
this.workflowContext =
new WorkflowContext(
namespace,
- commandHelper.getTask(),
+ workflowExecution,
startedAttributes,
runStartedTimestampMillis,
options.getContextPropagators());
- this.workflowClient = new ReplayChildWorkflowContext(commandHelper, workflowContext);
- this.workflowClock =
- new ReplayClockContext(
- commandHelper, laTaskPoller, workflowExecutor, options.getDataConverter());
this.enableLoggingInReplay = options.getEnableLoggingInReplay();
- this.metricsScope = new ReplayAwareScope(metricsScope, this, workflowClock::currentTimeMillis);
+ this.metricsScope =
+ new ReplayAwareScope(metricsScope, this, workflowStateMachines::currentTimeMillis);
}
@Override
@@ -96,12 +89,12 @@ public boolean getEnableLoggingInReplay() {
@Override
public UUID randomUUID() {
- return workflowClient.randomUUID();
+ return workflowStateMachines.randomUUID();
}
@Override
public Random newRandom() {
- return workflowClient.newRandom();
+ return workflowStateMachines.newRandom();
}
@Override
@@ -214,183 +207,127 @@ public Map getPropagatedContexts() {
}
@Override
- public Consumer scheduleActivityTask(
- ExecuteActivityParameters parameters, BiConsumer, Exception> callback) {
- return activityClient.scheduleActivityTask(parameters, callback);
+ public Functions.Proc1 scheduleActivityTask(
+ ExecuteActivityParameters parameters, Functions.Proc2, Failure> callback) {
+ ScheduleActivityTaskCommandAttributes.Builder attributes = parameters.getAttributes();
+ if (attributes.getActivityId().isEmpty()) {
+ attributes.setActivityId(workflowStateMachines.randomUUID().toString());
+ }
+ Functions.Proc cancellationHandler =
+ workflowStateMachines.scheduleActivityTask(parameters, callback);
+ return (exception) -> cancellationHandler.apply();
}
@Override
- public Consumer scheduleLocalActivityTask(
+ public Functions.Proc scheduleLocalActivityTask(
ExecuteLocalActivityParameters parameters,
- BiConsumer, Exception> callback) {
- return workflowClock.scheduleLocalActivityTask(parameters, callback);
+ Functions.Proc2, Failure> callback) {
+ return workflowStateMachines.scheduleLocalActivityTask(parameters, callback);
}
@Override
- public Consumer startChildWorkflow(
+ public Functions.Proc1 startChildWorkflow(
StartChildWorkflowExecutionParameters parameters,
- Consumer executionCallback,
- BiConsumer, Exception> callback) {
- return workflowClient.startChildWorkflow(parameters, executionCallback, callback);
+ Functions.Proc1 executionCallback,
+ Functions.Proc2, Exception> callback) {
+ Functions.Proc cancellationHandler =
+ workflowStateMachines.startChildWorkflow(parameters, executionCallback, callback);
+ return (exception) -> cancellationHandler.apply();
}
@Override
- public Consumer signalWorkflowExecution(
+ public Functions.Proc1 signalExternalWorkflowExecution(
SignalExternalWorkflowExecutionCommandAttributes.Builder attributes,
- BiConsumer callback) {
- return workflowClient.signalWorkflowExecution(attributes, callback);
+ Functions.Proc2 callback) {
+ Functions.Proc cancellationHandler =
+ workflowStateMachines.signalExternalWorkflowExecution(attributes.build(), callback);
+ return (e) -> cancellationHandler.apply();
}
@Override
- public Promise requestCancelWorkflowExecution(WorkflowExecution execution) {
- workflowClient.requestCancelWorkflowExecution(execution);
- // TODO: Make promise return success or failure of the cancellation request.
- return Workflow.newPromise(null);
+ public void requestCancelExternalWorkflowExecution(
+ WorkflowExecution execution, Functions.Proc2 callback) {
+ RequestCancelExternalWorkflowExecutionCommandAttributes attributes =
+ RequestCancelExternalWorkflowExecutionCommandAttributes.newBuilder()
+ .setWorkflowId(execution.getWorkflowId())
+ .setRunId(execution.getRunId())
+ .build();
+ workflowStateMachines.requestCancelExternalWorkflowExecution(attributes, callback);
}
@Override
public void continueAsNewOnCompletion(
ContinueAsNewWorkflowExecutionCommandAttributes attributes) {
- workflowClient.continueAsNewOnCompletion(attributes);
- }
-
- void setReplayCurrentTimeMilliseconds(long replayCurrentTimeMilliseconds) {
- if (replayCurrentTimeMilliseconds < workflowClock.currentTimeMillis()) {
- if (log.isWarnEnabled()) {
- log.warn(
- "Trying to set workflow clock back from "
- + workflowClock.currentTimeMillis()
- + " to "
- + replayCurrentTimeMilliseconds
- + ". This will be a no-op.");
- }
- return;
- }
- workflowClock.setReplayCurrentTimeMilliseconds(replayCurrentTimeMilliseconds);
+ workflowContext.setContinueAsNewOnCompletion(attributes);
}
long getReplayCurrentTimeMilliseconds() {
- return workflowClock.currentTimeMillis();
+ return workflowStateMachines.currentTimeMillis();
}
@Override
public boolean isReplaying() {
- return workflowClock.isReplaying();
+ return workflowStateMachines.isReplaying();
}
@Override
- public Consumer createTimer(Duration delay, Consumer callback) {
- return workflowClock.createTimer(delay, callback);
+ public Functions.Proc1 newTimer(
+ Duration delay, Functions.Proc1 callback) {
+ if (delay.compareTo(Duration.ZERO) <= 0) {
+ callback.apply(null);
+ return (e) -> {};
+ }
+ StartTimerCommandAttributes attributes =
+ StartTimerCommandAttributes.newBuilder()
+ .setStartToFireTimeout(ProtobufTimeUtils.toProtoDuration(delay))
+ .setTimerId(workflowStateMachines.randomUUID().toString())
+ .build();
+ Functions.Proc cancellationHandler =
+ workflowStateMachines.newTimer(attributes, (event) -> handleTimerCallback(callback, event));
+ return (e) -> cancellationHandler.apply();
+ }
+
+ private void handleTimerCallback(Functions.Proc1 callback, HistoryEvent event) {
+ switch (event.getEventType()) {
+ case EVENT_TYPE_TIMER_FIRED:
+ {
+ callback.apply(null);
+ return;
+ }
+ case EVENT_TYPE_TIMER_CANCELED:
+ {
+ CanceledFailure exception = new CanceledFailure("Canceled by request");
+ callback.apply(exception);
+ return;
+ }
+ default:
+ throw new IllegalArgumentException("Unexpected event type: " + event.getEventType());
+ }
}
@Override
- public Optional sideEffect(Func> func) {
- return workflowClock.sideEffect(func);
+ public void sideEffect(
+ Func> func, Functions.Proc1> callback) {
+ workflowStateMachines.sideEffect(func, callback);
}
@Override
- public Optional mutableSideEffect(
- String id, DataConverter converter, Func1, Optional> func) {
- return workflowClock.mutableSideEffect(id, converter, func);
+ public void mutableSideEffect(
+ String id,
+ Func1, Optional> func,
+ Functions.Proc1> callback) {
+ workflowStateMachines.mutableSideEffect(id, func, callback);
}
@Override
- public int getVersion(
- String changeId, DataConverter converter, int minSupported, int maxSupported) {
- return workflowClock.getVersion(changeId, converter, minSupported, maxSupported);
+ public void getVersion(
+ String changeId, int minSupported, int maxSupported, Functions.Proc1 callback) {
+ workflowStateMachines.getVersion(changeId, minSupported, maxSupported, callback);
}
@Override
public long currentTimeMillis() {
- return workflowClock.currentTimeMillis();
- }
-
- void setReplaying(boolean replaying) {
- workflowClock.setReplaying(replaying);
- }
-
- @Override
- public void handleActivityTaskCanceled(HistoryEvent event) {
- activityClient.handleActivityTaskCanceled(event);
- }
-
- @Override
- public void handleActivityTaskCompleted(HistoryEvent event) {
- activityClient.handleActivityTaskCompleted(event);
- }
-
- @Override
- public void handleActivityTaskFailed(HistoryEvent event) {
- activityClient.handleActivityTaskFailed(event);
- }
-
- @Override
- public void handleActivityTaskTimedOut(HistoryEvent event) {
- activityClient.handleActivityTaskTimedOut(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionCancelRequested(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionCancelRequested(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionCanceled(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionCanceled(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionStarted(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionStarted(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionTimedOut(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionTimedOut(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionTerminated(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionTerminated(event);
- }
-
- @Override
- public void handleStartChildWorkflowExecutionFailed(HistoryEvent event) {
- workflowClient.handleStartChildWorkflowExecutionFailed(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionFailed(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionFailed(event);
- }
-
- @Override
- public void handleChildWorkflowExecutionCompleted(HistoryEvent event) {
- workflowClient.handleChildWorkflowExecutionCompleted(event);
- }
-
- @Override
- public void handleTimerFired(TimerFiredEventAttributes attributes) {
- workflowClock.handleTimerFired(attributes);
- }
-
- @Override
- public void handleTimerCanceled(HistoryEvent event) {
- workflowClock.handleTimerCanceled(event);
- }
-
- void handleSignalExternalWorkflowExecutionFailed(HistoryEvent event) {
- workflowClient.handleSignalExternalWorkflowExecutionFailed(event);
- }
-
- @Override
- public void handleExternalWorkflowExecutionSignaled(HistoryEvent event) {
- workflowClient.handleExternalWorkflowExecutionSignaled(event);
- }
-
- @Override
- public void handleMarkerRecorded(HistoryEvent event) {
- workflowClock.handleMarkerRecorded(event);
+ return workflowStateMachines.currentTimeMillis();
}
public void handleWorkflowTaskFailed(HistoryEvent event) {
@@ -402,30 +339,34 @@ public void handleWorkflowTaskFailed(HistoryEvent event) {
}
boolean startUnstartedLaTasks(Duration maxWaitAllowed) {
- return workflowClock.startUnstartedLaTasks(maxWaitAllowed);
+ // return workflowClock.startUnstartedLaTasks(maxWaitAllowed);
+ throw new UnsupportedOperationException("TODO");
}
int numPendingLaTasks() {
- return workflowClock.numPendingLaTasks();
+ // return workflowClock.numPendingLaTasks();
+ // TODO(maxim): implement
+ return 0;
}
void awaitTaskCompletion(Duration duration) throws InterruptedException {
- workflowClock.awaitTaskCompletion(duration);
+ // workflowClock.awaitTaskCompletion(duration);
+ throw new UnsupportedOperationException("TODO");
}
@Override
public void upsertSearchAttributes(SearchAttributes searchAttributes) {
- workflowClock.upsertSearchAttributes(searchAttributes);
+ // workflowClock.upsertSearchAttributes(searchAttributes);
workflowContext.mergeSearchAttributes(searchAttributes);
}
- @Override
- public void handleUpsertSearchAttributes(HistoryEvent event) {
- UpsertWorkflowSearchAttributesEventAttributes attr =
- event.getUpsertWorkflowSearchAttributesEventAttributes();
- if (attr != null) {
- SearchAttributes searchAttributes = attr.getSearchAttributes();
- workflowContext.mergeSearchAttributes(searchAttributes);
- }
- }
+ // @Override
+ // public void handleUpsertSearchAttributes(HistoryEvent event) {
+ // UpsertWorkflowSearchAttributesEventAttributes attr =
+ // event.getUpsertWorkflowSearchAttributesEventAttributes();
+ // if (attr != null) {
+ // SearchAttributes searchAttributes = attr.getSearchAttributes();
+ // workflowContext.mergeSearchAttributes(searchAttributes);
+ // }
+ // }
}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayWorkflowExecutor.java b/src/main/java/io/temporal/internal/replay/ReplayWorkflowExecutor.java
index 60f83fd01..35dea4cdf 100644
--- a/src/main/java/io/temporal/internal/replay/ReplayWorkflowExecutor.java
+++ b/src/main/java/io/temporal/internal/replay/ReplayWorkflowExecutor.java
@@ -19,262 +19,54 @@
package io.temporal.internal.replay;
-import static io.temporal.internal.metrics.MetricsTag.METRICS_TAGS_CALL_OPTIONS_KEY;
-import static io.temporal.worker.WorkflowErrorPolicy.FailWorkflow;
-
-import com.google.common.base.Throwables;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Timestamp;
import com.google.protobuf.util.Timestamps;
import com.uber.m3.tally.Scope;
-import com.uber.m3.tally.Stopwatch;
-import io.grpc.Status;
import io.temporal.api.command.v1.ContinueAsNewWorkflowExecutionCommandAttributes;
import io.temporal.api.common.v1.Payloads;
-import io.temporal.api.enums.v1.EventType;
-import io.temporal.api.enums.v1.QueryResultType;
-import io.temporal.api.history.v1.History;
import io.temporal.api.history.v1.HistoryEvent;
-import io.temporal.api.history.v1.TimerFiredEventAttributes;
+import io.temporal.api.history.v1.WorkflowExecutionCancelRequestedEventAttributes;
import io.temporal.api.history.v1.WorkflowExecutionSignaledEventAttributes;
-import io.temporal.api.history.v1.WorkflowExecutionStartedEventAttributes;
import io.temporal.api.query.v1.WorkflowQuery;
-import io.temporal.api.query.v1.WorkflowQueryResult;
-import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest;
-import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse;
-import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponse;
-import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponseOrBuilder;
-import io.temporal.common.converter.DataConverter;
import io.temporal.failure.CanceledFailure;
-import io.temporal.internal.common.GrpcRetryer;
import io.temporal.internal.common.ProtobufTimeUtils;
-import io.temporal.internal.common.RpcRetryOptions;
import io.temporal.internal.metrics.MetricsType;
-import io.temporal.internal.replay.HistoryHelper.WorkflowTaskEvents;
-import io.temporal.internal.worker.LocalActivityWorker;
-import io.temporal.internal.worker.SingleWorkerOptions;
+import io.temporal.internal.statemachines.WorkflowStateMachines;
import io.temporal.internal.worker.WorkflowExecutionException;
-import io.temporal.internal.worker.WorkflowTaskWithHistoryIterator;
-import io.temporal.serviceclient.WorkflowServiceStubs;
-import io.temporal.workflow.Functions;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
+import io.temporal.worker.WorkflowImplementationOptions;
import java.util.Optional;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-/**
- * Implements workflow executor that relies on replay of a workflow code. An instance of this class
- * is created per cached workflow run.
- */
-class ReplayWorkflowExecutor implements WorkflowExecutor {
+final class ReplayWorkflowExecutor {
- private static final int MAXIMUM_PAGE_SIZE = 10000;
+ private final ReplayWorkflow workflow;
+
+ private final Scope metricsScope;
+
+ private final WorkflowStateMachines workflowStateMachines;
- private final CommandHelper commandHelper;
private final ReplayWorkflowContextImpl context;
- private final WorkflowServiceStubs service;
- private final ReplayWorkflow workflow;
- private boolean cancelRequested;
+
private boolean completed;
+
private WorkflowExecutionException failure;
- private long wakeUpTime;
- private Consumer timerCancellationHandler;
- private final Scope metricsScope;
- private final Timestamp wfStartTime;
- private final WorkflowExecutionStartedEventAttributes startedEvent;
- private final Lock lock = new ReentrantLock();
- private final Consumer localActivityCompletionSink;
- private final Map queryResults = new HashMap<>();
- private final DataConverter converter;
- ReplayWorkflowExecutor(
- WorkflowServiceStubs service,
- String namespace,
+ private boolean cancelRequested;
+
+ public ReplayWorkflowExecutor(
ReplayWorkflow workflow,
- PollWorkflowTaskQueueResponse.Builder workflowTask,
- SingleWorkerOptions options,
Scope metricsScope,
- BiFunction laTaskPoller) {
- this.service = service;
+ WorkflowStateMachines workflowStateMachines,
+ ReplayWorkflowContextImpl context) {
this.workflow = workflow;
- this.commandHelper = new CommandHelper(workflowTask);
this.metricsScope = metricsScope;
- this.converter = options.getDataConverter();
-
- HistoryEvent firstEvent = workflowTask.getHistory().getEvents(0);
- if (!firstEvent.hasWorkflowExecutionStartedEventAttributes()) {
- throw new IllegalArgumentException(
- "First event in the history is not WorkflowExecutionStarted");
- }
- startedEvent = firstEvent.getWorkflowExecutionStartedEventAttributes();
- wfStartTime = firstEvent.getEventTime();
-
- context =
- new ReplayWorkflowContextImpl(
- commandHelper,
- namespace,
- startedEvent,
- Timestamps.toMillis(firstEvent.getEventTime()),
- options,
- metricsScope,
- laTaskPoller,
- this);
-
- localActivityCompletionSink =
- historyEvent -> {
- lock.lock();
- try {
- processEvent(historyEvent);
- } finally {
- lock.unlock();
- }
- };
+ this.workflowStateMachines = workflowStateMachines;
+ this.context = context;
}
- Lock getLock() {
- return lock;
+ public boolean isCompleted() {
+ return completed;
}
- private void handleWorkflowExecutionStarted(HistoryEvent event) {
- workflow.start(event, context);
- }
-
- private void processEvent(HistoryEvent event) {
- EventType eventType = event.getEventType();
- switch (eventType) {
- case EVENT_TYPE_ACTIVITY_TASK_CANCELED:
- context.handleActivityTaskCanceled(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_COMPLETED:
- context.handleActivityTaskCompleted(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_FAILED:
- context.handleActivityTaskFailed(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_STARTED:
- commandHelper.handleActivityTaskStarted(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_TIMED_OUT:
- context.handleActivityTaskTimedOut(event);
- break;
- case EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_CANCEL_REQUESTED:
- context.handleChildWorkflowExecutionCancelRequested(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_CANCELED:
- context.handleChildWorkflowExecutionCanceled(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED:
- context.handleChildWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_FAILED:
- context.handleChildWorkflowExecutionFailed(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_STARTED:
- context.handleChildWorkflowExecutionStarted(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TERMINATED:
- context.handleChildWorkflowExecutionTerminated(event);
- break;
- case EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_TIMED_OUT:
- context.handleChildWorkflowExecutionTimedOut(event);
- break;
- case EVENT_TYPE_WORKFLOW_TASK_COMPLETED:
- // NOOP
- break;
- case EVENT_TYPE_WORKFLOW_TASK_SCHEDULED:
- // NOOP
- break;
- case EVENT_TYPE_WORKFLOW_TASK_STARTED:
- throw new IllegalArgumentException("not expected");
- case EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT:
- // Handled in the processEvent(event)
- break;
- case EVENT_TYPE_EXTERNAL_WORKFLOW_EXECUTION_SIGNALED:
- context.handleExternalWorkflowExecutionSignaled(event);
- break;
- case EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_FAILED:
- context.handleStartChildWorkflowExecutionFailed(event);
- break;
- case EVENT_TYPE_TIMER_FIRED:
- handleTimerFired(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_CANCEL_REQUESTED:
- handleWorkflowExecutionCancelRequested(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED:
- handleWorkflowExecutionSignaled(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_STARTED:
- handleWorkflowExecutionStarted(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_TERMINATED:
- // NOOP
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT:
- commandHelper.handleWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_SCHEDULED:
- d:
- commandHelper.handleActivityTaskScheduled(event);
- break;
- case EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED:
- commandHelper.handleActivityTaskCancelRequested(event);
- break;
- case EVENT_TYPE_MARKER_RECORDED:
- context.handleMarkerRecorded(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED:
- commandHelper.handleWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_FAILED:
- commandHelper.handleWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_CANCELED:
- commandHelper.handleWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW:
- commandHelper.handleWorkflowExecutionCompleted(event);
- break;
- case EVENT_TYPE_TIMER_STARTED:
- commandHelper.handleTimerStarted(event);
- break;
- case EVENT_TYPE_TIMER_CANCELED:
- context.handleTimerCanceled(event);
- break;
- case EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED:
- commandHelper.handleSignalExternalWorkflowExecutionInitiated(event);
- break;
- case EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED:
- context.handleSignalExternalWorkflowExecutionFailed(event);
- break;
- case EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED:
- commandHelper.handleRequestCancelExternalWorkflowExecutionInitiated(event);
- break;
- case EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED:
- commandHelper.handleRequestCancelExternalWorkflowExecutionFailed(event);
- break;
- case EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED:
- commandHelper.handleStartChildWorkflowExecutionInitiated(event);
- break;
- case EVENT_TYPE_WORKFLOW_TASK_FAILED:
- context.handleWorkflowTaskFailed(event);
- break;
- case EVENT_TYPE_UPSERT_WORKFLOW_SEARCH_ATTRIBUTES:
- context.handleUpsertSearchAttributes(event);
- break;
- }
- }
-
- private void eventLoop() {
+ public void eventLoop() {
if (completed) {
return;
}
@@ -295,96 +87,49 @@ private void eventLoop() {
failure = workflow.mapUnexpectedException(e);
completed = true;
}
- }
-
- private void mayBeCompleteWorkflow() {
if (completed) {
completeWorkflow();
- } else {
- updateTimers();
}
}
private void completeWorkflow() {
if (failure != null) {
- commandHelper.failWorkflowExecution(failure);
+ workflowStateMachines.newFailWorkflow(failure.getFailure());
metricsScope.counter(MetricsType.WORKFLOW_FAILED_COUNTER).inc(1);
} else if (cancelRequested) {
- commandHelper.cancelWorkflowExecution();
- metricsScope.counter(MetricsType.WORKFLOW_CANCELLED_COUNTER).inc(1);
+ workflowStateMachines.newCancelWorkflow();
+ metricsScope.counter(MetricsType.WORKFLOW_CANCELED_COUNTER).inc(1);
} else {
ContinueAsNewWorkflowExecutionCommandAttributes attributes =
context.getContinueAsNewOnCompletion();
if (attributes != null) {
- commandHelper.continueAsNewWorkflowExecution(attributes);
+ workflowStateMachines.newContinueAsNewWorkflow(attributes);
metricsScope.counter(MetricsType.WORKFLOW_CONTINUE_AS_NEW_COUNTER).inc(1);
} else {
Optional workflowOutput = workflow.getOutput();
- commandHelper.completeWorkflowExecution(workflowOutput);
+ workflowStateMachines.newCompleteWorkflow(workflowOutput);
metricsScope.counter(MetricsType.WORKFLOW_COMPLETED_COUNTER).inc(1);
}
}
com.uber.m3.util.Duration d =
- ProtobufTimeUtils.ToM3Duration(
- Timestamps.fromMillis(System.currentTimeMillis()), wfStartTime);
+ ProtobufTimeUtils.toM3Duration(
+ Timestamps.fromMillis(System.currentTimeMillis()),
+ Timestamps.fromMillis(context.getRunStartedTimestampMillis()));
metricsScope.timer(MetricsType.WORKFLOW_E2E_LATENCY).record(d);
}
- private void updateTimers() {
- long nextWakeUpTime = workflow.getNextWakeUpTime();
- if (nextWakeUpTime == 0) {
- if (timerCancellationHandler != null) {
- timerCancellationHandler.accept(null);
- timerCancellationHandler = null;
- }
- wakeUpTime = nextWakeUpTime;
- return;
- }
- if (wakeUpTime == nextWakeUpTime && timerCancellationHandler != null) {
- return; // existing timer
- }
- long delayMilliseconds = nextWakeUpTime - context.currentTimeMillis();
- if (delayMilliseconds < 0) {
- throw new IllegalStateException("Negative delayMilliseconds=" + delayMilliseconds);
- }
-
- if (timerCancellationHandler != null) {
- timerCancellationHandler.accept(null);
- timerCancellationHandler = null;
- }
- wakeUpTime = nextWakeUpTime;
- timerCancellationHandler =
- context.createTimer(
- Duration.ofMillis(delayMilliseconds),
- (t) -> {
- // Intentionally left empty.
- // Timer ensures that a workflow task is scheduled at the time workflow can make
- // progress.
- // But no specific timer related action is necessary as Workflow.sleep is just a
- // Workflow.await with a time based condition.
- });
- }
-
- private void handleWorkflowExecutionCancelRequested(HistoryEvent event) {
+ public void handleWorkflowExecutionCancelRequested(HistoryEvent event) {
+ WorkflowExecutionCancelRequestedEventAttributes attributes =
+ event.getWorkflowExecutionCancelRequestedEventAttributes();
context.setCancelRequested(true);
- String cause = event.getWorkflowExecutionCancelRequestedEventAttributes().getCause();
+ String cause = attributes.getCause();
workflow.cancel(cause);
cancelRequested = true;
}
- private void handleTimerFired(HistoryEvent event) {
- TimerFiredEventAttributes attributes = event.getTimerFiredEventAttributes();
- String timerId = attributes.getTimerId();
- if (timerId.equals(CommandHelper.FORCE_IMMEDIATE_WORKFLOW_TASK_TIMER)) {
- return;
- }
- context.handleTimerFired(attributes);
- }
-
- private void handleWorkflowExecutionSignaled(HistoryEvent event) {
- assert (event.getEventType() == EventType.EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED);
- final WorkflowExecutionSignaledEventAttributes signalAttributes =
+ public void handleWorkflowExecutionSignaled(HistoryEvent event) {
+ WorkflowExecutionSignaledEventAttributes signalAttributes =
event.getWorkflowExecutionSignaledEventAttributes();
if (completed) {
throw new IllegalStateException("Signal received after workflow is closed.");
@@ -394,334 +139,23 @@ private void handleWorkflowExecutionSignaled(HistoryEvent event) {
this.workflow.handleSignal(signalAttributes.getSignalName(), input, event.getEventId());
}
- @Override
- public WorkflowTaskResult handleWorkflowTask(PollWorkflowTaskQueueResponseOrBuilder workflowTask)
- throws Throwable {
- lock.lock();
- try {
- queryResults.clear();
- boolean forceCreateNewWorkflowTask = handleWorkflowTaskImpl(workflowTask, null);
- return new WorkflowTaskResult(
- commandHelper.getCommands(), queryResults, forceCreateNewWorkflowTask, completed);
- } finally {
- lock.unlock();
- }
+ public Optional query(WorkflowQuery query) {
+ return workflow.query(query);
}
- // Returns boolean to indicate whether we need to force create new workflow task for local
- // activity heartbeating.
- private boolean handleWorkflowTaskImpl(
- PollWorkflowTaskQueueResponseOrBuilder workflowTask, Functions.Proc legacyQueryCallback)
- throws Throwable {
- boolean forceCreateNewWorkflowTask = false;
- Stopwatch sw = metricsScope.timer(MetricsType.WORKFLOW_TASK_REPLAY_LATENCY).start();
- boolean timerStopped = false;
- try {
- long startTime = System.currentTimeMillis();
- WorkflowTaskWithHistoryIterator workflowTaskWithHistoryIterator =
- new WorkflowTaskWithHistoryIteratorImpl(
- workflowTask,
- ProtobufTimeUtils.ToJavaDuration(startedEvent.getWorkflowTaskTimeout()));
- HistoryHelper historyHelper =
- new HistoryHelper(
- workflowTaskWithHistoryIterator, context.getReplayCurrentTimeMilliseconds());
- Iterator iterator = historyHelper.getIterator();
- if (commandHelper.getLastStartedEventId() > 0
- && commandHelper.getLastStartedEventId() != historyHelper.getPreviousStartedEventId()
- && workflowTask.getHistory().getEventsCount() > 0) {
- throw new IllegalStateException(
- String.format(
- "ReplayWorkflowExecutor processed up to event id %d. History's previous started event id is %d",
- commandHelper.getLastStartedEventId(), historyHelper.getPreviousStartedEventId()));
- }
- while (iterator.hasNext()) {
- WorkflowTaskEvents taskEvents = iterator.next();
- if (!timerStopped && !taskEvents.isReplay()) {
- sw.stop();
- timerStopped = true;
- }
- context.setReplaying(taskEvents.isReplay());
- context.setReplayCurrentTimeMilliseconds(taskEvents.getReplayCurrentTimeMilliseconds());
-
- commandHelper.handleWorkflowTaskStartedEvent(taskEvents);
- // Markers must be cached first as their data is needed when processing events.
- for (HistoryEvent event : taskEvents.getMarkers()) {
- if (!event
- .getMarkerRecordedEventAttributes()
- .getMarkerName()
- .equals(ReplayClockContext.LOCAL_ACTIVITY_MARKER_NAME)) {
- processEvent(event);
- }
- }
-
- for (HistoryEvent event : taskEvents.getEvents()) {
- processEvent(event);
- }
-
- forceCreateNewWorkflowTask =
- processEventLoop(
- startTime,
- ProtobufTimeUtils.ToJavaDuration(startedEvent.getWorkflowTaskTimeout()),
- taskEvents,
- workflowTask.hasQuery());
-
- mayBeCompleteWorkflow();
- if (taskEvents.isReplay()) {
- commandHelper.notifyCommandSent();
- }
- // Updates state machines with results of the previous commands
- for (HistoryEvent event : taskEvents.getCommandEvents()) {
- processEvent(event);
- }
- // Reset state to before running the event loop
- commandHelper.handleWorkflowTaskStartedEvent(taskEvents);
- }
- return forceCreateNewWorkflowTask;
- } catch (Error e) {
- if (this.workflow.getWorkflowImplementationOptions().getWorkflowErrorPolicy()
- == FailWorkflow) {
- // fail workflow
- failure = workflow.mapError(e);
- completed = true;
- completeWorkflow();
- return false;
- } else {
- metricsScope.counter(MetricsType.WORKFLOW_TASK_NO_COMPLETION_COUNTER).inc(1);
- // fail workflow task, not a workflow
- throw e;
- }
- } finally {
- if (!timerStopped) {
- sw.stop();
- }
- Map queries = workflowTask.getQueriesMap();
- for (Map.Entry entry : queries.entrySet()) {
- WorkflowQuery query = entry.getValue();
- try {
- Optional queryResult = workflow.query(query);
- WorkflowQueryResult.Builder result =
- WorkflowQueryResult.newBuilder()
- .setResultType(QueryResultType.QUERY_RESULT_TYPE_ANSWERED);
- if (queryResult.isPresent()) {
- result.setAnswer(queryResult.get());
- }
- queryResults.put(entry.getKey(), result.build());
- } catch (Exception e) {
- String stackTrace = Throwables.getStackTraceAsString(e);
- queryResults.put(
- entry.getKey(),
- WorkflowQueryResult.newBuilder()
- .setResultType(QueryResultType.QUERY_RESULT_TYPE_FAILED)
- .setErrorMessage(e.getMessage())
- .setAnswer(converter.toPayloads(stackTrace).get())
- .build());
- }
- }
- if (legacyQueryCallback != null) {
- legacyQueryCallback.apply();
- }
- if (completed) {
- close();
- }
- }
+ public WorkflowImplementationOptions getWorkflowImplementationOptions() {
+ return workflow.getWorkflowImplementationOptions();
}
- private boolean processEventLoop(
- long startTime, Duration workflowTaskTimeout, WorkflowTaskEvents taskEvents, boolean isQuery)
- throws Throwable {
- eventLoop();
-
- if (taskEvents.isReplay() || isQuery) {
- return replayLocalActivities(taskEvents);
- } else {
- return executeLocalActivities(startTime, workflowTaskTimeout);
- }
+ public WorkflowExecutionException mapError(Throwable e) {
+ return workflow.mapError(e);
}
- private boolean replayLocalActivities(WorkflowTaskEvents taskEvents) throws Throwable {
- List localActivityMarkers = new ArrayList<>();
- for (HistoryEvent event : taskEvents.getMarkers()) {
- if (event
- .getMarkerRecordedEventAttributes()
- .getMarkerName()
- .equals(ReplayClockContext.LOCAL_ACTIVITY_MARKER_NAME)) {
- localActivityMarkers.add(event);
- }
- }
-
- if (localActivityMarkers.isEmpty()) {
- return false;
- }
-
- int processed = 0;
- while (context.numPendingLaTasks() > 0) {
- int numTasks = context.numPendingLaTasks();
- for (HistoryEvent event : localActivityMarkers) {
- processEvent(event);
- }
-
- eventLoop();
-
- processed += numTasks;
- if (processed == localActivityMarkers.size()) {
- return false;
- }
- }
- return false;
- }
-
- // Return whether we would need a new workflow task immediately.
- private boolean executeLocalActivities(long startTime, Duration workflowTaskTimeout) {
- Duration maxProcessingTime = workflowTaskTimeout.multipliedBy(4).dividedBy(5);
-
- while (context.numPendingLaTasks() > 0) {
- Duration processingTime = Duration.ofMillis(System.currentTimeMillis() - startTime);
- Duration maxWaitAllowed = maxProcessingTime.minus(processingTime);
-
- boolean started = context.startUnstartedLaTasks(maxWaitAllowed);
- if (!started) {
- // We were not able to send the current batch of la tasks before deadline.
- // Return true to indicate that we need a new workflow task immediately.
- return true;
- }
-
- try {
- context.awaitTaskCompletion(maxWaitAllowed);
- } catch (InterruptedException e) {
- return true;
- }
-
- eventLoop();
-
- if (context.numPendingLaTasks() == 0) {
- return false;
- }
-
- // Break local activity processing loop if we almost reach workflow task timeout.
- processingTime = Duration.ofMillis(System.currentTimeMillis() - startTime);
- if (processingTime.compareTo(maxProcessingTime) > 0) {
- return true;
- }
- }
- return false;
- }
-
- Duration getWorkflowTaskTimeout() {
- return ProtobufTimeUtils.ToJavaDuration(startedEvent.getWorkflowTaskTimeout());
- }
-
- @Override
public void close() {
- lock.lock();
- try {
- workflow.close();
- } finally {
- lock.unlock();
- }
+ workflow.close();
}
- @Override
- public Optional handleQueryWorkflowTask(
- PollWorkflowTaskQueueResponseOrBuilder response, WorkflowQuery query) throws Throwable {
- lock.lock();
- try {
- AtomicReference> result = new AtomicReference<>();
- handleWorkflowTaskImpl(response, () -> result.set(workflow.query(query)));
- return result.get();
- } finally {
- lock.unlock();
- }
- }
-
- public Consumer getLocalActivityCompletionSink() {
- return localActivityCompletionSink;
- }
-
- private class WorkflowTaskWithHistoryIteratorImpl implements WorkflowTaskWithHistoryIterator {
-
- private final Duration retryServiceOperationInitialInterval = Duration.ofMillis(200);
- private final Duration retryServiceOperationMaxInterval = Duration.ofSeconds(4);
- private final Duration paginationStart = Duration.ofMillis(System.currentTimeMillis());
- private Duration workflowTaskTimeout;
-
- private final PollWorkflowTaskQueueResponseOrBuilder task;
- private Iterator current;
- private ByteString nextPageToken;
-
- WorkflowTaskWithHistoryIteratorImpl(
- PollWorkflowTaskQueueResponseOrBuilder task, Duration workflowTaskTimeout) {
- this.task = Objects.requireNonNull(task);
- this.workflowTaskTimeout = Objects.requireNonNull(workflowTaskTimeout);
-
- History history = task.getHistory();
- current = history.getEventsList().iterator();
- nextPageToken = task.getNextPageToken();
- }
-
- @Override
- public PollWorkflowTaskQueueResponseOrBuilder getWorkflowTask() {
- lock.lock();
- try {
- return task;
- } finally {
- lock.unlock();
- }
- }
-
- @Override
- public Iterator getHistory() {
- return new Iterator() {
- @Override
- public boolean hasNext() {
- return current.hasNext() || !nextPageToken.isEmpty();
- }
-
- @Override
- public HistoryEvent next() {
- if (current.hasNext()) {
- return current.next();
- }
-
- Duration passed = Duration.ofMillis(System.currentTimeMillis()).minus(paginationStart);
- Duration expiration = workflowTaskTimeout.minus(passed);
- if (expiration.isZero() || expiration.isNegative()) {
- throw Status.DEADLINE_EXCEEDED
- .withDescription(
- "getWorkflowExecutionHistory pagination took longer than workflow task timeout")
- .asRuntimeException();
- }
- RpcRetryOptions retryOptions =
- RpcRetryOptions.newBuilder()
- .setExpiration(expiration)
- .setInitialInterval(retryServiceOperationInitialInterval)
- .setMaximumInterval(retryServiceOperationMaxInterval)
- .build();
-
- GetWorkflowExecutionHistoryRequest request =
- GetWorkflowExecutionHistoryRequest.newBuilder()
- .setNamespace(context.getNamespace())
- .setExecution(task.getWorkflowExecution())
- .setMaximumPageSize(MAXIMUM_PAGE_SIZE)
- .setNextPageToken(nextPageToken)
- .build();
-
- try {
- GetWorkflowExecutionHistoryResponse r =
- GrpcRetryer.retryWithResult(
- retryOptions,
- () ->
- service
- .blockingStub()
- .withOption(METRICS_TAGS_CALL_OPTIONS_KEY, metricsScope)
- .getWorkflowExecutionHistory(request));
- current = r.getHistory().getEventsList().iterator();
- nextPageToken = r.getNextPageToken();
- } catch (Exception e) {
- throw new Error(e);
- }
- return current.next();
- }
- };
- }
+ public void start(HistoryEvent startWorkflowEvent) {
+ workflow.start(startWorkflowEvent, context);
}
}
diff --git a/src/main/java/io/temporal/internal/replay/ReplayWorkflowRunTaskHandler.java b/src/main/java/io/temporal/internal/replay/ReplayWorkflowRunTaskHandler.java
new file mode 100644
index 000000000..50f277484
--- /dev/null
+++ b/src/main/java/io/temporal/internal/replay/ReplayWorkflowRunTaskHandler.java
@@ -0,0 +1,335 @@
+/*
+ * Copyright (C) 2020 Temporal Technologies, Inc. All Rights Reserved.
+ *
+ * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
+ *
+ * Modifications copyright (C) 2017 Uber Technologies, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"). You may not
+ * use this file except in compliance with the License. A copy of the License is
+ * located at
+ *
+ * http://aws.amazon.com/apache2.0
+ *
+ * or in the "license" file accompanying this file. This file is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
+ * express or implied. See the License for the specific language governing
+ * permissions and limitations under the License.
+ */
+
+package io.temporal.internal.replay;
+
+import static io.temporal.internal.common.ProtobufTimeUtils.toJavaDuration;
+import static io.temporal.worker.WorkflowErrorPolicy.FailWorkflow;
+
+import com.google.common.base.Throwables;
+import com.google.protobuf.util.Durations;
+import com.google.protobuf.util.Timestamps;
+import com.uber.m3.tally.Scope;
+import com.uber.m3.tally.Stopwatch;
+import io.temporal.api.command.v1.Command;
+import io.temporal.api.common.v1.Payloads;
+import io.temporal.api.enums.v1.QueryResultType;
+import io.temporal.api.history.v1.HistoryEvent;
+import io.temporal.api.history.v1.WorkflowExecutionStartedEventAttributes;
+import io.temporal.api.query.v1.WorkflowQuery;
+import io.temporal.api.query.v1.WorkflowQueryResult;
+import io.temporal.api.workflowservice.v1.PollWorkflowTaskQueueResponseOrBuilder;
+import io.temporal.common.converter.DataConverter;
+import io.temporal.internal.metrics.MetricsType;
+import io.temporal.internal.statemachines.EntityManagerListener;
+import io.temporal.internal.statemachines.WorkflowStateMachines;
+import io.temporal.internal.worker.ActivityTaskHandler;
+import io.temporal.internal.worker.LocalActivityWorker;
+import io.temporal.internal.worker.SingleWorkerOptions;
+import io.temporal.serviceclient.WorkflowServiceStubs;
+import io.temporal.worker.WorkflowImplementationOptions;
+import io.temporal.workflow.Functions;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.BiFunction;
+
+/**
+ * Implements workflow executor that relies on replay of a workflow code. An instance of this class
+ * is created per cached workflow run.
+ */
+class ReplayWorkflowRunTaskHandler implements WorkflowRunTaskHandler {
+
+ /** Force new decision task after workflow task timeout multiplied by this coefficient. */
+ public static final double FORCED_DECISION_TIME_COEFFICIENT = 4d / 5d;
+
+ private final WorkflowServiceStubs service;
+
+ private final String namespace;
+
+ private final Scope metricsScope;
+
+ private final WorkflowExecutionStartedEventAttributes startedEvent;
+
+ private final Lock lock = new ReentrantLock();
+
+ private final Functions.Proc1