-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
BaseHoodieWriteClient.java
1423 lines (1299 loc) · 64.9 KB
/
BaseHoodieWriteClient.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.hudi.client;
import org.apache.hudi.async.AsyncArchiveService;
import org.apache.hudi.async.AsyncCleanerService;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRestorePlan;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieRollbackPlan;
import org.apache.hudi.callback.HoodieWriteCommitCallback;
import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.heartbeat.HeartbeatUtils;
import org.apache.hudi.client.transaction.TransactionManager;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.HoodiePendingRollbackInfo;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.TableServiceType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieRestoreException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metrics.HoodieMetrics;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.rollback.RollbackUtils;
import org.apache.hudi.table.action.savepoint.SavepointHelpers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade;
import org.apache.hudi.table.upgrade.UpgradeDowngrade;
import com.codahale.metrics.Timer;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Abstract Write Client providing functionality for performing commit, index updates and rollback
* Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap
*
* @param <T> Sub type of HoodieRecordPayload
* @param <I> Type of inputs
* @param <K> Type of keys
* @param <O> Type of outputs
*/
public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K, O> extends BaseHoodieClient
implements RunsTableService {
protected static final String LOOKUP_STR = "lookup";
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(BaseHoodieWriteClient.class);
private final transient HoodieIndex<?, ?> index;
private final SupportsUpgradeDowngrade upgradeDowngradeHelper;
private transient WriteOperationType operationType;
private transient HoodieWriteCommitCallback commitCallback;
protected final transient HoodieMetrics metrics;
protected transient Timer.Context writeTimer = null;
protected transient Timer.Context compactionTimer;
protected transient Timer.Context clusteringTimer;
protected transient AsyncCleanerService asyncCleanerService;
protected transient AsyncArchiveService asyncArchiveService;
protected final TransactionManager txnManager;
protected Option<Pair<HoodieInstant, Map<String, String>>> lastCompletedTxnAndMetadata = Option.empty();
/**
* Create a write client, with new hudi index.
* @param context HoodieEngineContext
* @param writeConfig instance of HoodieWriteConfig
* @param upgradeDowngradeHelper engine-specific instance of {@link SupportsUpgradeDowngrade}
*/
@Deprecated
public BaseHoodieWriteClient(HoodieEngineContext context,
HoodieWriteConfig writeConfig,
SupportsUpgradeDowngrade upgradeDowngradeHelper) {
this(context, writeConfig, Option.empty(), upgradeDowngradeHelper);
}
/**
* Create a write client, allows to specify all parameters.
*
* @param context HoodieEngineContext
* @param writeConfig instance of HoodieWriteConfig
* @param timelineService Timeline Service that runs as part of write client.
*/
@Deprecated
public BaseHoodieWriteClient(HoodieEngineContext context,
HoodieWriteConfig writeConfig,
Option<EmbeddedTimelineService> timelineService,
SupportsUpgradeDowngrade upgradeDowngradeHelper) {
super(context, writeConfig, timelineService);
this.metrics = new HoodieMetrics(config);
this.index = createIndex(writeConfig);
this.txnManager = new TransactionManager(config, fs);
this.upgradeDowngradeHelper = upgradeDowngradeHelper;
}
protected abstract HoodieIndex<?, ?> createIndex(HoodieWriteConfig writeConfig);
public void setOperationType(WriteOperationType operationType) {
this.operationType = operationType;
}
public WriteOperationType getOperationType() {
return this.operationType;
}
/**
* Commit changes performed at the given instantTime marker.
*/
public boolean commit(String instantTime, O writeStatuses) {
return commit(instantTime, writeStatuses, Option.empty());
}
/**
*
* Commit changes performed at the given instantTime marker.
*/
public boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata) {
HoodieTableMetaClient metaClient = createMetaClient(false);
String actionType = metaClient.getCommitActionType();
return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap());
}
public abstract boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata,
String commitActionType, Map<String, List<String>> partitionToReplacedFileIds);
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
String commitActionType) {
return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap());
}
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
// Skip the empty commit if not allowed
if (!config.allowEmptyCommit() && stats.isEmpty()) {
return true;
}
LOG.info("Committing " + instantTime + " action " + commitActionType);
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = createTable(config, hadoopConf);
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds,
extraMetadata, operationType, config.getWriteSchema(), commitActionType);
HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime);
HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config);
this.txnManager.beginTransaction(Option.of(inflightInstant),
lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty());
try {
preCommit(inflightInstant, metadata);
commit(table, commitActionType, instantTime, metadata, stats);
postCommit(table, metadata, instantTime, extraMetadata);
LOG.info("Committed " + instantTime);
releaseResources();
} catch (IOException e) {
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e);
} finally {
this.txnManager.endTransaction(Option.of(inflightInstant));
}
// do this outside of lock since compaction, clustering can be time taking and we don't need a lock for the entire execution period
runTableServicesInline(table, metadata, extraMetadata);
emitCommitMetrics(instantTime, metadata, commitActionType);
// callback if needed.
if (config.writeCommitCallbackOn()) {
if (null == commitCallback) {
commitCallback = HoodieCommitCallbackFactory.create(config);
}
commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath(), stats));
}
return true;
}
protected void commit(HoodieTable table, String commitActionType, String instantTime, HoodieCommitMetadata metadata,
List<HoodieWriteStat> stats) throws IOException {
LOG.info("Committing " + instantTime + " action " + commitActionType);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
// Finalize write
finalizeWrite(table, instantTime, stats);
// update Metadata table
writeTableMetadata(table, instantTime, commitActionType, metadata);
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
}
protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) {
return createTable(config, hadoopConf, false);
}
protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline);
void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) {
try {
if (writeTimer != null) {
long durationInMs = metrics.getDurationInMs(writeTimer.stop());
metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(), durationInMs,
metadata, actionType);
writeTimer = null;
}
} catch (ParseException e) {
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime
+ "Instant time is not of valid format", e);
}
}
/**
* Any pre-commit actions like conflict resolution goes here.
* @param inflightInstant instant of inflight operation.
* @param metadata commit metadata for which pre commit is being invoked.
*/
protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
// To be overridden by specific engines to perform conflict resolution if any.
}
/**
* Write the HoodieCommitMetadata to metadata table if available.
* @param table {@link HoodieTable} of interest.
* @param instantTime instant time of the commit.
* @param actionType action type of the commit.
* @param metadata instance of {@link HoodieCommitMetadata}.
*/
protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) {
context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table");
table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime,
table.isTableServiceAction(actionType)));
}
/**
* Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication.
*
* @param hoodieRecords Input Hoodie records.
* @return A subset of hoodieRecords, with existing records filtered out.
*/
public abstract I filterExists(I hoodieRecords);
/**
* Main API to run bootstrap to hudi.
*/
public void bootstrap(Option<Map<String, String>> extraMetadata) {
// TODO : MULTIWRITER -> check if failed bootstrap files can be cleaned later
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
throw new HoodieException("Cannot bootstrap the table in multi-writer mode");
}
HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UPSERT, Option.ofNullable(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS));
rollbackFailedBootstrap();
table.bootstrap(context, extraMetadata);
}
/**
* Main API to rollback failed bootstrap.
*/
protected void rollbackFailedBootstrap() {
LOG.info("Rolling back pending bootstrap if present");
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
Option<String> instant = Option.fromJavaOptional(
inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst());
if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS,
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
LOG.info("Found pending bootstrap instants. Rolling them back");
table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime());
LOG.info("Finished rolling back pending bootstrap");
}
}
/**
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
*
* @param records hoodieRecords to upsert
* @param instantTime Instant time of the commit
* @return WriteStatus to inspect errors and counts
*/
public abstract O upsert(I records, final String instantTime);
/**
* Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
* <p>
* This implementation requires that the input records are already tagged, and de-duped if needed.
*
* @param preppedRecords Prepared HoodieRecords to upsert
* @param instantTime Instant time of the commit
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O upsertPreppedRecords(I preppedRecords, final String instantTime);
/**
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal writes.
* <p>
* This implementation skips the index check and is able to leverage benefits such as small file handling/blocking
* alignment, as with upsert(), by profiling the workload
*
* @param records HoodieRecords to insert
* @param instantTime Instant time of the commit
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O insert(I records, final String instantTime);
/**
* Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
* <p>
* This implementation skips the index check, skips de-duping and is able to leverage benefits such as small file
* handling/blocking alignment, as with insert(), by profiling the workload. The prepared HoodieRecords should be
* de-duped if needed.
*
* @param preppedRecords HoodieRecords to insert
* @param instantTime Instant time of the commit
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O insertPreppedRecords(I preppedRecords, final String instantTime);
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing table to Hoodie).
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
* the numbers of files with less memory compared to the {@link BaseHoodieWriteClient#insert(I, String)}
*
* @param records HoodieRecords to insert
* @param instantTime Instant time of the commit
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O bulkInsert(I records, final String instantTime);
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing table to Hoodie).
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
* the numbers of files with less memory compared to the {@link BaseHoodieWriteClient#insert(I, String)}. Optionally
* it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
* {@link BulkInsertPartitioner}.
*
* @param records HoodieRecords to insert
* @param instantTime Instant time of the commit
* @param userDefinedBulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
* into hoodie.
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O bulkInsert(I records, final String instantTime,
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner);
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
* table for the very first time (e.g: converting an existing table to Hoodie). The input records should contain no
* duplicates if needed.
* <p>
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
* the numbers of files with less memory compared to the {@link BaseHoodieWriteClient#insert(I, String)}. Optionally
* it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
* {@link BulkInsertPartitioner}.
*
* @param preppedRecords HoodieRecords to insert
* @param instantTime Instant time of the commit
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are inserted
* into hoodie.
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime,
Option<BulkInsertPartitioner> bulkInsertPartitioner);
/**
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
* de-duped and non existent keys will be removed before deleting.
*
* @param keys {@link List} of {@link HoodieKey}s to be deleted
* @param instantTime Commit time handle
* @return Collection of WriteStatus to inspect errors and counts
*/
public abstract O delete(K keys, final String instantTime);
/**
* Common method containing steps to be performed before write (upsert/insert/...
* @param instantTime
* @param writeOperationType
* @param metaClient
*/
protected void preWrite(String instantTime, WriteOperationType writeOperationType,
HoodieTableMetaClient metaClient) {
setOperationType(writeOperationType);
this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient);
if (null == this.asyncCleanerService) {
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this);
} else {
this.asyncCleanerService.start(null);
}
if (null == this.asyncArchiveService) {
this.asyncArchiveService = AsyncArchiveService.startAsyncArchiveIfEnabled(this);
} else {
this.asyncArchiveService.start(null);
}
}
/**
* Common method containing steps to be performed after write (upsert/insert/..) operations including auto-commit.
* @param result Commit Action Result
* @param instantTime Instant Time
* @param hoodieTable Hoodie Table
* @return Write Status
*/
protected abstract O postWrite(HoodieWriteMetadata<O> result, String instantTime, HoodieTable hoodieTable);
/**
* Post Commit Hook. Derived classes use this method to perform post-commit processing
*
* @param table table to commit on
* @param metadata Commit Metadata corresponding to committed instant
* @param instantTime Instant Time
* @param extraMetadata Additional Metadata passed by user
*/
protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
try {
// Delete the marker directory for the instant.
WriteMarkersFactory.get(config.getMarkersType(), table, instantTime)
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
autoCleanOnCommit();
autoArchiveOnCommit(table);
} finally {
this.heartbeatClient.stop(instantTime);
}
}
protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option<Map<String, String>> extraMetadata) {
if (!tableServicesEnabled(config)) {
return;
}
if (config.areAnyTableServicesExecutedInline() || config.areAnyTableServicesScheduledInline()) {
if (config.isMetadataTableEnabled()) {
table.getHoodieView().sync();
}
// Do an inline compaction if enabled
if (config.inlineCompactionEnabled()) {
runAnyPendingCompactions(table);
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "true");
inlineCompaction(extraMetadata);
} else {
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT.key(), "false");
}
// if just inline schedule is enabled
if (!config.inlineCompactionEnabled() && config.scheduleInlineCompaction()
&& !table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants().findAny().isPresent()) {
// proceed only if there are no pending compactions
metadata.addMetadata(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), "true");
inlineScheduleCompaction(extraMetadata);
}
// Do an inline clustering if enabled
if (config.inlineClusteringEnabled()) {
runAnyPendingClustering(table);
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "true");
inlineClustering(extraMetadata);
} else {
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING.key(), "false");
}
// if just inline schedule is enabled
if (!config.inlineClusteringEnabled() && config.scheduleInlineClustering()
&& !table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().findAny().isPresent()) {
// proceed only if there are no pending clustering
metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), "true");
inlineScheduleClustering(extraMetadata);
}
}
}
protected void runAnyPendingCompactions(HoodieTable table) {
table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants()
.forEach(instant -> {
LOG.info("Running previously failed inflight compaction at instant " + instant);
compact(instant.getTimestamp(), true);
});
}
protected void runAnyPendingClustering(HoodieTable table) {
table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> {
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
if (instantPlan.isPresent()) {
LOG.info("Running pending clustering at instant " + instantPlan.get().getLeft());
cluster(instant.getTimestamp(), true);
}
});
}
protected void autoCleanOnCommit() {
if (!config.isAutoClean()) {
return;
}
if (config.isAsyncClean()) {
LOG.info("Async cleaner has been spawned. Waiting for it to finish");
AsyncCleanerService.waitForCompletion(asyncCleanerService);
LOG.info("Async cleaner has finished");
} else {
LOG.info("Start to clean synchronously.");
// Do not reuse instantTime for clean as metadata table requires all changes to have unique instant timestamps.
clean(true);
}
}
protected void autoArchiveOnCommit(HoodieTable table) {
if (!config.isAutoArchive()) {
return;
}
if (config.isAsyncArchive()) {
LOG.info("Async archiver has been spawned. Waiting for it to finish");
AsyncArchiveService.waitForCompletion(asyncArchiveService);
LOG.info("Async archiver has finished");
} else {
LOG.info("Start to archive synchronously.");
archive(table);
}
}
/**
* Run any pending compactions.
*/
public void runAnyPendingCompactions() {
runAnyPendingCompactions(createTable(config, hadoopConf, config.isMetadataTableEnabled()));
}
/**
* Create a savepoint based on the latest commit action on the timeline.
*
* @param user - User creating the savepoint
* @param comment - Comment for the savepoint
*/
public void savepoint(String user, String comment) {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
if (table.getCompletedCommitsTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
}
String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().getTimestamp();
LOG.info("Savepointing latest commit " + latestCommit);
savepoint(latestCommit, user, comment);
}
/**
* Savepoint a specific commit instant time. Latest version of data files as of the passed in instantTime
* will be referenced in the savepoint and will never be cleaned. The savepointed commit will never be rolledback or archived.
* <p>
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be manually created and
* deleted.
* <p>
* Savepoint should be on a commit that could not have been cleaned.
*
* @param instantTime - commit that should be savepointed
* @param user - User creating the savepoint
* @param comment - Comment for the savepoint
*/
public void savepoint(String instantTime, String user, String comment) {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
table.savepoint(context, instantTime, user, comment);
}
/**
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be rolledback and cleaner may
* clean up data files.
*
* @param savepointTime - delete the savepoint
* @return true if the savepoint was deleted successfully
*/
public void deleteSavepoint(String savepointTime) {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
SavepointHelpers.deleteSavepoint(table, savepointTime);
}
/**
* Restore the data to the savepoint.
*
* WARNING: This rolls back recent commits and deleted data files and also pending compactions after savepoint time.
* Queries accessing the files will mostly fail. This is expected to be a manual operation and no concurrent write or
* compaction is expected to be running
*
* @param savepointTime - savepoint time to rollback to
* @return true if the savepoint was restored to successfully
*/
public void restoreToSavepoint(String savepointTime) {
HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty());
SavepointHelpers.validateSavepointPresence(table, savepointTime);
restoreToInstant(savepointTime);
SavepointHelpers.validateSavepointRestore(table, savepointTime);
}
@Deprecated
public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty());
Option<HoodiePendingRollbackInfo> pendingRollbackInfo = getPendingRollbackInfo(table.getMetaClient(), commitInstantTime);
return rollback(commitInstantTime, pendingRollbackInfo, false);
}
/**
* @Deprecated
* Rollback the inflight record changes with the given commit time. This
* will be removed in future in favor of {@link BaseHoodieWriteClient#restoreToInstant(String)}
*
* @param commitInstantTime Instant time of the commit
* @param pendingRollbackInfo pending rollback instant and plan if rollback failed from previous attempt.
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
* @throws HoodieRollbackException if rollback cannot be performed successfully
*/
@Deprecated
public boolean rollback(final String commitInstantTime, Option<HoodiePendingRollbackInfo> pendingRollbackInfo, boolean skipLocking) throws HoodieRollbackException {
LOG.info("Begin rollback of instant " + commitInstantTime);
final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
final Timer.Context timerContext = this.metrics.getRollbackCtx();
try {
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
.findFirst());
if (commitInstantOpt.isPresent() || pendingRollbackInfo.isPresent()) {
LOG.info(String.format("Scheduling Rollback at instant time : %s "
+ "(exists in active timeline: %s), with rollback plan: %s",
rollbackInstantTime, commitInstantOpt.isPresent(), pendingRollbackInfo.isPresent()));
Option<HoodieRollbackPlan> rollbackPlanOption = pendingRollbackInfo.map(entry -> Option.of(entry.getRollbackPlan()))
.orElseGet(() -> table.scheduleRollback(context, rollbackInstantTime, commitInstantOpt.get(), false, config.shouldRollbackUsingMarkers()));
if (rollbackPlanOption.isPresent()) {
// There can be a case where the inflight rollback failed after the instant files
// are deleted for commitInstantTime, so that commitInstantOpt is empty as it is
// not present in the timeline. In such a case, the hoodie instant instance
// is reconstructed to allow the rollback to be reattempted, and the deleteInstants
// is set to false since they are already deleted.
// Execute rollback
HoodieRollbackMetadata rollbackMetadata = commitInstantOpt.isPresent()
? table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true, skipLocking)
: table.rollback(context, rollbackInstantTime, new HoodieInstant(
true, rollbackPlanOption.get().getInstantToRollback().getAction(), commitInstantTime),
false, skipLocking);
if (timerContext != null) {
long durationInMs = metrics.getDurationInMs(timerContext.stop());
metrics.updateRollbackMetrics(durationInMs, rollbackMetadata.getTotalFilesDeleted());
}
return true;
} else {
throw new HoodieRollbackException("Failed to rollback " + config.getBasePath() + " commits " + commitInstantTime);
}
} else {
LOG.warn("Cannot find instant " + commitInstantTime + " in the timeline, for rollback");
return false;
}
} catch (Exception e) {
throw new HoodieRollbackException("Failed to rollback " + config.getBasePath() + " commits " + commitInstantTime, e);
}
}
/**
* NOTE : This action requires all writers (ingest and compact) to a table to be stopped before proceeding. Revert
* the (inflight/committed) record changes for all commits after the provided instant time.
*
* @param instantTime Instant time to which restoration is requested
*/
public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws HoodieRestoreException {
LOG.info("Begin restore to instant " + instantTime);
final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime();
Timer.Context timerContext = metrics.getRollbackCtx();
try {
HoodieTable<T, I, K, O> table = initTable(WriteOperationType.UNKNOWN, Option.empty());
Option<HoodieRestorePlan> restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime);
if (restorePlanOption.isPresent()) {
HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
if (timerContext != null) {
final long durationInMs = metrics.getDurationInMs(timerContext.stop());
final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
.flatMap(Collection::stream)
.mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
.sum();
metrics.updateRollbackMetrics(durationInMs, totalFilesDeleted);
}
return restoreMetadata;
} else {
throw new HoodieRestoreException("Failed to restore " + config.getBasePath() + " to commit " + instantTime);
}
} catch (Exception e) {
throw new HoodieRestoreException("Failed to restore to " + instantTime, e);
}
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
* cleaned)
*/
public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
return clean(cleanInstantTime, true, false);
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
* cleaned)
* @param cleanInstantTime instant time for clean.
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
* @return instance of {@link HoodieCleanMetadata}.
*/
public HoodieCleanMetadata clean(String cleanInstantTime, boolean skipLocking) throws HoodieIOException {
return clean(cleanInstantTime, true, skipLocking);
}
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
* cleaned). This API provides the flexibility to schedule clean instant asynchronously via
* {@link BaseHoodieWriteClient#scheduleTableService(String, Option, TableServiceType)} and disable inline scheduling
* of clean.
* @param cleanInstantTime instant time for clean.
* @param scheduleInline true if needs to be scheduled inline. false otherwise.
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
*/
public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline, boolean skipLocking) throws HoodieIOException {
if (!tableServicesEnabled(config)) {
return null;
}
final Timer.Context timerContext = metrics.getCleanCtx();
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites(skipLocking));
HoodieCleanMetadata metadata = null;
HoodieTable table = createTable(config, hadoopConf);
if (config.allowMultipleCleans() || !table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent()) {
LOG.info("Cleaner started");
// proceed only if multiple clean schedules are enabled or if there are no pending cleans.
if (scheduleInline) {
scheduleTableServiceInternal(cleanInstantTime, Option.empty(), TableServiceType.CLEAN);
table.getMetaClient().reloadActiveTimeline();
}
metadata = table.clean(context, cleanInstantTime, skipLocking);
if (timerContext != null && metadata != null) {
long durationMs = metrics.getDurationInMs(timerContext.stop());
metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
+ " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain()
+ " cleanerElapsedMs" + durationMs);
}
}
return metadata;
}
public HoodieCleanMetadata clean() {
return clean(false);
}
/**
* Triggers clean for the table. This refers to Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
* * configurations and CleaningPolicy used.
* @param skipLocking if this is triggered by another parent transaction, locking can be skipped.
* @return instance of {@link HoodieCleanMetadata}.
*/
public HoodieCleanMetadata clean(boolean skipLocking) {
return clean(HoodieActiveTimeline.createNewInstantTime(), skipLocking);
}
/**
* Trigger archival for the table. This ensures that the number of commits do not explode
* and keep increasing unbounded over time.
* @param table table to commit on.
*/
protected void archive(HoodieTable table) {
if (!tableServicesEnabled(config)) {
return;
}
try {
// We cannot have unbounded commit files. Archive commits if we have to archive
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table);
archiver.archiveIfRequired(context);
} catch (IOException ioe) {
throw new HoodieIOException("Failed to archive", ioe);
}
}
/**
* Trigger archival for the table. This ensures that the number of commits do not explode
* and keep increasing unbounded over time.
*/
public void archive() {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = createTable(config, hadoopConf);
archive(table);
}
/**
* Provides a new commit time for a write operation (insert/update/delete).
*/
public String startCommit() {
HoodieTableMetaClient metaClient = createMetaClient(true);
return startCommit(metaClient.getCommitActionType(), metaClient);
}
/**
* Provides a new commit time for a write operation (insert/update/delete/insert_overwrite/insert_overwrite_table) with specified action.
*/
public String startCommit(String actionType, HoodieTableMetaClient metaClient) {
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites());
String instantTime = HoodieActiveTimeline.createNewInstantTime();
startCommit(instantTime, actionType, metaClient);
return instantTime;
}
/**
* Provides a new commit time for a write operation (insert/update/delete/insert_overwrite/insert_overwrite_table) without specified action.
* @param instantTime Instant time to be generated
*/
public void startCommitWithTime(String instantTime) {
HoodieTableMetaClient metaClient = createMetaClient(true);
startCommitWithTime(instantTime, metaClient.getCommitActionType(), metaClient);
}
/**
* Completes a new commit time for a write operation (insert/update/delete/insert_overwrite/insert_overwrite_table) with specified action.
*/
public void startCommitWithTime(String instantTime, String actionType) {
HoodieTableMetaClient metaClient = createMetaClient(true);
startCommitWithTime(instantTime, actionType, metaClient);
}
/**
* Completes a new commit time for a write operation (insert/update/delete) with specified action.
*/
private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) {
CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(),
HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites());
startCommit(instantTime, actionType, metaClient);
}
private void startCommit(String instantTime, String actionType, HoodieTableMetaClient metaClient) {
LOG.info("Generate a new instant time: " + instantTime + " action: " + actionType);
// if there are pending compactions, their instantTime must not be greater than that of this instant time
metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending ->
ValidationUtils.checkArgument(
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime));
if (config.getFailedWritesCleanPolicy().isLazy()) {
this.heartbeatClient.start(instantTime);
}
if (actionType.equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
metaClient.getActiveTimeline().createRequestedReplaceCommit(instantTime, actionType);
} else {
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType,
instantTime));
}
}
/**
* Schedules a new compaction instant.
* @param extraMetadata Extra Metadata to be stored
*/
public Option<String> scheduleCompaction(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
String instantTime = HoodieActiveTimeline.createNewInstantTime();
return scheduleCompactionAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty();
}
/**
* Schedules a new compaction instant with passed-in instant time.
* @param instantTime Compaction Instant Time
* @param extraMetadata Extra Metadata to be stored
*/
public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
return scheduleTableService(instantTime, extraMetadata, TableServiceType.COMPACT).isPresent();
}
/**
* Performs Compaction for the workload stored in instant-time.
*
* @param compactionInstantTime Compaction Instant Time
* @return Collection of WriteStatus to inspect errors and counts
*/
public HoodieWriteMetadata<O> compact(String compactionInstantTime) {
return compact(compactionInstantTime, config.shouldAutoCommit());
}
/**
* Commit a compaction operation. Allow passing additional meta-data to be stored in commit instant file.
*
* @param compactionInstantTime Compaction Instant Time
* @param metadata All the metadata that gets stored along with a commit
* @param extraMetadata Extra Metadata to be stored
*/
public abstract void commitCompaction(String compactionInstantTime, HoodieCommitMetadata metadata,
Option<Map<String, String>> extraMetadata);
/**
* Commit Compaction and track metrics.
*/
protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime);
/**
* Get inflight time line exclude compaction and clustering.
* @param metaClient
* @return
*/
private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTableMetaClient metaClient) {
HoodieTimeline inflightTimelineWithReplaceCommit = metaClient.getCommitsTimeline().filterPendingExcludingCompaction();
HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> {
if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(metaClient, instant);
return !instantPlan.isPresent();
} else {
return true;
}
});
return inflightTimelineExcludeClusteringCommit;
}
protected Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) {
return getPendingRollbackInfo(metaClient, commitToRollback, true);
}
protected Option<HoodiePendingRollbackInfo> getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback, boolean ignoreCompactionAndClusteringInstants) {
return getPendingRollbackInfos(metaClient, ignoreCompactionAndClusteringInstants).getOrDefault(commitToRollback, Option.empty());
}
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient) {
return getPendingRollbackInfos(metaClient, true);
}
/**
* Fetch map of pending commits to be rolled-back to {@link HoodiePendingRollbackInfo}.
* @param metaClient instance of {@link HoodieTableMetaClient} to use.
* @return map of pending commits to be rolled-back instants to Rollback Instant and Rollback plan Pair.
*/
protected Map<String, Option<HoodiePendingRollbackInfo>> getPendingRollbackInfos(HoodieTableMetaClient metaClient, boolean ignoreCompactionAndClusteringInstants) {
List<HoodieInstant> instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().collect(Collectors.toList());
Map<String, Option<HoodiePendingRollbackInfo>> infoMap = new HashMap<>();
for (HoodieInstant instant : instants) {
try {
HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, instant);
String action = rollbackPlan.getInstantToRollback().getAction();
if (ignoreCompactionAndClusteringInstants) {
if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) {
boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)
&& ClusteringUtils.getClusteringPlan(metaClient, new HoodieInstant(true, rollbackPlan.getInstantToRollback().getAction(),
rollbackPlan.getInstantToRollback().getCommitTime())).isPresent();
if (!isClustering) {