Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[BEAM-12174] Samza Portable Runner Support (#14554)
- Loading branch information
Showing
12 changed files
with
226 additions
and
233 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
82 changes: 82 additions & 0 deletions
82
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvoker.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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); | ||
} | ||
} |
Oops, something went wrong.