Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.ratis.server.util.ServerStringUtils;
import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver;
import org.apache.ratis.thirdparty.io.grpc.stub.ClientCallStreamObserver;
import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto;
import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult;
Expand All @@ -57,10 +58,8 @@
import java.util.Optional;
import java.util.Queue;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;

/**
Expand Down Expand Up @@ -341,20 +340,37 @@ private boolean haveTooManyPendingRequests() {
}

static class StreamObservers {
private final CallStreamObserver<AppendEntriesRequestProto> appendLog;
private final CallStreamObserver<AppendEntriesRequestProto> heartbeat;
private final ClientCallStreamObserver<AppendEntriesRequestProto> appendLog;
private final ClientCallStreamObserver<AppendEntriesRequestProto> heartbeat;
private final TimeDuration waitForReady;
private final TimeDuration completeGracePeriod;
private volatile boolean running = true;

private final ScheduledExecutorService closer =
Executors.newSingleThreadScheduledExecutor(r -> {
Thread t = new Thread(r, "grpc-log-appender-stream-closer");
t.setDaemon(true);
return t;
});

// private final AtomicBoolean closing = new AtomicBoolean(false);
private final AtomicBoolean completed = new AtomicBoolean(false);
private final AtomicBoolean cancelled = new AtomicBoolean(false);

StreamObservers(GrpcServerProtocolClient client, AppendLogResponseHandler handler, boolean separateHeartbeat,
TimeDuration waitTimeMin) {
this.appendLog = client.appendEntries(handler, false);
this.heartbeat = separateHeartbeat? client.appendEntries(handler, true): null;
TimeDuration waitTimeMin, TimeDuration completeGracePeriod) {
this.appendLog = (ClientCallStreamObserver<AppendEntriesRequestProto>) client.appendEntries(handler, false);
this.heartbeat = separateHeartbeat?
(ClientCallStreamObserver<AppendEntriesRequestProto>) client.appendEntries(handler, true): null;
this.waitForReady = waitTimeMin.isPositive()? waitTimeMin: TimeDuration.ONE_MILLISECOND;
this.completeGracePeriod = completeGracePeriod.isPositive()? completeGracePeriod : TimeDuration.ONE_SECOND;
}

void onNext(AppendEntriesRequestProto proto)
throws InterruptedIOException {
if (!running) {
throw new InterruptedIOException("StreamObservers is stopping/closing");
}
CallStreamObserver<AppendEntriesRequestProto> stream;
boolean isHeartBeat = heartbeat != null && proto.getEntriesCount() == 0;
if (isHeartBeat) {
Expand All @@ -366,16 +382,73 @@ void onNext(AppendEntriesRequestProto proto)
while (!stream.isReady() && running) {
sleep(waitForReady, isHeartBeat);
}
stream.onNext(proto);
try {
stream.onNext(proto);
} catch (Exception e) {
InterruptedIOException ioe =
new InterruptedIOException("Failed to send request via stream");
ioe.initCause(e);
throw ioe;
}
}

void stop() {
running = false;
}

void onCompleted() {
appendLog.onCompleted();
Optional.ofNullable(heartbeat).ifPresent(StreamObserver::onCompleted);
if (completed.compareAndSet(false, true)) {
completeStreamGracefully(appendLog, "appendLog");
Optional.ofNullable(heartbeat)
.ifPresent(s -> completeStreamGracefully(s, "heartbeat"));
}
final long delayMs = Math.max(1L, completeGracePeriod.toLong(TimeUnit.MILLISECONDS));
closer.schedule(this::cancelIfStillNeeded, delayMs, TimeUnit.MILLISECONDS);
}

void cancelNow(String reason, Throwable cause) {
if (cancelled.compareAndSet(false, true)) {
running = false;
cancelStream(appendLog, "appendLog", reason, cause);
Optional.ofNullable(heartbeat)
.ifPresent(s -> cancelStream(s, "heartbeat", reason, cause));
shutdownCloser();
}
}

private void cancelIfStillNeeded() {
if (cancelled.compareAndSet(false, true)) {
cancelStream(appendLog, "appendLog", "Stream completion timeout", null);
Optional.ofNullable(heartbeat)
.ifPresent(s -> cancelStream(s, "heartbeat", "Stream completion timeout", null));
}
shutdownCloser();
}

private void completeStreamGracefully(
ClientCallStreamObserver<AppendEntriesRequestProto> stream,
String name) {
try {
stream.onCompleted();
} catch (Exception e) {
LOG.warn("Failed to call onCompleted on {}", name, e);
}
}

private void cancelStream(
ClientCallStreamObserver<AppendEntriesRequestProto> stream,
String name,
String reason,
Throwable cause) {
try {
stream.cancel(reason, cause);
} catch (Exception e) {
LOG.warn("Failed to cancel {}", name, e);
}
}

private void shutdownCloser() {
closer.shutdown();
}
}

Expand Down Expand Up @@ -404,7 +477,8 @@ private void appendLog(boolean heartbeat) throws IOException {
increaseNextIndex(pending);
if (appendLogRequestObserver == null) {
appendLogRequestObserver = new StreamObservers(
getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin());
getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin(),
getCompleteGracePeriod());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -686,6 +686,16 @@ static void setWaitTimeMin(RaftProperties properties, TimeDuration minDuration)
setTimeDuration(properties::setTimeDuration, WAIT_TIME_MIN_KEY, minDuration);
}

String COMPLETE_GRACE_PERIOD_KEY = PREFIX + ".complete.grace.period";
TimeDuration COMPLETE_GRACE_PERIOD_DEFAULT = TimeDuration.ONE_SECOND;
static TimeDuration completeGracePeriod(RaftProperties properties) {
return getTimeDuration(properties.getTimeDuration(COMPLETE_GRACE_PERIOD_DEFAULT.getUnit()),
COMPLETE_GRACE_PERIOD_KEY, COMPLETE_GRACE_PERIOD_DEFAULT, getDefaultLog());
}
static void setCompleteGracePeriod(RaftProperties properties, TimeDuration duration) {
setTimeDuration(properties::setTimeDuration, COMPLETE_GRACE_PERIOD_KEY, duration);
}

String RETRY_POLICY_KEY = PREFIX + ".retry.policy";
/**
* The min wait time as 1ms (0 is not allowed) for first 10,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ public abstract class LogAppenderBase implements LogAppender {

private final AtomicBoolean heartbeatTrigger = new AtomicBoolean();
private final TimeDuration waitTimeMin;
private final TimeDuration completeGracePeriod;

protected LogAppenderBase(RaftServer.Division server, LeaderState leaderState, FollowerInfo f) {
this.follower = f;
Expand All @@ -78,6 +79,7 @@ protected LogAppenderBase(RaftServer.Division server, LeaderState leaderState, F
this.eventAwaitForSignal = new AwaitForSignal(name);

this.waitTimeMin = RaftServerConfigKeys.Log.Appender.waitTimeMin(properties);
this.completeGracePeriod = RaftServerConfigKeys.Log.Appender.completeGracePeriod(properties);
}

@Override
Expand Down Expand Up @@ -144,6 +146,10 @@ protected TimeDuration getWaitTimeMin() {
return waitTimeMin;
}

protected TimeDuration getCompleteGracePeriod() {
return completeGracePeriod;
}

protected TimeDuration getRemainingWaitTime() {
return waitTimeMin.add(getFollower().getLastRpcSendTime().elapsedTime().negate());
}
Expand Down
Loading