You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/09/29 18:59:18 UTC

[GitHub] [nifi] simonbence opened a new pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

simonbence opened a new pull request #5423:
URL: https://github.com/apache/nifi/pull/5423


   [NIFI-9260](https://issues.apache.org/jira/browse/NIFI-9260)
   
   This is a small proposal in order to add some more flexibility to the PutHDFS: In some use cases -please see linked story- it is undesirable to write into a tempoarary file and then rename it. By adding a new property -with using default value results the current behaviour- this can be changed easily.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
simonbence commented on pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#issuecomment-952705311


   Thanks @mattyb149 for catching up this! Yeah, it became stall during the review, but I quickly solved it, so we should be good now.


-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r742180815



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -133,6 +136,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading  partially written files.");

Review comment:
       "This prevents..." or "This can prevent..." would describe its purpose/behaviour more exactly.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r720220570



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -205,6 +222,7 @@
                 .description("The parent HDFS directory to which files should be written. The directory will be created if it doesn't exist.")
                 .build());
         props.add(CONFLICT_RESOLUTION);
+        props.add(WRITING_STRATEGY);

Review comment:
       Enchancement: I think this can be wrapped in an unmodifiable list in the end.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r720284694



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -365,9 +388,12 @@ public void process(InputStream in) throws IOException {
                     final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
                     tempDotCopyFile = tempCopyFile;
 
-                    if (!conflictResponse.equals(APPEND_RESOLUTION)
-                            || (conflictResponse.equals(APPEND_RESOLUTION) && !destinationExists)) {
+                    if  (
+                        writingStrategy.equals(WRITE_AND_RENAME)
+                        && (!conflictResponse.equals(APPEND_RESOLUTION) || (conflictResponse.equals(APPEND_RESOLUTION) && !destinationExists))

Review comment:
       You are correct and I was playing with the idea but I did not want to change the existing expression and make it less readable




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719843019



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -129,6 +137,15 @@
             .allowableValues(REPLACE_RESOLUTION_AV, IGNORE_RESOLUTION_AV, FAIL_RESOLUTION_AV, APPEND_RESOLUTION_AV)
             .build();
 
+    protected static final PropertyDescriptor WRITING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("writing-strategy")
+            .displayName("Writing Strategy")
+            .description("Defines the approach for writing the FlowFile data.")

Review comment:
       Minor: I'd advise using "method" rather than "approach" here because it refers to the process.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5423:
URL: https://github.com/apache/nifi/pull/5423


   


-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719865861



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
##########
@@ -189,35 +193,76 @@ public void testValidators() {
 
     @Test
     public void testPutFile() throws IOException {
-        PutHDFS proc = new TestablePutHDFS(kerberosProperties, mockFileSystem);
-        TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(PutHDFS.DIRECTORY, "target/test-classes");
+        // given
+        final FileSystem spyFileSystem = Mockito.spy(mockFileSystem);
+        final PutHDFS proc = new TestablePutHDFS(kerberosProperties, spyFileSystem);
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHDFS.DIRECTORY, TARGET_DIRECTORY);
         runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");
-        try (FileInputStream fis = new FileInputStream("src/test/resources/testdata/randombytes-1")) {
-            Map<String, String> attributes = new HashMap<>();
-            attributes.put(CoreAttributes.FILENAME.key(), "randombytes-1");
+
+        // when
+        try (final FileInputStream fis = new FileInputStream(SOURCE_DIRECTORY + "/" + FILE_NAME)) {
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put(CoreAttributes.FILENAME.key(), FILE_NAME);
             runner.enqueue(fis, attributes);
             runner.run();
         }
 
-        List<MockFlowFile> failedFlowFiles = runner
-                .getFlowFilesForRelationship(new Relationship.Builder().name("failure").build());
+        // then
+        final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_FAILURE);
         assertTrue(failedFlowFiles.isEmpty());
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_SUCCESS);
         assertEquals(1, flowFiles.size());
-        MockFlowFile flowFile = flowFiles.get(0);
-        assertTrue(mockFileSystem.exists(new Path("target/test-classes/randombytes-1")));
-        assertEquals("randombytes-1", flowFile.getAttribute(CoreAttributes.FILENAME.key()));
-        assertEquals("target/test-classes", flowFile.getAttribute(PutHDFS.ABSOLUTE_HDFS_PATH_ATTRIBUTE));
+
+        final MockFlowFile flowFile = flowFiles.get(0);
+        assertTrue(spyFileSystem.exists(new Path(TARGET_DIRECTORY + "/" + FILE_NAME)));
+        assertEquals(FILE_NAME, flowFile.getAttribute(CoreAttributes.FILENAME.key()));
+        assertEquals(TARGET_DIRECTORY, flowFile.getAttribute(PutHDFS.ABSOLUTE_HDFS_PATH_ATTRIBUTE));
         assertEquals("true", flowFile.getAttribute(PutHDFS.TARGET_HDFS_DIR_CREATED_ATTRIBUTE));
 
         final List<ProvenanceEventRecord> provenanceEvents = runner.getProvenanceEvents();
         assertEquals(1, provenanceEvents.size());
         final ProvenanceEventRecord sendEvent = provenanceEvents.get(0);
         assertEquals(ProvenanceEventType.SEND, sendEvent.getEventType());
         // If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
-        assertTrue(sendEvent.getTransitUri().endsWith("target/test-classes/randombytes-1"));
+        assertTrue(sendEvent.getTransitUri().endsWith(TARGET_DIRECTORY + "/" + FILE_NAME));
+
+        Mockito.verify(spyFileSystem, Mockito.times(1)).rename(Mockito.any(Path.class), Mockito.any(Path.class));
+    }
+
+    @Test
+    public void testPutFileWithSimpleWrite() throws IOException {
+        // given
+        final FileSystem spyFileSystem = Mockito.spy(mockFileSystem);
+        final PutHDFS proc = new TestablePutHDFS(kerberosProperties, spyFileSystem);
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHDFS.DIRECTORY, TARGET_DIRECTORY);
+        runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");

Review comment:
       ```suggestion
           runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, PutHDFS.REPLACE_RESOLUTION);
   ```




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719864892



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
##########
@@ -189,35 +193,76 @@ public void testValidators() {
 
     @Test
     public void testPutFile() throws IOException {
-        PutHDFS proc = new TestablePutHDFS(kerberosProperties, mockFileSystem);
-        TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(PutHDFS.DIRECTORY, "target/test-classes");
+        // given
+        final FileSystem spyFileSystem = Mockito.spy(mockFileSystem);
+        final PutHDFS proc = new TestablePutHDFS(kerberosProperties, spyFileSystem);
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHDFS.DIRECTORY, TARGET_DIRECTORY);
         runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");
-        try (FileInputStream fis = new FileInputStream("src/test/resources/testdata/randombytes-1")) {
-            Map<String, String> attributes = new HashMap<>();
-            attributes.put(CoreAttributes.FILENAME.key(), "randombytes-1");
+
+        // when
+        try (final FileInputStream fis = new FileInputStream(SOURCE_DIRECTORY + "/" + FILE_NAME)) {
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put(CoreAttributes.FILENAME.key(), FILE_NAME);

Review comment:
       ```suggestion
               final Map<String, String> attributes = Collections.singletonMap(CoreAttributes.FILENAME.key(), FILE_NAME);
   ```




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719863380



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -365,9 +388,12 @@ public void process(InputStream in) throws IOException {
                     final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
                     tempDotCopyFile = tempCopyFile;
 
-                    if (!conflictResponse.equals(APPEND_RESOLUTION)
-                            || (conflictResponse.equals(APPEND_RESOLUTION) && !destinationExists)) {
+                    if  (
+                        writingStrategy.equals(WRITE_AND_RENAME)

Review comment:
       This can be simplified to:
   ```suggestion
                           writingStrategy.equals(WRITE_AND_RENAME) && (!conflictResponse.equals(APPEND_RESOLUTION) || !destinationExists)
   ```
   




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719866135



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/test/java/org/apache/nifi/processors/hadoop/PutHDFSTest.java
##########
@@ -189,35 +193,76 @@ public void testValidators() {
 
     @Test
     public void testPutFile() throws IOException {
-        PutHDFS proc = new TestablePutHDFS(kerberosProperties, mockFileSystem);
-        TestRunner runner = TestRunners.newTestRunner(proc);
-        runner.setProperty(PutHDFS.DIRECTORY, "target/test-classes");
+        // given
+        final FileSystem spyFileSystem = Mockito.spy(mockFileSystem);
+        final PutHDFS proc = new TestablePutHDFS(kerberosProperties, spyFileSystem);
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHDFS.DIRECTORY, TARGET_DIRECTORY);
         runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");
-        try (FileInputStream fis = new FileInputStream("src/test/resources/testdata/randombytes-1")) {
-            Map<String, String> attributes = new HashMap<>();
-            attributes.put(CoreAttributes.FILENAME.key(), "randombytes-1");
+
+        // when
+        try (final FileInputStream fis = new FileInputStream(SOURCE_DIRECTORY + "/" + FILE_NAME)) {
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put(CoreAttributes.FILENAME.key(), FILE_NAME);
             runner.enqueue(fis, attributes);
             runner.run();
         }
 
-        List<MockFlowFile> failedFlowFiles = runner
-                .getFlowFilesForRelationship(new Relationship.Builder().name("failure").build());
+        // then
+        final List<MockFlowFile> failedFlowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_FAILURE);
         assertTrue(failedFlowFiles.isEmpty());
 
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_SUCCESS);
+        final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutHDFS.REL_SUCCESS);
         assertEquals(1, flowFiles.size());
-        MockFlowFile flowFile = flowFiles.get(0);
-        assertTrue(mockFileSystem.exists(new Path("target/test-classes/randombytes-1")));
-        assertEquals("randombytes-1", flowFile.getAttribute(CoreAttributes.FILENAME.key()));
-        assertEquals("target/test-classes", flowFile.getAttribute(PutHDFS.ABSOLUTE_HDFS_PATH_ATTRIBUTE));
+
+        final MockFlowFile flowFile = flowFiles.get(0);
+        assertTrue(spyFileSystem.exists(new Path(TARGET_DIRECTORY + "/" + FILE_NAME)));
+        assertEquals(FILE_NAME, flowFile.getAttribute(CoreAttributes.FILENAME.key()));
+        assertEquals(TARGET_DIRECTORY, flowFile.getAttribute(PutHDFS.ABSOLUTE_HDFS_PATH_ATTRIBUTE));
         assertEquals("true", flowFile.getAttribute(PutHDFS.TARGET_HDFS_DIR_CREATED_ATTRIBUTE));
 
         final List<ProvenanceEventRecord> provenanceEvents = runner.getProvenanceEvents();
         assertEquals(1, provenanceEvents.size());
         final ProvenanceEventRecord sendEvent = provenanceEvents.get(0);
         assertEquals(ProvenanceEventType.SEND, sendEvent.getEventType());
         // If it runs with a real HDFS, the protocol will be "hdfs://", but with a local filesystem, just assert the filename.
-        assertTrue(sendEvent.getTransitUri().endsWith("target/test-classes/randombytes-1"));
+        assertTrue(sendEvent.getTransitUri().endsWith(TARGET_DIRECTORY + "/" + FILE_NAME));
+
+        Mockito.verify(spyFileSystem, Mockito.times(1)).rename(Mockito.any(Path.class), Mockito.any(Path.class));
+    }
+
+    @Test
+    public void testPutFileWithSimpleWrite() throws IOException {
+        // given
+        final FileSystem spyFileSystem = Mockito.spy(mockFileSystem);
+        final PutHDFS proc = new TestablePutHDFS(kerberosProperties, spyFileSystem);
+        final TestRunner runner = TestRunners.newTestRunner(proc);
+        runner.setProperty(PutHDFS.DIRECTORY, TARGET_DIRECTORY);
+        runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");
+        runner.setProperty(PutHDFS.WRITING_STRATEGY, PutHDFS.SIMPLE_WRITE);
+
+        // when
+        try (final FileInputStream fis = new FileInputStream(SOURCE_DIRECTORY + "/" + FILE_NAME)) {
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put(CoreAttributes.FILENAME.key(), FILE_NAME);

Review comment:
       ```suggestion
               final Map<String, String> attributes = Collections.singletonMap(CoreAttributes.FILENAME.key(), FILE_NAME);
   ```




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719841369



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -121,6 +124,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading half-written files.");

Review comment:
       Do you think "partially written" would make more sense here?




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r742180815



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -133,6 +136,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading  partially written files.");

Review comment:
       "This prevents..." or "This can prevent..." would describe its purpose/behaviour more exactly.

##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -133,6 +136,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading  partially written files.");

Review comment:
       "This prevents..." or "This can prevent..." would describe its purpose/behaviour more exactly.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r742180815



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -133,6 +136,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading  partially written files.");

Review comment:
       "This prevents..." or "This can prevent..." would describe its purpose/behaviour more exactly.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5423:
URL: https://github.com/apache/nifi/pull/5423






-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] mattyb149 commented on pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#issuecomment-951053800


   There's a merge conflict that needs to be resolved before this PR can be merged


-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719863600



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -365,9 +388,12 @@ public void process(InputStream in) throws IOException {
                     final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
                     tempDotCopyFile = tempCopyFile;
 
-                    if (!conflictResponse.equals(APPEND_RESOLUTION)
-                            || (conflictResponse.equals(APPEND_RESOLUTION) && !destinationExists)) {
+                    if  (
+                        writingStrategy.equals(WRITE_AND_RENAME)
+                        && (!conflictResponse.equals(APPEND_RESOLUTION) || (conflictResponse.equals(APPEND_RESOLUTION) && !destinationExists))

Review comment:
       This can be simplified to:
   ```suggestion
                           writingStrategy.equals(WRITE_AND_RENAME) && (!conflictResponse.equals(APPEND_RESOLUTION) || !destinationExists)
   ```




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] Lehel44 commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
Lehel44 commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r719858254



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -269,6 +287,11 @@ public Object run() {
                     final Path tempCopyFile = new Path(dirPath, "." + filename);
                     final Path copyFile = new Path(dirPath, filename);
 
+                    // Depending on the writing strategy, we might need a temporary file
+                    final Path actualCopyFile = (writingStrategy.equals(WRITE_AND_RENAME))

Review comment:
       Since temporary files are now only created if the WRITE AND RENAME strategy is chosen, I believe the comment on 335 is worth extending.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r720271434



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -121,6 +124,11 @@
     protected static final AllowableValue APPEND_RESOLUTION_AV = new AllowableValue(APPEND_RESOLUTION, APPEND_RESOLUTION,
             "Appends to the existing file if any, creates a new file otherwise.");
 
+    protected static final AllowableValue WRITE_AND_RENAME_AV = new AllowableValue(WRITE_AND_RENAME, "Write and rename",
+            "The processor writes FlowFile data into a temporary file and renames it after completion. This might prevent other processes from reading half-written files.");

Review comment:
       Sounds better, thanks for the advice




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #5423: NIFI-9260 Making the 'write and rename' behaviour optional for PutHDFS

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #5423:
URL: https://github.com/apache/nifi/pull/5423#discussion_r720273479



##########
File path: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java
##########
@@ -129,6 +137,15 @@
             .allowableValues(REPLACE_RESOLUTION_AV, IGNORE_RESOLUTION_AV, FAIL_RESOLUTION_AV, APPEND_RESOLUTION_AV)
             .build();
 
+    protected static final PropertyDescriptor WRITING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("writing-strategy")
+            .displayName("Writing Strategy")
+            .description("Defines the approach for writing the FlowFile data.")

Review comment:
       Maybe "approach" is not the most describing word, but "method" suggests something else. To be honest I did not want to reuse the word "strategy", especially because it's meaning is loaded with being heavy weight, so I found this description better.




-- 
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-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org