Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Process RAFT messages directly inside the role handlers, thereby removing the need to change role, and having to send the message back for later processing by that role. The tests have been restructured to reflect the requirement for some messages to be handled in mostly the same way, regardless of the current role.
- Loading branch information
Showing
20 changed files
with
1,035 additions
and
780 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
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
113 changes: 113 additions & 0 deletions
113
enterprise/core-edge/src/main/java/org/neo4j/coreedge/raft/roles/Appending.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 | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,113 @@ | |||
/* | |||
* Copyright (c) 2002-2016 "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.coreedge.raft.roles; | |||
|
|||
import org.neo4j.coreedge.raft.RaftMessages; | |||
import org.neo4j.coreedge.raft.log.RaftLogEntry; | |||
import org.neo4j.coreedge.raft.log.RaftStorageException; | |||
import org.neo4j.coreedge.raft.outcome.AppendLogEntry; | |||
import org.neo4j.coreedge.raft.outcome.BatchAppendLogEntries; | |||
import org.neo4j.coreedge.raft.outcome.Outcome; | |||
import org.neo4j.coreedge.raft.outcome.ShipCommand; | |||
import org.neo4j.coreedge.raft.outcome.TruncateLogCommand; | |||
import org.neo4j.coreedge.raft.replication.ReplicatedContent; | |||
import org.neo4j.coreedge.raft.state.ReadableRaftState; | |||
|
|||
public class Appending | |||
{ | |||
public static <MEMBER> void handleAppendEntriesRequest( | |||
ReadableRaftState<MEMBER> state, Outcome<MEMBER> outcome, RaftMessages.AppendEntries.Request<MEMBER> request ) | |||
throws RaftStorageException | |||
{ | |||
if ( request.leaderTerm() < state.term() ) | |||
{ | |||
RaftMessages.AppendEntries.Response<MEMBER> appendResponse = new RaftMessages.AppendEntries.Response<>( | |||
state.myself(), state.term(), false, -1, state.entryLog().appendIndex() ); | |||
|
|||
outcome.addOutgoingMessage( new RaftMessages.Directed<>( request.from(), appendResponse ) ); | |||
return; | |||
} | |||
|
|||
outcome.renewElectionTimeout(); | |||
outcome.setNextTerm( request.leaderTerm() ); | |||
outcome.setLeader( request.from() ); | |||
outcome.setLeaderCommit( request.leaderCommit() ); | |||
|
|||
if ( !Follower.logHistoryMatches( state, request.prevLogIndex(), request.prevLogTerm() ) ) | |||
{ | |||
assert request.prevLogIndex() > -1 && request.prevLogTerm() > -1; | |||
RaftMessages.AppendEntries.Response<MEMBER> appendResponse = new RaftMessages.AppendEntries.Response<>( | |||
state.myself(), request.leaderTerm(), false, -1, state.entryLog().appendIndex() ); | |||
|
|||
outcome.addOutgoingMessage( new RaftMessages.Directed<>( request.from(), appendResponse ) ); | |||
return; | |||
} | |||
|
|||
long baseIndex = request.prevLogIndex() + 1; | |||
int offset; | |||
|
|||
/* Find possible truncation point. */ | |||
for ( offset = 0; offset < request.entries().length; offset++ ) | |||
{ | |||
long logTerm = state.entryLog().readEntryTerm( baseIndex + offset ); | |||
|
|||
if( baseIndex + offset > state.entryLog().appendIndex() ) | |||
{ | |||
/* entry doesn't exist */ | |||
break; | |||
} | |||
else if ( logTerm != request.entries()[offset].term() ) | |||
{ | |||
outcome.addLogCommand( new TruncateLogCommand( baseIndex + offset ) ); | |||
break; | |||
} | |||
} | |||
|
|||
if( offset < request.entries().length ) | |||
{ | |||
outcome.addLogCommand( new BatchAppendLogEntries( baseIndex, offset, request.entries() ) ); | |||
} | |||
|
|||
Follower.commitToLogOnUpdate( state, request.prevLogIndex() + request.entries().length, request.leaderCommit | |||
(), outcome ); | |||
|
|||
long endMatchIndex = request.prevLogIndex() + request.entries().length; // this is the index of the last incoming entry | |||
if ( endMatchIndex >= 0 ) | |||
{ | |||
RaftMessages.AppendEntries.Response<MEMBER> appendResponse = new RaftMessages.AppendEntries.Response<>( state.myself(), request.leaderTerm(), true, endMatchIndex, endMatchIndex ); | |||
outcome.addOutgoingMessage( new RaftMessages.Directed<>( request.from(), appendResponse ) ); | |||
} | |||
} | |||
|
|||
public static <MEMBER> void appendNewEntry( ReadableRaftState<MEMBER> ctx, Outcome<MEMBER> outcome, ReplicatedContent | |||
content ) throws RaftStorageException | |||
{ | |||
long prevLogIndex = ctx.entryLog().appendIndex(); | |||
long prevLogTerm = prevLogIndex == -1 ? -1 : | |||
prevLogIndex > ctx.lastLogIndexBeforeWeBecameLeader() ? | |||
ctx.term() : | |||
ctx.entryLog().readLogEntry( prevLogIndex ).term(); | |||
|
|||
RaftLogEntry newLogEntry = new RaftLogEntry( ctx.term(), content ); | |||
|
|||
outcome.addShipCommand( new ShipCommand.NewEntry( prevLogIndex, prevLogTerm, newLogEntry ) ); | |||
outcome.addLogCommand( new AppendLogEntry( prevLogIndex + 1, newLogEntry ) ); | |||
} | |||
} |
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
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
48 changes: 48 additions & 0 deletions
48
enterprise/core-edge/src/main/java/org/neo4j/coreedge/raft/roles/Heart.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 | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,48 @@ | |||
/* | |||
* Copyright (c) 2002-2016 "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.coreedge.raft.roles; | |||
|
|||
import org.neo4j.coreedge.raft.RaftMessages; | |||
import org.neo4j.coreedge.raft.log.RaftStorageException; | |||
import org.neo4j.coreedge.raft.outcome.Outcome; | |||
import org.neo4j.coreedge.raft.state.ReadableRaftState; | |||
|
|||
public class Heart | |||
{ | |||
public static <MEMBER> void beat( ReadableRaftState<MEMBER> state, Outcome<MEMBER> outcome, RaftMessages.Heartbeat<MEMBER> request ) throws RaftStorageException | |||
{ | |||
if ( request.leaderTerm() < state.term() ) | |||
{ | |||
return; | |||
} | |||
|
|||
outcome.renewElectionTimeout(); | |||
outcome.setNextTerm( request.leaderTerm() ); | |||
outcome.setLeader( request.from() ); | |||
outcome.setLeaderCommit( request.commitIndex() ); | |||
|
|||
if ( !Follower.logHistoryMatches( state, request.commitIndex(), request.commitIndexTerm() ) ) | |||
{ | |||
return; | |||
} | |||
|
|||
Follower.commitToLogOnUpdate( state, request.commitIndex(), request.commitIndex(), outcome ); | |||
} | |||
} |
Oops, something went wrong.