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 @@ -369,7 +369,7 @@ class BeamModulePlugin implements Plugin<Project> {
google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev402-$google_clients_version",
google_api_services_clouddebugger : "com.google.apis:google-api-services-clouddebugger:v2-rev253-$google_clients_version",
google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev502-$google_clients_version",
google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev257-$google_clients_version",
google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev266-$google_clients_version",
google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev399-$google_clients_version",
google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev136-$google_clients_version",
google_auth_library_credentials : "com.google.auth:google-auth-library-credentials:$google_auth_version",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
@VisibleForTesting static final int GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT = 1024 * 1024;

@VisibleForTesting static final String PIPELINE_FILE_NAME = "pipeline.pb";
@VisibleForTesting static final String DATAFLOW_GRAPH_FILE_NAME = "dataflow_graph.pb";

private static final ObjectMapper MAPPER = new ObjectMapper();

Expand Down Expand Up @@ -860,6 +861,20 @@ public DataflowPipelineJob run(Pipeline pipeline) {
newJob.setTransformNameMapping(options.getTransformNameMapping());
newJob.setReplaceJobId(jobIdToUpdate);
}

// Upload the job to GCS and remove the graph object from the API call. The graph
// will be downloaded from GCS by the service.
if (hasExperiment(options, "upload_graph")) {
DataflowPackage stagedGraph =
options
.getStager()
.stageToFile(
DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8),
DATAFLOW_GRAPH_FILE_NAME);
newJob.getSteps().clear();
newJob.setStepsLocation(stagedGraph.getLocation());
}

Job jobResult;
try {
jobResult = dataflowClient.createJob(newJob);
Expand All @@ -869,9 +884,9 @@ public DataflowPipelineJob run(Pipeline pipeline) {
if (Utf8.encodedLength(newJob.toString()) >= CREATE_JOB_REQUEST_LIMIT_BYTES) {
errorMessages =
"The size of the serialized JSON representation of the pipeline "
+ "exceeds the allowable limit. "
+ "For more information, please check the FAQ link below:\n"
+ "https://cloud.google.com/dataflow/faq";
+ "exceeds the allowable limit for the API. Use experiment "
+ "'upload_graph' (--experiments=upload_graph) to direct the runner to "
+ "upload the JSON to your GCS staging bucket instead of embedding in the API request.";
} else {
errorMessages = e.getDetails().getMessage();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -602,6 +602,24 @@ public void testUpdate() throws IOException {
assertValidJob(jobCaptor.getValue());
}

@Test
public void testUploadGraph() throws IOException {
DataflowPipelineOptions options = buildPipelineOptions();
options.setExperiments(Arrays.asList("upload_graph"));
Pipeline p = buildDataflowPipeline(options);
DataflowPipelineJob job = (DataflowPipelineJob) p.run();

ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
Mockito.verify(mockJobs).create(eq(PROJECT_ID), eq(REGION_ID), jobCaptor.capture());
assertValidJob(jobCaptor.getValue());
assertTrue(jobCaptor.getValue().getSteps().isEmpty());
assertTrue(
jobCaptor
.getValue()
.getStepsLocation()
.startsWith("gs://valid-bucket/temp/staging/dataflow_graph"));
}

@Test
public void testUpdateNonExistentPipeline() throws IOException {
thrown.expect(IllegalArgumentException.class);
Expand Down