Skip to content

Commit

Permalink
Refactor commit logic out of StreamingDataflowWorker (apache#30312)
Browse files Browse the repository at this point in the history
  • Loading branch information
m-trieu authored and hjtran committed Apr 4, 2024
1 parent 260a143 commit c178724
Show file tree
Hide file tree
Showing 14 changed files with 1,105 additions and 205 deletions.

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.runners.dataflow.worker;

import com.google.auto.value.AutoValue;
import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.TextFormat;

Expand All @@ -29,6 +30,10 @@ public static WindmillComputationKey create(
return new AutoValue_WindmillComputationKey(computationId, key, shardingKey);
}

public static WindmillComputationKey create(String computationId, ShardedKey shardedKey) {
return create(computationId, shardedKey.key(), shardedKey.shardingKey());
}

public abstract String computationId();

public abstract ByteString key();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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 abstract StreamT stream();

abstract Runnable onClose();

@Override
public void close() throws Exception {
onClose().run();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.function.Supplier;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
Expand All @@ -36,6 +37,7 @@
* <p>The pool holds a fixed total number of streams, and keeps each stream open for a specified
* time to allow for better load-balancing.
*/
@Internal
@ThreadSafe
public class WindmillStreamPool<StreamT extends WindmillStream> {

Expand Down Expand Up @@ -131,6 +133,11 @@ public StreamT getStream() {
}
}

public CloseableStream<StreamT> getCloseableStream() {
StreamT stream = getStream();
return CloseableStream.create(stream, () -> releaseStream(stream));
}

private synchronized WindmillStreamPool.StreamData<StreamT> createAndCacheStream(int cacheKey) {
WindmillStreamPool.StreamData<StreamT> newStreamData =
new WindmillStreamPool.StreamData<>(streamSupplier.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.dataflow.worker.streaming;
package org.apache.beam.runners.dataflow.worker.windmill.client.commits;

import com.google.auto.value.AutoValue;
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.windmill.Windmill.WorkItemCommitRequest;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;

/** Value class for a queued commit. */
@Internal
@AutoValue
public abstract class Commit {

Expand All @@ -31,6 +35,10 @@ public static Commit create(
return new AutoValue_Commit(request, computationState, work);
}

public final String computationId() {
return computationState().getComputationId();
}

public abstract WorkItemCommitRequest request();

public abstract ComputationState computationState();
Expand Down
Original file line number Diff line number Diff line change
@@ -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.ABORTED);
}

public abstract String computationId();

public abstract ShardedKey shardedKey();

public abstract WorkId workId();

public abstract CommitStatus status();
}
Original file line number Diff line number Diff line change
@@ -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 MAX_COMMIT_QUEUE_BYTES = 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(
MAX_COMMIT_QUEUE_BYTES, commit -> Math.min(MAX_COMMIT_QUEUE_BYTES, 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.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()) {
// Appliance errors are propagated by exception on entire batch.
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));
}
}
}
}
Loading

0 comments on commit c178724

Please sign in to comment.