[ 
https://issues.apache.org/jira/browse/NIFI-7760?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17184654#comment-17184654
 ] 

StephenPJ commented on NIFI-7760:
---------------------------------

There are two methods for ingesting payloads.

One is the payload is already archived with TAR or GZIP and delivered by FTP.

The other is we collect the files from a share in the OS, and use the 
"MergeContent" processor to
archive (TAR) the necessary files ready to ingest, which then follows tha same 
path as the first method.

We have a set of Sample Files that we use for both scenarios, and they all 
produce the necessary results in the 1.11.4 version of NiFi.

In the updated version 1.12.0, the first method we have 40% of the sample set 
failing.

In the second method we have 100% failure rate.

If I manually create new archive files for the first method they all work.

If I create new archive files for the second method they always fail, but
once the files are archived with the "MergeContent" processor the payloads
will always stay in the flow and I have no way of checking the "fileModeOctal"

The "UnpackContent" processor is configured with the following:

Packaging Format - "use mime.type attribute"
File Filter - ".*"

The first block below depicts a failed payload.

If I then extract the file from the first archive and manually archive it again 
with TAR it works.

The second block below shows the payload that is successfully unpacked.

The only conclusion I derive from my investigation is that somehow the 
"MergeContent" processor is 
setting the permissions (if any) in a manner that "UnpackContent" rejects...

 

and thank you for your time...

--------------------------------------------------------------------------
Mimetype - application/x-tar

File: ‘Filename-Fail’
 Size: 376 Blocks: 8 IO Block: 4096 regular file
Device: fd00h/64768d Inode: 44158187 Links: 1
Access: (0644/-rw-r--r--) Uid: ( 1000/ pp) Gid: ( 1000/ pp)
Context: unconfined_u:object_r:usr_t:s0
Access: 2020-08-25 16:25:00.097133099 +0100
Modify: 2020-08-25 12:17:07.148243469 +0100
Change: 2020-08-25 16:24:39.337125913 +0100
 Birth: -

Contents of TAR
 
 File: ‘Filename.xml’
 Size: 482 Blocks: 8 IO Block: 4096 regular file
Device: fd00h/64768d Inode: 44158347 Links: 1
Access: (0644/-rw-r--r--) Uid: ( 0/ root) Gid: ( 0/ root)
Context: unconfined_u:object_r:usr_t:s0
Access: 2020-08-25 16:22:54.014555629 +0100
Modify: 2012-02-09 12:12:44.000000000 +0000
Change: 2020-08-25 16:22:54.014555629 +0100
 Birth: -
--------------------------------------------------------------------------
Mimetype - application/x-tar
 
 File: ‘Filename-Success’
 Size: 10240 Blocks: 24 IO Block: 4096 regular file
Device: fd00h/64768d Inode: 44158179 Links: 1
Access: (0644/-rw-r--r--) Uid: ( 1000/ pp) Gid: ( 1000/ pp)
Context: unconfined_u:object_r:usr_t:s0
Access: 2020-08-25 16:19:21.894026246 +0100
Modify: 2020-08-25 16:15:21.943050326 +0100
Change: 2020-08-25 16:19:04.260474598 +0100
 Birth: - 
 
Contents of TAR
 
 File: ‘Filename.xml’
 Size: 482 Blocks: 8 IO Block: 4096 regular file
Device: fd00h/64768d Inode: 44158347 Links: 1
Access: (0644/-rw-r--r--) Uid: ( 0/ root) Gid: ( 0/ root)
Context: unconfined_u:object_r:usr_t:s0
Access: 2020-08-25 16:22:54.014555629 +0100
Modify: 2012-02-09 12:12:44.000000000 +0000
Change: 2020-08-25 16:22:54.014555629 +0100
 Birth: -
--------------------------------------------------------------------------

What is logged in nifi-app.log.


2020-08-25 18:28:48,874 INFO [Timer-Driven Process Thread-4] 
o.a.n.c.s.TimerDrivenSchedulingAgent Scheduled 
UpdateAttribute[id=bbe31713-57b6-3ca8-2d00-a128a9f969b9] to run with 1 threads

2020-08-25 18:28:49,003 ERROR [Timer-Driven Process Thread-1] 
o.a.n.processors.standard.UnpackContent 
UnpackContent[id=ea9c0f1c-abe7-31e5-8e9b-9c894288c8da] Unable to unpack 
StandardFlowFileRecord[uuid=b60fe3ad-d5c0-4cb6-ac70-84ff1e81042e,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1598373938260-600, container=default, 
section=600], offset=765173, 
length=10241],offset=0,name=Filename-Fail,size=10241] due to 
java.lang.IllegalArgumentException: Invalid permission numerals; routing to 
failure: java.lang.IllegalArgumentException: Invalid permission numerals
java.lang.IllegalArgumentException: Invalid permission numerals
 at 
org.apache.nifi.processors.standard.util.FileInfo.permissionToString(FileInfo.java:224)
 at 
org.apache.nifi.processors.standard.UnpackContent$TarUnpacker$1.process(UnpackContent.java:342)
 at 
org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2324)
 at 
org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:2292)
 at 
org.apache.nifi.processors.standard.UnpackContent$TarUnpacker.unpack(UnpackContent.java:317)
 at 
org.apache.nifi.processors.standard.UnpackContent.onTrigger(UnpackContent.java:270)
 at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
 at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1174)
 at 
org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:213)
 at 
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117)
 at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)
2020-08-25 18:28:49,832 INFO [Flow Service Tasks Thread-1] 
o.a.nifi.controller.StandardFlowService Saved flow controller 
org.apache.nifi.controller.FlowController@7e3cfd0e // Another save pending = 
false
2020-08-25 18:28:50,828 INFO [pool-12-thread-1] 
o.a.n.c.r.WriteAheadFlowFileRepository Initiating checkpoint of FlowFile 
Repository
2020-08-25 18:28:50,831 INFO [pool-12-thread-1] 
o.a.n.wali.SequentialAccessWriteAheadLog Checkpointed Write-Ahead Log with 4 
Records and 0 Swap Files in 3 milliseconds (Stop-the-world time = 2 
milliseconds), max Transaction ID 583327

> UnpackContent failing on "Invalid permission numerals"
> ------------------------------------------------------
>
>                 Key: NIFI-7760
>                 URL: https://issues.apache.org/jira/browse/NIFI-7760
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>            Reporter: Matt Burgess
>            Assignee: Tamás Bunth
>            Priority: Blocker
>             Fix For: 1.13.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> A community user ran into an issue with UnpackContent in NiFi 1.12.0 that did 
> not occur in 1.11.4. Files going through UnpackContent now may generate an 
> error on "Invalid permission numerals", perhaps even if the file mode on the 
> files is legitimate (0744 for example).
> This check was added via NIFI-6128, a unit test should be added to reproduce 
> the issue alongside any fix.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to