scwhittle commented on code in PR #31883: URL: https://github.com/apache/beam/pull/31883#discussion_r1702238706
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkItemBuffer.java: ########## @@ -0,0 +1,133 @@ +/* + * 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.client.grpc; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link ByteString} buffer of {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk}(s). + * + * <p>Once all serialized chunks of an {@link WorkItem} have been received, provides functionality + * to flush (deserialize) the chunk of bytes into a {@link WorkItem}. + * + * @implNote This class is not thread safe, and provides no synchronization underneath. + */ +@NotThreadSafe +final class GetWorkItemBuffer { Review Comment: what about a little more specific name like GetWorkResponseChunkAssembler? clearer what it does versus buffer, and clearer that it is taking the chunks and producing work item instead of a queue of work items. could name ConstructedWorkItem to AssembledWorkItem then to match naming ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetHolder.java: ########## @@ -0,0 +1,32 @@ +/* + * 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.work.budget; + +/** + * Represents something that holds some {@link + * org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget} + */ +public interface GetWorkBudgetHolder { Review Comment: holder doesn't seem too bad, but what about GetWorkBudgetSpender? I like that it captures it is using it (without being confusing with a end user etc) ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java: ########## @@ -765,6 +757,11 @@ int numCommitThreads() { return workCommitter.parallelism(); } + @VisibleForTesting + CacheStats getStateCacheStats() { Review Comment: how about justhaving the test use getComputationStateCache().getCacheStats() instead of another testing method ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java: ########## @@ -0,0 +1,285 @@ +/* + * 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.harness; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; + +import com.google.auto.value.AutoBuilder; +import com.google.auto.value.AutoOneOf; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.RpcException; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; +import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link StreamingWorkerHarness} implementations that fetch {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from a single source. + */ +@Internal +public final class SingleSourceWorkerHarness implements StreamingWorkerHarness { + private static final Logger LOG = LoggerFactory.getLogger(SingleSourceWorkerHarness.class); + private static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; + + private final AtomicBoolean isRunning; + private final WorkCommitter workCommitter; + private final GetDataClient getDataClient; + private final HeartbeatSender heartbeatSender; + private final StreamingWorkScheduler streamingWorkScheduler; + private final Runnable waitForResources; + private final Function<String, Optional<ComputationState>> computationStateFetcher; + private final ExecutorService workProviderExecutor; + private final GetWorkSender getWorkSender; + + SingleSourceWorkerHarness( + WorkCommitter workCommitter, + GetDataClient getDataClient, + HeartbeatSender heartbeatSender, + StreamingWorkScheduler streamingWorkScheduler, + Runnable waitForResources, + Function<String, Optional<ComputationState>> computationStateFetcher, + GetWorkSender getWorkSender) { + this.workCommitter = workCommitter; + this.getDataClient = getDataClient; + this.heartbeatSender = heartbeatSender; + this.streamingWorkScheduler = streamingWorkScheduler; + this.waitForResources = waitForResources; + this.computationStateFetcher = computationStateFetcher; + this.workProviderExecutor = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("DispatchThread") + .build()); + this.isRunning = new AtomicBoolean(false); + this.getWorkSender = getWorkSender; + } + + public static SingleSourceWorkerHarness.Builder builder() { + return new AutoBuilder_SingleSourceWorkerHarness_Builder(); + } + + @SuppressWarnings("FutureReturnValueIgnored") + @Override + public void start() { + Preconditions.checkState( + isRunning.compareAndSet(false, true), + "Multiple calls to {}.start() are not allowed.", + getClass()); + workCommitter.start(); + workProviderExecutor.submit( Review Comment: I think since you don't want the future you could use execute(...) instead of submit(...) and remove the warning suppression ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java: ########## @@ -240,25 +234,46 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); - WorkItemBuffer workItemBuffer = - workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new WorkItemBuffer()); + GetWorkItemBuffer workItemBuffer = + workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkItemBuffer()); workItemBuffer.append(chunk); // The entire WorkItem has been received, it is ready to be processed. if (chunk.getRemainingBytesForWorkItem() == 0) { Review Comment: it seems a little nicer if the assembler did this since it already gets the chunk. Can append be changed to return the work item if it is completed? the size could be communicated in that Constructed/AssembledWorkItem object ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java: ########## @@ -240,25 +234,46 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); - WorkItemBuffer workItemBuffer = - workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new WorkItemBuffer()); + GetWorkItemBuffer workItemBuffer = + workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkItemBuffer()); workItemBuffer.append(chunk); // The entire WorkItem has been received, it is ready to be processed. if (chunk.getRemainingBytesForWorkItem() == 0) { - workItemBuffer.runAndReset(); + long size = workItemBuffer.bufferedSize(); + workItemBuffer.flushToWorkItem().ifPresent(this::consumeWorkItem); // Record the fact that there are now fewer outstanding messages and bytes on the stream. - inFlightBudget.updateAndGet(budget -> budget.subtract(1, workItemBuffer.bufferedSize())); + inFlightBudget.updateAndGet(budget -> budget.subtract(1, size)); } } + private void consumeWorkItem(ConstructedWorkItem constructedWorkItem) { + WorkItem workItem = constructedWorkItem.workItem(); + GetWorkItemBuffer.ComputationMetadata metadata = constructedWorkItem.computationMetadata(); + pendingResponseBudget.getAndUpdate(budget -> budget.apply(1, workItem.getSerializedSize())); + try { + workItemScheduler.scheduleWork( + workItem, + createWatermarks(workItem, Preconditions.checkNotNull(metadata)), + createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), + constructedWorkItem.latencyAttributions()); + } finally { + pendingResponseBudget.getAndUpdate(budget -> budget.apply(-1, -workItem.getSerializedSize())); + } + } + + private Work.ProcessingContext createProcessingContext(String computationId) { + return Work.createProcessingContext( + computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get()); + } + @Override protected void startThrottleTimer() { getWorkThrottleTimer.start(); } @Override - public synchronized void adjustBudget(long itemsDelta, long bytesDelta) { + public void adjustBudget(long itemsDelta, long bytesDelta) { nextBudgetAdjustment.set(nextBudgetAdjustment.get().apply(itemsDelta, bytesDelta)); Review Comment: use updateAndGet instead of set with get? -- 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]
