Skip to content

Commit 5944a30

Browse files
authored
BigQueryIO : control StorageWrite parallelism in batch, by reshuffling before write on the number of streams set for BigQueryIO.write() using .withNumStorageWriteApiStreams(numStorageWriteApiStreams) (#32805)
* BigQueryIO : control StorageWrite parallelism in batch, by reshuffling before write on the number of streams set for BigQueryIO.write() using .withNumStorageWriteApiStreams(numStorageWriteApiStreams) * fix unused dep and comment * spotlessApply * spotlessApply * fix typo
1 parent 067deed commit 5944a30

File tree

3 files changed

+68
-8
lines changed

3 files changed

+68
-8
lines changed

it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryIOLT.java

Lines changed: 47 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -79,11 +79,20 @@
7979
*
8080
* <p>Example trigger command for specific test running on Dataflow runner:
8181
*
82+
* <p><b>Maven</b>
83+
*
8284
* <pre>
8385
* mvn test -pl it/google-cloud-platform -am -Dtest="BigQueryIOLT#testAvroFileLoadsWriteThenRead" \
8486
* -Dconfiguration=medium -Dproject=[gcpProject] -DartifactBucket=[temp bucket] -DfailIfNoTests=false
8587
* </pre>
8688
*
89+
* <p><b>Gradle</b>
90+
*
91+
* <pre>
92+
* ./gradlew :it:google-cloud-platform:BigQueryPerformanceTest --tests='BigQueryIOLT.testAvroFileLoadsWriteThenRead' \
93+
* -Dconfiguration=medium -Dproject=[gcpProject] -DartifactBucket=[temp bucket] -DfailIfNoTests=false
94+
* </pre>
95+
*
8796
* <p>Example trigger command for specific test and custom data configuration:
8897
*
8998
* <pre>mvn test -pl it/google-cloud-platform -am \
@@ -172,11 +181,11 @@ public static void tearDownClass() {
172181
Configuration.class), // 1 MB
173182
"medium",
174183
Configuration.fromJsonString(
175-
"{\"numRecords\":10000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":20,\"runner\":\"DataflowRunner\"}",
184+
"{\"numRecords\":10000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":20,\"runner\":\"DataflowRunner\",\"workerMachineType\":\"e2-standard-2\",\"experiments\":\"disable_runner_v2\",\"numWorkers\":\"1\",\"maxNumWorkers\":\"1\"}",
176185
Configuration.class), // 10 GB
177186
"large",
178187
Configuration.fromJsonString(
179-
"{\"numRecords\":100000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":80,\"runner\":\"DataflowRunner\"}",
188+
"{\"numRecords\":100000000,\"valueSizeBytes\":1000,\"pipelineTimeout\":80,\"runner\":\"DataflowRunner\",\"workerMachineType\":\"e2-standard-2\",\"experiments\":\"disable_runner_v2\",\"numWorkers\":\"1\",\"maxNumWorkers\":\"1\",\"numStorageWriteApiStreams\":4,\"storageWriteApiTriggeringFrequencySec\":20}",
180189
Configuration.class) // 100 GB
181190
);
182191
} catch (IOException e) {
@@ -230,16 +239,19 @@ public void testWriteAndRead() throws IOException {
230239
writeIO =
231240
BigQueryIO.<byte[]>write()
232241
.withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)
242+
.withNumStorageWriteApiStreams(
243+
configuration.numStorageWriteApiStreams) // control the number of streams
233244
.withAvroFormatFunction(
234245
new AvroFormatFn(
235246
configuration.numColumns,
236247
!("STORAGE_WRITE_API".equalsIgnoreCase(configuration.writeMethod))));
237-
238248
break;
239249
case JSON:
240250
writeIO =
241251
BigQueryIO.<byte[]>write()
242252
.withSuccessfulInsertsPropagation(false)
253+
.withNumStorageWriteApiStreams(
254+
configuration.numStorageWriteApiStreams) // control the number of streams
243255
.withFormatFunction(new JsonFormatFn(configuration.numColumns));
244256
break;
245257
}
@@ -268,6 +280,10 @@ private void testWrite(BigQueryIO.Write<byte[]> writeIO) throws IOException {
268280
.setSdk(PipelineLauncher.Sdk.JAVA)
269281
.setPipeline(writePipeline)
270282
.addParameter("runner", configuration.runner)
283+
.addParameter("workerMachineType", configuration.workerMachineType)
284+
.addParameter("experiments", configuration.experiments)
285+
.addParameter("numWorkers", configuration.numWorkers)
286+
.addParameter("maxNumWorkers", configuration.maxNumWorkers)
271287
.build();
272288

273289
PipelineLauncher.LaunchInfo launchInfo = pipelineLauncher.launch(project, region, options);
@@ -304,6 +320,10 @@ private void testRead() throws IOException {
304320
.setSdk(PipelineLauncher.Sdk.JAVA)
305321
.setPipeline(readPipeline)
306322
.addParameter("runner", configuration.runner)
323+
.addParameter("workerMachineType", configuration.workerMachineType)
324+
.addParameter("experiments", configuration.experiments)
325+
.addParameter("numWorkers", configuration.numWorkers)
326+
.addParameter("maxNumWorkers", configuration.maxNumWorkers)
307327
.build();
308328

309329
PipelineLauncher.LaunchInfo launchInfo = pipelineLauncher.launch(project, region, options);
@@ -445,12 +465,36 @@ static class Configuration extends SyntheticSourceOptions {
445465
/** Runner specified to run the pipeline. */
446466
@JsonProperty public String runner = "DirectRunner";
447467

468+
/** Worker machine type specified to run the pipeline with Dataflow Runner. */
469+
@JsonProperty public String workerMachineType = "";
470+
471+
/** Experiments specified to run the pipeline. */
472+
@JsonProperty public String experiments = "";
473+
474+
/** Number of workers to start the pipeline. Must be a positive value. */
475+
@JsonProperty public String numWorkers = "1";
476+
477+
/** Maximum umber of workers for the pipeline. Must be a positive value. */
478+
@JsonProperty public String maxNumWorkers = "1";
479+
448480
/** BigQuery read method: DEFAULT/DIRECT_READ/EXPORT. */
449481
@JsonProperty public String readMethod = "DEFAULT";
450482

451483
/** BigQuery write method: DEFAULT/FILE_LOADS/STREAMING_INSERTS/STORAGE_WRITE_API. */
452484
@JsonProperty public String writeMethod = "DEFAULT";
453485

486+
/**
487+
* BigQuery number of streams for write method STORAGE_WRITE_API. 0 let's the runner determine
488+
* the number of streams. Remark : max limit for open connections per hour is 10K streams.
489+
*/
490+
@JsonProperty public int numStorageWriteApiStreams = 0;
491+
492+
/**
493+
* BigQuery triggering frequency in second in combination with the number of streams for write
494+
* method STORAGE_WRITE_API.
495+
*/
496+
@JsonProperty public int storageWriteApiTriggeringFrequencySec = 20;
497+
454498
/** BigQuery write format: AVRO/JSON. */
455499
@JsonProperty public String writeFormat = "AVRO";
456500
}

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3040,9 +3040,14 @@ public Write<T> withNumFileShards(int numFileShards) {
30403040
}
30413041

30423042
/**
3043-
* Control how many parallel streams are used when using Storage API writes. Applicable only for
3044-
* streaming pipelines, and when {@link #withTriggeringFrequency} is also set. To let runner
3045-
* determine the sharding at runtime, set this to zero, or {@link #withAutoSharding()} instead.
3043+
* Control how many parallel streams are used when using Storage API writes.
3044+
*
3045+
* <p>For streaming pipelines, and when {@link #withTriggeringFrequency} is also set. To let
3046+
* runner determine the sharding at runtime, set this to zero, or {@link #withAutoSharding()}
3047+
* instead.
3048+
*
3049+
* <p>For batch pipelines, it inserts a redistribute. To not reshufle and keep the pipeline
3050+
* parallelism as is, set this to zero.
30463051
*/
30473052
public Write<T> withNumStorageWriteApiStreams(int numStorageWriteApiStreams) {
30483053
return toBuilder().setNumStorageWriteApiStreams(numStorageWriteApiStreams).build();

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiLoads.java

Lines changed: 13 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import org.apache.beam.sdk.transforms.GroupIntoBatches;
3737
import org.apache.beam.sdk.transforms.PTransform;
3838
import org.apache.beam.sdk.transforms.ParDo;
39+
import org.apache.beam.sdk.transforms.Redistribute;
3940
import org.apache.beam.sdk.transforms.SerializableFunction;
4041
import org.apache.beam.sdk.transforms.errorhandling.BadRecord;
4142
import org.apache.beam.sdk.transforms.errorhandling.BadRecordRouter;
@@ -360,9 +361,19 @@ public WriteResult expandUntriggered(
360361
rowUpdateFn,
361362
badRecordRouter));
362363

364+
PCollection<KV<DestinationT, StorageApiWritePayload>> successfulConvertedRows =
365+
convertMessagesResult.get(successfulConvertedRowsTag);
366+
367+
if (numShards > 0) {
368+
successfulConvertedRows =
369+
successfulConvertedRows.apply(
370+
"ResdistibuteNumShards",
371+
Redistribute.<KV<DestinationT, StorageApiWritePayload>>arbitrarily()
372+
.withNumBuckets(numShards));
373+
}
374+
363375
PCollectionTuple writeRecordsResult =
364-
convertMessagesResult
365-
.get(successfulConvertedRowsTag)
376+
successfulConvertedRows
366377
.apply(
367378
"StorageApiWriteUnsharded",
368379
new StorageApiWriteUnshardedRecords<>(

0 commit comments

Comments
 (0)