lukecwik commented on code in PR #25186:
URL: https://github.com/apache/beam/pull/25186#discussion_r1095077081
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java:
##########
@@ -287,9 +284,24 @@ public void run() {
outboundObserver.onError(
Status.INTERNAL.withDescription(getStackTraceAsString(thrown)).asException());
throw new IllegalStateException(thrown);
- } else {
- outboundObserver.onCompleted();
}
+ if (inboundObserverCompletion.isDone()) {
+ try {
+ inboundObserverCompletion.get();
+ throw new IllegalStateException(
+ "Logging stream terminated unexpectedly with success before it
was closed by the client.");
+ } catch (ExecutionException e) {
+ throw new IllegalStateException(
+ "Logging stream terminated unexpectedly before it was closed by
the client with error: "
+ + e.getCause());
+ } catch (InterruptedException e) {
+ // Should never happen because of the isDone check.
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ }
+
+ outboundObserver.onCompleted();
Review Comment:
We can't call outboundObserver.onCompleted() if we already called
outboundObserver.onError()
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java:
##########
@@ -102,7 +104,7 @@ public BeamFnLoggingClient(
Function<Endpoints.ApiServiceDescriptor, ManagedChannel> channelFactory)
{
this.apiServiceDescriptor = apiServiceDescriptor;
this.inboundObserverCompletion = new CompletableFuture<>();
- this.phaser = new Phaser(1);
Review Comment:
makes sense
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java:
##########
@@ -122,7 +124,9 @@ public BeamFnLoggingClient(
logRecordHandler.setLevel(Level.ALL);
logRecordHandler.setFormatter(DEFAULT_FORMATTER);
logRecordHandler.executeOn(options.as(ExecutorOptions.class).getScheduledExecutorService());
Review Comment:
we have a race here, `executeOn` will start the logging thread before
`outboundObserver` has been initialized.
##########
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java:
##########
@@ -122,7 +124,9 @@ public BeamFnLoggingClient(
logRecordHandler.setLevel(Level.ALL);
logRecordHandler.setFormatter(DEFAULT_FORMATTER);
logRecordHandler.executeOn(options.as(ExecutorOptions.class).getScheduledExecutorService());
- outboundObserver = (CallStreamObserver<BeamFnApi.LogEntry.List>)
stub.logging(inboundObserver);
+ outboundObserver =
+ new DirectStreamObserver<BeamFnApi.LogEntry.List>(
+ phaser, (CallStreamObserver<BeamFnApi.LogEntry.List>)
stub.logging(inboundObserver));
Review Comment:
There was no point in using the `DirectStreamObserver` since we guarantee
that all writing to the stream happens from the single thread in
`LogRecordHandler#run()` which will never be the inbound gRPC thread so we
don't have to worry about waiting for the phaser and re-checking outbound
observer readiness.
--
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]