Skip to content

Commit 27d998a

Browse files
Add failure_reason to nexus_task_execution_failed (#2274)
Add failure_reason to nexus_task_execution_failed
1 parent b8c4b7b commit 27d998a

File tree

5 files changed

+362
-7
lines changed

5 files changed

+362
-7
lines changed

temporal-sdk/src/main/java/io/temporal/internal/nexus/NexusTaskHandlerImpl.java

+3-1
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,9 @@ private void convertKnownFailures(Throwable e) {
223223
if (failure instanceof Error) {
224224
throw (Error) failure;
225225
}
226-
throw new RuntimeException(failure);
226+
throw failure instanceof RuntimeException
227+
? (RuntimeException) failure
228+
: new RuntimeException(failure);
227229
}
228230

229231
private OperationStartResult<HandlerResultContent> startOperation(

temporal-sdk/src/main/java/io/temporal/internal/worker/NexusWorker.java

+26-5
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
package io.temporal.internal.worker;
2222

2323
import static io.temporal.serviceclient.MetricsTag.METRICS_TAGS_CALL_OPTIONS_KEY;
24+
import static io.temporal.serviceclient.MetricsTag.TASK_FAILURE_TYPE;
2425

2526
import com.google.protobuf.ByteString;
2627
import com.uber.m3.tally.Scope;
@@ -40,6 +41,7 @@
4041
import io.temporal.worker.MetricsType;
4142
import io.temporal.worker.WorkerMetricsTag;
4243
import io.temporal.worker.tuning.*;
44+
import java.util.Collections;
4345
import java.util.Objects;
4446
import java.util.concurrent.CompletableFuture;
4547
import java.util.concurrent.TimeUnit;
@@ -278,16 +280,35 @@ private void handleNexusTask(NexusTask task, Scope metricsScope) {
278280
Stopwatch sw = metricsScope.timer(MetricsType.NEXUS_EXEC_LATENCY).start();
279281
try {
280282
result = handler.handle(task, metricsScope);
281-
if (result.getHandlerError() != null
282-
|| (result.getResponse().hasStartOperation()
283-
&& result.getResponse().getStartOperation().hasOperationError())) {
284-
metricsScope.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER).inc(1);
283+
if (result.getHandlerError() != null) {
284+
metricsScope
285+
.tagged(
286+
Collections.singletonMap(
287+
TASK_FAILURE_TYPE,
288+
"handler_error_" + result.getHandlerError().getErrorType()))
289+
.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER)
290+
.inc(1);
291+
} else if (result.getResponse().hasStartOperation()
292+
&& result.getResponse().getStartOperation().hasOperationError()) {
293+
String operationState =
294+
result.getResponse().getStartOperation().getOperationError().getOperationState();
295+
metricsScope
296+
.tagged(Collections.singletonMap(TASK_FAILURE_TYPE, "operation_" + operationState))
297+
.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER)
298+
.inc(1);
285299
}
286300
} catch (TimeoutException e) {
287301
log.warn("Nexus task timed out while processing", e);
288-
metricsScope.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER).inc(1);
302+
metricsScope
303+
.tagged(Collections.singletonMap(TASK_FAILURE_TYPE, "timeout"))
304+
.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER)
305+
.inc(1);
289306
return;
290307
} catch (Throwable e) {
308+
metricsScope
309+
.tagged(Collections.singletonMap(TASK_FAILURE_TYPE, "internal_sdk_error"))
310+
.counter(MetricsType.NEXUS_EXEC_FAILED_COUNTER)
311+
.inc(1);
291312
// handler.handle if expected to never throw an exception and return result
292313
// that can be used for a workflow callback if this method throws, it's a bug.
293314
log.error("[BUG] Code that expected to never throw an exception threw an exception", e);
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,303 @@
1+
/*
2+
* Copyright (C) 2022 Temporal Technologies, Inc. All Rights Reserved.
3+
*
4+
* Copyright (C) 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved.
5+
*
6+
* Modifications copyright (C) 2017 Uber Technologies, Inc.
7+
*
8+
* Licensed under the Apache License, Version 2.0 (the "License");
9+
* you may not use this material except in compliance with the License.
10+
* You may obtain a copy of the License at
11+
*
12+
* http://www.apache.org/licenses/LICENSE-2.0
13+
*
14+
* Unless required by applicable law or agreed to in writing, software
15+
* distributed under the License is distributed on an "AS IS" BASIS,
16+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17+
* See the License for the specific language governing permissions and
18+
* limitations under the License.
19+
*/
20+
21+
package io.temporal.workflow.nexus;
22+
23+
import static io.temporal.testing.internal.SDKTestWorkflowRule.NAMESPACE;
24+
25+
import com.google.common.collect.ImmutableMap;
26+
import com.uber.m3.tally.RootScopeBuilder;
27+
import io.nexusrpc.OperationUnsuccessfulException;
28+
import io.nexusrpc.handler.OperationHandler;
29+
import io.nexusrpc.handler.OperationHandlerException;
30+
import io.nexusrpc.handler.OperationImpl;
31+
import io.nexusrpc.handler.ServiceImpl;
32+
import io.temporal.api.common.v1.WorkflowExecution;
33+
import io.temporal.client.WorkflowExecutionAlreadyStarted;
34+
import io.temporal.client.WorkflowFailedException;
35+
import io.temporal.common.reporter.TestStatsReporter;
36+
import io.temporal.failure.ApplicationFailure;
37+
import io.temporal.serviceclient.MetricsTag;
38+
import io.temporal.testUtils.Eventually;
39+
import io.temporal.testing.internal.SDKTestWorkflowRule;
40+
import io.temporal.worker.MetricsType;
41+
import io.temporal.worker.WorkerMetricsTag;
42+
import io.temporal.workflow.*;
43+
import io.temporal.workflow.shared.TestNexusServices;
44+
import io.temporal.workflow.shared.TestWorkflows.TestWorkflow1;
45+
import java.time.Duration;
46+
import java.util.Map;
47+
import java.util.concurrent.ConcurrentHashMap;
48+
import org.junit.Assert;
49+
import org.junit.Rule;
50+
import org.junit.Test;
51+
52+
public class OperationFailMetricTest {
53+
private final TestStatsReporter reporter = new TestStatsReporter();
54+
55+
@Rule
56+
public SDKTestWorkflowRule testWorkflowRule =
57+
SDKTestWorkflowRule.newBuilder()
58+
.setWorkflowTypes(TestNexus.class)
59+
.setNexusServiceImplementation(new TestNexusServiceImpl())
60+
.setMetricsScope(
61+
new RootScopeBuilder()
62+
.reporter(reporter)
63+
.reportEvery(com.uber.m3.util.Duration.ofMillis(10)))
64+
.build();
65+
66+
private ImmutableMap.Builder<String, String> getBaseTags() {
67+
return ImmutableMap.<String, String>builder()
68+
.putAll(MetricsTag.defaultTags(NAMESPACE))
69+
.put(MetricsTag.WORKER_TYPE, WorkerMetricsTag.WorkerType.NEXUS_WORKER.getValue())
70+
.put(MetricsTag.TASK_QUEUE, testWorkflowRule.getTaskQueue());
71+
}
72+
73+
private ImmutableMap.Builder<String, String> getOperationTags() {
74+
return getBaseTags()
75+
.put(MetricsTag.NEXUS_SERVICE, "TestNexusService1")
76+
.put(MetricsTag.NEXUS_OPERATION, "operation");
77+
}
78+
79+
@Test
80+
public void failOperationMetrics() {
81+
TestWorkflow1 workflowStub =
82+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
83+
84+
Assert.assertThrows(WorkflowFailedException.class, () -> workflowStub.execute("fail"));
85+
86+
Map<String, String> execFailedTags =
87+
getOperationTags().put(MetricsTag.TASK_FAILURE_TYPE, "operation_failed").buildKeepingLast();
88+
Eventually.assertEventually(
89+
Duration.ofSeconds(3),
90+
() -> {
91+
reporter.assertTimer(
92+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
93+
reporter.assertTimer(
94+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
95+
reporter.assertTimer(
96+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
97+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
98+
});
99+
}
100+
101+
@Test
102+
public void failHandlerBadRequestMetrics() {
103+
TestWorkflow1 workflowStub =
104+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
105+
Assert.assertThrows(WorkflowFailedException.class, () -> workflowStub.execute("handlererror"));
106+
107+
Map<String, String> execFailedTags =
108+
getOperationTags()
109+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_BAD_REQUEST")
110+
.buildKeepingLast();
111+
Eventually.assertEventually(
112+
Duration.ofSeconds(3),
113+
() -> {
114+
reporter.assertTimer(
115+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
116+
reporter.assertTimer(
117+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
118+
reporter.assertTimer(
119+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
120+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
121+
});
122+
}
123+
124+
@Test
125+
public void failHandlerAlreadyStartedMetrics() {
126+
TestWorkflow1 workflowStub =
127+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
128+
Assert.assertThrows(
129+
WorkflowFailedException.class, () -> workflowStub.execute("already-started"));
130+
131+
Map<String, String> execFailedTags =
132+
getOperationTags()
133+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_BAD_REQUEST")
134+
.buildKeepingLast();
135+
Eventually.assertEventually(
136+
Duration.ofSeconds(3),
137+
() -> {
138+
reporter.assertTimer(
139+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
140+
reporter.assertTimer(
141+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
142+
reporter.assertTimer(
143+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
144+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
145+
});
146+
}
147+
148+
@Test
149+
public void failHandlerRetryableApplicationFailureMetrics() {
150+
TestWorkflow1 workflowStub =
151+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
152+
Assert.assertThrows(
153+
WorkflowFailedException.class, () -> workflowStub.execute("retryable-application-failure"));
154+
155+
Map<String, String> execFailedTags =
156+
getOperationTags()
157+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_INTERNAL")
158+
.buildKeepingLast();
159+
Eventually.assertEventually(
160+
Duration.ofSeconds(3),
161+
() -> {
162+
reporter.assertTimer(
163+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
164+
reporter.assertTimer(
165+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
166+
reporter.assertTimer(
167+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
168+
reporter.assertCounter(
169+
MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, c -> c >= 1);
170+
});
171+
}
172+
173+
@Test
174+
public void failHandlerNonRetryableApplicationFailureMetrics() {
175+
TestWorkflow1 workflowStub =
176+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
177+
Assert.assertThrows(
178+
WorkflowFailedException.class,
179+
() -> workflowStub.execute("non-retryable-application-failure"));
180+
181+
Map<String, String> execFailedTags =
182+
getOperationTags()
183+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_BAD_REQUEST")
184+
.buildKeepingLast();
185+
Eventually.assertEventually(
186+
Duration.ofSeconds(3),
187+
() -> {
188+
reporter.assertTimer(
189+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
190+
reporter.assertTimer(
191+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
192+
reporter.assertTimer(
193+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
194+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
195+
});
196+
}
197+
198+
@Test(timeout = 20000)
199+
public void failHandlerSleepMetrics() throws InterruptedException {
200+
TestWorkflow1 workflowStub =
201+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
202+
Assert.assertThrows(WorkflowFailedException.class, () -> workflowStub.execute("sleep"));
203+
204+
Map<String, String> execFailedTags =
205+
getOperationTags().put(MetricsTag.TASK_FAILURE_TYPE, "timeout").buildKeepingLast();
206+
Eventually.assertEventually(
207+
Duration.ofSeconds(3),
208+
() -> {
209+
reporter.assertTimer(
210+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
211+
reporter.assertTimer(
212+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
213+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
214+
});
215+
}
216+
217+
@Test
218+
public void failHandlerErrorMetrics() {
219+
TestWorkflow1 workflowStub =
220+
testWorkflowRule.newWorkflowStubTimeoutOptions(TestWorkflow1.class);
221+
Assert.assertThrows(WorkflowFailedException.class, () -> workflowStub.execute("error"));
222+
Map<String, String> execFailedTags =
223+
getOperationTags()
224+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_INTERNAL")
225+
.buildKeepingLast();
226+
Eventually.assertEventually(
227+
Duration.ofSeconds(3),
228+
() -> {
229+
reporter.assertTimer(
230+
MetricsType.NEXUS_SCHEDULE_TO_START_LATENCY, getBaseTags().buildKeepingLast());
231+
reporter.assertTimer(
232+
MetricsType.NEXUS_EXEC_LATENCY, getOperationTags().buildKeepingLast());
233+
reporter.assertTimer(
234+
MetricsType.NEXUS_TASK_E2E_LATENCY, getOperationTags().buildKeepingLast());
235+
reporter.assertCounter(
236+
MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, c -> c >= 1);
237+
});
238+
}
239+
240+
public static class TestNexus implements TestWorkflow1 {
241+
@Override
242+
public String execute(String operation) {
243+
TestNexusServices.TestNexusService1 testNexusService =
244+
Workflow.newNexusServiceStub(
245+
TestNexusServices.TestNexusService1.class,
246+
NexusServiceOptions.newBuilder()
247+
.setOperationOptions(
248+
NexusOperationOptions.newBuilder()
249+
.setScheduleToCloseTimeout(Duration.ofSeconds(10))
250+
.build())
251+
.build());
252+
return testNexusService.operation(operation);
253+
}
254+
}
255+
256+
@ServiceImpl(service = TestNexusServices.TestNexusService1.class)
257+
public class TestNexusServiceImpl {
258+
Map<String, Integer> invocationCount = new ConcurrentHashMap<>();
259+
260+
@OperationImpl
261+
public OperationHandler<String, String> operation() {
262+
// Implemented inline
263+
return OperationHandler.sync(
264+
(ctx, details, operation) -> {
265+
invocationCount.put(
266+
details.getRequestId(),
267+
invocationCount.getOrDefault(details.getRequestId(), 0) + 1);
268+
if (invocationCount.get(details.getRequestId()) > 1) {
269+
throw new OperationUnsuccessfulException("exceeded invocation count");
270+
}
271+
switch (operation) {
272+
case "success":
273+
return operation;
274+
case "fail":
275+
throw new OperationUnsuccessfulException("fail");
276+
case "handlererror":
277+
throw new OperationHandlerException(
278+
OperationHandlerException.ErrorType.BAD_REQUEST, "handlererror");
279+
case "already-started":
280+
throw new WorkflowExecutionAlreadyStarted(
281+
WorkflowExecution.getDefaultInstance(), "TestWorkflowType", null);
282+
case "retryable-application-failure":
283+
throw ApplicationFailure.newFailure("fail", "TestFailure");
284+
case "non-retryable-application-failure":
285+
throw ApplicationFailure.newNonRetryableFailure("fail", "TestFailure");
286+
case "sleep":
287+
try {
288+
Thread.sleep(11000);
289+
} catch (InterruptedException e) {
290+
throw new RuntimeException(e);
291+
}
292+
return operation;
293+
case "error":
294+
throw new Error("error");
295+
default:
296+
// Should never happen
297+
Assert.fail();
298+
}
299+
return operation;
300+
});
301+
}
302+
}
303+
}

temporal-sdk/src/test/java/io/temporal/workflow/nexus/SyncClientOperationTest.java

+6-1
Original file line numberDiff line numberDiff line change
@@ -106,7 +106,12 @@ public void syncClientOperationFail() {
106106
.buildKeepingLast();
107107
reporter.assertTimer(MetricsType.NEXUS_EXEC_LATENCY, operationTags);
108108
reporter.assertTimer(MetricsType.NEXUS_TASK_E2E_LATENCY, operationTags);
109-
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, operationTags, 1);
109+
Map<String, String> execFailedTags =
110+
ImmutableMap.<String, String>builder()
111+
.putAll(operationTags)
112+
.put(MetricsTag.TASK_FAILURE_TYPE, "handler_error_BAD_REQUEST")
113+
.buildKeepingLast();
114+
reporter.assertCounter(MetricsType.NEXUS_EXEC_FAILED_COUNTER, execFailedTags, 1);
110115
}
111116

112117
@WorkflowInterface

0 commit comments

Comments
 (0)