2 * Copyright (c) 2016 Brocade Communications Systems, Inc. and others. All rights reserved.
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
8 package org.opendaylight.controller.cluster.raft.behaviors;
10 import org.apache.pekko.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 import org.slf4j.Logger;
16 import org.slf4j.LoggerFactory;
19 * The behavior of a RaftActor when it is in the PreLeader state. This state performs all the duties of
20 * Leader with the added behavior of attempting to commit all uncommitted entries from the previous leader's
21 * term. Raft does not allow a leader to commit entries from a previous term by simply counting replicas -
22 * only entries from the leader's current term can be committed (§5.4.2). Rather then waiting for a client
23 * interaction to commit a new entry, the PreLeader state immediately appends a no-op entry (NoopPayload) to
24 * the log with the leader's current term. Once the no-op entry is committed, all prior entries are committed
25 * indirectly. Once all entries are committed, ie commitIndex matches the last log index, it switches to the
26 * normal Leader state.
28 * <p>The use of a no-op entry in this manner is outlined in the last paragraph in §8 of the
29 * <a href="https://raft.github.io/raft.pdf">extended raft version</a>.
31 * @author Thomas Pantelis
33 public final class PreLeader extends AbstractLeader {
34 private static final Logger LOG = LoggerFactory.getLogger(PreLeader.class);
36 PreLeader(final RaftActorContext context) {
37 super(context, RaftState.PreLeader);
39 context.getActor().tell(NoopPayload.INSTANCE, context.getActor());
43 public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
44 if (message instanceof ApplyState) {
45 final var lastApplied = context.getLastApplied();
46 final var lastIndex = context.getReplicatedLog().lastIndex();
47 LOG.debug("{}: Received {} - lastApplied: {}, lastIndex: {}", logName, message, lastApplied, lastIndex);
48 return lastApplied < lastIndex ? this
49 // We've applied all entries - we can switch to Leader.
50 : internalSwitchBehavior(new Leader(context, this));
52 return super.handleMessage(sender, message);