m-trieu commented on code in PR #32774:
URL: https://github.com/apache/beam/pull/32774#discussion_r1830042998
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -153,113 +163,172 @@ private static long debugDuration(long nowMs, long
startMs) {
protected abstract void startThrottleTimer();
/** Reflects that {@link #shutdown()} was explicitly called. */
- protected boolean isShutdown() {
- return isShutdown.get();
- }
-
- private StreamObserver<RequestT> requestObserver() {
- if (requestObserver == null) {
- throw new NullPointerException(
- "requestObserver cannot be null. Missing a call to startStream() to
initialize.");
+ protected boolean hasReceivedShutdownSignal() {
+ synchronized (shutdownLock) {
+ return isShutdown;
}
-
- return requestObserver;
}
/** Send a request to the server. */
protected final void send(RequestT request) {
- lastSendTimeMs.set(Instant.now().getMillis());
synchronized (this) {
- if (streamClosed.get()) {
+ if (hasReceivedShutdownSignal()) {
+ return;
+ }
+
+ if (streamClosed) {
+ // TODO(m-trieu): throw a more specific exception here (i.e
StreamClosedException)
throw new IllegalStateException("Send called on a client closed
stream.");
}
- requestObserver().onNext(request);
+ try {
+ verify(!Thread.holdsLock(shutdownLock), "shutdownLock should not be
held during send.");
+ debugMetrics.recordSend();
+ requestObserver.onNext(request);
+ } catch (StreamObserverCancelledException e) {
+ if (hasReceivedShutdownSignal()) {
+ logger.debug("Stream was shutdown during send.", e);
+ return;
+ }
+
+ requestObserver.onError(e);
+ }
+ }
+ }
+
+ @Override
+ public final void start() {
+ boolean shouldStartStream = false;
+ synchronized (shutdownLock) {
+ if (!isShutdown && !started) {
+ started = true;
+ shouldStartStream = true;
+ }
+ }
+
+ if (shouldStartStream) {
+ startStream();
}
}
/** Starts the underlying stream. */
- protected final void startStream() {
+ private void startStream() {
// Add the stream to the registry after it has been fully constructed.
streamRegistry.add(this);
while (true) {
try {
synchronized (this) {
- startTimeMs.set(Instant.now().getMillis());
- lastResponseTimeMs.set(0);
- streamClosed.set(false);
- // lazily initialize the requestObserver. Gets reset whenever the
stream is reopened.
- requestObserver = requestObserverSupplier.get();
+ if (hasReceivedShutdownSignal()) {
+ break;
+ }
+ debugMetrics.recordStart();
+ streamClosed = false;
+ requestObserver.reset();
onNewStream();
- if (clientClosed.get()) {
+ if (clientClosed) {
halfClose();
}
return;
}
+ } catch (WindmillStreamShutdownException e) {
+ logger.debug("Stream was shutdown waiting to start.", e);
} catch (Exception e) {
- LOG.error("Failed to create new stream, retrying: ", e);
+ logger.error("Failed to create new stream, retrying: ", e);
try {
long sleep = backoff.nextBackOffMillis();
- sleepUntil.set(Instant.now().getMillis() + sleep);
- Thread.sleep(sleep);
- } catch (InterruptedException | IOException i) {
+ debugMetrics.recordSleep(sleep);
+ sleeper.sleep(sleep);
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ logger.info(
+ "Interrupted during {} creation backoff. The stream will not be
created.",
+ getClass());
+ break;
+ } catch (IOException ioe) {
// Keep trying to create the stream.
}
}
}
+
+ // We were never able to start the stream, remove it from the stream
registry. Otherwise, it is
+ // removed when closed.
+ streamRegistry.remove(this);
}
- protected final Executor executor() {
- return executor;
+ /**
+ * Execute the runnable using the {@link #executor} handling the executor
being in a shutdown
+ * state.
+ */
+ protected final void executeSafely(Runnable runnable) {
+ try {
+ executor.execute(runnable);
+ } catch (RejectedExecutionException e) {
+ logger.debug("{}-{} has been shutdown.", getClass(), backendWorkerToken);
+ }
}
- public final synchronized void maybeSendHealthCheck(Instant
lastSendThreshold) {
- if (lastSendTimeMs.get() < lastSendThreshold.getMillis() &&
!clientClosed.get()) {
+ public final void maybeSendHealthCheck(Instant lastSendThreshold) {
+ if (!clientClosed && debugMetrics.getLastSendTimeMs() <
lastSendThreshold.getMillis()) {
try {
sendHealthCheck();
} catch (RuntimeException e) {
- LOG.debug("Received exception sending health check.", e);
+ logger.debug("Received exception sending health check.", e);
}
}
}
protected abstract void sendHealthCheck();
- // Care is taken that synchronization on this is unnecessary for all status
page information.
- // Blocking sends are made beneath this stream object's lock which could
block status page
- // rendering.
+ /**
+ * @implNote Care is taken that synchronization on this is unnecessary for
all status page
+ * information. Blocking sends are made beneath this stream object's
lock which could block
+ * status page rendering.
+ */
public final void appendSummaryHtml(PrintWriter writer) {
appendSpecificHtml(writer);
- if (errorCount.get() > 0) {
- writer.format(
- ", %d errors, last error [ %s ] at [%s]",
- errorCount.get(), lastError.get(), lastErrorTime.get());
- }
- if (clientClosed.get()) {
+ StreamDebugMetrics.Snapshot summaryMetrics =
debugMetrics.getSummaryMetrics();
+ summaryMetrics
+ .restartMetrics()
+ .ifPresent(
+ metrics ->
+ writer.format(
+ ", %d restarts, last restart reason [ %s ] at [%s], %d
errors",
+ metrics.restartCount(),
+ metrics.lastRestartReason(),
+ metrics.lastRestartTime(),
+ metrics.errorCount()));
+
+ if (clientClosed) {
writer.write(", client closed");
}
- long nowMs = Instant.now().getMillis();
- long sleepLeft = sleepUntil.get() - nowMs;
- if (sleepLeft > 0) {
- writer.format(", %dms backoff remaining", sleepLeft);
+
+ if (summaryMetrics.sleepLeft() > 0) {
+ writer.format(", %dms backoff remaining", summaryMetrics.sleepLeft());
}
+
writer.format(
- ", current stream is %dms old, last send %dms, last response %dms,
closed: %s",
- debugDuration(nowMs, startTimeMs.get()),
- debugDuration(nowMs, lastSendTimeMs.get()),
- debugDuration(nowMs, lastResponseTimeMs.get()),
- streamClosed.get());
+ ", current stream is %dms old, last send %dms, last response %dms,
closed: %s, "
+ + "isShutdown: %s, shutdown time: %s",
+ summaryMetrics.streamAge(),
+ summaryMetrics.timeSinceLastSend(),
+ summaryMetrics.timeSinceLastResponse(),
+ streamClosed,
+ hasReceivedShutdownSignal(),
+ summaryMetrics.shutdownTime().orElse(null));
}
- // Don't require synchronization on stream, see the appendSummaryHtml
comment.
+ /**
+ * @implNote Don't require synchronization on stream, see the {@link
+ * #appendSummaryHtml(PrintWriter)} comment.
+ */
protected abstract void appendSpecificHtml(PrintWriter writer);
@Override
public final synchronized void halfClose() {
// Synchronization of close and onCompleted necessary for correct retry
logic in onNewStream.
- clientClosed.set(true);
- requestObserver().onCompleted();
- streamClosed.set(true);
+ clientClosed = true;
+ requestObserver.onCompleted();
Review Comment:
done
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]