NIFI-9713 : TagS3Object do not have provenance data

Changed from AmazonS3 to AmazonS3client.
Add REMOTE_INVOCATION provenance event with URL and "Object tagged"

This closes #5785

Signed-off-by: Mike Thomsen <mthomsen@apache.org>
This commit is contained in:
lawye1973 2022-02-21 16:37:28 +08:00 committed by Mike Thomsen
parent 07131a66ea
commit 54b4209798
No known key found for this signature in database
GPG Key ID: 88511C3D4CAD246F
1 changed files with 4 additions and 2 deletions

View File

@ -17,7 +17,7 @@
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.GetObjectTaggingRequest;
import com.amazonaws.services.s3.model.GetObjectTaggingResult;
import com.amazonaws.services.s3.model.ObjectTagging;
@ -153,7 +153,7 @@ public class TagS3Object extends AbstractS3Processor {
final String version = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue();
final AmazonS3 s3 = getClient();
final AmazonS3Client s3 = getClient();
SetObjectTaggingRequest r;
List<Tag> tags = new ArrayList<>();
@ -186,8 +186,10 @@ public class TagS3Object extends AbstractS3Processor {
flowFile = setTagAttributes(session, flowFile, tags);
session.transfer(flowFile, REL_SUCCESS);
final String url = s3.getResourceUrl(bucket, key);
final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
getLogger().info("Successfully tagged S3 Object for {} in {} millis; routing to success", new Object[]{flowFile, transferMillis});
session.getProvenanceReporter().invokeRemoteProcess(flowFile, url, "Object tagged");
}
private void failFlowWithBlankEvaluatedProperty(ProcessSession session, FlowFile flowFile, PropertyDescriptor pd) {