Skip to content

Commit 003c291

Browse files
committed
RATIS-2228. Refactor the offered map in LogAppenderBase.nextAppendEntriesRequest.
1 parent accb612 commit 003c291

File tree

4 files changed

+126
-77
lines changed

4 files changed

+126
-77
lines changed

ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java

Lines changed: 112 additions & 62 deletions
Original file line numberDiff line numberDiff line change
@@ -42,22 +42,82 @@
4242
import java.util.List;
4343
import java.util.Map;
4444
import java.util.Objects;
45-
import java.util.Optional;
4645
import java.util.concurrent.CompletableFuture;
47-
import java.util.concurrent.TimeUnit;
4846
import java.util.concurrent.atomic.AtomicBoolean;
4947
import java.util.function.LongUnaryOperator;
5048

5149
/**
5250
* An abstract implementation of {@link LogAppender}.
5351
*/
5452
public abstract class LogAppenderBase implements LogAppender {
53+
/** For storing log entries to create an {@link AppendEntriesRequestProto}. */
54+
private class EntryBuffer {
55+
/** A queue for limiting the byte size and element size. */
56+
private final DataQueue<EntryWithData> queue;
57+
/** A map for releasing {@link ReferenceCountedObject}s. */
58+
private final Map<Long, ReferenceCountedObject<EntryWithData>> map = new HashMap<>();
59+
60+
EntryBuffer() {
61+
final RaftProperties properties = server.getRaftServer().getProperties();
62+
final SizeInBytes bufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties);
63+
final int bufferElementLimit = RaftServerConfigKeys.Log.Appender.bufferElementLimit(properties);
64+
this.queue = new DataQueue<>(name, bufferByteLimit, bufferElementLimit, EntryWithData::getSerializedSize);
65+
}
66+
67+
void retain() {
68+
for (ReferenceCountedObject<EntryWithData> ref : map.values()) {
69+
ref.retain();
70+
}
71+
}
72+
73+
void release() {
74+
for (ReferenceCountedObject<EntryWithData> ref : map.values()) {
75+
ref.release();
76+
}
77+
}
78+
79+
int size() {
80+
return map.size();
81+
}
82+
83+
boolean putNew(long index, ReferenceCountedObject<EntryWithData> ref) {
84+
if (!queue.offer(ref.get())) {
85+
ref.release();
86+
return false;
87+
}
88+
final ReferenceCountedObject<EntryWithData> previous = map.put(index, ref);
89+
Preconditions.assertNull(previous, () -> "previous with index " + index);
90+
return true;
91+
}
92+
93+
void releaseAndClear() {
94+
release();
95+
map.clear();
96+
}
97+
98+
List<LogEntryProto> pollList(long heartbeatWaitTimeMs) throws RaftLogIOException {
99+
try {
100+
return queue.pollList(heartbeatWaitTimeMs, EntryWithData::getEntry, null);
101+
} catch (RaftLogIOException e) {
102+
releaseAndClear();
103+
throw e;
104+
} finally {
105+
for (EntryWithData entry : queue) {
106+
// Release remaining entries.
107+
final ReferenceCountedObject<EntryWithData> removed = map.remove(entry.getIndex());
108+
Objects.requireNonNull(removed, "removed == null");
109+
removed.release();
110+
}
111+
queue.clear();
112+
}
113+
}
114+
}
115+
55116
private final String name;
56117
private final RaftServer.Division server;
57118
private final LeaderState leaderState;
58119
private final FollowerInfo follower;
59120

60-
private final DataQueue<EntryWithData> buffer;
61121
private final int snapshotChunkMaxSize;
62122

63123
private final LogAppenderDaemon daemon;
@@ -75,9 +135,6 @@ protected LogAppenderBase(RaftServer.Division server, LeaderState leaderState, F
75135
final RaftProperties properties = server.getRaftServer().getProperties();
76136
this.snapshotChunkMaxSize = RaftServerConfigKeys.Log.Appender.snapshotChunkSizeMax(properties).getSizeInt();
77137

78-
final SizeInBytes bufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties);
79-
final int bufferElementLimit = RaftServerConfigKeys.Log.Appender.bufferElementLimit(properties);
80-
this.buffer = new DataQueue<>(this, bufferByteLimit, bufferElementLimit, EntryWithData::getSerializedSize);
81138
this.daemon = new LogAppenderDaemon(this);
82139
this.eventAwaitForSignal = new AwaitForSignal(name);
83140

@@ -210,13 +267,13 @@ protected LongUnaryOperator getNextIndexForError(long newNextIndex) {
210267
final long n = oldNextIndex <= 0L ? oldNextIndex : Math.min(oldNextIndex - 1, newNextIndex);
211268
if (m > n) {
212269
if (m > newNextIndex) {
213-
LOG.info("Set nextIndex to matchIndex + 1 (= " + m + ")");
270+
LOG.info("{}: Set nextIndex to matchIndex + 1 (= {})", name, m);
214271
}
215272
return m;
216273
} else if (oldNextIndex <= 0L) {
217274
return oldNextIndex; // no change.
218275
} else {
219-
LOG.info("Decrease nextIndex to " + n);
276+
LOG.info("{}: Decrease nextIndex to {}", name, n);
220277
return n;
221278
}
222279
};
@@ -227,18 +284,18 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he
227284
throw new UnsupportedOperationException("Use nextAppendEntriesRequest(" + callId + ", " + heartbeat +") instead.");
228285
}
229286

230-
/**
231-
* Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}.
232-
* The {@link AppendEntriesRequestProto} object may contain zero or more log entries.
233-
* When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat.
234-
*
235-
* @param callId The call id of the returned request.
236-
* @param heartbeat the returned request must be a heartbeat.
237-
*
238-
* @return a retained reference of {@link AppendEntriesRequestProto} object.
239-
* Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}}
240-
* after use.
241-
*/
287+
/**
288+
* Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}.
289+
* The {@link AppendEntriesRequestProto} object may contain zero or more log entries.
290+
* When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat.
291+
*
292+
* @param callId The call id of the returned request.
293+
* @param heartbeat the returned request must be a heartbeat.
294+
*
295+
* @return a retained reference of {@link AppendEntriesRequestProto} object.
296+
* Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}}
297+
* after use.
298+
*/
242299
protected ReferenceCountedObject<AppendEntriesRequestProto> nextAppendEntriesRequest(long callId, boolean heartbeat)
243300
throws RaftLogIOException {
244301
final long heartbeatWaitTimeMs = getHeartbeatWaitTimeMs();
@@ -253,56 +310,24 @@ protected ReferenceCountedObject<AppendEntriesRequestProto> nextAppendEntriesReq
253310
return ref;
254311
}
255312

256-
Preconditions.assertTrue(buffer.isEmpty(), () -> "buffer has " + buffer.getNumElements() + " elements.");
257-
258313
final long snapshotIndex = follower.getSnapshotIndex();
259-
final long leaderNext = getRaftLog().getNextIndex();
260314
final long followerNext = follower.getNextIndex();
261-
final long halfMs = heartbeatWaitTimeMs/2;
262-
final Map<Long, ReferenceCountedObject<EntryWithData>> offered = new HashMap<>();
263-
for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) {
264-
final ReferenceCountedObject<EntryWithData> entryWithData;
265-
try {
266-
entryWithData = getRaftLog().retainEntryWithData(next);
267-
if (!buffer.offer(entryWithData.get())) {
268-
entryWithData.release();
269-
break;
270-
}
271-
offered.put(next, entryWithData);
272-
} catch (Exception e){
273-
for (ReferenceCountedObject<EntryWithData> ref : offered.values()) {
274-
ref.release();
275-
}
276-
offered.clear();
277-
throw e;
278-
}
279-
}
280-
if (buffer.isEmpty()) {
315+
final EntryBuffer entryBuffer = readLogEntries(followerNext, heartbeatWaitTimeMs);
316+
if (entryBuffer == null) {
281317
return null;
282318
}
283319

284-
final List<LogEntryProto> protos;
285-
try {
286-
protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry,
287-
(entry, time, exception) -> LOG.warn("Failed to get {} in {}",
288-
entry, time.toString(TimeUnit.MILLISECONDS, 3), exception));
289-
} catch (RaftLogIOException e) {
290-
for (ReferenceCountedObject<EntryWithData> ref : offered.values()) {
291-
ref.release();
292-
}
293-
offered.clear();
294-
throw e;
295-
} finally {
296-
for (EntryWithData entry : buffer) {
297-
// Release remaining entries.
298-
Optional.ofNullable(offered.remove(entry.getIndex())).ifPresent(ReferenceCountedObject::release);
299-
}
300-
buffer.clear();
301-
}
320+
final List<LogEntryProto> protos = entryBuffer.pollList(heartbeatWaitTimeMs);
321+
Preconditions.assertSame(entryBuffer.size(), protos.size(), "#protos");
302322
assertProtos(protos, followerNext, previous, snapshotIndex);
303323
AppendEntriesRequestProto appendEntriesProto =
304324
leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId);
305-
return ReferenceCountedObject.delegateFrom(offered.values(), appendEntriesProto);
325+
326+
final ReferenceCountedObject<AppendEntriesRequestProto> ref = ReferenceCountedObject.wrap(
327+
appendEntriesProto, entryBuffer::retain, entryBuffer::release);
328+
ref.retain();
329+
entryBuffer.release();
330+
return ref;
306331
}
307332

308333
private void assertProtos(List<LogEntryProto> protos, long nextIndex, TermIndex previous, long snapshotIndex) {
@@ -324,6 +349,31 @@ private void assertProtos(List<LogEntryProto> protos, long nextIndex, TermIndex
324349
}
325350
}
326351

352+
private EntryBuffer readLogEntries(long followerNext, long heartbeatWaitTimeMs) throws RaftLogIOException {
353+
final RaftLog raftLog = getRaftLog();
354+
final long leaderNext = raftLog.getNextIndex();
355+
final long halfMs = heartbeatWaitTimeMs/2;
356+
EntryBuffer entryBuffer = null;
357+
for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) {
358+
final ReferenceCountedObject<EntryWithData> ref;
359+
try {
360+
ref = raftLog.retainEntryWithData(next);
361+
if (entryBuffer == null) {
362+
entryBuffer = new EntryBuffer();
363+
}
364+
if (!entryBuffer.putNew(next, ref)) {
365+
break;
366+
}
367+
} catch (Exception e){
368+
if (entryBuffer != null) {
369+
entryBuffer.releaseAndClear();
370+
}
371+
throw e;
372+
}
373+
}
374+
return entryBuffer;
375+
}
376+
327377
@Override
328378
public InstallSnapshotRequestProto newInstallSnapshotNotificationRequest(TermIndex firstAvailableLogTermIndex) {
329379
Preconditions.assertTrue(firstAvailableLogTermIndex.getIndex() >= 0);

ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java

Lines changed: 9 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@
2626
import org.apache.ratis.server.raftlog.RaftLogIOException;
2727
import org.apache.ratis.server.storage.RaftStorage;
2828
import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting;
29-
import org.apache.ratis.thirdparty.com.google.common.cache.CacheLoader;
3029
import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream;
3130
import org.apache.ratis.util.FileUtils;
3231
import org.apache.ratis.util.JavaUtils;
@@ -266,15 +265,14 @@ private void assertSegment(long expectedStart, int expectedEntryCount, boolean c
266265
*
267266
* In the future we can make the cache loader configurable if necessary.
268267
*/
269-
class LogEntryLoader extends CacheLoader<LogRecord, ReferenceCountedObject<LogEntryProto>> {
268+
class LogEntryLoader {
270269
private final SegmentedRaftLogMetrics raftLogMetrics;
271270

272271
LogEntryLoader(SegmentedRaftLogMetrics raftLogMetrics) {
273272
this.raftLogMetrics = raftLogMetrics;
274273
}
275274

276-
@Override
277-
public ReferenceCountedObject<LogEntryProto> load(LogRecord key) throws IOException {
275+
ReferenceCountedObject<LogEntryProto> load(TermIndex key) throws IOException {
278276
final File file = getFile();
279277
// note the loading should not exceed the endIndex: it is possible that
280278
// the on-disk log file should be truncated but has not been done yet.
@@ -285,17 +283,16 @@ public ReferenceCountedObject<LogEntryProto> load(LogRecord key) throws IOExcept
285283
try {
286284
final TermIndex ti = TermIndex.valueOf(entry);
287285
putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE);
288-
if (ti.equals(key.getTermIndex())) {
286+
if (ti.equals(key)) {
287+
entryRef.retain();
289288
toReturn.set(entryRef);
290-
} else {
291-
entryRef.release();
292289
}
293-
} catch (Exception e) {
290+
} finally {
294291
entryRef.release();
295292
}
296293
});
297294
loadingTimes.incrementAndGet();
298-
return Objects.requireNonNull(toReturn.get());
295+
return Objects.requireNonNull(toReturn.get(), () -> "toReturn == null for " + key);
299296
}
300297
}
301298

@@ -492,8 +489,8 @@ ReferenceCountedObject<LogEntryProto> getEntryFromCache(TermIndex ti) {
492489
/**
493490
* Acquire LogSegment's monitor so that there is no concurrent loading.
494491
*/
495-
synchronized ReferenceCountedObject<LogEntryProto> loadCache(LogRecord record) throws RaftLogIOException {
496-
ReferenceCountedObject<LogEntryProto> entry = entryCache.get(record.getTermIndex());
492+
synchronized ReferenceCountedObject<LogEntryProto> loadCache(TermIndex ti) throws RaftLogIOException {
493+
ReferenceCountedObject<LogEntryProto> entry = entryCache.get(ti);
497494
if (entry != null) {
498495
try {
499496
entry.retain();
@@ -504,7 +501,7 @@ synchronized ReferenceCountedObject<LogEntryProto> loadCache(LogRecord record) t
504501
}
505502
}
506503
try {
507-
return cacheLoader.load(record);
504+
return cacheLoader.load(ti);
508505
} catch (Exception e) {
509506
throw new RaftLogIOException(e);
510507
}

ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -304,7 +304,8 @@ public ReferenceCountedObject<LogEntryProto> retainLog(long index) throws RaftLo
304304
if (record == null) {
305305
return null;
306306
}
307-
final ReferenceCountedObject<LogEntryProto> entry = segment.getEntryFromCache(record.getTermIndex());
307+
final TermIndex ti = record.getTermIndex();
308+
final ReferenceCountedObject<LogEntryProto> entry = segment.getEntryFromCache(ti);
308309
if (entry != null) {
309310
try {
310311
entry.retain();
@@ -319,7 +320,7 @@ public ReferenceCountedObject<LogEntryProto> retainLog(long index) throws RaftLo
319320
// the entry is not in the segment's cache. Load the cache without holding the lock.
320321
getRaftLogMetrics().onRaftLogCacheMiss();
321322
cacheEviction.signal();
322-
return segment.loadCache(record);
323+
return segment.loadCache(ti);
323324
}
324325

325326
@Override

ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -139,14 +139,15 @@ static void checkLogSegment(LogSegment segment, long start, long end,
139139
long offset = SegmentedRaftLogFormat.getHeaderLength();
140140
for (long i = start; i <= end; i++) {
141141
LogSegment.LogRecord record = segment.getLogRecord(i);
142+
Assertions.assertNotNull(record);
142143
final TermIndex ti = record.getTermIndex();
143144
Assertions.assertEquals(i, ti.getIndex());
144145
Assertions.assertEquals(term, ti.getTerm());
145146
Assertions.assertEquals(offset, record.getOffset());
146147

147148
ReferenceCountedObject<LogEntryProto> entry = segment.getEntryFromCache(ti);
148149
if (entry == null) {
149-
entry = segment.loadCache(record);
150+
entry = segment.loadCache(ti);
150151
}
151152
offset += getEntrySize(entry.get(), Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE);
152153
}

0 commit comments

Comments
 (0)