Skip to content

Commit

Permalink
[BEAM-12174] Samza Portable Runner Support (#14554)
Browse files Browse the repository at this point in the history
  • Loading branch information
kw2542 committed Apr 21, 2021
1 parent 2012180 commit 9ab200c
Show file tree
Hide file tree
Showing 12 changed files with 226 additions and 233 deletions.
2 changes: 1 addition & 1 deletion runners/samza/build.gradle
Expand Up @@ -52,14 +52,14 @@ dependencies {
compile library.java.jackson_annotations
compile library.java.slf4j_api
compile library.java.joda_time
compile library.java.args4j
compile library.java.commons_io
runtimeOnly "org.rocksdb:rocksdbjni:6.15.2"
runtimeOnly "org.scala-lang:scala-library:2.11.8"
compile "org.apache.samza:samza-api:$samza_version"
compile "org.apache.samza:samza-core_2.11:$samza_version"
runtimeOnly "org.apache.samza:samza-kafka_2.11:$samza_version"
runtimeOnly "org.apache.samza:samza-kv_2.11:$samza_version"
compile project(":sdks:java:expansion-service")
compile "org.apache.samza:samza-kv-rocksdb_2.11:$samza_version"
compile "org.apache.samza:samza-kv-inmemory_2.11:$samza_version"
compile "org.apache.samza:samza-yarn_2.11:$samza_version"
Expand Down
Expand Up @@ -17,53 +17,22 @@
*/
package org.apache.beam.runners.samza;

import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;

import java.time.Duration;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.beam.runners.fnexecution.GrpcFnServer;
import org.apache.beam.runners.fnexecution.ServerFactory;
import org.apache.beam.runners.fnexecution.control.ControlClientPool;
import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
import org.apache.beam.runners.fnexecution.control.JobBundleFactory;
import org.apache.beam.runners.fnexecution.control.MapControlClientPool;
import org.apache.beam.runners.fnexecution.control.SingleEnvironmentInstanceJobBundleFactory;
import org.apache.beam.runners.fnexecution.data.GrpcDataService;
import org.apache.beam.runners.fnexecution.environment.EnvironmentFactory;
import org.apache.beam.runners.fnexecution.environment.RemoteEnvironment;
import org.apache.beam.runners.fnexecution.state.GrpcStateService;
import org.apache.beam.runners.samza.metrics.SamzaMetricsContainer;
import org.apache.beam.sdk.fn.IdGenerator;
import org.apache.beam.sdk.fn.IdGenerators;
import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
import org.apache.samza.context.ApplicationContainerContext;
import org.apache.samza.context.ApplicationContainerContextFactory;
import org.apache.samza.context.ContainerContext;
import org.apache.samza.context.ExternalContext;
import org.apache.samza.context.JobContext;
import org.apache.samza.metrics.MetricsRegistryMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** Runtime context for the Samza runner. */
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SamzaExecutionContext implements ApplicationContainerContext {
private static final Logger LOG = LoggerFactory.getLogger(SamzaExecutionContext.class);
private static final String SAMZA_WORKER_ID = "samza_py_worker_id";

private final SamzaPipelineOptions options;
private SamzaMetricsContainer metricsContainer;
private JobBundleFactory jobBundleFactory;
private GrpcFnServer<FnApiControlClientPoolService> fnControlServer;
private GrpcFnServer<GrpcDataService> fnDataServer;
private GrpcFnServer<GrpcStateService> fnStateServer;
private ControlClientPool controlClientPool;
private ExecutorService dataExecutor;
private IdGenerator idGenerator = IdGenerators.incrementingLongs();

public SamzaExecutionContext(SamzaPipelineOptions options) {
this.options = options;
Expand All @@ -81,93 +50,11 @@ void setMetricsContainer(SamzaMetricsContainer metricsContainer) {
this.metricsContainer = metricsContainer;
}

public JobBundleFactory getJobBundleFactory() {
return this.jobBundleFactory;
}

void setJobBundleFactory(JobBundleFactory jobBundleFactory) {
this.jobBundleFactory = jobBundleFactory;
}

@Override
public void start() {
checkState(getJobBundleFactory() == null, "jobBundleFactory has been created!");

if (SamzaRunnerOverrideConfigs.isPortableMode(options)) {
try {
controlClientPool = MapControlClientPool.create();
dataExecutor = Executors.newCachedThreadPool();

fnControlServer =
GrpcFnServer.allocatePortAndCreateFor(
FnApiControlClientPoolService.offeringClientsToPool(
controlClientPool.getSink(), () -> SAMZA_WORKER_ID),
ServerFactory.createWithPortSupplier(
() -> SamzaRunnerOverrideConfigs.getFnControlPort(options)));
LOG.info("Started control server on port {}", fnControlServer.getServer().getPort());

fnDataServer =
GrpcFnServer.allocatePortAndCreateFor(
GrpcDataService.create(
options, dataExecutor, OutboundObserverFactory.serverDirect()),
ServerFactory.createDefault());
LOG.info("Started data server on port {}", fnDataServer.getServer().getPort());

fnStateServer =
GrpcFnServer.allocatePortAndCreateFor(
GrpcStateService.create(), ServerFactory.createDefault());
LOG.info("Started state server on port {}", fnStateServer.getServer().getPort());

final long waitTimeoutMs =
SamzaRunnerOverrideConfigs.getControlClientWaitTimeoutMs(options);
LOG.info("Control client wait timeout config: " + waitTimeoutMs);

final InstructionRequestHandler instructionHandler =
controlClientPool.getSource().take(SAMZA_WORKER_ID, Duration.ofMillis(waitTimeoutMs));
final EnvironmentFactory environmentFactory =
(environment, workerId) ->
RemoteEnvironment.forHandler(environment, instructionHandler);
// TODO: use JobBundleFactoryBase.WrappedSdkHarnessClient.wrapping
jobBundleFactory =
SingleEnvironmentInstanceJobBundleFactory.create(
environmentFactory, fnDataServer, fnStateServer, idGenerator);
LOG.info("Started job bundle factory");
} catch (Exception e) {
throw new RuntimeException(
"Running samza in Beam portable mode but failed to create job bundle factory", e);
}

setJobBundleFactory(jobBundleFactory);
}
}
public void start() {}

@Override
public void stop() {
closeAutoClosable(fnControlServer, "controlServer");
fnControlServer = null;
closeAutoClosable(fnDataServer, "dataServer");
fnDataServer = null;
closeAutoClosable(fnStateServer, "stateServer");
fnStateServer = null;
if (dataExecutor != null) {
dataExecutor.shutdown();
dataExecutor = null;
}
controlClientPool = null;
closeAutoClosable(jobBundleFactory, "jobBundle");
jobBundleFactory = null;
}

private static void closeAutoClosable(AutoCloseable closeable, String name) {
try (AutoCloseable closer = closeable) {
LOG.info("Closed {}", name);
} catch (Exception e) {
LOG.error(
"Failed to close {}. Ignore since this is shutdown process...",
closeable.getClass().getSimpleName(),
e);
}
}
public void stop() {}

/** The factory to return this {@link SamzaExecutionContext}. */
public class Factory implements ApplicationContainerContextFactory<SamzaExecutionContext> {
Expand Down
@@ -0,0 +1,82 @@
/*
* 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;

import java.util.UUID;
import javax.annotation.Nullable;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
import org.apache.beam.runners.jobsubmission.JobInvocation;
import org.apache.beam.runners.jobsubmission.JobInvoker;
import org.apache.beam.runners.jobsubmission.PortablePipelineJarCreator;
import org.apache.beam.runners.jobsubmission.PortablePipelineRunner;
import org.apache.beam.sdk.options.PortablePipelineOptions;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SamzaJobInvoker extends JobInvoker {

private static final Logger LOG = LoggerFactory.getLogger(SamzaJobInvoker.class);
private final SamzaJobServerDriver.SamzaServerConfiguration configuration;

public static SamzaJobInvoker create(
SamzaJobServerDriver.SamzaServerConfiguration configuration) {
return new SamzaJobInvoker(configuration);
}

private SamzaJobInvoker(SamzaJobServerDriver.SamzaServerConfiguration configuration) {
super("samza-runner-job-invoker-%d");
this.configuration = configuration;
}

@Override
protected JobInvocation invokeWithExecutor(
RunnerApi.Pipeline pipeline,
Struct options,
@Nullable String retrievalToken,
ListeningExecutorService executorService) {
LOG.trace("Parsing pipeline options");
final SamzaPortablePipelineOptions samzaOptions =
PipelineOptionsTranslation.fromProto(options).as(SamzaPortablePipelineOptions.class);

final PortablePipelineRunner pipelineRunner;
if (Strings.isNullOrEmpty(
samzaOptions.as(PortablePipelineOptions.class).getOutputExecutablePath())) {
pipelineRunner = new SamzaPipelineRunner(samzaOptions);
} else {
/*
* To support --output_executable_path where bundles the input pipeline along with all
* artifacts, etc. required to run the pipeline into a jar that can be executed later.
*/
pipelineRunner = new PortablePipelineJarCreator(SamzaPipelineRunner.class);
}

final String invocationId =
String.format("%s_%s", samzaOptions.getJobName(), UUID.randomUUID().toString());
final JobInfo jobInfo =
JobInfo.create(invocationId, samzaOptions.getJobName(), retrievalToken, options);
return new JobInvocation(jobInfo, executorService, pipeline, pipelineRunner);
}
}

0 comments on commit 9ab200c

Please sign in to comment.