m-trieu commented on code in PR #27593:
URL: https://github.com/apache/beam/pull/27593#discussion_r1274437354


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamPool.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.runners.dataflow.worker.windmill;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/**
+ * Pool of homogeneous streams to Windmill.
+ *
+ * <p>The pool holds a fixed total number of streams, and keeps each stream 
open for a specified
+ * time to allow for better load-balancing.
+ */
+@ThreadSafe
+public class WindmillStreamPool<StreamT extends 
WindmillServerStub.WindmillStream> {
+
+  private final Duration streamTimeout;
+  private final Supplier<StreamT> streamSupplier;
+  private final List<WindmillStreamPool.StreamData<StreamT>> streams;
+  private final Map<StreamT, StreamData<StreamT>> holds;
+
+  private WindmillStreamPool(
+      Duration streamTimeout,
+      Supplier<StreamT> streamSupplier,
+      List<WindmillStreamPool.StreamData<StreamT>> streams,
+      ConcurrentHashMap<StreamT, StreamData<StreamT>> holds) {
+    this.streams = streams;
+    this.streamTimeout = streamTimeout;
+    this.streamSupplier = streamSupplier;
+    this.holds = holds;
+  }
+
+  public static <StreamT extends WindmillServerStub.WindmillStream>
+      WindmillStreamPool<StreamT> create(
+          int numStreams, Duration streamTimeout, Supplier<StreamT> 
streamSupplier) {
+    return new WindmillStreamPool<>(
+        streamTimeout, streamSupplier, emptyStreamList(numStreams), new 
ConcurrentHashMap<>());
+  }
+
+  /**
+   * Return a list of the given capacity populated with null. This list is 
used to load balance
+   * {@link WindmillStreamPool#getStream()} requests. It never grows in size 
and must be filled to
+   * capacity since the random used to grab a random stream is bound by the 
size of the list.
+   */
+  @SuppressWarnings("nullness")

Review Comment:
   ack yea, i tried this route.  unfortunately if you annotate the list as 
nullable, it only checks allows that the list itself is null and will throw the 
following error if we are trying to call `add(null)`
   > error: [argument] incompatible argument for parameter arg0 of add.
         streamz.add(null);
                     ^
     found   : null (NullType)
     required: @Initialized @NonNull StreamData<StreamT extends @Initialized 
@NonNull WindmillStream>
   `
   
   i added a StreamLoadBalancer helper class to handle this using `Map<Integer, 
Stream>` with the key space bounded to the `capacity` which is `numStreams`.  
so now we don't have to deal with `null`



-- 
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]

Reply via email to