From 47daf136d8f98555886119ba0f2b3885422c5844 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bal=C3=A1zs=20N=C3=A9meth?= Date: Thu, 30 Nov 2023 05:14:06 +0100 Subject: [PATCH] Fix UnsupportedOperationException because of ImmutableCollection.add --- .../org/apache/beam/runners/dataflow/DataflowRunner.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index d2b10f91c064..0d9328fe136d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -1339,9 +1339,9 @@ public DataflowPipelineJob run(Pipeline pipeline) { int jobGraphByteSize = jobGraphBytes.length; if (jobGraphByteSize >= CREATE_JOB_REQUEST_LIMIT_BYTES && !hasExperiment(options, "upload_graph")) { - List experiments = firstNonNull(options.getExperiments(), new ArrayList<>()); - experiments.add("upload_graph"); - options.setExperiments(ImmutableList.copyOf(experiments)); + List experiments = firstNonNull(options.getExperiments(), Collections.emptyList()); + options.setExperiments( + ImmutableList.builder().addAll(experiments).add("upload_graph").build()); LOG.info( "The job graph size ({} in bytes) is larger than {}. Automatically add " + "the upload_graph option to experiments.",