Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.joda.time.Duration;

Expand Down Expand Up @@ -613,12 +614,32 @@ private void executeAsync(Callable<Void> callable) throws UserCodeExecutionExcep
private static <T> void parseAndThrow(Future<T> future, ExecutionException e)
throws UserCodeExecutionException {
future.cancel(true);
if (e.getCause() == null) {
throw new UserCodeExecutionException(e);

try {
UserCodeExecutionException genericException = null;
for (Throwable throwable : Throwables.getCausalChain(e)) {
if (throwable instanceof UserCodeQuotaException) {
throw (UserCodeQuotaException) throwable;
} else if (throwable instanceof UserCodeTimeoutException) {
throw (UserCodeTimeoutException) throwable;
} else if (throwable instanceof UserCodeRemoteSystemException) {
throw (UserCodeRemoteSystemException) throwable;
} else if (genericException == null && throwable instanceof UserCodeExecutionException) {
genericException = (UserCodeExecutionException) throwable;
}
}
if (genericException != null) {
throw genericException;
}
} catch (IllegalArgumentException iae) {
// Circular reference detected in causal chain
throw new UserCodeExecutionException(
"circular reference detected in exception causal chain", e);
}
Throwable cause = checkStateNotNull(e.getCause());
if (cause instanceof UserCodeQuotaException) {
throw new UserCodeQuotaException(cause);

Throwable cause = e.getCause();
if (cause == null) {
throw new UserCodeExecutionException(e);
}
throw new UserCodeExecutionException(cause);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,34 @@ public void givenCallerThrowsUserCodeExecutionException_emitsIntoFailurePCollect
pipeline.run();
}

@Test
public void givenCallerThrowsNonUserCodeException_emitsWrappedUserCodeExecutionException() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(Call.of(new CallerThrowsRuntimeException(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(1L);

pipeline.run();
}

@Test
public void givenCallerThrowsCircularCausalChain_emitsUserCodeExecutionException() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(Call.of(new CallerThrowsCircularCause(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(1L);

pipeline.run();
}

@Test
public void givenCallerThrowsQuotaException_emitsIntoFailurePCollection() {
Result<Response> result =
Expand Down Expand Up @@ -142,22 +170,169 @@ public void givenCallerTimeout_emitsFailurePCollection() {
}

@Test
public void givenCallerThrowsTimeoutException_emitsFailurePCollection() {
public void givenCallerThrowsTimeoutException_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(Call.of(new CallerThrowsTimeout(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(1L);
.isEqualTo(0L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(0L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class))
.isEqualTo(1L);

pipeline.run();
}

@Test
public void givenCallerThrowsRemoteSystemException_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(new CallerThrowsRemoteSystemException(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeRemoteSystemException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenNestedExecutionException_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(
new CallerThrowsNestedExecutionException(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class))
.isEqualTo(0L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeRemoteSystemException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenCallerThrowsGenericWrappingTimeout_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(
new CallerThrowsGenericWrappingTimeout(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenCallerThrowsGenericWrappingQuota_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(new CallerThrowsGenericWrappingQuota(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeQuotaException.class)).isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenCallerThrowsGenericWrappingRemoteSystem_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(
new CallerThrowsGenericWrappingRemoteSystem(),
NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeRemoteSystemException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void
givenCallerThrowsUncheckedExecutionExceptionWrappingTimeout_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(
new CallerThrowsUncheckedExecutionExceptionWrappingTimeout(),
NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void
givenCallerThrowsUncheckedExecutionExceptionWrappingRemoteSystem_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(
new CallerThrowsUncheckedExecutionExceptionWrappingRemoteSystem(),
NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeRemoteSystemException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenCallerThrowsTripleNestedTimeout_thenPreservesExceptionType() {
Result<Response> result =
pipeline
.apply(Create.of(new Request("a")))
.apply(
Call.of(new CallerThrowsTripleNestedTimeout(), NON_DETERMINISTIC_RESPONSE_CODER));

PCollection<ApiIOError> failures = result.getFailures();
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeTimeoutException.class))
.isEqualTo(1L);
PAssert.thatSingleton(countStackTracesOf(failures, UserCodeExecutionException.class))
.isEqualTo(0L);

pipeline.run();
}

@Test
public void givenSetupThrowsUserCodeExecutionException_throwsError() {
pipeline
Expand Down Expand Up @@ -376,6 +551,25 @@ public Response call(Request request) throws UserCodeExecutionException {
}
}

private static class CallerThrowsRuntimeException implements Caller<Request, Response> {

@Override
public Response call(Request request) {
throw new RuntimeException("unexpected error");
}
}

private static class CallerThrowsCircularCause implements Caller<Request, Response> {

@Override
public Response call(Request request) {
Exception a = new Exception("a");
Exception b = new Exception("b", a);
a.initCause(b); // a -> b -> a (circular reference)
throw new RuntimeException("boom", a);
}
}

private static class CallerThrowsTimeout implements Caller<Request, Response> {

@Override
Expand All @@ -384,6 +578,74 @@ public Response call(Request request) throws UserCodeExecutionException {
}
}

private static class CallerThrowsRemoteSystemException implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UserCodeRemoteSystemException("");
}
}

private static class CallerThrowsNestedExecutionException implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UncheckedExecutionException(new UserCodeExecutionException("nested"));
}
}

private static class CallerThrowsGenericWrappingTimeout implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UserCodeExecutionException("generic", new UserCodeTimeoutException("timeout"));
}
}

private static class CallerThrowsGenericWrappingQuota implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UserCodeExecutionException("generic", new UserCodeQuotaException("quota"));
}
}

private static class CallerThrowsGenericWrappingRemoteSystem
implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UserCodeExecutionException("generic", new UserCodeRemoteSystemException("remote"));
}
}

private static class CallerThrowsUncheckedExecutionExceptionWrappingTimeout
implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UncheckedExecutionException(new UserCodeTimeoutException("timeout"));
}
}

private static class CallerThrowsUncheckedExecutionExceptionWrappingRemoteSystem
implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UncheckedExecutionException(new UserCodeRemoteSystemException("remote"));
}
}

private static class CallerThrowsTripleNestedTimeout implements Caller<Request, Response> {

@Override
public Response call(Request request) throws UserCodeExecutionException {
throw new UncheckedExecutionException(
new RuntimeException(new UserCodeTimeoutException("deep timeout")));
}
}

private static class CallerInvokesQuotaException implements Caller<Request, Response> {

@Override
Expand Down
11 changes: 9 additions & 2 deletions sdks/python/apache_beam/transforms/ptransform.py
Original file line number Diff line number Diff line change
Expand Up @@ -883,8 +883,15 @@ def __init__(self, fn, *args, **kwargs):
# Ensure fn and side inputs are picklable for remote execution.
try:
self.fn = pickler.roundtrip(self.fn)
except RuntimeError as e:
raise RuntimeError('Unable to pickle fn %s: %s' % (self.fn, e))
except (RuntimeError, TypeError, Exception) as e:
raise RuntimeError(
'Unable to pickle fn %s: %s. '
'User code must be serializable (picklable) for distributed '
'execution. This usually happens when lambdas or closures capture '
'non-serializable objects like file handles, database connections, '
'or thread locks. Try: (1) using module-level functions instead of '
'lambdas, (2) initializing resources in setup() methods, '
'(3) checking what your closure captures.' % (self.fn, e)) from e

self.args = pickler.roundtrip(self.args)
self.kwargs = pickler.roundtrip(self.kwargs)
Expand Down
19 changes: 19 additions & 0 deletions sdks/python/apache_beam/transforms/ptransform_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,25 @@ def test_do_with_side_input_as_keyword_arg(self):
lambda x, addon: [x + addon], addon=pvalue.AsSingleton(side))
assert_that(result, equal_to([11, 12, 13]))

def test_callable_non_serializable_error_message(self):
class NonSerializable:
def __getstate__(self):
raise RuntimeError('nope')

bad = NonSerializable()

with self.assertRaises(RuntimeError) as context:
_ = beam.Map(lambda x: bad)

message = str(context.exception)
self.assertIn('Unable to pickle fn', message)
self.assertIn(
'User code must be serializable (picklable) for distributed execution.',
message)
self.assertIn('non-serializable objects like file handles', message)
self.assertIn(
'Try: (1) using module-level functions instead of lambdas', message)

def test_do_with_do_fn_returning_string_raises_warning(self):
ex_details = r'.*Returning a str from a ParDo or FlatMap is discouraged.'

Expand Down
Loading