You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/07/16 18:59:20 UTC

[GitHub] [cassandra] maedhroz opened a new pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

maedhroz opened a new pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110


   an isolated reproduction of the assertion failure that occurs when the Accumulator size hasn't become visible by the time we check for digest mismatches


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#issuecomment-883701665


   Latest Circle runs are clean: https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-16807-trunk


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#issuecomment-884381543


   see https://github.com/apache/cassandra/commit/29cc615fafe5f246dbc6668bcf85bfe4467152ee


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672415239



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);

Review comment:
       Yeah, I think this might just be an artifact from before I added the start latch earlier.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r671688310



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException

Review comment:
       Nit: doesn't need `throws InterruptedException`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672239464



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +79,69 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+
+        try
+        {
+            for (int i = 0; i < 10_000_000; i++)

Review comment:
       I think that the time bound of 1/2 of the JUnit timeout is a nice solution. If we find it problematic in the future we can always move it to long tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r671689441



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);

Review comment:
       Nit: wouldn't it be enough with 2 threads?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r671500382



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +79,69 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+
+        try
+        {
+            for (int i = 0; i < 10_000_000; i++)

Review comment:
       This test can obviously run for quite a while, but if it doesn't run long enough to catch the problem, it doesn't add much value. I'm open to throwing an `@Ignore` on it for now, leaving it to a suite of tests we don't run on every commit, or any other ideas...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r671688480



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException

Review comment:
       Nit: We could mention the ticket number in a comment, maybe in the JavaDoc for the test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672239464



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +79,69 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+
+        try
+        {
+            for (int i = 0; i < 10_000_000; i++)

Review comment:
       I think that the time bound of 1/2 of the JUnit timeout is a nice solution. If we find it problematic in the future we can always move it to long tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672205210



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+        long endTime = System.nanoTime() + TimeUnit.MINUTES.toNanos(2);
+
+        try
+        {
+            while (System.nanoTime() < endTime)
+            {
+                final long startNanos = System.nanoTime();
+                final DigestResolver<EndpointsForToken, ReplicaPlan.ForTokenRead> resolver = new DigestResolver<>(command, plan, startNanos);
+                final ReadCallback<EndpointsForToken, ReplicaPlan.ForTokenRead> callback = new ReadCallback<>(resolver, command, plan, startNanos);
+                
+                final CountDownLatch startlatch = new CountDownLatch(2);
+
+                pool.execute(() ->
+                             {
+                                 startlatch.countDown();
+
+                                 try
+                                 {
+                                     startlatch.await();
+                                 }
+                                 catch (InterruptedException e)
+                                 {
+                                     Thread.currentThread().interrupt();
+                                 }
+
+                                 callback.onResponse(response(command, EP1, iter(response), true));
+                             });
+
+                pool.execute(() ->
+                             {
+                                 startlatch.countDown();
+
+                                 try
+                                 {
+                                     startlatch.await();
+                                 }
+                                 catch (InterruptedException e)
+                                 {
+                                     Thread.currentThread().interrupt();
+                                 }
+
+                                 callback.onResponse(response(command, EP2, iter(response), true));
+                             });
+
+                callback.awaitResults();
+                Assert.assertTrue(resolver.isDataPresent());
+                resolver.responsesMatch();

Review comment:
       Perhaps we can put this inside an `assertTrue`, just in case.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672415239



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);

Review comment:
       Yeah, I think this might just be an artifact from before I added the start latch earlier.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz closed pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz closed pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672205210



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+        long endTime = System.nanoTime() + TimeUnit.MINUTES.toNanos(2);
+
+        try
+        {
+            while (System.nanoTime() < endTime)
+            {
+                final long startNanos = System.nanoTime();
+                final DigestResolver<EndpointsForToken, ReplicaPlan.ForTokenRead> resolver = new DigestResolver<>(command, plan, startNanos);
+                final ReadCallback<EndpointsForToken, ReplicaPlan.ForTokenRead> callback = new ReadCallback<>(resolver, command, plan, startNanos);
+                
+                final CountDownLatch startlatch = new CountDownLatch(2);
+
+                pool.execute(() ->
+                             {
+                                 startlatch.countDown();
+
+                                 try
+                                 {
+                                     startlatch.await();
+                                 }
+                                 catch (InterruptedException e)
+                                 {
+                                     Thread.currentThread().interrupt();
+                                 }
+
+                                 callback.onResponse(response(command, EP1, iter(response), true));
+                             });
+
+                pool.execute(() ->
+                             {
+                                 startlatch.countDown();
+
+                                 try
+                                 {
+                                     startlatch.await();
+                                 }
+                                 catch (InterruptedException e)
+                                 {
+                                     Thread.currentThread().interrupt();
+                                 }
+
+                                 callback.onResponse(response(command, EP2, iter(response), true));
+                             });
+
+                callback.awaitResults();
+                Assert.assertTrue(resolver.isDataPresent());
+                resolver.responsesMatch();

Review comment:
       Perhaps we can put this inside an `assertTrue`, just in case.

##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +79,69 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);
+
+        try
+        {
+            for (int i = 0; i < 10_000_000; i++)

Review comment:
       I think that the time bound of 1/2 of the JUnit timeout is a nice solution. If we find it problematic in the future we can always move it to long tests.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #1110: CASSANDRA-16807 Weak visibility guarantees of Accumulator lead to failed assertions during digest comparison

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #1110:
URL: https://github.com/apache/cassandra/pull/1110#discussion_r672415239



##########
File path: test/unit/org/apache/cassandra/service/reads/DigestResolverTest.java
##########
@@ -78,6 +80,70 @@ public void noRepairNeeded()
         assertPartitionsEqual(filter(iter(response)), resolver.getData());
     }
 
+    @Test
+    public void multiThreadedNoRepairNeededReadCallback() throws InterruptedException
+    {
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.fullPartitionRead(cfm, nowInSec, dk);
+        EndpointsForToken targetReplicas = EndpointsForToken.of(dk.getToken(), full(EP1), full(EP2));
+        PartitionUpdate response = update(row(1000, 4, 4), row(1000, 5, 5)).build();
+        ReplicaPlan.SharedForTokenRead plan = plan(ConsistencyLevel.ONE, targetReplicas);
+
+        ExecutorService pool = Executors.newFixedThreadPool(4);

Review comment:
       Yeah, I think this might just be an artifact from before I added the start latch earlier.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org