forked from apache/hive
/
ReplDumpTask.java
1839 lines (1715 loc) · 94.5 KB
/
ReplDumpTask.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.hadoop.hive.ql.exec.repl;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.ValidTxnList;
import org.apache.hadoop.hive.common.ValidWriteIdList;
import org.apache.hadoop.hive.common.repl.ReplConst;
import org.apache.hadoop.hive.conf.Constants;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.ReplChangeManager;
import org.apache.hadoop.hive.metastore.TableType;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.Function;
import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.hadoop.hive.metastore.api.SQLAllTableConstraints;
import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
import org.apache.hadoop.hive.metastore.api.TxnType;
import org.apache.hadoop.hive.metastore.messaging.event.filters.AndFilter;
import org.apache.hadoop.hive.metastore.messaging.event.filters.CatalogFilter;
import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter;
import org.apache.hadoop.hive.metastore.messaging.event.filters.ReplEventFilter;
import org.apache.hadoop.hive.metastore.txn.TxnErrorMsg;
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
import org.apache.hadoop.hive.metastore.utils.StringUtils;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.Task;
import org.apache.hadoop.hive.ql.exec.TaskFactory;
import org.apache.hadoop.hive.ql.exec.repl.util.AddDependencyToLeaves;
import org.apache.hadoop.hive.ql.exec.repl.util.FileList;
import org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils;
import org.apache.hadoop.hive.ql.exec.repl.util.SnapshotUtils;
import org.apache.hadoop.hive.ql.exec.repl.util.TaskTracker;
import org.apache.hadoop.hive.ql.exec.util.DAGTraversal;
import org.apache.hadoop.hive.ql.exec.util.Retryable;
import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hadoop.hive.ql.lockmgr.DbLockManager;
import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager;
import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
import org.apache.hadoop.hive.ql.lockmgr.LockException;
import org.apache.hadoop.hive.ql.metadata.Hive;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.metadata.InvalidTableException;
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.hive.ql.metadata.events.EventUtils;
import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.TableSpec;
import org.apache.hadoop.hive.ql.parse.EximUtil;
import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.parse.repl.DumpType;
import org.apache.hadoop.hive.ql.parse.repl.dump.ExportService;
import org.apache.hadoop.hive.ql.parse.repl.dump.EventsDumpMetadata;
import org.apache.hadoop.hive.ql.parse.repl.dump.HiveWrapper;
import org.apache.hadoop.hive.ql.parse.repl.dump.TableExport;
import org.apache.hadoop.hive.ql.parse.repl.dump.Utils;
import org.apache.hadoop.hive.ql.parse.repl.dump.events.EventHandler;
import org.apache.hadoop.hive.ql.parse.repl.dump.events.EventHandlerFactory;
import org.apache.hadoop.hive.ql.parse.repl.dump.io.ConstraintsSerializer;
import org.apache.hadoop.hive.ql.parse.repl.dump.io.FunctionSerializer;
import org.apache.hadoop.hive.ql.parse.repl.dump.io.JsonWriter;
import org.apache.hadoop.hive.ql.parse.repl.dump.log.BootstrapDumpLogger;
import org.apache.hadoop.hive.ql.parse.repl.dump.log.IncrementalDumpLogger;
import org.apache.hadoop.hive.ql.parse.repl.dump.metric.BootstrapDumpMetricCollector;
import org.apache.hadoop.hive.ql.parse.repl.dump.metric.IncrementalDumpMetricCollector;
import org.apache.hadoop.hive.ql.parse.repl.dump.metric.OptimizedBootstrapDumpMetricCollector;
import org.apache.hadoop.hive.ql.parse.repl.dump.metric.PreOptimizedBootstrapDumpMetricCollector;
import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
import org.apache.hadoop.hive.ql.parse.repl.load.FailoverMetaData;
import org.apache.hadoop.hive.ql.parse.repl.metric.ReplicationMetricCollector;
import org.apache.hadoop.hive.ql.parse.repl.metric.event.Metadata.ReplicationType;
import org.apache.hadoop.hive.ql.parse.repl.metric.event.Status;
import org.apache.hadoop.hive.ql.plan.ExportWork.MmContext;
import org.apache.hadoop.hive.ql.plan.api.StageType;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.thrift.TException;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.Set;
import java.util.HashSet;
import java.util.List;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.UUID;
import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_RESUME_STARTED_AFTER_FAILOVER;
import static org.apache.hadoop.hive.common.repl.ReplConst.REPL_TARGET_DB_PROPERTY;
import static org.apache.hadoop.hive.common.repl.ReplConst.TARGET_OF_REPLICATION;
import static org.apache.hadoop.hive.conf.Constants.SCHEDULED_QUERY_SCHEDULENAME;
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT;
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY;
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK;
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK_PATHS;
import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY;
import static org.apache.hadoop.hive.common.repl.ReplConst.SOURCE_OF_REPLICATION;
import static org.apache.hadoop.hive.metastore.ReplChangeManager.getReplPolicyIdString;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.EVENT_ACK_FILE;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.TABLE_DIFF_COMPLETE_DIRECTORY;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.checkFileExists;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.createAndGetEventAckFile;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.createBootstrapTableList;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.getEventIdFromFile;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.getReplEventIdFromDatabase;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.getTablesFromTableDiffFile;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.getTargetEventId;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.isDbTargetOfFailover;
import static org.apache.hadoop.hive.ql.exec.repl.OptimisedBootstrapUtils.isFirstIncrementalPending;
import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.LOAD_ACKNOWLEDGEMENT;
import static org.apache.hadoop.hive.ql.exec.repl.ReplAck.NON_RECOVERABLE_MARKER;
import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.RANGER_AUTHORIZER;
import static org.apache.hadoop.hive.ql.exec.repl.util.ReplUtils.getOpenTxns;
import static org.apache.hadoop.hive.ql.exec.repl.util.SnapshotUtils.cleanupSnapshots;
import static org.apache.hadoop.hive.ql.exec.repl.util.SnapshotUtils.getDFS;
import static org.apache.hadoop.hive.ql.exec.repl.util.SnapshotUtils.getListFromFileList;
import static org.apache.hadoop.hive.ql.parse.ReplicationSpec.KEY.CURR_STATE_ID_SOURCE;
import static org.apache.hadoop.hive.ql.parse.ReplicationSpec.KEY.CURR_STATE_ID_TARGET;
public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
private static final long serialVersionUID = 1L;
private static final String dumpSchema = "dump_dir,last_repl_id#string,string";
private static final String FUNCTION_METADATA_FILE_NAME = EximUtil.METADATA_NAME;
private static final long SLEEP_TIME = 5 * 60000;
private static final long SLEEP_TIME_FOR_TESTS = 30000;
private Set<String> tablesForBootstrap = new HashSet<>();
private List<TxnType> excludedTxns = Arrays.asList(TxnType.READ_ONLY, TxnType.REPL_CREATED);
private boolean createEventMarker = false;
public enum ConstraintFileType {COMMON("common", "c_"), FOREIGNKEY("fk", "f_");
private final String name;
private final String prefix;
ConstraintFileType(String name, String prefix) {
this.name = name;
this.prefix = prefix;
}
public String getName() {
return this.name;
}
public String getPrefix() {
return prefix;
}
}
private Logger LOG = LoggerFactory.getLogger(ReplDumpTask.class);
@Override
public String getName() {
return "REPL_DUMP";
}
@Override
public int execute() {
try {
SecurityUtils.reloginExpiringKeytabUser();
if (work.dataCopyIteratorsInitialized()) {
initiateDataCopyTasks();
} else {
Path dumpRoot = ReplUtils.getEncodedDumpRootPath(conf, work.dbNameOrPattern.toLowerCase());
Path latestDumpPath = ReplUtils.getLatestDumpPath(dumpRoot, conf);
if (ReplUtils.failedWithNonRecoverableError(latestDumpPath, conf)) {
LOG.error("Previous dump failed with non recoverable error. Needs manual intervention. ");
Path nonRecoverableFile = new Path(latestDumpPath, NON_RECOVERABLE_MARKER.toString());
ReplUtils.reportStatusInReplicationMetrics(getName(), Status.SKIPPED, nonRecoverableFile.toString(), conf, work.dbNameOrPattern, null);
setException(new SemanticException(ErrorMsg.REPL_FAILED_WITH_NON_RECOVERABLE_ERROR.format()));
return ErrorMsg.REPL_FAILED_WITH_NON_RECOVERABLE_ERROR.getErrorCode();
}
Path previousValidHiveDumpPath = getPreviousValidDumpMetadataPath(dumpRoot);
boolean isFailoverMarkerPresent = false;
boolean isFailoverTarget = isDbTargetOfFailover(work.dbNameOrPattern, getHive());
LOG.debug("Database {} is {} going through failover", work.dbNameOrPattern, isFailoverTarget ? "" : "not");
if (previousValidHiveDumpPath == null && !isFailoverTarget) {
work.setBootstrap(true);
} else {
work.setOldReplScope(isFailoverTarget ? null : new DumpMetaData(previousValidHiveDumpPath, conf).getReplScope());
isFailoverMarkerPresent = !isFailoverTarget && isDumpFailoverReady(previousValidHiveDumpPath);
}
//Proceed with dump operation in following cases:
//1. No previous dump is present.
//2. Previous dump is already loaded and it is not in failover ready status.
if (shouldDump(previousValidHiveDumpPath, isFailoverMarkerPresent, isFailoverTarget)) {
Path currentDumpPath = getCurrentDumpPath(dumpRoot, work.isBootstrap());
Path hiveDumpRoot = new Path(currentDumpPath, ReplUtils.REPL_HIVE_BASE_DIR);
if (!work.isBootstrap() && !isFailoverTarget) {
preProcessFailoverIfRequired(previousValidHiveDumpPath, isFailoverMarkerPresent);
}
// check if we need to create event marker
if (previousValidHiveDumpPath == null) {
createEventMarker = isFailoverTarget;
} else {
if (isFailoverTarget) {
boolean isEventAckFilePresent = checkFileExists(previousValidHiveDumpPath.getParent(), conf, EVENT_ACK_FILE);
if (!isEventAckFilePresent) {
// If this is optimised bootstrap failover cycle and _event_ack file is not present, then create it
createEventMarker = true;
}
}
}
// Set distCp custom name corresponding to the replication policy.
String mapRedCustomName = ReplUtils.getDistCpCustomName(conf, work.dbNameOrPattern);
conf.set(JobContext.JOB_NAME, mapRedCustomName);
work.setCurrentDumpPath(currentDumpPath);
// Initialize repl dump metric collector for all replication stage (Bootstrap, incremental, pre-optimised and optimised bootstrap)
ReplicationMetricCollector dumpMetricCollector = initReplicationDumpMetricCollector(hiveDumpRoot, work.isBootstrap(), createEventMarker /*isPreOptimisedBootstrap*/, isFailoverTarget);
work.setMetricCollector(dumpMetricCollector);
if (shouldDumpAtlasMetadata()) {
addAtlasDumpTask(work.isBootstrap(), previousValidHiveDumpPath);
LOG.info("Added task to dump atlas metadata.");
}
if (shouldDumpAuthorizationMetadata()) {
initiateAuthorizationDumpTask();
}
DumpMetaData dmd = new DumpMetaData(hiveDumpRoot, conf);
// Initialize ReplChangeManager instance since we will require it to encode file URI.
ReplChangeManager.getInstance(conf);
Path cmRoot = new Path(conf.getVar(HiveConf.ConfVars.REPLCMDIR));
Long lastReplId;
LOG.info("Data copy at load enabled : {}", conf.getBoolVar(HiveConf.ConfVars.REPL_RUN_DATA_COPY_TASKS_ON_TARGET));
if (isFailoverTarget) {
if (createEventMarker) {
LOG.info("Optimised Bootstrap Dump triggered for {}.", work.dbNameOrPattern);
// Before starting optimised bootstrap, check if the first incremental is done to ensure database is in
// consistent state.
isFirstIncrementalPending(work.dbNameOrPattern, getHive());
Database database = getHive().getDatabase(work.dbNameOrPattern);
if (database != null) {
HashMap<String, String> params = new HashMap<>(database.getParameters());
long failbackStartTime = System.currentTimeMillis();
params.put(ReplConst.REPL_METRICS_LAST_FAILBACK_STARTTIME, Long.toString(failbackStartTime));
LOG.info("Replication Metrics: Setting replication metrics failback start time for database: {} to: {} ", work.dbNameOrPattern, failbackStartTime);
if (!MetaStoreUtils.isDbBeingPlannedFailedOver(database)) { // if this is failback due to unplanned failover
LOG.info("Replication Metrics: Setting last failover type for database: {} to: {} ", work.dbNameOrPattern, ReplConst.FailoverType.UNPLANNED.toString());
params.put(ReplConst.REPL_METRICS_LAST_FAILOVER_TYPE, ReplConst.FailoverType.UNPLANNED.toString());
int failoverCount = 1 + NumberUtils.toInt(params.getOrDefault(ReplConst.REPL_METRICS_FAILOVER_COUNT, "0"), 0);
LOG.info("Replication Metrics: Setting replication metrics failover count for database: {} to: {} ", work.dbNameOrPattern, failoverCount);
params.put(ReplConst.REPL_METRICS_FAILOVER_COUNT, Integer.toString(failoverCount));
}
database.setParameters(params);
getHive().alterDatabase(work.dbNameOrPattern, database);
} else {
LOG.debug("Database {} does not exist. Cannot set replication failover failback metrics", work.dbNameOrPattern);
}
// Get the last replicated event id from the database.
String dbEventId = getReplEventIdFromDatabase(work.dbNameOrPattern, getHive());
// Get the last replicated event id from the database with respect to target.
String targetDbEventId = getTargetEventId(work.dbNameOrPattern, getHive());
LOG.info("Creating event_ack file for database {} with event id {}.", work.dbNameOrPattern, dbEventId);
Map<String, Long> metricMap = new HashMap<>();
metricMap.put(ReplUtils.MetricName.EVENTS.name(), 0L);
work.getMetricCollector().reportStageStart(getName(), metricMap);
lastReplId =
createAndGetEventAckFile(currentDumpPath, dmd, cmRoot, dbEventId, targetDbEventId, conf, work);
finishRemainingTasks();
work.getMetricCollector().reportStageEnd(getName(), Status.SUCCESS);
} else {
// We should be here only if TableDiff is Present.
boolean isTableDiffDirectoryPresent =
checkFileExists(previousValidHiveDumpPath.getParent(), conf, TABLE_DIFF_COMPLETE_DIRECTORY);
boolean isAbortTxnsListPresent =
checkFileExists(previousValidHiveDumpPath.getParent(), conf, OptimisedBootstrapUtils.ABORT_TXNS_FILE);
assert isTableDiffDirectoryPresent;
work.setSecondDumpAfterFailover(true);
long fromEventId = Long.parseLong(getEventIdFromFile(previousValidHiveDumpPath.getParent(), conf)[1]);
LOG.info("Starting optimised bootstrap from event id {} for database {}", fromEventId,
work.dbNameOrPattern);
work.setEventFrom(fromEventId);
// Get the tables to be bootstrapped from the table diff
tablesForBootstrap = getTablesFromTableDiffFile(previousValidHiveDumpPath.getParent(), conf);
if (isAbortTxnsListPresent) {
abortReplCreatedTxnsPriorToFailover(previousValidHiveDumpPath.getParent(), conf);
}
// Generate the bootstrapped table list and put it in the new dump directory for the load to consume.
createBootstrapTableList(currentDumpPath, tablesForBootstrap, conf);
dumpDbMetadata(work.dbNameOrPattern, new Path(hiveDumpRoot, EximUtil.METADATA_PATH_NAME),
fromEventId, getHive());
// Call the normal dump with the tablesForBootstrap set.
lastReplId = incrementalDump(hiveDumpRoot, dmd, cmRoot, getHive());
}
}
else if (work.isBootstrap()) {
lastReplId = bootStrapDump(hiveDumpRoot, dmd, cmRoot, getHive());
} else {
work.setEventFrom(getEventFromPreviousDumpMetadata(previousValidHiveDumpPath));
lastReplId = incrementalDump(hiveDumpRoot, dmd, cmRoot, getHive());
}
// The datacopy doesn't need to be initialised in case of optimised bootstrap first dump.
if (lastReplId >= 0) {
work.setResultValues(Arrays.asList(currentDumpPath.toUri().toString(), String.valueOf(lastReplId)));
initiateDataCopyTasks();
}
} else {
if (isFailoverMarkerPresent) {
LOG.info("Previous Dump is failover ready. Skipping this iteration.");
} else {
LOG.info("Previous Dump is not yet loaded. Skipping this iteration.");
}
ReplUtils.reportStatusInReplicationMetrics(getName(), Status.SKIPPED, null, conf,
work.dbNameOrPattern, work.isBootstrap() ? ReplicationType.BOOTSTRAP: ReplicationType.INCREMENTAL);
}
}
} catch (RuntimeException e) {
LOG.error("replication failed with run time exception", e);
setException(e);
try{
ReplUtils.handleException(true, e, work.getCurrentDumpPath().toString(),
work.getMetricCollector(), getName(), conf);
} catch (Exception ex){
LOG.error("Failed to collect replication metrics: ", ex);
}
throw e;
} catch (Exception e) {
setException(e);
int errorCode;
if (e instanceof SnapshotException) {
errorCode = ErrorMsg.getErrorMsg("SNAPSHOT_ERROR").getErrorCode();
} else {
errorCode = ErrorMsg.getErrorMsg(e.getMessage()).getErrorCode();
}
try{
return ReplUtils.handleException(true, e, work.getCurrentDumpPath().toString(),
work.getMetricCollector(), getName(), conf);
}
catch (Exception ex){
LOG.error("Failed to collect replication metrics: ", ex);
return errorCode;
}
}
return 0;
}
private void abortReplCreatedTxnsPriorToFailover(Path dumpPath, HiveConf conf) throws LockException, IOException {
List<Long> replCreatedTxnsToAbort = OptimisedBootstrapUtils.getTxnIdFromAbortTxnsFile(dumpPath, conf);
String replPolicy = HiveUtils.getReplPolicy(work.dbNameOrPattern);
HiveTxnManager hiveTxnManager = getTxnMgr();
for (Long txnId : replCreatedTxnsToAbort) {
LOG.info("Rolling back Repl_Created txns:" + replCreatedTxnsToAbort.toString() + " opened prior to failover.");
hiveTxnManager.replRollbackTxn(replPolicy, txnId);
}
}
private void preProcessFailoverIfRequired(Path previousValidHiveDumpDir, boolean isPrevFailoverReadyMarkerPresent)
throws HiveException, IOException {
FileSystem fs = previousValidHiveDumpDir.getFileSystem(conf);
Database db = getHive().getDatabase(work.dbNameOrPattern);
if (isPrevFailoverReadyMarkerPresent) {
if (MetaStoreUtils.isDbBeingPlannedFailedOverAtEndpoint(db, MetaStoreUtils.FailoverEndpoint.SOURCE)) {
//Since previous valid dump is failover ready and repl.failover.endpoint is set for source, just rollback
// the failover process initiated in the previous iteration.
LOG.info("Rolling back failover initiated in previous dump iteration.");
fs.delete(new Path(previousValidHiveDumpDir, ReplAck.FAILOVER_READY_MARKER.toString()), true);
} else if (MetaStoreUtils.isDbBeingPlannedFailedOverAtEndpoint(db, MetaStoreUtils.FailoverEndpoint.TARGET)) {
//Since previous valid dump is failover ready and repl.failover.endpoint is set for target,
// this means it is first dump operation in the reverse direction.
LOG.info("Switching to bootstrap dump as this is the first dump execution after failover.");
work.setFirstDumpAfterFailover(true);
}
}
if (!shouldFailover() && !work.isFirstDumpAfterFailover()) {
//If this is first dump operation, don't unset this property until first incremental dump.
ReplUtils.unsetDbPropIfSet(db, ReplConst.REPL_FAILOVER_ENDPOINT, getHive());
}
}
private boolean isDumpFailoverReady(Path previousValidHiveDumpPath) throws IOException {
FileSystem fs = previousValidHiveDumpPath.getFileSystem(conf);
Path failoverReadyMarkerFile = new Path(previousValidHiveDumpPath, ReplAck.FAILOVER_READY_MARKER.toString());
return fs.exists(failoverReadyMarkerFile);
}
private void initiateAuthorizationDumpTask() throws SemanticException {
if (RANGER_AUTHORIZER.equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.REPL_AUTHORIZATION_PROVIDER_SERVICE))) {
Path rangerDumpRoot = new Path(work.getCurrentDumpPath(), ReplUtils.REPL_RANGER_BASE_DIR);
LOG.info("Exporting Authorization Metadata from {} at {} ", RANGER_AUTHORIZER, rangerDumpRoot);
RangerDumpWork rangerDumpWork = new RangerDumpWork(rangerDumpRoot, work.dbNameOrPattern,
work.getMetricCollector());
Task<RangerDumpWork> rangerDumpTask = TaskFactory.get(rangerDumpWork, conf);
if (childTasks == null) {
childTasks = new ArrayList<>();
}
childTasks.add(rangerDumpTask);
} else {
throw new SemanticException(ErrorMsg.REPL_INVALID_CONFIG_FOR_SERVICE.format("Authorizer "
+ conf.getVar(HiveConf.ConfVars.REPL_AUTHORIZATION_PROVIDER_SERVICE)
+ " not supported for replication ", ReplUtils.REPL_RANGER_SERVICE));
}
}
private boolean shouldDumpAuthorizationMetadata() {
return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_AUTHORIZATION_METADATA);
}
private boolean shouldDumpAtlasMetadata() {
return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_ATLAS_METADATA);
}
private Path getCurrentDumpPath(Path dumpRoot, boolean isBootstrap) throws IOException {
Path lastDumpPath = ReplUtils.getLatestDumpPath(dumpRoot, conf);
if (lastDumpPath != null && shouldResumePreviousDump(lastDumpPath, isBootstrap)) {
//Resume previous dump
LOG.info("Resuming the dump with existing dump directory {}", lastDumpPath);
if (!shouldFailover()) {
//If failoverReadyMarker was created in previous dump iteration, Just delete and proceed.
Path hiveDumpRoot = new Path(lastDumpPath, ReplUtils.REPL_HIVE_BASE_DIR);
Path failoverReadyMarkerFile = new Path(hiveDumpRoot, ReplAck.FAILOVER_READY_MARKER.toString());
FileSystem fs = failoverReadyMarkerFile.getFileSystem(conf);
if (fs.exists(failoverReadyMarkerFile)) {
LOG.info("Deleting previous failover ready marker file: {}.", failoverReadyMarkerFile);
fs.delete(failoverReadyMarkerFile, true);
}
}
work.setShouldOverwrite(true);
return lastDumpPath;
} else {
return new Path(dumpRoot, getNextDumpDir());
}
}
private void initiateDataCopyTasks() throws HiveException, IOException {
TaskTracker taskTracker = new TaskTracker(conf.getIntVar(HiveConf.ConfVars.REPL_APPROX_MAX_LOAD_TASKS));
if (childTasks == null) {
childTasks = new ArrayList<>();
}
List<Task<?>> externalTableCopyTasks = work.externalTableCopyTasks(taskTracker, conf);
childTasks.addAll(externalTableCopyTasks);
LOG.debug("Scheduled {} external table copy tasks", externalTableCopyTasks.size());
// If external table data copy tasks are present add a task to mark the end of data copy
if (!externalTableCopyTasks.isEmpty() && !work.getExternalTblCopyPathIterator().hasNext()) {
ReplUtils.addLoggerTask(work.getReplLogger(), childTasks, conf);
}
childTasks.addAll(work.managedTableCopyTasks(taskTracker, conf));
childTasks.addAll(work.functionsBinariesCopyTasks(taskTracker, conf));
if (childTasks.isEmpty()) {
//All table data copy work finished.
finishRemainingTasks();
} else {
DAGTraversal.traverse(childTasks, new AddDependencyToLeaves(TaskFactory.get(work, conf)));
}
}
private void addAtlasDumpTask(boolean bootstrap, Path prevHiveDumpDir) {
Path atlasDumpDir = new Path(work.getCurrentDumpPath(), ReplUtils.REPL_ATLAS_BASE_DIR);
Path prevAtlasDumpDir = prevHiveDumpDir == null ? null
: new Path(prevHiveDumpDir.getParent(), ReplUtils.REPL_ATLAS_BASE_DIR);
Path tableListLoc = null;
if (!work.replScope.includeAllTables()) {
Path tableListDir = new Path(work.getCurrentDumpPath(), ReplUtils.REPL_HIVE_BASE_DIR + "/" + ReplUtils.REPL_TABLE_LIST_DIR_NAME);
tableListLoc = new Path(tableListDir, work.dbNameOrPattern.toLowerCase());
}
AtlasDumpWork atlasDumpWork = new AtlasDumpWork(work.dbNameOrPattern, atlasDumpDir, bootstrap, prevAtlasDumpDir,
tableListLoc, work.getMetricCollector());
Task<?> atlasDumpTask = TaskFactory.get(atlasDumpWork, conf);
childTasks = new ArrayList<>();
childTasks.add(atlasDumpTask);
}
private void finishRemainingTasks() throws HiveException {
Database database = getHive().getDatabase(work.dbNameOrPattern);
boolean isFailoverInProgress = shouldFailover() && !work.isBootstrap() && !createEventMarker;
if (isFailoverInProgress) {
Utils.create(new Path(work.getCurrentDumpPath(), ReplUtils.REPL_HIVE_BASE_DIR + File.separator
+ ReplAck.FAILOVER_READY_MARKER), conf);
LOG.info("Dump marked as failover ready.");
}
Path dumpAckFile = new Path(work.getCurrentDumpPath(), ReplUtils.REPL_HIVE_BASE_DIR + File.separator
+ ReplAck.DUMP_ACKNOWLEDGEMENT);
// Check if we need to unset database properties after successful optimised bootstrap.
if (work.isSecondDumpAfterFailover()) {
if (database != null) {
HashMap<String, String> dbParams = new HashMap<>(database.getParameters());
LOG.debug("Database {} params before removal {}", work.dbNameOrPattern, dbParams);
dbParams.remove(TARGET_OF_REPLICATION);
dbParams.remove(CURR_STATE_ID_TARGET.toString());
dbParams.remove(CURR_STATE_ID_SOURCE.toString());
dbParams.remove(REPL_TARGET_DB_PROPERTY);
dbParams.remove(ReplConst.REPL_ENABLE_BACKGROUND_THREAD);
dbParams.remove(REPL_RESUME_STARTED_AFTER_FAILOVER);
if (!isFailoverInProgress) {
// if we have failover endpoint from controlled failover remove it.
dbParams.remove(ReplConst.REPL_FAILOVER_ENDPOINT);
}
LOG.info("Removing {} property from the database {} after successful optimised bootstrap dump", String.join(",",
new String[]{TARGET_OF_REPLICATION, CURR_STATE_ID_TARGET.toString(), CURR_STATE_ID_SOURCE.toString(),
REPL_TARGET_DB_PROPERTY}), work.dbNameOrPattern);
int failbackCount = 1 + NumberUtils.toInt(dbParams.getOrDefault(ReplConst.REPL_METRICS_FAILBACK_COUNT, "0"), 0);
LOG.info("Replication Metrics: Setting replication metrics failback count for database: {} to: {} ", work.dbNameOrPattern, failbackCount);
dbParams.put(ReplConst.REPL_METRICS_FAILBACK_COUNT, Integer.toString(failbackCount));
long failbackEndTime = System.currentTimeMillis();
dbParams.put(ReplConst.REPL_METRICS_LAST_FAILBACK_ENDTIME, Long.toString(failbackEndTime));
LOG.info("Replication Metrics: Setting replication metrics failback end time for database: {} to: {} ", work.dbNameOrPattern, failbackEndTime);
database.setParameters(dbParams);
getHive().alterDatabase(work.dbNameOrPattern, database);
LOG.debug("Database {} params after removal {}", work.dbNameOrPattern, dbParams);
} else {
LOG.debug("Database {} does not exist. Cannot set replication failover and failback metrics", work.dbNameOrPattern);
}
}
Utils.create(dumpAckFile, conf);
prepareReturnValues(work.getResultValues());
if (isFailoverInProgress) {
work.getMetricCollector().reportEnd(Status.FAILOVER_READY);
} else {
work.getMetricCollector().reportEnd(isFirstCycleOfResume(database) ?
Status.RESUME_READY :
Status.SUCCESS);
}
deleteAllPreviousDumpMeta(work.getCurrentDumpPath());
}
private boolean isFirstCycleOfResume(Database database) {
return createEventMarker && database.getParameters().containsKey(REPL_RESUME_STARTED_AFTER_FAILOVER);
}
private void prepareReturnValues(List<String> values) throws SemanticException {
LOG.debug("prepareReturnValues : " + dumpSchema);
for (String s : values) {
LOG.debug(" > " + s);
}
Utils.writeOutput(Collections.singletonList(values), new Path(work.resultTempPath), conf);
}
private void deleteAllPreviousDumpMeta(Path currentDumpPath) {
try {
Path dumpRoot = getDumpRoot(currentDumpPath);
if(dumpRoot == null) {
return;
}
FileSystem fs = dumpRoot.getFileSystem(conf);
if (fs.exists(dumpRoot)) {
FileStatus[] statuses = fs.listStatus(dumpRoot,
path -> !path.equals(currentDumpPath) && !path.toUri().getPath().equals(currentDumpPath.toString()));
int retainPrevDumpDirCount = conf.getIntVar(HiveConf.ConfVars.REPL_RETAIN_PREV_DUMP_DIR_COUNT);
int numDumpDirs = statuses.length;
if(shouldRetainPrevDumpDirs()) {
Arrays.sort(statuses, (Comparator.<FileStatus>
comparingLong(fileStatus1 -> fileStatus1.getModificationTime())
.thenComparingLong(fileStatus2 -> fileStatus2.getModificationTime())));
}
for (FileStatus status : statuses) {
//based on config, either delete all previous dump-dirs
//or delete a minimum number of oldest dump-directories
if (numDumpDirs == 1 && work.isFirstDumpAfterFailover()) {
LOG.info("Skipping deletion of last failover ready dump dir: ", status.getPath());
break;
}
if(!shouldRetainPrevDumpDirs() || numDumpDirs > retainPrevDumpDirCount){
fs.delete(status.getPath(), true);
numDumpDirs--;
}
}
}
} catch (Exception ex) {
LOG.warn("Possible leak on disk, could not delete the previous dump directory:" + currentDumpPath, ex);
}
}
private Path getDumpRoot(Path currentDumpPath) {
if (ReplDumpWork.testDeletePreviousDumpMetaPath
&& (conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)
|| conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST_REPL))) {
//testDeleteDumpMetaDumpPath to be used only for test.
return null;
} else {
return currentDumpPath.getParent();
}
}
private Long getEventFromPreviousDumpMetadata(Path previousDumpPath) throws SemanticException {
if (previousDumpPath != null) {
DumpMetaData dmd = new DumpMetaData(previousDumpPath, conf);
if (dmd.isIncrementalDump()) {
return dmd.getEventTo();
}
//bootstrap case return event from
return dmd.getEventFrom();
}
return 0L;
}
private Path getPreviousValidDumpMetadataPath(Path dumpRoot) throws IOException {
FileStatus latestValidStatus = null;
FileSystem fs = dumpRoot.getFileSystem(conf);
if (fs.exists(dumpRoot)) {
FileStatus[] statuses = fs.listStatus(dumpRoot);
for (FileStatus status : statuses) {
LOG.info("Evaluating previous dump dir path:{}", status.getPath());
if (latestValidStatus == null) {
latestValidStatus = validDump(status.getPath()) ? status : null;
} else if (validDump(status.getPath())
&& status.getModificationTime() > latestValidStatus.getModificationTime()) {
latestValidStatus = status;
}
}
}
Path latestDumpDir = (latestValidStatus == null)
? null : new Path(latestValidStatus.getPath(), ReplUtils.REPL_HIVE_BASE_DIR);
LOG.info("Selecting latest valid dump dir as {}", (latestDumpDir == null) ? "null" : latestDumpDir.toString());
return latestDumpDir;
}
private boolean validDump(Path dumpDir) throws IOException {
//Check if it was a successful dump
if (dumpDir != null) {
FileSystem fs = dumpDir.getFileSystem(conf);
Path hiveDumpDir = new Path(dumpDir, ReplUtils.REPL_HIVE_BASE_DIR);
return fs.exists(new Path(hiveDumpDir, ReplAck.DUMP_ACKNOWLEDGEMENT.toString()));
}
return false;
}
private boolean shouldDump(Path previousDumpPath, boolean isFailoverMarkerPresent, boolean isFailover)
throws IOException, HiveException {
/** a) If there is no previous dump dir found, the current run is bootstrap case.
* b) If the previous dump was successful and it contains failover marker file as well as
* HiveConf.ConfVars.HIVE_REPL_FAILOVER_START == true, last dump was a controlled failover dump,
* skip doing any further dump.
*/
if (previousDumpPath == null) {
return true;
} else if (isFailoverMarkerPresent && shouldFailover()) {
return false;
} else if (isFailover) {
// In case of OptimisedBootstrap Failover, We need to do a dump in case:
// 1. No EVENT_ACK file is there.
// 2. EVENT_ACK file and TABLE_DIFF_COMPLETE file is also there and the current database id is same as that in
// the EVENT_ACK file
boolean isEventAckFilePresent = checkFileExists(previousDumpPath.getParent(), conf, EVENT_ACK_FILE);
if (!isEventAckFilePresent) {
// If in the previous valid dump path, Event_Ack isn't there that means the previous one was a normal dump,
// we need to trigger the failover dump
LOG.debug("EVENT_ACK file not found in {}. Proceeding with OptimisedBootstrap Failover",
previousDumpPath.getParent());
return true;
}
// Event_ACK file is present check if it contains correct value or not.
String fileEventId = getEventIdFromFile(previousDumpPath.getParent(), conf)[0];
String dbEventId = getReplEventIdFromDatabase(work.dbNameOrPattern, getHive()).trim();
if (!dbEventId.equalsIgnoreCase(fileEventId)) {
// In case the database event id changed post table_diff_complete generation, that means both forward &
// backward policies are operational, We fail in that case with non-recoverable error.
LOG.error("The database eventID {} and the event id in the EVENT_ACK file {} both mismatch. FilePath {}",
dbEventId, fileEventId, previousDumpPath.getParent());
throw new RuntimeException("Database event id changed post table diff generation.");
} else {
// Check table_diff_complete and Load_ACK
return checkFileExists(previousDumpPath.getParent(), conf, TABLE_DIFF_COMPLETE_DIRECTORY) && checkFileExists(previousDumpPath,
conf, LOAD_ACKNOWLEDGEMENT.toString());
}
} else {
FileSystem fs = previousDumpPath.getFileSystem(conf);
return fs.exists(new Path(previousDumpPath, LOAD_ACKNOWLEDGEMENT.toString()));
}
}
/**
* Decide whether to examine all the tables to dump. We do this if
* 1. External tables are going to be part of the dump : In which case we need to list their
* locations.
* 2. External or ACID tables are being bootstrapped for the first time : so that we can dump
* those tables as a whole.
* 3. If replication policy is changed/replaced, then need to examine all the tables to see if
* any of them need to be bootstrapped as old policy doesn't include it but new one does.
* 4. Some tables are renamed and the new name satisfies the table list filter while old name was not.
* @return true if need to examine tables for dump and false if not.
*/
private boolean shouldExamineTablesToDump() {
return (previousReplScopeModified())
|| !tablesForBootstrap.isEmpty()
|| conf.getBoolVar(HiveConf.ConfVars.REPL_BOOTSTRAP_ACID_TABLES)
|| conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES);
}
private boolean shouldFailover() {
return conf.getBoolVar(HiveConf.ConfVars.HIVE_REPL_FAILOVER_START);
}
private boolean previousReplScopeModified() {
return work.oldReplScope != null && !work.oldReplScope.equals(work.replScope);
}
/**
* Decide whether to dump external tables data. If external tables are enabled for replication,
* then need to dump it's data in all the incremental dumps.
* @param conf Hive Configuration.
* @return true if need to dump external table data and false if not.
*/
public static boolean shouldDumpExternalTableLocation(HiveConf conf) {
return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)
&& (!conf.getBoolVar(REPL_DUMP_METADATA_ONLY) &&
!conf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE));
}
/**
* Decide whether to dump external tables.
* @param tableName - Name of external table to be replicated
* @return true if need to bootstrap dump external table and false if not.
*/
private boolean shouldBootstrapDumpExternalTable(String tableName) {
// Note: If repl policy is replaced, then need to dump external tables if table is getting replicated
// for the first time in current dump. So, need to check if table is included in old policy.
return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)
&& (conf.getBoolVar(HiveConf.ConfVars.REPL_BOOTSTRAP_EXTERNAL_TABLES)
|| !ReplUtils.tableIncludedInReplScope(work.oldReplScope, tableName));
}
/**
* Decide whether to dump materialized views.
*/
private boolean isMaterializedViewsReplEnabled() {
return conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_MATERIALIZED_VIEWS);
}
/**
* Decide whether to retain previous dump-directories after repl-dump
*/
private boolean shouldRetainPrevDumpDirs() {
return conf.getBoolVar(HiveConf.ConfVars.REPL_RETAIN_PREV_DUMP_DIR);
}
/**
* Decide whether to dump ACID tables.
* @param tableName - Name of ACID table to be replicated
* @return true if need to bootstrap dump ACID table and false if not.
*/
private boolean shouldBootstrapDumpAcidTable(String tableName) {
// Note: If repl policy is replaced, then need to dump ACID tables if table is getting replicated
// for the first time in current dump. So, need to check if table is included in old policy.
return ReplUtils.includeAcidTableInDump(conf)
&& (conf.getBoolVar(HiveConf.ConfVars.REPL_BOOTSTRAP_ACID_TABLES)
|| !ReplUtils.tableIncludedInReplScope(work.oldReplScope, tableName));
}
private boolean shouldBootstrapDumpTable(Table table) {
// Note: If control reaches here, it means, table is already included in new replication policy.
if (TableType.EXTERNAL_TABLE.equals(table.getTableType())
&& shouldBootstrapDumpExternalTable(table.getTableName())) {
return true;
}
if (AcidUtils.isTransactionalTable(table)
&& shouldBootstrapDumpAcidTable(table.getTableName())) {
return true;
}
// If the table is renamed and the new name satisfies the filter but the old name does not then the table needs to
// be bootstrapped.
if (tablesForBootstrap.contains(table.getTableName().toLowerCase())) {
return true;
}
// If replication policy is changed with new included/excluded tables list, then tables which
// are not included in old policy but included in new policy should be bootstrapped along with
// the current incremental replication dump.
// Control reaches for Non-ACID tables.
return !ReplUtils.tableIncludedInReplScope(work.oldReplScope, table.getTableName());
}
private boolean doesTableSatisfyConfig(Table table) {
if (table == null) {
return false;
}
if (TableType.EXTERNAL_TABLE.equals(table.getTableType())
&& !conf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES)) {
return false;
}
if (AcidUtils.isTransactionalTable(table)
&& !ReplUtils.includeAcidTableInDump(conf)) {
return false;
}
return true;
}
private void fetchFailoverMetadata(Hive hiveDb) throws HiveException, IOException, TException {
Path hiveDumpDir = new Path(work.getCurrentDumpPath(), ReplUtils.REPL_HIVE_BASE_DIR);
FailoverMetaData fmd = new FailoverMetaData(hiveDumpDir, conf);
FileSystem fs = hiveDumpDir.getFileSystem(conf);
if (fs.exists(new Path(hiveDumpDir, FailoverMetaData.FAILOVER_METADATA)) && fmd.isValidMetadata()) {
work.setFailoverMetadata(fmd);
return;
}
HiveTxnManager hiveTxnManager = getTxnMgr();
List<Long> txnsForDb = getOpenTxns(hiveTxnManager, hiveTxnManager.getValidTxns(excludedTxns), work.dbNameOrPattern);
if (!txnsForDb.isEmpty()) {
LOG.debug("Going to abort transactions: {} for database: {}.", txnsForDb, work.dbNameOrPattern);
hiveDb.abortTransactions(txnsForDb, TxnErrorMsg.ABORT_FETCH_FAILOVER_METADATA.getErrorCode());
}
fmd.setAbortedTxns(txnsForDb);
fmd.setCursorPoint(currentNotificationId(hiveDb));
ValidTxnList allValidTxns = getTxnMgr().getValidTxns(excludedTxns);
List<Long> openTxns = getOpenTxns(allValidTxns);
fmd.setOpenTxns(openTxns);
fmd.setTxnsWithoutLock(getTxnsNotPresentInHiveLocksTable(openTxns));
txnsForDb = getOpenTxns(hiveTxnManager, allValidTxns, work.dbNameOrPattern);
if (!txnsForDb.isEmpty()) {
LOG.debug("Going to abort transactions: {} for database: {}.", txnsForDb, work.dbNameOrPattern);
hiveDb.abortTransactions(txnsForDb, TxnErrorMsg.ABORT_FETCH_FAILOVER_METADATA.getErrorCode());
fmd.addToAbortedTxns(txnsForDb);
}
fmd.setFailoverEventId(currentNotificationId(hiveDb));
fmd.write();
work.setFailoverMetadata(fmd);
}
private Long incrementalDump(Path dumpRoot, DumpMetaData dmd, Path cmRoot, Hive hiveDb) throws Exception {
Long lastReplId;// get list of events matching dbPattern & tblPattern
// go through each event, and dump out each event to a event-level dump dir inside dumproot
String validTxnList = null;
long waitUntilTime = 0;
long bootDumpBeginReplId = -1;
List<String> tableList = work.replScope.includeAllTables() ? null : new ArrayList<>();
SnapshotUtils.ReplSnapshotCount snapshotCount = null;
// If we are bootstrapping ACID tables, we need to perform steps similar to a regular
// bootstrap (See bootstrapDump() for more details. Only difference here is instead of
// waiting for the concurrent transactions to finish, we start dumping the incremental events
// and wait only for the remaining time if any.
if (needBootstrapAcidTablesDuringIncrementalDump()) {
work.setBootstrap(true);
bootDumpBeginReplId = queryState.getConf().getLong(ReplUtils.LAST_REPL_ID_KEY, -1L);
assert (bootDumpBeginReplId >= 0);
LOG.info("Dump for bootstrapping ACID tables during an incremental dump for db {}",
work.dbNameOrPattern);
long timeoutInMs = HiveConf.getTimeVar(conf,
HiveConf.ConfVars.REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT, TimeUnit.MILLISECONDS);
waitUntilTime = System.currentTimeMillis() + timeoutInMs;
}
// TODO : instead of simply restricting by message format, we should eventually
// move to a jdbc-driver-stype registering of message format, and picking message
// factory per event to decode. For now, however, since all messages have the
// same factory, restricting by message format is effectively a guard against
// older leftover data that would cause us problems.
String dbName = work.dbNameOrPattern;
Database db = hiveDb.getDatabase(dbName);
if (!HiveConf.getBoolVar(conf, REPL_DUMP_METADATA_ONLY)) {
setReplSourceFor(hiveDb, dbName, db);
}
if (shouldFailover()) {
if (!MetaStoreUtils.isDbBeingPlannedFailedOverAtEndpoint(db, MetaStoreUtils.FailoverEndpoint.SOURCE)) {
// set repl failover enabled at source
HashMap<String, String> params = new HashMap<>(db.getParameters());
params.put(ReplConst.REPL_FAILOVER_ENDPOINT, MetaStoreUtils.FailoverEndpoint.SOURCE.toString());
params.put(ReplConst.REPL_METRICS_LAST_FAILOVER_TYPE, ReplConst.FailoverType.PLANNED.toString());
LOG.info("Replication Metrics: Setting last failover type for database: {} to: {} ", dbName, ReplConst.FailoverType.PLANNED.toString());
int failoverCount = 1 + NumberUtils.toInt(params.getOrDefault(ReplConst.REPL_METRICS_FAILOVER_COUNT, "0"), 0);
LOG.info("Replication Metrics: Setting replication metrics failover count for target database: {} to: {} ", dbName, failoverCount);
params.put(ReplConst.REPL_METRICS_FAILOVER_COUNT, Integer.toString(failoverCount));
db.setParameters(params);
getHive().alterDatabase(work.dbNameOrPattern, db);
}
fetchFailoverMetadata(hiveDb);
assert work.getFailoverMetadata().isValidMetadata();
work.overrideLastEventToDump(hiveDb, bootDumpBeginReplId, work.getFailoverMetadata().getFailoverEventId());
} else {
work.overrideLastEventToDump(hiveDb, bootDumpBeginReplId, -1);
}
IMetaStoreClient.NotificationFilter evFilter = new AndFilter(
new ReplEventFilter(work.replScope),
new CatalogFilter(MetaStoreUtils.getDefaultCatalog(conf)),
new EventBoundaryFilter(work.eventFrom, work.eventTo));
EventUtils.MSClientNotificationFetcher evFetcher
= new EventUtils.MSClientNotificationFetcher(hiveDb);
int maxEventLimit = getMaxEventAllowed(work.maxEventLimit());
EventUtils.NotificationEventIterator evIter = new EventUtils.NotificationEventIterator(
evFetcher, work.eventFrom, maxEventLimit, evFilter);
lastReplId = work.eventTo;
Path ackFile = new Path(dumpRoot, ReplAck.EVENTS_DUMP.toString());
boolean shouldBatch = conf.getBoolVar(HiveConf.ConfVars.REPL_BATCH_INCREMENTAL_EVENTS);
EventsDumpMetadata eventsDumpMetadata =
Utils.fileExists(ackFile, conf) ? EventsDumpMetadata.deserialize(ackFile, conf)
: new EventsDumpMetadata(work.eventFrom, 0, shouldBatch);
long resumeFrom = eventsDumpMetadata.getLastReplId();
long estimatedNumEvents = evFetcher.getDbNotificationEventsCount(work.eventFrom, dbName, work.eventTo,
maxEventLimit);
try {
IncrementalDumpLogger replLogger =
new IncrementalDumpLogger(dbName, dumpRoot.toString(), estimatedNumEvents, work.eventFrom, work.eventTo,
maxEventLimit);
work.setReplLogger(replLogger);
replLogger.startLog();
Map<String, Long> metricMap = new HashMap<>();
metricMap.put(ReplUtils.MetricName.EVENTS.name(), estimatedNumEvents);
int size = tablesForBootstrap.size();
if (db != null && db.getParameters()!=null &&
Boolean.parseBoolean(db.getParameters().get(REPL_RESUME_STARTED_AFTER_FAILOVER))) {
Collection<String> allTables = Utils.getAllTables(hiveDb, dbName, work.replScope);
allTables.retainAll(tablesForBootstrap);
size = allTables.size();
}
if (size > 0) {
metricMap.put(ReplUtils.MetricName.TABLES.name(), (long) size);
}
if (shouldFailover()) {
Map<String, String> params = db.getParameters();
String dbFailoverEndPoint = "";
if (params != null) {
dbFailoverEndPoint = params.get(ReplConst.REPL_FAILOVER_ENDPOINT);
LOG.debug("Replication Metrics: setting failover endpoint to {} ", dbFailoverEndPoint);
} else {
LOG.warn("Replication Metrics: Cannot obtained failover endpoint info, setting failover endpoint to null ");
}
work.getMetricCollector().reportFailoverStart(getName(), metricMap, work.getFailoverMetadata(), dbFailoverEndPoint, ReplConst.FailoverType.PLANNED.toString());
} else {
work.getMetricCollector().reportStageStart(getName(), metricMap);
}
long dumpedCount = resumeFrom - work.eventFrom;
if (dumpedCount > 0) {
LOG.info("Event id {} to {} are already dumped, skipping {} events", work.eventFrom, resumeFrom, dumpedCount);
}
boolean isStagingDirCheckedForFailedEvents = false;
int batchNo = 0, eventCount = 0;
final int maxEventsPerBatch = conf.getIntVar(HiveConf.ConfVars.REPL_APPROX_MAX_LOAD_TASKS);
Path eventRootDir = dumpRoot;
if (shouldBatch && maxEventsPerBatch == 0) {
throw new SemanticException(String.format(
"batch size configured via %s cannot be set to zero since batching is enabled",
HiveConf.ConfVars.REPL_APPROX_MAX_LOAD_TASKS.varname));
}
if (eventsDumpMetadata.isEventsBatched() != shouldBatch) {
LOG.error("Failed to resume from previous dump. {} was set to {} in previous dump but currently it's" +
" set to {}. Cannot dump events in {} manner because they were {} batched in " +
"the previous incomplete run",
HiveConf.ConfVars.REPL_BATCH_INCREMENTAL_EVENTS.varname, eventsDumpMetadata.isEventsBatched(),
shouldBatch, shouldBatch ? "batched" : "sequential", shouldBatch ? "not" : ""
);
throw new HiveException(
String.format("Failed to resume from previous dump. %s must be set to %s, but currently it's set to %s",
HiveConf.ConfVars.REPL_BATCH_INCREMENTAL_EVENTS,
eventsDumpMetadata.isEventsBatched(), shouldBatch)
);
}
while (evIter.hasNext()) {
NotificationEvent ev = evIter.next();