From f922c0655c0b78898f87c4225edd5b94e3b57530 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 16 Aug 2023 08:15:02 -0600 Subject: [PATCH] Fix bug in VersionStateMachine causing wrong version to get returned (#1841) Fix a bug in VersionStateMachine causing DEFAULT_VERSION to get returned incorrectly if the same change ID was read multiple times in a Workflow. --- .../statemachines/VersionStateMachine.java | 19 +- .../statemachines/WorkflowStateMachines.java | 22 +- .../GetVersionDefaultInSignalTest.java | 100 ++++++ .../GetVersionMultipleCallsDefaultTest.java | 95 ++++++ .../GetVersionMultipleCallsTest.java | 91 ++++++ .../GetVersionOutOfOrderFailTest.java | 112 +++++++ .../testGetVersionMultipleCallsHistory.json | 304 ++++++++++++++++++ ...GetVersionMultipleCallsHistoryDefault.json | 267 +++++++++++++++ .../testGetVersionOutOfOrderFail.json | 204 ++++++++++++ 9 files changed, 1199 insertions(+), 15 deletions(-) create mode 100644 temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionDefaultInSignalTest.java create mode 100644 temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsDefaultTest.java create mode 100644 temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsTest.java create mode 100644 temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionOutOfOrderFailTest.java create mode 100644 temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistory.json create mode 100644 temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistoryDefault.json create mode 100644 temporal-sdk/src/test/resources/testGetVersionOutOfOrderFail.json diff --git a/temporal-sdk/src/main/java/io/temporal/internal/statemachines/VersionStateMachine.java b/temporal-sdk/src/main/java/io/temporal/internal/statemachines/VersionStateMachine.java index 7577408cd..ac934390e 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/statemachines/VersionStateMachine.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/statemachines/VersionStateMachine.java @@ -368,13 +368,26 @@ private VersionStateMachine( this.commandSink = Objects.requireNonNull(commandSink); this.stateMachineSink = stateMachineSink; } - - public State getVersion( + /** + * Get the version for this state machine. + * + * @param minSupported min version supported for the change + * @param maxSupported max version supported for the change + * @param callback used to return version + * @return True if the identifier is not present in history + */ + public boolean getVersion( int minSupported, int maxSupported, Functions.Proc2 callback) { InvocationStateMachine ism = new InvocationStateMachine(minSupported, maxSupported, callback); ism.explicitEvent(ExplicitEvent.CHECK_EXECUTION_STATE); ism.explicitEvent(ExplicitEvent.SCHEDULE); - return ism.getState(); + // If the state is SKIPPED_REPLAYING that means we: + // 1. Are replaying + // 2. We don't have a preloadedVersion + // This means either this version marker did not exist in the original execution or + // the version marker did exist, but was in an earlier WFT. If the version marker was in a + // previous WFT then the version field should have a value. + return !(ism.getState() == VersionStateMachine.State.SKIPPED_REPLAYING && version == null); } public void handleNonMatchingEvent(HistoryEvent event) { diff --git a/temporal-sdk/src/main/java/io/temporal/internal/statemachines/WorkflowStateMachines.java b/temporal-sdk/src/main/java/io/temporal/internal/statemachines/WorkflowStateMachines.java index d5dbc6257..47a8bc74f 100644 --- a/temporal-sdk/src/main/java/io/temporal/internal/statemachines/WorkflowStateMachines.java +++ b/temporal-sdk/src/main/java/io/temporal/internal/statemachines/WorkflowStateMachines.java @@ -926,18 +926,16 @@ public boolean getVersion( return VersionStateMachine.newInstance( changeId, this::isReplaying, commandSink, stateMachineSink); }); - VersionStateMachine.State state = - stateMachine.getVersion( - minSupported, - maxSupported, - (v, e) -> { - callback.apply(v, e); - // without this getVersion call will trigger the end of WFT, - // instead we want to prepare subsequent commands and unblock the execution one more - // time. - eventLoop(); - }); - return state != VersionStateMachine.State.SKIPPED_REPLAYING; + return stateMachine.getVersion( + minSupported, + maxSupported, + (v, e) -> { + callback.apply(v, e); + // without this getVersion call will trigger the end of WFT, + // instead we want to prepare subsequent commands and unblock the execution one more + // time. + eventLoop(); + }); } public List takeLocalActivityRequests() { diff --git a/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionDefaultInSignalTest.java b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionDefaultInSignalTest.java new file mode 100644 index 000000000..0cfecde86 --- /dev/null +++ b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionDefaultInSignalTest.java @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.temporal.workflow.versionTests; + +import static org.junit.Assert.assertEquals; + +import io.temporal.client.WorkflowClient; +import io.temporal.client.WorkflowStub; +import io.temporal.testing.internal.SDKTestOptions; +import io.temporal.testing.internal.SDKTestWorkflowRule; +import io.temporal.worker.WorkerOptions; +import io.temporal.workflow.Workflow; +import io.temporal.workflow.shared.TestActivities.TestActivitiesImpl; +import io.temporal.workflow.shared.TestActivities.VariousTestActivities; +import io.temporal.workflow.shared.TestWorkflows; +import java.time.Duration; +import org.junit.Rule; +import org.junit.Test; + +public class GetVersionDefaultInSignalTest { + + @Rule + public SDKTestWorkflowRule testWorkflowRule = + SDKTestWorkflowRule.newBuilder() + .setWorkflowTypes(TestGetVersionWorkflowImpl.class) + .setActivityImplementations(new TestActivitiesImpl()) + // Forcing a replay. Full history arrived from a normal queue causing a replay. + .setWorkerOptions( + WorkerOptions.newBuilder() + .setStickyQueueScheduleToStartTimeout(Duration.ZERO) + .build()) + .build(); + + @Test + public void testGetVersionDefaultInSignal() throws InterruptedException { + TestWorkflows.TestSignaledWorkflow workflow = + testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflows.TestSignaledWorkflow.class); + WorkflowClient.start(workflow::execute); + + WorkflowStub workflowStub = WorkflowStub.fromTyped(workflow); + SDKTestWorkflowRule.waitForOKQuery(workflowStub); + + workflow.signal(testWorkflowRule.getTaskQueue()); + workflow.signal(testWorkflowRule.getTaskQueue()); + testWorkflowRule.invalidateWorkflowCache(); + workflow.signal(testWorkflowRule.getTaskQueue()); + + String result = workflowStub.getResult(String.class); + assertEquals("1", result); + } + + public static class TestGetVersionWorkflowImpl implements TestWorkflows.TestSignaledWorkflow { + int signalCounter = 0; + + @Override + public String execute() { + int version = + io.temporal.workflow.Workflow.getVersion( + "testMarker", io.temporal.workflow.Workflow.DEFAULT_VERSION, 1); + Workflow.await(() -> signalCounter >= 3); + return String.valueOf(version); + } + + @Override + public void signal(String taskQueue) { + VariousTestActivities testActivities = + Workflow.newActivityStub( + VariousTestActivities.class, + SDKTestOptions.newActivityOptionsForTaskQueue(taskQueue)); + + int version = + io.temporal.workflow.Workflow.getVersion( + "testMarker", io.temporal.workflow.Workflow.DEFAULT_VERSION, 1); + if (version == 1) { + testActivities.activity1(1); + } else { + testActivities.activity(); + } + signalCounter++; + } + } +} diff --git a/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsDefaultTest.java b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsDefaultTest.java new file mode 100644 index 000000000..fd8d36c71 --- /dev/null +++ b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsDefaultTest.java @@ -0,0 +1,95 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.temporal.workflow.versionTests; + +import static org.junit.Assert.assertEquals; + +import io.temporal.testing.WorkflowReplayer; +import io.temporal.testing.internal.SDKTestOptions; +import io.temporal.testing.internal.SDKTestWorkflowRule; +import io.temporal.worker.WorkerOptions; +import io.temporal.workflow.Workflow; +import io.temporal.workflow.shared.TestActivities.TestActivitiesImpl; +import io.temporal.workflow.shared.TestActivities.VariousTestActivities; +import io.temporal.workflow.shared.TestWorkflows.TestWorkflow1; +import io.temporal.workflow.unsafe.WorkflowUnsafe; +import java.time.Duration; +import org.junit.Rule; +import org.junit.Test; + +public class GetVersionMultipleCallsDefaultTest { + @Rule + public SDKTestWorkflowRule testWorkflowRule = + SDKTestWorkflowRule.newBuilder() + .setWorkflowTypes(TestGetVersionWorkflowImpl.class) + .setActivityImplementations(new TestActivitiesImpl()) + // Forcing a replay. Full history arrived from a normal queue causing a replay. + .setWorkerOptions( + WorkerOptions.newBuilder() + .setStickyQueueScheduleToStartTimeout(Duration.ZERO) + .build()) + .build(); + + @Test + public void testGetVersionMultipleCallsDefault() { + TestWorkflow1 workflowStub = + testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class); + String result = workflowStub.execute(testWorkflowRule.getTaskQueue()); + assertEquals("activity1", result); + } + + @Test + public void testGetVersionMultipleCallsReplay() throws Exception { + WorkflowReplayer.replayWorkflowExecutionFromResource( + "testGetVersionMultipleCallsHistoryDefault.json", + GetVersionMultipleCallsDefaultTest.TestGetVersionWorkflowImpl.class); + } + + public static class TestGetVersionWorkflowImpl implements TestWorkflow1 { + @Override + public String execute(String taskQueue) { + VariousTestActivities testActivities = + Workflow.newActivityStub( + VariousTestActivities.class, + SDKTestOptions.newActivityOptionsForTaskQueue(taskQueue)); + System.out.println("Calling getVersion for the fist time"); + if (WorkflowUnsafe.isReplaying()) { + int version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, Workflow.DEFAULT_VERSION); + + // Try again in the same WFT + version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, Workflow.DEFAULT_VERSION); + } + + // Create a new WFT by sleeping + Workflow.sleep(1000); + int version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, Workflow.DEFAULT_VERSION); + + String result = "activity" + testActivities.activity1(1); + + version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, Workflow.DEFAULT_VERSION); + return result; + } + } +} diff --git a/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsTest.java b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsTest.java new file mode 100644 index 000000000..a999d644a --- /dev/null +++ b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionMultipleCallsTest.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.temporal.workflow.versionTests; + +import static org.junit.Assert.assertEquals; + +import io.temporal.testing.WorkflowReplayer; +import io.temporal.testing.internal.SDKTestOptions; +import io.temporal.testing.internal.SDKTestWorkflowRule; +import io.temporal.worker.WorkerOptions; +import io.temporal.workflow.Workflow; +import io.temporal.workflow.shared.TestActivities.TestActivitiesImpl; +import io.temporal.workflow.shared.TestActivities.VariousTestActivities; +import io.temporal.workflow.shared.TestWorkflows.TestWorkflow1; +import java.time.Duration; +import org.junit.Rule; +import org.junit.Test; + +public class GetVersionMultipleCallsTest { + @Rule + public SDKTestWorkflowRule testWorkflowRule = + SDKTestWorkflowRule.newBuilder() + .setWorkflowTypes(TestGetVersionWorkflowImpl.class) + .setActivityImplementations(new TestActivitiesImpl()) + // Forcing a replay. Full history arrived from a normal queue causing a replay. + .setWorkerOptions( + WorkerOptions.newBuilder() + .setStickyQueueScheduleToStartTimeout(Duration.ZERO) + .build()) + .build(); + + @Test + public void testGetVersionMultipleCalls() { + TestWorkflow1 workflowStub = + testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class); + String result = workflowStub.execute(testWorkflowRule.getTaskQueue()); + assertEquals("activity1", result); + } + + @Test + public void testGetVersionMultipleCallsReplay() throws Exception { + WorkflowReplayer.replayWorkflowExecutionFromResource( + "testGetVersionMultipleCallsHistory.json", + GetVersionMultipleCallsTest.TestGetVersionWorkflowImpl.class); + } + + public static class TestGetVersionWorkflowImpl implements TestWorkflow1 { + @Override + public String execute(String taskQueue) { + VariousTestActivities testActivities = + Workflow.newActivityStub( + VariousTestActivities.class, + SDKTestOptions.newActivityOptionsForTaskQueue(taskQueue)); + int version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, 1); + + // Try again in the same WFT + version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, 1); + + // Create a new WFT by sleeping + Workflow.sleep(1000); + version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, 1); + String result = "activity" + testActivities.activity1(1); + + version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, 1); + + return result; + } + } +} diff --git a/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionOutOfOrderFailTest.java b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionOutOfOrderFailTest.java new file mode 100644 index 000000000..0243a6eb2 --- /dev/null +++ b/temporal-sdk/src/test/java/io/temporal/workflow/versionTests/GetVersionOutOfOrderFailTest.java @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved. + * + * Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Modifications copyright (C) 2017 Uber Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this material except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.temporal.workflow.versionTests; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import io.temporal.client.WorkflowFailedException; +import io.temporal.failure.ApplicationFailure; +import io.temporal.testing.WorkflowReplayer; +import io.temporal.testing.internal.SDKTestOptions; +import io.temporal.testing.internal.SDKTestWorkflowRule; +import io.temporal.worker.NonDeterministicException; +import io.temporal.worker.WorkerOptions; +import io.temporal.worker.WorkflowImplementationOptions; +import io.temporal.workflow.Promise; +import io.temporal.workflow.Workflow; +import io.temporal.workflow.shared.TestActivities.TestActivitiesImpl; +import io.temporal.workflow.shared.TestActivities.VariousTestActivities; +import io.temporal.workflow.shared.TestWorkflows.TestWorkflow1; +import io.temporal.workflow.unsafe.WorkflowUnsafe; +import java.time.Duration; +import org.junit.Rule; +import org.junit.Test; + +public class GetVersionOutOfOrderFailTest { + + @Rule + public SDKTestWorkflowRule testWorkflowRule = + SDKTestWorkflowRule.newBuilder() + // Make the workflow fail on any exception to catch NonDeterministicException + .setWorkflowTypes( + WorkflowImplementationOptions.newBuilder() + .setFailWorkflowExceptionTypes(Throwable.class) + .build(), + TestGetVersionWorkflowImpl.class) + .setActivityImplementations(new TestActivitiesImpl()) + // Forcing a replay. Full history arrived from a normal queue causing a replay. + .setWorkerOptions( + WorkerOptions.newBuilder() + .setStickyQueueScheduleToStartTimeout(Duration.ZERO) + .build()) + .build(); + + @Test + public void testGetVersionOutOfOrderFail() { + TestWorkflow1 workflowStub = + testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class); + WorkflowFailedException e = + assertThrows( + WorkflowFailedException.class, + () -> workflowStub.execute(testWorkflowRule.getTaskQueue())); + assertThat(e.getCause(), is(instanceOf(ApplicationFailure.class))); + assertEquals( + NonDeterministicException.class.getName(), + ((ApplicationFailure) e.getCause().getCause().getCause()).getType()); + } + + @Test + public void testGetVersionOutOfOrderFailReplay() throws Exception { + assertThrows( + RuntimeException.class, + () -> + WorkflowReplayer.replayWorkflowExecutionFromResource( + "testGetVersionOutOfOrderFail.json", + GetVersionMultipleCallsTest.TestGetVersionWorkflowImpl.class)); + } + + public static class TestGetVersionWorkflowImpl implements TestWorkflow1 { + @Override + public String execute(String taskQueue) { + VariousTestActivities testActivities = + Workflow.newActivityStub( + VariousTestActivities.class, + SDKTestOptions.newActivityOptionsForTaskQueue(taskQueue)); + if (WorkflowUnsafe.isReplaying()) { + Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + } + // Create a timer to generate a command + Promise timer = Workflow.newTimer(Duration.ofSeconds(5)); + int version = Workflow.getVersion("changeId", Workflow.DEFAULT_VERSION, 1); + assertEquals(version, 1); + + timer.get(); + + String result = "activity" + testActivities.activity1(1); + + return result; + } + } +} diff --git a/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistory.json b/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistory.json new file mode 100644 index 000000000..ca1d476c6 --- /dev/null +++ b/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistory.json @@ -0,0 +1,304 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2023-08-15T14:27:11.738531921Z", + "eventType": "WorkflowExecutionStarted", + "taskId": "1056345", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "TestWorkflow1" + }, + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCalls-00f2c580-c8d1-4f43-9edb-6cbeae784492", + "kind": "Normal" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IldvcmtmbG93VGVzdC10ZXN0R2V0VmVyc2lvbk11bHRpcGxlQ2FsbHMtMDBmMmM1ODAtYzhkMS00ZjQzLTllZGItNmNiZWFlNzg0NDkyIg==" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "200s", + "workflowTaskTimeout": "5s", + "originalExecutionRunId": "9385c678-2d70-48ab-9ab7-c04e4fb26aef", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "firstExecutionRunId": "9385c678-2d70-48ab-9ab7-c04e4fb26aef", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": { + + } + } + }, + { + "eventId": "2", + "eventTime": "2023-08-15T14:27:11.738604504Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056346", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCalls-00f2c580-c8d1-4f43-9edb-6cbeae784492", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2023-08-15T14:27:11.762467879Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056352", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "requestId": "bf3d471e-a6b8-49fc-8fca-59c15c96b855", + "historySizeBytes": "489" + } + }, + { + "eventId": "4", + "eventTime": "2023-08-15T14:27:12.039614171Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056356", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + + } + } + }, + { + "eventId": "5", + "eventTime": "2023-08-15T14:27:12.039643046Z", + "eventType": "MarkerRecorded", + "taskId": "1056357", + "markerRecordedEventAttributes": { + "markerName": "Version", + "details": { + "changeId": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImNoYW5nZUlkIg==" + } + ] + }, + "version": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + } + }, + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "6", + "eventTime": "2023-08-15T14:27:12.039646796Z", + "eventType": "TimerStarted", + "taskId": "1056358", + "timerStartedEventAttributes": { + "timerId": "6fecc032-8b55-3bd5-a648-352d98c8f59e", + "startToFireTimeout": "1s", + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "7", + "eventTime": "2023-08-15T14:27:13.048472338Z", + "eventType": "TimerFired", + "taskId": "1056361", + "timerFiredEventAttributes": { + "timerId": "6fecc032-8b55-3bd5-a648-352d98c8f59e", + "startedEventId": "6" + } + }, + { + "eventId": "8", + "eventTime": "2023-08-15T14:27:13.048504255Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056362", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCalls-00f2c580-c8d1-4f43-9edb-6cbeae784492", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "9", + "eventTime": "2023-08-15T14:27:13.072938088Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056365", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "8", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "requestId": "425cf92d-7e40-428a-b3c3-652fe0a61168", + "historySizeBytes": "1056" + } + }, + { + "eventId": "10", + "eventTime": "2023-08-15T14:27:13.172328005Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056369", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "8", + "startedEventId": "9", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + + } + } + }, + { + "eventId": "11", + "eventTime": "2023-08-15T14:27:13.172373130Z", + "eventType": "ActivityTaskScheduled", + "taskId": "1056370", + "activityTaskScheduledEventAttributes": { + "activityId": "68f2db31-0cea-353f-8a62-1c6caecff57d", + "activityType": { + "name": "customActivity1" + }, + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCalls-00f2c580-c8d1-4f43-9edb-6cbeae784492", + "kind": "Normal" + }, + "header": { + + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "scheduleToCloseTimeout": "5s", + "scheduleToStartTimeout": "5s", + "startToCloseTimeout": "5s", + "heartbeatTimeout": "5s", + "workflowTaskCompletedEventId": "10", + "retryPolicy": { + "initialInterval": "1s", + "backoffCoefficient": 2, + "maximumInterval": "100s" + } + } + }, + { + "eventId": "12", + "eventTime": "2023-08-15T14:27:13.185790588Z", + "eventType": "ActivityTaskStarted", + "taskId": "1056376", + "activityTaskStartedEventAttributes": { + "scheduledEventId": "11", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "requestId": "c9a2b23e-29bf-4d3f-8fd7-ce6d1f67983e", + "attempt": 1 + } + }, + { + "eventId": "13", + "eventTime": "2023-08-15T14:27:13.210008213Z", + "eventType": "ActivityTaskCompleted", + "taskId": "1056377", + "activityTaskCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "scheduledEventId": "11", + "startedEventId": "12", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local" + } + }, + { + "eventId": "14", + "eventTime": "2023-08-15T14:27:13.210011672Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056378", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCalls-00f2c580-c8d1-4f43-9edb-6cbeae784492", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "15", + "eventTime": "2023-08-15T14:27:13.214602672Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056381", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "14", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "requestId": "d3c7363f-a036-4884-b6b2-39a5d2a0e808", + "historySizeBytes": "1795" + } + }, + { + "eventId": "16", + "eventTime": "2023-08-15T14:27:13.239348880Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056385", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "14", + "startedEventId": "15", + "identity": "55015@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + + } + } + }, + { + "eventId": "17", + "eventTime": "2023-08-15T14:27:13.239358713Z", + "eventType": "WorkflowExecutionCompleted", + "taskId": "1056386", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImFjdGl2aXR5MSI=" + } + ] + }, + "workflowTaskCompletedEventId": "16" + } + } + ] +} \ No newline at end of file diff --git a/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistoryDefault.json b/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistoryDefault.json new file mode 100644 index 000000000..b088570f9 --- /dev/null +++ b/temporal-sdk/src/test/resources/testGetVersionMultipleCallsHistoryDefault.json @@ -0,0 +1,267 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2023-08-15T18:34:16.145218296Z", + "eventType": "WorkflowExecutionStarted", + "taskId": "1056684", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "TestWorkflow1" + }, + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCallsDefault-b971598e-5202-479f-88b5-e5ecade10f01", + "kind": "Normal" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IldvcmtmbG93VGVzdC10ZXN0R2V0VmVyc2lvbk11bHRpcGxlQ2FsbHNEZWZhdWx0LWI5NzE1OThlLTUyMDItNDc5Zi04OGI1LWU1ZWNhZGUxMGYwMSI=" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "200s", + "workflowTaskTimeout": "5s", + "originalExecutionRunId": "3a1d6707-a783-4b7a-8932-25dc7b82564a", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "firstExecutionRunId": "3a1d6707-a783-4b7a-8932-25dc7b82564a", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": { + } + } + }, + { + "eventId": "2", + "eventTime": "2023-08-15T18:34:16.145269129Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056685", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCallsDefault-b971598e-5202-479f-88b5-e5ecade10f01", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2023-08-15T18:34:16.168159462Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056691", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "requestId": "941a5aae-4e88-4410-a078-8989e01f770a", + "historySizeBytes": "508" + } + }, + { + "eventId": "4", + "eventTime": "2023-08-15T18:34:16.463598088Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056695", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + } + } + }, + { + "eventId": "5", + "eventTime": "2023-08-15T18:34:16.463613546Z", + "eventType": "TimerStarted", + "taskId": "1056696", + "timerStartedEventAttributes": { + "timerId": "8c7dffc4-5227-3dba-90b3-a1e4149048ea", + "startToFireTimeout": "1s", + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "6", + "eventTime": "2023-08-15T18:34:17.487567630Z", + "eventType": "TimerFired", + "taskId": "1056699", + "timerFiredEventAttributes": { + "timerId": "8c7dffc4-5227-3dba-90b3-a1e4149048ea", + "startedEventId": "5" + } + }, + { + "eventId": "7", + "eventTime": "2023-08-15T18:34:17.487576296Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056700", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCallsDefault-b971598e-5202-479f-88b5-e5ecade10f01", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "8", + "eventTime": "2023-08-15T18:34:17.500961255Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056703", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "7", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "requestId": "b587bda5-af12-43d7-8ae3-b11d74384d0b", + "historySizeBytes": "953" + } + }, + { + "eventId": "9", + "eventTime": "2023-08-15T18:34:17.586989588Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056707", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "7", + "startedEventId": "8", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + } + } + }, + { + "eventId": "10", + "eventTime": "2023-08-15T18:34:17.587015880Z", + "eventType": "ActivityTaskScheduled", + "taskId": "1056708", + "activityTaskScheduledEventAttributes": { + "activityId": "5d10f437-c4a6-3139-9071-0ad935cacfb7", + "activityType": { + "name": "customActivity1" + }, + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCallsDefault-b971598e-5202-479f-88b5-e5ecade10f01", + "kind": "Normal" + }, + "header": { + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "scheduleToCloseTimeout": "5s", + "scheduleToStartTimeout": "5s", + "startToCloseTimeout": "5s", + "heartbeatTimeout": "5s", + "workflowTaskCompletedEventId": "9", + "retryPolicy": { + "initialInterval": "1s", + "backoffCoefficient": 2, + "maximumInterval": "100s" + } + } + }, + { + "eventId": "11", + "eventTime": "2023-08-15T18:34:17.593816338Z", + "eventType": "ActivityTaskStarted", + "taskId": "1056714", + "activityTaskStartedEventAttributes": { + "scheduledEventId": "10", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "requestId": "4c3819a4-8df3-470a-af8b-622498873924", + "attempt": 1 + } + }, + { + "eventId": "12", + "eventTime": "2023-08-15T18:34:17.619242088Z", + "eventType": "ActivityTaskCompleted", + "taskId": "1056715", + "activityTaskCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "scheduledEventId": "10", + "startedEventId": "11", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local" + } + }, + { + "eventId": "13", + "eventTime": "2023-08-15T18:34:17.619246713Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056716", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionMultipleCallsDefault-b971598e-5202-479f-88b5-e5ecade10f01", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "14", + "eventTime": "2023-08-15T18:34:17.624955421Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056719", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "13", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "requestId": "cff0ff55-6db1-468f-b4f1-b06b93afded1", + "historySizeBytes": "1712" + } + }, + { + "eventId": "15", + "eventTime": "2023-08-15T18:34:17.656496338Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056723", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "13", + "startedEventId": "14", + "identity": "71662@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + } + } + }, + { + "eventId": "16", + "eventTime": "2023-08-15T18:34:17.656505588Z", + "eventType": "WorkflowExecutionCompleted", + "taskId": "1056724", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImFjdGl2aXR5MSI=" + } + ] + }, + "workflowTaskCompletedEventId": "15" + } + } + ] +} \ No newline at end of file diff --git a/temporal-sdk/src/test/resources/testGetVersionOutOfOrderFail.json b/temporal-sdk/src/test/resources/testGetVersionOutOfOrderFail.json new file mode 100644 index 000000000..f9a0f967c --- /dev/null +++ b/temporal-sdk/src/test/resources/testGetVersionOutOfOrderFail.json @@ -0,0 +1,204 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2023-08-15T16:20:36.157005589Z", + "eventType": "WorkflowExecutionStarted", + "taskId": "1056562", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "TestWorkflow1" + }, + "taskQueue": { + "name": "WorkflowTest-testGetVersionOutOfOrderFail-ac421c1b-c3e8-4631-9095-5252fc072fe9", + "kind": "Normal" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IldvcmtmbG93VGVzdC10ZXN0R2V0VmVyc2lvbk91dE9mT3JkZXJGYWlsLWFjNDIxYzFiLWMzZTgtNDYzMS05MDk1LTUyNTJmYzA3MmZlOSI=" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "200s", + "workflowTaskTimeout": "5s", + "originalExecutionRunId": "fa94156b-634c-4558-8d32-827ef8743d68", + "identity": "62122@Quinn-Klassens-MacBook-Pro.local", + "firstExecutionRunId": "fa94156b-634c-4558-8d32-827ef8743d68", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": { + + } + } + }, + { + "eventId": "2", + "eventTime": "2023-08-15T16:20:36.157096756Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056563", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionOutOfOrderFail-ac421c1b-c3e8-4631-9095-5252fc072fe9", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2023-08-15T16:20:36.185117047Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056569", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "62122@Quinn-Klassens-MacBook-Pro.local", + "requestId": "bc978856-1bfb-4863-94f8-7fd718fd957a", + "historySizeBytes": "490" + } + }, + { + "eventId": "4", + "eventTime": "2023-08-15T16:20:36.444703464Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056573", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "62122@Quinn-Klassens-MacBook-Pro.local", + "meteringMetadata": { + + } + } + }, + { + "eventId": "5", + "eventTime": "2023-08-15T16:20:36.444729464Z", + "eventType": "TimerStarted", + "taskId": "1056574", + "timerStartedEventAttributes": { + "timerId": "2e092005-ee14-3e7d-9642-e583ce5e47fe", + "startToFireTimeout": "5s", + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "6", + "eventTime": "2023-08-15T16:20:36.444747131Z", + "eventType": "MarkerRecorded", + "taskId": "1056575", + "markerRecordedEventAttributes": { + "markerName": "Version", + "details": { + "changeId": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImNoYW5nZUlkIg==" + } + ] + }, + "version": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + } + }, + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "7", + "eventTime": "2023-08-15T16:20:41.448042925Z", + "eventType": "TimerFired", + "taskId": "1056578", + "timerFiredEventAttributes": { + "timerId": "2e092005-ee14-3e7d-9642-e583ce5e47fe", + "startedEventId": "5" + } + }, + { + "eventId": "8", + "eventTime": "2023-08-15T16:20:41.448077092Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1056579", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "WorkflowTest-testGetVersionOutOfOrderFail-ac421c1b-c3e8-4631-9095-5252fc072fe9", + "kind": "Normal" + }, + "startToCloseTimeout": "5s", + "attempt": 1 + } + }, + { + "eventId": "9", + "eventTime": "2023-08-15T16:20:41.474595967Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1056582", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "8", + "identity": "62122@Quinn-Klassens-MacBook-Pro.local", + "requestId": "e866c946-80e1-4d7b-896b-2b5f55f950c8", + "historySizeBytes": "1057" + } + }, + { + "eventId": "10", + "eventTime": "2023-08-15T16:20:41.555594092Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1056586", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "8", + "startedEventId": "9", + "identity": "62122@Quinn-Klassens-MacBook-Pro.local" + } + }, + { + "eventId": "11", + "eventTime": "2023-08-15T16:20:41.555605008Z", + "eventType": "WorkflowExecutionFailed", + "taskId": "1056587", + "workflowExecutionFailedEventAttributes": { + "failure": { + "message": "Failure handling event 5 of type 'EVENT_TYPE_TIMER_STARTED' during replay. {WorkflowTaskStartedEventId=9, CurrentStartedEventId=3}", + "source": "JavaSDK", + "stackTrace": "io.temporal.internal.statemachines.WorkflowStateMachines.createEventProcessingException(WorkflowStateMachines.java:314)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEventsBatch(WorkflowStateMachines.java:275)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEvent(WorkflowStateMachines.java:221)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.applyServerHistory(ReplayWorkflowRunTaskHandler.java:234)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTaskImpl(ReplayWorkflowRunTaskHandler.java:218)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTask(ReplayWorkflowRunTaskHandler.java:158)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTaskWithQuery(ReplayWorkflowTaskHandler.java:132)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTask(ReplayWorkflowTaskHandler.java:97)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handleTask(WorkflowWorker.java:415)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:319)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:261)\nio.temporal.internal.worker.PollTaskExecutor.lambda$process$0(PollTaskExecutor.java:105)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\njava.base/java.lang.Thread.run(Thread.java:829)\n", + "cause": { + "message": "Version: failure executing RESULT_NOTIFIED_REPLAYING-\u003eNON_MATCHING_EVENT, transition history is [CREATED-\u003eCHECK_EXECUTION_STATE, REPLAYING-\u003eSCHEDULE, MARKER_COMMAND_CREATED_REPLAYING-\u003eRECORD_MARKER]", + "source": "JavaSDK", + "stackTrace": "io.temporal.internal.statemachines.StateMachine.executeTransition(StateMachine.java:163)\nio.temporal.internal.statemachines.StateMachine.handleExplicitEvent(StateMachine.java:93)\nio.temporal.internal.statemachines.EntityStateMachineBase.explicitEvent(EntityStateMachineBase.java:95)\nio.temporal.internal.statemachines.VersionStateMachine$InvocationStateMachine.handleEvent(VersionStateMachine.java:167)\nio.temporal.internal.statemachines.CancellableCommand.handleEvent(CancellableCommand.java:73)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleCommandEvent(WorkflowStateMachines.java:415)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleSingleEvent(WorkflowStateMachines.java:346)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEventsBatch(WorkflowStateMachines.java:273)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEvent(WorkflowStateMachines.java:221)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.applyServerHistory(ReplayWorkflowRunTaskHandler.java:234)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTaskImpl(ReplayWorkflowRunTaskHandler.java:218)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTask(ReplayWorkflowRunTaskHandler.java:158)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTaskWithQuery(ReplayWorkflowTaskHandler.java:132)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTask(ReplayWorkflowTaskHandler.java:97)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handleTask(WorkflowWorker.java:415)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:319)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:261)\nio.temporal.internal.worker.PollTaskExecutor.lambda$process$0(PollTaskExecutor.java:105)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\njava.base/java.lang.Thread.run(Thread.java:829)\n", + "cause": { + "message": "getVersion call before the existing version marker event. The most probable cause is retroactive addition of a getVersion call with an existing 'changeId'", + "source": "JavaSDK", + "stackTrace": "io.temporal.internal.statemachines.VersionStateMachine$InvocationStateMachine.missingMarkerReplaying(VersionStateMachine.java:316)\nio.temporal.internal.statemachines.FixedTransitionAction.apply(FixedTransitionAction.java:46)\nio.temporal.internal.statemachines.StateMachine.executeTransition(StateMachine.java:159)\nio.temporal.internal.statemachines.StateMachine.handleExplicitEvent(StateMachine.java:93)\nio.temporal.internal.statemachines.EntityStateMachineBase.explicitEvent(EntityStateMachineBase.java:95)\nio.temporal.internal.statemachines.VersionStateMachine$InvocationStateMachine.handleEvent(VersionStateMachine.java:167)\nio.temporal.internal.statemachines.CancellableCommand.handleEvent(CancellableCommand.java:73)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleCommandEvent(WorkflowStateMachines.java:415)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleSingleEvent(WorkflowStateMachines.java:346)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEventsBatch(WorkflowStateMachines.java:273)\nio.temporal.internal.statemachines.WorkflowStateMachines.handleEvent(WorkflowStateMachines.java:221)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.applyServerHistory(ReplayWorkflowRunTaskHandler.java:234)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTaskImpl(ReplayWorkflowRunTaskHandler.java:218)\nio.temporal.internal.replay.ReplayWorkflowRunTaskHandler.handleWorkflowTask(ReplayWorkflowRunTaskHandler.java:158)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTaskWithQuery(ReplayWorkflowTaskHandler.java:132)\nio.temporal.internal.replay.ReplayWorkflowTaskHandler.handleWorkflowTask(ReplayWorkflowTaskHandler.java:97)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handleTask(WorkflowWorker.java:415)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:319)\nio.temporal.internal.worker.WorkflowWorker$TaskHandlerImpl.handle(WorkflowWorker.java:261)\nio.temporal.internal.worker.PollTaskExecutor.lambda$process$0(PollTaskExecutor.java:105)\njava.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\njava.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\njava.base/java.lang.Thread.run(Thread.java:829)\n", + "applicationFailureInfo": { + "type": "io.temporal.worker.NonDeterministicException" + } + }, + "applicationFailureInfo": { + "type": "java.lang.RuntimeException" + } + }, + "applicationFailureInfo": { + "type": "io.temporal.internal.statemachines.InternalWorkflowTaskException" + } + }, + "retryState": "RetryPolicyNotSet", + "workflowTaskCompletedEventId": "10" + } + } + ] +} \ No newline at end of file