Jens,

Also what type of file system/storage system are you running NiFi on
in this case?  We'll need to know this for the NiFi
content/flowfile/provenance repositories? Is it NFS?

Thanks

On Wed, Oct 20, 2021 at 11:14 AM Joe Witt <[email protected]> wrote:
>
> Jens,
>
> And to further narrow this down
>
> "I have a test flow, where a GenerateFlowfile has created 6x 1GB files
> (2 files per node) and next process was a hashcontent before it run
> into a test loop. Where files are uploaded via PutSFTP to a test
> server, and downloaded again and recalculated the hash. I have had one
> issue after 3 days of running."
>
> So to be clear with GenerateFlowFile making these files and then you
> looping the content is wholly and fully exclusively within the control
> of NiFI.  No Get/Fetch/Put-SFTP of any kind at all. In by looping the
> same files over and over in nifi itself you can make this happen or
> cannot?
>
> Thanks
>
> On Wed, Oct 20, 2021 at 11:08 AM Joe Witt <[email protected]> wrote:
> >
> > Jens,
> >
> > "After fetching a FlowFile-stream file and unpacked it back into NiFi
> > I calculate a sha256. 1 minutes later I recalculate the sha256 on the
> > exact same file. And got a new hash. That is what worry’s me.
> > The fact that the same file can be recalculated and produce two
> > different hashes, is very strange, but it happens. "
> >
> > Ok so to confirm you are saying that in each case this happens you see
> > it first compute the wrong hash, but then if you retry the same
> > flowfile it then provides the correct hash?
> >
> > Can you please also show/share the lineage history for such a flow
> > file then?  It should have events for the initial hash, second hash,
> > the unpacking, trace to the original stream, etc...
> >
> > Thanks
> >
> > On Wed, Oct 20, 2021 at 11:00 AM Jens M. Kofoed <[email protected]> 
> > wrote:
> > >
> > > Dear Mark and Joe
> > >
> > > I know my setup isn’t normal for many people. But if we only looks at my 
> > > receive side, which the last mails is about. Every thing is happening at 
> > > the same NIFI instance. It is the same 3 node NIFI cluster.
> > > After fetching a FlowFile-stream file and unpacked it back into NiFi I 
> > > calculate a sha256. 1 minutes later I recalculate the sha256 on the exact 
> > > same file. And got a new hash. That is what worry’s me.
> > > The fact that the same file can be recalculated and produce two different 
> > > hashes, is very strange, but it happens. Over the last 5 months it have 
> > > only happen 35-40 times.
> > >
> > > I can understand if the file is not completely loaded and saved into the 
> > > content repository before the hashing starts. But I believe that the 
> > > unpack process don’t forward the flow file to the next process before it 
> > > is 100% finish unpacking and saving the new content to the repository.
> > >
> > > I have a test flow, where a GenerateFlowfile has created 6x 1GB files (2 
> > > files per node) and next process was a hashcontent before it run into a 
> > > test loop. Where files are uploaded via PutSFTP to a test server, and 
> > > downloaded again and recalculated the hash. I have had one issue after 3 
> > > days of running.
> > > Now the test flow is running without the Put/Fetch sftp processors.
> > >
> > > Another problem is that I can’t find any correlation to other events. Not 
> > > within NIFI, nor the server itself or VMWare. If I just could find any 
> > > other event which happens at the same time, I might be able to force some 
> > > kind of event to trigger the issue.
> > > I have tried to force VMware to migrate a NiFi node to another host. 
> > > Forcing it to do a snapshot and deleting snapshots, but nothing can 
> > > trigger and error.
> > >
> > > I know it will be very very difficult to reproduce. But I will setup 
> > > multiple NiFi instances running different test flows to see if I can find 
> > > any reason why it behaves as it does.
> > >
> > > Kind Regards
> > > Jens M. Kofoed
> > >
> > > Den 20. okt. 2021 kl. 16.39 skrev Mark Payne <[email protected]>:
> > >
> > > Jens,
> > >
> > > Thanks for sharing the images.
> > >
> > > I tried to setup a test to reproduce the issue. I’ve had it running for 
> > > quite some time. Running through millions of iterations.
> > >
> > > I’ve used 5 KB files, 50 KB files, 50 MB files, and larger (to the tune 
> > > of hundreds of MB). I’ve been unable to reproduce an issue after millions 
> > > of iterations.
> > >
> > > So far I cannot replicate. And since you’re pulling the data via SFTP and 
> > > then unpacking, which preserves all original attributes from a different 
> > > system, this can easily become confusing.
> > >
> > > Recommend trying to reproduce with SFTP-related processors out of the 
> > > picture, as Joe is mentioning. Either using GetFile/FetchFile or 
> > > GenerateFlowFile. Then immediately use CryptographicHashContent to 
> > > generate an ‘initial hash’, copy that value to another attribute, and 
> > > then loop, generating the hash and comparing against the original one. 
> > > I’ll attach a flow that does this, but not sure if the email server will 
> > > strip out the attachment or not.
> > >
> > > This way we remove any possibility of actual corruption between the two 
> > > nifi instances. If we can still see corruption / different hashes within 
> > > a single nifi instance, then it certainly warrants further investigation 
> > > but i can’t see any issues so far.
> > >
> > > Thanks
> > > -Mark
> > >
> > >
> > >
> > >
> > >
> > > On Oct 20, 2021, at 10:21 AM, Joe Witt <[email protected]> wrote:
> > >
> > > Jens
> > >
> > > Actually is this current loop test contained within a single nifi and 
> > > there you see corruption happen?
> > >
> > > Joe
> > >
> > > On Wed, Oct 20, 2021 at 7:14 AM Joe Witt <[email protected]> wrote:
> > >
> > > Jens,
> > >
> > > You have a very involved setup including other systems (non NiFi).  Have 
> > > you removed those systems from the equation so you have more evidence to 
> > > support your expectation that NiFi is doing something other than you 
> > > expect?
> > >
> > > Joe
> > >
> > > On Wed, Oct 20, 2021 at 7:10 AM Jens M. Kofoed <[email protected]> 
> > > wrote:
> > >
> > > Hi
> > >
> > > Today I have another file which have been running through the retry loop 
> > > one time. To test the processors and the algorithm I added the 
> > > HashContent processor and also added hashing by SHA-1.
> > > I file have been going through the system, and both the SHA-1 and SHA-256 
> > > are both different than expected. with a 1 minutes delay the file is 
> > > going back into the hashing content flow and this time it calculates both 
> > > hashes fine.
> > >
> > > I don't believe that the hashing is buggy, but something is very very 
> > > strange. What can influence the processors/algorithm to calculate a 
> > > different hash???
> > > All the input/output claim information is exactly the same. It is the 
> > > same flow/content file going in a loop. It happens on all 3 nodes.
> > >
> > > Any suggestions for where to dig ?
> > >
> > > Regards
> > > Jens M. Kofoed
> > >
> > >
> > >
> > > Den ons. 20. okt. 2021 kl. 06.34 skrev Jens M. Kofoed 
> > > <[email protected]>:
> > >
> > > Hi Mark
> > >
> > > Thanks for replaying and the suggestion to look at the content Claim.
> > > These 3 pictures is from the first attempt:
> > > <image.png>   <image.png>   <image.png>
> > >
> > > Yesterday I realized that the content was still in the archive, so I 
> > > could Replay the file.
> > > <image.png>
> > > So here are the same pictures but for the replay and as you can see the 
> > > Identifier, offset and Size are all the same.
> > > <image.png>   <image.png>   <image.png>
> > >
> > > In my flow if the hash does not match my original first calculated hash, 
> > > it goes into a retry loop. Here are the pictures for the 4th time the 
> > > file went through:
> > > <image.png>   <image.png>   <image.png>
> > > Here the content Claim is all the same.
> > >
> > > It is very rare that we see these issues <1 : 1.000.000 files and only 
> > > with large files. Only once have I seen the error with a 110MB file, the 
> > > other times the files size are above 800MB.
> > > This time it was a Nifi-Flowstream v3 file, which has been exported from 
> > > one system and imported in another. But while the file has been imported 
> > > it is the same file inside NIFI and it stays at the same node. Going 
> > > through the same loop of processors multiple times and in the end the 
> > > CryptographicHashContent calculate a different SHA256 than it did 
> > > earlier. This should not be possible!!! And that is what concern my the 
> > > most.
> > > What can influence the same processor to calculate 2 different sha256 on 
> > > the exact same content???
> > >
> > > Regards
> > > Jens M. Kofoed
> > >
> > >
> > > Den tir. 19. okt. 2021 kl. 16.51 skrev Mark Payne <[email protected]>:
> > >
> > > Jens,
> > >
> > > In the two provenance events - one showing a hash of dd4cc… and the other 
> > > showing f6f0….
> > > If you go to the Content tab, do they both show the same Content Claim? 
> > > I.e., do the Input Claim / Output Claim show the same values for 
> > > Container, Section, Identifier, Offset, and Size?
> > >
> > > Thanks
> > > -Mark
> > >
> > > On Oct 19, 2021, at 1:22 AM, Jens M. Kofoed <[email protected]> 
> > > wrote:
> > >
> > > Dear NIFI Users
> > >
> > > I have posted this mail in the developers mailing list and just want to 
> > > inform all of our about a very odd behavior we are facing.
> > > The background:
> > > We have data going between 2 different NIFI systems which has no direct 
> > > network access to each other. Therefore we calculate a SHA256 hash value 
> > > of the content at system 1, before the flowfile and data are combined and 
> > > saved as a "flowfile-stream-v3" pkg file. The file is then transported to 
> > > system 2, where the pkg file is unpacked and the flow can continue. To be 
> > > sure about file integrity we calculate a new sha256 at system 2. But 
> > > sometimes we see that the sha256 gets another value, which might suggest 
> > > the file was corrupted. But recalculating the sha256 again gives a new 
> > > hash value.
> > >
> > > ----
> > >
> > > Tonight I had yet another file which didn't match the expected sha256 
> > > hash value. The content is a 1.7GB file and the Event Duration was 
> > > "00:00:17.539" to calculate the hash.
> > > I have created a Retry loop, where the file will go to a Wait process for 
> > > delaying the file 1 minute and going back to the CryptographicHashContent 
> > > for a new calculation. After 3 retries the file goes to the 
> > > retries_exceeded and goes to a disabled process just to be in a queue so 
> > > I manually can look at it. This morning I rerouted the file from my 
> > > retries_exceeded queue back to the CryptographicHashContent for a new 
> > > calculation and this time it calculated the correct hash value.
> > >
> > > THIS CAN'T BE TRUE :-( :-( But it is. - Something very very strange is 
> > > happening.
> > > <image.png>
> > >
> > > We are running NiFi 1.13.2 in a 3 node cluster at Ubuntu 20.04.02 with 
> > > openjdk version "1.8.0_292", OpenJDK Runtime Environment (build 
> > > 1.8.0_292-8u292-b10-0ubuntu1~20.04-b10), OpenJDK 64-Bit Server VM (build 
> > > 25.292-b10, mixed mode). Each server is a VM with 4 CPU, 8GB Ram on 
> > > VMware ESXi, 7.0.2. Each NIFI node is running at different vm physical 
> > > hosts.
> > > I have inspected different logs to see if I can find any correlation what 
> > > happened at the same time as the file is going through my loop, but there 
> > > are no event/task at that exact time.
> > >
> > > System 1:
> > > At 10/19/2021 00:15:11.247 CEST my file is going through a 
> > > CryptographicHashContent: SHA256 value: 
> > > dd4cc7ef8dbc8d70528e8aa788581f0ab88d297c9c9f39b6b542df68952efd20
> > > The file is exported as a "FlowFile Stream, v3" to System 2
> > >
> > > SYSTEM 2:
> > > At 10/19/2021 00:18:10.528 CEST the file is going through a 
> > > CryptographicHashContent: SHA256 value: 
> > > f6f0909aacae4952f10f6fa7704f3e55d0481ec211d495993550aedbb3fe0819
> > > <image.png>
> > > At 10/19/2021 00:19:08.996 CEST the file is going through the same 
> > > CryptographicHashContent at system 2: SHA256 value: 
> > > f6f0909aacae4952f10f6fa7704f3e55d0481ec211d495993550aedbb3fe0819
> > > At 10/19/2021 00:20:04.376 CEST the file is going through the same a 
> > > CryptographicHashContent at system 2: SHA256 value: 
> > > f6f0909aacae4952f10f6fa7704f3e55d0481ec211d495993550aedbb3fe0819
> > > At 10/19/2021 00:21:01.711 CEST the file is going through the same a 
> > > CryptographicHashContent at system 2: SHA256 value: 
> > > f6f0909aacae4952f10f6fa7704f3e55d0481ec211d495993550aedbb3fe0819
> > >
> > > At 10/19/2021 06:07:43.376 CEST the file is going through the same a 
> > > CryptographicHashContent at system 2: SHA256 value: 
> > > dd4cc7ef8dbc8d70528e8aa788581f0ab88d297c9c9f39b6b542df68952efd20
> > > <image.png>
> > >
> > > How on earth can this happen???
> > >
> > > Kind Regards
> > > Jens M. Kofoed
> > >
> > >
> > >
> > > <Repro.json>

Reply via email to