017794679c1e2deb4a5ece3c25f641cca32799ef
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / messages / AppendEntries.java
1 /*
2  * Copyright (c) 2014 Cisco 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
9 package org.opendaylight.controller.cluster.raft.messages;
10
11 import java.io.Externalizable;
12 import java.io.IOException;
13 import java.io.ObjectInput;
14 import java.io.ObjectOutput;
15 import java.util.ArrayList;
16 import java.util.List;
17 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
18 import org.opendaylight.controller.cluster.raft.persisted.SimpleReplicatedLogEntry;
19 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
20
21 /**
22  * Invoked by leader to replicate log entries (§5.3); also used as
23  * heartbeat (§5.2).
24  */
25 public class AppendEntries extends AbstractRaftRPC {
26     private static final long serialVersionUID = 1L;
27
28     // So that follower can redirect clients
29     private final String leaderId;
30
31     // Index of log entry immediately preceding new ones
32     private final long prevLogIndex;
33
34     // term of prevLogIndex entry
35     private final long prevLogTerm;
36
37     // log entries to store (empty for heart beat - may send more than one for efficiency)
38     private final List<ReplicatedLogEntry> entries;
39
40     // leader's commitIndex
41     private final long leaderCommit;
42
43     // index which has been replicated successfully to all followers, -1 if none
44     private final long replicatedToAllIndex;
45
46     private final short payloadVersion;
47
48     public AppendEntries(long term, String leaderId, long prevLogIndex, long prevLogTerm,
49             List<ReplicatedLogEntry> entries, long leaderCommit, long replicatedToAllIndex, short payloadVersion) {
50         super(term);
51         this.leaderId = leaderId;
52         this.prevLogIndex = prevLogIndex;
53         this.prevLogTerm = prevLogTerm;
54         this.entries = entries;
55         this.leaderCommit = leaderCommit;
56         this.replicatedToAllIndex = replicatedToAllIndex;
57         this.payloadVersion = payloadVersion;
58     }
59
60     public String getLeaderId() {
61         return leaderId;
62     }
63
64     public long getPrevLogIndex() {
65         return prevLogIndex;
66     }
67
68     public long getPrevLogTerm() {
69         return prevLogTerm;
70     }
71
72     public List<ReplicatedLogEntry> getEntries() {
73         return entries;
74     }
75
76     public long getLeaderCommit() {
77         return leaderCommit;
78     }
79
80     public long getReplicatedToAllIndex() {
81         return replicatedToAllIndex;
82     }
83
84     public short getPayloadVersion() {
85         return payloadVersion;
86     }
87
88     @Override
89     public String toString() {
90         StringBuilder builder = new StringBuilder();
91         builder.append("AppendEntries [leaderId=").append(leaderId).append(", prevLogIndex=").append(prevLogIndex)
92                 .append(", prevLogTerm=").append(prevLogTerm).append(", leaderCommit=").append(leaderCommit)
93                 .append(", replicatedToAllIndex=").append(replicatedToAllIndex).append(", payloadVersion=")
94                 .append(payloadVersion).append(", entries=").append(entries).append("]");
95         return builder.toString();
96     }
97
98     private Object writeReplace() {
99         return new Proxy(this);
100     }
101
102     private static class Proxy implements Externalizable {
103         private static final long serialVersionUID = 1L;
104
105         private AppendEntries appendEntries;
106
107         // checkstyle flags the public modifier as redundant which really doesn't make sense since it clearly isn't
108         // redundant. It is explicitly needed for Java serialization to be able to create instances via reflection.
109         @SuppressWarnings("checkstyle:RedundantModifier")
110         public Proxy() {
111         }
112
113         Proxy(AppendEntries appendEntries) {
114             this.appendEntries = appendEntries;
115         }
116
117         @Override
118         public void writeExternal(ObjectOutput out) throws IOException {
119             out.writeLong(appendEntries.getTerm());
120             out.writeObject(appendEntries.leaderId);
121             out.writeLong(appendEntries.prevLogTerm);
122             out.writeLong(appendEntries.prevLogIndex);
123             out.writeLong(appendEntries.leaderCommit);
124             out.writeLong(appendEntries.replicatedToAllIndex);
125             out.writeShort(appendEntries.payloadVersion);
126
127             out.writeInt(appendEntries.entries.size());
128             for (ReplicatedLogEntry e: appendEntries.entries) {
129                 out.writeLong(e.getIndex());
130                 out.writeLong(e.getTerm());
131                 out.writeObject(e.getData());
132             }
133         }
134
135         @Override
136         public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
137             long term = in.readLong();
138             String leaderId = (String) in.readObject();
139             long prevLogTerm = in.readLong();
140             long prevLogIndex = in.readLong();
141             long leaderCommit = in.readLong();
142             long replicatedToAllIndex = in.readLong();
143             short payloadVersion = in.readShort();
144
145             int size = in.readInt();
146             List<ReplicatedLogEntry> entries = new ArrayList<>(size);
147             for (int i = 0; i < size; i++) {
148                 entries.add(new SimpleReplicatedLogEntry(in.readLong(), in.readLong(), (Payload) in.readObject()));
149             }
150
151             appendEntries = new AppendEntries(term, leaderId, prevLogIndex, prevLogTerm, entries, leaderCommit,
152                     replicatedToAllIndex, payloadVersion);
153         }
154
155         private Object readResolve() {
156             return appendEntries;
157         }
158     }
159 }