m-trieu commented on code in PR #32774: URL: https://github.com/apache/beam/pull/32774#discussion_r1801759099
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java: ########## @@ -160,80 +179,125 @@ protected boolean isShutdown() { private StreamObserver<RequestT> requestObserver() { if (requestObserver == null) { throw new NullPointerException( - "requestObserver cannot be null. Missing a call to startStream() to initialize."); + "requestObserver cannot be null. Missing a call to start() to initialize stream."); } return requestObserver; } /** Send a request to the server. */ protected final void send(RequestT request) { - lastSendTimeMs.set(Instant.now().getMillis()); synchronized (this) { + if (isShutdown()) { + return; + } + if (streamClosed.get()) { throw new IllegalStateException("Send called on a client closed stream."); } - requestObserver().onNext(request); + try { + lastSendTimeMs.set(Instant.now().getMillis()); + requestObserver.onNext(request); + } catch (StreamObserverCancelledException e) { + if (isShutdown()) { + logger.debug("Stream was closed or shutdown during send.", e); + return; + } + + requestObserver.onError(e); + } + } + } + + @Override + public final void start() { + if (!isShutdown.get() && started.compareAndSet(false, true)) { + // start() should only be executed once during the lifetime of the stream for idempotency and + // when shutdown() has not been called. + 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) { + if (isShutdown.get()) { + break; + } 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(); + requestObserver.reset(); onNewStream(); if (clientClosed.get()) { halfClose(); } return; } } 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) { + sleeper.sleep(sleep); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + logger.info( + "Interrupted during stream creation backoff. The stream will not be created."); + 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); + } catch (IllegalStateException e) { Review Comment: we don't ever do anything with the IllegalStateException anyways except LOG so I think this is ok -- 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org