-
Notifications
You must be signed in to change notification settings - Fork 4.6k
Fix/add experiment refactor #38252
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Fix/add experiment refactor #38252
Changes from all commits
24923ba
fedda96
f3e822c
13549dc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -1243,16 +1243,21 @@ private static boolean includesTransformUpgrades(Pipeline pipeline) { | |||||||||||||
| @SuppressWarnings("Slf4jFormatShouldBeConst") | ||||||||||||||
| @Override | ||||||||||||||
| public DataflowPipelineJob run(Pipeline pipeline) { | ||||||||||||||
| // Ensure the experiments list is mutable before any experiments are added. | ||||||||||||||
| if (options.getExperiments() != null) { | ||||||||||||||
| options.setExperiments(new ArrayList<>(options.getExperiments())); | ||||||||||||||
| } | ||||||||||||||
| // Multi-language pipelines and pipelines that include upgrades should automatically be upgraded | ||||||||||||||
|
Comment on lines
+1246
to
1250
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This pre-emptive copy of the experiments list to an ArrayList is redundant. ExperimentalOptions.addExperiment already handles null-initialization and copies the existing list into a new mutable ArrayList internally before adding any new experiments. You can safely remove this block.
Suggested change
|
||||||||||||||
| // to Dataflow Portable Runner. | ||||||||||||||
| if (DataflowRunner.isMultiLanguagePipeline(pipeline) || includesTransformUpgrades(pipeline)) { | ||||||||||||||
| if (!useUnifiedWorker(options)) { | ||||||||||||||
| List<String> experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); | ||||||||||||||
| LOG.info( | ||||||||||||||
| "Automatically enabling Dataflow Portable Runner since the pipeline used cross-language" | ||||||||||||||
| + " transforms or pipeline needed a transform upgrade."); | ||||||||||||||
| options.setExperiments( | ||||||||||||||
| ImmutableList.<String>builder().addAll(experiments).add("use_runner_v2").build()); | ||||||||||||||
| if (!firstNonNull(options.getExperiments(), Collections.emptyList()) | ||||||||||||||
| .contains("use_runner_v2")) { | ||||||||||||||
| LOG.info( | ||||||||||||||
| "Automatically enabling Dataflow Portable Runner since the pipeline used cross-language" | ||||||||||||||
| + " transforms or pipeline needed a transform upgrade."); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "use_runner_v2"); | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
| } | ||||||||||||||
| if (useUnifiedWorker(options)) { | ||||||||||||||
|
|
@@ -1264,21 +1269,10 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||||||
| throw new IllegalArgumentException( | ||||||||||||||
| "Dataflow Portable Runner both disabled and enabled: at least one of ['enable_portable_runner', 'beam_fn_api', 'use_unified_worker', 'use_runner_v2', 'use_portable_job_submission'] is set and also one of ['enable_streaming_java_runner', 'disable_portable_runner', 'disable_runner_v2', 'disable_runner_v2_until_2023', 'disable_prime_runner_v2'] is set."); | ||||||||||||||
| } | ||||||||||||||
| List<String> experiments = | ||||||||||||||
| new ArrayList<>(options.getExperiments()); // non-null if useUnifiedWorker is true | ||||||||||||||
| if (!experiments.contains("use_runner_v2")) { | ||||||||||||||
| experiments.add("use_runner_v2"); | ||||||||||||||
| } | ||||||||||||||
| if (!experiments.contains("use_unified_worker")) { | ||||||||||||||
| experiments.add("use_unified_worker"); | ||||||||||||||
| } | ||||||||||||||
| if (!experiments.contains("beam_fn_api")) { | ||||||||||||||
| experiments.add("beam_fn_api"); | ||||||||||||||
| } | ||||||||||||||
| if (!experiments.contains("use_portable_job_submission")) { | ||||||||||||||
| experiments.add("use_portable_job_submission"); | ||||||||||||||
| } | ||||||||||||||
| options.setExperiments(ImmutableList.copyOf(experiments)); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "use_runner_v2"); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "use_unified_worker"); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "beam_fn_api"); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "use_portable_job_submission"); | ||||||||||||||
| // Ensure that logging via the FnApi is enabled | ||||||||||||||
| options.as(SdkHarnessOptions.class).setEnableLogViaFnApi(true); | ||||||||||||||
| } | ||||||||||||||
|
|
@@ -1305,19 +1299,16 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||||||
| options.setStreaming(true); | ||||||||||||||
|
|
||||||||||||||
| { | ||||||||||||||
| List<String> experiments = | ||||||||||||||
| options.getExperiments() == null | ||||||||||||||
| ? new ArrayList<>() | ||||||||||||||
| : new ArrayList<>(options.getExperiments()); | ||||||||||||||
| // Experiment marking that the harness supports tag encoding v2 | ||||||||||||||
| // Backend will enable tag encoding v2 only if the harness supports it. | ||||||||||||||
| experiments.add("streaming_engine_state_tag_encoding_v2_supported"); | ||||||||||||||
| ExperimentalOptions.addExperiment( | ||||||||||||||
| options, "streaming_engine_state_tag_encoding_v2_supported"); | ||||||||||||||
| // Experiment requesting tag encoding v2 on new jobs starting with 2.75.0. During job | ||||||||||||||
| // updates old job's tag encoding version is carried over by the backend. | ||||||||||||||
| if (!StreamingOptions.updateCompatibilityVersionLessThan(options, "2.75.0")) { | ||||||||||||||
| experiments.add("enable_streaming_engine_state_tag_encoding_v2"); | ||||||||||||||
| ExperimentalOptions.addExperiment( | ||||||||||||||
| options, "enable_streaming_engine_state_tag_encoding_v2"); | ||||||||||||||
| } | ||||||||||||||
| options.setExperiments(ImmutableList.copyOf(experiments)); | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| if (useUnifiedWorker(options)) { | ||||||||||||||
|
|
@@ -1443,15 +1434,7 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||||||
| pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); | ||||||||||||||
|
|
||||||||||||||
| if (!isNullOrEmpty(dataflowOptions.getDataflowWorkerJar()) && !useUnifiedWorker(options)) { | ||||||||||||||
| List<String> experiments = | ||||||||||||||
| firstNonNull(dataflowOptions.getExperiments(), Collections.emptyList()); | ||||||||||||||
| if (!experiments.contains("use_staged_dataflow_worker_jar")) { | ||||||||||||||
| dataflowOptions.setExperiments( | ||||||||||||||
| ImmutableList.<String>builder() | ||||||||||||||
| .addAll(experiments) | ||||||||||||||
| .add("use_staged_dataflow_worker_jar") | ||||||||||||||
| .build()); | ||||||||||||||
| } | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "use_staged_dataflow_worker_jar"); | ||||||||||||||
| } | ||||||||||||||
|
|
||||||||||||||
| Job newJob = jobSpecification.getJob(); | ||||||||||||||
|
|
@@ -1510,11 +1493,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||||||
| .collect(Collectors.toList()); | ||||||||||||||
|
|
||||||||||||||
| if (minCpuFlags.isEmpty()) { | ||||||||||||||
| dataflowOptions.setExperiments( | ||||||||||||||
| ImmutableList.<String>builder() | ||||||||||||||
| .addAll(experiments) | ||||||||||||||
| .add("min_cpu_platform=" + dataflowOptions.getMinCpuPlatform()) | ||||||||||||||
| .build()); | ||||||||||||||
| ExperimentalOptions.addExperiment( | ||||||||||||||
| dataflowOptions, "min_cpu_platform=" + dataflowOptions.getMinCpuPlatform()); | ||||||||||||||
| } else { | ||||||||||||||
| LOG.warn( | ||||||||||||||
| "Flag min_cpu_platform is defined in both top level PipelineOption, " | ||||||||||||||
|
|
@@ -1550,12 +1530,8 @@ public DataflowPipelineJob run(Pipeline pipeline) { | |||||||||||||
| // enable upload_graph when the graph is too large | ||||||||||||||
| byte[] jobGraphBytes = DataflowPipelineTranslator.jobToString(newJob).getBytes(UTF_8); | ||||||||||||||
| int jobGraphByteSize = jobGraphBytes.length; | ||||||||||||||
| if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES | ||||||||||||||
| && !hasExperiment(options, "upload_graph") | ||||||||||||||
| && !useUnifiedWorker(options)) { | ||||||||||||||
| List<String> experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); | ||||||||||||||
| options.setExperiments( | ||||||||||||||
| ImmutableList.<String>builder().addAll(experiments).add("upload_graph").build()); | ||||||||||||||
| if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES && !useUnifiedWorker(options)) { | ||||||||||||||
| ExperimentalOptions.addExperiment(options, "upload_graph"); | ||||||||||||||
| LOG.info( | ||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: looks like this log line will now be printed regardless even if the experiment was already enabled. |
||||||||||||||
| "The job graph size ({} in bytes) is larger than {}. Automatically add " | ||||||||||||||
| + "the upload_graph option to experiments.", | ||||||||||||||
|
|
||||||||||||||
| Original file line number | Diff line number | Diff line change | ||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -55,6 +55,7 @@ | |||||||||||||
| import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillStubFactoryFactory; | ||||||||||||||
| import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; | ||||||||||||||
| import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; | ||||||||||||||
| import org.apache.beam.sdk.options.ExperimentalOptions; | ||||||||||||||
| import org.apache.beam.sdk.options.PipelineOptionsFactory; | ||||||||||||||
| import org.apache.beam.sdk.util.BackOff; | ||||||||||||||
| import org.apache.beam.sdk.util.BackOffUtils; | ||||||||||||||
|
|
@@ -110,16 +111,20 @@ private static DataflowWorkerHarnessOptions testOptions( | |||||||||||||
| boolean enableStreamingEngine, List<String> additionalExperiments) { | ||||||||||||||
| DataflowWorkerHarnessOptions options = | ||||||||||||||
| PipelineOptionsFactory.create().as(DataflowWorkerHarnessOptions.class); | ||||||||||||||
| // Ensure the experiments list is mutable before any experiments are added. | ||||||||||||||
| if (options.getExperiments() != null) { | ||||||||||||||
| options.setExperiments(new ArrayList<>(options.getExperiments())); | ||||||||||||||
| } | ||||||||||||||
| options.setProject("project"); | ||||||||||||||
|
Comment on lines
+114
to
118
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This pre-emptive copy of the experiments list to an ArrayList is redundant. ExperimentalOptions.addExperiment already handles null-initialization and copies the existing list into a new mutable ArrayList internally before adding any new experiments. You can safely remove this block.
Suggested change
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. here and in other places: why was this copy necessary? |
||||||||||||||
| options.setJobId("job"); | ||||||||||||||
| options.setWorkerId("worker"); | ||||||||||||||
| List<String> experiments = | ||||||||||||||
| options.getExperiments() == null ? new ArrayList<>() : options.getExperiments(); | ||||||||||||||
|
|
||||||||||||||
| if (enableStreamingEngine) { | ||||||||||||||
| experiments.add(GcpOptions.STREAMING_ENGINE_EXPERIMENT); | ||||||||||||||
| ExperimentalOptions.addExperiment(options, GcpOptions.STREAMING_ENGINE_EXPERIMENT); | ||||||||||||||
| } | ||||||||||||||
| for (String experiment : additionalExperiments) { | ||||||||||||||
| ExperimentalOptions.addExperiment(options, experiment); | ||||||||||||||
| } | ||||||||||||||
| experiments.addAll(additionalExperiments); | ||||||||||||||
| options.setExperiments(experiments); | ||||||||||||||
|
|
||||||||||||||
| options.setWindmillServiceStreamingRpcBatchLimit(Integer.MAX_VALUE); | ||||||||||||||
| options.setWindmillServiceStreamingRpcHealthCheckPeriodMs(NO_HEALTH_CHECK); | ||||||||||||||
|
|
||||||||||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This pre-emptive copy of the experiments list to an ArrayList is redundant. ExperimentalOptions.addExperiment already handles null-initialization and copies the existing list into a new mutable ArrayList internally before adding any new experiments. You can safely remove this block.