scwhittle commented on code in PR #32408:
URL: https://github.com/apache/beam/pull/32408#discussion_r1756498116


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.streaming.config;
+
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public interface StreamingGlobalConfigHandle {
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig

Review Comment:
   update comment



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.streaming.config;
+
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public interface StreamingGlobalConfigHandle {
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig
+   */
+  StreamingGlobalConfig getConfig();
+
+  /*
+   * Subscribe to config updates by registering a callback.
+   * Callback should be called the first time with settings, if any, inline 
before the method returns.
+   */
+  void onConfig(@Nonnull Consumer<StreamingGlobalConfig> callback);

Review Comment:
   how about registerConfigObserver which is a little clearer what it does?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.streaming.config;
+
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public class StreamingGlobalConfigHandleImpl implements 
StreamingGlobalConfigHandle {
+
+  private final AtomicReference<StreamingGlobalConfig> streamingEngineConfig =
+      new AtomicReference<>();
+
+  private final CopyOnWriteArrayList<Consumer<StreamingGlobalConfig>> 
config_callbacks =
+      new CopyOnWriteArrayList<>();
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig

Review Comment:
   nit: rm these override comments, the base methods can be consulted



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.streaming.config;
+
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public class StreamingGlobalConfigHandleImpl implements 
StreamingGlobalConfigHandle {
+
+  private final AtomicReference<StreamingGlobalConfig> streamingEngineConfig =
+      new AtomicReference<>();
+
+  private final CopyOnWriteArrayList<Consumer<StreamingGlobalConfig>> 
config_callbacks =
+      new CopyOnWriteArrayList<>();
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig
+   */
+  @Override
+  public StreamingGlobalConfig getConfig() {
+    Preconditions.checkState(
+        streamingEngineConfig.get() != null,
+        "Global config should be set before any processing is done");
+    return streamingEngineConfig.get();
+  }
+
+  /*
+   * Subscribe to config updates by registering a callback.
+   * Callback will be called the first time with settings, if any, inline 
before the method returns.
+   */
+  @Override
+  public void onConfig(@Nonnull Consumer<StreamingGlobalConfig> callback) {
+    StreamingGlobalConfig config;
+    synchronized (this) {
+      config_callbacks.add(callback);
+      config = streamingEngineConfig.get();
+    }
+    if (config != null) {
+      callback.accept(config);

Review Comment:
   this is racy with setConfig and could result in callback accepting v1 after 
v2 is provided below
   
   Could always call accept with new streamingEngineConfig.get() result so it 
is monotonic?
   Or could expand synchronized blocks and just note that the callbacks should 
be quick.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java:
##########
@@ -196,13 +200,12 @@ public void start(
       Work work,
       WindmillStateReader stateReader,
       SideInputStateFetcher sideInputStateFetcher,
-      OperationalLimits operationalLimits,
       Windmill.WorkItemCommitRequest.Builder outputBuilder) {
     this.key = key;
     this.work = work;
     this.computationKey = WindmillComputationKey.create(computationId, 
work.getShardedKey());
     this.sideInputStateFetcher = sideInputStateFetcher;
-    this.operationalLimits = operationalLimits;
+    this.operationalLimits = configManager.getConfig().operationalLimits();

Review Comment:
   // Snapshot the limits for entire bundle processing.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.streaming.config;
+
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public class StreamingGlobalConfigHandleImpl implements 
StreamingGlobalConfigHandle {
+
+  private final AtomicReference<StreamingGlobalConfig> streamingEngineConfig =
+      new AtomicReference<>();
+
+  private final CopyOnWriteArrayList<Consumer<StreamingGlobalConfig>> 
config_callbacks =
+      new CopyOnWriteArrayList<>();
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig
+   */
+  @Override
+  public StreamingGlobalConfig getConfig() {
+    Preconditions.checkState(
+        streamingEngineConfig.get() != null,
+        "Global config should be set before any processing is done");
+    return streamingEngineConfig.get();
+  }
+
+  /*
+   * Subscribe to config updates by registering a callback.
+   * Callback will be called the first time with settings, if any, inline 
before the method returns.
+   */
+  @Override
+  public void onConfig(@Nonnull Consumer<StreamingGlobalConfig> callback) {
+    StreamingGlobalConfig config;
+    synchronized (this) {
+      config_callbacks.add(callback);
+      config = streamingEngineConfig.get();
+    }
+    if (config != null) {
+      callback.accept(config);
+    }
+  }
+
+  /*
+   * Package private setter for setting config

Review Comment:
   I'd just remove this one too, doesn't add much.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java:
##########
@@ -85,7 +84,7 @@ public final class StreamingWorkScheduler {
   private final HotKeyLogger hotKeyLogger;
   private final ConcurrentMap<String, StageInfo> stageInfoMap;
   private final DataflowExecutionStateSampler sampler;
-  private final AtomicReference<OperationalLimits> operationalLimits;
+  private final StreamingGlobalConfigHandle configManager;

Review Comment:
   nit: change to globalConfigHandle to match type?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandle.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.streaming.config;
+
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.sdk.annotations.Internal;
+
+@Internal
+@ThreadSafe
+public interface StreamingGlobalConfigHandle {
+
+  /*
+   * Returns the latest StreamingEnginePipelineConfig
+   */
+  StreamingGlobalConfig getConfig();
+
+  /*

Review Comment:
   change these comments to /** so they are javadoc?



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