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

Sunny Chan commented on IGNITE-5960:
------------------------------------

This issue has affected our application preventing live restart for our node. I 
have further investigated the issue and tired the propose fix but it didn't 
completely resolved the issue.

Using my own extra debugging statements, I have determine that the missing 
events is down to the fact that if you have lots of concurrent threads updating 
the cache entries, the listener updates could potentially be out of sequence as 
there is no lock, like this:

1) T1 test lsnrs!=null, assume no listener and obtain update counter 1
2) T2 test lsnrs==null, assume no listener and obtain update counter 2
3) T3 test lsnrs!=null, so there is a listener and obtain update counter 3

So we end up with T1, T3 will send update event but T2 won't and this caused a 
gap in the update sequence.

I will propose a fix which introduces a ReadWriteLock in the CCQM, and when we 
update the listener list we will obtain the write lock. Then in 
GridCacheMapEntry before we enter the synchronized block for the GridEntry, we 
obtain the Read lock. This way we ensure that the before the listener is 
updated all update/set cache entry will be completed. I have ran the unit test 
repeatedly using this fix and it seems to pass 100% of the time.

> Ignite Continuous Query (Queries 3): 
> CacheContinuousQueryConcurrentPartitionUpdateTest::testConcurrentUpdatesAndQueryStartAtomic
>  is flaky
> -----------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-5960
>                 URL: https://issues.apache.org/jira/browse/IGNITE-5960
>             Project: Ignite
>          Issue Type: Bug
>    Affects Versions: 2.1
>            Reporter: Sergey Chugunov
>            Assignee: Alexey Kuznetsov
>              Labels: MakeTeamcityGreenAgain, test-failure
>             Fix For: 2.4
>
>
> According to [TC 
> history|http://ci.ignite.apache.org/project.html?projectId=Ignite20Tests&testNameId=6546112007182082024&tab=testDetails&branch_Ignite20Tests=%3Cdefault%3E]
>  test is flaky.
> It is possible to reproduce it locally, sample run shows 9 failed tests out 
> of 30 overall executed.
> Test fails with jUnit assertion check: 
> {noformat}
> junit.framework.AssertionFailedError: 
> Expected :10000
> Actual   :0
>  <Click to see difference>
>       at junit.framework.Assert.fail(Assert.java:57)
>       at junit.framework.Assert.failNotEquals(Assert.java:329)
>       at junit.framework.Assert.assertEquals(Assert.java:78)
>       at junit.framework.Assert.assertEquals(Assert.java:234)
>       at junit.framework.Assert.assertEquals(Assert.java:241)
>       at junit.framework.TestCase.assertEquals(TestCase.java:409)
>       at 
> org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryConcurrentPartitionUpdateTest.concurrentUpdatesAndQueryStart(CacheContinuousQueryConcurrentPartitionUpdateTest.java:385)
>       at 
> org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryConcurrentPartitionUpdateTest.testConcurrentUpdatesAndQueryStartTx(CacheContinuousQueryConcurrentPartitionUpdateTest.java:245)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at junit.framework.TestCase.runTest(TestCase.java:176)
>       at 
> org.apache.ignite.testframework.junits.GridAbstractTest.runTestInternal(GridAbstractTest.java:2000)
>       at 
> org.apache.ignite.testframework.junits.GridAbstractTest.access$000(GridAbstractTest.java:132)
>       at 
> org.apache.ignite.testframework.junits.GridAbstractTest$5.run(GridAbstractTest.java:1915)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}



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

Reply via email to