/
BigQueryIO.java
3020 lines (2667 loc) · 131 KB
/
BigQueryIO.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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 static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.resolveTempLocation;
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryResourceNaming.createTempTableReference;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
import com.google.api.client.json.JsonFactory;
import com.google.api.services.bigquery.model.Clustering;
import com.google.api.services.bigquery.model.Job;
import com.google.api.services.bigquery.model.JobConfigurationQuery;
import com.google.api.services.bigquery.model.JobReference;
import com.google.api.services.bigquery.model.JobStatistics;
import com.google.api.services.bigquery.model.Table;
import com.google.api.services.bigquery.model.TableCell;
import com.google.api.services.bigquery.model.TableFieldSchema;
import com.google.api.services.bigquery.model.TableReference;
import com.google.api.services.bigquery.model.TableRow;
import com.google.api.services.bigquery.model.TableSchema;
import com.google.api.services.bigquery.model.TimePartitioning;
import com.google.auto.value.AutoValue;
import com.google.cloud.bigquery.storage.v1.CreateReadSessionRequest;
import com.google.cloud.bigquery.storage.v1.DataFormat;
import com.google.cloud.bigquery.storage.v1.ReadSession;
import com.google.cloud.bigquery.storage.v1.ReadStream;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.PipelineRunner;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.extensions.gcp.util.Transport;
import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath;
import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.fs.MoveOptions;
import org.apache.beam.sdk.io.fs.ResolveOptions;
import org.apache.beam.sdk.io.fs.ResourceId;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonTableRefToTableRef;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToJson;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TimePartitioningToJson;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead.Method;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryResourceNaming.JobType;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient;
import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.ExtractResult;
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantSchemaDestinations;
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningDestinations;
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.SchemaFromViewDestinations;
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.TableFunctionDestinations;
import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation;
import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.ContextContainer;
import org.apache.beam.sdk.io.gcp.bigquery.RowWriterFactory.OutputType;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SerializableFunctions;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Predicates;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* {@link PTransform}s for reading and writing <a
* href="https://developers.google.com/bigquery/">BigQuery</a> tables.
*
* <h3>Table References</h3>
*
* <p>A fully-qualified BigQuery table name consists of three components:
*
* <ul>
* <li>{@code projectId}: the Cloud project id (defaults to {@link GcpOptions#getProject()}).
* <li>{@code datasetId}: the BigQuery dataset id, unique within a project.
* <li>{@code tableId}: a table id, unique within a dataset.
* </ul>
*
* <p>BigQuery table references are stored as a {@link TableReference}, which comes from the <a
* href="https://cloud.google.com/bigquery/client-libraries">BigQuery Java Client API</a>. Tables
* can be referred to as Strings, with or without the {@code projectId}. A helper function is
* provided ({@link BigQueryHelpers#parseTableSpec(String)}) that parses the following string forms
* into a {@link TableReference}:
*
* <ul>
* <li>[{@code project_id}]:[{@code dataset_id}].[{@code table_id}]
* <li>[{@code dataset_id}].[{@code table_id}]
* </ul>
*
* <h3>BigQuery Concepts</h3>
*
* <p>Tables have rows ({@link TableRow}) and each row has cells ({@link TableCell}). A table has a
* schema ({@link TableSchema}), which in turn describes the schema of each cell ({@link
* TableFieldSchema}). The terms field and cell are used interchangeably.
*
* <p>{@link TableSchema}: describes the schema (types and order) for values in each row. It has one
* attribute, 'fields', which is list of {@link TableFieldSchema} objects.
*
* <p>{@link TableFieldSchema}: describes the schema (type, name) for one field. It has several
* attributes, including 'name' and 'type'. Common values for the type attribute are: 'STRING',
* 'INTEGER', 'FLOAT', 'BOOLEAN', 'NUMERIC', 'GEOGRAPHY'. All possible values are described at: <a
* href="https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types">
* https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types</a>
*
* <p>{@link TableRow}: Holds all values in a table row. Has one attribute, 'f', which is a list of
* {@link TableCell} instances.
*
* <p>{@link TableCell}: Holds the value for one cell (or field). Has one attribute, 'v', which is
* the value of the table cell.
*
* <p>As of Beam 2.7.0, the NUMERIC data type is supported. This data type supports high-precision
* decimal numbers (precision of 38 digits, scale of 9 digits). The GEOGRAPHY data type works with
* Well-Known Text (See <a href="https://en.wikipedia.org/wiki/Well-known_text">
* https://en.wikipedia.org/wiki/Well-known_text</a>) format for reading and writing to BigQuery.
* BigQuery IO requires values of BYTES datatype to be encoded using base64 encoding when writing to
* BigQuery. When bytes are read from BigQuery they are returned as base64-encoded strings.
*
* <h3>Reading</h3>
*
* <p>Reading from BigQuery is supported by {@link #read(SerializableFunction)}, which parses
* records in <a href="https://cloud.google.com/bigquery/data-formats#avro_format">AVRO format</a>
* into a custom type (see the table below for type conversion) using a specified parse function,
* and by {@link #readTableRows} which parses them into {@link TableRow}, which may be more
* convenient but has lower performance.
*
* <p>Both functions support reading either from a table or from the result of a query, via {@link
* TypedRead#from(String)} and {@link TypedRead#fromQuery} respectively. Exactly one of these must
* be specified.
*
* <p>If you are reading from an authorized view wih {@link TypedRead#fromQuery}, you need to use
* {@link TypedRead#withQueryLocation(String)} to set the location of the BigQuery job. Otherwise,
* Beam will ty to determine that location by reading the metadata of the dataset that contains the
* underlying tables. With authorized views, that will result in a 403 error and the query will not
* be resolved.
*
* <p><b>Type Conversion Table</b>
*
* <table border="1" cellspacing="1">
* <tr>
* <td> <b>BigQuery standard SQL type</b> </td> <td> <b>Avro type</b> </td> <td> <b>Java type</b> </td>
* </tr>
* <tr>
* <td> BOOLEAN </td> <td> boolean </td> <td> Boolean </td>
* </tr>
* <tr>
* <td> INT64 </td> <td> long </td> <td> Long </td>
* </tr>
* <tr>
* <td> FLOAT64 </td> <td> double </td> <td> Double </td>
* </tr>
* <tr>
* <td> BYTES </td> <td> bytes </td> <td> java.nio.ByteBuffer </td>
* </tr>
* <tr>
* <td> STRING </td> <td> string </td> <td> CharSequence </td>
* </tr>
* <tr>
* <td> DATE </td> <td> int </td> <td> Integer </td>
* </tr>
* <tr>
* <td> DATETIME </td> <td> string </td> <td> CharSequence </td>
* </tr>
* <tr>
* <td> TIMESTAMP </td> <td> long </td> <td> Long </td>
* </tr>
* <tr>
* <td> TIME </td> <td> long </td> <td> Long </td>
* </tr>
* <tr>
* <td> NUMERIC </td> <td> bytes </td> <td> java.nio.ByteBuffer </td>
* </tr>
* <tr>
* <td> GEOGRAPHY </td> <td> string </td> <td> CharSequence </td>
* </tr>
* <tr>
* <td> ARRAY </td> <td> array </td> <td> java.util.Collection </td>
* </tr>
* <tr>
* <td> STRUCT </td> <td> record </td> <td> org.apache.avro.generic.GenericRecord </td>
* </tr>
* </table>
*
* <p><b>Example: Reading rows of a table as {@link TableRow}.</b>
*
* <pre>{@code
* PCollection<TableRow> weatherData = pipeline.apply(
* BigQueryIO.readTableRows().from("clouddataflow-readonly:samples.weather_stations"));
* }</pre>
*
* <b>Example: Reading rows of a table and parsing them into a custom type.</b>
*
* <pre>{@code
* PCollection<WeatherRecord> weatherData = pipeline.apply(
* BigQueryIO
* .read(new SerializableFunction<SchemaAndRecord, WeatherRecord>() {
* public WeatherRecord apply(SchemaAndRecord schemaAndRecord) {
* return new WeatherRecord(...);
* }
* })
* .from("clouddataflow-readonly:samples.weather_stations"))
* .withCoder(SerializableCoder.of(WeatherRecord.class));
* }</pre>
*
* <p>Note: When using {@link #read(SerializableFunction)}, you may sometimes need to use {@link
* TypedRead#withCoder(Coder)} to specify a {@link Coder} for the result type, if Beam fails to
* infer it automatically.
*
* <p><b>Example: Reading results of a query as {@link TableRow}.</b>
*
* <pre>{@code
* PCollection<TableRow> meanTemperatureData = pipeline.apply(BigQueryIO.readTableRows()
* .fromQuery("SELECT year, mean_temp FROM [samples.weather_stations]"));
* }</pre>
*
* <p>Users can optionally specify a query priority using {@link
* TypedRead#withQueryPriority(TypedRead.QueryPriority)} and a geographic location where the query
* will be executed using {@link TypedRead#withQueryLocation(String)}. Query location must be
* specified for jobs that are not executed in US or EU, or if you are reading from an authorized
* view. See <a href="https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs/query">BigQuery
* Jobs: query</a>.
*
* <h3>Writing</h3>
*
* <p>To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. This consumes a
* {@link PCollection} of a user-defined type when using {@link BigQueryIO#write()} (recommended),
* or a {@link PCollection} of {@link TableRow TableRows} as input when using {@link
* BigQueryIO#writeTableRows()} (not recommended). When using a user-defined type, one of the
* following must be provided.
*
* <ul>
* <li>{@link BigQueryIO.Write#withAvroFormatFunction(SerializableFunction)} (recommended) to
* write data using avro records.
* <li>{@link BigQueryIO.Write#withAvroWriter} to write avro data using a user-specified {@link
* DatumWriter} (and format function).
* <li>{@link BigQueryIO.Write#withFormatFunction(SerializableFunction)} to write data as json
* encoded {@link TableRow TableRows}.
* </ul>
*
* If {@link BigQueryIO.Write#withAvroFormatFunction(SerializableFunction)} or {@link
* BigQueryIO.Write#withAvroWriter} is used, the table schema MUST be specified using one of the
* {@link Write#withJsonSchema(String)}, {@link Write#withJsonSchema(ValueProvider)}, {@link
* Write#withSchemaFromView(PCollectionView)} methods, or {@link Write#to(DynamicDestinations)}.
*
* <pre>{@code
* class Quote {
* final Instant timestamp;
* final String exchange;
* final String symbol;
* final double price;
*
* Quote(Instant timestamp, String exchange, String symbol, double price) {
* // initialize all member variables.
* }
* }
*
* PCollection<Quote> quotes = ...
*
* quotes.apply(BigQueryIO
* .<Quote>write()
* .to("my-project:my_dataset.my_table")
* .withSchema(new TableSchema().setFields(
* ImmutableList.of(
* new TableFieldSchema().setName("timestamp").setType("TIMESTAMP"),
* new TableFieldSchema().setName("exchange").setType("STRING"),
* new TableFieldSchema().setName("symbol").setType("STRING"),
* new TableFieldSchema().setName("price").setType("FLOAT"))))
* .withFormatFunction(quote -> new TableRow().set(..set the columns..))
* .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
* }</pre>
*
* <p>See {@link BigQueryIO.Write} for details on how to specify if a write should append to an
* existing table, replace the table, or verify that the table is empty. Note that the dataset being
* written to must already exist. Unbounded PCollections can only be written using {@link
* Write.WriteDisposition#WRITE_EMPTY} or {@link Write.WriteDisposition#WRITE_APPEND}.
*
* <p>BigQueryIO supports automatically inferring the BigQuery table schema from the Beam schema on
* the input PCollection. Beam can also automatically format the input into a TableRow in this case,
* if no format function is provide. In the above example, the quotes PCollection has a schema that
* Beam infers from the Quote POJO. So the write could be done more simply as follows:
*
* <pre>{@code
* {@literal @}DefaultSchema(JavaFieldSchema.class)
* class Quote {
* final Instant timestamp;
* final String exchange;
* final String symbol;
* final double price;
*
* {@literal @}SchemaCreate
* Quote(Instant timestamp, String exchange, String symbol, double price) {
* // initialize all member variables.
* }
* }
*
* PCollection<Quote> quotes = ...
*
* quotes.apply(BigQueryIO
* .<Quote>write()
* .to("my-project:my_dataset.my_table")
* .useBeamSchema()
* .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
* }</pre>
*
* <h3>Loading historical data into time-partitioned BigQuery tables</h3>
*
* <p>To load historical data into a time-partitioned BigQuery table, specify {@link
* BigQueryIO.Write#withTimePartitioning} with a {@link TimePartitioning#setField(String) field}
* used for <a
* href="https://cloud.google.com/bigquery/docs/partitioned-tables#partitioned_tables">column-based
* partitioning</a>. For example:
*
* <pre>{@code
* PCollection<Quote> quotes = ...;
*
* quotes.apply(BigQueryIO.write()
* .withSchema(schema)
* .withFormatFunction(quote -> new TableRow()
* .set("timestamp", quote.getTimestamp())
* .set(..other columns..))
* .to("my-project:my_dataset.my_table")
* .withTimePartitioning(new TimePartitioning().setField("time")));
* }</pre>
*
* <h3>Writing different values to different tables</h3>
*
* <p>A common use case is to dynamically generate BigQuery table names based on the current value.
* To support this, {@link BigQueryIO.Write#to(SerializableFunction)} accepts a function mapping the
* current element to a tablespec. For example, here's code that outputs quotes of different stocks
* to different tables:
*
* <pre>{@code
* PCollection<Quote> quotes = ...;
*
* quotes.apply(BigQueryIO.write()
* .withSchema(schema)
* .withFormatFunction(quote -> new TableRow()...)
* .to((ValueInSingleWindow<Quote> quote) -> {
* String symbol = quote.getSymbol();
* return new TableDestination(
* "my-project:my_dataset.quotes_" + symbol, // Table spec
* "Quotes of stock " + symbol // Table description
* );
* });
* }</pre>
*
* <p>Per-table schemas can also be provided using {@link BigQueryIO.Write#withSchemaFromView}. This
* allows you the schemas to be calculated based on a previous pipeline stage or statically via a
* {@link org.apache.beam.sdk.transforms.Create} transform. This method expects to receive a
* map-valued {@link PCollectionView}, mapping table specifications (project:dataset.table-id), to
* JSON formatted {@link TableSchema} objects. All destination tables must be present in this map,
* or the pipeline will fail to create tables. Care should be taken if the map value is based on a
* triggered aggregation over and unbounded {@link PCollection}; the side input will contain the
* entire history of all table schemas ever generated, which might blow up memory usage. This method
* can also be useful when writing to a single table, as it allows a previous stage to calculate the
* schema (possibly based on the full collection of records being written to BigQuery).
*
* <p>For the most general form of dynamic table destinations and schemas, look at {@link
* BigQueryIO.Write#to(DynamicDestinations)}.
*
* <h3>Insertion Method</h3>
*
* {@link BigQueryIO.Write} supports two methods of inserting data into BigQuery specified using
* {@link BigQueryIO.Write#withMethod}. If no method is supplied, then a default method will be
* chosen based on the input PCollection. See {@link BigQueryIO.Write.Method} for more information
* about the methods. The different insertion methods provide different tradeoffs of cost, quota,
* and data consistency; please see BigQuery documentation for more information about these
* tradeoffs.
*
* <h3>Usage with templates</h3>
*
* <p>When using {@link #read} or {@link #readTableRows()} in a template, it's required to specify
* {@link Read#withTemplateCompatibility()}. Specifying this in a non-template pipeline is not
* recommended because it has somewhat lower performance.
*
* <p>When using {@link #write()} or {@link #writeTableRows()} with batch loads in a template, it is
* recommended to specify {@link Write#withCustomGcsTempLocation}. Writing to BigQuery via batch
* loads involves writing temporary files to this location, so the location must be accessible at
* pipeline execution time. By default, this location is captured at pipeline <i>construction</i>
* time, may be inaccessible if the template may be reused from a different project or at a moment
* when the original location no longer exists. {@link
* Write#withCustomGcsTempLocation(ValueProvider)} allows specifying the location as an argument to
* the template invocation.
*
* <h3>Permissions</h3>
*
* <p>Permission requirements depend on the {@link PipelineRunner} that is used to execute the
* pipeline. Please refer to the documentation of corresponding {@link PipelineRunner}s for more
* details.
*
* <p>Please see <a href="https://cloud.google.com/bigquery/access-control">BigQuery Access Control
* </a> for security and permission related information specific to BigQuery.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class BigQueryIO {
/**
* Template for BigQuery jobs created by BigQueryIO. This template is: {@code
* "beam_bq_job_{TYPE}_{JOB_ID}_{STEP}_{RANDOM}"}, where:
*
* <ul>
* <li>{@code TYPE} represents the BigQuery job type (e.g. extract / copy / load / query)
* <li>{@code JOB_ID} is the Beam job name.
* <li>{@code STEP} is a UUID representing the the Dataflow step that created the BQ job.
* <li>{@code RANDOM} is a random string.
* </ul>
*
* <p><b>NOTE:</b> This job name template does not have backwards compatibility guarantees.
*/
public static final String BIGQUERY_JOB_TEMPLATE = "beam_bq_job_{TYPE}_{JOB_ID}_{STEP}_{RANDOM}";
private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class);
/** Singleton instance of the JSON factory used to read and write JSON formatted rows. */
static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
/**
* Project IDs must contain 6-63 lowercase letters, digits, or dashes. IDs must start with a
* letter and may not end with a dash. This regex isn't exact - this allows for patterns that
* would be rejected by the service, but this is sufficient for basic parsing of table references.
*/
private static final String PROJECT_ID_REGEXP = "[a-z][-a-z0-9:.]{4,61}[a-z0-9]";
/** Regular expression that matches Dataset IDs. */
private static final String DATASET_REGEXP = "[-\\w.]{1,1024}";
/** Regular expression that matches Table IDs. */
private static final String TABLE_REGEXP = "[-\\w$@]{1,1024}";
/**
* Matches table specifications in the form {@code "[project_id]:[dataset_id].[table_id]"} or
* {@code "[dataset_id].[table_id]"}.
*/
private static final String DATASET_TABLE_REGEXP =
String.format(
"((?<PROJECT>%s)[:\\.])?(?<DATASET>%s)\\.(?<TABLE>%s)",
PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP);
static final Pattern TABLE_SPEC = Pattern.compile(DATASET_TABLE_REGEXP);
/**
* Matches table specifications in the form {@code "projects/[project_id]/datasets/[dataset_id]/tables[table_id]".
*/
private static final String TABLE_URN_REGEXP =
String.format(
"projects/(?<PROJECT>%s)/datasets/(?<DATASET>%s)/tables/(?<TABLE>%s)",
PROJECT_ID_REGEXP, DATASET_REGEXP, TABLE_REGEXP);
static final Pattern TABLE_URN_SPEC = Pattern.compile(TABLE_URN_REGEXP);
/**
* A formatting function that maps a TableRow to itself. This allows sending a {@code
* PCollection<TableRow>} directly to BigQueryIO.Write.
*/
static final SerializableFunction<TableRow, TableRow> IDENTITY_FORMATTER = input -> input;
static final SerializableFunction<org.apache.avro.Schema, DatumWriter<GenericRecord>>
GENERIC_DATUM_WRITER_FACTORY = schema -> new GenericDatumWriter<>();
private static final SerializableFunction<TableSchema, org.apache.avro.Schema>
DEFAULT_AVRO_SCHEMA_FACTORY =
(SerializableFunction<TableSchema, org.apache.avro.Schema>)
input -> BigQueryAvroUtils.toGenericAvroSchema("root", input.getFields());
/**
* @deprecated Use {@link #read(SerializableFunction)} or {@link #readTableRows} instead. {@link
* #readTableRows()} does exactly the same as {@link #read}, however {@link
* #read(SerializableFunction)} performs better.
*/
@Deprecated
public static Read read() {
return new Read();
}
/**
* Like {@link #read(SerializableFunction)} but represents each row as a {@link TableRow}.
*
* <p>This method is more convenient to use in some cases, but usually has significantly lower
* performance than using {@link #read(SerializableFunction)} directly to parse data into a
* domain-specific type, due to the overhead of converting the rows to {@link TableRow}.
*/
public static TypedRead<TableRow> readTableRows() {
return read(new TableRowParser()).withCoder(TableRowJsonCoder.of());
}
/** Like {@link #readTableRows()} but with {@link Schema} support. */
public static TypedRead<TableRow> readTableRowsWithSchema() {
return read(new TableRowParser())
.withCoder(TableRowJsonCoder.of())
.withBeamRowConverters(
TypeDescriptor.of(TableRow.class),
BigQueryUtils.tableRowToBeamRow(),
BigQueryUtils.tableRowFromBeamRow());
}
/**
* Reads from a BigQuery table or query and returns a {@link PCollection} with one element per
* each row of the table or query result, parsed from the BigQuery AVRO format using the specified
* function.
*
* <p>Each {@link SchemaAndRecord} contains a BigQuery {@link TableSchema} and a {@link
* GenericRecord} representing the row, indexed by column name. Here is a sample parse function
* that parses click events from a table.
*
* <pre>{@code
* class ClickEvent { long userId; String url; ... }
*
* p.apply(BigQueryIO.read(new SerializableFunction<SchemaAndRecord, ClickEvent>() {
* public ClickEvent apply(SchemaAndRecord record) {
* GenericRecord r = record.getRecord();
* return new ClickEvent((Long) r.get("userId"), (String) r.get("url"));
* }
* }).from("...");
* }</pre>
*/
public static <T> TypedRead<T> read(SerializableFunction<SchemaAndRecord, T> parseFn) {
return new AutoValue_BigQueryIO_TypedRead.Builder<T>()
.setValidate(true)
.setWithTemplateCompatibility(false)
.setBigQueryServices(new BigQueryServicesImpl())
.setParseFn(parseFn)
.setMethod(TypedRead.Method.DEFAULT)
.setUseAvroLogicalTypes(false)
.setFormat(DataFormat.AVRO)
.build();
}
@VisibleForTesting
static class TableRowParser implements SerializableFunction<SchemaAndRecord, TableRow> {
public static final TableRowParser INSTANCE = new TableRowParser();
@Override
public TableRow apply(SchemaAndRecord schemaAndRecord) {
return BigQueryAvroUtils.convertGenericRecordToTableRow(
schemaAndRecord.getRecord(), schemaAndRecord.getTableSchema());
}
}
/** Implementation of {@link BigQueryIO#read()}. */
public static class Read extends PTransform<PBegin, PCollection<TableRow>> {
private final TypedRead<TableRow> inner;
Read() {
this(BigQueryIO.read(TableRowParser.INSTANCE).withCoder(TableRowJsonCoder.of()));
}
Read(TypedRead<TableRow> inner) {
this.inner = inner;
}
@Override
public PCollection<TableRow> expand(PBegin input) {
return input.apply(inner);
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
this.inner.populateDisplayData(builder);
}
boolean getValidate() {
return this.inner.getValidate();
}
ValueProvider<String> getQuery() {
return this.inner.getQuery();
}
public Read withTestServices(BigQueryServices testServices) {
return new Read(this.inner.withTestServices(testServices));
}
///////////////////////////////////////////////////////////////////
/** Returns the table to read, or {@code null} if reading from a query instead. */
public @Nullable ValueProvider<TableReference> getTableProvider() {
return this.inner.getTableProvider();
}
/** Returns the table to read, or {@code null} if reading from a query instead. */
public @Nullable TableReference getTable() {
return this.inner.getTable();
}
/**
* Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or {@code
* "[dataset_id].[table_id]"} for tables within the current project.
*/
public Read from(String tableSpec) {
return new Read(this.inner.from(tableSpec));
}
/** Same as {@code from(String)}, but with a {@link ValueProvider}. */
public Read from(ValueProvider<String> tableSpec) {
return new Read(this.inner.from(tableSpec));
}
/** Read from table specified by a {@link TableReference}. */
public Read from(TableReference table) {
return new Read(this.inner.from(table));
}
/**
* Reads results received after executing the given query.
*
* <p>By default, the query results will be flattened -- see "flattenResults" in the <a
* href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">Jobs documentation</a> for
* more information. To disable flattening, use {@link Read#withoutResultFlattening}.
*
* <p>By default, the query will use BigQuery's legacy SQL dialect. To use the BigQuery Standard
* SQL dialect, use {@link Read#usingStandardSql}.
*/
public Read fromQuery(String query) {
return new Read(this.inner.fromQuery(query));
}
/** Same as {@code fromQuery(String)}, but with a {@link ValueProvider}. */
public Read fromQuery(ValueProvider<String> query) {
return new Read(this.inner.fromQuery(query));
}
/**
* Disable validation that the table exists or the query succeeds prior to pipeline submission.
* Basic validation (such as ensuring that a query or table is specified) still occurs.
*/
public Read withoutValidation() {
return new Read(this.inner.withoutValidation());
}
/**
* Disable <a href="https://cloud.google.com/bigquery/docs/reference/v2/jobs">flattening of
* query results</a>.
*
* <p>Only valid when a query is used ({@link #fromQuery}). Setting this option when reading
* from a table will cause an error during validation.
*/
public Read withoutResultFlattening() {
return new Read(this.inner.withoutResultFlattening());
}
/**
* Enables BigQuery's Standard SQL dialect when reading from a query.
*
* <p>Only valid when a query is used ({@link #fromQuery}). Setting this option when reading
* from a table will cause an error during validation.
*/
public Read usingStandardSql() {
return new Read(this.inner.usingStandardSql());
}
/**
* Use new template-compatible source implementation.
*
* <p>Use new template-compatible source implementation. This implementation is compatible with
* repeated template invocations. It does not support dynamic work rebalancing.
*/
public Read withTemplateCompatibility() {
return new Read(this.inner.withTemplateCompatibility());
}
}
/////////////////////////////////////////////////////////////////////////////
/** Implementation of {@link BigQueryIO#read(SerializableFunction)}. */
@AutoValue
public abstract static class TypedRead<T> extends PTransform<PBegin, PCollection<T>> {
/** Determines the method used to read data from BigQuery. */
public enum Method {
/** The default behavior if no method is explicitly set. Currently {@link #EXPORT}. */
DEFAULT,
/**
* Export data to Google Cloud Storage in Avro format and read data files from that location.
*/
EXPORT,
/** Read the contents of a table directly using the BigQuery storage API. */
DIRECT_READ,
}
interface ToBeamRowFunction<T>
extends SerializableFunction<Schema, SerializableFunction<T, Row>> {}
interface FromBeamRowFunction<T>
extends SerializableFunction<Schema, SerializableFunction<Row, T>> {}
abstract Builder<T> toBuilder();
@AutoValue.Builder
abstract static class Builder<T> {
abstract Builder<T> setJsonTableRef(ValueProvider<String> jsonTableRef);
abstract Builder<T> setQuery(ValueProvider<String> query);
abstract Builder<T> setValidate(boolean validate);
abstract Builder<T> setFlattenResults(Boolean flattenResults);
abstract Builder<T> setUseLegacySql(Boolean useLegacySql);
abstract Builder<T> setWithTemplateCompatibility(Boolean useTemplateCompatibility);
abstract Builder<T> setBigQueryServices(BigQueryServices bigQueryServices);
abstract Builder<T> setQueryPriority(QueryPriority priority);
abstract Builder<T> setQueryLocation(String location);
abstract Builder<T> setQueryTempDataset(String queryTempDataset);
abstract Builder<T> setMethod(TypedRead.Method method);
@Experimental(Experimental.Kind.SOURCE_SINK)
abstract Builder<T> setFormat(DataFormat method);
abstract Builder<T> setSelectedFields(ValueProvider<List<String>> selectedFields);
abstract Builder<T> setRowRestriction(ValueProvider<String> rowRestriction);
abstract TypedRead<T> build();
abstract Builder<T> setParseFn(SerializableFunction<SchemaAndRecord, T> parseFn);
abstract Builder<T> setCoder(Coder<T> coder);
abstract Builder<T> setKmsKey(String kmsKey);
@Experimental(Kind.SCHEMAS)
abstract Builder<T> setTypeDescriptor(TypeDescriptor<T> typeDescriptor);
@Experimental(Kind.SCHEMAS)
abstract Builder<T> setToBeamRowFn(ToBeamRowFunction<T> toRowFn);
@Experimental(Kind.SCHEMAS)
abstract Builder<T> setFromBeamRowFn(FromBeamRowFunction<T> fromRowFn);
abstract Builder<T> setUseAvroLogicalTypes(Boolean useAvroLogicalTypes);
}
abstract @Nullable ValueProvider<String> getJsonTableRef();
abstract @Nullable ValueProvider<String> getQuery();
abstract boolean getValidate();
abstract @Nullable Boolean getFlattenResults();
abstract @Nullable Boolean getUseLegacySql();
abstract Boolean getWithTemplateCompatibility();
abstract BigQueryServices getBigQueryServices();
abstract SerializableFunction<SchemaAndRecord, T> getParseFn();
abstract @Nullable QueryPriority getQueryPriority();
abstract @Nullable String getQueryLocation();
abstract @Nullable String getQueryTempDataset();
abstract TypedRead.Method getMethod();
@Experimental(Experimental.Kind.SOURCE_SINK)
abstract DataFormat getFormat();
abstract @Nullable ValueProvider<List<String>> getSelectedFields();
abstract @Nullable ValueProvider<String> getRowRestriction();
abstract @Nullable Coder<T> getCoder();
abstract @Nullable String getKmsKey();
@Experimental(Kind.SCHEMAS)
abstract @Nullable TypeDescriptor<T> getTypeDescriptor();
@Experimental(Kind.SCHEMAS)
abstract @Nullable ToBeamRowFunction<T> getToBeamRowFn();
@Experimental(Kind.SCHEMAS)
abstract @Nullable FromBeamRowFunction<T> getFromBeamRowFn();
abstract Boolean getUseAvroLogicalTypes();
/**
* An enumeration type for the priority of a query.
*
* @see <a href="https://cloud.google.com/bigquery/docs/running-queries">Running Interactive and
* Batch Queries in the BigQuery documentation</a>
*/
public enum QueryPriority {
/**
* Specifies that a query should be run with an INTERACTIVE priority.
*
* <p>Interactive mode allows for BigQuery to execute the query as soon as possible. These
* queries count towards your concurrent rate limit and your daily limit.
*/
INTERACTIVE,
/**
* Specifies that a query should be run with a BATCH priority.
*
* <p>Batch mode queries are queued by BigQuery. These are started as soon as idle resources
* are available, usually within a few minutes. Batch queries don’t count towards your
* concurrent rate limit.
*/
BATCH
}
@VisibleForTesting
Coder<T> inferCoder(CoderRegistry coderRegistry) {
if (getCoder() != null) {
return getCoder();
}
try {
return coderRegistry.getCoder(TypeDescriptors.outputOf(getParseFn()));
} catch (CannotProvideCoderException e) {
throw new IllegalArgumentException(
"Unable to infer coder for output of parseFn. Specify it explicitly using withCoder().",
e);
}
}
private BigQuerySourceDef createSourceDef() {
BigQuerySourceDef sourceDef;
if (getQuery() == null) {
sourceDef = BigQueryTableSourceDef.create(getBigQueryServices(), getTableProvider());
} else {
sourceDef =
BigQueryQuerySourceDef.create(
getBigQueryServices(),
getQuery(),
getFlattenResults(),
getUseLegacySql(),
MoreObjects.firstNonNull(getQueryPriority(), QueryPriority.BATCH),
getQueryLocation(),
getQueryTempDataset(),
getKmsKey());
}
return sourceDef;
}
private BigQueryStorageQuerySource<T> createStorageQuerySource(
String stepUuid, Coder<T> outputCoder) {
return BigQueryStorageQuerySource.create(
stepUuid,
getQuery(),
getFlattenResults(),
getUseLegacySql(),
MoreObjects.firstNonNull(getQueryPriority(), QueryPriority.BATCH),
getQueryLocation(),
getQueryTempDataset(),
getKmsKey(),
getFormat(),
getParseFn(),
outputCoder,
getBigQueryServices());
}
private static final String QUERY_VALIDATION_FAILURE_ERROR =
"Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
+ " pipeline, This validation can be disabled using #withoutValidation.";
@Override
public void validate(PipelineOptions options) {
// Even if existence validation is disabled, we need to make sure that the BigQueryIO
// read is properly specified.
BigQueryOptions bqOptions = options.as(BigQueryOptions.class);
if (getMethod() != TypedRead.Method.DIRECT_READ) {
String tempLocation = bqOptions.getTempLocation();
checkArgument(
!Strings.isNullOrEmpty(tempLocation),
"BigQueryIO.Read needs a GCS temp location to store temp files."
+ "This can be set with option --tempLocation.");
if (getBigQueryServices() == null) {
try {
GcsPath.fromUri(tempLocation);
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException(
String.format(
"BigQuery temp location expected a valid 'gs://' path, but was given '%s'",
tempLocation),
e);
}
}
}
ValueProvider<TableReference> table = getTableProvider();
// Note that a table or query check can fail if the table or dataset are created by
// earlier stages of the pipeline or if a query depends on earlier stages of a pipeline.
// For these cases the withoutValidation method can be used to disable the check.
if (getValidate()) {
try (DatasetService datasetService = getBigQueryServices().getDatasetService(bqOptions)) {
if (table != null) {
checkArgument(
table.isAccessible(), "Cannot call validate if table is dynamically set.");
}
if (table != null && table.get().getProjectId() != null) {
// Check for source table presence for early failure notification.
BigQueryHelpers.verifyDatasetPresence(datasetService, table.get());
BigQueryHelpers.verifyTablePresence(datasetService, table.get());
} else if (getQuery() != null) {
checkArgument(
getQuery().isAccessible(), "Cannot call validate if query is dynamically set.");
JobService jobService = getBigQueryServices().getJobService(bqOptions);
try {
jobService.dryRunQuery(
bqOptions.getBigQueryProject() == null
? bqOptions.getProject()
: bqOptions.getBigQueryProject(),
new JobConfigurationQuery()
.setQuery(getQuery().get())
.setFlattenResults(getFlattenResults())
.setUseLegacySql(getUseLegacySql()),
getQueryLocation());
} catch (Exception e) {
throw new IllegalArgumentException(
String.format(QUERY_VALIDATION_FAILURE_ERROR, getQuery().get()), e);
}
// If the user provided a temp dataset, check if the dataset exists before launching the
// query
if (getQueryTempDataset() != null) {
// The temp table is only used for dataset and project id validation, not for table
// name
// validation
TableReference tempTable =
new TableReference()
.setProjectId(