[
https://issues.apache.org/jira/browse/NIFI-4715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16309192#comment-16309192
]
ASF GitHub Bot commented on NIFI-4715:
--------------------------------------
Github user adamlamar commented on the issue:
https://github.com/apache/nifi/pull/2361
> Do we risk making duplication by updating currentKeys in the middle of
the loop?
Yes, I think we do! I identified a similar (possibly the same) bug, and I
agree with all of your suggestions. The question in my mind is whether we
should fix all of these issues in this JIRA or defer to another. As far as the
original JIRA goes, I believe the current commit will address the issue. I also
did a fair bit of manual testing so I would be comfortable moving forward with
this change as-is.
Before refactoring, I'd like to put some additional unit tests in place for
safety. Its clear from the discussion that there is some meat here and I'd
really like to enumerate a few cases I've seen while testing in unit tests.
So its up to you - would you prefer that I start the unit tests and address
(potentially) multiple bugs in this PR, or should we merge this and create
another JIRA?
> 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)