Skip to content

Commit b210965

Browse files
authored
RATIS-2184. Improve TestRaftWithGrpc test stability (#1177)
1 parent 0514e09 commit b210965

File tree

4 files changed

+38
-20
lines changed

4 files changed

+38
-20
lines changed

ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -302,8 +302,14 @@ private long errorWaitTimeMs() {
302302

303303
@Override
304304
public CompletableFuture<LifeCycle.State> stopAsync() {
305-
grpcServerMetrics.unregister();
306-
return super.stopAsync();
305+
try (AutoCloseableLock ignored = lock.writeLock(caller, LOG::trace)) {
306+
if (appendLogRequestObserver != null) {
307+
appendLogRequestObserver.stop();
308+
appendLogRequestObserver = null;
309+
}
310+
grpcServerMetrics.unregister();
311+
return super.stopAsync();
312+
}
307313
}
308314

309315
@Override
@@ -382,6 +388,9 @@ private void appendLog(boolean heartbeat) throws IOException {
382388
final ReferenceCountedObject<AppendEntriesRequestProto> pending;
383389
final AppendEntriesRequest request;
384390
try (AutoCloseableLock writeLock = lock.writeLock(caller, LOG::trace)) {
391+
if (!isRunning()) {
392+
return;
393+
}
385394
// Prepare and send the append request.
386395
// Note changes on follower's nextIndex and ops on pendingRequests should always be done under the write-lock
387396
pending = nextAppendEntriesRequest(callId.getAndIncrement(), heartbeat);

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

Lines changed: 12 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -261,12 +261,19 @@ protected ReferenceCountedObject<AppendEntriesRequestProto> nextAppendEntriesReq
261261
final long halfMs = heartbeatWaitTimeMs/2;
262262
final Map<Long, ReferenceCountedObject<EntryWithData>> offered = new HashMap<>();
263263
for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) {
264-
final ReferenceCountedObject<EntryWithData> entryWithData = getRaftLog().retainEntryWithData(next);
265-
if (!buffer.offer(entryWithData.get())) {
266-
entryWithData.release();
267-
break;
264+
ReferenceCountedObject<EntryWithData> entryWithData = null;
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+
if (entryWithData != null) {
274+
entryWithData.release();
275+
}
268276
}
269-
offered.put(next, entryWithData);
270277
}
271278
if (buffer.isEmpty()) {
272279
return null;

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

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -282,11 +282,15 @@ public ReferenceCountedObject<LogEntryProto> load(LogRecord key) throws IOExcept
282282
final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen);
283283
readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), raftLogMetrics, entryRef -> {
284284
final LogEntryProto entry = entryRef.retain();
285-
final TermIndex ti = TermIndex.valueOf(entry);
286-
putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE);
287-
if (ti.equals(key.getTermIndex())) {
288-
toReturn.set(entryRef);
289-
} else {
285+
try {
286+
final TermIndex ti = TermIndex.valueOf(entry);
287+
putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE);
288+
if (ti.equals(key.getTermIndex())) {
289+
toReturn.set(entryRef);
290+
} else {
291+
entryRef.release();
292+
}
293+
} catch (Exception e) {
290294
entryRef.release();
291295
}
292296
});

ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -859,12 +859,11 @@ public void shutdown() {
859859
getServers().forEach(proxy -> executor.submit(() -> JavaUtils.runAsUnchecked(proxy::close)));
860860
final int maxRetries = 30;
861861
final TimeDuration retrySleep = TimeDuration.ONE_SECOND;
862-
try {
863-
executor.shutdown();
864-
// just wait for a few seconds
865-
boolean terminated = false;
862+
executor.shutdown();
863+
boolean terminated = false;
866864

867-
for(int i = 0; i < maxRetries && !terminated; ) {
865+
for(int i = 0; i < maxRetries && !terminated; ) {
866+
try {
868867
terminated = executor.awaitTermination(retrySleep.getDuration(), retrySleep.getUnit());
869868
if (!terminated) {
870869
i++;
@@ -874,10 +873,9 @@ public void shutdown() {
874873
LOG.error("Failed to shutdown executor, some servers may be still running:\n{}", printServers());
875874
}
876875
}
877-
}
878-
} catch (InterruptedException e) {
876+
} catch (InterruptedException e) {
879877
LOG.warn("shutdown interrupted", e);
880-
Thread.currentThread().interrupt();
878+
}
881879
}
882880

883881
Optional.ofNullable(timer.get()).ifPresent(Timer::cancel);

0 commit comments

Comments
 (0)