[ https://issues.apache.org/jira/browse/CASSANDRA-15676?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17102135#comment-17102135 ]
Yifan Cai commented on CASSANDRA-15676: --------------------------------------- Interesting approach. If I am reading the code correctly, the test makes sure the response from peers are always false. So the first CAS backs off and returns unknown result. I think overwriting the response is not necessary. You can precisely construct such a contention condition with {{CountDownLatch}} and 2 filters. The test can block the PROPOSE req until the second CAS is completed. So the first CAS is guaranteed to get unknown result. Something looks like the following. Therefore, you do not need to open the access to methods in {{Instance.java}} and {{DistributedTestSnitch.java}} and overwrites messages. {code:java} CountDownLatch ready = new CountDownLatch(1); cluster.filters().verbs(Verb.PAXOS_PROPOSE_REQ.id).from(1).to(2, 3).messagesMatching((from, to, msg) -> { if (to == 2) { // Inject a single CAS request in-between prepare and propose phases cluster.coordinator(2).execute(mkCasInsertQuery((a) -> pk, 1, 2), ConsistencyLevel.QUORUM); ready.countDown(); } else { Uninterruptibles.awaitUninterruptibly(ready); } return false; }).drop(); // the message intercepting syntax is sort of confusing... {code} > 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 > > Attachments: Screen Shot 2020-05-07 at 7.25.19 PM.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org