Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

NIFI-8120 Added RuntimeException handling on HttpContextMap.complete() #4747

Closed
wants to merge 2 commits into from
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -65,8 +65,6 @@
@SeeAlso(value = {HandleHttpRequest.class}, classNames = {"org.apache.nifi.http.StandardHttpContextMap", "org.apache.nifi.ssl.StandardSSLContextService"})
public class HandleHttpResponse extends AbstractProcessor {

public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+");

public static final PropertyDescriptor STATUS_CODE = new PropertyDescriptor.Builder()
.name("HTTP Status Code")
.description("The HTTP Status Code to use when responding to the HTTP Request. See Section 10 of RFC 2616 for more information.")
Expand Down Expand Up @@ -136,25 +134,25 @@ public void onTrigger(final ProcessContext context, final ProcessSession session

final String contextIdentifier = flowFile.getAttribute(HTTPUtils.HTTP_CONTEXT_ID);
if (contextIdentifier == null) {
session.transfer(flowFile, REL_FAILURE);
getLogger().warn("Failed to respond to HTTP request for {} because FlowFile did not have an '" + HTTPUtils.HTTP_CONTEXT_ID + "' attribute",
new Object[]{flowFile});
session.transfer(flowFile, REL_FAILURE);
return;
}

final String statusCodeValue = context.getProperty(STATUS_CODE).evaluateAttributeExpressions(flowFile).getValue();
if (!isNumber(statusCodeValue)) {
session.transfer(flowFile, REL_FAILURE);
getLogger().error("Failed to respond to HTTP request for {} because status code was '{}', which is not a valid number", new Object[]{flowFile, statusCodeValue});
session.transfer(flowFile, REL_FAILURE);
return;
}

final HttpContextMap contextMap = context.getProperty(HTTP_CONTEXT_MAP).asControllerService(HttpContextMap.class);
final HttpServletResponse response = contextMap.getResponse(contextIdentifier);
if (response == null) {
session.transfer(flowFile, REL_FAILURE);
getLogger().error("Failed to respond to HTTP request for {} because FlowFile had an '{}' attribute of {} but could not find an HTTP Response Object for this identifier",
new Object[]{flowFile, HTTPUtils.HTTP_CONTEXT_ID, contextIdentifier});
session.transfer(flowFile, REL_FAILURE);
return;
}

Expand Down Expand Up @@ -192,27 +190,31 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
session.exportTo(flowFile, response.getOutputStream());
response.flushBuffer();
} catch (final ProcessException e) {
session.transfer(flowFile, REL_FAILURE);
getLogger().error("Failed to respond to HTTP request for {} due to {}", new Object[]{flowFile, e});
contextMap.complete(contextIdentifier);
try {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not a new issue but it could be improved around exception handling:
The typical (by convention) error handling flow is logging first, then sending the FF to FAILURE (in case of an exception in session.transfer()). Like in the last catch block at lines 210-211.

This catch and the others throughout onTrigger() do it vice versa.
I would consider moving the
session.transfer(flowFile, REL_FAILURE);
lines just before the return statements.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense, reordered this and other session.transfer() calls to occur after logging.

contextMap.complete(contextIdentifier);
} catch (final RuntimeException ce) {
getLogger().error("Failed to complete HTTP Transaction for {} due to {}", new Object[]{flowFile, ce});
}
session.transfer(flowFile, REL_FAILURE);
return;
} catch (final Exception e) {
session.transfer(flowFile, REL_FAILURE);
getLogger().error("Failed to respond to HTTP request for {} due to {}", new Object[]{flowFile, e});
session.transfer(flowFile, REL_FAILURE);
return;
}

try {
contextMap.complete(contextIdentifier);
} catch (final IllegalStateException ise) {
getLogger().error("Failed to complete HTTP Transaction for {} due to {}", new Object[]{flowFile, ise});
} catch (final RuntimeException ce) {
getLogger().error("Failed to complete HTTP Transaction for {} due to {}", new Object[]{flowFile, ce});
session.transfer(flowFile, REL_FAILURE);
return;
}

session.getProvenanceReporter().send(flowFile, HTTPUtils.getURI(flowFile.getAttributes()), stopWatch.getElapsed(TimeUnit.MILLISECONDS));
session.transfer(flowFile, REL_SUCCESS);
getLogger().info("Successfully responded to HTTP Request for {} with status code {}", new Object[]{flowFile, statusCode});
session.transfer(flowFile, REL_SUCCESS);
}

private static boolean isNumber(final String value) {
Expand Down