You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2016/04/12 17:10:24 UTC

[01/10] mesos git commit: Speeded up MasterAllocatorTest.SlaveLost test.

Repository: mesos
Updated Branches:
  refs/heads/master 371072e97 -> a326e71b7


Speeded up MasterAllocatorTest.SlaveLost test.

Set the `executor_shutdown_grace_period` to a small value so that
the agent does not wait for executors to clean up for too long.

Review: https://reviews.apache.org/r/43329/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/37f382a7
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/37f382a7
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/37f382a7

Branch: refs/heads/master
Commit: 37f382a7901e702fb48c13f2f11139328934d473
Parents: 371072e
Author: Shuai Lin <li...@gmail.com>
Authored: Tue Apr 12 12:53:59 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 13:00:55 2016 +0200

----------------------------------------------------------------------
 src/tests/master_allocator_tests.cpp | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/37f382a7/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 404ff09..17607df 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -666,6 +666,10 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   TestContainerizer containerizer(&exec);
 
   slave::Flags flags1 = this->CreateSlaveFlags();
+
+  // Set the `executor_shutdown_grace_period` to a small value so that
+  // the agent does not wait for executors to clean up for too long.
+  flags1.executor_shutdown_grace_period = Milliseconds(50);
   flags1.resources = Some("cpus:2;mem:1024");
 
   EXPECT_CALL(allocator, addSlave(_, _, _, _, _));


[08/10] mesos git commit: Speeded up OversubscriptionTest.RemoveCapabilitiesOnSchedulerFailover.

Posted by al...@apache.org.
Speeded up OversubscriptionTest.RemoveCapabilitiesOnSchedulerFailover.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43522/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/f350dbb6
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/f350dbb6
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/f350dbb6

Branch: refs/heads/master
Commit: f350dbb61368947885989fdf955d45fe7770b381
Parents: 30483d9
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 15:07:24 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 17:01:45 2016 +0200

----------------------------------------------------------------------
 src/tests/oversubscription_tests.cpp | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f350dbb6/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index aa623e8..07c6658 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -1034,7 +1034,8 @@ TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
 TEST_F(OversubscriptionTest, RemoveCapabilitiesOnSchedulerFailover)
 {
   // Start the master.
-  Try<Owned<cluster::Master>> master = StartMaster();
+  master::Flags masterFlags = MesosTest::CreateMasterFlags();
+  Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
   // Start the slave with mock executor and test resource estimator.
@@ -1124,6 +1125,12 @@ TEST_F(OversubscriptionTest, RemoveCapabilitiesOnSchedulerFailover)
 
   driver2.start();
 
+  // Ensure resources are be recovered before a batch allocation is triggered.
+  Clock::pause();
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   AWAIT_READY(offers3);
   EXPECT_NE(0u, offers3.get().size());
   EXPECT_TRUE(Resources(offers3.get()[0].resources()).revocable().empty());


[07/10] mesos git commit: Speeded up OversubscriptionTest.UpdateAllocatorOnSchedulerFailover.

Posted by al...@apache.org.
Speeded up OversubscriptionTest.UpdateAllocatorOnSchedulerFailover.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43521/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/30483d90
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/30483d90
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/30483d90

Branch: refs/heads/master
Commit: 30483d9011fc0a8e9a94a20beb994266e1fcd90a
Parents: 424b105
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 15:02:27 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 15:06:19 2016 +0200

----------------------------------------------------------------------
 src/tests/oversubscription_tests.cpp | 16 +++++++++++-----
 1 file changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/30483d90/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index 2367174..aa623e8 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -912,7 +912,8 @@ TEST_F(OversubscriptionTest, QoSCorrectionKill)
 //   5. Check if revocable offers are being sent to the framework.
 TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
 {
-  Try<Owned<cluster::Master>> master = StartMaster();
+  master::Flags masterFlags = MesosTest::CreateMasterFlags();
+  Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -996,14 +997,19 @@ TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
 
   driver2.start();
 
-  AWAIT_READY(offers1);
-  EXPECT_NE(0u, offers1.get().size());
-  EXPECT_TRUE(Resources(offers1.get()[0].resources()).revocable().empty());
-
   AWAIT_READY(sched2Registered);
 
   AWAIT_READY(sched1Error);
 
+  // Advance the clock and trigger a batch allocation.
+  Clock::pause();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
+  AWAIT_READY(offers1);
+  EXPECT_NE(0u, offers1.get().size());
+  EXPECT_TRUE(Resources(offers1.get()[0].resources()).revocable().empty());
+
   // Check if framework receives revocable offers.
   Future<vector<Offer>> offers2;
   EXPECT_CALL(sched2, resourceOffers(&driver2, _))


[09/10] mesos git commit: Speeded up SchedulerTest.Decline.

Posted by al...@apache.org.
Speeded up SchedulerTest.Decline.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43321/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/a326e71b
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/a326e71b
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/a326e71b

Branch: refs/heads/master
Commit: a326e71b77bac3e243ec341addf71c984da57301
Parents: 29b4267
Author: Shuai Lin <li...@gmail.com>
Authored: Tue Apr 12 16:13:19 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 17:01:45 2016 +0200

----------------------------------------------------------------------
 src/tests/scheduler_tests.cpp | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a326e71b/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
index 7956da3..b630944 100644
--- a/src/tests/scheduler_tests.cpp
+++ b/src/tests/scheduler_tests.cpp
@@ -1041,6 +1041,9 @@ TEST_P(SchedulerTest, Decline)
   EXPECT_CALL(*scheduler, offers(_, _))
     .WillOnce(FutureArg<1>(&offers2));
 
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &MesosAllocatorProcess::recoverResources);
+
   {
     Call call;
     call.mutable_framework_id()->CopyFrom(frameworkId);
@@ -1057,6 +1060,13 @@ TEST_P(SchedulerTest, Decline)
     mesos.send(call);
   }
 
+  // Make sure the dispatch event for `recoverResources` has been enqueued.
+  AWAIT_READY(recoverResources);
+
+  Clock::pause();
+  Clock::advance(flags.allocation_interval);
+  Clock::resume();
+
   // If the resources were properly declined, the scheduler should
   // get another offer with same amount of resources.
   AWAIT_READY(offers2);


[04/10] mesos git commit: Speeded up MasterTest.MasterInfoOnReElection.

Posted by al...@apache.org.
Speeded up MasterTest.MasterInfoOnReElection.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43515/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/f9a068d0
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/f9a068d0
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/f9a068d0

Branch: refs/heads/master
Commit: f9a068d0c40b687f6120e4985b3d79a35def8acf
Parents: 2989e90
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 14:06:47 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 14:58:26 2016 +0200

----------------------------------------------------------------------
 src/tests/master_tests.cpp | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f9a068d0/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 8deb7fc..09fb9ff 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -1006,7 +1006,8 @@ TEST_F(MasterTest, MasterInfo)
 
 TEST_F(MasterTest, MasterInfoOnReElection)
 {
-  Try<Owned<cluster::Master>> master = StartMaster();
+  master::Flags masterFlags = CreateMasterFlags();
+  Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
   StandaloneMasterDetector detector(master.get()->pid);
@@ -1059,6 +1060,11 @@ TEST_F(MasterTest, MasterInfoOnReElection)
 
   EXPECT_EQ(MESOS_VERSION, masterInfo.get().version());
 
+  // Advance the clock and trigger a batch allocation.
+  Clock::pause();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   // The re-registered framework should get offers.
   AWAIT_READY(resourceOffers2);
 


[06/10] mesos git commit: Speeded up MasterTest.RecoverResources.

Posted by al...@apache.org.
Speeded up MasterTest.RecoverResources.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43514/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/2989e90d
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/2989e90d
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/2989e90d

Branch: refs/heads/master
Commit: 2989e90d45fd71f15429c83139294527224b74a7
Parents: e9ea989
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 13:40:55 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 14:58:26 2016 +0200

----------------------------------------------------------------------
 src/tests/master_tests.cpp | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2989e90d/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index a5b21d3..8deb7fc 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -662,7 +662,8 @@ TEST_F(MasterTest, StatusUpdateAck)
 
 TEST_F(MasterTest, RecoverResources)
 {
-  Try<Owned<cluster::Master>> master = StartMaster();
+  master::Flags masterFlags = CreateMasterFlags();
+  Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -764,6 +765,14 @@ TEST_F(MasterTest, RecoverResources)
   // Now kill the executor, scheduler should get an offer it's resources.
   containerizer.destroy(offer.framework_id(), executorInfo.executor_id());
 
+  // Ensure the container is destroyed, `ExitedExecutorMessage` message
+  // is received by the master and hence its resources will be recovered
+  // before a batch allocation is triggered.
+  Clock::pause();
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   // TODO(benh): We can't do driver.reviveOffers() because we need to
   // wait for the killed executors resources to get aggregated! We
   // should wait for the allocator to recover the resources first. See


[10/10] mesos git commit: Speeded up HookTest.VerifySlaveLaunchExecutorHook.

Posted by al...@apache.org.
Speeded up HookTest.VerifySlaveLaunchExecutorHook.

Review: https://reviews.apache.org/r/42241/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/29b42674
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/29b42674
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/29b42674

Branch: refs/heads/master
Commit: 29b426743fc684ac0839bfa009197de5dd827e56
Parents: f350dbb
Author: Jian Qiu <qi...@cn.ibm.com>
Authored: Tue Apr 12 15:22:25 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 17:01:45 2016 +0200

----------------------------------------------------------------------
 src/tests/hook_tests.cpp | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/29b42674/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 0bbb7ac..97ff55a 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -365,7 +365,9 @@ TEST_F(HookTest, VerifySlaveLaunchExecutorHook)
 
   // Executor shutdown would force the Slave to execute the
   // remove-executor hook.
-  EXPECT_CALL(exec, shutdown(_));
+  Future<Nothing> shutdown;
+  EXPECT_CALL(exec, shutdown(_))
+    .WillOnce(FutureSatisfy(&shutdown));;
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -384,6 +386,10 @@ TEST_F(HookTest, VerifySlaveLaunchExecutorHook)
   driver.stop();
   driver.join();
 
+  // Explicitly destroy the container.
+  AWAIT_READY(shutdown);
+  containerizer.destroy(offers.get()[0].framework_id(), DEFAULT_EXECUTOR_ID);
+
   // The scheduler shutdown from above forces the executor to
   // shutdown. This in turn should force the Slave to execute
   // the remove-executor hook.


[03/10] mesos git commit: Speeded up MasterTest.LaunchCombinedOfferTest.

Posted by al...@apache.org.
Speeded up MasterTest.LaunchCombinedOfferTest.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43516/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/b7770c6e
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/b7770c6e
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/b7770c6e

Branch: refs/heads/master
Commit: b7770c6e161fa013fb321e4b054c7250f83c8456
Parents: f9a068d
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 14:53:07 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 14:58:26 2016 +0200

----------------------------------------------------------------------
 src/tests/master_tests.cpp | 13 ++++++++++++-
 1 file changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b7770c6e/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 09fb9ff..ff9fa0e 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -1212,7 +1212,8 @@ TEST_F(MasterTest, MasterLost)
 // all slave resources and a single task should be able to run on these.
 TEST_F(MasterTest, LaunchCombinedOfferTest)
 {
-  Try<Owned<cluster::Master>> master = StartMaster();
+  master::Flags masterFlags = CreateMasterFlags();
+  Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -1280,6 +1281,11 @@ TEST_F(MasterTest, LaunchCombinedOfferTest)
   AWAIT_READY(status1);
   EXPECT_EQ(TASK_RUNNING, status1.get().state());
 
+  // Advance the clock and trigger a batch allocation.
+  Clock::pause();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   // Await 2nd offer.
   AWAIT_READY(offers2);
   EXPECT_NE(0u, offers2.get().size());
@@ -1307,6 +1313,11 @@ TEST_F(MasterTest, LaunchCombinedOfferTest)
   AWAIT_READY(status2);
   EXPECT_EQ(TASK_KILLED, status2.get().state());
 
+  // Advance the clock and trigger a batch allocation.
+  Clock::pause();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   // Await 3rd offer - 2nd and 3rd offer to same slave are now ready.
   AWAIT_READY(offers3);
   EXPECT_NE(0u, offers3.get().size());


[05/10] mesos git commit: Speeded up MasterTest.OfferTimeout.

Posted by al...@apache.org.
Speeded up MasterTest.OfferTimeout.

Advance the clock to trigger a batch allocation instantly.

Review: https://reviews.apache.org/r/43517/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/424b1050
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/424b1050
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/424b1050

Branch: refs/heads/master
Commit: 424b105046ae2847e570d7d62c57c1de09a0bd7f
Parents: b7770c6
Author: haosdent huang <ha...@gmail.com>
Authored: Tue Apr 12 14:55:28 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 14:58:26 2016 +0200

----------------------------------------------------------------------
 src/tests/master_tests.cpp | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/424b1050/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index ff9fa0e..806a958 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -2598,6 +2598,11 @@ TEST_F(MasterTest, OfferTimeout)
 
   AWAIT_READY(recoverResources);
 
+  // Advance the clock and trigger a batch allocation.
+  Clock::pause();
+  Clock::advance(masterFlags.allocation_interval);
+  Clock::resume();
+
   // Expect that the resources are re-offered to the framework after
   // the rescind.
   AWAIT_READY(offers2);


[02/10] mesos git commit: Speeded up RecoverTest.AutoInitialization.

Posted by al...@apache.org.
Speeded up RecoverTest.AutoInitialization.

Advance the clock to avoid waiting for a retry delay
used by log replicas in recovery protocol.

Review: https://reviews.apache.org/r/43524/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/e9ea9890
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/e9ea9890
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/e9ea9890

Branch: refs/heads/master
Commit: e9ea9890bcc0c75cc6a5d5447b9f63cdf25fd28e
Parents: 37f382a
Author: Shuai Lin <li...@gmail.com>
Authored: Tue Apr 12 13:15:17 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Tue Apr 12 14:58:18 2016 +0200

----------------------------------------------------------------------
 src/tests/log_tests.cpp | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e9ea9890/src/tests/log_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/log_tests.cpp b/src/tests/log_tests.cpp
index 8f208ba..85fc9d4 100644
--- a/src/tests/log_tests.cpp
+++ b/src/tests/log_tests.cpp
@@ -1865,6 +1865,16 @@ TEST_F(RecoverTest, AutoInitialization)
 
   Future<Owned<Replica> > recovering3 = recover(2, replica3, network, true);
 
+  Clock::pause();
+  Clock::settle();
+
+  // At this moment `replica1` and `replica2` are in EMPTY status, and
+  // are retrying with a random interval between [0.5 sec, 1 sec]. Since
+  // the retry interval is hard coded and is not configurable, we need
+  // to advance the clock here to avoid waiting for the backoff time.
+  Clock::advance(Seconds(1));
+  Clock::resume();
+
   AWAIT_READY(recovering1);
   AWAIT_READY(recovering2);
   AWAIT_READY(recovering3);