|
| 1 | +// Copyright 2025 The Cockroach Authors. |
| 2 | +// |
| 3 | +// Use of this software is governed by the CockroachDB Software License |
| 4 | +// included in the /LICENSE file. |
| 5 | + |
| 6 | +package bulkingest |
| 7 | + |
| 8 | +import ( |
| 9 | + "github.com/cockroachdb/cockroach/pkg/keys" |
| 10 | + "github.com/cockroachdb/cockroach/pkg/roachpb" |
| 11 | + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" |
| 12 | + "github.com/cockroachdb/errors" |
| 13 | +) |
| 14 | + |
| 15 | +// pickSplits picks which spans to split on based on the input SSTs. The splits are chosen |
| 16 | +// so that each SST is contained within exactly one output span. The output spans are contiguous |
| 17 | +// and non-overlapping. Splits are chosen based on the start key of the following SST. |
| 18 | +// |
| 19 | +// Every SST must be contained within exactly one of the input spans. |
| 20 | +// |
| 21 | +// The input spans must be ordered by start key and be non-overlapping. The |
| 22 | +// input SSTs must be contained within the input spans, must be ordered by start |
| 23 | +// key, and must be non-overlapping. |
| 24 | +func pickSplits( |
| 25 | + spans []roachpb.Span, ssts []execinfrapb.BulkMergeSpec_SST, |
| 26 | +) ([]roachpb.Span, error) { |
| 27 | + if len(ssts) == 0 { |
| 28 | + return spans, nil |
| 29 | + } |
| 30 | + if len(spans) == 0 { |
| 31 | + return nil, errors.New("no spans provided") |
| 32 | + } |
| 33 | + |
| 34 | + // Validate spans are ordered and non-overlapping |
| 35 | + for i := 1; i < len(spans); i++ { |
| 36 | + if !less(spans[i-1].Key, spans[i].Key) { |
| 37 | + return nil, errors.Newf("spans not ordered: %s >= %s", spans[i-1].Key, spans[i].Key) |
| 38 | + } |
| 39 | + if overlaps(spans[i-1], spans[i]) { |
| 40 | + return nil, errors.Newf("spans are overlapping: %s overlaps with %s", spans[i-1].EndKey, spans[i].Key) |
| 41 | + } |
| 42 | + } |
| 43 | + |
| 44 | + // Validate SSTs are ordered and non-overlapping. |
| 45 | + for i := 1; i < len(ssts); i++ { |
| 46 | + prev, curr := roachpb.Key(ssts[i-1].StartKey), roachpb.Key(ssts[i].StartKey) |
| 47 | + if !less(prev, curr) { |
| 48 | + return nil, errors.Newf("SSTs not in order: %s >= %s", prev, curr) |
| 49 | + } |
| 50 | + if overlaps(spanFromSST(ssts[i-1]), spanFromSST(ssts[i])) { |
| 51 | + return nil, errors.Newf("overlapping SSTs: %s overlaps with %s", ssts[i-1].EndKey, ssts[i].StartKey) |
| 52 | + } |
| 53 | + } |
| 54 | + |
| 55 | + result := make([]roachpb.Span, 0, len(ssts)) |
| 56 | + sstIdx := 0 |
| 57 | + |
| 58 | + for _, span := range spans { |
| 59 | + spanSSTStartIdx := sstIdx |
| 60 | + for ; sstIdx < len(ssts); sstIdx++ { |
| 61 | + sstStart := roachpb.Key(ssts[sstIdx].StartKey) |
| 62 | + if !less(sstStart, span.EndKey) { |
| 63 | + break |
| 64 | + } |
| 65 | + |
| 66 | + sstEnd := roachpb.Key(ssts[sstIdx].EndKey) |
| 67 | + if !less(sstEnd, span.EndKey) && !sstEnd.Equal(span.EndKey) { |
| 68 | + return nil, errors.Newf("SST ending at %s extends beyond containing span ending at %s", |
| 69 | + sstEnd, span.EndKey) |
| 70 | + } |
| 71 | + if less(sstStart, span.Key) { |
| 72 | + return nil, errors.Newf("SST starting at %s begins before containing span starting at %s", |
| 73 | + sstStart, span.Key) |
| 74 | + } |
| 75 | + } |
| 76 | + |
| 77 | + if spanSSTStartIdx == sstIdx { |
| 78 | + result = append(result, span) |
| 79 | + continue |
| 80 | + } |
| 81 | + |
| 82 | + spanSplits, err := pickSplitsForSpan(span, ssts[spanSSTStartIdx:sstIdx]) |
| 83 | + if err != nil { |
| 84 | + return nil, err |
| 85 | + } |
| 86 | + result = append(result, spanSplits...) |
| 87 | + } |
| 88 | + |
| 89 | + // Ensure all SSTs were contained within the input spans. |
| 90 | + if sstIdx < len(ssts) { |
| 91 | + return nil, errors.Newf("SST starting at %s not contained in any span", ssts[sstIdx].StartKey) |
| 92 | + } |
| 93 | + |
| 94 | + return result, nil |
| 95 | +} |
| 96 | + |
| 97 | +// pickSplitsForSpan splits a single span based on the SSTs that overlap with |
| 98 | +// it. The output spans cover the entire input span, are non-overlapping, and |
| 99 | +// are contiguous. Each output span is assigned exactly one SST. |
| 100 | +// |
| 101 | +// This function validates that SST boundaries are already at safe split points |
| 102 | +// (i.e., row boundaries, not column family boundaries). |
| 103 | +func pickSplitsForSpan( |
| 104 | + span roachpb.Span, ssts []execinfrapb.BulkMergeSpec_SST, |
| 105 | +) ([]roachpb.Span, error) { |
| 106 | + if len(ssts) == 0 { |
| 107 | + return []roachpb.Span{span}, nil |
| 108 | + } |
| 109 | + |
| 110 | + result := make([]roachpb.Span, 0, len(ssts)) |
| 111 | + |
| 112 | + spanStart := span.Key |
| 113 | + |
| 114 | + for i := 1; i < len(ssts); i++ { |
| 115 | + splitPoint := roachpb.Key(ssts[i].StartKey) |
| 116 | + |
| 117 | + // Validate that the split point is already at a safe split point. |
| 118 | + safeSplitPoint, err := keys.EnsureSafeSplitKey(splitPoint) |
| 119 | + if err != nil { |
| 120 | + return nil, errors.NewAssertionErrorWithWrappedErrf(err, "SST %d has unsafe start key %s", i, splitPoint) |
| 121 | + } |
| 122 | + if !safeSplitPoint.Equal(splitPoint) { |
| 123 | + return nil, errors.AssertionFailedf( |
| 124 | + "SST %d start key %s is not at a safe split point (safe point would be %s); "+ |
| 125 | + "SST writer should have ensured safe boundaries", |
| 126 | + i, splitPoint, safeSplitPoint) |
| 127 | + } |
| 128 | + |
| 129 | + result = append(result, roachpb.Span{ |
| 130 | + Key: spanStart, |
| 131 | + EndKey: splitPoint, |
| 132 | + }) |
| 133 | + spanStart = splitPoint |
| 134 | + } |
| 135 | + |
| 136 | + // Create the final span covering from the last split point to the end |
| 137 | + result = append(result, roachpb.Span{ |
| 138 | + Key: spanStart, |
| 139 | + EndKey: span.EndKey, |
| 140 | + }) |
| 141 | + |
| 142 | + return result, nil |
| 143 | +} |
| 144 | + |
| 145 | +// less returns true if a is less than b |
| 146 | +func less(a, b roachpb.Key) bool { |
| 147 | + return a.Compare(b) < 0 |
| 148 | +} |
| 149 | + |
| 150 | +// overlaps returns true if span a overlaps with span b |
| 151 | +func overlaps(a, b roachpb.Span) bool { |
| 152 | + // Two spans overlap if one's end key is greater than the other's start key |
| 153 | + return a.EndKey.Compare(b.Key) > 0 && b.EndKey.Compare(a.Key) > 0 |
| 154 | +} |
| 155 | + |
| 156 | +// spanFromSST returns the span that matches the SST's start and end keys. |
| 157 | +func spanFromSST(sst execinfrapb.BulkMergeSpec_SST) roachpb.Span { |
| 158 | + return roachpb.Span{ |
| 159 | + Key: roachpb.Key(sst.StartKey), |
| 160 | + EndKey: roachpb.Key(sst.EndKey), |
| 161 | + } |
| 162 | +} |
0 commit comments