From: <mar...@us...> - 2014-01-08 17:10:15
|
Revision: 7748 http://bigdata.svn.sourceforge.net/bigdata/?rev=7748&view=rev Author: martyncutcher Date: 2014-01-08 17:10:07 +0000 (Wed, 08 Jan 2014) Log Message: ----------- Modification to support possible future non-thick future for receiveAndReplicate to check for regression on CI. The intention is to comment out this code once we have verified it is a plausible approach. Modified Paths: -------------- branches/MGC_1_3_0/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java Modified: branches/MGC_1_3_0/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java =================================================================== --- branches/MGC_1_3_0/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2014-01-08 15:14:09 UTC (rev 7747) +++ branches/MGC_1_3_0/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2014-01-08 17:10:07 UTC (rev 7748) @@ -1951,27 +1951,65 @@ * continuously. This does rather beg the question of * whether we should only be checking futRec at this stage. */ - while (!(futSnd.isDone() || futRec.isDone())) { + + /* + * Await the Futures, but spend more time waiting on the + * local Future and only check the remote Future every + * second. Timeouts are ignored during this loop - they + * are used to let us wait longer on the local Future + * than on the remote Future. ExecutionExceptions are + * also ignored. We want to continue this loop until + * both Futures are done. Interrupts are not trapped, so + * an interrupt will still exit the loop. + */ + while (!futSnd.isDone() || !futRec.isDone()) { /* * Make sure leader's quorum token remains valid for * ALL writes. */ member.assertLeader(token); try { - futSnd.get(1L, TimeUnit.SECONDS); + futSnd.get(10L, TimeUnit.MILLISECONDS); } catch (TimeoutException ignore) { } catch (ExecutionException ignore) { + /* + * Try the other Future with timeout and cancel + * if not done. + */ + try { + futRec.get(1L, TimeUnit.SECONDS); + } catch(TimeoutException ex) { // Ignore. + } catch(ExecutionException ex) { // Ignore. + } finally { + futRec.cancel(true/* mayInterruptIfRunning */); + } + /* + * Note: Both futures are DONE at this point. + */ } try { futRec.get(10L, TimeUnit.MILLISECONDS); } catch (TimeoutException ignore) { } catch (ExecutionException ignore) { + /* + * Try the other Future with timeout and cancel + * if not done. + */ + try { + futSnd.get(10L, TimeUnit.MILLISECONDS); + } catch(TimeoutException ex) { // Ignore. + } catch(ExecutionException ex) { // Ignore. + } finally { + futSnd.cancel(true/* mayInterruptIfRunning */); + } + /* + * Note: Both futures are DONE at this point. + */ } } - + /* - * Note: Both futures are DONE at this point. However, - * we want to check the remote Future for the downstream + * Note: We want to check the remote Future for the downstream * service first in order to accurately report the * service that was the source of a pipeline replication * problem. @@ -2534,7 +2572,7 @@ * TODO: check the comparative logic with this and robustReplicate * to confirm the equivalence of checking the different futures. */ - while (!(futRec.isDone() || futRep.isDone())) { + while (!futRec.isDone() || !futRep.isDone()) { /* * The token must remain valid, even if this service * is not joined with the met quorum. If fact, @@ -2547,11 +2585,39 @@ futRec.get(1L, TimeUnit.SECONDS); } catch (TimeoutException ignore) { } catch (ExecutionException ignore) { + /* + * Try the other Future with timeout and cancel + * if not done. + */ + try { + futRep.get(1L, TimeUnit.SECONDS); + } catch(TimeoutException ex) { // Ignore. + } catch(ExecutionException ex) { // Ignore. + } finally { + futRep.cancel(true/* mayInterruptIfRunning */); + } + /* + * Note: Both futures are DONE at this point. + */ } try { futRep.get(10L, TimeUnit.MILLISECONDS); } catch (TimeoutException ignore) { } catch (ExecutionException ignore) { + /* + * Try the other Future with timeout and cancel + * if not done. + */ + try { + futRec.get(1L, TimeUnit.SECONDS); + } catch(TimeoutException ex) { // Ignore. + } catch(ExecutionException ex) { // Ignore. + } finally { + futRec.cancel(true/* mayInterruptIfRunning */); + } + /* + * Note: Both futures are DONE at this point. + */ } } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |