Skip to content

NIFI-4651 Add error message and code to PutSQL FlowFile attributes#7007

Closed
krisztina-zsihovszki wants to merge 2 commits intoapache:mainfrom
krisztina-zsihovszki:NIFI-4651
Closed

NIFI-4651 Add error message and code to PutSQL FlowFile attributes#7007
krisztina-zsihovszki wants to merge 2 commits intoapache:mainfrom
krisztina-zsihovszki:NIFI-4651

Conversation

@krisztina-zsihovszki
Copy link
Contributor

Summary

NIFI-4651

Tracking

Please complete the following tracking steps prior to pull request creation.

Issue Tracking

Pull Request Tracking

  • Pull Request title starts with Apache NiFi Jira issue number, such as NIFI-00000
  • Pull Request commit message starts with Apache NiFi Jira issue number, as such NIFI-00000

Pull Request Formatting

  • Pull Request based on current revision of the main branch
  • Pull Request refers to a feature branch with one commit containing changes

Verification

Please indicate the verification steps performed prior to pull request creation.

Build

  • Build completed using mvn clean install -P contrib-check
    • JDK 11
    • JDK 17

Licensing

  • New dependencies are compatible with the Apache License 2.0 according to the License Policy
  • New dependencies are documented in applicable LICENSE and NOTICE files

Documentation

  • Documentation formatting appears as expected in rendered files

@krisztina-zsihovszki
Copy link
Contributor Author

recheck

@Lehel44
Copy link
Contributor

Lehel44 commented Mar 9, 2023

Thanks for the contribution @krisztina-zsihovszki. Reviewing.

switch (errorTypesResult.destination()) {
case Failure:
getLogger().error("Failed to update database for {} due to {}; routing to failure", new Object[] {i, e}, e);
getLogger().error("Failed to update database for {} due to {}; routing to failure", new Object[] {flowFile, exception}, exception);
Copy link
Contributor

Choose a reason for hiding this comment

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

The logger can handle arbitrary number of arguments.

Suggested change
getLogger().error("Failed to update database for {} due to {}; routing to failure", new Object[] {flowFile, exception}, exception);
getLogger().error("Failed to update database for {}; routing to failure", flowFile, exception);

Comment on lines +472 to +473
getLogger().error("Failed to update database for {} due to {}; it is possible that retrying the operation will succeed, so routing to retry",
new Object[] {i, e}, e);
new Object[] {flowFile, exception}, exception);
Copy link
Contributor

Choose a reason for hiding this comment

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

Same here.

break;
case Self:
getLogger().error("Failed to update database for {} due to {};", new Object[] {i, e}, e);
getLogger().error("Failed to update database for {} due to {};", new Object[] {flowFile, exception}, exception);
Copy link
Contributor

Choose a reason for hiding this comment

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

Same here.

Comment on lines +483 to +502
private ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError
= ExceptionHandler.createOnGroupError(context, session, result, REL_FAILURE, REL_RETRY);
onGroupError = onGroupError.andThen((ctx, flowFileGroup, errorTypesResult, exception) -> {

switch (errorTypesResult.destination()) {
case Failure:
List<FlowFile> flowFilesToFailure = getFlowFilesOnRelationShip(result, REL_FAILURE);
Optional.ofNullable(flowFilesToFailure).map(flowFiles ->
result.getRoutedFlowFiles().put(REL_FAILURE, addErrorAttributesToFlowFilesInGroup(session, flowFiles, flowFileGroup.getFlowFiles(), exception)));
break;
case Retry:
List<FlowFile> flowFilesToRetry = getFlowFilesOnRelationShip(result, REL_RETRY);
Optional.ofNullable(flowFilesToRetry).map(flowFiles ->
result.getRoutedFlowFiles().put(REL_RETRY, addErrorAttributesToFlowFilesInGroup(session, flowFiles, flowFileGroup.getFlowFiles(), exception)));
break;
}
});
return onGroupError;
}
Copy link
Contributor

@Lehel44 Lehel44 Mar 10, 2023

Choose a reason for hiding this comment

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

The RoutingResult::getRoutedFlowFiles already returns an empty map, so the getFlowFilesOnRelationShip does not need to use Optional and should return an empty list in case the flowfiles for the specified relationship are missing. This means that flowFilesToFailure variable in onGroupError cannot be null and also doesn't need to be wrapped in Optionals. I'd also recommend simplifying the 2-branched switch to an if statement for visibility and extracting the condition on getting the Relationship based on the ErrorTypesResult.Destination since both branches depend on that value i.e. -> the if statement is not needed.

Suggested change
private ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError
= ExceptionHandler.createOnGroupError(context, session, result, REL_FAILURE, REL_RETRY);
onGroupError = onGroupError.andThen((ctx, flowFileGroup, errorTypesResult, exception) -> {
switch (errorTypesResult.destination()) {
case Failure:
List<FlowFile> flowFilesToFailure = getFlowFilesOnRelationShip(result, REL_FAILURE);
Optional.ofNullable(flowFilesToFailure).map(flowFiles ->
result.getRoutedFlowFiles().put(REL_FAILURE, addErrorAttributesToFlowFilesInGroup(session, flowFiles, flowFileGroup.getFlowFiles(), exception)));
break;
case Retry:
List<FlowFile> flowFilesToRetry = getFlowFilesOnRelationShip(result, REL_RETRY);
Optional.ofNullable(flowFilesToRetry).map(flowFiles ->
result.getRoutedFlowFiles().put(REL_RETRY, addErrorAttributesToFlowFilesInGroup(session, flowFiles, flowFileGroup.getFlowFiles(), exception)));
break;
}
});
return onGroupError;
}
private ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError(final ProcessContext context, final ProcessSession session, final RoutingResult result) {
ExceptionHandler.OnError<RollbackOnFailure, FlowFileGroup> onGroupError =
ExceptionHandler.createOnGroupError(context, session, result, REL_FAILURE, REL_RETRY);
onGroupError = onGroupError.andThen((ctx, flowFileGroup, errorTypesResult, exception) -> {
Relationship relationship = errorTypesResult.destination() == ErrorTypes.Destination.Failure ? REL_FAILURE : REL_RETRY;
List<FlowFile> flowFilesToRelationship = getFlowFilesOnRelationship(result, relationship);
result.getRoutedFlowFiles().put(relationship, addErrorAttributesToFlowFilesInGroup(session, flowFilesToRelationship, flowFileGroup.getFlowFiles(), exception));
});
return onGroupError;
}

Comment on lines +510 to +514
private List<FlowFile> getFlowFilesOnRelationShip(RoutingResult result, final Relationship relationship) {
return Optional.of(result.getRoutedFlowFiles())
.orElse(emptyMap())
.get(relationship);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Since the getRoutedFlowFiles already returns an empty map, the optional is not necessary here.

Suggested change
private List<FlowFile> getFlowFilesOnRelationShip(RoutingResult result, final Relationship relationship) {
return Optional.of(result.getRoutedFlowFiles())
.orElse(emptyMap())
.get(relationship);
}
private List<FlowFile> getFlowFilesOnRelationship(RoutingResult result, final Relationship relationship) {
return Optional.ofNullable(result.getRoutedFlowFiles().get(relationship))
.orElse(emptyList());
}

attributes.put(ERROR_SQL_STATE_ATTR, valueOf(((SQLException) exception).getSQLState()));
}

return session.putAllAttributes(flowFile, attributes);
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it guaranteed that the flowfile doesn't have original attributes before being updated with the error ones or does the method only update and not replace the flowfile attributes?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The method adds extra attributes to the FlowFile, it does not remove any existing attribute. (Added a check in unit tests to verify this.)

Copy link
Contributor

@Lehel44 Lehel44 left a comment

Choose a reason for hiding this comment

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

Thanks for making the changes @krisztina-zsihovszki!
LGTM+1

mattyb149 pushed a commit that referenced this pull request Mar 21, 2023
NIFI-4651 Review comments

Signed-off-by: Matthew Burgess <mattyb149@apache.org>

This closes #7007
@mattyb149
Copy link
Contributor

+1 LGTM, Merging to main and support/nifi-1.x

@mattyb149 mattyb149 closed this Mar 21, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants