6d2f429de278d2f3ad47d671c582847fc81e8320
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / PreLeader.java
1 /*
2  * Copyright (c) 2016 Brocade Communications Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8 package org.opendaylight.controller.cluster.raft.behaviors;
9
10 import akka.actor.ActorRef;
11 import org.opendaylight.controller.cluster.raft.RaftActorContext;
12 import org.opendaylight.controller.cluster.raft.RaftState;
13 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
14 import org.opendaylight.controller.cluster.raft.persisted.NoopPayload;
15
16 /**
17  * The behavior of a RaftActor when it is in the PreLeader state. This state performs all the duties of
18  * Leader with the added behavior of attempting to commit all uncommitted entries from the previous leader's
19  * term. Raft does not allow a leader to commit entries from a previous term by simply counting replicas -
20  * only entries from the leader's current term can be committed (§5.4.2). Rather then waiting for a client
21  * interaction to commit a new entry, the PreLeader state immediately appends a no-op entry (NoopPayload) to
22  * the log with the leader's current term. Once the no-op entry is committed, all prior entries are committed
23  * indirectly. Once all entries are committed, ie commitIndex matches the last log index, it switches to the
24  * normal Leader state.
25  *
26  * <p>
27  * The use of a no-op entry in this manner is outlined in the last paragraph in §8 of the
28  * <a href="https://raft.github.io/raft.pdf">extended raft version</a>.
29  *
30  * @author Thomas Pantelis
31  */
32 public class PreLeader extends AbstractLeader {
33
34     public PreLeader(RaftActorContext context) {
35         super(context, RaftState.PreLeader);
36
37         context.getActor().tell(NoopPayload.INSTANCE, context.getActor());
38     }
39
40     @Override
41     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
42         if (message instanceof ApplyState) {
43             log.debug("{}: Received {} - lastApplied: {}, lastIndex: {}", logName(), message, context.getLastApplied(),
44                     context.getReplicatedLog().lastIndex());
45             if (context.getLastApplied() >= context.getReplicatedLog().lastIndex()) {
46                 // We've applied all entries - we can switch to Leader.
47                 return internalSwitchBehavior(new Leader(context, this));
48             } else {
49                 return this;
50             }
51         } else {
52             return super.handleMessage(sender, message);
53         }
54     }
55 }