Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
asfgit closed pull request #8295: NIFI-12642 Added support for FileResourceService in PutS3Object URL: https://github.com/apache/nifi/pull/8295 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
balazsgerner commented on code in PR #8295: URL: https://github.com/apache/nifi/pull/8295#discussion_r1467694135 ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java: ## @@ -88,6 +100,74 @@ public static void setupClass() { kmsKeyId = getKMSKey(); } +@AfterEach +public void tearDown() { +// clear bucket content +AmazonS3 client = getClient(); +List objectSummaries = client.listObjects(BUCKET_NAME).getObjectSummaries(); +if (!objectSummaries.isEmpty()) { +client.deleteObjects(new DeleteObjectsRequest(BUCKET_NAME) +.withKeys(objectSummaries.stream() +.map(S3ObjectSummary::getKey) +.toArray(String[]::new))); +} Review Comment: Fixed in [7b26f3b](https://github.com/apache/nifi/pull/8295/commits/7b26f3b4fd6641388eebf4ac0d59f17aa92a6a0e) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
turcsanyip commented on code in PR #8295: URL: https://github.com/apache/nifi/pull/8295#discussion_r1467623687 ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/ITPutS3Object.java: ## @@ -88,6 +100,74 @@ public static void setupClass() { kmsKeyId = getKMSKey(); } +@AfterEach +public void tearDown() { +// clear bucket content +AmazonS3 client = getClient(); +List objectSummaries = client.listObjects(BUCKET_NAME).getObjectSummaries(); +if (!objectSummaries.isEmpty()) { +client.deleteObjects(new DeleteObjectsRequest(BUCKET_NAME) +.withKeys(objectSummaries.stream() +.map(S3ObjectSummary::getKey) +.toArray(String[]::new))); +} Review Comment: [AbstractS3IT](https://github.com/apache/nifi/blob/49702f1f510cc33c9cb52bb5b9580009a1d8afe8/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/AbstractS3IT.java#L133-L149) contains similar functionality. To avoid duplication, that logic could be extracted to `emptyBucket()` method which could be called here too. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
balazsgerner commented on PR #8295: URL: https://github.com/apache/nifi/pull/8295#issuecomment-1911903310 > @balazsgerner It looks the processor does not upload the file's content in File Resource Service mode (an empty file is created only). Could you please double check it? Thanks for spotting that, the reason for this issue was that we always considered the flowfile content's size, and in the case the source is `FILE_RESOURCE_SERVICE`, the incoming flowfile content was empty. Fixed in [73f49bb](https://github.com/apache/nifi/pull/8295/commits/73f49bbc665ebfd946bfdd41b8c7f5d1419e2370) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
balazsgerner commented on code in PR #8295: URL: https://github.com/apache/nifi/pull/8295#discussion_r1467525114 ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java: ## @@ -517,331 +526,328 @@ public void onTrigger(final ProcessContext context, final ProcessSession session /* * Then */ -try { -final FlowFile flowFileCopy = flowFile; -session.read(flowFile, new InputStreamCallback() { -@Override -public void process(final InputStream in) throws IOException { -final ObjectMetadata objectMetadata = new ObjectMetadata(); -objectMetadata.setContentLength(ff.getSize()); - -final String contentType = context.getProperty(CONTENT_TYPE) -.evaluateAttributeExpressions(ff).getValue(); -if (contentType != null) { -objectMetadata.setContentType(contentType); -attributes.put(S3_CONTENT_TYPE, contentType); -} +final FlowFile flowFileCopy = session.clone(flowFile); Review Comment: Cloning the flowfile was only used as a workaround for an issue which occurred during unit test execution. If the inputstream of the flowfile was open, the `session.putAllAttributes(flowFile, attributes)` call threw exception, because the inputstream was already in use. I agree that the cloning is not needed, after some examination, I managed to eliminate the need for that. See 73f49bbc -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
turcsanyip commented on code in PR #8295: URL: https://github.com/apache/nifi/pull/8295#discussion_r1466940824 ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java: ## @@ -501,6 +508,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final FlowFile ff = flowFile; final Map attributes = new HashMap<>(); final String ffFilename = ff.getAttributes().get(CoreAttributes.FILENAME.key()); +final ResourceTransferSource resourceTransferSource = ResourceTransferSource.valueOf(context.getProperty(RESOURCE_TRANSFER_SOURCE).getValue()); Review Comment: Please use the recently added `PropertyValue.asAllowableValue()` method: ```suggestion final ResourceTransferSource resourceTransferSource = context.getProperty(RESOURCE_TRANSFER_SOURCE).asAllowableValue(ResourceTransferSource.class); ``` ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java: ## @@ -517,331 +526,328 @@ public void onTrigger(final ProcessContext context, final ProcessSession session /* * Then */ -try { -final FlowFile flowFileCopy = flowFile; -session.read(flowFile, new InputStreamCallback() { -@Override -public void process(final InputStream in) throws IOException { -final ObjectMetadata objectMetadata = new ObjectMetadata(); -objectMetadata.setContentLength(ff.getSize()); - -final String contentType = context.getProperty(CONTENT_TYPE) -.evaluateAttributeExpressions(ff).getValue(); -if (contentType != null) { -objectMetadata.setContentType(contentType); -attributes.put(S3_CONTENT_TYPE, contentType); -} +final FlowFile flowFileCopy = session.clone(flowFile); Review Comment: I don't think we need to clone the FlowFile. What is the reason for that? ## nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java: ## @@ -852,25 +858,27 @@ public void process(final InputStream in) throws IOException { final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); session.getProvenanceReporter().send(flowFile, url, millis); -getLogger().info("Successfully put {} to Amazon S3 in {} milliseconds", new Object[] {ff, millis}); +getLogger().info("Successfully put {} to Amazon S3 in {} milliseconds", new Object[]{ff, millis}); try { removeLocalState(cacheKey); } catch (IOException e) { getLogger().info("Error trying to delete key {} from cache: {}", new Object[]{cacheKey, e.getMessage()}); } -} catch (final ProcessException | AmazonClientException pe) { + +} catch (final ProcessException | AmazonClientException | IOException pe) { Review Comment: ```suggestion } catch (final ProcessException | AmazonClientException | IOException e) { ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[PR] NIFI-12642 Added support for FileResourceService in PutS3Object [nifi]
balazsgerner opened a new pull request, #8295: URL: https://github.com/apache/nifi/pull/8295 # Summary Added support to upload content directly from local file's inputstream (by-passing the content repository) inside the PutS3Object processor. The default behaviour is upload from flowfile content to be backward compatible. This can be overridden if the RESOURCE_TRANSFER_SOURCE property value is set to FILE_RESOURCE_SERVICE, and a FileResourceService controller service is registered. [NIFI-12642](https://issues.apache.org/jira/browse/NIFI-12642) # Tracking Please complete the following tracking steps prior to pull request creation. ### Issue Tracking - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created ### Pull Request Tracking - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-0` - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-0` ### Pull Request Formatting - [x] Pull Request based on current revision of the `main` branch - [x] 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 - [x] Build completed using `mvn clean install -P contrib-check` - [x] JDK 21 ### Licensing - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html) - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files ### Documentation - [ ] Documentation formatting appears as expected in rendered files -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org