scwhittle commented on code in PR #32408: URL: https://github.com/apache/beam/pull/32408#discussion_r1758370482
########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/FixedGlobalConfigHandleTest.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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class FixedGlobalConfigHandleTest { + + @Test + public void getConfig() { + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + FixedGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(config); + assertEquals(config, globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver() throws InterruptedException { + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + FixedGlobalConfigHandle globalConfigHandle = new FixedGlobalConfigHandle(config); + AtomicReference<StreamingGlobalConfig> configFromCallback = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + globalConfigHandle.registerConfigObserver( + cbConfig -> { + configFromCallback.set(cbConfig); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); Review Comment: the contract is to call it inline so it seems you could just verify configFromCallback is equal without waiting. If it is racy/flaky it means it isn't calling inline. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java: ########## @@ -0,0 +1,152 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class StreamingGlobalConfigHandleImplTest { + + @Test + public void getConfig() { + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + globalConfigHandle.setConfig(config); + assertEquals(config, globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver_configSetAfterRegisteringCallback() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference<StreamingGlobalConfig> configFromCallback = new AtomicReference<>(); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + globalConfigHandle.setConfig(configToSet); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver_configSetBeforeRegisteringCallback() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference<StreamingGlobalConfig> configFromCallback = new AtomicReference<>(); + globalConfigHandle.setConfig(configToSet); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); Review Comment: ditto supposed to call inline ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerStatusPages.java: ########## @@ -150,6 +157,16 @@ private void addStreamingEngineStatusPages() { statusPages.addCapturePage(Preconditions.checkNotNull(channelzServlet)); statusPages.addStatusDataProvider( "streaming", "Streaming RPCs", Preconditions.checkNotNull(windmillStreamFactory)); + statusPages.addStatusDataProvider( + "jobSettings", + "User Worker Job Settings", + writer -> { + if (seConfig.get() == null) { Review Comment: seems safer to store result of get() on stack instead of calling twice since it can change ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java: ########## @@ -0,0 +1,78 @@ +/* + * 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; Review Comment: use vendored ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImpl.java: ########## @@ -0,0 +1,78 @@ +/* + * 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<>(); + + @Override + public StreamingGlobalConfig getConfig() { + Preconditions.checkState( + streamingEngineConfig.get() != null, + "Global config should be set before any processing is done"); + return streamingEngineConfig.get(); + } + + @Override + public void registerConfigObserver(@Nonnull Consumer<StreamingGlobalConfig> callback) { + StreamingGlobalConfig config; + synchronized (this) { + config_callbacks.add(callback); + config = streamingEngineConfig.get(); + } + if (config != null) { + // read config from streamingEngineConfig again + // to prevent calling callback with stale config. + // The cached `config` will be stale if setConfig + // ran after the synchronized block. + callback.accept(streamingEngineConfig.get()); Review Comment: on second thought this is still racy, we can call get() get the old result, pause before we call accept and then set config runs and accepts new result and then this accept runs. Maybe we should call under some lock to prevent that? Or could have some single-threaded executor we use for running all the callbacks and here we schedule on that and then block for it to run? Below in setConfig we could just schedule it to run and not have setConfig block on running the callbacks. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingGlobalConfigHandleImplTest.java: ########## @@ -0,0 +1,152 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.OperationalLimits; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.UserWorkerRunnerV1Settings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class StreamingGlobalConfigHandleImplTest { + + @Test + public void getConfig() { + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig config = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + globalConfigHandle.setConfig(config); + assertEquals(config, globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver_configSetAfterRegisteringCallback() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference<StreamingGlobalConfig> configFromCallback = new AtomicReference<>(); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + globalConfigHandle.setConfig(configToSet); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver_configSetBeforeRegisteringCallback() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + StreamingGlobalConfig configToSet = + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + AtomicReference<StreamingGlobalConfig> configFromCallback = new AtomicReference<>(); + globalConfigHandle.setConfig(configToSet); + globalConfigHandle.registerConfigObserver( + config -> { + configFromCallback.set(config); + latch.countDown(); + }); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertEquals(configFromCallback.get(), globalConfigHandle.getConfig()); + } + + @Test + public void registerConfigObserver_shouldNotCallCallbackForIfConfigRemainsSame() + throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callbackCount = new AtomicInteger(0); + StreamingGlobalConfigHandleImpl globalConfigHandle = new StreamingGlobalConfigHandleImpl(); + Supplier<StreamingGlobalConfig> configToSet = + () -> + StreamingGlobalConfig.builder() + .setOperationalLimits( + OperationalLimits.builder() + .setMaxOutputValueBytes(123) + .setMaxOutputKeyBytes(324) + .setMaxWorkItemCommitBytes(456) + .build()) + .setWindmillServiceEndpoints(ImmutableSet.of(HostAndPort.fromHost("windmillHost"))) + .setUserWorkerJobSettings( + UserWorkerRunnerV1Settings.newBuilder() + .setUseSeparateWindmillHeartbeatStreams(false) + .build()) + .build(); + globalConfigHandle.registerConfigObserver( + config -> { + callbackCount.incrementAndGet(); + latch.countDown(); + }); + globalConfigHandle.setConfig(configToSet.get()); + // call setter again with same config + globalConfigHandle.setConfig(configToSet.get()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); Review Comment: seems like a sleep would be better to catch possible second call than latch which is going to stop blocking after first. -- 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]
