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 2020/04/09 23:20:06 UTC

[GitHub] [cassandra] jonmeredith opened a new pull request #534: Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

jonmeredith opened a new pull request #534: Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534
 
 
   Thread scheduling is not guaranteed to be fair and having the BusyWork tasks reschedule itself makes sure there is always more work for the SEPWorker once it finishes, so it can hog all the CPU if run with a low number of cores.  To randomize the scheduling better, introduce a second thread that keeps the executor primed with work, but guarantees a thread switch by waiting on the sempahore.
   
   Also resolves a cleanup bug - the sharedPool was not being shutdown correctly.
   
   Patch by Jon Meredith; reviewed by ? for CASSANDRA-15709

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406836614
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -75,48 +78,67 @@ public void write(int b) { }
     }
 
     @Test
-    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException
+    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException, TimeoutException
     {
-        final int numBusyWorkers = 2; // Number of busy worker threads to run and gum things up
+        // Number of busy worker threads to run and gum things up. Chosen to be
+        // between the low and high max pool size so the test exercises resizing
+        // under a number of different conditions.
+        final int numBusyWorkers = 2;
         SharedExecutorPool sharedPool = new SharedExecutorPool("ChangingMaxWorkersMeetsConcurrencyGoalsTest");
         final AtomicInteger notifiedMaxPoolSize = new AtomicInteger();
 
         LocalAwareExecutorService executor = sharedPool.newExecutor(0, notifiedMaxPoolSize::set, 4, "internal", "resizetest");
 
+        // Keep feeding the executor work while resizing
+        // so it stays under load.
         AtomicBoolean stayBusy = new AtomicBoolean(true);
-        for (int i = 0; i < numBusyWorkers; i++)
-        {
-            executor.execute(new BusyWork(executor, stayBusy));
-        }
+        Semaphore busyWorkerPermits = new Semaphore(numBusyWorkers);
+        Thread makeBusy = new Thread(() -> {
+            while (stayBusy.get() == true)
 
 Review comment:
   nit: or `while (stayBusy.get())`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406837445
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -75,48 +78,67 @@ public void write(int b) { }
     }
 
     @Test
-    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException
+    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException, TimeoutException
     {
-        final int numBusyWorkers = 2; // Number of busy worker threads to run and gum things up
+        // Number of busy worker threads to run and gum things up. Chosen to be
+        // between the low and high max pool size so the test exercises resizing
+        // under a number of different conditions.
+        final int numBusyWorkers = 2;
         SharedExecutorPool sharedPool = new SharedExecutorPool("ChangingMaxWorkersMeetsConcurrencyGoalsTest");
         final AtomicInteger notifiedMaxPoolSize = new AtomicInteger();
 
         LocalAwareExecutorService executor = sharedPool.newExecutor(0, notifiedMaxPoolSize::set, 4, "internal", "resizetest");
 
+        // Keep feeding the executor work while resizing
+        // so it stays under load.
         AtomicBoolean stayBusy = new AtomicBoolean(true);
-        for (int i = 0; i < numBusyWorkers; i++)
-        {
-            executor.execute(new BusyWork(executor, stayBusy));
-        }
+        Semaphore busyWorkerPermits = new Semaphore(numBusyWorkers);
+        Thread makeBusy = new Thread(() -> {
+            while (stayBusy.get() == true)
+            {
+                try
+                {
+                    if (busyWorkerPermits.tryAcquire(1, MILLISECONDS)) {
+                        executor.execute(new BusyWork(busyWorkerPermits));
+                    }
+                }
+                catch (InterruptedException e)
+                {
+                    // ignore, will either stop looping if done or retry the lock
+                }
+            }
+        });
 
-        final int previousConcurrency = executor.getMaximumPoolSize();
+        makeBusy.start();
         try
         {
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+            for (int repeat = 0; repeat < 1000; repeat++)
+            {
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 2);
-            Assert.assertEquals(2, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 2);
+                Assert.assertEquals(2, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 3);
-            Assert.assertEquals(3, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 3);
+                Assert.assertEquals(3, notifiedMaxPoolSize.get());
 
-            executor.setMaximumPoolSize(0);
-            Assert.assertEquals(0, notifiedMaxPoolSize.get());
+                executor.setMaximumPoolSize(0);
+                Assert.assertEquals(0, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 4);
-            Assert.assertEquals(4, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 4);
+                Assert.assertEquals(4, notifiedMaxPoolSize.get());
+            }
         }
         finally
         {
             stayBusy.set(false);
-            executor.setMaximumPoolSize(previousConcurrency);
-            executor.shutdownNow();
-            Assert.assertTrue(executor.isShutdown());
-            Assert.assertTrue(executor.awaitTermination(1L, TimeUnit.MINUTES));
+            makeBusy.join(TimeUnit.SECONDS.toNanos(5));
 
 Review comment:
   `java.lang.Thread#join(long)` is milliseconds, you are providing nanos.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406841137
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -174,9 +191,9 @@ void assertMaxTaskConcurrency(LocalAwareExecutorService executor, int concurrenc
         CountDownLatch concurrencyGoal = new CountDownLatch(concurrency);
         for (int i = 0; i < concurrency; i++)
         {
-            executor.execute(new LatchWaiter(concurrencyGoal, 5L, TimeUnit.SECONDS));
+            executor.execute(new LatchWaiter(concurrencyGoal, 5000L, TimeUnit.MILLISECONDS));
         }
         // Will return true if all of the LatchWaiters count down before the timeout
-        Assert.assertEquals(true, concurrencyGoal.await(3L, TimeUnit.SECONDS));
+        Assert.assertEquals(true, concurrencyGoal.await(3000L, TimeUnit.MILLISECONDS));
 
 Review comment:
   while you are here, can you provide a more meaningful error message? =D

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406844616
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -75,48 +78,67 @@ public void write(int b) { }
     }
 
     @Test
-    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException
+    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException, TimeoutException
     {
-        final int numBusyWorkers = 2; // Number of busy worker threads to run and gum things up
+        // Number of busy worker threads to run and gum things up. Chosen to be
+        // between the low and high max pool size so the test exercises resizing
+        // under a number of different conditions.
+        final int numBusyWorkers = 2;
         SharedExecutorPool sharedPool = new SharedExecutorPool("ChangingMaxWorkersMeetsConcurrencyGoalsTest");
         final AtomicInteger notifiedMaxPoolSize = new AtomicInteger();
 
         LocalAwareExecutorService executor = sharedPool.newExecutor(0, notifiedMaxPoolSize::set, 4, "internal", "resizetest");
 
+        // Keep feeding the executor work while resizing
+        // so it stays under load.
         AtomicBoolean stayBusy = new AtomicBoolean(true);
-        for (int i = 0; i < numBusyWorkers; i++)
-        {
-            executor.execute(new BusyWork(executor, stayBusy));
-        }
+        Semaphore busyWorkerPermits = new Semaphore(numBusyWorkers);
+        Thread makeBusy = new Thread(() -> {
+            while (stayBusy.get() == true)
+            {
+                try
+                {
+                    if (busyWorkerPermits.tryAcquire(1, MILLISECONDS)) {
+                        executor.execute(new BusyWork(busyWorkerPermits));
+                    }
+                }
+                catch (InterruptedException e)
+                {
+                    // ignore, will either stop looping if done or retry the lock
+                }
+            }
+        });
 
-        final int previousConcurrency = executor.getMaximumPoolSize();
+        makeBusy.start();
         try
         {
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+            for (int repeat = 0; repeat < 1000; repeat++)
+            {
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 2);
-            Assert.assertEquals(2, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 2);
+                Assert.assertEquals(2, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 3);
-            Assert.assertEquals(3, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 3);
+                Assert.assertEquals(3, notifiedMaxPoolSize.get());
 
-            executor.setMaximumPoolSize(0);
-            Assert.assertEquals(0, notifiedMaxPoolSize.get());
+                executor.setMaximumPoolSize(0);
+                Assert.assertEquals(0, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 4);
-            Assert.assertEquals(4, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 4);
+                Assert.assertEquals(4, notifiedMaxPoolSize.get());
+            }
         }
         finally
         {
             stayBusy.set(false);
-            executor.setMaximumPoolSize(previousConcurrency);
-            executor.shutdownNow();
-            Assert.assertTrue(executor.isShutdown());
-            Assert.assertTrue(executor.awaitTermination(1L, TimeUnit.MINUTES));
+            makeBusy.join(TimeUnit.SECONDS.toNanos(5));
 
 Review comment:
   heh, that's why symbolic names are good. Amazing you can forget what you read in the docs between changing back to the source and modifying.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406836614
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -75,48 +78,67 @@ public void write(int b) { }
     }
 
     @Test
-    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException
+    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException, TimeoutException
     {
-        final int numBusyWorkers = 2; // Number of busy worker threads to run and gum things up
+        // Number of busy worker threads to run and gum things up. Chosen to be
+        // between the low and high max pool size so the test exercises resizing
+        // under a number of different conditions.
+        final int numBusyWorkers = 2;
         SharedExecutorPool sharedPool = new SharedExecutorPool("ChangingMaxWorkersMeetsConcurrencyGoalsTest");
         final AtomicInteger notifiedMaxPoolSize = new AtomicInteger();
 
         LocalAwareExecutorService executor = sharedPool.newExecutor(0, notifiedMaxPoolSize::set, 4, "internal", "resizetest");
 
+        // Keep feeding the executor work while resizing
+        // so it stays under load.
         AtomicBoolean stayBusy = new AtomicBoolean(true);
-        for (int i = 0; i < numBusyWorkers; i++)
-        {
-            executor.execute(new BusyWork(executor, stayBusy));
-        }
+        Semaphore busyWorkerPermits = new Semaphore(numBusyWorkers);
+        Thread makeBusy = new Thread(() -> {
+            while (stayBusy.get() == true)
 
 Review comment:
   or `while (stayBusy.get())`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406838537
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -174,9 +191,9 @@ void assertMaxTaskConcurrency(LocalAwareExecutorService executor, int concurrenc
         CountDownLatch concurrencyGoal = new CountDownLatch(concurrency);
         for (int i = 0; i < concurrency; i++)
         {
-            executor.execute(new LatchWaiter(concurrencyGoal, 5L, TimeUnit.SECONDS));
+            executor.execute(new LatchWaiter(concurrencyGoal, 5000L, TimeUnit.MILLISECONDS));
         }
         // Will return true if all of the LatchWaiters count down before the timeout
-        Assert.assertEquals(true, concurrencyGoal.await(3L, TimeUnit.SECONDS));
+        Assert.assertEquals(true, concurrencyGoal.await(3000L, TimeUnit.MILLISECONDS));
 
 Review comment:
   is there a reason you changed the units?  the values are the same so not sure if there is a reason.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406837587
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -75,48 +78,67 @@ public void write(int b) { }
     }
 
     @Test
-    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException
+    public void changingMaxWorkersMeetsConcurrencyGoalsTest() throws InterruptedException, TimeoutException
     {
-        final int numBusyWorkers = 2; // Number of busy worker threads to run and gum things up
+        // Number of busy worker threads to run and gum things up. Chosen to be
+        // between the low and high max pool size so the test exercises resizing
+        // under a number of different conditions.
+        final int numBusyWorkers = 2;
         SharedExecutorPool sharedPool = new SharedExecutorPool("ChangingMaxWorkersMeetsConcurrencyGoalsTest");
         final AtomicInteger notifiedMaxPoolSize = new AtomicInteger();
 
         LocalAwareExecutorService executor = sharedPool.newExecutor(0, notifiedMaxPoolSize::set, 4, "internal", "resizetest");
 
+        // Keep feeding the executor work while resizing
+        // so it stays under load.
         AtomicBoolean stayBusy = new AtomicBoolean(true);
-        for (int i = 0; i < numBusyWorkers; i++)
-        {
-            executor.execute(new BusyWork(executor, stayBusy));
-        }
+        Semaphore busyWorkerPermits = new Semaphore(numBusyWorkers);
+        Thread makeBusy = new Thread(() -> {
+            while (stayBusy.get() == true)
+            {
+                try
+                {
+                    if (busyWorkerPermits.tryAcquire(1, MILLISECONDS)) {
+                        executor.execute(new BusyWork(busyWorkerPermits));
+                    }
+                }
+                catch (InterruptedException e)
+                {
+                    // ignore, will either stop looping if done or retry the lock
+                }
+            }
+        });
 
-        final int previousConcurrency = executor.getMaximumPoolSize();
+        makeBusy.start();
         try
         {
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+            for (int repeat = 0; repeat < 1000; repeat++)
+            {
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 2);
-            Assert.assertEquals(2, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 2);
+                Assert.assertEquals(2, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 1);
-            Assert.assertEquals(1, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 1);
+                Assert.assertEquals(1, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 3);
-            Assert.assertEquals(3, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 3);
+                Assert.assertEquals(3, notifiedMaxPoolSize.get());
 
-            executor.setMaximumPoolSize(0);
-            Assert.assertEquals(0, notifiedMaxPoolSize.get());
+                executor.setMaximumPoolSize(0);
+                Assert.assertEquals(0, notifiedMaxPoolSize.get());
 
-            assertMaxTaskConcurrency(executor, 4);
-            Assert.assertEquals(4, notifiedMaxPoolSize.get());
+                assertMaxTaskConcurrency(executor, 4);
+                Assert.assertEquals(4, notifiedMaxPoolSize.get());
+            }
         }
         finally
         {
             stayBusy.set(false);
-            executor.setMaximumPoolSize(previousConcurrency);
-            executor.shutdownNow();
-            Assert.assertTrue(executor.isShutdown());
-            Assert.assertTrue(executor.awaitTermination(1L, TimeUnit.MINUTES));
+            makeBusy.join(TimeUnit.SECONDS.toNanos(5));
+            Assert.assertFalse(makeBusy.isAlive());
 
 Review comment:
   nit: useful failure message would be nice.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #534: CASSANDRA-15709 Fix flaky SEPExecutor.changingMaxWorkersMeetsConcurrencyGoalsTest
URL: https://github.com/apache/cassandra/pull/534#discussion_r406844635
 
 

 ##########
 File path: test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java
 ##########
 @@ -174,9 +191,9 @@ void assertMaxTaskConcurrency(LocalAwareExecutorService executor, int concurrenc
         CountDownLatch concurrencyGoal = new CountDownLatch(concurrency);
         for (int i = 0; i < concurrency; i++)
         {
-            executor.execute(new LatchWaiter(concurrencyGoal, 5L, TimeUnit.SECONDS));
+            executor.execute(new LatchWaiter(concurrencyGoal, 5000L, TimeUnit.MILLISECONDS));
         }
         // Will return true if all of the LatchWaiters count down before the timeout
-        Assert.assertEquals(true, concurrencyGoal.await(3L, TimeUnit.SECONDS));
+        Assert.assertEquals(true, concurrencyGoal.await(3000L, TimeUnit.MILLISECONDS));
 
 Review comment:
   leftover cruft from local reproduction - will revert and improve message.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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