Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.runners.TransformHierarchy;
Expand Down Expand Up @@ -121,10 +122,17 @@ public class DataflowPipelineTranslator {
private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineTranslator.class);
private static final ObjectMapper MAPPER = new ObjectMapper();

private static byte[] serializeWindowingStrategy(WindowingStrategy<?, ?> windowingStrategy) {
private static byte[] serializeWindowingStrategy(
WindowingStrategy<?, ?> windowingStrategy, PipelineOptions options) {
try {
SdkComponents sdkComponents = SdkComponents.create();
sdkComponents.registerEnvironment(Environments.JAVA_SDK_HARNESS_ENVIRONMENT);

String workerHarnessContainerImageURL =
DataflowRunner.getContainerImageForJob(options.as(DataflowPipelineOptions.class));
RunnerApi.Environment defaultEnvironmentForDataflow =
Environments.createDockerEnvironment(workerHarnessContainerImageURL);
sdkComponents.registerEnvironment(defaultEnvironmentForDataflow);

return WindowingStrategyTranslation.toMessageProto(windowingStrategy, sdkComponents)
.toByteArray();
} catch (Exception e) {
Expand Down Expand Up @@ -164,7 +172,13 @@ public JobSpecification translate(

// Capture the sdkComponents for look up during step translations
SdkComponents sdkComponents = SdkComponents.create();
sdkComponents.registerEnvironment(Environments.JAVA_SDK_HARNESS_ENVIRONMENT);

String workerHarnessContainerImageURL =
DataflowRunner.getContainerImageForJob(options.as(DataflowPipelineOptions.class));
RunnerApi.Environment defaultEnvironmentForDataflow =
Environments.createDockerEnvironment(workerHarnessContainerImageURL);
sdkComponents.registerEnvironment(defaultEnvironmentForDataflow);

RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(pipeline, sdkComponents, true);

LOG.debug("Portable pipeline proto:\n{}", TextFormat.printToString(pipelineProto));
Expand Down Expand Up @@ -754,7 +768,8 @@ private <ElemT, ViewT> void translateTyped(
WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
stepContext.addInput(
PropertyNames.WINDOWING_STRATEGY,
byteArrayToJsonString(serializeWindowingStrategy(windowingStrategy)));
byteArrayToJsonString(
serializeWindowingStrategy(windowingStrategy, context.getPipelineOptions())));
stepContext.addInput(
PropertyNames.IS_MERGING_WINDOW_FN,
!windowingStrategy.getWindowFn().isNonMerging());
Expand Down Expand Up @@ -898,7 +913,8 @@ private <K, V> void groupByKeyHelper(
stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, !allowCombinerLifting);
stepContext.addInput(
PropertyNames.SERIALIZED_FN,
byteArrayToJsonString(serializeWindowingStrategy(windowingStrategy)));
byteArrayToJsonString(
serializeWindowingStrategy(windowingStrategy, context.getPipelineOptions())));
stepContext.addInput(
PropertyNames.IS_MERGING_WINDOW_FN,
!windowingStrategy.getWindowFn().isNonMerging());
Expand Down Expand Up @@ -1039,7 +1055,8 @@ private <T> void translateHelper(Window.Assign<T> transform, TranslationContext
stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));

WindowingStrategy<?, ?> strategy = context.getOutput(transform).getWindowingStrategy();
byte[] serializedBytes = serializeWindowingStrategy(strategy);
byte[] serializedBytes =
serializeWindowingStrategy(strategy, context.getPipelineOptions());
String serializedJson = byteArrayToJsonString(serializedBytes);
stepContext.addInput(PropertyNames.SERIALIZED_FN, serializedJson);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,8 @@
import java.util.Set;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload;
import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
import org.apache.beam.runners.core.construction.PTransformTranslation;
import org.apache.beam.runners.core.construction.ParDoTranslation;
Expand Down Expand Up @@ -956,6 +958,34 @@ public void populateDisplayData(DisplayData.Builder builder) {
assertEquals(expectedFn2DisplayData, ImmutableSet.copyOf(fn2displayData));
}

/**
* Tests that when {@link DataflowPipelineOptions#setWorkerHarnessContainerImage(String)} pipeline
* option is set, {@link DataflowRunner} sets that value as the {@link
* DockerPayload#getContainerImage()} of the default {@link Environment} used when generating the
* model pipeline proto.
*/
@Test
public void testSetWorkerHarnessContainerImageInPipelineProto() throws Exception {
DataflowPipelineOptions options = buildPipelineOptions();
String containerImage = "gcr.io/IMAGE/foo";
options.as(DataflowPipelineOptions.class).setWorkerHarnessContainerImage(containerImage);

JobSpecification specification =
DataflowPipelineTranslator.fromOptions(options)
.translate(
Pipeline.create(options),
DataflowRunner.fromOptions(options),
Collections.emptyList());
RunnerApi.Pipeline pipelineProto = specification.getPipelineProto();

assertEquals(1, pipelineProto.getComponents().getEnvironmentsCount());
Environment defaultEnvironment =
Iterables.getOnlyElement(pipelineProto.getComponents().getEnvironmentsMap().values());

DockerPayload payload = DockerPayload.parseFrom(defaultEnvironment.getPayload());
assertEquals(DataflowRunner.getContainerImageForJob(options), payload.getContainerImage());
}

private static void assertAllStepOutputsHaveUniqueIds(Job job) throws Exception {
List<String> outputIds = new ArrayList<>();
for (Step step : job.getSteps()) {
Expand Down