You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@aurora.apache.org by Zameer Manji <zm...@apache.org> on 2016/12/02 03:19:01 UTC

Review Request 54288: Make leader elections resilient to ZK disconnections.

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157844
-----------------------------------------------------------



Master (4bc5246) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.exec.EstablishBuildEnvironment.doBuild(EstablishBuildEnvironment.java:72)
	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.HintGCAfterBuild.execute(HintGCAfterBuild.java:44)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
:test

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testLeadAdvertise FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:84
    java.lang.AssertionError
I1202 22:25:10.665 [ShutdownHook, SchedulerMain] Stopping scheduler services. 

1068 tests completed, 1 failed, 2 skipped
:test FAILED
:jacocoTestReport
Coverage report generated: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/jacoco/test/html/index.html
:analyzeReport
Instruction coverage is 0.8879984514130855, but must be greater than 0.89
Branch coverage is 0.8017550857598723, but must be greater than 0.835

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':test'.
> There were failing tests. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/tests/test/index.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 3 mins 59.199 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 2, 2016, 3:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 3:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157840
-----------------------------------------------------------



@ReviewBot retry

- Zameer Manji


On Dec. 1, 2016, 7:19 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 1, 2016, 7:19 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158815
-----------------------------------------------------------



Master (c8e8953) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.EstablishBuildEnvironment.doBuild(EstablishBuildEnvironment.java:72)
	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.HintGCAfterBuild.execute(HintGCAfterBuild.java:44)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
:test

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testZKDisconnection FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:233
I1212 01:48:33.031 [ShutdownHook, SchedulerMain] Stopping scheduler services. 

1068 tests completed, 1 failed, 2 skipped
:test FAILED
:jacocoTestReport
Coverage report generated: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/jacoco/test/html/index.html
:analyzeReport
Instruction coverage is 0.8877981722428748, but must be greater than 0.89
Branch coverage is 0.8013562026326286, but must be greater than 0.835

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':test'.
> There were failing tests. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/tests/test/index.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 3 mins 46.037 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 12, 2016, 12:43 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 12, 2016, 12:43 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162687
-----------------------------------------------------------



NB: The [5-6 interdiff](https://reviews.apache.org/r/54288/diff/5-6/) is from me and represents minor changes to remove flakiness of tests. These changes now withstand >25 serial runs whereas they failed generally with <10 serial runs before.

I took this opportunity to review Zameer's changes in detail, including the use of a single thread executor - which looked scary in isolation - and this all looks good to me.

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162697
-----------------------------------------------------------



@ReviewBot retry

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162703
-----------------------------------------------------------



@ReviewBot retry

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162696
-----------------------------------------------------------


Ship it!




Master (dc6f27e) is green with this patch.
  ./build-support/jenkins/build.sh

I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Joshua Cohen <jc...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162693
-----------------------------------------------------------


Ship it!




Ship It!

- Joshua Cohen


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162709
-----------------------------------------------------------


Ship it!




Master (75129b6) is green with this patch.
  ./build-support/jenkins/build.sh

I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.

> On Jan. 23, 2017, 2:16 p.m., Aurora ReviewBot wrote:
> > Master (75129b6) is red with this patch.
> >   ./build-support/jenkins/build.sh
> > 
> >   Test coverage missing for org/apache/aurora/scheduler/events/Webhook
> >   Test coverage missing for org/apache/aurora/scheduler/events/WebhookInfo
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/EntrySerializer$EntrySerializerImpl$1
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$8
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$7
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$4
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$3
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$6
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$5
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$2
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$1
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorage$Settings
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorage$ScheduledExecutorSchedulingService
> >   Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorageModule
> >   Test coverage missing for org/apache/aurora/scheduler/storage/backup/TemporaryStorage$TemporaryStorageFactory$1
> >   Test coverage missing for org/apache/aurora/scheduler/storage/backup/BackupModule
> >   Test coverage missing for org/apache/aurora/scheduler/storage/backup/Recovery$RecoveryImpl
> >   Test coverage missing for org/apache/aurora/scheduler/storage/backup/TemporaryStorage$TemporaryStorageFactory
> >   Test coverage missing for org/apache/aurora/scheduler/storage/backup/Recovery$RecoveryImpl$PendingRecovery
> >   Test coverage missing for org/apache/aurora/scheduler/TaskVars
> >   Test coverage missing for org/apache/aurora/scheduler/SchedulerLifecycle$DefaultDelayedActions
> >   Test coverage missing for org/apache/aurora/scheduler/TierManager$TierManagerImpl$TierConfig
> >   Test coverage missing for org/apache/aurora/scheduler/TaskVars$Counter
> >   Test coverage missing for org/apache/aurora/scheduler/TaskVars$1
> >   Test coverage missing for org/apache/aurora/scheduler/SchedulerModule$TaskEventBatchWorker
> >   Test coverage missing for org/apache/aurora/scheduler/HostOffer$1
> >   Test coverage missing for org/apache/aurora/scheduler/SchedulerModule
> >   Test coverage missing for org/apache/aurora/scheduler/TaskIdGenerator$TaskIdGeneratorImpl
> >   Test coverage missing for org/apache/aurora/scheduler/SchedulerModule$1
> >   Test coverage missing for org/apache/aurora/scheduler/TaskStatusHandlerImpl
> >   Test coverage missing for org/apache/aurora/scheduler/TaskStatusHandlerImpl$1
> > 
> > * Try:
> > Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.
> > ==============================================================================
> > 
> > BUILD FAILED
> > 
> > Total time: 4 mins 50.396 secs
> > 
> > 
> > I will refresh this build result if you post a review containing "@ReviewBot retry"

This was an incidence of the now well-known:
```
:testJava HotSpot(TM) 64-Bit Server VM warning: INFO: os::commit_memory(0x00000007a8e80000, 298319872, 0) failed; error='Cannot allocate memory' (errno=12)
```

I'll spin one more retry and then submit this if green.


- John


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162700
-----------------------------------------------------------


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162700
-----------------------------------------------------------



Master (75129b6) is red with this patch.
  ./build-support/jenkins/build.sh

  Test coverage missing for org/apache/aurora/scheduler/events/Webhook
  Test coverage missing for org/apache/aurora/scheduler/events/WebhookInfo
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl
  Test coverage missing for org/apache/aurora/scheduler/storage/log/EntrySerializer$EntrySerializerImpl$1
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$8
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$7
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$4
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$3
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$6
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$5
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$2
  Test coverage missing for org/apache/aurora/scheduler/storage/log/SnapshotStoreImpl$1
  Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorage$Settings
  Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorage$ScheduledExecutorSchedulingService
  Test coverage missing for org/apache/aurora/scheduler/storage/log/LogStorageModule
  Test coverage missing for org/apache/aurora/scheduler/storage/backup/TemporaryStorage$TemporaryStorageFactory$1
  Test coverage missing for org/apache/aurora/scheduler/storage/backup/BackupModule
  Test coverage missing for org/apache/aurora/scheduler/storage/backup/Recovery$RecoveryImpl
  Test coverage missing for org/apache/aurora/scheduler/storage/backup/TemporaryStorage$TemporaryStorageFactory
  Test coverage missing for org/apache/aurora/scheduler/storage/backup/Recovery$RecoveryImpl$PendingRecovery
  Test coverage missing for org/apache/aurora/scheduler/TaskVars
  Test coverage missing for org/apache/aurora/scheduler/SchedulerLifecycle$DefaultDelayedActions
  Test coverage missing for org/apache/aurora/scheduler/TierManager$TierManagerImpl$TierConfig
  Test coverage missing for org/apache/aurora/scheduler/TaskVars$Counter
  Test coverage missing for org/apache/aurora/scheduler/TaskVars$1
  Test coverage missing for org/apache/aurora/scheduler/SchedulerModule$TaskEventBatchWorker
  Test coverage missing for org/apache/aurora/scheduler/HostOffer$1
  Test coverage missing for org/apache/aurora/scheduler/SchedulerModule
  Test coverage missing for org/apache/aurora/scheduler/TaskIdGenerator$TaskIdGeneratorImpl
  Test coverage missing for org/apache/aurora/scheduler/SchedulerModule$1
  Test coverage missing for org/apache/aurora/scheduler/TaskStatusHandlerImpl
  Test coverage missing for org/apache/aurora/scheduler/TaskStatusHandlerImpl$1

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.
==============================================================================

BUILD FAILED

Total time: 4 mins 50.396 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162721
-----------------------------------------------------------


Ship it!




Ship It!

- Zameer Manji


On Jan. 23, 2017, 12:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 12:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162691
-----------------------------------------------------------



@ReviewBot retry

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162688
-----------------------------------------------------------


Ship it!




Ship It!

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162690
-----------------------------------------------------------


Ship it!




Master (dc6f27e) is green with this patch.
  ./build-support/jenkins/build.sh

I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by David McLaughlin <da...@dmclaughlin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162698
-----------------------------------------------------------


Ship it!




Ship It!

- David McLaughlin


On Jan. 23, 2017, 8:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 8:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review162710
-----------------------------------------------------------



Spoke with Zameer offline and he'll actually land this once he reviews the last change.

- John Sirois


On Jan. 23, 2017, 1:21 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Jan. 23, 2017, 1:21 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

(Updated Jan. 23, 2017, 1:21 p.m.)


Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Changes
-------

Fixup tests to be robust to interleaving variations.

 src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java     |  4 ++--
 src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java    |  1 -
 src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java | 33 ++++++++++++++++++---------------
 3 files changed, 20 insertions(+), 18 deletions(-)


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs (updated)
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158812
-----------------------------------------------------------



@ReviewBot retry

- Zameer Manji


On Dec. 11, 2016, 4:43 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 11, 2016, 4:43 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158811
-----------------------------------------------------------


Ship it!




Master (c8e8953) is green with this patch.
  ./build-support/jenkins/build.sh

I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 12, 2016, 12:43 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 12, 2016, 12:43 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

(Updated Dec. 11, 2016, 4:43 p.m.)


Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Changes
-------

Fix more possible timing issues.


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs (updated)
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158599
-----------------------------------------------------------



Master (f5749ce) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
:test

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testAbdicateTransition FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:125
    java.lang.AssertionError

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testLeadAdvertise FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:94
    java.lang.AssertionError
I1208 23:33:14.100 [ShutdownHook, SchedulerMain] Stopping scheduler services. 

1068 tests completed, 2 failed, 2 skipped
:test FAILED
:jacocoTestReport
Coverage report generated: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/jacoco/test/html/index.html
:analyzeReport
Instruction coverage is 0.887991790582404, but must be greater than 0.89
Branch coverage is 0.8021539688871161, but must be greater than 0.835

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':test'.
> There were failing tests. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/tests/test/index.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 4 mins 19.969 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 8, 2016, 11:28 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 11:28 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.

> On Dec. 8, 2016, 3:42 p.m., Zameer Manji wrote:
> > Does anyone know how to get the test reports from jenkins or have an idea of what's going on?
> 
> John Sirois wrote:
>     Yes, these are legit failures, no Jenkins logs needed, just this is enough:
>     ```
>     org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testAbdicateTransition FAILED
>         java.lang.AssertionError at CuratorSingletonServiceTest.java:125
>         java.lang.AssertionError
>     
>     org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testLeadAdvertise FAILED
>         java.lang.AssertionError at CuratorSingletonServiceTest.java:94
>         java.lang.AssertionError
>     ```
>     
>     I looked at the testLeadAdvertise one 1st which [blocks until an ephemeral node is added](https://github.com/apache/aurora/blob/master/src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java#L82-L83) to the leadership group path as the signal that leadership has been already synchronously obtained. The assumption is broken with LeaderSelector since the transition is [fired on an executor runnable](https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java#L239) and could be delayed arbitrarily past the test assertion.
>     
>     In short, you're getting lucky having these 2 (and maybe more) pass on your machine.  Needs careful re-review of the test infra and the new behavior of the LeaderSelector using the thread pool to do leadership transitions async.

Good catch.

The tests already have an `awaitCapture` and since I am adding a timeout, we can use that to ensure the listeners are fired instead of plain asserts.


- Zameer


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158601
-----------------------------------------------------------


On Dec. 8, 2016, 3:28 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 3:28 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by John Sirois <js...@apache.org>.

> On Dec. 8, 2016, 4:42 p.m., Zameer Manji wrote:
> > Does anyone know how to get the test reports from jenkins or have an idea of what's going on?

Yes, these are legit failures, no Jenkins logs needed, just this is enough:
```
org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testAbdicateTransition FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:125
    java.lang.AssertionError

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testLeadAdvertise FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:94
    java.lang.AssertionError
```

I looked at the testLeadAdvertise one 1st which [blocks until an ephemeral node is added](https://github.com/apache/aurora/blob/master/src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java#L82-L83) to the leadership group path as the signal that leadership has been already synchronously obtained. The assumption is broken with LeaderSelector since the transition is [fired on an executor runnable](https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java#L239) and could be delayed arbitrarily past the test assertion.

In short, you're getting lucky having these 2 (and maybe more) pass on your machine.  Needs careful re-review of the test infra and the new behavior of the LeaderSelector using the thread pool to do leadership transitions async.


- John


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158601
-----------------------------------------------------------


On Dec. 8, 2016, 4:28 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 4:28 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158601
-----------------------------------------------------------



Does anyone know how to get the test reports from jenkins or have an idea of what's going on?

- Zameer Manji


On Dec. 8, 2016, 3:28 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 3:28 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

(Updated Dec. 8, 2016, 3:28 p.m.)


Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Changes
-------

Take Karthik's suggestion and fix a possible timing issue in the failing test.


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs (updated)
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158597
-----------------------------------------------------------



Master (f5749ce) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.HintGCAfterBuild.execute(HintGCAfterBuild.java:44)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
:test

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testDefeatTransition FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:171
    java.lang.AssertionError
I1208 23:14:51.000 [SessionTracker, SessionTrackerImpl] SessionTrackerImpl exited loop! 
I1208 23:14:51.030 [ShutdownHook, SchedulerMain] Stopping scheduler services. 

1068 tests completed, 1 failed, 2 skipped
:test FAILED
:jacocoTestReport
Coverage report generated: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/jacoco/test/html/index.html
:analyzeReport
Instruction coverage is 0.8877981722428748, but must be greater than 0.89
Branch coverage is 0.8017550857598723, but must be greater than 0.835

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':test'.
> There were failing tests. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/tests/test/index.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 4 mins 2.711 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 8, 2016, 11:06 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 11:06 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

(Updated Dec. 8, 2016, 3:06 p.m.)


Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Changes
-------

Appease PMD


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs (updated)
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158593
-----------------------------------------------------------



Master (f5749ce) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.exec.RequestStopIfSingleUsedDaemon.execute(RequestStopIfSingleUsedDaemon.java:34)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.ForwardClientInput$2.call(ForwardClientInput.java:74)
	at org.gradle.launcher.daemon.server.exec.ForwardClientInput$2.call(ForwardClientInput.java:72)
	at org.gradle.util.Swapper.swap(Swapper.java:38)
	at org.gradle.launcher.daemon.server.exec.ForwardClientInput.execute(ForwardClientInput.java:72)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.LogAndCheckHealth.execute(LogAndCheckHealth.java:55)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.LogToClient.doBuild(LogToClient.java:60)
	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.EstablishBuildEnvironment.doBuild(EstablishBuildEnvironment.java:72)
	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.HintGCAfterBuild.execute(HintGCAfterBuild.java:44)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
/home/jenkins/jenkins-slave/workspace/AuroraBot/src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java:47:	Private field 'timeout' could be made final; it is only initialized in the declaration or constructor.
:pmdTest FAILED

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':pmdTest'.
> 1 PMD rule violations were found. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/pmd/test.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 3 mins 19.579 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 8, 2016, 10:24 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 8, 2016, 10:24 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/
-----------------------------------------------------------

(Updated Dec. 8, 2016, 2:24 p.m.)


Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.


Changes
-------

Add timeout rule.


Bugs: AURORA-1669
    https://issues.apache.org/jira/browse/AURORA-1669


Repository: aurora


Description
-------

As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
leadership if the ZK connection is lost or if there is a timeout. This is not
compatible with the commons based implementation which would only abdicate
leadership if the ZK session timeout occurred.

This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
custom listener that only loses leadership if a connection loss occurs.


Diffs (updated)
-----

  commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
  src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
  src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
  src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 

Diff: https://reviews.apache.org/r/54288/diff/


Testing
-------


Thanks,

Zameer Manji


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Aurora ReviewBot <wf...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157696
-----------------------------------------------------------



Master (3ea0331) is red with this patch.
  ./build-support/jenkins/build.sh

	at org.gradle.launcher.daemon.server.exec.EstablishBuildEnvironment.doBuild(EstablishBuildEnvironment.java:72)
	at org.gradle.launcher.daemon.server.exec.BuildCommandOnly.execute(BuildCommandOnly.java:36)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.HintGCAfterBuild.execute(HintGCAfterBuild.java:44)
	at org.gradle.launcher.daemon.server.api.DaemonCommandExecution.proceed(DaemonCommandExecution.java:120)
	at org.gradle.launcher.daemon.server.exec.StartBuildOrRespondWithBusy$1.run(StartBuildOrRespondWithBusy.java:50)
	at org.gradle.launcher.daemon.server.DaemonStateCoordinator$1.run(DaemonStateCoordinator.java:293)
	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
:pmdTest
:test

org.apache.aurora.scheduler.discovery.CuratorSingletonServiceTest > testLeadAdvertise FAILED
    java.lang.AssertionError at CuratorSingletonServiceTest.java:84
    java.lang.AssertionError
I1202 04:35:28.388 [ShutdownHook, SchedulerMain:101] Stopping scheduler services. 

1068 tests completed, 1 failed, 2 skipped
:test FAILED
:jacocoTestReport
Coverage report generated: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/jacoco/test/html/index.html
:analyzeReport
Instruction coverage is 0.8882695620197798, but must be greater than 0.89
Branch coverage is 0.8025528520143598, but must be greater than 0.835

FAILURE: Build failed with an exception.

* What went wrong:
Execution failed for task ':test'.
> There were failing tests. See the report at: file:///home/jenkins/jenkins-slave/workspace/AuroraBot/dist/reports/tests/test/index.html

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug option to get more log output.

BUILD FAILED

Total time: 4 mins 2.75 secs


I will refresh this build result if you post a review containing "@ReviewBot retry"

- Aurora ReviewBot


On Dec. 2, 2016, 3:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 3:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Karthik Anantha Padmanabhan <tr...@gmail.com>.

> On Dec. 2, 2016, 10:37 p.m., Zameer Manji wrote:
> > Can someone else validate that the tests pass for them locally? I can't reproduce the jenkins failure.

I wonder if this is a timing issue in the tests ?

`expectGroupEvent(PathChildrenCacheEvent.Type.CHILD_ADDED)` returns as soon as there is a CHILD_ADDED event. Is it possible `listener.onLeading` isn't called by then ?

assertTrue(capture.hasCaptured())


- Karthik


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157847
-----------------------------------------------------------


On Dec. 2, 2016, 3:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 3:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157847
-----------------------------------------------------------



Can someone else validate that the tests pass for them locally? I can't reproduce the jenkins failure.

- Zameer Manji


On Dec. 1, 2016, 7:19 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 1, 2016, 7:19 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Stephan Erb <se...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review158567
-----------------------------------------------------------


Ship it!




LGTM. Please also add an entry to the changelog.

- Stephan Erb


On Dec. 2, 2016, 4:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 4:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Zameer Manji <zm...@apache.org>.

> On Dec. 2, 2016, 7:58 a.m., Joshua Cohen wrote:
> > Thanks for picking this up! This is a basic question, but I just want to be sure: by mimicking the old behavior, we're not running the risk of re-introducing the same deadlock we were trying to fix by moving to Curator, right? I'm not sure where the deadlock was caused... was it in our implementation of the `SingletonService` recipe, was it in the ZK client itself, or somewhere else entirely?

From my understanding it was our implementation of the SingletonService code.


> On Dec. 2, 2016, 7:58 a.m., Joshua Cohen wrote:
> > src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java, lines 215-218
> > <https://reviews.apache.org/r/54288/diff/1/?file=1574553#file1574553line215>
> >
> >     Should we have an escape hatch for the case where we never become leader in this test (i.e. sleep for up to N seconds then `fail()`)?

Perhaps I could just add a timeout to the test? JUnit allows me to do that.


- Zameer


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157759
-----------------------------------------------------------


On Dec. 1, 2016, 7:19 p.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 1, 2016, 7:19 p.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Joshua Cohen <jc...@apache.org>.

> On Dec. 2, 2016, 3:58 p.m., Joshua Cohen wrote:
> > src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java, lines 215-218
> > <https://reviews.apache.org/r/54288/diff/1/?file=1574553#file1574553line215>
> >
> >     Should we have an escape hatch for the case where we never become leader in this test (i.e. sleep for up to N seconds then `fail()`)?
> 
> Zameer Manji wrote:
>     Perhaps I could just add a timeout to the test? JUnit allows me to do that.

Timeout on the test works for me.


- Joshua


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157759
-----------------------------------------------------------


On Dec. 2, 2016, 3:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 3:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>


Re: Review Request 54288: Make leader elections resilient to ZK disconnections.

Posted by Joshua Cohen <jc...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/54288/#review157759
-----------------------------------------------------------



Thanks for picking this up! This is a basic question, but I just want to be sure: by mimicking the old behavior, we're not running the risk of re-introducing the same deadlock we were trying to fix by moving to Curator, right? I'm not sure where the deadlock was caused... was it in our implementation of the `SingletonService` recipe, was it in the ZK client itself, or somewhere else entirely?


src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java (lines 215 - 218)
<https://reviews.apache.org/r/54288/#comment228374>

    Should we have an escape hatch for the case where we never become leader in this test (i.e. sleep for up to N seconds then `fail()`)?


- Joshua Cohen


On Dec. 2, 2016, 3:19 a.m., Zameer Manji wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/54288/
> -----------------------------------------------------------
> 
> (Updated Dec. 2, 2016, 3:19 a.m.)
> 
> 
> Review request for Aurora, David McLaughlin, Joshua Cohen, John Sirois, and Stephan Erb.
> 
> 
> Bugs: AURORA-1669
>     https://issues.apache.org/jira/browse/AURORA-1669
> 
> 
> Repository: aurora
> 
> 
> Description
> -------
> 
> As documented in AURORA-1840 the Curator `LeaderLatch` recipe abdicates
> leadership if the ZK connection is lost or if there is a timeout. This is not
> compatible with the commons based implementation which would only abdicate
> leadership if the ZK session timeout occurred.
> 
> This replaces the `LeaderLatch` recipe with the `LeaderSelector` recipe with a
> custom listener that only loses leadership if a connection loss occurs.
> 
> 
> Diffs
> -----
> 
>   commons/src/main/java/org/apache/aurora/common/zookeeper/testing/ZooKeeperTestServer.java 50acaeba82e163f8f2970a264cbd889c9eb3b5ed 
>   src/main/java/org/apache/aurora/scheduler/discovery/CuratorSingletonService.java c378172c850aafe0a9381552b5067277b40dbfab 
>   src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java a2b4125369d1f6c0a79bc4ac0fb3d2dab8a6c583 
>   src/test/java/org/apache/aurora/scheduler/discovery/CuratorSingletonServiceTest.java 6ea49b0c690d288ff59d1d4798144bfa2d153d3a 
> 
> Diff: https://reviews.apache.org/r/54288/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Zameer Manji
> 
>