-
Notifications
You must be signed in to change notification settings - Fork 7
/
Copy pathBigQueryEventConsumer.java
1666 lines (1559 loc) · 80 KB
/
BigQueryEventConsumer.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
/*
* Copyright © 2019 Cask Data, Inc.
*
* Licensed 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 io.cdap.delta.bigquery;
import avro.shaded.com.google.common.collect.ImmutableList;
import com.google.cloud.bigquery.BigQuery;
import com.google.cloud.bigquery.BigQueryError;
import com.google.cloud.bigquery.BigQueryException;
import com.google.cloud.bigquery.Clustering;
import com.google.cloud.bigquery.DatasetId;
import com.google.cloud.bigquery.DatasetInfo;
import com.google.cloud.bigquery.EncryptionConfiguration;
import com.google.cloud.bigquery.Field;
import com.google.cloud.bigquery.FieldList;
import com.google.cloud.bigquery.FormatOptions;
import com.google.cloud.bigquery.Job;
import com.google.cloud.bigquery.JobId;
import com.google.cloud.bigquery.JobInfo;
import com.google.cloud.bigquery.JobStatus;
import com.google.cloud.bigquery.LoadJobConfiguration;
import com.google.cloud.bigquery.QueryJobConfiguration;
import com.google.cloud.bigquery.StandardTableDefinition;
import com.google.cloud.bigquery.Table;
import com.google.cloud.bigquery.TableDefinition;
import com.google.cloud.bigquery.TableId;
import com.google.cloud.bigquery.TableInfo;
import com.google.cloud.storage.BlobId;
import com.google.cloud.storage.Bucket;
import com.google.cloud.storage.Storage;
import com.google.common.annotations.VisibleForTesting;
import com.google.gson.Gson;
import io.cdap.cdap.api.common.Bytes;
import io.cdap.cdap.api.data.schema.Schema;
import io.cdap.delta.api.DDLEvent;
import io.cdap.delta.api.DDLOperation;
import io.cdap.delta.api.DMLEvent;
import io.cdap.delta.api.DeltaFailureException;
import io.cdap.delta.api.DeltaTargetContext;
import io.cdap.delta.api.EventConsumer;
import io.cdap.delta.api.Offset;
import io.cdap.delta.api.ReplicationError;
import io.cdap.delta.api.Sequenced;
import io.cdap.delta.api.SortKey;
import io.cdap.delta.api.SourceProperties;
import net.jodah.failsafe.Failsafe;
import net.jodah.failsafe.FailsafeException;
import net.jodah.failsafe.RetryPolicy;
import net.jodah.failsafe.TimeoutExceededException;
import net.jodah.failsafe.event.ExecutionAttemptedEvent;
import net.jodah.failsafe.function.ContextualRunnable;
import org.apache.twill.common.Threads;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.time.Duration;
import java.time.temporal.ChronoUnit;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
/**
* Consumes change events and applies them to BigQuery.
* <p>
* Writes to BigQuery in three steps.
* <p>
* Step 1 - Write a batch of changes to GCS
* <p>
* Each batch of changes is written to GCS as an object with path:
* <p>
* [staging bucket]/cdap/cdc/[app name]/[table id]/[batch id]
* <p>
* Batch id is the timestamp that the first event in the batch was processed.
* The size of the batch is determined through configuration.
* There is a maximum number of rows to include in each batch and a maximum amount of time to wait in between batches.
* Each object is written in avro format and contains the columns in the destination table plus two additional columns:
* _op: CREATE | UPDATE | DELETE
* _batch_id: the batch id
* <p>
* Changes in the batch do not span across a DDL event, so they are guaranteed to conform to the same schema.
* Failure scenarios are:
* <p>
* 1. The program dies after the object is written, but before the offset is persisted.
* When the program starts up again, events for the batch will be replayed.
* The consumer will not know which events are duplicates, so duplicate events will be written out to GCS.
* This will not matter because of the behavior of later steps.
* 2. The program dies before the object is written, which is always before the offset is persisted.
* In this case, nothing was ever persisted to GCS and everything behaves as if it was the first time
* the events were seen.
* 3. The write to GCS fails for some reason. For example, permissions were revoked, quota was hit,
* there was a temporary outage, etc. In this scenario, the write will be repeatedly retried until it
* succeeds. It may need manual intervention to succeed.
* <p>
* Step 2 - Load data from GCS into staging BigQuery table
* <p>
* This step happens after the offset from Step 1 is successfully persisted. This will load the object
* into a staging table in BigQuery. The staging table has the same schema as the rows in the GCS object.
* It is clustered on _batch_id in order to make reads and deletes on the _batch_id efficient.
* The job id for the load is of the form [app name]_stage_[dataset]_[table]_[batch id]_[retry num].
* Failure scenarios are:
* <p>
* 1. The load job fails for some reason. For example, permissions were revoked, quota was hit, temporary outage, etc.
* The load will be repeatedly retried until is succeeds. It may need manual intervention to succeed.
* 2. The program dies. When the program starts up again, events will be replayed from the last committed offset.
* <p>
* Step 3 - Merge a batch of data from the staging BigQuery table into the target table
* <p>
* This step happens after the load job to the staging table has succeeded. The consumer runs a merge query of the form:
* <p>
* MERGE [dataset].[target table] as T
* USING (SELECT * FROM [dataset].[staging table] WHERE _batch_id = [batch id]) as S
* ON [row equality condition]
* WHEN MATCHED AND S._OP = "DELETE"
* DELETE
* WHEN MATCHED AND S._OP = "UPDATE"
* UPDATE(...)
* WHEN NOT MATCHED AND S._OP = "INSERT"
* INSERT(...)
* VALUES(...)
* <p>
* The job id is of the form [app name]_merge_[dataset]_[table]_[batch id]_[retry_num].
* This query ensures that it does not matter if there are duplicate events in the batch objects on GCS.
* Duplicate inserts and deletes will not match and be ignored.
* Duplicate updates will update the target row to be the same that it already is.
* Once the job succeeds, the corresponding GCS object is deleted and the offset of the latest event is committed.
* Failure scenarios are:
* <p>
* 1. The merge query fails for some reason. The consumer will retry until it succeeds.
* It may need manual intervention to succeed.
* 2. The program dies. Events are replayed from the last committed offset when the program starts back up.
* 3. The GCS delete fails. The error is logged, but the consumer proceeds on.
* Manual deletion of the object is required.
*/
public class BigQueryEventConsumer implements EventConsumer {
private static final Logger LOG = LoggerFactory.getLogger(BigQueryEventConsumer.class);
private static final Gson GSON = new Gson();
private static final String RETAIN_STAGING_TABLE = "retain.staging.table";
private static final String DIRECT_LOADING_IN_PROGRESS_PREFIX = "bigquery-direct-load-in-progress-";
private final DeltaTargetContext context;
private final BigQuery bigQuery;
private final int loadIntervalSeconds;
private final String stagingTablePrefix;
private final MultiGCSWriter gcsWriter;
private final Bucket bucket;
private final String project;
private final EncryptionConfiguration encryptionConfig;
private final RetryPolicy<Object> commitRetryPolicy;
private final Map<String, Long> latestSeenSequence;
private final Map<String, Long> latestMergedSequence;
private final Map<TableId, List<String>> primaryKeyStore;
private final Map<TableId, SortKeyState> sortKeyStore;
private final boolean requireManualDrops;
private final long baseRetryDelay;
private final int maxClusteringColumns;
private final boolean sourceRowIdSupported;
private final SourceProperties.Ordering sourceEventOrdering;
private final String datasetName;
private final boolean retainStagingTable;
private final boolean softDeletesEnabled;
private ScheduledExecutorService scheduledExecutorService;
private ScheduledFuture<?> scheduledFlush;
private ExecutorService executorService;
private Offset latestOffset;
private long latestSequenceNum;
private Exception flushException;
private final AtomicBoolean shouldStop;
private final SchemaMappingCache schemaMappingCache;
private RetryPolicy<Object> gcsWriterRetryPolicy = new RetryPolicy<>()
.withMaxAttempts(25)
.withMaxDuration(Duration.of(2, ChronoUnit.MINUTES))
.withBackoff(1, 30, ChronoUnit.SECONDS)
.withJitter(0.1);
// have to keep all the records in memory in case there is a failure writing to GCS
// cannot write to a temporary file on local disk either in case there is a failure writing to disk
// Without keeping the entire batch in memory, there would be no way to recover the records that failed to write
BigQueryEventConsumer(DeltaTargetContext context, Storage storage, BigQuery bigQuery, Bucket bucket,
String project, int loadIntervalSeconds, String stagingTablePrefix, boolean requireManualDrops,
@Nullable EncryptionConfiguration encryptionConfig, @Nullable Long baseRetryDelay,
@Nullable String datasetName, boolean softDeletesEnabled) {
this.context = context;
this.bigQuery = bigQuery;
this.loadIntervalSeconds = loadIntervalSeconds;
this.stagingTablePrefix = stagingTablePrefix;
this.bucket = bucket;
this.project = project;
this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
this.latestSequenceNum = 0L;
this.encryptionConfig = encryptionConfig;
// these maps are only accessed in synchronized methods so they do not need to be thread safe.
this.latestMergedSequence = new HashMap<>();
this.latestSeenSequence = new HashMap<>();
this.primaryKeyStore = new HashMap<>();
this.sortKeyStore = new HashMap<>();
this.schemaMappingCache = new SchemaMappingCache();
this.commitRetryPolicy = new RetryPolicy<>()
.withMaxAttempts(Integer.MAX_VALUE)
.withMaxDuration(Duration.of(5, ChronoUnit.MINUTES))
.withBackoff(1, 60, ChronoUnit.SECONDS)
.onFailedAttempt(failureContext -> {
// log on the first failure and then every fifth failed attempt
if (failureContext.getAttemptCount() == 1 || failureContext.getAttemptCount() % 5 == 0) {
LOG.warn("Error committing offset (Attempt:{}). Changes will be blocked until this succeeds, ",
failureContext.getAttemptCount(), failureContext.getLastFailure());
}
})
.onSuccess(successContext -> {
if (successContext.getAttemptCount() > 1) {
LOG.info("Commited offset successfully after {} retries", (successContext.getAttemptCount() - 1));
}
});
this.requireManualDrops = requireManualDrops;
this.executorService = Executors.newCachedThreadPool(Threads.createDaemonThreadFactory("bq-daemon-%d"));
this.gcsWriter = new MultiGCSWriter(storage, bucket.getName(),
String.format("cdap/delta/%s/", context.getApplicationName()),
context, executorService);
this.baseRetryDelay = baseRetryDelay == null ? 10L : baseRetryDelay;
String maxClusteringColumnsStr = context.getRuntimeArguments().get("gcp.bigquery.max.clustering.columns");
// current max clustering columns is set as 4 in big query side, use that as default max value
// https://cloud.google.com/bigquery/docs/creating-clustered-tables#limitations
this.maxClusteringColumns = maxClusteringColumnsStr == null ? 4 : Integer.parseInt(maxClusteringColumnsStr);
this.sourceRowIdSupported =
context.getSourceProperties() != null && context.getSourceProperties().isRowIdSupported();
this.sourceEventOrdering = context.getSourceProperties() == null ? SourceProperties.Ordering.ORDERED :
context.getSourceProperties().getOrdering();
this.datasetName = datasetName;
this.retainStagingTable = Boolean.parseBoolean(context.getRuntimeArguments().get(RETAIN_STAGING_TABLE));
this.softDeletesEnabled = softDeletesEnabled;
this.shouldStop = new AtomicBoolean(false);
}
@Override
public void start() {
scheduledFlush = scheduledExecutorService.scheduleAtFixedRate(() -> {
try {
flush();
} catch (InterruptedException e) {
// just return and let things end
Thread.currentThread().interrupt();
} catch (Exception e) {
flushException = e;
}
}, loadIntervalSeconds, loadIntervalSeconds, TimeUnit.SECONDS);
}
@Override
public void stop() {
if (scheduledFlush != null) {
scheduledFlush.cancel(true);
}
scheduledExecutorService.shutdownNow();
executorService.shutdownNow();
shouldStop.set(true);
try {
scheduledExecutorService.awaitTermination(10, TimeUnit.SECONDS);
executorService.awaitTermination(10, TimeUnit.SECONDS);
} catch (InterruptedException e) {
// just return and let everything end
Thread.currentThread().interrupt();
}
}
@Override
public synchronized void applyDDL(Sequenced<DDLEvent> sequencedEvent) throws Exception {
// this is non-null if an error happened during a time scheduled flush
if (flushException != null) {
throw flushException;
}
DDLEvent event = sequencedEvent.getEvent();
DDLOperation ddlOperation = event.getOperation();
String normalizedDatabaseName = BigQueryUtils.getNormalizedDatasetName(datasetName,
event.getOperation().getDatabaseName());
String normalizedTableName = BigQueryUtils.normalizeTableName(ddlOperation.getTableName());
String normalizedStagingTableName = normalizedTableName == null ? null :
BigQueryUtils.normalizeTableName(stagingTablePrefix + normalizedTableName);
RetryPolicy<Object> retryPolicy = createBaseRetryPolicy(baseRetryDelay)
.abortOn(ex -> ex instanceof DeltaFailureException)
.onFailedAttempt(failureContext ->
handleBigQueryFailure(normalizedDatabaseName, event.getOperation().getSchemaName(), normalizedTableName,
String.format("Failed to apply '%s' DDL event", GSON.toJson(event)), failureContext)
);
runWithRetryPolicy(
ctx -> {
try {
handleDDL(event, normalizedDatabaseName, normalizedTableName, normalizedStagingTableName);
} catch (BigQueryException ex) {
logBigQueryError(ex);
if (isInvalidOperationError(ex)) {
//Unsupported DDL Operation
throw new DeltaFailureException("Non recoverable error in applying DDL event, aborting", ex);
}
throw ex;
}
},
String.format("Exhausted retries trying to apply '%s' DDL event",
event.getOperation()), retryPolicy
);
latestOffset = event.getOffset();
if (LOG.isTraceEnabled()) {
LOG.trace("DDL offset: {} seq num: {}", latestOffset.get(), latestSequenceNum);
}
context.incrementCount(event.getOperation());
if (event.isSnapshot()) {
context.setTableSnapshotting(normalizedDatabaseName, normalizedTableName);
} else {
context.setTableReplicating(normalizedDatabaseName, normalizedTableName);
}
}
private void handleDDL(DDLEvent event, String normalizedDatabaseName, String normalizedTableName,
String normalizedStagingTableName)
throws IOException, DeltaFailureException, InterruptedException {
LOG.info("DDL Event={}", event);
if (LOG.isDebugEnabled() && event.getSchema() != null) {
LOG.debug("Schema={}", event.getSchema());
}
switch (event.getOperation().getType()) {
case CREATE_DATABASE:
DatasetId datasetId = DatasetId.of(project, normalizedDatabaseName);
if (bigQuery.getDataset(datasetId) == null) {
DatasetInfo datasetInfo = DatasetInfo.newBuilder(datasetId).setLocation(bucket.getLocation()).build();
try {
bigQuery.create(datasetInfo);
} catch (BigQueryException e) {
// It is possible that in multiple worker instances scenario
// dataset is created by another worker instance after this worker instance
// determined that dataset does not exists. Ignore error if dataset is created.
if (e.getCode() != BigQueryTarget.CONFLICT) {
throw e;
}
}
}
break;
case DROP_DATABASE:
datasetId = DatasetId.of(project, normalizedDatabaseName);
primaryKeyStore.clear();
if (bigQuery.getDataset(datasetId) != null) {
if (requireManualDrops) {
String message = String.format("Encountered an event to drop dataset '%s' in project '%s', " +
"but the target is configured to require manual drops. " +
"Please manually drop the dataset to make progress.",
normalizedDatabaseName, project);
LOG.error(message);
throw new RuntimeException(message);
}
bigQuery.delete(datasetId);
}
break;
case CREATE_TABLE:
TableId tableId = TableId.of(project, normalizedDatabaseName, normalizedTableName);
Table table = bigQuery.getTable(tableId);
// SNAPSHOT data is directly loaded in the target table. Check if any such direct load was in progress
// for the current table when target received CREATE_TABLE ddl. This indicates that the snapshot was abandoned
// because of some failure scenario. Delete the existing table if any.
byte[] state = context.getState(String.format(DIRECT_LOADING_IN_PROGRESS_PREFIX + "%s-%s",
normalizedDatabaseName, normalizedTableName));
if (table != null && state != null && state.length != 0 && Bytes.toBoolean(state)) {
bigQuery.delete(tableId);
}
List<String> primaryKeys = event.getPrimaryKey();
List<String> normalizedPrimaryKeys = primaryKeys.stream()
.map(BigQueryUtils::normalizeFieldName)
.collect(Collectors.toList());
updatePrimaryKeys(tableId, normalizedPrimaryKeys);
// TODO: check schema of table if it exists already
if (table == null) {
TableDefinition tableDefinition = StandardTableDefinition.newBuilder()
.setSchema(Schemas.convert(addSupplementaryColumnsToTargetSchema(event.getSchema(), tableId)))
.setClustering(getClustering(event.getSchema(), primaryKeys))
.build();
TableInfo.Builder builder = TableInfo.newBuilder(tableId, tableDefinition);
if (encryptionConfig != null) {
builder.setEncryptionConfiguration(encryptionConfig);
}
TableInfo tableInfo = builder.build();
bigQuery.create(tableInfo);
}
break;
case DROP_TABLE:
// need to flush changes before dropping the table, otherwise the next flush will write data that
// shouldn't exist
flush();
tableId = TableId.of(project, normalizedDatabaseName, normalizedTableName);
primaryKeyStore.remove(tableId);
table = bigQuery.getTable(tableId);
if (table != null) {
if (requireManualDrops) {
String message = String.format("Encountered an event to drop table '%s' in dataset '%s' in project '%s', " +
"but the target is configured to require manual drops. " +
"Please manually drop the table to make progress.",
normalizedTableName, normalizedDatabaseName, project);
LOG.error(message);
throw new RuntimeException(message);
}
bigQuery.delete(tableId);
}
TableId stagingTableId = TableId.of(project, normalizedDatabaseName, normalizedStagingTableName);
Table stagingTable = bigQuery.getTable(stagingTableId);
if (stagingTable != null) {
bigQuery.delete(stagingTableId);
}
break;
case ALTER_TABLE:
// need to flush any changes before altering the table to ensure all changes before the schema change
// are in the table when it is altered.
flush();
schemaMappingCache.reset();
// after a flush, the staging table will be gone, so no need to alter it.
tableId = TableId.of(project, normalizedDatabaseName, normalizedTableName);
table = bigQuery.getTable(tableId);
primaryKeys = event.getPrimaryKey();
TableDefinition tableDefinition = StandardTableDefinition.newBuilder()
.setSchema(Schemas.convert(addSupplementaryColumnsToTargetSchema(event.getSchema(), tableId)))
.setClustering(getClustering(event.getSchema(), primaryKeys))
.build();
TableInfo.Builder builder = TableInfo.newBuilder(tableId, tableDefinition);
if (encryptionConfig != null) {
builder.setEncryptionConfiguration(encryptionConfig);
}
TableInfo tableInfo = builder.build();
if (table == null) {
bigQuery.create(tableInfo);
} else {
bigQuery.update(tableInfo);
}
normalizedPrimaryKeys = primaryKeys.stream()
.map(BigQueryUtils::normalizeFieldName)
.collect(Collectors.toList());
updatePrimaryKeys(tableId, normalizedPrimaryKeys);
break;
case RENAME_TABLE:
// TODO: flush changes, execute a copy job, delete previous table, drop old staging table, remove old entry
// in primaryKeyStore, put new entry in primaryKeyStore
LOG.warn("Rename DDL events are not supported. Ignoring rename event in database {} from table {} to table {}.",
event.getOperation().getDatabaseName(), event.getOperation().getPrevTableName(),
event.getOperation().getTableName());
break;
case TRUNCATE_TABLE:
flush();
tableId = TableId.of(project, normalizedDatabaseName, normalizedTableName);
table = bigQuery.getTable(tableId);
if (table != null) {
tableDefinition = table.getDefinition();
bigQuery.delete(tableId);
} else {
primaryKeys = event.getPrimaryKey();
tableDefinition = StandardTableDefinition.newBuilder()
.setSchema(Schemas.convert(addSupplementaryColumnsToTargetSchema(event.getSchema(), tableId)))
.setClustering(getClustering(event.getSchema(), primaryKeys))
.build();
}
builder = TableInfo.newBuilder(tableId, tableDefinition);
if (encryptionConfig != null) {
builder.setEncryptionConfiguration(encryptionConfig);
}
tableInfo = builder.build();
bigQuery.create(tableInfo);
break;
}
}
@Nullable
private Clustering getClustering(Schema recordSchema, List<String> primaryKeys) {
List<String> clusteringSupportedKeys = getClusteringSupportedKeys(primaryKeys, recordSchema);
Clustering clustering = maxClusteringColumns <= 0 || clusteringSupportedKeys.isEmpty() ? null :
Clustering.newBuilder()
.setFields(clusteringSupportedKeys.subList(0, Math.min(maxClusteringColumns,
clusteringSupportedKeys.size())))
.build();
return clustering;
}
@VisibleForTesting
static List<String> getClusteringSupportedKeys(List<String> primaryKeys, Schema recordSchema) {
List<String> result = new ArrayList<>();
for (String key : primaryKeys) {
if (Schemas.isClusteringSupported(recordSchema.getField(key))) {
result.add(BigQueryUtils.normalizeFieldName(key));
}
}
return result;
}
private void updatePrimaryKeys(TableId tableId, List<String> primaryKeys) throws DeltaFailureException, IOException {
if (primaryKeys.isEmpty()) {
throw new DeltaFailureException(
String.format("Table '%s' in database '%s' has no primary key. Tables without a primary key are" +
" not supported.", tableId.getTable(), tableId.getDataset()));
}
List<String> existingKey = primaryKeyStore.get(tableId);
if (primaryKeys.equals(existingKey)) {
return;
}
primaryKeyStore.put(tableId, primaryKeys);
context.putState(getTableStateKey(tableId),
Bytes.toBytes(GSON.toJson(new BigQueryTableState(primaryKeys, getSortKeys(tableId)))));
}
private List<String> getPrimaryKeys(TableId targetTableId) throws IOException, DeltaFailureException {
List<String> primaryKeys = primaryKeyStore.get(targetTableId);
if (primaryKeys == null) {
byte[] stateBytes = context.getState(getTableStateKey(targetTableId));
if (stateBytes == null || stateBytes.length == 0) {
throw new DeltaFailureException(
String.format("Primary key information for table '%s' in dataset '%s' could not be found. This can only " +
"happen if state was corrupted. Please create a new replicator and start again.",
targetTableId.getTable(), targetTableId.getDataset()));
}
BigQueryTableState targetTableState = GSON.fromJson(new String(stateBytes), BigQueryTableState.class);
primaryKeys = targetTableState.getPrimaryKeys();
primaryKeyStore.put(targetTableId, primaryKeys);
}
return primaryKeys;
}
private Schema addSupplementaryColumnsToTargetSchema(Schema original, TableId tableId) throws IOException {
List<Schema.Field> fields = new ArrayList<>(original.getFields().size() + 4);
fields.add(Schema.Field.of(Constants.SEQUENCE_NUM, Schema.of(Schema.Type.LONG)));
fields.add(Schema.Field.of(Constants.IS_DELETED, Schema.nullableOf(Schema.of(Schema.Type.BOOLEAN))));
fields.add(Schema.Field.of(Constants.ROW_ID, Schema.nullableOf(Schema.of(Schema.Type.STRING))));
fields.add(Schema.Field.of(Constants.SOURCE_TIMESTAMP, Schema.nullableOf(Schema.of(Schema.Type.LONG))));
if (sourceEventOrdering == SourceProperties.Ordering.UN_ORDERED) {
List<Schema.Type> sortKeys = getSortKeys(tableId);
if (sortKeys != null) {
fields.add(Schema.Field.of(Constants.SORT_KEYS, Schemas.getSortKeysSchema(sortKeys)));
}
}
fields.addAll(original.getFields());
return Schema.recordOf(original.getRecordName() + ".sequenced", fields);
}
private void commitOffset() throws DeltaFailureException {
try {
Failsafe.with(commitRetryPolicy).run(() -> {
if (latestOffset != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Committing offset : {} and seq num: {}", latestOffset.get(), latestSequenceNum);
}
context.commitOffset(latestOffset, latestSequenceNum);
}
});
} catch (Exception e) {
throw new DeltaFailureException(e.getMessage(), e);
}
}
@Override
public synchronized void applyDML(Sequenced<DMLEvent> sequencedEvent) throws Exception {
// this is non-null if an error happened during a time scheduled flush
if (flushException != null) {
throw flushException;
}
DMLEvent event = sequencedEvent.getEvent();
long sequenceNumber = sequencedEvent.getSequenceNumber();
if (LOG.isTraceEnabled()) {
LOG.trace("DML Event={}, sequenceNumber={}", GSON.toJson(event), sequenceNumber);
} else if (LOG.isDebugEnabled()) {
LOG.debug("DML Event={}, sequenceNumber={}", event, sequenceNumber);
}
String normalizedDatabaseName = BigQueryUtils.getNormalizedDatasetName(datasetName,
event.getOperation().getDatabaseName());
String normalizedTableName = BigQueryUtils.normalizeTableName(event.getOperation().getTableName());
DMLEvent normalizedDMLEvent = BigQueryUtils.normalize(event, schemaMappingCache)
.setDatabaseName(normalizedDatabaseName)
.setTableName(normalizedTableName)
.build();
TableId tableId = TableId.of(project, normalizedDatabaseName, normalizedTableName);
String tableKey = getTableKey(tableId);
Long latestMergedSequencedNum = latestMergedSequence.get(tableKey);
if (latestMergedSequencedNum == null) {
// first event of the table
latestMergedSequencedNum = getLatestSequenceNum(tableId);
latestMergedSequence.put(tableKey, latestMergedSequencedNum);
// latestSeenSequence will replace the latestMergedSequence at the end of flush()
// set this default value to avoid dup query of max merged sequence num in next `flush()`
latestSeenSequence.put(tableKey, latestMergedSequencedNum);
}
// it's possible that some previous events were merged to target table but offset were not committed
// because offset is committed when the whole batch of all the tables were merged.
// so it's possible we see an event that was already merged to target table
if (sequenceNumber > latestMergedSequencedNum) {
latestSeenSequence.put(tableKey, sequenceNumber);
//Only write events which have not already been applied
Failsafe.with(gcsWriterRetryPolicy)
.run(() -> gcsWriter.write(new Sequenced<>(normalizedDMLEvent, sequenceNumber)));
}
latestOffset = event.getOffset();
latestSequenceNum = sequenceNumber;
context.incrementCount(event.getOperation());
if (event.isSnapshot()) {
context.setTableSnapshotting(normalizedDatabaseName, normalizedTableName);
} else {
context.setTableReplicating(normalizedDatabaseName, normalizedTableName);
}
if (sourceEventOrdering == SourceProperties.Ordering.UN_ORDERED && !getCachedSortKeys(tableId).isPresent()) {
storeSortKeys(tableId, event.getSortKeys());
}
}
private String getTableKey(TableId tableId) {
return tableId.getDataset() + ":" + tableId.getTable();
}
@VisibleForTesting
synchronized void flush() throws InterruptedException, IOException, DeltaFailureException {
Map<MultiGCSWriter.BlobType, Collection<TableBlob>> tableBlobsByBlobType;
// if this throws an IOException, we want to propagate it, since we need the app to reset state to the last
// commit and replay events. This is because previous events are written directly to an outputstream to GCS
// and then dropped, so we cannot simply retry the flush here.
try {
tableBlobsByBlobType = gcsWriter.flush();
} catch (IOException e) {
flushException = e;
throw e;
}
processBlobsInParallel(tableBlobsByBlobType.get(MultiGCSWriter.BlobType.SNAPSHOT));
processBlobsInParallel(tableBlobsByBlobType.get(MultiGCSWriter.BlobType.STREAMING));
latestMergedSequence.clear();
latestMergedSequence.putAll(latestSeenSequence);
commitOffset();
}
private void processBlobsInParallel(Collection<TableBlob> tableBlobs)
throws InterruptedException, DeltaFailureException {
List<Future<?>> mergeFutures = new ArrayList<>(tableBlobs.size());
for (TableBlob blob : tableBlobs) {
// submit a callable instead of a runnable so that it can throw checked exceptions
mergeFutures.add(executorService.submit((Callable<Void>) () -> {
if (blob.isSnapshotOnly()) {
context.putState(String.format(DIRECT_LOADING_IN_PROGRESS_PREFIX + "%s-%s", blob.getDataset(),
blob.getTable()),
Bytes.toBytes(true));
directLoadToTarget(blob);
} else {
context.putState(String.format(DIRECT_LOADING_IN_PROGRESS_PREFIX + "%s-%s", blob.getDataset(),
blob.getTable()),
Bytes.toBytes(false));
mergeTableChanges(blob);
}
return null;
}));
}
DeltaFailureException exception = null;
for (Future<?> mergeFuture : mergeFutures) {
try {
getMergeFuture(mergeFuture);
} catch (InterruptedException e) {
throw e;
} catch (DeltaFailureException e) {
if (exception != null) {
exception.addSuppressed(e);
} else {
exception = e;
}
}
}
if (exception != null) {
throw exception;
}
}
private void directLoadToTarget(TableBlob blob) throws Exception {
LOG.debug("Direct loading batch {} of {} events into target table {}.{}", blob.getBatchId(), blob.getNumEvents(),
blob.getDataset(), blob.getTable());
TableId targetTableId = TableId.of(project, blob.getDataset(), blob.getTable());
long retryDelay = Math.min(91, context.getMaxRetrySeconds()) - 1;
runWithRetries(runContext -> loadTable(targetTableId, blob, JobType.LOAD_TARGET, runContext.getAttemptCount()),
retryDelay,
blob.getDataset(),
blob.getSourceDbSchemaName(),
blob.getTable(),
String.format("Failed to load a batch of changes from GCS into target table for %s.%s",
blob.getDataset(), blob.getTable()),
"Exhausted retries while attempting to load changed to the staging table.");
try {
blob.getBlob().delete();
} catch (Exception e) {
// there is no retry for this cleanup error since it will not affect future functionality.
LOG.warn("Failed to delete temporary GCS object {} in bucket {}. The object will need to be manually deleted.",
blob.getBlob().getBlobId().getName(), blob.getBlob().getBlobId().getBucket(), e);
}
LOG.debug("Completed direct loading of batch {} of {} events into target table {}.{}", blob.getBatchId(),
blob.getNumEvents(), blob.getDataset(), blob.getTable());
}
private void mergeTableChanges(TableBlob blob) throws DeltaFailureException, InterruptedException {
String normalizedStagingTableName = BigQueryUtils.normalizeTableName(stagingTablePrefix + blob.getTable());
TableId stagingTableId = TableId.of(project, blob.getDataset(), normalizedStagingTableName);
long retryDelay = Math.min(91, context.getMaxRetrySeconds()) - 1;
runWithRetries(runContext -> loadTable(stagingTableId, blob, JobType.LOAD_STAGING, runContext.getAttemptCount()),
retryDelay,
blob.getDataset(),
blob.getSourceDbSchemaName(),
blob.getTable(),
String.format("Failed to load a batch of changes from GCS into staging table for %s.%s",
blob.getDataset(), blob.getTable()),
"Exhausted retries while attempting to load changed to the staging table.");
runWithRetries(runContext -> mergeStagingTable(stagingTableId, blob, runContext.getAttemptCount()),
retryDelay,
blob.getDataset(),
blob.getSourceDbSchemaName(),
blob.getTable(),
String.format("Failed to merge a batch of changes from the staging table into %s.%s",
blob.getDataset(), blob.getTable()),
String.format("Exhausted retries while attempting to merge changes into target table %s.%s. "
+ "Check that the service account has the right permissions "
+ "and the table was not modified.", blob.getDataset(), blob.getTable()));
try {
blob.getBlob().delete();
} catch (Exception e) {
// there is no retry for this cleanup error since it will not affect future functionality.
LOG.warn("Failed to delete temporary GCS object {} in bucket {}. The object will need to be manually deleted.",
blob.getBlob().getBlobId().getName(), blob.getBlob().getBlobId().getBucket(), e);
}
// clean up staging table after merging is done, there is no retry for this clean up since it will not affect
// future functionality
if (!retainStagingTable) {
bigQuery.delete(stagingTableId);
}
}
private void loadTable(TableId tableId, TableBlob blob, JobType jobType, int attemptNumber)
throws InterruptedException, IOException, DeltaFailureException {
LOG.info("Loading batch {} of {} events into {} table for {}.{} {}", blob.getBatchId(), blob.getNumEvents(),
jobType.isForTargetTable() ? "target" : "staging", blob.getDataset(), blob.getTable(),
attemptNumber > 0 ? "attempt: " + attemptNumber : "");
Job loadJob = null;
if (attemptNumber > 0) {
// Check if any job from previous attempts was successful to avoid loading the same data multiple times
// which can lead to data inconsistency
loadJob = getPreviousJobIfNotFailed(blob, attemptNumber, jobType);
}
if (loadJob == null) {
loadJob = createLoadJob(tableId, blob, attemptNumber, jobType);
}
Job completedJob = loadJob.waitFor();
if (completedJob == null) {
// should not happen since we just submitted the job
throw new IOException("Load job no longer exists. Will be retried till retry timeout is reached.");
}
if (completedJob.getStatus().getError() != null) {
// load job failed
throw new IOException(String.format("Failed to execute BigQuery load job: %s",
completedJob.getStatus().getError()));
}
if (LOG.isDebugEnabled()) {
LOG.debug("Loaded batch {} into staging table for {}.{}", blob.getBatchId(), blob.getDataset(), blob.getTable());
}
}
private Job getPreviousJobIfNotFailed(TableBlob blob, int attemptNumber, JobType jobType) {
Job previousJob = getJobFromPreviousAttemptsIfExists(blob, attemptNumber, jobType);
if (previousJob != null) {
if (isFailedJob(previousJob)) {
LOG.warn("Previous job {} failed with error {} attempting to run a new job", previousJob,
previousJob.getStatus().getError());
} else {
return previousJob;
}
}
return null;
}
private Job createLoadJob(TableId tableId, TableBlob blob, int attemptNumber, JobType jobType)
throws IOException, DeltaFailureException {
Table table = bigQuery.getTable(tableId);
if (table == null) {
Schema schema = jobType.isForTargetTable() ? blob.getTargetSchema() : blob.getStagingSchema();
List<String> primaryKeys = getPrimaryKeys(TableId.of(project, blob.getDataset(), blob.getTable()));
TableDefinition tableDefinition = StandardTableDefinition.newBuilder()
.setLocation(bucket.getLocation())
.setSchema(Schemas.convert(schema))
.setClustering(getClustering(schema, primaryKeys))
.build();
TableInfo.Builder builder = TableInfo.newBuilder(tableId, tableDefinition);
if (encryptionConfig != null) {
builder.setEncryptionConfiguration(encryptionConfig);
}
TableInfo tableInfo = builder.build();
bigQuery.create(tableInfo);
}
// load data from GCS object into staging BQ table
// batch id is a timestamp generated at the time the first event was seen, so the job id is
// guaranteed to be different from the previous batch for the table
JobId jobId = JobId.newBuilder()
.setLocation(bucket.getLocation())
.setJob(getJobId(jobType, blob.getDataset(), blob.getTable(), blob.getBatchId(), attemptNumber))
.build();
BlobId blobId = blob.getBlob().getBlobId();
String uri = String.format("gs://%s/%s", blobId.getBucket(), blobId.getName());
// Explicitly set schema for load jobs
com.google.cloud.bigquery.Schema bqSchema
= Schemas.convert(jobType.isForTargetTable() ? blob.getTargetSchema() : blob.getStagingSchema());
LoadJobConfiguration.Builder jobConfigBuilder = LoadJobConfiguration
.newBuilder(tableId, uri)
.setSchema(bqSchema)
.setSchemaUpdateOptions(ImmutableList.of(JobInfo.SchemaUpdateOption.ALLOW_FIELD_ADDITION));
if (encryptionConfig != null) {
jobConfigBuilder.setDestinationEncryptionConfiguration(encryptionConfig);
}
if (blob.isJsonFormat()) {
jobConfigBuilder.setFormatOptions(FormatOptions.json());
} else {
jobConfigBuilder.setFormatOptions(FormatOptions.avro());
jobConfigBuilder.setUseAvroLogicalTypes(true);
}
LoadJobConfiguration loadJobConf = jobConfigBuilder.build();
JobInfo jobInfo = JobInfo.newBuilder(loadJobConf)
.setJobId(jobId)
.build();
return BigQueryUtils.createBigQueryJob(bigQuery, jobInfo);
}
private void mergeStagingTable(TableId stagingTableId, TableBlob blob,
int attemptNumber) throws InterruptedException, IOException, DeltaFailureException {
LOG.info("Merging batch {} for {}.{} {}", blob.getBatchId(), blob.getDataset(), blob.getTable(),
attemptNumber > 0 ? "attempt: " + attemptNumber : "");
Job mergeJob = null;
if (attemptNumber > 0) {
// Check if any job from previous attempts was successful to avoid merging the same data multiple times
// which can lead to data inconsistency
mergeJob = getPreviousJobIfNotFailed(blob, attemptNumber, JobType.MERGE_TARGET);
}
if (mergeJob == null) {
mergeJob = createMergeJob(stagingTableId, blob, attemptNumber);
}
Job completedJob = mergeJob.waitFor();
if (completedJob == null) {
// should not happen since we just submitted the job
throw new IOException("Merge query job no longer exists. Will be retried till retry timeout is reached.");
}
if (completedJob.getStatus().getError() != null) {
// merge job failed
throw new IOException(String.format("Failed to execute BigQuery merge query job: %s",
completedJob.getStatus().getError()));
}
if (LOG.isDebugEnabled()) {
LOG.debug("Merged batch {} into {}.{}", blob.getBatchId(), blob.getDataset(), blob.getTable());
}
}
private Job createMergeJob(TableId stagingTableId, TableBlob blob, int attemptNumber)
throws IOException, DeltaFailureException {
TableId targetTableId = TableId.of(project, blob.getDataset(), blob.getTable());
List<String> primaryKeys = getPrimaryKeys(targetTableId);
Optional<List<Schema.Type>> sortKeys = getCachedSortKeys(targetTableId);
/*
* Merge data from staging BQ table into target table.
*
* Four independent cases to be considered while performing merge operation:
*
* Case 1: Source generates events without row id and events are ordered.
*
* If the source table has two columns -- id and name -- the staging table will look something like:
*
* | _batch_id | _sequence_num | _op | _before_id | _before_name | id | name
* | 1234567890 | 2 | INSERT | | | 0 | alice
* | 1234567890 | 3 | UPDATE | 0 | alice | 1 | alice
* | 1234567890 | 4 | UPDATE | 1 | alice | 2 | alice
* | 1234567890 | 5 | DELETE | 2 | alice | 2 | alice
* | 1234567890 | 6 | INSERT | | | 0 | Alice
* | 1234567890 | 7 | INSERT | | | 1 | blob
* | 1234567890 | 8 | UPDATE | 1 | blob | 1 | Bob
*
* If the primary key is the 'id' field, the merge is performed by running the following query:
*
* MERGE [target table] as T
* USING ($DIFF_QUERY) as D
* ON T.id = D._before_id
* WHEN MATCHED AND D._op = "DELETE"
* DELETE
* WHEN MATCHED AND D._op IN ("INSERT", "UPDATE")
* UPDATE id = D.id, name = D.name
* WHEN NOT MATCHED AND D._op IN ("INSERT", "UPDATE")
* INSERT (_sequence_num, id, name) VALUES (D._sequence_num, name)
*
* where the $DIFF_QUERY is:
*
* SELECT A.* FROM
* (SELECT * FROM [staging table] WHERE _batch_id = 1234567890 AND _sequence_num > $LATEST_APPLIED) as A
* LEFT OUTER JOIN
* (SELECT * FROM [staging table] WHERE _batch_id = 1234567890 AND _sequence_num > $LATEST_APPLIED) as B
* ON A.id = B._before_id AND A._sequence_num < B._sequence_num
* WHERE B._before_id IS NULL
*
* The purpose of the query is to flatten events within the same batch that have the same primary key.
* For example, with the example data given above, the result of the diff query is:
*
* | _batch_id | _sequence_num | _op | _before_id | _before_name | id | name
* | 1234567890 | 5 | DELETE | | | 2 | alice
* | 1234567890 | 6 | INSERT | | | 0 | Alice
* | 1234567890 | 8 | UPDATE | | | 1 | Bob
*
* The $LATEST_APPLIED part of the query is required for idempotency. If a previous run of the pipeline merged
* some results into the target table, but died before it could commit its offset, the merge query could end up
* doing the wrong thing because what goes into a batch is not deterministic due to the time bound on batches.
*
* Case 2: Source generates events without row id and events are unordered.
* The major differences between Case 1 and Case 2 are that:
* a. For diff query, when join staging table with its self, join condition should contain:
* $ORDERING_CONDITION instead of A._sequence_num < B._sequence_num
* to make sure events in A are joining with events happening later in B. Because for ordered events, sequence
* num can decide the ordering while for unordered events, source timestamp can decide the ordering.
*
* Where $ORDERING_CONDITION is of the form
*
* (A._sort._key_0 < B._sort._key_0) OR
* (A._sort._key_0 = B._sort._key_0 AND A._sort._key_1 < B._sort._key_1) OR
* ...
* (A._sort._key_0 = B._sort._key_0 ... AND A._sort._key_n-1 = B._sort._key_n-1 AND
* A._sort._key_n < B._sort._key_n)
*
* _sort is a struct column consisting of list of fields (_key_0, _key_1 ... _key_n) that each record
* can be sorted by, in the order that the comparison should be performed
*
* b. When merging a delete event, instead of deleting the row, we update the '_is_delete' column to true.
* Because it's possible that an earlier happening update event comes late, if we delete the row, this late
* coming update event will insert a new row. second difference makes sure such event will be ignored.
* c. When merging delete and update event, we add an additional condition $ORDERING_CONDITION
* Because it's possible that an earlier happening update event comes late, we should ignore such event, if
* events happening later than this event has already been applied to target table.
*
* So the merge query would be :
*
* MERGE [target table] as T
* USING ($DIFF_QUERY) as D
* ON T.id = D._before_id
* WHEN MATCHED AND D._op = "DELETE" AND $ORDERING_CONDITION
* UPDATE SET T._is_deleted = true
* WHEN MATCHED AND D._op IN ("INSERT", "UPDATE") AND $ORDERING_CONDITION
* UPDATE id = D.id, name = D.name
* WHEN NOT MATCHED AND D._op IN ("INSERT", "UPDATE")
* INSERT (_sequence_num, _is_deleted, _source_timestamp, _sort, id, name) VALUES (D._sequence_num, false, D
* ._source_timestamp, D._sort, id, name)
*