|
79 | 79 | * |
80 | 80 | * <p>Example trigger command for specific test running on Dataflow runner: |
81 | 81 | * |
| 82 | + * <p><b>Maven</b> |
| 83 | + * |
82 | 84 | * <pre> |
83 | 85 | * mvn test -pl it/google-cloud-platform -am -Dtest="BigQueryIOLT#testAvroFileLoadsWriteThenRead" \ |
84 | 86 | * -Dconfiguration=medium -Dproject=[gcpProject] -DartifactBucket=[temp bucket] -DfailIfNoTests=false |
85 | 87 | * </pre> |
86 | 88 | * |
| 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 | + * |
87 | 96 | * <p>Example trigger command for specific test and custom data configuration: |
88 | 97 | * |
89 | 98 | * <pre>mvn test -pl it/google-cloud-platform -am \ |
@@ -172,11 +181,11 @@ public static void tearDownClass() { |
172 | 181 | Configuration.class), // 1 MB |
173 | 182 | "medium", |
174 | 183 | 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\"}", |
176 | 185 | Configuration.class), // 10 GB |
177 | 186 | "large", |
178 | 187 | 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}", |
180 | 189 | Configuration.class) // 100 GB |
181 | 190 | ); |
182 | 191 | } catch (IOException e) { |
@@ -230,16 +239,19 @@ public void testWriteAndRead() throws IOException { |
230 | 239 | writeIO = |
231 | 240 | BigQueryIO.<byte[]>write() |
232 | 241 | .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) |
| 242 | + .withNumStorageWriteApiStreams( |
| 243 | + configuration.numStorageWriteApiStreams) // control the number of streams |
233 | 244 | .withAvroFormatFunction( |
234 | 245 | new AvroFormatFn( |
235 | 246 | configuration.numColumns, |
236 | 247 | !("STORAGE_WRITE_API".equalsIgnoreCase(configuration.writeMethod)))); |
237 | | - |
238 | 248 | break; |
239 | 249 | case JSON: |
240 | 250 | writeIO = |
241 | 251 | BigQueryIO.<byte[]>write() |
242 | 252 | .withSuccessfulInsertsPropagation(false) |
| 253 | + .withNumStorageWriteApiStreams( |
| 254 | + configuration.numStorageWriteApiStreams) // control the number of streams |
243 | 255 | .withFormatFunction(new JsonFormatFn(configuration.numColumns)); |
244 | 256 | break; |
245 | 257 | } |
@@ -268,6 +280,10 @@ private void testWrite(BigQueryIO.Write<byte[]> writeIO) throws IOException { |
268 | 280 | .setSdk(PipelineLauncher.Sdk.JAVA) |
269 | 281 | .setPipeline(writePipeline) |
270 | 282 | .addParameter("runner", configuration.runner) |
| 283 | + .addParameter("workerMachineType", configuration.workerMachineType) |
| 284 | + .addParameter("experiments", configuration.experiments) |
| 285 | + .addParameter("numWorkers", configuration.numWorkers) |
| 286 | + .addParameter("maxNumWorkers", configuration.maxNumWorkers) |
271 | 287 | .build(); |
272 | 288 |
|
273 | 289 | PipelineLauncher.LaunchInfo launchInfo = pipelineLauncher.launch(project, region, options); |
@@ -304,6 +320,10 @@ private void testRead() throws IOException { |
304 | 320 | .setSdk(PipelineLauncher.Sdk.JAVA) |
305 | 321 | .setPipeline(readPipeline) |
306 | 322 | .addParameter("runner", configuration.runner) |
| 323 | + .addParameter("workerMachineType", configuration.workerMachineType) |
| 324 | + .addParameter("experiments", configuration.experiments) |
| 325 | + .addParameter("numWorkers", configuration.numWorkers) |
| 326 | + .addParameter("maxNumWorkers", configuration.maxNumWorkers) |
307 | 327 | .build(); |
308 | 328 |
|
309 | 329 | PipelineLauncher.LaunchInfo launchInfo = pipelineLauncher.launch(project, region, options); |
@@ -445,12 +465,36 @@ static class Configuration extends SyntheticSourceOptions { |
445 | 465 | /** Runner specified to run the pipeline. */ |
446 | 466 | @JsonProperty public String runner = "DirectRunner"; |
447 | 467 |
|
| 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 | + |
448 | 480 | /** BigQuery read method: DEFAULT/DIRECT_READ/EXPORT. */ |
449 | 481 | @JsonProperty public String readMethod = "DEFAULT"; |
450 | 482 |
|
451 | 483 | /** BigQuery write method: DEFAULT/FILE_LOADS/STREAMING_INSERTS/STORAGE_WRITE_API. */ |
452 | 484 | @JsonProperty public String writeMethod = "DEFAULT"; |
453 | 485 |
|
| 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 | + |
454 | 498 | /** BigQuery write format: AVRO/JSON. */ |
455 | 499 | @JsonProperty public String writeFormat = "AVRO"; |
456 | 500 | } |
|
0 commit comments