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 @@ -1244,8 +1244,8 @@ public DataflowPipelineJob run(Pipeline pipeline) {
// Multi-language pipelines and pipelines that include upgrades should automatically be upgraded
// to Runner v2.
if (DataflowRunner.isMultiLanguagePipeline(pipeline) || includesTransformUpgrades(pipeline)) {
List<String> experiments = firstNonNull(options.getExperiments(), Collections.emptyList());
if (!experiments.contains("use_runner_v2")) {
if (!useUnifiedWorker(options)) {
Comment thread
TongruiLi marked this conversation as resolved.
List<String> experiments = firstNonNull(options.getExperiments(), Collections.emptyList());
LOG.info(
"Automatically enabling Dataflow Runner v2 since the pipeline used cross-language"
+ " transforms or pipeline needed a transform upgrade.");
Expand All @@ -1256,7 +1256,9 @@ public DataflowPipelineJob run(Pipeline pipeline) {
if (useUnifiedWorker(options)) {
if (hasExperiment(options, "disable_runner_v2")
|| hasExperiment(options, "disable_runner_v2_until_2023")
|| hasExperiment(options, "disable_prime_runner_v2")) {
|| hasExperiment(options, "disable_prime_runner_v2")
|| hasExperiment(options, "disable_portable_runner")
|| hasExperiment(options, "enable_streaming_java_runner")) {
Comment thread
TongruiLi marked this conversation as resolved.
throw new IllegalArgumentException(
"Runner V2 both disabled and enabled: at least one of ['beam_fn_api', 'use_unified_worker', 'use_runner_v2', 'use_portable_job_submission'] is set and also one of ['disable_runner_v2', 'disable_runner_v2_until_2023', 'disable_prime_runner_v2'] is set.");
Comment thread
TongruiLi marked this conversation as resolved.
}
Expand Down Expand Up @@ -2729,7 +2731,8 @@ static boolean useUnifiedWorker(DataflowPipelineOptions options) {
return hasExperiment(options, "beam_fn_api")
|| hasExperiment(options, "use_runner_v2")
|| hasExperiment(options, "use_unified_worker")
|| hasExperiment(options, "use_portable_job_submission");
|| hasExperiment(options, "use_portable_job_submission")
|| hasExperiment(options, "enable_portable_runner");
}

static void verifyDoFnSupported(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1783,7 +1783,11 @@ public void testSdkHarnessConfigurationPrime() throws IOException {
public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception {
for (String experiment :
ImmutableList.of(
"beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) {
"beam_fn_api",
"use_runner_v2",
"use_unified_worker",
"use_portable_job_submission",
"enable_portable_runner")) {
DataflowPipelineOptions options = buildPipelineOptions();
ExperimentalOptions.addExperiment(options, experiment);
Pipeline p = Pipeline.create(options);
Expand All @@ -1798,7 +1802,11 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception

for (String experiment :
ImmutableList.of(
"beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) {
"beam_fn_api",
"use_runner_v2",
"use_unified_worker",
"use_portable_job_submission",
"enable_portable_runner")) {
DataflowPipelineOptions options = buildPipelineOptions();
options.setStreaming(true);
ExperimentalOptions.addExperiment(options, experiment);
Expand All @@ -1822,10 +1830,18 @@ public void testSettingAnyFnApiExperimentEnablesUnifiedWorker() throws Exception
public void testSettingConflictingEnableAndDisableExperimentsThrowsException() throws Exception {
for (String experiment :
ImmutableList.of(
"beam_fn_api", "use_runner_v2", "use_unified_worker", "use_portable_job_submission")) {
"beam_fn_api",
"use_runner_v2",
"use_unified_worker",
"use_portable_job_submission",
"enable_portable_runner")) {
for (String disabledExperiment :
ImmutableList.of(
"disable_runner_v2", "disable_runner_v2_until_2023", "disable_prime_runner_v2")) {
"disable_runner_v2",
"disable_runner_v2_until_2023",
"disable_prime_runner_v2",
"enable_streaming_java_runner",
"disable_portable_runner")) {
DataflowPipelineOptions options = buildPipelineOptions();
ExperimentalOptions.addExperiment(options, experiment);
ExperimentalOptions.addExperiment(options, disabledExperiment);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -625,7 +625,7 @@ public void testThreadsAreAddedOnlyAsNeededWithContention() throws Exception {
LOG.info("Created {} threads to execute at most 100 parallel tasks", largestPool);
// Ideally we would never create more than 100, however with contention it is still possible
// some extra threads will be created.
assertTrue(largestPool <= 110);
assertTrue(largestPool <= 120);
executorService.shutdown();
}
}
Loading