/
BigQueryIOTest.java
2034 lines (1770 loc) · 81.8 KB
/
BigQueryIOTest.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 com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.createJobIdToken;
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.createTempTableReference;
import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.toJsonString;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.hasItem;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import com.google.api.client.util.Data;
import com.google.api.services.bigquery.model.Job;
import com.google.api.services.bigquery.model.JobStatistics;
import com.google.api.services.bigquery.model.JobStatistics2;
import com.google.api.services.bigquery.model.JobStatistics4;
import com.google.api.services.bigquery.model.JobStatus;
import com.google.api.services.bigquery.model.Table;
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.common.collect.HashBasedTable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.Serializable;
import java.math.BigDecimal;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.Coder.Context;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.CountingSource;
import org.apache.beam.sdk.io.GenerateSequence;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation;
import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.StreamingOptions;
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.testing.CoderProperties;
import org.apache.beam.sdk.testing.ExpectedLogs;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.SourceTestUtils;
import org.apache.beam.sdk.testing.SourceTestUtils.ExpectedSplitOutcome;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.DoFnTester;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
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.transforms.display.DisplayDataEvaluator;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.util.IOChannelUtils;
import org.apache.beam.sdk.util.MimeTypes;
import org.apache.beam.sdk.util.PCollectionViews;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowingStrategy;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.hamcrest.CoreMatchers;
import org.hamcrest.Matchers;
import org.joda.time.Instant;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
/**
* Tests for BigQueryIO.
*/
@RunWith(JUnit4.class)
public class BigQueryIOTest implements Serializable {
private static Path tempFolder;
// Table information must be static, as each ParDo will get a separate instance of
// FakeDatasetServices, and they must all modify the same storage.
static com.google.common.collect.Table<String, String, Map<String, TableContainer>>
tables = HashBasedTable.create();
@Rule public final transient TestPipeline p = TestPipeline.create();
@Rule public transient ExpectedException thrown = ExpectedException.none();
@Rule public transient ExpectedLogs loggedBigQueryIO = ExpectedLogs.none(BigQueryIO.class);
@Rule public transient ExpectedLogs loggedWriteRename = ExpectedLogs.none(WriteRename.class);
@Rule public transient ExpectedLogs loggedWriteTables = ExpectedLogs.none(WriteTables.class);
@Rule public transient TemporaryFolder testFolder = new TemporaryFolder();
private void checkReadTableObject(
BigQueryIO.Read read, String project, String dataset, String table) {
checkReadTableObjectWithValidate(read, project, dataset, table, true);
}
private void checkReadQueryObject(BigQueryIO.Read read, String query) {
checkReadQueryObjectWithValidate(read, query, true);
}
private void checkReadTableObjectWithValidate(
BigQueryIO.Read read, String project, String dataset, String table, boolean validate) {
assertEquals(project, read.getTable().getProjectId());
assertEquals(dataset, read.getTable().getDatasetId());
assertEquals(table, read.getTable().getTableId());
assertNull(read.getQuery());
assertEquals(validate, read.getValidate());
}
private void checkReadQueryObjectWithValidate(
BigQueryIO.Read read, String query, boolean validate) {
assertNull(read.getTable());
assertEquals(query, read.getQuery().get());
assertEquals(validate, read.getValidate());
}
private void checkWriteObject(
BigQueryIO.Write write, String project, String dataset, String table,
TableSchema schema, CreateDisposition createDisposition,
WriteDisposition writeDisposition, String tableDescription) {
checkWriteObjectWithValidate(
write,
project,
dataset,
table,
schema,
createDisposition,
writeDisposition,
tableDescription,
true);
}
private void checkWriteObjectWithValidate(
BigQueryIO.Write<TableRow> write, String project, String dataset, String table,
TableSchema schema, CreateDisposition createDisposition,
WriteDisposition writeDisposition, String tableDescription, boolean validate) {
assertEquals(project, write.getTable().get().getProjectId());
assertEquals(dataset, write.getTable().get().getDatasetId());
assertEquals(table, write.getTable().get().getTableId());
assertEquals(schema, write.getSchema());
assertEquals(createDisposition, write.getCreateDisposition());
assertEquals(writeDisposition, write.getWriteDisposition());
assertEquals(tableDescription, write.getTableDescription());
assertEquals(validate, write.getValidate());
}
@BeforeClass
public static void setupClass() throws IOException {
tempFolder = Files.createTempDirectory("BigQueryIOTest");
}
@Before
public void setUp() throws IOException {
tables = HashBasedTable.create();
BigQueryIO.clearCreatedTables();
}
@Test
public void testBuildTableBasedSource() {
BigQueryIO.Read read = BigQueryIO.read().from("foo.com:project:somedataset.sometable");
checkReadTableObject(read, "foo.com:project", "somedataset", "sometable");
}
@Test
public void testBuildQueryBasedSource() {
BigQueryIO.Read read = BigQueryIO.read().fromQuery("foo_query");
checkReadQueryObject(read, "foo_query");
}
@Test
public void testBuildTableBasedSourceWithoutValidation() {
// This test just checks that using withoutValidation will not trigger object
// construction errors.
BigQueryIO.Read read =
BigQueryIO.read().from("foo.com:project:somedataset.sometable").withoutValidation();
checkReadTableObjectWithValidate(read, "foo.com:project", "somedataset", "sometable", false);
}
@Test
public void testBuildQueryBasedSourceWithoutValidation() {
// This test just checks that using withoutValidation will not trigger object
// construction errors.
BigQueryIO.Read read =
BigQueryIO.read().fromQuery("some_query").withoutValidation();
checkReadQueryObjectWithValidate(read, "some_query", false);
}
@Test
public void testBuildTableBasedSourceWithDefaultProject() {
BigQueryIO.Read read =
BigQueryIO.read().from("somedataset.sometable");
checkReadTableObject(read, null, "somedataset", "sometable");
}
@Test
public void testBuildSourceWithTableReference() {
TableReference table = new TableReference()
.setProjectId("foo.com:project")
.setDatasetId("somedataset")
.setTableId("sometable");
BigQueryIO.Read read = BigQueryIO.read().from(table);
checkReadTableObject(read, "foo.com:project", "somedataset", "sometable");
}
@Test
public void testValidateReadSetsDefaultProject() throws Exception {
String projectId = "someproject";
String datasetId = "somedataset";
String tableId = "sometable";
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject(projectId);
Path baseDir = Files.createTempDirectory(tempFolder, "testValidateReadSetsDefaultProject");
bqOptions.setTempLocation(baseDir.toString());
FakeDatasetService fakeDatasetService = new FakeDatasetService();
fakeDatasetService.createDataset(projectId, datasetId, "", "");
TableReference tableReference =
new TableReference().setProjectId(projectId).setDatasetId(datasetId).setTableId(tableId);
fakeDatasetService.createTable(new Table()
.setTableReference(tableReference)
.setSchema(new TableSchema()
.setFields(
ImmutableList.of(
new TableFieldSchema().setName("name").setType("STRING"),
new TableFieldSchema().setName("number").setType("INTEGER")))));
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withJobService(new FakeJobService())
.withDatasetService(fakeDatasetService);
List<TableRow> expected = ImmutableList.of(
new TableRow().set("name", "a").set("number", 1L),
new TableRow().set("name", "b").set("number", 2L),
new TableRow().set("name", "c").set("number", 3L),
new TableRow().set("name", "d").set("number", 4L),
new TableRow().set("name", "e").set("number", 5L),
new TableRow().set("name", "f").set("number", 6L));
fakeDatasetService.insertAll(tableReference, expected, null);
Pipeline p = TestPipeline.create(bqOptions);
TableReference tableRef = new TableReference();
tableRef.setDatasetId(datasetId);
tableRef.setTableId(tableId);
PCollection<KV<String, Long>> output =
p.apply(BigQueryIO.read().from(tableRef).withTestServices(fakeBqServices))
.apply(ParDo.of(new DoFn<TableRow, KV<String, Long>>() {
@ProcessElement
public void processElement(ProcessContext c) throws Exception {
c.output(KV.of((String) c.element().get("name"),
Long.valueOf((String) c.element().get("number"))));
}
}));
PAssert.that(output).containsInAnyOrder(ImmutableList.of(KV.of("a", 1L), KV.of("b", 2L),
KV.of("c", 3L), KV.of("d", 4L), KV.of("e", 5L), KV.of("f", 6L)));
p.run();
}
@Test
public void testBuildSourceWithTableAndFlatten() {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation("gs://testbucket/testdir");
Pipeline p = TestPipeline.create(bqOptions);
thrown.expect(IllegalStateException.class);
thrown.expectMessage(
"Invalid BigQueryIO.Read: Specifies a table with a result flattening preference,"
+ " which only applies to queries");
p.apply("ReadMyTable",
BigQueryIO.read()
.from("foo.com:project:somedataset.sometable")
.withoutResultFlattening());
p.run();
}
@Test
public void testBuildSourceWithTableAndFlattenWithoutValidation() {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation("gs://testbucket/testdir");
Pipeline p = TestPipeline.create(bqOptions);
thrown.expect(IllegalStateException.class);
thrown.expectMessage(
"Invalid BigQueryIO.Read: Specifies a table with a result flattening preference,"
+ " which only applies to queries");
p.apply(
BigQueryIO.read()
.from("foo.com:project:somedataset.sometable")
.withoutValidation()
.withoutResultFlattening());
p.run();
}
@Test
public void testBuildSourceWithTableAndSqlDialect() {
BigQueryOptions bqOptions = PipelineOptionsFactory.as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation("gs://testbucket/testdir");
Pipeline p = TestPipeline.create(bqOptions);
thrown.expect(IllegalStateException.class);
thrown.expectMessage(
"Invalid BigQueryIO.Read: Specifies a table with a SQL dialect preference,"
+ " which only applies to queries");
p.apply(
BigQueryIO.read()
.from("foo.com:project:somedataset.sometable")
.usingStandardSql());
p.run();
}
@Test
public void testReadFromTable() throws IOException, InterruptedException {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
Job job = new Job();
JobStatus status = new JobStatus();
job.setStatus(status);
JobStatistics jobStats = new JobStatistics();
job.setStatistics(jobStats);
JobStatistics4 extract = new JobStatistics4();
jobStats.setExtract(extract);
extract.setDestinationUriFileCounts(ImmutableList.of(1L));
Table sometable = new Table();
sometable.setSchema(
new TableSchema()
.setFields(
ImmutableList.of(
new TableFieldSchema().setName("name").setType("STRING"),
new TableFieldSchema().setName("number").setType("INTEGER"))));
sometable.setTableReference(
new TableReference()
.setProjectId("non-executing-project")
.setDatasetId("somedataset")
.setTableId("sometable"));
sometable.setNumBytes(1024L * 1024L);
FakeDatasetService fakeDatasetService = new FakeDatasetService();
fakeDatasetService.createDataset("non-executing-project", "somedataset", "", "");
fakeDatasetService.createTable(sometable);
List<TableRow> records = Lists.newArrayList(
new TableRow().set("name", "a").set("number", 1L),
new TableRow().set("name", "b").set("number", 2L),
new TableRow().set("name", "c").set("number", 3L));
fakeDatasetService.insertAll(sometable.getTableReference(), records, null);
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withJobService(new FakeJobService())
.withDatasetService(fakeDatasetService);
Pipeline p = TestPipeline.create(bqOptions);
PCollection<KV<String, Long>> output = p
.apply(BigQueryIO.read().from("non-executing-project:somedataset.sometable")
.withTestServices(fakeBqServices)
.withoutValidation())
.apply(ParDo.of(new DoFn<TableRow, KV<String, Long>>() {
@ProcessElement
public void processElement(ProcessContext c) throws Exception {
c.output(KV.of((String) c.element().get("name"),
Long.valueOf((String) c.element().get("number"))));
}
}));
PAssert.that(output)
.containsInAnyOrder(ImmutableList.of(KV.of("a", 1L), KV.of("b", 2L), KV.of("c", 3L)));
p.run();
}
@Test
public void testWrite() throws Exception {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
FakeDatasetService datasetService = new FakeDatasetService();
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withJobService(new FakeJobService())
.withDatasetService(datasetService);
datasetService.createDataset("defaultproject", "dataset-id", "", "");
Pipeline p = TestPipeline.create(bqOptions);
p.apply(Create.of(
new TableRow().set("name", "a").set("number", 1),
new TableRow().set("name", "b").set("number", 2),
new TableRow().set("name", "c").set("number", 3))
.withCoder(TableRowJsonCoder.of()))
.apply(BigQueryIO.writeTableRows().to("dataset-id.table-id")
.withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
.withSchema(new TableSchema().setFields(
ImmutableList.of(
new TableFieldSchema().setName("name").setType("STRING"),
new TableFieldSchema().setName("number").setType("INTEGER"))))
.withTestServices(fakeBqServices)
.withoutValidation());
p.run();
File tempDir = new File(bqOptions.getTempLocation());
testNumFiles(tempDir, 0);
}
@Test
public void testStreamingWrite() throws Exception {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
FakeDatasetService datasetService = new FakeDatasetService();
datasetService.createDataset("project-id", "dataset-id", "", "");
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withDatasetService(datasetService);
Pipeline p = TestPipeline.create(bqOptions);
p.apply(Create.of(
new TableRow().set("name", "a").set("number", 1),
new TableRow().set("name", "b").set("number", 2),
new TableRow().set("name", "c").set("number", 3),
new TableRow().set("name", "d").set("number", 4))
.withCoder(TableRowJsonCoder.of()))
.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED)
.apply(BigQueryIO.writeTableRows().to("project-id:dataset-id.table-id")
.withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
.withSchema(new TableSchema().setFields(
ImmutableList.of(
new TableFieldSchema().setName("name").setType("STRING"),
new TableFieldSchema().setName("number").setType("INTEGER"))))
.withTestServices(fakeBqServices)
.withoutValidation());
p.run();
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id"),
containsInAnyOrder(
new TableRow().set("name", "a").set("number", 1),
new TableRow().set("name", "b").set("number", 2),
new TableRow().set("name", "c").set("number", 3),
new TableRow().set("name", "d").set("number", 4)));
}
/**
* A generic window function that allows partitioning data into windows by a string value.
*
* <p>Logically, creates multiple global windows, and the user provides a function that
* decides which global window a value should go into.
*/
private static class PartitionedGlobalWindows<T> extends
NonMergingWindowFn<T, PartitionedGlobalWindow> {
private SerializableFunction<T, String> extractPartition;
public PartitionedGlobalWindows(SerializableFunction<T, String> extractPartition) {
this.extractPartition = extractPartition;
}
@Override
public Collection<PartitionedGlobalWindow> assignWindows(AssignContext c) {
return Collections.singletonList(new PartitionedGlobalWindow(
extractPartition.apply(c.element())));
}
@Override
public boolean isCompatible(WindowFn<?, ?> o) {
return o instanceof PartitionedGlobalWindows;
}
@Override
public Coder<PartitionedGlobalWindow> windowCoder() {
return new PartitionedGlobalWindowCoder();
}
@Override
public WindowMappingFn<PartitionedGlobalWindow> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException(
"PartitionedGlobalWindows is not allowed in side inputs");
}
@Override
public Instant getOutputTime(Instant inputTimestamp, PartitionedGlobalWindow window) {
return inputTimestamp;
}
}
/**
* Custom Window object that encodes a String value.
*/
private static class PartitionedGlobalWindow extends BoundedWindow {
String value;
public PartitionedGlobalWindow(String value) {
this.value = value;
}
@Override
public Instant maxTimestamp() {
return GlobalWindow.INSTANCE.maxTimestamp();
}
// The following methods are only needed due to BEAM-1022. Once this issue is fixed, we will
// no longer need these.
@Override
public boolean equals(Object other) {
if (other instanceof PartitionedGlobalWindow) {
return value.equals(((PartitionedGlobalWindow) other).value);
}
return false;
}
@Override
public int hashCode() {
return value.hashCode();
}
}
/**
* Coder for @link{PartitionedGlobalWindow}.
*/
private static class PartitionedGlobalWindowCoder extends CustomCoder<PartitionedGlobalWindow> {
@Override
public void encode(PartitionedGlobalWindow window, OutputStream outStream, Context context)
throws IOException, CoderException {
StringUtf8Coder.of().encode(window.value, outStream, context);
}
@Override
public PartitionedGlobalWindow decode(InputStream inStream, Context context)
throws IOException, CoderException {
return new PartitionedGlobalWindow(StringUtf8Coder.of().decode(inStream, context));
}
@Override
public void verifyDeterministic() {}
}
@Test
public void testStreamingWriteWithDynamicTables() throws Exception {
testWriteWithDynamicTables(true);
}
@Test
public void testBatchWriteWithDynamicTables() throws Exception {
testWriteWithDynamicTables(false);
}
public void testWriteWithDynamicTables(boolean streaming) throws Exception {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
FakeDatasetService datasetService = new FakeDatasetService();
datasetService.createDataset("project-id", "dataset-id", "", "");
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withDatasetService(datasetService)
.withJobService(new FakeJobService());
List<Integer> inserts = new ArrayList<>();
for (int i = 0; i < 10; i++) {
inserts.add(i);
}
// Create a windowing strategy that puts the input into five different windows depending on
// record value.
WindowFn<Integer, PartitionedGlobalWindow> window = new PartitionedGlobalWindows(
new SerializableFunction<Integer, String>() {
@Override
public String apply(Integer i) {
return Integer.toString(i % 5);
}
}
);
SerializableFunction<ValueInSingleWindow<Integer>, TableDestination> tableFunction =
new SerializableFunction<ValueInSingleWindow<Integer>, TableDestination>() {
@Override
public TableDestination apply(ValueInSingleWindow<Integer> input) {
PartitionedGlobalWindow window = (PartitionedGlobalWindow) input.getWindow();
// Check that we can access the element as well here.
checkArgument(window.value.equals(Integer.toString(input.getValue() % 5)),
"Incorrect element");
return new TableDestination("project-id:dataset-id.table-id-" + window.value, "");
}
};
Pipeline p = TestPipeline.create(bqOptions);
PCollection<Integer> input = p.apply(Create.of(inserts));
if (streaming) {
input = input.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED);
}
input.apply(Window.<Integer>into(window))
.apply(BigQueryIO.<Integer>write()
.to(tableFunction)
.withFormatFunction(new SerializableFunction<Integer, TableRow>() {
@Override
public TableRow apply(Integer i) {
return new TableRow().set("name", "number" + i).set("number", i);
}})
.withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
.withSchema(new TableSchema().setFields(
ImmutableList.of(
new TableFieldSchema().setName("name").setType("STRING"),
new TableFieldSchema().setName("number").setType("INTEGER"))))
.withTestServices(fakeBqServices)
.withoutValidation());
p.run();
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-0"),
containsInAnyOrder(
new TableRow().set("name", "number0").set("number", 0),
new TableRow().set("name", "number5").set("number", 5)));
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-1"),
containsInAnyOrder(
new TableRow().set("name", "number1").set("number", 1),
new TableRow().set("name", "number6").set("number", 6)));
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-2"),
containsInAnyOrder(
new TableRow().set("name", "number2").set("number", 2),
new TableRow().set("name", "number7").set("number", 7)));
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-3"),
containsInAnyOrder(
new TableRow().set("name", "number3").set("number", 3),
new TableRow().set("name", "number8").set("number", 8)));
assertThat(datasetService.getAllRows("project-id", "dataset-id", "table-id-4"),
containsInAnyOrder(
new TableRow().set("name", "number4").set("number", 4),
new TableRow().set("name", "number9").set("number", 9)));
}
@Test
public void testWriteUnknown() throws Exception {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
FakeDatasetService datasetService = new FakeDatasetService();
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withJobService(new FakeJobService())
.withDatasetService(datasetService);
datasetService.createDataset("project-id", "dataset-id", "", "");
Pipeline p = TestPipeline.create(bqOptions);
p.apply(Create.of(
new TableRow().set("name", "a").set("number", 1),
new TableRow().set("name", "b").set("number", 2),
new TableRow().set("name", "c").set("number", 3))
.withCoder(TableRowJsonCoder.of()))
.apply(BigQueryIO.writeTableRows().to("project-id:dataset-id.table-id")
.withCreateDisposition(CreateDisposition.CREATE_NEVER)
.withTestServices(fakeBqServices)
.withoutValidation());
thrown.expect(RuntimeException.class);
thrown.expectMessage("Failed to create load job");
try {
p.run();
} finally {
File tempDir = new File(bqOptions.getTempLocation());
testNumFiles(tempDir, 0);
}
}
@Test
public void testWriteFailedJobs() throws Exception {
BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
bqOptions.setProject("defaultproject");
bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath());
FakeDatasetService datasetService = new FakeDatasetService();
FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
.withJobService(new FakeJobService())
.withDatasetService(datasetService);
Pipeline p = TestPipeline.create(bqOptions);
p.apply(Create.of(
new TableRow().set("name", "a").set("number", 1),
new TableRow().set("name", "b").set("number", 2),
new TableRow().set("name", "c").set("number", 3))
.withCoder(TableRowJsonCoder.of()))
.apply(BigQueryIO.writeTableRows().to("dataset-id.table-id")
.withCreateDisposition(CreateDisposition.CREATE_NEVER)
.withTestServices(fakeBqServices)
.withoutValidation());
thrown.expect(RuntimeException.class);
thrown.expectMessage("Failed to create load job with id prefix");
thrown.expectMessage("reached max retries");
thrown.expectMessage("last failed load job");
try {
p.run();
} finally {
File tempDir = new File(bqOptions.getTempLocation());
testNumFiles(tempDir, 0);
}
}
@Test
public void testBuildSourceDisplayDataTable() {
String tableSpec = "project:dataset.tableid";
BigQueryIO.Read read = BigQueryIO.read()
.from(tableSpec)
.withoutResultFlattening()
.usingStandardSql()
.withoutValidation();
DisplayData displayData = DisplayData.from(read);
assertThat(displayData, hasDisplayItem("table", tableSpec));
assertThat(displayData, hasDisplayItem("flattenResults", false));
assertThat(displayData, hasDisplayItem("useLegacySql", false));
assertThat(displayData, hasDisplayItem("validation", false));
}
@Test
public void testBuildSourceDisplayDataQuery() {
BigQueryIO.Read read = BigQueryIO.read()
.fromQuery("myQuery")
.withoutResultFlattening()
.usingStandardSql()
.withoutValidation();
DisplayData displayData = DisplayData.from(read);
assertThat(displayData, hasDisplayItem("query", "myQuery"));
assertThat(displayData, hasDisplayItem("flattenResults", false));
assertThat(displayData, hasDisplayItem("useLegacySql", false));
assertThat(displayData, hasDisplayItem("validation", false));
}
@Test
@Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient")
public void testTableSourcePrimitiveDisplayData() throws IOException, InterruptedException {
DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
BigQueryIO.Read read = BigQueryIO.read()
.from("project:dataset.tableId")
.withTestServices(new FakeBigQueryServices()
.withDatasetService(new FakeDatasetService())
.withJobService(new FakeJobService()))
.withoutValidation();
Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
assertThat("BigQueryIO.Read should include the table spec in its primitive display data",
displayData, hasItem(hasDisplayItem("table")));
}
@Test
@Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient")
public void testQuerySourcePrimitiveDisplayData() throws IOException, InterruptedException {
DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
BigQueryIO.Read read = BigQueryIO.read()
.fromQuery("foobar")
.withTestServices(new FakeBigQueryServices()
.withDatasetService(new FakeDatasetService())
.withJobService(new FakeJobService()))
.withoutValidation();
Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
assertThat("BigQueryIO.Read should include the query in its primitive display data",
displayData, hasItem(hasDisplayItem("query")));
}
@Test
public void testBuildWrite() {
BigQueryIO.Write<TableRow> write =
BigQueryIO.writeTableRows().to("foo.com:project:somedataset.sometable");
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, "");
}
@Test
@Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient")
public void testBatchWritePrimitiveDisplayData() throws IOException, InterruptedException {
testWritePrimitiveDisplayData(/* streaming: */ false);
}
@Test
@Ignore("[BEAM-436] DirectRunner tempLocation configuration insufficient")
public void testStreamingWritePrimitiveDisplayData() throws IOException, InterruptedException {
testWritePrimitiveDisplayData(/* streaming: */ true);
}
private void testWritePrimitiveDisplayData(boolean streaming) throws IOException,
InterruptedException {
PipelineOptions options = TestPipeline.testingPipelineOptions();
options.as(StreamingOptions.class).setStreaming(streaming);
DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(options);
BigQueryIO.Write write = BigQueryIO.writeTableRows()
.to("project:dataset.table")
.withSchema(new TableSchema().set("col1", "type1").set("col2", "type2"))
.withTestServices(new FakeBigQueryServices()
.withDatasetService(new FakeDatasetService())
.withJobService(new FakeJobService()))
.withoutValidation();
Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
assertThat("BigQueryIO.Write should include the table spec in its primitive display data",
displayData, hasItem(hasDisplayItem("tableSpec")));
assertThat("BigQueryIO.Write should include the table schema in its primitive display data",
displayData, hasItem(hasDisplayItem("schema")));
}
@Test
public void testBuildWriteWithoutValidation() {
// This test just checks that using withoutValidation will not trigger object
// construction errors.
BigQueryIO.Write write =
BigQueryIO.<TableRow>write().to("foo.com:project:somedataset.sometable")
.withoutValidation();
checkWriteObjectWithValidate(
write,
"foo.com:project",
"somedataset",
"sometable",
null,
CreateDisposition.CREATE_IF_NEEDED,
WriteDisposition.WRITE_EMPTY,
"",
false);
}
@Test
public void testBuildWriteDefaultProject() {
BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows()
.to("somedataset" + ".sometable");
checkWriteObject(
write, null, "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY,
"");
}
@Test
public void testBuildWriteWithTableReference() {
TableReference table = new TableReference()
.setProjectId("foo.com:project")
.setDatasetId("somedataset")
.setTableId("sometable");
BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows().to(table);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, "");
}
@Test
public void testBuildWriteWithSchema() {
TableSchema schema = new TableSchema();
BigQueryIO.Write<TableRow> write =
BigQueryIO.<TableRow>write().to("foo.com:project:somedataset.sometable").withSchema(schema);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, "");
}
@Test
public void testBuildWriteWithCreateDispositionNever() {
BigQueryIO.Write<TableRow> write = BigQueryIO.<TableRow>write()
.to("foo.com:project:somedataset.sometable")
.withCreateDisposition(CreateDisposition.CREATE_NEVER);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_NEVER, WriteDisposition.WRITE_EMPTY, "");
}
@Test
public void testBuildWriteWithCreateDispositionIfNeeded() {
BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows()
.to("foo.com:project:somedataset.sometable")
.withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, "");
}
@Test
public void testBuildWriteWithWriteDispositionTruncate() {
BigQueryIO.Write<TableRow> write = BigQueryIO.<TableRow>write()
.to("foo.com:project:somedataset.sometable")
.withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_TRUNCATE, "");
}
@Test
public void testBuildWriteWithWriteDispositionAppend() {
BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows()
.to("foo.com:project:somedataset.sometable")
.withWriteDisposition(WriteDisposition.WRITE_APPEND);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_APPEND, "");
}
@Test
public void testBuildWriteWithWriteDispositionEmpty() {
BigQueryIO.Write<TableRow> write = BigQueryIO.<TableRow>write()
.to("foo.com:project:somedataset.sometable")
.withWriteDisposition(WriteDisposition.WRITE_EMPTY);
checkWriteObject(
write, "foo.com:project", "somedataset", "sometable",
null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, "");
}
@Test
public void testBuildWriteWithWriteWithTableDescription() {
final String tblDescription = "foo bar table";
BigQueryIO.Write<TableRow> write = BigQueryIO.writeTableRows()
.to("foo.com:project:somedataset.sometable")
.withTableDescription(tblDescription);
checkWriteObject(
write,
"foo.com:project",
"somedataset",
"sometable",
null,
CreateDisposition.CREATE_IF_NEEDED,
WriteDisposition.WRITE_EMPTY,
tblDescription);
}
@Test
public void testBuildWriteDisplayData() {