diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 56bd14318be4..41fbf8e9dfee 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -485,7 +485,7 @@ public WriteResult expandUntriggered(PCollection> inp // loading. PCollectionTuple partitions = results - .apply("ReifyResults", new ReifyAsIterable<>()) + .apply("ReifyResults", new CombineAsIterable<>()) .setCoder(IterableCoder.of(WriteBundlesToFiles.ResultCoder.of(destinationCoder))) .apply( "WritePartitionUntriggered", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CombineAsIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CombineAsIterable.java new file mode 100644 index 000000000000..a537f70e9eea --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CombineAsIterable.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.gcp.bigquery; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class CombineAsIterable extends PTransform, PCollection>> { + @Override + public PCollection> expand(PCollection input) { + return input + .apply( + "assign single key", + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(@Element T element, OutputReceiver> o) { + o.output(KV.of("key", element)); + } + })) + .apply(GroupByKey.create()) + .apply(Values.create()); + } +}