From: <tho...@us...> - 2013-03-11 16:02:54
|
Revision: 6980 http://bigdata.svn.sourceforge.net/bigdata/?rev=6980&view=rev Author: thompsonbry Date: 2013-03-11 16:02:46 +0000 (Mon, 11 Mar 2013) Log Message: ----------- Exposed public method for getCastVoteIfConsensus() and new test suites for pipeline reorg and getCastVoteIfConsensus() from Martyn. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java branches/READ_CACHE/bigdata/src/test/com/bigdata/quorum/TestHA3QuorumSemantics.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/quorum/zk/TestZkHA3QuorumSemantics.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java 2013-03-11 15:44:56 UTC (rev 6979) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java 2013-03-11 16:02:46 UTC (rev 6980) @@ -896,21 +896,7 @@ } } - /* - * Helper methods. - */ - - /** - * Search for the vote for the service. - * - * @param serviceId - * The service identifier. - * - * @return The lastCommitTime for which the service has cast its vote -or- - * <code>null</code> if the service is not participating in a - * consensus. - */ - private Long getCastVoteIfConsensus(final UUID serviceId) { + public Long getCastVoteIfConsensus(final UUID serviceId) { lock.lock(); try { final Iterator<Map.Entry<Long, LinkedHashSet<UUID>>> itr = votes @@ -932,6 +918,10 @@ } } + /* + * Helper methods. + */ + // /** // * Search for the vote for the service. // * Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java 2013-03-11 15:44:56 UTC (rev 6979) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java 2013-03-11 16:02:46 UTC (rev 6980) @@ -224,6 +224,18 @@ Long getCastVote(final UUID serviceId); /** + * Search for the vote for the service. + * + * @param serviceId + * The service identifier. + * + * @return The lastCommitTime for which the service has cast its vote -or- + * <code>null</code> if the service is not participating in a + * consensus. + */ + Long getCastVoteIfConsensus(final UUID serviceId); + + /** * Return the identifiers for the member services joined with this quorum. * If the quorum was met at the moment the request was processed, then the * first element of the array was the quorum leader as of that moment and Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/quorum/TestHA3QuorumSemantics.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/quorum/TestHA3QuorumSemantics.java 2013-03-11 15:44:56 UTC (rev 6979) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/quorum/TestHA3QuorumSemantics.java 2013-03-11 16:02:46 UTC (rev 6980) @@ -27,6 +27,7 @@ package com.bigdata.quorum; +import java.util.Arrays; import java.util.Map; import java.util.UUID; @@ -1234,4 +1235,157 @@ } + + /** + * Variations on standard pipelineReorganization to test conditions that + * occasionally fail in TestHA3 + * + * @throws InterruptedException + */ + public void test_pipelineReorganization2() throws InterruptedException { + + final Quorum<?, ?> quorum0 = quorums[0]; + final MockQuorumMember<?> client0 = clients[0]; + final QuorumActor<?, ?> actor0 = actors[0]; + final UUID serviceId0 = client0.getServiceId(); + + final Quorum<?, ?> quorum1 = quorums[1]; + final MockQuorumMember<?> client1 = clients[1]; + final QuorumActor<?, ?> actor1 = actors[1]; + final UUID serviceId1 = client1.getServiceId(); + + final Quorum<?, ?> quorum2 = quorums[2]; + final MockQuorumMember<?> client2 = clients[2]; + final QuorumActor<?, ?> actor2 = actors[2]; + final UUID serviceId2 = client2.getServiceId(); + + final long lastCommitTime = 0L; + + // declare the services as a quorum members. + actor0.memberAdd(); + actor1.memberAdd(); + actor2.memberAdd(); + + /* + * Have the services join the pipeline a different order from the order + * in which they will cast their votes. + */ + actor2.pipelineAdd(); + actor1.pipelineAdd(); + actor0.pipelineAdd(); + + assertCondition(new Runnable() { + public void run() { + /* + * The service which we will cause to vote first (and hence will + * become the leader) is NOT at the head of the pipeline. + */ + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum0.getPipeline()); + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum1.getPipeline()); + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum2.getPipeline()); + } + }); + + /* + * Have two services cast a vote for a lastCommitTime. This will cause + * the quorum to meet. + * + * But actor2 should be moved to end of pipeline + */ + final long token1; + { + + actor1.castVote(lastCommitTime); + actor0.castVote(lastCommitTime); + + // validate the token was assigned. + log.warn("Awaiting quorums to report met."); + token1 = quorum0.awaitQuorum(); + log.warn("Quorum0 reports met."); + assertEquals(token1, quorum1.awaitQuorum()); + log.warn("Quorum1 reports met."); + assertEquals(token1, quorum2.awaitQuorum()); + log.warn("Quorum2 reports met."); + assertEquals(token1, quorum1.lastValidToken()); + assertEquals(token1, quorum2.lastValidToken()); + assertEquals(Quorum.NO_QUORUM + 1, quorum0.lastValidToken()); + assertEquals(Quorum.NO_QUORUM + 1, quorum0.token()); + assertTrue(quorum0.isQuorumMet()); + + assertCondition(new Runnable() { + public void run() { + + // services have voted for a single lastCommitTime. + assertEquals(1, quorum0.getVotes().size()); + assertEquals(1, quorum1.getVotes().size()); + assertEquals(1, quorum2.getVotes().size()); + + // verify the vote order. + log.warn("expected: " + + Arrays.toString(new UUID[] { serviceId1, + serviceId0 })); + log.warn("actual : " + + Arrays.toString(quorum0.getVotes().get( + lastCommitTime))); + + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum0 + .getVotes().get(lastCommitTime)); + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum1 + .getVotes().get(lastCommitTime)); + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum2 + .getVotes().get(lastCommitTime)); + + // verify the consensus was updated. + assertEquals(lastCommitTime, client0.lastConsensusValue); + assertEquals(lastCommitTime, client1.lastConsensusValue); + assertEquals(lastCommitTime, client2.lastConsensusValue); + + /* + * Service join in the same order in which they cast their + * votes. + */ + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum0.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum1.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum2.getJoined()); + + // The leader is now at the front of the pipeline, with + // service2 moved to end. + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum0.getPipeline()); + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum1.getPipeline()); + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum2.getPipeline()); + + } + }); + + // check votes cast + assertTrue(quorum1.getCastVote(serviceId1) == quorum1.getCastVoteIfConsensus(serviceId0)); + assertTrue(quorum1.getCastVote(serviceId1) != quorum1.getCastVote(serviceId2)); + actor2.castVote(lastCommitTime); + assertTrue(quorum1.getCastVoteIfConsensus(serviceId1) == quorum1.getCastVoteIfConsensus(serviceId2)); + + Thread.sleep(100); + + final long token = quorum2.awaitQuorum(); + + assertTrue(quorum2.isQuorumFullyMet(token)); + + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum0.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum1.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum2.getJoined()); + } + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/quorum/zk/TestZkHA3QuorumSemantics.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/quorum/zk/TestZkHA3QuorumSemantics.java 2013-03-11 15:44:56 UTC (rev 6979) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/quorum/zk/TestZkHA3QuorumSemantics.java 2013-03-11 16:02:46 UTC (rev 6980) @@ -1214,4 +1214,157 @@ } + + /** + * Variations on standard pipelineReorganization to test conditions that + * occasionally fail in TestHA3 + * + * @throws InterruptedException + */ + public void test_pipelineReorganization2() throws InterruptedException { + + final Quorum<?, ?> quorum0 = quorums[0]; + final MockQuorumMember<?> client0 = clients[0]; + final QuorumActor<?, ?> actor0 = actors[0]; + final UUID serviceId0 = client0.getServiceId(); + + final Quorum<?, ?> quorum1 = quorums[1]; + final MockQuorumMember<?> client1 = clients[1]; + final QuorumActor<?, ?> actor1 = actors[1]; + final UUID serviceId1 = client1.getServiceId(); + + final Quorum<?, ?> quorum2 = quorums[2]; + final MockQuorumMember<?> client2 = clients[2]; + final QuorumActor<?, ?> actor2 = actors[2]; + final UUID serviceId2 = client2.getServiceId(); + + final long lastCommitTime = 0L; + + // declare the services as a quorum members. + actor0.memberAdd(); + actor1.memberAdd(); + actor2.memberAdd(); + + /* + * Have the services join the pipeline a different order from the order + * in which they will cast their votes. + */ + actor2.pipelineAdd(); + actor1.pipelineAdd(); + actor0.pipelineAdd(); + + assertCondition(new Runnable() { + public void run() { + /* + * The service which we will cause to vote first (and hence will + * become the leader) is NOT at the head of the pipeline. + */ + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum0.getPipeline()); + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum1.getPipeline()); + assertEquals(new UUID[] { serviceId2, serviceId1, serviceId0 }, + quorum2.getPipeline()); + } + }); + + /* + * Have two services cast a vote for a lastCommitTime. This will cause + * the quorum to meet. + * + * But actor2 should be moved to end of pipeline + */ + final long token1; + { + + actor1.castVote(lastCommitTime); + actor0.castVote(lastCommitTime); + + // validate the token was assigned. + log.warn("Awaiting quorums to report met."); + token1 = quorum0.awaitQuorum(); + log.warn("Quorum0 reports met."); + assertEquals(token1, quorum1.awaitQuorum()); + log.warn("Quorum1 reports met."); + assertEquals(token1, quorum2.awaitQuorum()); + log.warn("Quorum2 reports met."); + assertEquals(token1, quorum1.lastValidToken()); + assertEquals(token1, quorum2.lastValidToken()); + assertEquals(Quorum.NO_QUORUM + 1, quorum0.lastValidToken()); + assertEquals(Quorum.NO_QUORUM + 1, quorum0.token()); + assertTrue(quorum0.isQuorumMet()); + + assertCondition(new Runnable() { + public void run() { + + // services have voted for a single lastCommitTime. + assertEquals(1, quorum0.getVotes().size()); + assertEquals(1, quorum1.getVotes().size()); + assertEquals(1, quorum2.getVotes().size()); + + // verify the vote order. + log.warn("expected: " + + Arrays.toString(new UUID[] { serviceId1, + serviceId0 })); + log.warn("actual : " + + Arrays.toString(quorum0.getVotes().get( + lastCommitTime))); + + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum0 + .getVotes().get(lastCommitTime)); + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum1 + .getVotes().get(lastCommitTime)); + assertEquals(new UUID[] { serviceId1, serviceId0 }, quorum2 + .getVotes().get(lastCommitTime)); + + // verify the consensus was updated. + assertEquals(lastCommitTime, client0.lastConsensusValue); + assertEquals(lastCommitTime, client1.lastConsensusValue); + assertEquals(lastCommitTime, client2.lastConsensusValue); + + /* + * Service join in the same order in which they cast their + * votes. + */ + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum0.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum1.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0 }, + quorum2.getJoined()); + + // The leader is now at the front of the pipeline, with + // service2 moved to end. + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum0.getPipeline()); + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum1.getPipeline()); + assertEquals(new UUID[] { serviceId1, serviceId0, + serviceId2 }, quorum2.getPipeline()); + + } + }); + + // check votes cast + assertTrue(quorum1.getCastVote(serviceId1) == quorum1.getCastVoteIfConsensus(serviceId0)); + assertTrue(quorum1.getCastVote(serviceId1) != quorum1.getCastVote(serviceId2)); + actor2.castVote(lastCommitTime); + assertTrue(quorum1.getCastVoteIfConsensus(serviceId1) == quorum1.getCastVoteIfConsensus(serviceId2)); + + Thread.sleep(100); + + final long token = quorum2.awaitQuorum(); + + assertTrue(quorum2.isQuorumFullyMet(token)); + + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum0.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum1.getJoined()); + assertEquals(new UUID[] { serviceId1, serviceId0, serviceId2 }, + quorum2.getJoined()); + } + + } + } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |