-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[Java Dataflow Streaming] Add support to AbstractWindmillStream to transition between physical streams within the same logical stream #35523
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
9b76715
b595a97
faa5942
ca93e77
cd7e3ab
d5e4247
20f8a8e
8d3b0dd
4849e66
ed01dd1
48e3ded
1f1139b
296ac57
b9a298f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,13 +17,19 @@ | |
| */ | ||
| package org.apache.beam.runners.dataflow.worker.windmill.client; | ||
|
|
||
| import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; | ||
| import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; | ||
|
|
||
| import com.google.errorprone.annotations.CanIgnoreReturnValue; | ||
| import java.io.PrintWriter; | ||
| import java.time.Duration; | ||
| import java.util.HashSet; | ||
| import java.util.Set; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.RejectedExecutionException; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
|
|
@@ -79,7 +85,7 @@ public abstract class AbstractWindmillStream<RequestT, ResponseT> implements Win | |
| protected final Sleeper sleeper; | ||
|
|
||
| private final Logger logger; | ||
| private final ExecutorService executor; | ||
| private final ScheduledExecutorService executor; | ||
| private final BackOff backoff; | ||
| private final CountDownLatch finishLatch; | ||
| private final Set<AbstractWindmillStream<?, ?>> streamRegistry; | ||
|
|
@@ -89,6 +95,7 @@ public abstract class AbstractWindmillStream<RequestT, ResponseT> implements Win | |
| private final Function<StreamObserver<ResponseT>, TerminatingStreamObserver<RequestT>> | ||
| physicalStreamFactory; | ||
| protected final long physicalStreamDeadlineSeconds; | ||
| private final Duration halfClosePhysicalStreamAfter; | ||
| private final ResettableThrowingStreamObserver<RequestT> requestObserver; | ||
|
|
||
| private final StreamDebugMetrics debugMetrics; | ||
|
|
@@ -106,6 +113,14 @@ public abstract class AbstractWindmillStream<RequestT, ResponseT> implements Win | |
| @GuardedBy("this") | ||
| protected @Nullable PhysicalStreamHandler currentPhysicalStream; | ||
|
|
||
| @GuardedBy("this") | ||
| @Nullable | ||
| Future<?> halfCloseFuture = null; | ||
|
|
||
| // Physical streams that have been half-closed and are waiting for responses or stream failure. | ||
| @GuardedBy("this") | ||
| protected final Set<PhysicalStreamHandler> closingPhysicalStreams; | ||
|
|
||
| // Generally the same as currentPhysicalStream, set under synchronization of this but can be read | ||
| // without. | ||
| private final AtomicReference<PhysicalStreamHandler> currentPhysicalStreamForDebug = | ||
|
|
@@ -114,6 +129,9 @@ public abstract class AbstractWindmillStream<RequestT, ResponseT> implements Win | |
| @GuardedBy("this") | ||
| private boolean started; | ||
|
|
||
| // If halfClosePhysicalStream is non-zero, substreams created for the logical | ||
| // AbstractWindmillStream | ||
| // will be half-closed and a new physical stream will be created after this duraction. | ||
| protected AbstractWindmillStream( | ||
| Logger logger, | ||
| String debugStreamType, | ||
|
|
@@ -122,13 +140,23 @@ protected AbstractWindmillStream( | |
| StreamObserverFactory streamObserverFactory, | ||
| Set<AbstractWindmillStream<?, ?>> streamRegistry, | ||
| int logEveryNStreamFailures, | ||
| String backendWorkerToken) { | ||
| String backendWorkerToken, | ||
| Duration halfClosePhysicalStreamAfter) { | ||
| checkArgument(!halfClosePhysicalStreamAfter.isNegative()); | ||
| this.backendWorkerToken = backendWorkerToken; | ||
| this.physicalStreamFactory = | ||
| (StreamObserver<ResponseT> observer) -> streamObserverFactory.from(clientFactory, observer); | ||
| this.physicalStreamDeadlineSeconds = streamObserverFactory.getDeadlineSeconds(); | ||
| if (!halfClosePhysicalStreamAfter.isZero() | ||
| && halfClosePhysicalStreamAfter.compareTo(Duration.ofSeconds(physicalStreamDeadlineSeconds)) | ||
| >= 0) { | ||
| logger.debug("Not attempting to half-close cleanly as stream deadline is shorter."); | ||
| halfClosePhysicalStreamAfter = Duration.ZERO; | ||
| } | ||
| this.halfClosePhysicalStreamAfter = halfClosePhysicalStreamAfter; | ||
| this.closingPhysicalStreams = new HashSet<>(); | ||
| this.executor = | ||
| Executors.newSingleThreadExecutor( | ||
| Executors.newSingleThreadScheduledExecutor( | ||
| new ThreadFactoryBuilder() | ||
| .setDaemon(true) | ||
| .setNameFormat(createThreadName(debugStreamType, backendWorkerToken)) | ||
|
|
@@ -182,7 +210,8 @@ protected abstract class PhysicalStreamHandler { | |
|
|
||
| protected abstract PhysicalStreamHandler newResponseHandler(); | ||
|
|
||
| protected abstract void onNewStream() throws WindmillStreamShutdownException; | ||
| protected abstract void onFlushPending(boolean isNewStream) | ||
| throws WindmillStreamShutdownException; | ||
|
|
||
| /** Try to send a request to the server. Returns true if the request was successfully sent. */ | ||
| @CanIgnoreReturnValue | ||
|
|
@@ -214,26 +243,34 @@ public final void start() { | |
| } | ||
|
|
||
| if (shouldStartStream) { | ||
| // Add the stream to the registry after it has been fully constructed. | ||
| streamRegistry.add(this); | ||
| startStream(); | ||
| } | ||
| } | ||
|
|
||
| /** Starts the underlying stream. */ | ||
| private void startStream() { | ||
| // Add the stream to the registry after it has been fully constructed. | ||
| streamRegistry.add(this); | ||
| while (true) { | ||
| @NonNull PhysicalStreamHandler streamHandler = newResponseHandler(); | ||
| try { | ||
| synchronized (this) { | ||
| checkState(currentPhysicalStream == null, "Overwriting existing physical stream"); | ||
| checkState(halfCloseFuture == null, "Unexpected half-close future"); | ||
| debugMetrics.recordStart(); | ||
| streamHandler.streamDebugMetrics.recordStart(); | ||
| currentPhysicalStream = streamHandler; | ||
| currentPhysicalStreamForDebug.set(currentPhysicalStream); | ||
| requestObserver.reset(physicalStreamFactory.apply(new ResponseObserver(streamHandler))); | ||
| onNewStream(); | ||
| onFlushPending(true); | ||
| if (clientClosed) { | ||
| halfClose(); | ||
| } else if (!halfClosePhysicalStreamAfter.isZero()) { | ||
| halfCloseFuture = | ||
| executor.schedule( | ||
| () -> onHalfClosePhysicalStreamTimeout(streamHandler), | ||
| halfClosePhysicalStreamAfter.getSeconds(), | ||
| TimeUnit.SECONDS); | ||
| } | ||
| return; | ||
| } | ||
|
|
@@ -243,8 +280,17 @@ private void startStream() { | |
| break; | ||
| } catch (Exception e) { | ||
| logger.error("Failed to create new stream, retrying: ", e); | ||
| synchronized (this) { | ||
| currentPhysicalStream = null; | ||
| if (halfCloseFuture != null) { | ||
| halfCloseFuture.cancel(false); | ||
| halfCloseFuture = null; | ||
| } | ||
| clearPhysicalStreamForDebug(); | ||
| } | ||
| try { | ||
| long sleep = backoff.nextBackOffMillis(); | ||
| debugMetrics.recordRestartReason("Failed to create new stream, retrying: " + e); | ||
| debugMetrics.recordSleep(sleep); | ||
| sleeper.sleep(sleep); | ||
| } catch (InterruptedException ie) { | ||
|
|
@@ -375,7 +421,12 @@ public final void appendSummaryHtml(PrintWriter writer) { | |
|
|
||
| @Override | ||
| public final synchronized void halfClose() { | ||
| // Synchronization of close and onCompleted necessary for correct retry logic in onNewStream. | ||
| if (clientClosed) { | ||
| logger.warn("Stream was previously closed."); | ||
| return; | ||
| } | ||
| // Synchronization of close and onCompleted necessary for correct retry logic in | ||
| // onPhysicalStreamCompleted. | ||
| debugMetrics.recordHalfClose(); | ||
| clientClosed = true; | ||
| try { | ||
|
|
@@ -423,24 +474,6 @@ public final void shutdown() { | |
|
|
||
| protected synchronized void shutdownInternal() {} | ||
|
|
||
| /** Returns true if the stream was torn down and should not be restarted internally. */ | ||
| private synchronized boolean maybeTearDownStream(PhysicalStreamHandler doneStream) { | ||
| if (clientClosed && !doneStream.hasPendingRequests()) { | ||
| shutdown(); | ||
| } | ||
|
|
||
| if (isShutdown) { | ||
| // Once we have background closing physicalStreams we will need to improve this to wait for | ||
| // all of the work of the logical stream to be complete. | ||
| streamRegistry.remove(AbstractWindmillStream.this); | ||
| finishLatch.countDown(); | ||
| executor.shutdownNow(); | ||
| return true; | ||
| } | ||
|
|
||
| return false; | ||
| } | ||
|
|
||
| private class ResponseObserver implements StreamObserver<ResponseT> { | ||
| private final PhysicalStreamHandler handler; | ||
|
|
||
|
|
@@ -472,17 +505,74 @@ private void clearPhysicalStreamForDebug() { | |
| currentPhysicalStreamForDebug.set(null); | ||
| } | ||
|
|
||
| private void onHalfClosePhysicalStreamTimeout(PhysicalStreamHandler handler) { | ||
| synchronized (this) { | ||
| if (currentPhysicalStream != handler || clientClosed || isShutdown) { | ||
| return; | ||
| } | ||
| try { | ||
| handler.streamDebugMetrics.recordHalfClose(); | ||
| closingPhysicalStreams.add(handler); | ||
| currentPhysicalStream = null; | ||
| halfCloseFuture = null; // This is the currently running future. | ||
| clearPhysicalStreamForDebug(); | ||
| requestObserver.onCompleted(); | ||
| } catch (Exception e) { | ||
| // XXX figure out | ||
| } | ||
| try { | ||
| @NonNull PhysicalStreamHandler streamHandler = newResponseHandler(); | ||
|
||
| streamHandler.streamDebugMetrics.recordStart(); | ||
| currentPhysicalStream = streamHandler; | ||
| currentPhysicalStreamForDebug.set(currentPhysicalStream); | ||
| requestObserver.reset(physicalStreamFactory.apply(new ResponseObserver(streamHandler))); | ||
| onFlushPending(true); | ||
| } catch (Exception e) { | ||
| // XXX figure out | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private void onPhysicalStreamCompletion(Status status, PhysicalStreamHandler handler) { | ||
| synchronized (this) { | ||
| final boolean wasActiveStream = currentPhysicalStream == handler; | ||
| if (currentPhysicalStream == handler) { | ||
| clearPhysicalStreamForDebug(); | ||
| currentPhysicalStream = null; | ||
| if (halfCloseFuture != null) { | ||
| halfCloseFuture.cancel(false); | ||
| halfCloseFuture = null; | ||
|
||
| } | ||
| } else { | ||
| checkState(closingPhysicalStreams.remove(handler)); | ||
| } | ||
| handler.onDone(status); | ||
| if (wasActiveStream | ||
| && clientClosed | ||
| && !handler.hasPendingRequests() | ||
| && closingPhysicalStreams.stream().noneMatch(PhysicalStreamHandler::hasPendingRequests)) { | ||
| shutdown(); | ||
| } | ||
| if (isShutdown && currentPhysicalStream == null && closingPhysicalStreams.isEmpty()) { | ||
| logger.debug("Completing shutdown of stream after shutdown and all streams terminated."); | ||
| streamRegistry.remove(AbstractWindmillStream.this); | ||
| finishLatch.countDown(); | ||
| executor.shutdownNow(); | ||
| return; | ||
| } | ||
| if (currentPhysicalStream != null) { | ||
| try { | ||
| onFlushPending(false); | ||
| } catch (WindmillStreamShutdownException e) { | ||
| // XXX figure out | ||
| } | ||
| return; | ||
| } | ||
| if (!wasActiveStream) { | ||
| return; | ||
| } | ||
| } | ||
| handler.onDone(status); | ||
| if (maybeTearDownStream(handler)) { | ||
| return; | ||
| } | ||
|
|
||
| // Backoff on errors.; | ||
| if (!status.isOk()) { | ||
| try { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.