You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/10/21 21:43:20 UTC

[GitHub] [beam] lukecwik commented on a change in pull request #15747: [BEAM-13015] Create a multiplexer that sends Elements based upon instruction id allowing for an inbound observer responsible for the entire instruction id.

lukecwik commented on a change in pull request #15747:
URL: https://github.com/apache/beam/pull/15747#discussion_r734066644



##########
File path: sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer2.java
##########
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.fn.data;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.sdk.fn.CancellableQueue;
+import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
+import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A gRPC multiplexer for a specific {@link Endpoints.ApiServiceDescriptor}.
+ *
+ * <p>Multiplexes data for inbound consumers based upon their {@code instructionId}.
+ *
+ * <p>Multiplexing inbound and outbound streams is as thread safe as the consumers of those streams.
+ * For inbound streams, this is as thread safe as the inbound observers. For outbound streams, this
+ * is as thread safe as the underlying stream observer.
+ *
+ * <p>TODO: Add support for multiplexing over multiple outbound observers by stickying the output
+ * location with a specific outbound observer.
+ */
+public class BeamFnDataGrpcMultiplexer2 implements AutoCloseable {
+  private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataGrpcMultiplexer2.class);
+  private final Endpoints.@Nullable ApiServiceDescriptor apiServiceDescriptor;
+  private final StreamObserver<BeamFnApi.Elements> inboundObserver;
+  private final StreamObserver<BeamFnApi.Elements> outboundObserver;
+  private final ConcurrentMap<
+          /*instructionId=*/ String, CompletableFuture<CloseableFnDataReceiver<BeamFnApi.Elements>>>
+      receivers;
+  private final ConcurrentMap<String, Boolean> erroredInstructionIds;
+  private final List<CancellableQueue<Elements>> unusedQueues;
+
+  public BeamFnDataGrpcMultiplexer2(
+      Endpoints.@Nullable ApiServiceDescriptor apiServiceDescriptor,
+      OutboundObserverFactory outboundObserverFactory,
+      OutboundObserverFactory.BasicFactory<BeamFnApi.Elements, BeamFnApi.Elements>
+          baseOutboundObserverFactory) {
+    this.apiServiceDescriptor = apiServiceDescriptor;
+    this.receivers = new ConcurrentHashMap<>();
+    this.inboundObserver = new InboundObserver();
+    this.outboundObserver =
+        outboundObserverFactory.outboundObserverFor(baseOutboundObserverFactory, inboundObserver);
+    this.erroredInstructionIds = new ConcurrentHashMap<>();
+    this.unusedQueues = new ArrayList<>(100);
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .omitNullValues()
+        .add("apiServiceDescriptor", apiServiceDescriptor)
+        .add("consumers", receivers)
+        .toString();
+  }
+
+  public StreamObserver<BeamFnApi.Elements> getInboundObserver() {
+    return inboundObserver;
+  }
+
+  public StreamObserver<BeamFnApi.Elements> getOutboundObserver() {
+    return outboundObserver;
+  }
+
+  private CompletableFuture<CloseableFnDataReceiver<BeamFnApi.Elements>> receiverFuture(
+      String instructionId) {
+    return receivers.computeIfAbsent(instructionId, (unused) -> new CompletableFuture<>());
+  }
+
+  /**
+   * Registers a consumer for the specified intruction id.
+   *
+   * <p>The {@link BeamFnDataGrpcMultiplexer2} partitions {@link BeamFnApi.Elements} with multiple
+   * instruction ids ensuring that the receiver will only see {@link BeamFnApi.Elements} with a
+   * single instruction id.
+   *
+   * <p>The caller must {@link #unregisterConsumer unregister the consumer} when they no longer wish
+   * to receive messages.
+   */
+  public void registerConsumer(
+      String instructionId, CloseableFnDataReceiver<BeamFnApi.Elements> receiver) {
+    receiverFuture(instructionId).complete(receiver);
+  }
+
+  /** Unregisters a consumer. */
+  public void unregisterConsumer(String instructionId) {
+    receivers.remove(instructionId);
+  }
+
+  @VisibleForTesting
+  boolean hasConsumer(String instructionId) {
+    return receivers.containsKey(instructionId);
+  }
+
+  @Override
+  public void close() throws Exception {
+    Exception exception = null;
+    for (CompletableFuture<CloseableFnDataReceiver<BeamFnApi.Elements>> receiver :
+        ImmutableList.copyOf(receivers.values())) {
+      // Cancel any observer waiting for the client to complete. If the receiver has already been
+      // completed or cancelled, this call will be ignored.
+      receiver.cancel(true);
+      if (!receiver.isCompletedExceptionally()) {
+        try {
+          receiver.get().close();
+        } catch (Exception e) {
+          if (exception == null) {
+            exception = e;
+          } else {
+            exception.addSuppressed(e);
+          }
+        }
+      }
+    }
+    // Cancel any outbound calls and complete any inbound calls, as this multiplexer is hanging up
+    outboundObserver.onError(
+        Status.CANCELLED.withDescription("Multiplexer hanging up").asException());
+    inboundObserver.onCompleted();
+    if (exception != null) {
+      throw exception;
+    }
+  }
+
+  /**
+   * A multiplexing {@link StreamObserver} that selects the inbound {@link Consumer} to pass the
+   * elements to.
+   *
+   * <p>The inbound observer blocks until the {@link Consumer} is bound allowing for the sending
+   * harness to initiate transmitting data without needing for the receiving harness to signal that
+   * it is ready to consume that data.
+   */
+  private final class InboundObserver implements StreamObserver<BeamFnApi.Elements> {
+    @Override
+    public void onNext(BeamFnApi.Elements value) {
+      // Have a fast path to handle the common case and provide a short circuit to exit if we detect
+      // multiple instruction ids.
+      SINGLE:
+      {
+        String instructionId = null;
+        for (BeamFnApi.Elements.Data data : value.getDataList()) {
+          if (instructionId == null) {
+            instructionId = data.getInstructionId();
+          } else if (!instructionId.equals(data.getInstructionId())) {
+            break SINGLE;
+          }
+        }
+        for (BeamFnApi.Elements.Timers timers : value.getTimersList()) {
+          if (instructionId == null) {
+            instructionId = timers.getInstructionId();
+          } else if (!instructionId.equals(timers.getInstructionId())) {
+            break SINGLE;
+          }
+        }

Review comment:
       It is not as performant.
   
   I initially used a HashSet variant and that cost us 0.3% of CPU time which is a significant amount when the whole change will improve performance by 3-5%
   
   This thread that reads from gRPC is quite sensitive to code changes since each message must pass through it.




-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org