Hi Mark Thanks for the clarification. I will implement the script when I return to the office at Monday next week ( November 1st). I don’t use NFS, but ext4. But I will implement the script so we can check if it’s the case here. But I think the issue might be after the processors writing content to the repository. I have a test flow running for more than 2 weeks without any errors. But this flow only calculate hash and comparing.
Two other flows both create errors. One flow use PutSFTP->FetchSFTP->CryptographicHashContent->compares. The other flow use MergeContent->UnpackContent->CryptographicHashContent->compares. The last flow is totally inside nifi, excluding other network/server issues. In both cases the CryptographicHashContent is right after a process which writes new content to the repository. But in one case a file in our production flow did calculate a wrong hash 4 times with a 1 minutes delay between each calculation. A few hours later I looped the file back and this time it was OK. Just like the case in step 5 and 12 in the pdf file I will let you all know more later next week Kind regards Jens > Den 27. okt. 2021 kl. 15.43 skrev Mark Payne <[email protected]>: > > And the actual script: > > > import org.apache.nifi.flowfile.FlowFile > > import java.util.stream.Collectors > > Map<String, String> getPreviousHistogram(final FlowFile flowFile) { > final Map<String, String> histogram = > flowFile.getAttributes().entrySet().stream() > .filter({ entry -> entry.getKey().startsWith("histogram.") }) > .collect(Collectors.toMap({ entry -> entry.key}, { entry -> > entry.value })) > return histogram; > } > > Map<String, String> createHistogram(final FlowFile flowFile, final > InputStream inStream) { > final Map<String, String> histogram = new HashMap<>(); > final int[] distribution = new int[256]; > Arrays.fill(distribution, 0); > > long total = 0L; > final byte[] buffer = new byte[8192]; > int len; > while ((len = inStream.read(buffer)) > 0) { > for (int i=0; i < len; i++) { > final int val = buffer[i]; > distribution[val]++; > total++; > } > } > > for (int i=0; i < 256; i++) { > histogram.put("histogram." + i, String.valueOf(distribution[i])); > } > histogram.put("histogram.totalBytes", String.valueOf(total)); > > return histogram; > } > > void logHistogramDifferences(final Map<String, String> previous, final > Map<String, String> updated) { > final StringBuilder sb = new StringBuilder("There are differences in the > histogram\n"); > final Map<String, String> sorted = new TreeMap<>(previous) > for (final Map.Entry<String, String> entry : sorted.entrySet()) { > final String key = entry.getKey(); > final String previousValue = entry.getValue(); > final String updatedValue = updated.get(entry.getKey()) > > if (!Objects.equals(previousValue, updatedValue)) { > sb.append("Byte Value: ").append(key).append(", Previous Count: > ").append(previousValue).append(", New Count: > ").append(updatedValue).append("\n"); > } > } > > log.error(sb.toString()); > } > > > def flowFile = session.get() > if (flowFile == null) { > return > } > > final Map<String, String> previousHistogram = getPreviousHistogram(flowFile) > Map<String, String> histogram = null; > > final InputStream inStream = session.read(flowFile); > try { > histogram = createHistogram(flowFile, inStream); > } finally { > inStream.close() > } > > if (!previousHistogram.isEmpty()) { > if (previousHistogram.equals(histogram)) { > log.info("Histograms match") > } else { > logHistogramDifferences(previousHistogram, histogram) > session.transfer(flowFile, REL_FAILURE) > return; > } > } > > flowFile = session.putAllAttributes(flowFile, histogram) > session.transfer(flowFile, REL_SUCCESS) > > > > > >> On Oct 27, 2021, at 9:43 AM, Mark Payne <[email protected]> wrote: >> >> Jens, >> >> For a bit of background here, the reason that Joe and I have expressed >> interest in NFS file systems is that the way the protocol works, it is >> allowed to receive packets/chunks of the file out-of-order. So, what happens >> is let’s say a 1 MB file is being written. The first 500 KB are received. >> Then instead of the the 501st KB it receives the 503rd KB. What happens is >> that the size of the file on the file system becomes 503 KB. But what about >> 501 & 502? Well when you read the data, the file system just returns ASCII >> NUL characters (byte 0) for those bytes. Once the NFS server receives those >> bytes, it then goes back and fills in the proper bytes. So if you’re running >> on NFS, it is possible for the contents of the file on the underlying file >> system to change out from under you. It’s not clear to me what other types >> of file system might do something similar. >> >> So, one thing that we can do is to find out whether or not the contents of >> the underlying file have changed in some way, or if there’s something else >> happening that could perhaps result in the hashes being wrong. I’ve put >> together a script that should help diagnose this. >> >> Can you insert an ExecuteScript processor either just before or just after >> your CryptographicHashContent processor? Doesn’t really matter whether it’s >> run just before or just after. I’ll attach the script here. It’s a Groovy >> Script so you should be able to use ExecuteScript with Script Engine = >> Groovy and the following script as the Script Body. No other changes needed. >> >> The way the script works, it reads in the contents of the FlowFile, and then >> it builds up a histogram of all byte values (0-255) that it sees in the >> contents, and then adds that as attributes. So it adds attributes such as: >> histogram.0 = 280273 >> histogram.1 = 2820 >> histogram.2 = 48202 >> histogram.3 = 3820 >> … >> histogram.totalBytes = 1780928732 >> >> It then checks if those attributes have already been added. If so, after >> calculating that histogram, it checks against the previous values (in the >> attributes). If they are the same, the FlowFile goes to ’success’. If they >> are different, it logs an error indicating the before/after value for any >> byte whose distribution was different, and it routes to failure. >> >> So, if for example, the first time through it sees 280,273 bytes with a >> value of ‘0’, and the second times it only sees 12,001 then we know there >> were a bunch of 0’s previously that were updated to be some other value. And >> it includes the total number of bytes in case somehow we find that we’re >> reading too many bytes or not enough bytes or something like that. This >> should help narrow down what’s happening. >> >> Thanks >> -Mark >> >> >> >>> On Oct 26, 2021, at 6:25 PM, Joe Witt <[email protected]> wrote: >>> >>> Jens >>> >>> Attached is the flow I was using (now running yours and this one). Curious >>> if that one reproduces the issue for you as well. >>> >>> Thanks >>> >>>> On Tue, Oct 26, 2021 at 3:09 PM Joe Witt <[email protected]> wrote: >>>> Jens >>>> >>>> I have your flow running and will keep it running for several days/week to >>>> see if I can reproduce. Also of note please use your same test flow but >>>> use HashContent instead of crypto hash. Curious if that matters for any >>>> reason... >>>> >>>> Still want to know more about your underlying storage system. >>>> >>>> You could also try updating nifi.properties and changing the following >>>> lines: >>>> nifi.flowfile.repository.always.sync=true >>>> nifi.content.repository.always.sync=true >>>> nifi.provenance.repository.always.sync=true >>>> >>>> It will hurt performance but can be useful/necessary on certain storage >>>> subsystems. >>>> >>>> Thanks >>>> >>>>> On Tue, Oct 26, 2021 at 12:05 PM Joe Witt <[email protected]> wrote: >>>>> Ignore "For the scenario where you can replicate this please share the >>>>> flow.xml.gz for which it is reproducible." I see the uploaded JSON >>>>> >>>>>> On Tue, Oct 26, 2021 at 12:04 PM Joe Witt <[email protected]> wrote: >>>>>> Jens, >>>>>> >>>>>> We asked about the underlying storage system. You replied with some >>>>>> info but not the specifics. Do you know precisely what the underlying >>>>>> storage is and how it is presented to the operating system? For >>>>>> instance is it NFS or something similar? >>>>>> >>>>>> I've setup a very similar flow at extremely high rates running for the >>>>>> past several days with no issue. In my case though I know precisely >>>>>> what the config is and the disk setup is. Didn't do anything special to >>>>>> be clear but still it is important to know. >>>>>> >>>>>> For the scenario where you can replicate this please share the >>>>>> flow.xml.gz for which it is reproducible. >>>>>> >>>>>> Thanks >>>>>> Joe >>>>>> >>>>>>> On Sun, Oct 24, 2021 at 9:53 PM Jens M. Kofoed <[email protected]> >>>>>>> wrote: >>>>>>> Dear Joe and Mark >>>>>>> >>>>>>> I have created a test flow without the sftp processors, which don't >>>>>>> create any errors. Therefore I created a new test flow where I use a >>>>>>> MergeContent and UnpackContent instead of the sftp processors. This >>>>>>> keeps all data internal in NIFI, but force NIFI to write and read new >>>>>>> files totally local. >>>>>>> My flow have been running for 7 days and this morning there where 2 >>>>>>> files where the sha256 has been given another has value than original. >>>>>>> I have set this flow up in another nifi cluster only for testing, and >>>>>>> the cluster is not doing anything else. It is using Nifi 1.14.0 >>>>>>> So I can reproduce issues at different nifi clusters and versions >>>>>>> (1.13.2 and 1.14.0) where the calculation of a hash on content can give >>>>>>> different outputs. Is doesn't make any sense, but it happens. In all my >>>>>>> cases the issues happens where the calculations of the hashcontent >>>>>>> happens right after NIFI writes the content to the content repository. >>>>>>> I don't know if there cut be some kind of delay writing the content >>>>>>> 100% before the next processors begin reading the content??? >>>>>>> >>>>>>> Please see attach test flow, and the previous mail with a pdf showing >>>>>>> the lineage of a production file which also had issues. In the pdf >>>>>>> check step 5 and 12. >>>>>>> >>>>>>> Kind regards >>>>>>> Jens M. Kofoed >>>>>>> >>>>>>> >>>>>>>> Den tor. 21. okt. 2021 kl. 08.28 skrev Jens M. Kofoed >>>>>>>> <[email protected]>: >>>>>>>> Joe, >>>>>>>> >>>>>>>> To start from the last mail :-) >>>>>>>> All the repositories has it's own disk, and I'm using ext4 >>>>>>>> /dev/VG_b/LV_b /nifiRepo ext4 defaults,noatime 0 0 >>>>>>>> /dev/VG_c/LV_c /provRepo01 ext4 defaults,noatime 0 0 >>>>>>>> /dev/VG_d/LV_d /contRepo01 ext4 defaults,noatime 0 0 >>>>>>>> >>>>>>>> My test flow WITH sftp looks like this: >>>>>>>> <image.png> >>>>>>>> And this flow has produced 1 error within 3 days. After many many >>>>>>>> loops the file fails and went out via the "unmatched" output to the >>>>>>>> disabled UpdateAttribute, which is doing nothing. Just for keeping the >>>>>>>> failed flowfile in a queue. I enabled the UpdateAttribute and looped >>>>>>>> the file back to the CryptographicHashContent and now it calculated >>>>>>>> the hash correct again. But in this flow I have a FetchSFTP Process >>>>>>>> right before the Hashing. >>>>>>>> Right now my flow is running without the 2 sftp processors, and the >>>>>>>> last 24hours there has been no errors. >>>>>>>> >>>>>>>> About the Lineage: >>>>>>>> Are there a way to export all the lineage data? The export only >>>>>>>> generate a svg file. >>>>>>>> This is only for the receiving nifi which is internally calculate 2 >>>>>>>> different hashes on the same content with ca. 1 minutes delay. >>>>>>>> Attached is a pdf-document with the lineage, the flow and all the >>>>>>>> relevant Provenance information's for each step in the lineage. >>>>>>>> The interesting steps are step 5 and 12. >>>>>>>> >>>>>>>> Can the issues be that data is not written 100% to disk between step 4 >>>>>>>> and 5 in the flow? >>>>>>>> >>>>>>>> Kind regards >>>>>>>> Jens M. Kofoed >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>>> Den ons. 20. okt. 2021 kl. 23.49 skrev Joe Witt <[email protected]>: >>>>>>>>> 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> >>> <Try_to_recreate_Jens_Challenge.json> >> >
