Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Renew election timeout on IO threads instead of processing thread
This is to avoid timeouts not being refreshed quickly, and elections triggered, when the processing thread is slow, e.g. blocking for snapshot download. LeaderAvailabilityTimers has been extracted to encapsulate timers for elections and heartbeats as well as timeouts and last refresh times.
- Loading branch information
1 parent
71ed837
commit 9ef2eb1
Showing
13 changed files
with
482 additions
and
111 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
100 changes: 100 additions & 0 deletions
100
...ng/src/main/java/org/neo4j/causalclustering/core/consensus/LeaderAvailabilityHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,100 @@ | ||
/* | ||
* Copyright (c) 2002-2017 "Neo Technology," | ||
* Network Engine for Objects in Lund AB [http://neotechnology.com] | ||
* | ||
* This file is part of Neo4j. | ||
* | ||
* Neo4j is free software: you can redistribute it and/or modify | ||
* it under the terms of the GNU Affero General Public License as | ||
* published by the Free Software Foundation, either version 3 of the | ||
* License, or (at your option) any later version. | ||
* | ||
* This program is distributed in the hope that it will be useful, | ||
* but WITHOUT ANY WARRANTY; without even the implied warranty of | ||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||
* GNU Affero General Public License for more details. | ||
* | ||
* You should have received a copy of the GNU Affero General Public License | ||
* along with this program. If not, see <http://www.gnu.org/licenses/>. | ||
*/ | ||
package org.neo4j.causalclustering.core.consensus; | ||
|
||
import java.util.Objects; | ||
import java.util.function.LongSupplier; | ||
|
||
import org.neo4j.causalclustering.identity.ClusterId; | ||
import org.neo4j.causalclustering.messaging.Inbound; | ||
import org.neo4j.logging.Log; | ||
import org.neo4j.logging.LogProvider; | ||
|
||
public class LeaderAvailabilityHandler implements Inbound.MessageHandler<RaftMessages.ClusterIdAwareMessage> | ||
{ | ||
private final Inbound.MessageHandler<RaftMessages.ClusterIdAwareMessage> delegateHandler; | ||
private final LeaderAvailabilityTimers leaderAvailabilityTimers; | ||
private final LongSupplier term; | ||
private final Log log; | ||
private volatile ClusterId boundClusterId; | ||
|
||
public LeaderAvailabilityHandler( Inbound.MessageHandler<RaftMessages.ClusterIdAwareMessage> delegateHandler, LeaderAvailabilityTimers leaderAvailabilityTimers, | ||
LongSupplier term, LogProvider logProvider ) | ||
{ | ||
this.delegateHandler = delegateHandler; | ||
this.leaderAvailabilityTimers = leaderAvailabilityTimers; | ||
this.term = term; | ||
this.log = logProvider.getLog( getClass() ); | ||
} | ||
|
||
public synchronized void start( ClusterId clusterId ) | ||
{ | ||
boundClusterId = clusterId; | ||
} | ||
|
||
public synchronized void stop() | ||
{ | ||
boundClusterId = null; | ||
} | ||
|
||
@Override | ||
public void handle( RaftMessages.ClusterIdAwareMessage message ) | ||
{ | ||
if ( Objects.isNull( boundClusterId ) ) | ||
{ | ||
log.debug( "This pre handler has been stopped, dropping the message: %s", message.message() ); | ||
} | ||
else if ( !Objects.equals( message.clusterId(), boundClusterId ) ) | ||
{ | ||
log.info( "Discarding message[%s] owing to mismatched clusterId. Expected: %s, Encountered: %s", | ||
message.message(), boundClusterId, message.clusterId() ); | ||
} | ||
else | ||
{ | ||
handleTimeouts( message ); | ||
|
||
delegateHandler.handle( message ); | ||
} | ||
} | ||
|
||
private void handleTimeouts( RaftMessages.ClusterIdAwareMessage message ) | ||
{ | ||
if ( shouldRenewElectionTimeout( message.message() ) ) | ||
{ | ||
leaderAvailabilityTimers.renewElection(); | ||
} | ||
} | ||
|
||
// TODO replace with visitor pattern | ||
private boolean shouldRenewElectionTimeout( RaftMessages.RaftMessage message ) | ||
{ | ||
switch ( message.type() ) | ||
{ | ||
case HEARTBEAT: | ||
RaftMessages.Heartbeat heartbeat = (RaftMessages.Heartbeat) message; | ||
return heartbeat.leaderTerm() >= term.getAsLong(); | ||
case APPEND_ENTRIES_REQUEST: | ||
RaftMessages.AppendEntries.Request request = (RaftMessages.AppendEntries.Request) message; | ||
return request.leaderTerm() >= term.getAsLong(); | ||
default: | ||
return false; | ||
} | ||
} | ||
} |
127 changes: 127 additions & 0 deletions
127
...ing/src/main/java/org/neo4j/causalclustering/core/consensus/LeaderAvailabilityTimers.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,127 @@ | ||
/* | ||
* Copyright (c) 2002-2017 "Neo Technology," | ||
* Network Engine for Objects in Lund AB [http://neotechnology.com] | ||
* | ||
* This file is part of Neo4j. | ||
* | ||
* Neo4j is free software: you can redistribute it and/or modify | ||
* it under the terms of the GNU Affero General Public License as | ||
* published by the Free Software Foundation, either version 3 of the | ||
* License, or (at your option) any later version. | ||
* | ||
* This program is distributed in the hope that it will be useful, | ||
* but WITHOUT ANY WARRANTY; without even the implied warranty of | ||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the | ||
* GNU Affero General Public License for more details. | ||
* | ||
* You should have received a copy of the GNU Affero General Public License | ||
* along with this program. If not, see <http://www.gnu.org/licenses/>. | ||
*/ | ||
package org.neo4j.causalclustering.core.consensus; | ||
|
||
import java.time.Clock; | ||
import java.time.Duration; | ||
|
||
import org.neo4j.causalclustering.core.consensus.schedule.RenewableTimeoutService; | ||
import org.neo4j.function.ThrowingAction; | ||
import org.neo4j.logging.Log; | ||
import org.neo4j.logging.LogProvider; | ||
|
||
class LeaderAvailabilityTimers | ||
{ | ||
private final long electionTimeout; | ||
private final long heartbeatInterval; | ||
private final Clock clock; | ||
private final RenewableTimeoutService renewableTimeoutService; | ||
private final Log log; | ||
|
||
private volatile long lastElectionRenewalMillis; | ||
|
||
private RenewableTimeoutService.RenewableTimeout heartbeatTimer; | ||
private RenewableTimeoutService.RenewableTimeout electionTimer; | ||
|
||
LeaderAvailabilityTimers( Duration electionTimeout, Duration heartbeatInterval, Clock clock, RenewableTimeoutService renewableTimeoutService, | ||
LogProvider logProvider ) | ||
{ | ||
this.electionTimeout = electionTimeout.toMillis(); | ||
this.heartbeatInterval = heartbeatInterval.toMillis(); | ||
this.clock = clock; | ||
this.renewableTimeoutService = renewableTimeoutService; | ||
this.log = logProvider.getLog( getClass() ); | ||
|
||
if ( this.electionTimeout < this.heartbeatInterval ) | ||
{ | ||
throw new IllegalArgumentException( String.format( | ||
"Election timeout %s should not be shorter than heartbeat interval %s", this.electionTimeout, this.heartbeatInterval | ||
) ); | ||
} | ||
} | ||
|
||
synchronized void start( ThrowingAction<Exception> electionAction, ThrowingAction<Exception> heartbeatAction ) | ||
{ | ||
this.electionTimer = renewableTimeoutService.create( RaftMachine.Timeouts.ELECTION, getElectionTimeout(), randomTimeoutRange(), | ||
renewing( electionAction ) ); | ||
this.heartbeatTimer = renewableTimeoutService.create( RaftMachine.Timeouts.HEARTBEAT, getHeartbeatInterval(), 0, | ||
renewing( heartbeatAction ) ); | ||
lastElectionRenewalMillis = clock.millis(); | ||
} | ||
|
||
synchronized void stop() | ||
{ | ||
if ( electionTimer != null ) | ||
{ | ||
electionTimer.cancel(); | ||
} | ||
if ( heartbeatTimer != null ) | ||
{ | ||
heartbeatTimer.cancel(); | ||
} | ||
|
||
} | ||
|
||
synchronized void renewElection() | ||
{ | ||
lastElectionRenewalMillis = clock.millis(); | ||
if ( electionTimer != null ) | ||
{ | ||
electionTimer.renew(); | ||
} | ||
} | ||
|
||
synchronized boolean isElectionTimedOut() | ||
{ | ||
return clock.millis() - lastElectionRenewalMillis >= electionTimeout; | ||
} | ||
|
||
// Getters for immutable values | ||
long getElectionTimeout() | ||
{ | ||
return electionTimeout; | ||
} | ||
|
||
long getHeartbeatInterval() | ||
{ | ||
return heartbeatInterval; | ||
} | ||
|
||
private long randomTimeoutRange() | ||
{ | ||
return getElectionTimeout(); | ||
} | ||
|
||
private RenewableTimeoutService.TimeoutHandler renewing( ThrowingAction<Exception> action ) | ||
{ | ||
return timeout -> | ||
{ | ||
try | ||
{ | ||
action.apply(); | ||
} | ||
catch ( Exception e ) | ||
{ | ||
log.error( "Failed to process timeout.", e ); | ||
} | ||
timeout.renew(); | ||
}; | ||
} | ||
} |
Oops, something went wrong.