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

ASF GitHub Bot commented on NIFI-4715:
--------------------------------------

Github user adamlamar commented on the issue:

    https://github.com/apache/nifi/pull/2361
  
    @ijokarumawak I did as you suggested and pulled `persistState` out in the 
case when no new keys have been listed, but this actually caused unit tests to 
fail. This is because `currentTimestamp` never changes during the main loop, so 
even though `commit` calls `persistState`, the value of `currentTimestamp` 
doesn't change until the main loop exits. Which is why `persistState` is 
required in both exit paths.
    
    Instead, I took a slightly different approach with the change just pushed. 
Since `currentTimestamp` is the current value persisted to the state manager, 
`maxTimestamp` is the highest timestamp seen in the main loop, and 
`currentKeys` is tied to `maxTimestamp` (not `currentTimestamp`), I removed the 
`persistState` call in `commit`, and did `persistState` at the end of 
`onTrigger` only. While this does continue to `persistState` on each exit, it 
reduces the number of `persistState` calls to once per `onTrigger` rather than 
once per 1000 keys iterated (which was done previously in `commit`).
    
    I did a bunch of manual testing with concurrent `PutS3Object` and `ListS3` 
and always got the correct number of listed keys, even when uploading 20k+ 
objects using 10 threads. I tried a few strategies to skip `persistState` if 
nothing had changed, but in manual testing it always produced the wrong number 
of keys, although sometimes only off by 1. The current code should be quite an 
improvement to the load on the state manager, even if it isn't ideal.
    
    I also introduced `totalListCount` which helps tighten up the log messages 
a bit. Previously it would "successfully list X objects" followed by "no new 
objects to list" in a single `onTrigger` (this was apparent in the unit test 
output). `totalListCount` also avoids an unnecessary `yield`.
    
    There's a lot going on in this one - let me know if you have any other 
questions!


> ListS3 produces duplicates in frequently updated buckets
> --------------------------------------------------------
>
>                 Key: NIFI-4715
>                 URL: https://issues.apache.org/jira/browse/NIFI-4715
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Core Framework
>    Affects Versions: 1.2.0, 1.3.0, 1.4.0
>         Environment: All
>            Reporter: Milan Das
>         Attachments: List-S3-dup-issue.xml, screenshot-1.png
>
>
> ListS3 state is implemented using HashSet. HashSet is not thread safe. When 
> ListS3 operates in multi threaded mode, sometimes it  tries to list  same 
> file from S3 bucket.  Seems like HashSet data is getting corrupted.
> currentKeys = new HashSet<>(); // need to be implemented Thread Safe like 
> currentKeys = //ConcurrentHashMap.newKeySet();
> *{color:red}+Update+{color}*:
> This is not a HashSet issue:
> Root cause is: 
> When the file gets uploaded to S3 simultaneously  when List S3 is in progress.
> onTrigger-->  maxTimestamp is initiated as 0L.
> This is clearing keys as per the code below
> When lastModifiedTime on S3 object is same as currentTimestamp for the listed 
> key it should be skipped. As the key is cleared, it is loading the same file 
> again. 
> I think fix should be to initiate the maxTimestamp with currentTimestamp not 
> 0L.
> {code}
>  long maxTimestamp = currentTimestamp;
> {code}
> Following block is clearing keys.
> {code:title=org.apache.nifi.processors.aws.s3.ListS3.java|borderStyle=solid}
>  if (lastModified > maxTimestamp) {
>                     maxTimestamp = lastModified;
>                     currentKeys.clear();
>                     getLogger().debug("clearing keys");
>                 }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to