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

Gianluca Righetto commented on CASSANDRA-15676:
-----------------------------------------------

The current implementation of this test is non-deterministic, it executes two 
diverging compare-and-set statements in a while-true loop until it breaks 
QUORUM. The issue is it may throw a WriteTimeoutException before that happens, 
because the loop creates too many requests. At the time this ticket was opened, 
the timeout exception was masked due to a poor assert statement, but that 
specifically has been addressed already.

Another issue was that the generated PK for the INSERT statements was non-final 
in the test method, it was overwritten at every iteration of the loop, so 
multiple threads would see only the last assigned value of the PK, which 
contributed to the overhead.

I have re-written the test to make it deterministic. Now it will intercept the 
{{PAXOS_PROPOSE_RSP}} messages from the nodes back to the coordinator and 
replace them with failure responses, such that QUORUM is never reached and it 
will throw the expected {{CasWriteUnknownResultException}}. This way it removes 
the "chance" aspect, but still exercises the same code path (which is mostly 
testing o.a.c.s.StorageProxy::cas).

[PR|https://github.com/grighetto/cassandra/pull/2]

[JVM 11 
Tests|https://app.circleci.com/pipelines/github/grighetto/cassandra/17/workflows/f9930861-5765-469f-970e-d4cf6a08563b/jobs/91]

[JVM 8 
Tests|https://app.circleci.com/pipelines/github/grighetto/cassandra/17/workflows/2cfc7b5f-6acc-420f-9fcd-ee4abfe3aa8a/jobs/92]

> flaky test testWriteUnknownResult- 
> org.apache.cassandra.distributed.test.CasWriteTest
> -------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-15676
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15676
>             Project: Cassandra
>          Issue Type: Task
>          Components: Test/dtest
>            Reporter: Kevin Gallardo
>            Assignee: Gianluca Righetto
>            Priority: Normal
>             Fix For: 4.0-alpha
>
>
> Failure observed in: 
> https://app.circleci.com/pipelines/github/newkek/cassandra/33/workflows/54007cf7-4424-4ec1-9655-665f6044e6d1/jobs/187/tests
> {noformat}
> testWriteUnknownResult - org.apache.cassandra.distributed.test.CasWriteTest
> junit.framework.AssertionFailedError: Expecting cause to be 
> CasWriteUncertainException
>       at 
> org.apache.cassandra.distributed.test.CasWriteTest.testWriteUnknownResult(CasWriteTest.java:257)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> {noformat}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to