Skip to content

Commit

Permalink
Fix bug in VersionStateMachine causing wrong version to get returned (#…
Browse files Browse the repository at this point in the history
…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.
  • Loading branch information
Quinn-With-Two-Ns committed Aug 16, 2023
1 parent 622edc0 commit f922c06
Show file tree
Hide file tree
Showing 9 changed files with 1,199 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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<Integer, RuntimeException> 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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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<ExecuteLocalActivityParameters> takeLocalActivityRequests() {
Expand Down
Original file line number Diff line number Diff line change
@@ -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++;
}
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
}
}

0 comments on commit f922c06

Please sign in to comment.