Skip to content

Commit

Permalink
Adds support in Samza Runner to run DoFn.processElement in parallel i…
Browse files Browse the repository at this point in the history
…nside Samza tasks (#23313)
  • Loading branch information
xinyuiscool authored Sep 28, 2022
1 parent a393efa commit 406da4a
Show file tree
Hide file tree
Showing 11 changed files with 465 additions and 58 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,14 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.DefaultValueFactory;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.Hidden;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.samza.config.ConfigLoaderFactory;
import org.apache.samza.config.loaders.PropertiesConfigLoaderFactory;
import org.apache.samza.metrics.MetricsReporter;
Expand Down Expand Up @@ -129,4 +134,30 @@ public interface SamzaPipelineOptions extends PipelineOptions {
long getMaxBundleTimeMs();

void setMaxBundleTimeMs(long maxBundleTimeMs);

@Description(
"The number of threads to run DoFn.processElements in parallel within a bundle. Used only in non-portable mode.")
@Default.Integer(1)
int getNumThreadsForProcessElement();

void setNumThreadsForProcessElement(int numThreads);

@JsonIgnore
@Description(
"The ExecutorService instance to run DoFN.processElements in parallel within a bundle. Used only in non-portable mode.")
@Default.InstanceFactory(ProcessElementExecutorServiceFactory.class)
@Hidden
ExecutorService getExecutorServiceForProcessElement();

void setExecutorServiceForProcessElement(ExecutorService executorService);

class ProcessElementExecutorServiceFactory implements DefaultValueFactory<ExecutorService> {

@Override
public ExecutorService create(PipelineOptions options) {
return Executors.newFixedThreadPool(
options.as(SamzaPipelineOptions.class).getNumThreadsForProcessElement(),
new ThreadFactoryBuilder().setNameFormat("Process Element Thread-%d").build());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,14 +44,11 @@ static void validateBundlingRelatedOptions(SamzaPipelineOptions pipelineOptions)
: pipelineOptions.getConfigOverride();
final JobConfig jobConfig = new JobConfig(new MapConfig(configs));

// TODO: once Samza supports a better thread pool modle, e.g. thread
// per-task/key-range, this can be supported.
// Validate that the threadPoolSize is not override in the code
checkArgument(
jobConfig.getThreadPoolSize() <= 1,
JOB_CONTAINER_THREAD_POOL_SIZE
+ " cannot be configured to"
+ " greater than 1 for max bundle size: "
+ pipelineOptions.getMaxBundleSize());
+ " config should be replaced with SamzaPipelineOptions.numThreadsForProcessElement");
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
/*
* 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.samza.runtime;

import java.util.Collection;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.stream.Collectors;
import org.apache.beam.runners.core.DoFnRunner;
import org.apache.beam.runners.samza.SamzaPipelineOptions;
import org.apache.beam.sdk.state.TimeDomain;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* This {@link DoFnRunner} adds the capability of executing the {@link
* org.apache.beam.sdk.transforms.DoFn.ProcessElement} in the thread pool, and returns the future to
* the collector for the underlying async execution.
*/
public class AsyncDoFnRunner<InT, OutT> implements DoFnRunner<InT, OutT> {
private static final Logger LOG = LoggerFactory.getLogger(AsyncDoFnRunner.class);

private final DoFnRunner<InT, OutT> underlying;
private final ExecutorService executor;
private final OpEmitter<OutT> emitter;
private final FutureCollector<OutT> futureCollector;

public static <InT, OutT> AsyncDoFnRunner<InT, OutT> create(
DoFnRunner<InT, OutT> runner,
OpEmitter<OutT> emitter,
FutureCollector<OutT> futureCollector,
SamzaPipelineOptions options) {

LOG.info("Run DoFn with " + AsyncDoFnRunner.class.getName());
return new AsyncDoFnRunner<>(runner, emitter, futureCollector, options);
}

private AsyncDoFnRunner(
DoFnRunner<InT, OutT> runner,
OpEmitter<OutT> emitter,
FutureCollector<OutT> futureCollector,
SamzaPipelineOptions options) {
this.underlying = runner;
this.executor = options.getExecutorServiceForProcessElement();
this.emitter = emitter;
this.futureCollector = futureCollector;
}

@Override
public void startBundle() {
underlying.startBundle();
}

@Override
public void processElement(WindowedValue<InT> elem) {
final CompletableFuture<Void> future =
CompletableFuture.runAsync(
() -> {
underlying.processElement(elem);
},
executor);

final CompletableFuture<Collection<WindowedValue<OutT>>> outputFutures =
future.thenApply(
x ->
emitter.collectOutput().stream()
.map(OpMessage::getElement)
.collect(Collectors.toList()));

futureCollector.addAll(outputFutures);
}

@Override
public <KeyT> void onTimer(
String timerId,
String timerFamilyId,
KeyT key,
BoundedWindow window,
Instant timestamp,
Instant outputTimestamp,
TimeDomain timeDomain) {
underlying.onTimer(timerId, timerFamilyId, key, window, timestamp, outputTimestamp, timeDomain);
}

@Override
public void finishBundle() {
underlying.finishBundle();
}

@Override
public <KeyT> void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {
underlying.onWindowExpiration(window, timestamp, key);
}

@Override
public DoFn<InT, OutT> getFn() {
return underlying.getFn();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,12 @@

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.ServiceLoader;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
Expand Down Expand Up @@ -259,7 +259,9 @@ public void open(
sideOutputTags,
outputCoders,
doFnSchemaInformation,
(Map<String, PCollectionView<?>>) sideInputMapping);
(Map<String, PCollectionView<?>>) sideInputMapping,
emitter,
outputFutureCollector);
}

this.pushbackFnRunner =
Expand Down Expand Up @@ -479,30 +481,50 @@ static <T, OutT> CompletionStage<WindowedValue<OutT>> createOutputFuture(
}

static class FutureCollectorImpl<OutT> implements FutureCollector<OutT> {
private final List<CompletionStage<WindowedValue<OutT>>> outputFutures;
private final AtomicBoolean collectorSealed;
private CompletionStage<Collection<WindowedValue<OutT>>> outputFuture;

FutureCollectorImpl() {
/*
* Choosing synchronized list here since the concurrency is low as the message dispatch thread is single threaded.
* We need this guard against scenarios when watermark/finish bundle trigger outputs.
*/
outputFutures = Collections.synchronizedList(new ArrayList<>());
outputFuture = CompletableFuture.completedFuture(new ArrayList<>());
collectorSealed = new AtomicBoolean(true);
}

@Override
public void add(CompletionStage<WindowedValue<OutT>> element) {
checkState(
!collectorSealed.get(),
"Cannot add element to an unprepared collector. Make sure prepare() is invoked before adding elements.");

// We need synchronize guard against scenarios when watermark/finish bundle trigger outputs.
synchronized (this) {
outputFuture =
outputFuture.thenCombine(
element,
(collection, event) -> {
collection.add(event);
return collection;
});
}
}

@Override
public void addAll(CompletionStage<Collection<WindowedValue<OutT>>> elements) {
checkState(
!collectorSealed.get(),
"Cannot add elements to an unprepared collector. Make sure prepare() is invoked before adding elements.");
outputFutures.add(element);

synchronized (this) {
outputFuture = FutureUtils.combineFutures(outputFuture, elements);
}
}

@Override
public void discard() {
collectorSealed.compareAndSet(false, true);
outputFutures.clear();

synchronized (this) {
outputFuture = CompletableFuture.completedFuture(new ArrayList<>());
}
}

@Override
Expand All @@ -513,10 +535,11 @@ public CompletionStage<Collection<WindowedValue<OutT>>> finish() {
*/
collectorSealed.compareAndSet(false, true);

CompletionStage<Collection<WindowedValue<OutT>>> sealedOutputFuture =
FutureUtils.flattenFutures(outputFutures);
outputFutures.clear();
return sealedOutputFuture;
synchronized (this) {
final CompletionStage<Collection<WindowedValue<OutT>>> sealedOutputFuture = outputFuture;
outputFuture = CompletableFuture.completedFuture(new ArrayList<>());
return sealedOutputFuture;
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,13 @@ public interface FutureCollector<OutT> {
*/
void add(CompletionStage<WindowedValue<OutT>> element);

/**
* Outputs a collection of elements to the collector.
*
* @param elements to add to the collector
*/
void addAll(CompletionStage<Collection<WindowedValue<OutT>>> elements);

/**
* Discards the elements within the collector. Once the elements have been discarded, callers need
* to prepare the collector again before invoking {@link #add(CompletionStage)}.
Expand Down
Loading

0 comments on commit 406da4a

Please sign in to comment.