scwhittle commented on code in PR #30312: URL: https://github.com/apache/beam/pull/30312#discussion_r1510917576
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java: ########## @@ -0,0 +1,210 @@ +/* + * 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.commits; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +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.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Streaming engine implementation of {@link WorkCommitter}. Commits work back to Streaming Engine + * backend. + */ +@Internal +@ThreadSafe +public final class StreamingEngineWorkCommitter implements WorkCommitter { + private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineWorkCommitter.class); + private static final int COMMIT_BATCH_SIZE = 5; + private static final int TARGET_COMMIT_BATCH_SIZE = 500 << 20; // 500MB + + private final Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory; + private final WeightedBoundedQueue<Commit> commitQueue; + private final ExecutorService commitSenders; + private final AtomicLong activeCommitBytes; + private final Consumer<CompleteCommit> onCommitComplete; + private final int numCommitSenders; + + private StreamingEngineWorkCommitter( + Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory, + int numCommitSenders, + Consumer<CompleteCommit> onCommitComplete) { + this.commitWorkStreamFactory = commitWorkStreamFactory; + this.commitQueue = + WeightedBoundedQueue.create( + TARGET_COMMIT_BATCH_SIZE, + commit -> Math.min(TARGET_COMMIT_BATCH_SIZE, commit.getSize())); + this.commitSenders = + Executors.newFixedThreadPool( + numCommitSenders, + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MAX_PRIORITY) + .setNameFormat("CommitThread-%d") + .build()); + this.activeCommitBytes = new AtomicLong(); + this.onCommitComplete = onCommitComplete; + this.numCommitSenders = numCommitSenders; + } + + public static StreamingEngineWorkCommitter create( + Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory, + int numCommitSenders, + Consumer<CompleteCommit> onCommitComplete) { + return new StreamingEngineWorkCommitter( + commitWorkStreamFactory, numCommitSenders, onCommitComplete); + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public void start() { + if (!commitSenders.isShutdown() || !commitSenders.isTerminated()) { Review Comment: seems like should be &&, or just have the shutdown check since I think it has to precede termination. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitter.java: ########## @@ -0,0 +1,167 @@ +/* + * 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.commits; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.streaming.WorkId; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitWorkRequest; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Streaming appliance implementation of {@link WorkCommitter}. */ +@Internal +@ThreadSafe +public final class StreamingApplianceWorkCommitter implements WorkCommitter { + private static final Logger LOG = LoggerFactory.getLogger(StreamingApplianceWorkCommitter.class); + private static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20; + private static final int TARGET_COMMIT_BATCH_SIZE = 500 << 20; // 500MB + + private final Consumer<CommitWorkRequest> commitWorkFn; + private final WeightedBoundedQueue<Commit> commitQueue; + private final ExecutorService commitWorkers; + private final AtomicLong activeCommitBytes; + private final Consumer<CompleteCommit> onCommitComplete; + + private StreamingApplianceWorkCommitter( + Consumer<CommitWorkRequest> commitWorkFn, Consumer<CompleteCommit> onCommitComplete) { + this.commitWorkFn = commitWorkFn; + this.commitQueue = + WeightedBoundedQueue.create( + TARGET_COMMIT_BATCH_SIZE, + commit -> Math.min(TARGET_COMMIT_BATCH_SIZE, commit.getSize())); + this.commitWorkers = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MAX_PRIORITY) + .setNameFormat("CommitThread-%d") + .build()); + this.activeCommitBytes = new AtomicLong(); + this.onCommitComplete = onCommitComplete; + } + + public static StreamingApplianceWorkCommitter create( + Consumer<CommitWorkRequest> commitWork, Consumer<CompleteCommit> onCommitComplete) { + return new StreamingApplianceWorkCommitter(commitWork, onCommitComplete); + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public void start() { + if (!commitWorkers.isTerminated() || !commitWorkers.isShutdown()) { + commitWorkers.submit(this::commitLoop); + } + } + + @Override + public void commit(Commit commit) { + commitQueue.put(commit); + } + + @Override + public long currentActiveCommitBytes() { + return activeCommitBytes.get(); + } + + @Override + public void stop() { + commitWorkers.shutdownNow(); + } + + @Override + public int parallelism() { + return 1; + } + + private void commitLoop() { + Map<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> computationRequestMap = + new HashMap<>(); + while (true) { + computationRequestMap.clear(); + CommitWorkRequest.Builder commitRequestBuilder = CommitWorkRequest.newBuilder(); + long commitBytes = 0; + // Block until we have a commit, then batch with additional commits. + Commit commit; + try { + commit = commitQueue.take(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + continue; + } + while (commit != null) { + ComputationState computationState = commit.computationState(); + commit.work().setState(Work.State.COMMITTING); + Windmill.ComputationCommitWorkRequest.Builder computationRequestBuilder = + computationRequestMap.get(computationState); + if (computationRequestBuilder == null) { + computationRequestBuilder = commitRequestBuilder.addRequestsBuilder(); + computationRequestBuilder.setComputationId(computationState.getComputationId()); + computationRequestMap.put(computationState, computationRequestBuilder); + } + computationRequestBuilder.addRequests(commit.request()); + // Send the request if we've exceeded the bytes or there is no more + // pending work. commitBytes is a long, so this cannot overflow. + commitBytes += commit.getSize(); + if (commitBytes >= TARGET_COMMIT_BUNDLE_BYTES) { + break; + } + commit = commitQueue.poll(); + } + commitWork(commitRequestBuilder.build(), commitBytes); + completeWork(computationRequestMap); + } + } + + private void commitWork(CommitWorkRequest commitRequest, long commitBytes) { + LOG.trace("Commit: {}", commitRequest); + activeCommitBytes.addAndGet(commitBytes); + commitWorkFn.accept(commitRequest); + activeCommitBytes.addAndGet(-commitBytes); + } + + private void completeWork( + Map<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> committedWork) { + for (Map.Entry<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> entry : + committedWork.entrySet()) { + for (Windmill.WorkItemCommitRequest workRequest : entry.getValue().getRequestsList()) { + onCommitComplete.accept( + CompleteCommit.create( + entry.getKey().getComputationId(), + ShardedKey.create(workRequest.getKey(), workRequest.getShardingKey()), + WorkId.builder() + .setCacheToken(workRequest.getCacheToken()) + .setWorkToken(workRequest.getWorkToken()) + .build(), + Windmill.CommitStatus.OK)); Review Comment: // Appliance errors are propagated by exception on entire batch. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingApplianceWorkCommitterTest.java: ########## @@ -0,0 +1,147 @@ +/* + * 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.commits; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertNotNull; + +import com.google.api.services.dataflow.model.MapTask; +import com.google.common.truth.Correspondence; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Consumer; +import org.apache.beam.runners.dataflow.worker.FakeWindmillServer; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Instant; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ErrorCollector; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +@RunWith(JUnit4.class) +public class StreamingApplianceWorkCommitterTest { + @Rule public ErrorCollector errorCollector = new ErrorCollector(); + private FakeWindmillServer fakeWindmillServer; + private StreamingApplianceWorkCommitter workCommitter; + + private static Work createMockWork(long workToken, Consumer<Work> processWorkFn) { + return Work.create( + Windmill.WorkItem.newBuilder() + .setKey(ByteString.EMPTY) + .setWorkToken(workToken) + .setShardingKey(workToken) + .setCacheToken(workToken) + .build(), + Instant::now, + Collections.emptyList(), + processWorkFn); + } + + private static ComputationState createComputationState(String computationId) { + return new ComputationState( + computationId, + new MapTask().setSystemName("system").setStageName("stage"), + Mockito.mock(BoundedQueueExecutor.class), + ImmutableMap.of(), + null); + } + + private StreamingApplianceWorkCommitter createWorkCommitter( + Consumer<CompleteCommit> onCommitComplete) { + return StreamingApplianceWorkCommitter.create(fakeWindmillServer::commitWork, onCommitComplete); + } + + @Before + public void setUp() { + fakeWindmillServer = + new FakeWindmillServer( + errorCollector, ignored -> Optional.of(Mockito.mock(ComputationState.class))); + } + + @After + public void cleanUp() { + workCommitter.stop(); + } + + @Test + public void testParallelism() { + workCommitter = createWorkCommitter(ignored -> {}); Review Comment: doesn't seem worth testing IMO ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java: ########## @@ -310,6 +311,10 @@ public void invalidate(ByteString processingKey, long shardingKey) { keyIndex.remove(key); } + public void invalidate(ShardedKey shardedKey) { Review Comment: final ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java: ########## @@ -1445,163 +1434,21 @@ private WorkItemCommitRequest buildWorkItemTruncationRequest( return outputBuilder.build(); } - private void commitLoop() { - Map<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> computationRequestMap = - new HashMap<>(); - while (running.get()) { - computationRequestMap.clear(); - Windmill.CommitWorkRequest.Builder commitRequestBuilder = - Windmill.CommitWorkRequest.newBuilder(); - long commitBytes = 0; - // Block until we have a commit, then batch with additional commits. - Commit commit = null; - try { - commit = commitQueue.take(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - continue; - } - while (commit != null) { - ComputationState computationState = commit.computationState(); - commit.work().setState(Work.State.COMMITTING); - Windmill.ComputationCommitWorkRequest.Builder computationRequestBuilder = - computationRequestMap.get(computationState); - if (computationRequestBuilder == null) { - computationRequestBuilder = commitRequestBuilder.addRequestsBuilder(); - computationRequestBuilder.setComputationId(computationState.getComputationId()); - computationRequestMap.put(computationState, computationRequestBuilder); - } - computationRequestBuilder.addRequests(commit.request()); - // Send the request if we've exceeded the bytes or there is no more - // pending work. commitBytes is a long, so this cannot overflow. - commitBytes += commit.getSize(); - if (commitBytes >= TARGET_COMMIT_BUNDLE_BYTES) { - break; - } - commit = commitQueue.poll(); - } - Windmill.CommitWorkRequest commitRequest = commitRequestBuilder.build(); - LOG.trace("Commit: {}", commitRequest); - activeCommitBytes.addAndGet(commitBytes); - windmillServer.commitWork(commitRequest); - activeCommitBytes.addAndGet(-commitBytes); - for (Map.Entry<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> entry : - computationRequestMap.entrySet()) { - ComputationState computationState = entry.getKey(); - for (Windmill.WorkItemCommitRequest workRequest : entry.getValue().getRequestsList()) { - computationState.completeWorkAndScheduleNextWorkForKey( - ShardedKey.create(workRequest.getKey(), workRequest.getShardingKey()), - WorkId.builder() - .setCacheToken(workRequest.getCacheToken()) - .setWorkToken(workRequest.getWorkToken()) - .build()); - } - } - } - } - - // Adds the commit to the commitStream if it fits, returning true iff it is consumed. - private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { - Preconditions.checkNotNull(commit); - final ComputationState state = commit.computationState(); - final Windmill.WorkItemCommitRequest request = commit.request(); - // Drop commits for failed work. Such commits will be dropped by Windmill anyway. - if (commit.work().isFailed()) { + private void onStreamingCommitComplete(CompleteCommit completeCommit) { Review Comment: rename to onCommitComplete ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/CompleteCommit.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.commits; + +import com.google.auto.value.AutoValue; +import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; +import org.apache.beam.runners.dataflow.worker.streaming.WorkId; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; + +/** + * A {@link Commit} is marked as complete when it has been attempted to be committed back to + * Streaming Engine/Appliance via {@link + * org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub#commitWorkStream(StreamObserver)} + * for Streaming Engine or {@link + * org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub#commitWork(Windmill.CommitWorkRequest, + * StreamObserver)} for Streaming Appliance. + */ +@Internal +@AutoValue +public abstract class CompleteCommit { + + public static CompleteCommit create(Commit commit, CommitStatus commitStatus) { + return new AutoValue_CompleteCommit( + commit.computationId(), + ShardedKey.create(commit.request().getKey(), commit.request().getShardingKey()), + WorkId.builder() + .setWorkToken(commit.request().getWorkToken()) + .setCacheToken(commit.request().getCacheToken()) + .build(), + commitStatus); + } + + public static CompleteCommit create( + String computationId, ShardedKey shardedKey, WorkId workId, CommitStatus status) { + return new AutoValue_CompleteCommit(computationId, shardedKey, workId, status); + } + + public static CompleteCommit forFailedWork(Commit commit) { + return create(commit, CommitStatus.DEFAULT); Review Comment: DEFAULT doesn't seem like a good status for failed ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/CloseableStream.java: ########## @@ -0,0 +1,47 @@ +/* + * 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; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.annotations.Internal; + +/** + * Wrapper for a {@link WindmillStream} that allows callers to tie an action after the stream is + * finished being used. Has an option for closing code to be a no-op. + */ +@Internal +@AutoValue +public abstract class CloseableStream<StreamT extends WindmillStream> implements AutoCloseable { + public static <StreamT extends WindmillStream> CloseableStream<StreamT> create( + StreamT stream, Runnable onClose) { + return new AutoValue_CloseableStream<>(stream, onClose); + } + + public static <StreamT extends WindmillStream> CloseableStream<StreamT> create(StreamT stream) { Review Comment: is this used? can it be done at call sites instead if it is? Then at call-site it's more obvious it isn't doing anything on close. ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.commits; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertNotNull; + +import com.google.api.services.dataflow.model.MapTask; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.FakeWindmillServer; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ErrorCollector; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +@RunWith(JUnit4.class) +public class StreamingEngineWorkCommitterTest { + + @Rule public ErrorCollector errorCollector = new ErrorCollector(); + private StreamingEngineWorkCommitter workCommitter; + private FakeWindmillServer fakeWindmillServer; + private Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory; + + private static Work createMockWork(long workToken, Consumer<Work> processWorkFn) { + return Work.create( + Windmill.WorkItem.newBuilder() + .setKey(ByteString.EMPTY) + .setWorkToken(workToken) + .setShardingKey(workToken) + .setCacheToken(workToken) + .build(), + Instant::now, + Collections.emptyList(), + processWorkFn); + } + + private static ComputationState createComputationState(String computationId) { + return new ComputationState( + computationId, + new MapTask().setSystemName("system").setStageName("stage"), + Mockito.mock(BoundedQueueExecutor.class), + ImmutableMap.of(), + null); + } + + private static CompleteCommit asCompleteCommit(Commit commit) { + if (commit.work().isFailed()) { + return CompleteCommit.forFailedWork(commit); + } + + return CompleteCommit.create(commit, Windmill.CommitStatus.OK); + } + + @Before + public void setUp() { + fakeWindmillServer = + new FakeWindmillServer( + errorCollector, ignored -> Optional.of(Mockito.mock(ComputationState.class))); + commitWorkStreamFactory = + WindmillStreamPool.create( + 1, Duration.standardMinutes(1), fakeWindmillServer::commitWorkStream) + ::getCloseableStream; + } + + @After + public void cleanUp() { + workCommitter.stop(); + } + + private StreamingEngineWorkCommitter createWorkCommitter( + Consumer<CompleteCommit> onCommitComplete) { + return StreamingEngineWorkCommitter.create(commitWorkStreamFactory, 1, onCommitComplete); + } + + @Test + public void testCommit_sendsCommitsToStreamingEngine() { + workCommitter = createWorkCommitter(ignored -> {}); + List<Commit> commits = new ArrayList<>(); + for (int i = 1; i <= 5; i++) { + Work work = createMockWork(i, ignored -> {}); + WorkItemCommitRequest commitRequest = + WorkItemCommitRequest.newBuilder() + .setKey(work.getWorkItem().getKey()) + .setShardingKey(work.getWorkItem().getShardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .build(); + commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work)); + } + + workCommitter.start(); + commits.forEach(workCommitter::commit); + + Map<Long, WorkItemCommitRequest> committed = + fakeWindmillServer.waitForAndGetCommits(commits.size()); + + for (Commit commit : commits) { + WorkItemCommitRequest request = committed.get(commit.work().getWorkItem().getWorkToken()); + assertNotNull(request); + assertThat(request).isEqualTo(commit.request()); + } + } + + @Test + public void testCommit_handlesFailedCommits() { + Set<CompleteCommit> failedCommits = new HashSet<>(); + workCommitter = createWorkCommitter(failedCommits::add); + List<Commit> commits = new ArrayList<>(); + for (int i = 1; i <= 10; i++) { + Work work = createMockWork(i, ignored -> {}); + // Fail half of the work. + if (i % 2 == 0) { + work.setFailed(); + } + WorkItemCommitRequest commitRequest = + WorkItemCommitRequest.newBuilder() + .setKey(work.getWorkItem().getKey()) + .setShardingKey(work.getWorkItem().getShardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .build(); + commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work)); + } + + workCommitter.start(); + commits.forEach(workCommitter::commit); + + Map<Long, WorkItemCommitRequest> committed = + fakeWindmillServer.waitForAndGetCommits(commits.size() / 2); + + for (Commit commit : commits) { + if (commit.work().isFailed()) { + assertThat(failedCommits).contains(asCompleteCommit(commit)); + assertThat(committed).doesNotContainKey(commit.work().getWorkItem().getWorkToken()); + } + } + } + + @Test + public void testCommit_handlesCompleteCommits() { + Set<CompleteCommit> completeCommits = new HashSet<>(); + workCommitter = createWorkCommitter(completeCommits::add); + List<Commit> commits = new ArrayList<>(); + for (int i = 1; i <= 5; i++) { + Work work = createMockWork(i, ignored -> {}); + WorkItemCommitRequest commitRequest = + WorkItemCommitRequest.newBuilder() + .setKey(work.getWorkItem().getKey()) + .setShardingKey(work.getWorkItem().getShardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .build(); + commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work)); + } + + workCommitter.start(); + commits.forEach(workCommitter::commit); + + Map<Long, WorkItemCommitRequest> committed = + fakeWindmillServer.waitForAndGetCommits(commits.size()); + + for (Commit commit : commits) { + WorkItemCommitRequest request = committed.get(commit.work().getWorkItem().getWorkToken()); + assertNotNull(request); + assertThat(request).isEqualTo(commit.request()); + assertThat(completeCommits).contains(asCompleteCommit(commit)); Review Comment: add a test that verifies some particular keys failing in windmill, that the non-ok commit status is plumbed correctly ########## runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitterTest.java: ########## @@ -0,0 +1,208 @@ +/* + * 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.commits; + +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertNotNull; + +import com.google.api.services.dataflow.model.MapTask; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Supplier; +import org.apache.beam.runners.dataflow.worker.FakeWindmillServer; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ErrorCollector; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +@RunWith(JUnit4.class) +public class StreamingEngineWorkCommitterTest { + + @Rule public ErrorCollector errorCollector = new ErrorCollector(); + private StreamingEngineWorkCommitter workCommitter; + private FakeWindmillServer fakeWindmillServer; + private Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory; + + private static Work createMockWork(long workToken, Consumer<Work> processWorkFn) { + return Work.create( + Windmill.WorkItem.newBuilder() + .setKey(ByteString.EMPTY) + .setWorkToken(workToken) + .setShardingKey(workToken) + .setCacheToken(workToken) + .build(), + Instant::now, + Collections.emptyList(), + processWorkFn); + } + + private static ComputationState createComputationState(String computationId) { + return new ComputationState( + computationId, + new MapTask().setSystemName("system").setStageName("stage"), + Mockito.mock(BoundedQueueExecutor.class), + ImmutableMap.of(), + null); + } + + private static CompleteCommit asCompleteCommit(Commit commit) { + if (commit.work().isFailed()) { + return CompleteCommit.forFailedWork(commit); + } + + return CompleteCommit.create(commit, Windmill.CommitStatus.OK); + } + + @Before + public void setUp() { + fakeWindmillServer = + new FakeWindmillServer( + errorCollector, ignored -> Optional.of(Mockito.mock(ComputationState.class))); + commitWorkStreamFactory = + WindmillStreamPool.create( + 1, Duration.standardMinutes(1), fakeWindmillServer::commitWorkStream) + ::getCloseableStream; + } + + @After + public void cleanUp() { + workCommitter.stop(); + } + + private StreamingEngineWorkCommitter createWorkCommitter( + Consumer<CompleteCommit> onCommitComplete) { + return StreamingEngineWorkCommitter.create(commitWorkStreamFactory, 1, onCommitComplete); + } + + @Test + public void testCommit_sendsCommitsToStreamingEngine() { + workCommitter = createWorkCommitter(ignored -> {}); + List<Commit> commits = new ArrayList<>(); + for (int i = 1; i <= 5; i++) { + Work work = createMockWork(i, ignored -> {}); + WorkItemCommitRequest commitRequest = + WorkItemCommitRequest.newBuilder() + .setKey(work.getWorkItem().getKey()) + .setShardingKey(work.getWorkItem().getShardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .build(); + commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work)); + } + + workCommitter.start(); + commits.forEach(workCommitter::commit); + + Map<Long, WorkItemCommitRequest> committed = + fakeWindmillServer.waitForAndGetCommits(commits.size()); + + for (Commit commit : commits) { + WorkItemCommitRequest request = committed.get(commit.work().getWorkItem().getWorkToken()); + assertNotNull(request); + assertThat(request).isEqualTo(commit.request()); + } + } + + @Test + public void testCommit_handlesFailedCommits() { + Set<CompleteCommit> failedCommits = new HashSet<>(); + workCommitter = createWorkCommitter(failedCommits::add); + List<Commit> commits = new ArrayList<>(); + for (int i = 1; i <= 10; i++) { + Work work = createMockWork(i, ignored -> {}); + // Fail half of the work. + if (i % 2 == 0) { + work.setFailed(); + } + WorkItemCommitRequest commitRequest = + WorkItemCommitRequest.newBuilder() + .setKey(work.getWorkItem().getKey()) + .setShardingKey(work.getWorkItem().getShardingKey()) + .setWorkToken(work.getWorkItem().getWorkToken()) + .setCacheToken(work.getWorkItem().getCacheToken()) + .build(); + commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work)); + } + + workCommitter.start(); + commits.forEach(workCommitter::commit); + + Map<Long, WorkItemCommitRequest> committed = + fakeWindmillServer.waitForAndGetCommits(commits.size() / 2); + + for (Commit commit : commits) { + if (commit.work().isFailed()) { Review Comment: verify that non-failed ones are in committed ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/commits/StreamingEngineWorkCommitter.java: ########## @@ -0,0 +1,210 @@ +/* + * 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.commits; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.client.CloseableStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.CommitWorkStream; +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.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Streaming engine implementation of {@link WorkCommitter}. Commits work back to Streaming Engine + * backend. + */ +@ThreadSafe +final class StreamingEngineWorkCommitter implements WorkCommitter { + private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineWorkCommitter.class); + private static final int COMMIT_BATCH_SIZE = 5; Review Comment: I think you might have changed the wrong one, the bytes is still a max since that is the queue limit. I meant what is now COMMIT_BATCH_SIZE to be TARGET_COMMIT_BATCH_SIZE (could use KEYS nstead of SIZE as well) -- 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]
