-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
IndexerSQLMetadataStorageCoordinator.java
2999 lines (2713 loc) · 113 KB
/
IndexerSQLMetadataStorageCoordinator.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.druid.metadata;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.hash.Hashing;
import com.google.common.io.BaseEncoding;
import com.google.inject.Inject;
import org.apache.commons.lang.StringEscapeUtils;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.SegmentCreateRequest;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.segment.SegmentMetadata;
import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.SegmentSchemaManager;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.server.http.DataSegmentPlus;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.PartitionIds;
import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.PreparedBatch;
import org.skife.jdbi.v2.PreparedBatchPart;
import org.skife.jdbi.v2.Query;
import org.skife.jdbi.v2.ResultIterator;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.TransactionCallback;
import org.skife.jdbi.v2.TransactionStatus;
import org.skife.jdbi.v2.Update;
import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.util.ByteArrayMapper;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
*
*/
public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
{
private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class);
private static final int MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE = 100;
private static final String UPGRADED_PENDING_SEGMENT_PREFIX = "upgraded_to_version__";
private final ObjectMapper jsonMapper;
private final MetadataStorageTablesConfig dbTables;
private final SQLMetadataConnector connector;
private final SegmentSchemaManager segmentSchemaManager;
private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig;
private final boolean schemaPersistEnabled;
@Inject
public IndexerSQLMetadataStorageCoordinator(
ObjectMapper jsonMapper,
MetadataStorageTablesConfig dbTables,
SQLMetadataConnector connector,
SegmentSchemaManager segmentSchemaManager,
CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig
)
{
this.jsonMapper = jsonMapper;
this.dbTables = dbTables;
this.connector = connector;
this.segmentSchemaManager = segmentSchemaManager;
this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig;
this.schemaPersistEnabled =
centralizedDatasourceSchemaConfig.isEnabled()
&& !centralizedDatasourceSchemaConfig.isTaskSchemaPublishDisabled();
}
@LifecycleStart
public void start()
{
connector.createDataSourceTable();
connector.createPendingSegmentsTable();
if (centralizedDatasourceSchemaConfig.isEnabled()) {
connector.createSegmentSchemasTable();
}
connector.createSegmentTable();
connector.createUpgradeSegmentsTable();
}
@Override
public Collection<DataSegment> retrieveUsedSegmentsForIntervals(
final String dataSource,
final List<Interval> intervals,
final Segments visibility
)
{
if (intervals == null || intervals.isEmpty()) {
throw new IAE("null/empty intervals");
}
return doRetrieveUsedSegments(dataSource, intervals, visibility);
}
@Override
public Collection<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility)
{
return doRetrieveUsedSegments(dataSource, Collections.emptyList(), visibility);
}
/**
* @param intervals empty list means unrestricted interval.
*/
private Collection<DataSegment> doRetrieveUsedSegments(
final String dataSource,
final List<Interval> intervals,
final Segments visibility
)
{
return connector.retryWithHandle(
handle -> {
if (visibility == Segments.ONLY_VISIBLE) {
final SegmentTimeline timeline =
getTimelineForIntervalsWithHandle(handle, dataSource, intervals);
return timeline.findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE);
} else {
return retrieveAllUsedSegmentsForIntervalsWithHandle(handle, dataSource, intervals);
}
}
);
}
@Override
public List<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource, List<Interval> intervals)
{
StringBuilder queryBuilder = new StringBuilder(
"SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true"
);
final boolean compareIntervalEndpointsAsString = intervals.stream()
.allMatch(Intervals::canCompareEndpointsAsStrings);
final SqlSegmentsMetadataQuery.IntervalMode intervalMode = SqlSegmentsMetadataQuery.IntervalMode.OVERLAPS;
queryBuilder.append(
SqlSegmentsMetadataQuery.getConditionForIntervalsAndMatchMode(
compareIntervalEndpointsAsString ? intervals : Collections.emptyList(),
intervalMode,
connector.getQuoteString()
)
);
final String queryString = StringUtils.format(queryBuilder.toString(), dbTables.getSegmentsTable());
return connector.retryWithHandle(
handle -> {
Query<Map<String, Object>> query = handle
.createQuery(queryString)
.bind("dataSource", dataSource);
if (compareIntervalEndpointsAsString) {
SqlSegmentsMetadataQuery.bindIntervalsToQuery(query, intervals);
}
final List<Pair<DataSegment, String>> segmentsWithCreatedDates = query
.map((int index, ResultSet r, StatementContext ctx) ->
new Pair<>(
JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), DataSegment.class),
r.getString("created_date")
)
)
.list();
if (intervals.isEmpty() || compareIntervalEndpointsAsString) {
return segmentsWithCreatedDates;
} else {
return segmentsWithCreatedDates
.stream()
.filter(pair -> {
for (Interval interval : intervals) {
if (intervalMode.apply(interval, pair.lhs.getInterval())) {
return true;
}
}
return false;
}).collect(Collectors.toList());
}
}
);
}
@Override
public List<DataSegment> retrieveUnusedSegmentsForInterval(
String dataSource,
Interval interval,
@Nullable List<String> versions,
@Nullable Integer limit,
@Nullable DateTime maxUsedStatusLastUpdatedTime
)
{
final List<DataSegment> matchingSegments = connector.inReadOnlyTransaction(
(handle, status) -> {
try (final CloseableIterator<DataSegment> iterator =
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
.retrieveUnusedSegments(
dataSource,
Collections.singletonList(interval),
versions,
limit,
null,
null,
maxUsedStatusLastUpdatedTime
)
) {
return ImmutableList.copyOf(iterator);
}
}
);
log.info("Found [%,d] unused segments for datasource[%s] in interval[%s] and versions[%s] with maxUsedStatusLastUpdatedTime[%s].",
matchingSegments.size(), dataSource, interval, versions, maxUsedStatusLastUpdatedTime);
return matchingSegments;
}
@Override
public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interval)
{
final Integer numSegmentsMarkedUnused = connector.retryTransaction(
(handle, status) ->
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
.markSegmentsUnused(dataSource, interval),
3,
SQLMetadataConnector.DEFAULT_MAX_TRIES
);
log.info("Marked %,d segments unused for %s for interval %s.", numSegmentsMarkedUnused, dataSource, interval);
return numSegmentsMarkedUnused;
}
/**
* Fetches all the pending segments, whose interval overlaps with the given search interval, from the metadata store.
*/
@VisibleForTesting
List<PendingSegmentRecord> getPendingSegmentsForIntervalWithHandle(
final Handle handle,
final String dataSource,
final Interval interval
)
{
final boolean compareIntervalEndpointsAsStrings = Intervals.canCompareEndpointsAsStrings(interval);
String sql = "SELECT payload, sequence_name, sequence_prev_id, task_allocator_id, upgraded_from_segment_id"
+ " FROM " + dbTables.getPendingSegmentsTable()
+ " WHERE dataSource = :dataSource";
if (compareIntervalEndpointsAsStrings) {
sql = sql
+ " AND start < :end"
+ StringUtils.format(" AND %1$send%1$s > :start", connector.getQuoteString());
}
Query<Map<String, Object>> query = handle.createQuery(sql)
.bind("dataSource", dataSource);
if (compareIntervalEndpointsAsStrings) {
query = query.bind("start", interval.getStart().toString())
.bind("end", interval.getEnd().toString());
}
final ResultIterator<PendingSegmentRecord> pendingSegmentIterator =
query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper))
.iterator();
final ImmutableList.Builder<PendingSegmentRecord> pendingSegments = ImmutableList.builder();
while (pendingSegmentIterator.hasNext()) {
final PendingSegmentRecord pendingSegment = pendingSegmentIterator.next();
if (compareIntervalEndpointsAsStrings || pendingSegment.getId().getInterval().overlaps(interval)) {
pendingSegments.add(pendingSegment);
}
}
pendingSegmentIterator.close();
return pendingSegments.build();
}
List<PendingSegmentRecord> getPendingSegmentsForTaskAllocatorIdWithHandle(
final Handle handle,
final String dataSource,
final String taskAllocatorId
)
{
String sql = "SELECT payload, sequence_name, sequence_prev_id, task_allocator_id, upgraded_from_segment_id"
+ " FROM " + dbTables.getPendingSegmentsTable()
+ " WHERE dataSource = :dataSource AND task_allocator_id = :task_allocator_id";
Query<Map<String, Object>> query = handle.createQuery(sql)
.bind("dataSource", dataSource)
.bind("task_allocator_id", taskAllocatorId);
final ResultIterator<PendingSegmentRecord> pendingSegmentRecords =
query.map((index, r, ctx) -> PendingSegmentRecord.fromResultSet(r, jsonMapper))
.iterator();
final List<PendingSegmentRecord> pendingSegments = new ArrayList<>();
while (pendingSegmentRecords.hasNext()) {
pendingSegments.add(pendingSegmentRecords.next());
}
pendingSegmentRecords.close();
return pendingSegments;
}
private SegmentTimeline getTimelineForIntervalsWithHandle(
final Handle handle,
final String dataSource,
final List<Interval> intervals
) throws IOException
{
try (final CloseableIterator<DataSegment> iterator =
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
.retrieveUsedSegments(dataSource, intervals)) {
return SegmentTimeline.forSegments(iterator);
}
}
private Collection<DataSegment> retrieveAllUsedSegmentsForIntervalsWithHandle(
final Handle handle,
final String dataSource,
final List<Interval> intervals
) throws IOException
{
try (final CloseableIterator<DataSegment> iterator =
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
.retrieveUsedSegments(dataSource, intervals)) {
final List<DataSegment> retVal = new ArrayList<>();
iterator.forEachRemaining(retVal::add);
return retVal;
}
}
@Override
public Set<DataSegment> commitSegments(
final Set<DataSegment> segments,
@Nullable final SegmentSchemaMapping segmentSchemaMapping
) throws IOException
{
final SegmentPublishResult result =
commitSegmentsAndMetadata(
segments,
null,
null,
segmentSchemaMapping
);
// Metadata transaction cannot fail because we are not trying to do one.
if (!result.isSuccess()) {
throw new ISE("announceHistoricalSegments failed with null metadata, should not happen.");
}
return result.getSegments();
}
@Override
public SegmentPublishResult commitSegmentsAndMetadata(
final Set<DataSegment> segments,
@Nullable final DataSourceMetadata startMetadata,
@Nullable final DataSourceMetadata endMetadata,
@Nullable final SegmentSchemaMapping segmentSchemaMapping
) throws IOException
{
verifySegmentsToCommit(segments);
if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) {
throw new IllegalArgumentException("start/end metadata pair must be either null or non-null");
}
final String dataSource = segments.iterator().next().getDataSource();
// Find which segments are used (i.e. not overshadowed).
final Set<DataSegment> usedSegments = new HashSet<>();
List<TimelineObjectHolder<String, DataSegment>> segmentHolders =
SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY);
for (TimelineObjectHolder<String, DataSegment> holder : segmentHolders) {
for (PartitionChunk<DataSegment> chunk : holder.getObject()) {
usedSegments.add(chunk.getObject());
}
}
final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false);
try {
return connector.retryTransaction(
(handle, transactionStatus) -> {
// Set definitelyNotUpdated back to false upon retrying.
definitelyNotUpdated.set(false);
if (startMetadata != null) {
final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle(
handle,
dataSource,
startMetadata,
endMetadata
);
if (result.isFailed()) {
// Metadata was definitely not updated.
transactionStatus.setRollbackOnly();
definitelyNotUpdated.set(true);
if (result.canRetry()) {
throw new RetryTransactionException(result.getErrorMsg());
} else {
throw InvalidInput.exception(result.getErrorMsg());
}
}
}
final Set<DataSegment> inserted =
announceHistoricalSegmentBatch(
handle,
segments,
usedSegments,
segmentSchemaMapping
);
return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted));
},
3,
getSqlMetadataMaxRetry()
);
}
catch (CallbackFailedException e) {
if (definitelyNotUpdated.get()) {
return SegmentPublishResult.fail(e.getMessage());
} else {
// Must throw exception if we are not sure if we updated or not.
throw e;
}
}
}
@Override
public SegmentPublishResult commitReplaceSegments(
final Set<DataSegment> replaceSegments,
final Set<ReplaceTaskLock> locksHeldByReplaceTask,
@Nullable final SegmentSchemaMapping segmentSchemaMapping
)
{
verifySegmentsToCommit(replaceSegments);
try {
return connector.retryTransaction(
(handle, transactionStatus) -> {
final Set<DataSegment> segmentsToInsert = new HashSet<>(replaceSegments);
Set<DataSegmentPlus> upgradedSegments =
createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask);
Map<SegmentId, SegmentMetadata> upgradeSegmentMetadata = new HashMap<>();
for (DataSegmentPlus dataSegmentPlus : upgradedSegments) {
segmentsToInsert.add(dataSegmentPlus.getDataSegment());
if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) {
upgradeSegmentMetadata.put(
dataSegmentPlus.getDataSegment().getId(),
new SegmentMetadata(dataSegmentPlus.getNumRows(), dataSegmentPlus.getSchemaFingerprint())
);
}
}
SegmentPublishResult result = SegmentPublishResult.ok(
insertSegments(
handle,
segmentsToInsert,
segmentSchemaMapping,
upgradeSegmentMetadata,
Collections.emptyMap()
),
upgradePendingSegmentsOverlappingWith(segmentsToInsert)
);
return result;
},
3,
getSqlMetadataMaxRetry()
);
}
catch (CallbackFailedException e) {
return SegmentPublishResult.fail(e.getMessage());
}
}
@Override
public SegmentPublishResult commitAppendSegments(
final Set<DataSegment> appendSegments,
final Map<DataSegment, ReplaceTaskLock> appendSegmentToReplaceLock,
final String taskAllocatorId,
@Nullable final SegmentSchemaMapping segmentSchemaMapping
)
{
return commitAppendSegmentsAndMetadataInTransaction(
appendSegments,
appendSegmentToReplaceLock,
null,
null,
taskAllocatorId,
segmentSchemaMapping
);
}
@Override
public SegmentPublishResult commitAppendSegmentsAndMetadata(
Set<DataSegment> appendSegments,
Map<DataSegment, ReplaceTaskLock> appendSegmentToReplaceLock,
DataSourceMetadata startMetadata,
DataSourceMetadata endMetadata,
String taskAllocatorId,
@Nullable SegmentSchemaMapping segmentSchemaMapping
)
{
return commitAppendSegmentsAndMetadataInTransaction(
appendSegments,
appendSegmentToReplaceLock,
startMetadata,
endMetadata,
taskAllocatorId,
segmentSchemaMapping
);
}
@Override
public SegmentPublishResult commitMetadataOnly(
String dataSource,
DataSourceMetadata startMetadata,
DataSourceMetadata endMetadata
)
{
if (dataSource == null) {
throw new IllegalArgumentException("datasource name cannot be null");
}
if (startMetadata == null) {
throw new IllegalArgumentException("start metadata cannot be null");
}
if (endMetadata == null) {
throw new IllegalArgumentException("end metadata cannot be null");
}
final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false);
try {
return connector.retryTransaction(
new TransactionCallback<SegmentPublishResult>()
{
@Override
public SegmentPublishResult inTransaction(
final Handle handle,
final TransactionStatus transactionStatus
) throws Exception
{
// Set definitelyNotUpdated back to false upon retrying.
definitelyNotUpdated.set(false);
final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle(
handle,
dataSource,
startMetadata,
endMetadata
);
if (result.isFailed()) {
// Metadata was definitely not updated.
transactionStatus.setRollbackOnly();
definitelyNotUpdated.set(true);
if (result.canRetry()) {
throw new RetryTransactionException(result.getErrorMsg());
} else {
throw new RuntimeException(result.getErrorMsg());
}
}
return SegmentPublishResult.ok(ImmutableSet.of());
}
},
3,
getSqlMetadataMaxRetry()
);
}
catch (CallbackFailedException e) {
if (definitelyNotUpdated.get()) {
return SegmentPublishResult.fail(e.getMessage());
} else {
// Must throw exception if we are not sure if we updated or not.
throw e;
}
}
}
@VisibleForTesting
public int getSqlMetadataMaxRetry()
{
return SQLMetadataConnector.DEFAULT_MAX_TRIES;
}
@Override
public Map<SegmentCreateRequest, SegmentIdWithShardSpec> allocatePendingSegments(
String dataSource,
Interval allocateInterval,
boolean skipSegmentLineageCheck,
List<SegmentCreateRequest> requests
)
{
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(allocateInterval, "interval");
final Interval interval = allocateInterval.withChronology(ISOChronology.getInstanceUTC());
return connector.retryWithHandle(
handle -> allocatePendingSegments(handle, dataSource, interval, skipSegmentLineageCheck, requests)
);
}
@Override
public SegmentIdWithShardSpec allocatePendingSegment(
final String dataSource,
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval,
final PartialShardSpec partialShardSpec,
final String maxVersion,
final boolean skipSegmentLineageCheck,
String taskAllocatorId
)
{
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(sequenceName, "sequenceName");
Preconditions.checkNotNull(interval, "interval");
Preconditions.checkNotNull(maxVersion, "version");
final Interval allocateInterval = interval.withChronology(ISOChronology.getInstanceUTC());
return connector.retryWithHandle(
handle -> {
// Get the time chunk and associated data segments for the given interval, if any
final List<TimelineObjectHolder<String, DataSegment>> existingChunks =
getTimelineForIntervalsWithHandle(handle, dataSource, ImmutableList.of(interval))
.lookup(interval);
if (existingChunks.size() > 1) {
// Not possible to expand more than one chunk with a single segment.
log.warn(
"Cannot allocate new segment for dataSource[%s], interval[%s] as it already has [%,d] versions.",
dataSource, interval, existingChunks.size()
);
return null;
}
if (skipSegmentLineageCheck) {
return allocatePendingSegment(
handle,
dataSource,
sequenceName,
allocateInterval,
partialShardSpec,
maxVersion,
existingChunks,
taskAllocatorId
);
} else {
return allocatePendingSegmentWithSegmentLineageCheck(
handle,
dataSource,
sequenceName,
previousSegmentId,
allocateInterval,
partialShardSpec,
maxVersion,
existingChunks,
taskAllocatorId
);
}
}
);
}
@Override
public List<PendingSegmentRecord> upgradePendingSegmentsOverlappingWith(
Set<DataSegment> replaceSegments
)
{
if (replaceSegments.isEmpty()) {
return Collections.emptyList();
}
// Any replace interval has exactly one version of segments
final Map<Interval, DataSegment> replaceIntervalToMaxId = new HashMap<>();
for (DataSegment segment : replaceSegments) {
DataSegment committedMaxId = replaceIntervalToMaxId.get(segment.getInterval());
if (committedMaxId == null
|| committedMaxId.getShardSpec().getPartitionNum() < segment.getShardSpec().getPartitionNum()) {
replaceIntervalToMaxId.put(segment.getInterval(), segment);
}
}
final String datasource = replaceSegments.iterator().next().getDataSource();
return connector.retryWithHandle(
handle -> upgradePendingSegments(handle, datasource, replaceIntervalToMaxId)
);
}
/**
* Creates and inserts new IDs for the pending segments contained in each replace
* interval. The newly created pending segment IDs
* <ul>
* <li>Have the same interval and version as that of an overlapping segment
* committed by the REPLACE task.</li>
* <li>Cannot be committed but are only used to serve realtime queries against
* those versions.</li>
* </ul>
*
* @return Inserted pending segment records
*/
private List<PendingSegmentRecord> upgradePendingSegments(
Handle handle,
String datasource,
Map<Interval, DataSegment> replaceIntervalToMaxId
) throws JsonProcessingException
{
final List<PendingSegmentRecord> upgradedPendingSegments = new ArrayList<>();
for (Map.Entry<Interval, DataSegment> entry : replaceIntervalToMaxId.entrySet()) {
final Interval replaceInterval = entry.getKey();
final DataSegment maxSegmentId = entry.getValue();
final String replaceVersion = maxSegmentId.getVersion();
final int numCorePartitions = maxSegmentId.getShardSpec().getNumCorePartitions();
int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum();
final List<PendingSegmentRecord> overlappingPendingSegments
= getPendingSegmentsForIntervalWithHandle(handle, datasource, replaceInterval);
for (PendingSegmentRecord overlappingPendingSegment : overlappingPendingSegments) {
final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getId();
if (shouldUpgradePendingSegment(overlappingPendingSegment, replaceInterval, replaceVersion)) {
// Ensure unique sequence_name_prev_id_sha1 by setting
// sequence_prev_id -> pendingSegmentId
// sequence_name -> prefix + replaceVersion
SegmentIdWithShardSpec newId = new SegmentIdWithShardSpec(
datasource,
replaceInterval,
replaceVersion,
new NumberedShardSpec(++currentPartitionNumber, numCorePartitions)
);
upgradedPendingSegments.add(
new PendingSegmentRecord(
newId,
UPGRADED_PENDING_SEGMENT_PREFIX + replaceVersion,
pendingSegmentId.toString(),
pendingSegmentId.toString(),
overlappingPendingSegment.getTaskAllocatorId()
)
);
}
}
}
// Do not skip lineage check so that the sequence_name_prev_id_sha1
// includes hash of both sequence_name and prev_segment_id
int numInsertedPendingSegments = insertPendingSegmentsIntoMetastore(
handle,
upgradedPendingSegments,
datasource,
false
);
log.info(
"Inserted total [%d] new versions for [%d] pending segments.",
numInsertedPendingSegments, upgradedPendingSegments.size()
);
return upgradedPendingSegments;
}
private boolean shouldUpgradePendingSegment(
PendingSegmentRecord pendingSegment,
Interval replaceInterval,
String replaceVersion
)
{
if (pendingSegment.getTaskAllocatorId() == null) {
return false;
} else if (pendingSegment.getId().getVersion().compareTo(replaceVersion) >= 0) {
return false;
} else if (!replaceInterval.contains(pendingSegment.getId().getInterval())) {
return false;
} else {
// Do not upgrade already upgraded pending segment
return pendingSegment.getSequenceName() == null
|| !pendingSegment.getSequenceName().startsWith(UPGRADED_PENDING_SEGMENT_PREFIX);
}
}
@Nullable
private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck(
final Handle handle,
final String dataSource,
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval,
final PartialShardSpec partialShardSpec,
final String maxVersion,
final List<TimelineObjectHolder<String, DataSegment>> existingChunks,
final String taskAllocatorId
) throws IOException
{
final String previousSegmentIdNotNull = previousSegmentId == null ? "" : previousSegmentId;
final String sql = StringUtils.format(
"SELECT payload FROM %s WHERE "
+ "dataSource = :dataSource AND "
+ "sequence_name = :sequence_name AND "
+ "sequence_prev_id = :sequence_prev_id",
dbTables.getPendingSegmentsTable()
);
final Query<Map<String, Object>> query
= handle.createQuery(sql)
.bind("dataSource", dataSource)
.bind("sequence_name", sequenceName)
.bind("sequence_prev_id", previousSegmentIdNotNull);
final String usedSegmentVersion = existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion();
final CheckExistingSegmentIdResult result = findExistingPendingSegment(
query,
interval,
sequenceName,
previousSegmentIdNotNull,
usedSegmentVersion
);
if (result.found) {
// The found existing segment identifier can be null if its interval doesn't match with the given interval
return result.segmentIdentifier;
}
final SegmentIdWithShardSpec newIdentifier = createNewSegment(
handle,
dataSource,
interval,
partialShardSpec,
maxVersion,
existingChunks
);
if (newIdentifier == null) {
return null;
}
// SELECT -> INSERT can fail due to races; callers must be prepared to retry.
// Avoiding ON DUPLICATE KEY since it's not portable.
// Avoiding try/catch since it may cause inadvertent transaction-splitting.
// UNIQUE key for the row, ensuring sequences do not fork in two directions.
// Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines
// have difficulty with large unique keys (see https://github.com/apache/druid/issues/2319)
final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode(
Hashing.sha1()
.newHasher()
.putBytes(StringUtils.toUtf8(sequenceName))
.putByte((byte) 0xff)
.putBytes(StringUtils.toUtf8(previousSegmentIdNotNull))
.putByte((byte) 0xff)
.putBytes(StringUtils.toUtf8(newIdentifier.getVersion()))
.hash()
.asBytes()
);
insertPendingSegmentIntoMetastore(
handle,
newIdentifier,
dataSource,
interval,
previousSegmentIdNotNull,
sequenceName,
sequenceNamePrevIdSha1,
taskAllocatorId
);
return newIdentifier;
}
private Map<SegmentCreateRequest, SegmentIdWithShardSpec> allocatePendingSegments(
final Handle handle,
final String dataSource,
final Interval interval,
final boolean skipSegmentLineageCheck,
final List<SegmentCreateRequest> requests
) throws IOException
{
// Get the time chunk and associated data segments for the given interval, if any
final List<TimelineObjectHolder<String, DataSegment>> existingChunks =
getTimelineForIntervalsWithHandle(handle, dataSource, Collections.singletonList(interval))
.lookup(interval);
if (existingChunks.size() > 1) {
log.warn(
"Cannot allocate new segments for dataSource[%s], interval[%s] as interval already has [%,d] chunks.",
dataSource, interval, existingChunks.size()
);
return Collections.emptyMap();
}
final String existingVersion = existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion();
final Map<SegmentCreateRequest, CheckExistingSegmentIdResult> existingSegmentIds;
if (skipSegmentLineageCheck) {
existingSegmentIds =
getExistingSegmentIdsSkipLineageCheck(handle, dataSource, interval, existingVersion, requests);
} else {
existingSegmentIds =
getExistingSegmentIdsWithLineageCheck(handle, dataSource, interval, existingVersion, requests);
}
// For every request see if a segment id already exists
final Map<SegmentCreateRequest, SegmentIdWithShardSpec> allocatedSegmentIds = new HashMap<>();
final List<SegmentCreateRequest> requestsForNewSegments = new ArrayList<>();
for (SegmentCreateRequest request : requests) {
CheckExistingSegmentIdResult existingSegmentId = existingSegmentIds.get(request);
if (existingSegmentId == null || !existingSegmentId.found) {
requestsForNewSegments.add(request);
} else if (existingSegmentId.segmentIdentifier != null) {
log.info("Found valid existing segment [%s] for request.", existingSegmentId.segmentIdentifier);
allocatedSegmentIds.put(request, existingSegmentId.segmentIdentifier);
} else {
log.info("Found clashing existing segment [%s] for request.", existingSegmentId);
}
}
// For each of the remaining requests, create a new segment
final Map<SegmentCreateRequest, PendingSegmentRecord> createdSegments = createNewSegments(
handle,
dataSource,
interval,
skipSegmentLineageCheck,
existingChunks,
requestsForNewSegments
);
// SELECT -> INSERT can fail due to races; callers must be prepared to retry.
// Avoiding ON DUPLICATE KEY since it's not portable.
// Avoiding try/catch since it may cause inadvertent transaction-splitting.
// UNIQUE key for the row, ensuring we don't have more than one segment per sequence per interval.
// Using a single column instead of (sequence_name, sequence_prev_id) as some MySQL storage engines