/
SpannerIO.java
2353 lines (2042 loc) · 95.4 KB
/
SpannerIO.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.spanner;
import static java.util.stream.Collectors.toList;
import static org.apache.beam.sdk.io.gcp.spanner.MutationUtils.isPointDelete;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_CHANGE_STREAM_NAME;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_INCLUSIVE_END_AT;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_INCLUSIVE_START_AT;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.DEFAULT_RPC_PRIORITY;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.MAX_INCLUSIVE_END_AT;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants.THROUGHPUT_WINDOW_SECONDS;
import static org.apache.beam.sdk.io.gcp.spanner.changestreams.NameGenerator.generatePartitionMetadataTableName;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
import com.google.api.gax.retrying.RetrySettings;
import com.google.api.gax.rpc.StatusCode.Code;
import com.google.auth.Credentials;
import com.google.auto.value.AutoValue;
import com.google.cloud.ServiceFactory;
import com.google.cloud.Timestamp;
import com.google.cloud.spanner.AbortedException;
import com.google.cloud.spanner.DatabaseClient;
import com.google.cloud.spanner.DatabaseId;
import com.google.cloud.spanner.Dialect;
import com.google.cloud.spanner.ErrorCode;
import com.google.cloud.spanner.KeySet;
import com.google.cloud.spanner.Mutation;
import com.google.cloud.spanner.Mutation.Op;
import com.google.cloud.spanner.Options;
import com.google.cloud.spanner.Options.RpcPriority;
import com.google.cloud.spanner.PartitionOptions;
import com.google.cloud.spanner.Spanner;
import com.google.cloud.spanner.SpannerException;
import com.google.cloud.spanner.SpannerOptions;
import com.google.cloud.spanner.Statement;
import com.google.cloud.spanner.Struct;
import com.google.cloud.spanner.TimestampBound;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.OptionalInt;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers;
import org.apache.beam.runners.core.metrics.MonitoringInfoConstants;
import org.apache.beam.runners.core.metrics.ServiceCallMetric;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamMetrics;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamsConstants;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.MetadataSpannerConfigFactory;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.action.ActionFactory;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.DaoFactory;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.CleanUpReadChangeStreamDoFn;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.DetectNewPartitionsDoFn;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.InitializeDoFn;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.PostProcessingMetricsDoFn;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.dofn.ReadChangeStreamPartitionDoFn;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.estimator.BytesThroughputEstimator;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.estimator.SizeEstimator;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.mapper.MapperFactory;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.DataChangeRecord;
import org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.options.ValueProvider;
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.Flatten;
import org.apache.beam.sdk.transforms.Impulse;
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.View;
import org.apache.beam.sdk.transforms.Wait;
import org.apache.beam.sdk.transforms.WithTimestamps;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.sdk.util.FluentBackoff;
import org.apache.beam.sdk.util.Sleeper;
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.PCollectionList;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.PInput;
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.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedBytes;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*
*
* <h3>Reading from Cloud Spanner</h3>
*
* <h4>Bulk reading of a single query or table</h4>
*
* <p>To perform a single read from Cloud Spanner, construct a {@link Read} transform using {@link
* SpannerIO#read() SpannerIO.read()}. It will return a {@link PCollection} of {@link Struct
* Structs}, where each element represents an individual row returned from the read operation. Both
* Query and Read APIs are supported. See more information about <a
* href="https://cloud.google.com/spanner/docs/reads">reading from Cloud Spanner</a>
*
* <p>To execute a <strong>Query</strong>, specify a {@link Read#withQuery(Statement)} or {@link
* Read#withQuery(String)} during the construction of the transform.
*
* <pre>{@code
* PCollection<Struct> rows = p.apply(
* SpannerIO.read()
* .withInstanceId(instanceId)
* .withDatabaseId(dbId)
* .withQuery("SELECT id, name, email FROM users"));
* }</pre>
*
* <p>Reads by default use the <a
* href="https://cloud.google.com/spanner/docs/reads#read_data_in_parallel">PartitionQuery API</a>
* which enforces some limitations on the type of queries that can be used so that the data can be
* read in parallel. If the query is not supported by the PartitionQuery API, then you can specify a
* non-partitioned read by setting {@link Read#withBatching(boolean) withBatching(false)}. If the
* amount of data being read by a non-partitioned read is very large, it may be useful to add a
* {@link Reshuffle#viaRandomKey()} transform on the output so that the downstream transforms can
* run in parallel.
*
* <p>To read an entire <strong>Table</strong>, use {@link Read#withTable(String)} and optionally
* specify a {@link Read#withColumns(List) list of columns}.
*
* <pre>{@code
* PCollection<Struct> rows = p.apply(
* SpannerIO.read()
* .withInstanceId(instanceId)
* .withDatabaseId(dbId)
* .withTable("users")
* .withColumns("id", "name", "email"));
* }</pre>
*
* <p>To read using an <strong>Index</strong>, specify the index name using {@link
* Read#withIndex(String)}.
*
* <pre>{@code
* PCollection<Struct> rows = p.apply(
* SpannerIO.read()
* .withInstanceId(instanceId)
* .withDatabaseId(dbId)
* .withTable("users")
* .withIndex("users_by_name")
* .withColumns("id", "name", "email"));
* }</pre>
*
* <h4>Read consistency</h4>
*
* <p>The transform is guaranteed to be executed on a consistent snapshot of data, utilizing the
* power of read only transactions. Staleness of data can be controlled using {@link
* Read#withTimestampBound} or {@link Read#withTimestamp(Timestamp)} methods. <a
* href="https://cloud.google.com/spanner/docs/transactions#read-only_transactions">Read more</a>
* about transactions in Cloud Spanner.
*
* <p>It is possible to read several {@link PCollection PCollections} within a single transaction.
* Apply {@link SpannerIO#createTransaction()} transform, that lazily creates a transaction. The
* result of this transformation can be passed to read operation using {@link
* Read#withTransaction(PCollectionView)}.
*
* <pre>{@code
* SpannerConfig spannerConfig = ...
*
* PCollectionView<Transaction> tx = p.apply(
* SpannerIO.createTransaction()
* .withSpannerConfig(spannerConfig)
* .withTimestampBound(TimestampBound.strong()));
*
* PCollection<Struct> users = p.apply(
* SpannerIO.read()
* .withSpannerConfig(spannerConfig)
* .withQuery("SELECT name, email FROM users")
* .withTransaction(tx));
*
* PCollection<Struct> tweets = p.apply(
* SpannerIO.read()
* .withSpannerConfig(spannerConfig)
* .withQuery("SELECT user, tweet, date FROM tweets")
* .withTransaction(tx));
* }</pre>
*
* <h4>Bulk reading of multiple queries or tables</h4>
*
* You can perform multiple consistent reads on a set of tables or using a set of queries by
* constructing a {@link ReadAll} transform using {@link SpannerIO#readAll() SpannerIO.readAll()}.
* This transform takes a {@link PCollection} of {@link ReadOperation} elements, and performs the
* partitioned read on each of them using the same Read Only Transaction for consistent results.
*
* <p>Note that this transform should <strong>not</strong> be used in Streaming pipelines. This is
* because the same Read Only Transaction, which is created once when the pipeline is first
* executed, will be used for all reads. The data being read will therefore become stale, and if no
* reads are made for more than 1 hour, the transaction will automatically timeout and be closed by
* the Spanner server, meaning that any subsequent reads will fail.
*
* <pre>{@code
* // Build a collection of ReadOperations.
* PCollection<ReadOperation> reads = ...
*
* PCollection<Struct> rows = reads.apply(
* SpannerIO.readAll()
* .withInstanceId(instanceId)
* .withDatabaseId(dbId)
* }</pre>
*
* <h3>Writing to Cloud Spanner</h3>
*
* <p>The Cloud Spanner {@link Write} transform writes to Cloud Spanner by executing a collection of
* input row {@link Mutation Mutations}. The mutations are grouped into batches for efficiency.
*
* <p>To configure the write transform, create an instance using {@link #write()} and then specify
* the destination Cloud Spanner instance ({@link Write#withInstanceId(String)} and destination
* database ({@link Write#withDatabaseId(String)}). For example:
*
* <pre>{@code
* // Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
* PCollection<Mutation> mutations = ...;
* // Write mutations.
* SpannerWriteResult result = mutations.apply(
* "Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
* }</pre>
*
* <h3>SpannerWriteResult</h3>
*
* <p>The {@link SpannerWriteResult SpannerWriteResult} object contains the results of the
* transform, including a {@link PCollection} of MutationGroups that failed to write, and a {@link
* PCollection} that can be used in batch pipelines as a completion signal to {@link Wait
* Wait.OnSignal} to indicate when all input has been written. Note that in streaming pipelines,
* this signal will never be triggered as the input is unbounded and this {@link PCollection} is
* using the {@link GlobalWindow}.
*
* <h3>Batching and Grouping</h3>
*
* <p>To reduce the number of transactions sent to Spanner, the {@link Mutation Mutations} are
* grouped into batches. The default maximum size of the batch is set to 1MB or 5000 mutated cells,
* or 500 rows (whichever is reached first). To override this use {@link
* Write#withBatchSizeBytes(long) withBatchSizeBytes()}, {@link Write#withMaxNumMutations(long)
* withMaxNumMutations()} or {@link Write#withMaxNumMutations(long) withMaxNumRows()}. Setting
* either to a small value or zero disables batching.
*
* <p>Note that the <a
* href="https://cloud.google.com/spanner/quotas#limits_for_creating_reading_updating_and_deleting_data">maximum
* size of a single transaction</a> is 20,000 mutated cells - including cells in indexes. If you
* have a large number of indexes and are getting exceptions with message: <tt>INVALID_ARGUMENT: The
* transaction contains too many mutations</tt> you will need to specify a smaller number of {@code
* MaxNumMutations}.
*
* <p>The batches written are obtained from by grouping enough {@link Mutation Mutations} from the
* Bundle provided by Beam to form several batches. This group of {@link Mutation Mutations} is then
* sorted by table and primary key, and the batches are created from the sorted group. Each batch
* will then have rows for the same table, with keys that are 'close' to each other, thus optimising
* write efficiency by each batch affecting as few table splits as possible performance.
*
* <p>This grouping factor (number of batches) is controlled by the parameter {@link
* Write#withGroupingFactor(int) withGroupingFactor()}.
*
* <p>Note that each worker will need enough memory to hold {@code GroupingFactor x
* MaxBatchSizeBytes} Mutations, so if you have a large {@code MaxBatchSize} you may need to reduce
* {@code GroupingFactor}
*
* <p>While Grouping and Batching increases write efficiency, it dramatically increases the latency
* between when a Mutation is received by the transform, and when it is actually written to the
* database. This is because enough Mutations need to be received to fill the grouped batches. In
* Batch pipelines (bounded sources), this is not normally an issue, but in Streaming (unbounded)
* pipelines, this latency is often seen as unacceptable.
*
* <p>There are therefore 3 different ways that this transform can be configured:
*
* <ul>
* <li>With Grouping and Batching. <br>
* This is the default for Batch pipelines, where sorted batches of Mutations are created and
* written. This is the most efficient way to ingest large amounts of data, but the highest
* latency before writing
* <li>With Batching but no Grouping <br>
* If {@link Write#withGroupingFactor(int) .withGroupingFactor(1)}, is set, grouping is
* disabled. This is the default for Streaming pipelines. Unsorted batches are created and
* written as soon as enough mutations to fill a batch are received. This reflects a
* compromise where a small amount of additional latency enables more efficient writes
* <li>Without any Batching <br>
* If {@link Write#withBatchSizeBytes(long) .withBatchSizeBytes(0)} is set, no batching is
* performed and the Mutations are written to the database as soon as they are received.
* ensuring the lowest latency before Mutations are written.
* </ul>
*
* <h3>Monitoring</h3>
*
* <p>Several counters are provided for monitoring purpooses:
*
* <ul>
* <li><tt>batchable_mutation_groups</tt><br>
* Counts the mutations that are batched for writing to Spanner.
* <li><tt>unbatchable_mutation_groups</tt><br>
* Counts the mutations that can not be batched and are applied individually - either because
* they are too large to fit into a batch, or they are ranged deletes.
* <li><tt>mutation_group_batches_received, mutation_group_batches_write_success,
* mutation_group_batches_write_failed</tt><br>
* Count the number of batches that are processed. If Failure Mode is set to {@link
* FailureMode#REPORT_FAILURES REPORT_FAILURES}, then failed batches will be split up and the
* individual mutation groups retried separately.
* <li><tt>mutation_groups_received, mutation_groups_write_success,
* mutation_groups_write_fail</tt><br>
* Count the number of individual MutationGroups that are processed.
* <li><tt>spanner_write_success, spanner_write_fail</tt><br>
* The number of writes to Spanner that have occurred.
* <li><tt>spanner_write_retries</tt><br>
* The number of times a write is retried after a failure - either due to a timeout, or when
* batches fail and {@link FailureMode#REPORT_FAILURES REPORT_FAILURES} is set so that
* individual Mutation Groups are retried.
* <li><tt>spanner_write_timeouts</tt><br>
* The number of timeouts that occur when writing to Spanner. Writes that timed out are
* retried after a backoff. Large numbers of timeouts suggest an overloaded Spanner instance.
* <li><tt>spanner_write_total_latency_ms</tt><br>
* The total amount of time spent writing to Spanner, in milliseconds.
* </ul>
*
* <h3>Database Schema Preparation</h3>
*
* <p>The Write transform reads the database schema on pipeline start to know which columns are used
* as primary keys of the tables and indexes. This is so that the transform knows how to sort the
* grouped Mutations by table name and primary key as described above.
*
* <p>If the database schema, any additional tables or indexes are created in the same pipeline then
* there will be a race condition, leading to a situation where the schema is read before the table
* is created its primary key will not be known. This will mean that the sorting/batching will not
* be optimal and performance will be reduced (warnings will be logged for rows using unknown
* tables)
*
* <p>To prevent this race condition, use {@link Write#withSchemaReadySignal(PCollection)} to pass a
* signal {@link PCollection} (for example the output of the transform that creates the table(s))
* which will be used with {@link Wait.OnSignal} to prevent the schema from being read until it is
* ready. The Write transform will be paused until this signal {@link PCollection} is closed.
*
* <h3>Transactions</h3>
*
* <p>The transform does not provide same transactional guarantees as Cloud Spanner. In particular,
*
* <ul>
* <li>Individual Mutations are submitted atomically, but all Mutations are not submitted in the
* same transaction.
* <li>A Mutation is applied at least once;
* <li>If the pipeline was unexpectedly stopped, mutations that were already applied will not get
* rolled back.
* </ul>
*
* <p>Use {@link MutationGroup MutationGroups} with the {@link WriteGrouped} transform to ensure
* that a small set mutations is bundled together. It is guaranteed that mutations in a {@link
* MutationGroup} are submitted in the same transaction. Note that a MutationGroup must not exceed
* the Spanner transaction limits.
*
* <pre>{@code
* // Earlier in the pipeline, create a PCollection of MutationGroups to be written to Cloud Spanner.
* PCollection<MutationGroup> mutationGroups = ...;
* // Write mutation groups.
* SpannerWriteResult result = mutationGroups.apply(
* "Write",
* SpannerIO.write().withInstanceId("instance").withDatabaseId("database").grouped());
* }</pre>
*
* <h3>Streaming Support</h3>
*
* <p>{@link Write} can be used as a streaming sink, however as with batch mode note that the write
* order of individual {@link Mutation}/{@link MutationGroup} objects is not guaranteed.
*
* <p>{@link Read} and {@link ReadAll} can be used in Streaming pipelines to read a set of Facts on
* pipeline startup.
*
* <p>{@link ReadAll} should not be used on an unbounded {@code PCollection<ReadOperation>}, for the
* reasons stated above.
*
* <h3>Updates to the I/O connector code</h3>
*
* For any significant significant updates to this I/O connector, please consider involving
* corresponding code reviewers mentioned <a
* href="https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/OWNERS">
* here</a>.
*/
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class SpannerIO {
private static final Logger LOG = LoggerFactory.getLogger(SpannerIO.class);
private static final long DEFAULT_BATCH_SIZE_BYTES = 1024L * 1024L; // 1 MB
// Max number of mutations to batch together.
private static final int DEFAULT_MAX_NUM_MUTATIONS = 5000;
// Max number of mutations to batch together.
private static final int DEFAULT_MAX_NUM_ROWS = 500;
// Multiple of mutation size to use to gather and sort mutations
private static final int DEFAULT_GROUPING_FACTOR = 1000;
// Size of caches for read/write ServiceCallMetric objects .
// This is a reasonable limit, as for reads, each worker will process very few different table
// read requests, and for writes, batching will ensure that write operations for the same
// table occur at the same time (within a bundle).
static final int METRICS_CACHE_SIZE = 100;
/**
* Creates an uninitialized instance of {@link Read}. Before use, the {@link Read} must be
* configured with a {@link Read#withInstanceId} and {@link Read#withDatabaseId} that identify the
* Cloud Spanner database.
*/
public static Read read() {
return new AutoValue_SpannerIO_Read.Builder()
.setSpannerConfig(SpannerConfig.create())
.setTimestampBound(TimestampBound.strong())
.setReadOperation(ReadOperation.create())
.setBatching(true)
.build();
}
/**
* A {@link PTransform} that works like {@link #read}, but executes read operations coming from a
* {@link PCollection}.
*/
public static ReadAll readAll() {
return new AutoValue_SpannerIO_ReadAll.Builder()
.setSpannerConfig(SpannerConfig.create())
.setTimestampBound(TimestampBound.strong())
.setBatching(true)
.build();
}
/**
* Returns a transform that creates a batch transaction. By default, {@link
* TimestampBound#strong()} transaction is created, to override this use {@link
* CreateTransaction#withTimestampBound(TimestampBound)}.
*/
public static CreateTransaction createTransaction() {
return new AutoValue_SpannerIO_CreateTransaction.Builder()
.setSpannerConfig(SpannerConfig.create())
.setTimestampBound(TimestampBound.strong())
.build();
}
/**
* Creates an uninitialized instance of {@link Write}. Before use, the {@link Write} must be
* configured with a {@link Write#withInstanceId} and {@link Write#withDatabaseId} that identify
* the Cloud Spanner database being written.
*/
public static Write write() {
return new AutoValue_SpannerIO_Write.Builder()
.setSpannerConfig(SpannerConfig.create())
.setBatchSizeBytes(DEFAULT_BATCH_SIZE_BYTES)
.setMaxNumMutations(DEFAULT_MAX_NUM_MUTATIONS)
.setMaxNumRows(DEFAULT_MAX_NUM_ROWS)
.setFailureMode(FailureMode.FAIL_FAST)
.build();
}
/**
* Creates an uninitialized instance of {@link ReadChangeStream}. Before use, the {@link
* ReadChangeStream} must be configured with a {@link ReadChangeStream#withProjectId}, {@link
* ReadChangeStream#withInstanceId}, and {@link ReadChangeStream#withDatabaseId} that identify the
* Cloud Spanner database being written. It must also be configured with the start time and the
* change stream name.
*/
public static ReadChangeStream readChangeStream() {
return new AutoValue_SpannerIO_ReadChangeStream.Builder()
.setSpannerConfig(SpannerConfig.create())
.setChangeStreamName(DEFAULT_CHANGE_STREAM_NAME)
.setRpcPriority(DEFAULT_RPC_PRIORITY)
.setInclusiveStartAt(DEFAULT_INCLUSIVE_START_AT)
.setInclusiveEndAt(DEFAULT_INCLUSIVE_END_AT)
.build();
}
/** Implementation of {@link #readAll}. */
@AutoValue
public abstract static class ReadAll
extends PTransform<PCollection<ReadOperation>, PCollection<Struct>> {
abstract SpannerConfig getSpannerConfig();
abstract @Nullable PCollectionView<Transaction> getTransaction();
abstract @Nullable TimestampBound getTimestampBound();
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setSpannerConfig(SpannerConfig spannerConfig);
abstract Builder setTransaction(PCollectionView<Transaction> transaction);
abstract Builder setTimestampBound(TimestampBound timestampBound);
abstract Builder setBatching(Boolean batching);
abstract ReadAll build();
}
/** Specifies the Cloud Spanner configuration. */
public ReadAll withSpannerConfig(SpannerConfig spannerConfig) {
return toBuilder().setSpannerConfig(spannerConfig).build();
}
/** Specifies the Cloud Spanner project. */
public ReadAll withProjectId(String projectId) {
return withProjectId(ValueProvider.StaticValueProvider.of(projectId));
}
/** Specifies the Cloud Spanner project. */
public ReadAll withProjectId(ValueProvider<String> projectId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withProjectId(projectId));
}
/** Specifies the Cloud Spanner instance. */
public ReadAll withInstanceId(String instanceId) {
return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId));
}
/** Specifies the Cloud Spanner instance. */
public ReadAll withInstanceId(ValueProvider<String> instanceId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withInstanceId(instanceId));
}
/** Specifies the Cloud Spanner database. */
public ReadAll withDatabaseId(String databaseId) {
return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId));
}
/** Specifies the Cloud Spanner host. */
public ReadAll withHost(ValueProvider<String> host) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withHost(host));
}
public ReadAll withHost(String host) {
return withHost(ValueProvider.StaticValueProvider.of(host));
}
/** Specifies the Cloud Spanner emulator host. */
public ReadAll withEmulatorHost(ValueProvider<String> emulatorHost) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withEmulatorHost(emulatorHost));
}
public ReadAll withEmulatorHost(String emulatorHost) {
return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost));
}
/** Specifies the Cloud Spanner database. */
public ReadAll withDatabaseId(ValueProvider<String> databaseId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withDatabaseId(databaseId));
}
@VisibleForTesting
ReadAll withServiceFactory(ServiceFactory<Spanner, SpannerOptions> serviceFactory) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withServiceFactory(serviceFactory));
}
public ReadAll withTransaction(PCollectionView<Transaction> transaction) {
return toBuilder().setTransaction(transaction).build();
}
public ReadAll withTimestamp(Timestamp timestamp) {
return withTimestampBound(TimestampBound.ofReadTimestamp(timestamp));
}
public ReadAll withTimestampBound(TimestampBound timestampBound) {
return toBuilder().setTimestampBound(timestampBound).build();
}
/**
* By default the <a
* href="https://cloud.google.com/spanner/docs/reads#read_data_in_parallel">PartitionQuery
* API</a> is used to read data from Cloud Spanner. It is useful to disable batching when the
* underlying query is not root-partitionable.
*/
public ReadAll withBatching(boolean batching) {
return toBuilder().setBatching(batching).build();
}
public ReadAll withLowPriority() {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withRpcPriority(RpcPriority.LOW));
}
public ReadAll withHighPriority() {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withRpcPriority(RpcPriority.HIGH));
}
abstract Boolean getBatching();
@Override
public PCollection<Struct> expand(PCollection<ReadOperation> input) {
if (PCollection.IsBounded.UNBOUNDED == input.isBounded()) {
// Warn that SpannerIO.ReadAll should not be used on unbounded inputs.
LOG.warn(
"SpannerIO.ReadAll({}) is being applied to an unbounded input. "
+ "This is not supported and can lead to runtime failures.",
this.getName());
}
PTransform<PCollection<ReadOperation>, PCollection<Struct>> readTransform;
if (getBatching()) {
readTransform =
BatchSpannerRead.create(getSpannerConfig(), getTransaction(), getTimestampBound());
} else {
readTransform =
NaiveSpannerRead.create(getSpannerConfig(), getTransaction(), getTimestampBound());
}
return input
.apply("Reshuffle", Reshuffle.viaRandomKey())
.apply("Read from Cloud Spanner", readTransform);
}
/** Helper function to create ServiceCallMetrics. */
static ServiceCallMetric buildServiceCallMetricForReadOp(
SpannerConfig config, ReadOperation op) {
HashMap<String, String> baseLabels = buildServiceCallMetricLabels(config);
baseLabels.put(MonitoringInfoConstants.Labels.METHOD, "Read");
if (op.getQuery() != null) {
String queryName = op.getQueryName();
if (queryName == null || queryName.isEmpty()) {
// if queryName is not specified, use a hash of the SQL statement string.
queryName = String.format("UNNAMED_QUERY#%08x", op.getQuery().getSql().hashCode());
}
baseLabels.put(
MonitoringInfoConstants.Labels.RESOURCE,
GcpResourceIdentifiers.spannerQuery(
baseLabels.get(MonitoringInfoConstants.Labels.SPANNER_PROJECT_ID),
config.getInstanceId().get(),
config.getDatabaseId().get(),
queryName));
baseLabels.put(MonitoringInfoConstants.Labels.SPANNER_QUERY_NAME, queryName);
} else {
baseLabels.put(
MonitoringInfoConstants.Labels.RESOURCE,
GcpResourceIdentifiers.spannerTable(
baseLabels.get(MonitoringInfoConstants.Labels.SPANNER_PROJECT_ID),
config.getInstanceId().get(),
config.getDatabaseId().get(),
op.getTable()));
baseLabels.put(MonitoringInfoConstants.Labels.TABLE_ID, op.getTable());
}
return new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels);
}
}
/** Implementation of {@link #read}. */
@AutoValue
public abstract static class Read extends PTransform<PBegin, PCollection<Struct>> {
abstract SpannerConfig getSpannerConfig();
abstract ReadOperation getReadOperation();
abstract @Nullable TimestampBound getTimestampBound();
abstract @Nullable PCollectionView<Transaction> getTransaction();
abstract @Nullable PartitionOptions getPartitionOptions();
abstract Boolean getBatching();
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setSpannerConfig(SpannerConfig spannerConfig);
abstract Builder setReadOperation(ReadOperation readOperation);
abstract Builder setTimestampBound(TimestampBound timestampBound);
abstract Builder setTransaction(PCollectionView<Transaction> transaction);
abstract Builder setPartitionOptions(PartitionOptions partitionOptions);
abstract Builder setBatching(Boolean batching);
abstract Read build();
}
/** Specifies the Cloud Spanner configuration. */
public Read withSpannerConfig(SpannerConfig spannerConfig) {
return toBuilder().setSpannerConfig(spannerConfig).build();
}
/** Specifies the Cloud Spanner project. */
public Read withProjectId(String projectId) {
return withProjectId(ValueProvider.StaticValueProvider.of(projectId));
}
/** Specifies the Cloud Spanner project. */
public Read withProjectId(ValueProvider<String> projectId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withProjectId(projectId));
}
/** Specifies the Cloud Spanner instance. */
public Read withInstanceId(String instanceId) {
return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId));
}
/** Specifies the Cloud Spanner instance. */
public Read withInstanceId(ValueProvider<String> instanceId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withInstanceId(instanceId));
}
/** Specifies the Cloud Spanner database. */
public Read withDatabaseId(String databaseId) {
return withDatabaseId(ValueProvider.StaticValueProvider.of(databaseId));
}
/** Specifies the Cloud Spanner database. */
public Read withDatabaseId(ValueProvider<String> databaseId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withDatabaseId(databaseId));
}
/** Specifies the Cloud Spanner host. */
public Read withHost(ValueProvider<String> host) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withHost(host));
}
public Read withHost(String host) {
return withHost(ValueProvider.StaticValueProvider.of(host));
}
/** Specifies the Cloud Spanner emulator host. */
public Read withEmulatorHost(ValueProvider<String> emulatorHost) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withEmulatorHost(emulatorHost));
}
public Read withEmulatorHost(String emulatorHost) {
return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost));
}
/** If true the uses Cloud Spanner batch API. */
public Read withBatching(boolean batching) {
return toBuilder().setBatching(batching).build();
}
@VisibleForTesting
Read withServiceFactory(ServiceFactory<Spanner, SpannerOptions> serviceFactory) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withServiceFactory(serviceFactory));
}
public Read withTransaction(PCollectionView<Transaction> transaction) {
return toBuilder().setTransaction(transaction).build();
}
public Read withTimestamp(Timestamp timestamp) {
return withTimestampBound(TimestampBound.ofReadTimestamp(timestamp));
}
public Read withTimestampBound(TimestampBound timestampBound) {
return toBuilder().setTimestampBound(timestampBound).build();
}
public Read withTable(String table) {
return withReadOperation(getReadOperation().withTable(table));
}
public Read withReadOperation(ReadOperation operation) {
return toBuilder().setReadOperation(operation).build();
}
public Read withColumns(String... columns) {
return withColumns(Arrays.asList(columns));
}
public Read withColumns(List<String> columns) {
return withReadOperation(getReadOperation().withColumns(columns));
}
public Read withQuery(Statement statement) {
return withReadOperation(getReadOperation().withQuery(statement));
}
public Read withQuery(String sql) {
return withQuery(Statement.of(sql));
}
public Read withQueryName(String queryName) {
return withReadOperation(getReadOperation().withQueryName(queryName));
}
public Read withKeySet(KeySet keySet) {
return withReadOperation(getReadOperation().withKeySet(keySet));
}
public Read withIndex(String index) {
return withReadOperation(getReadOperation().withIndex(index));
}
/**
* Note that {@link PartitionOptions} are currently ignored. See <a
* href="https://cloud.google.com/spanner/docs/reference/rpc/google.spanner.v1#google.spanner.v1.PartitionOptions">
* PartitionOptions in RPC documents</a>
*/
public Read withPartitionOptions(PartitionOptions partitionOptions) {
return withReadOperation(getReadOperation().withPartitionOptions(partitionOptions));
}
public Read withLowPriority() {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withRpcPriority(RpcPriority.LOW));
}
public Read withHighPriority() {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withRpcPriority(RpcPriority.HIGH));
}
@Override
public PCollection<Struct> expand(PBegin input) {
getSpannerConfig().validate();
checkArgument(
getTimestampBound() != null,
"SpannerIO.read() runs in a read only transaction and requires timestamp to be set "
+ "with withTimestampBound or withTimestamp method");
if (getReadOperation().getQuery() != null) {
// TODO: validate query?
} else if (getReadOperation().getTable() != null) {
// Assume read
checkNotNull(
getReadOperation().getColumns(),
"For a read operation SpannerIO.read() requires a list of "
+ "columns to set with withColumns method");
checkArgument(
!getReadOperation().getColumns().isEmpty(),
"For a read operation SpannerIO.read() requires a non-empty"
+ " list of columns to set with withColumns method");
} else {
throw new IllegalArgumentException(
"SpannerIO.read() requires configuring query or read operation.");
}
ReadAll readAll =
readAll()
.withSpannerConfig(getSpannerConfig())
.withTimestampBound(getTimestampBound())
.withBatching(getBatching())
.withTransaction(getTransaction());
return input.apply(Create.of(getReadOperation())).apply("Execute query", readAll);
}
SerializableFunction<Struct, Row> getFormatFn() {
return (SerializableFunction<Struct, Row>)
input ->
Row.withSchema(Schema.builder().addInt64Field("Key").build())
.withFieldValue("Key", 3L)
.build();
}
}
static class ReadRows extends PTransform<PBegin, PCollection<Row>> {
Read read;
Schema schema;
public ReadRows(Read read, Schema schema) {
super("Read rows");
this.read = read;
this.schema = schema;
}
@Override
public PCollection<Row> expand(PBegin input) {
return input
.apply(read)
.apply(
MapElements.into(TypeDescriptor.of(Row.class))
.via(
(SerializableFunction<Struct, Row>)
struct -> StructUtils.structToBeamRow(struct, schema)))
.setRowSchema(schema);
}
}
/**
* A {@link PTransform} that create a transaction. If applied to a {@link PCollection}, it will
* create a transaction after the {@link PCollection} is closed.
*
* @see SpannerIO
* @see Wait
*/
@AutoValue
public abstract static class CreateTransaction
extends PTransform<PInput, PCollectionView<Transaction>> {
abstract SpannerConfig getSpannerConfig();
abstract @Nullable TimestampBound getTimestampBound();
abstract Builder toBuilder();
@Override
public PCollectionView<Transaction> expand(PInput input) {
getSpannerConfig().validate();
PCollection<?> collection = input.getPipeline().apply(Create.of(1));
if (input instanceof PCollection) {
collection = collection.apply(Wait.on((PCollection<?>) input));
} else if (!(input instanceof PBegin)) {
throw new RuntimeException("input must be PBegin or PCollection");
}
return collection
.apply(
"Create transaction",
ParDo.of(new CreateTransactionFn(this.getSpannerConfig(), this.getTimestampBound())))
.apply("As PCollectionView", View.asSingleton());
}
/** Specifies the Cloud Spanner configuration. */
public CreateTransaction withSpannerConfig(SpannerConfig spannerConfig) {
return toBuilder().setSpannerConfig(spannerConfig).build();
}
/** Specifies the Cloud Spanner project. */
public CreateTransaction withProjectId(String projectId) {
return withProjectId(ValueProvider.StaticValueProvider.of(projectId));
}
/** Specifies the Cloud Spanner project. */
public CreateTransaction withProjectId(ValueProvider<String> projectId) {
SpannerConfig config = getSpannerConfig();
return withSpannerConfig(config.withProjectId(projectId));
}
/** Specifies the Cloud Spanner instance. */
public CreateTransaction withInstanceId(String instanceId) {
return withInstanceId(ValueProvider.StaticValueProvider.of(instanceId));
}