scwhittle commented on code in PR #30439: URL: https://github.com/apache/beam/pull/30439#discussion_r1526703939
########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java: ########## @@ -0,0 +1,196 @@ +/* + * 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.util; + +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor}. */ +@RunWith(JUnit4.class) +// TODO(https://github.com/apache/beam/issues/21230): Remove when new version of errorprone is +// released (2.11.0) +@SuppressWarnings("unused") +public class BoundedQueueExecutorTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(300); + private static final long MAXIMUM_BYTES_OUTSTANDING = 10000000; + private static final int DEFAULT_MAX_THREADS = 2; + private static final int DEFAULT_THREAD_EXPIRATION_SEC = 60; + + private BoundedQueueExecutor executor; + + private Runnable createSleepProcessWorkFn(CountDownLatch latch, AtomicBoolean stop) { + Runnable runnable = + () -> { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + return; + } + latch.countDown(); + int count = 0; + while (!stop.get()) { + count += 1; Review Comment: add Thread.sleep(10) or something to avoid burning cpu or could make stop a countdown latch as well to signal. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java: ########## @@ -149,18 +191,22 @@ public String summaryHtml() { builder.append(executor.getMaximumPoolSize()); builder.append("<br>/n"); + builder.append("Maximum Threads: "); + builder.append(maximumThreadCount()); + builder.append("<br>/n"); + builder.append("Active Threads: "); builder.append(executor.getActiveCount()); builder.append("<br>/n"); builder.append("Work Queue Size: "); - builder.append(elementsOutstanding); + builder.append(elementsOutstanding()); Review Comment: should the monitor.enter() at the top of summaryHtml be removed if using the accessors that grab the monitor? not sure if it is reentrant. Would be good to add renderHtml to the unit test either way. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutorTest.java: ########## @@ -0,0 +1,196 @@ +/* + * 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.util; + +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Unit tests for {@link org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor}. */ +@RunWith(JUnit4.class) +// TODO(https://github.com/apache/beam/issues/21230): Remove when new version of errorprone is +// released (2.11.0) +@SuppressWarnings("unused") +public class BoundedQueueExecutorTest { + @Rule public transient Timeout globalTimeout = Timeout.seconds(300); + private static final long MAXIMUM_BYTES_OUTSTANDING = 10000000; + private static final int DEFAULT_MAX_THREADS = 2; + private static final int DEFAULT_THREAD_EXPIRATION_SEC = 60; + + private BoundedQueueExecutor executor; + + private Runnable createSleepProcessWorkFn(CountDownLatch latch, AtomicBoolean stop) { + Runnable runnable = + () -> { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + return; + } + latch.countDown(); + int count = 0; + while (!stop.get()) { + count += 1; + } + }; + return runnable; + } + + @Before + public void setUp() { + this.executor = + new BoundedQueueExecutor( + DEFAULT_MAX_THREADS, + DEFAULT_THREAD_EXPIRATION_SEC, + TimeUnit.SECONDS, + DEFAULT_MAX_THREADS + 100, + MAXIMUM_BYTES_OUTSTANDING, + new ThreadFactoryBuilder() + .setNameFormat("DataflowWorkUnits-%d") + .setDaemon(true) + .build()); + } + Review Comment: add a basic test since we have a test for this now below we never schedule new work after previous work started for example -- 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]
