6
6
package sql
7
7
8
8
import (
9
+ "bytes"
9
10
"context"
10
11
"math/rand"
12
+ "sort"
11
13
"time"
12
14
13
15
"github.com/cockroachdb/cockroach/pkg/keys"
@@ -19,31 +21,177 @@ import (
19
21
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
20
22
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
21
23
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
24
+ "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside"
22
25
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
23
26
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
27
+ "github.com/cockroachdb/cockroach/pkg/sql/stats"
24
28
"github.com/cockroachdb/cockroach/pkg/util/encoding"
25
29
"github.com/cockroachdb/cockroach/pkg/util/log"
26
30
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
31
+ "github.com/cockroachdb/errors"
27
32
)
28
33
29
34
type indexSplitAndScatter struct {
30
- db * kv.DB
31
- codec keys.SQLCodec
32
- sv * settings.Values
33
- rangeIter rangedesc.IteratorFactory
34
- nodeDescs kvclient.NodeDescStore
35
+ db * kv.DB
36
+ codec keys.SQLCodec
37
+ sv * settings.Values
38
+ rangeIter rangedesc.IteratorFactory
39
+ nodeDescs kvclient.NodeDescStore
40
+ statsCache * stats.TableStatisticsCache
41
+ testingKnobs * ExecutorTestingKnobs
35
42
}
36
43
44
+ var SplitAndScatterWithStats = settings .RegisterBoolSetting (
45
+ settings .ApplicationLevel ,
46
+ "schemachanger.backfiller.split_with_stats.enabled" ,
47
+ "when enabled the index backfiller will generate split and " +
48
+ "scatter points based table statistics" ,
49
+ false ,
50
+ )
51
+
37
52
// NewIndexSplitAndScatter creates a new scexec.IndexSpanSplitter implementation.
38
53
func NewIndexSplitAndScatter (execCfg * ExecutorConfig ) scexec.IndexSpanSplitter {
39
-
40
54
return & indexSplitAndScatter {
41
- db : execCfg .DB ,
42
- codec : execCfg .Codec ,
43
- sv : & execCfg .Settings .SV ,
44
- rangeIter : execCfg .RangeDescIteratorFactory ,
45
- nodeDescs : execCfg .NodeDescs ,
55
+ db : execCfg .DB ,
56
+ codec : execCfg .Codec ,
57
+ sv : & execCfg .Settings .SV ,
58
+ rangeIter : execCfg .RangeDescIteratorFactory ,
59
+ nodeDescs : execCfg .NodeDescs ,
60
+ statsCache : execCfg .TableStatsCache ,
61
+ testingKnobs : & execCfg .TestingKnobs ,
62
+ }
63
+ }
64
+
65
+ func (is * indexSplitAndScatter ) getSplitPointsWithStats (
66
+ ctx context.Context , table catalog.TableDescriptor , indexToBackfill catalog.Index , nSplits int ,
67
+ ) ([][]byte , error ) {
68
+ // Split and scatter with statistics is disabled.
69
+ if ! SplitAndScatterWithStats .Get (is .sv ) {
70
+ return nil , nil
71
+ }
72
+ // Fetch the current statistics for this table.
73
+ tableStats , err := is .statsCache .GetTableStats (ctx , table , nil )
74
+ if err != nil {
75
+ return nil , err
76
+ }
77
+ // Nothing can be done since no stats exist.
78
+ if len (tableStats ) == 0 {
79
+ return nil , errors .New ("no stats exist for this table" )
80
+ }
81
+ // Gather the latest stats for each column.
82
+ keyCols := indexToBackfill .CollectKeyColumnIDs ()
83
+ statsForColumns := make (map [descpb.ColumnID ]* stats.TableStatistic )
84
+ keyCols .ForEach (func (col descpb.ColumnID ) {
85
+ for _ , stat := range tableStats {
86
+ // Skip stats that:
87
+ // 1) Do not contain this column.
88
+ // 2) Consist of multiple columns.
89
+ // 3) Have no histogram information.
90
+ if stat .Histogram == nil || len (stat .ColumnIDs ) != 1 || stat .ColumnIDs [0 ] != col {
91
+ continue
92
+ }
93
+ statsForColumns [col ] = stat
94
+ break
95
+ }
96
+ })
97
+ rowsPerRange := tableStats [0 ].RowCount / uint64 (nSplits )
98
+ // Helper function that will append split points, and if necessary, downsample
99
+ // them if they get too big.
100
+ var splitPoints [][]byte
101
+ appendAndShrinkSplitPoint := func (existing [][]byte , add []byte ) [][]byte {
102
+ maxSplitPoints := nSplits * 2
103
+ if len (existing ) < maxSplitPoints {
104
+ return append (existing , add )
105
+ }
106
+ // Otherwise, we can sample these split points.
107
+ sort .Slice (existing , func (i , j int ) bool {
108
+ return bytes .Compare (existing [i ], existing [j ]) < 0
109
+ })
110
+ // Next get this down to capacity again by taking a uniform sample of the
111
+ // existing split points.
112
+ newSplitPoints := make ([][]byte , 0 , nSplits + 1 )
113
+ step := float64 (len (existing )) / float64 (nSplits )
114
+ for i := 0 ; i < nSplits ; i ++ {
115
+ newSplitPoints = append (newSplitPoints , existing [int (float64 (i )* step )])
116
+ }
117
+ newSplitPoints = append (newSplitPoints , add )
118
+ return newSplitPoints
46
119
}
120
+ // The following code generates split points for an index by iterating through
121
+ // each column of the index. For each column, it uses histogram statistics to
122
+ // identify points where the data can be divided into chunks of a target size
123
+ // (`rowsPerRange`).
124
+ //
125
+ // For the first column, it creates initial split points. For each subsequent
126
+ // column, it expands on the previously generated split points. It does this by
127
+ // appending the new column's split values to each of the existing split points from
128
+ // prior columns. This causes us to iterate combinatorially over all possible split points,
129
+ // so the `appendAndShrinkSplitPoint` function is used to downsample and keep the total number
130
+ // of points controlled.
131
+
132
+ // Note: Sadly, only the primary key or columns in indexes will have
133
+ // detailed information that we can use. All other columns will have
134
+ // limited splits.
135
+ for colIdx := 0 ; colIdx < indexToBackfill .NumKeyColumns (); colIdx ++ {
136
+ lastSplitPoints := append ([][]byte {}, splitPoints ... )
137
+ splitPoints = splitPoints [:0 ]
138
+ keyColID := indexToBackfill .GetKeyColumnID (colIdx )
139
+ // Look up the stats and skip if they are missing.
140
+ stat , ok := statsForColumns [keyColID ]
141
+ if ! ok {
142
+ break
143
+ }
144
+ numInBucket := uint64 (0 )
145
+ for bucketIdx , bucket := range stat .Histogram {
146
+ numInBucket += uint64 (bucket .NumRange ) + uint64 (bucket .NumEq )
147
+ // If we have hit the target rows, then emit a split point. Or
148
+ // if we are on the last bucket, we should always emit one.
149
+ if numInBucket >= rowsPerRange || bucketIdx == len (stat .Histogram )- 1 {
150
+ var prevKeys [][]byte
151
+ // For the first column, we are going to start fresh with the base index prefix.
152
+ if colIdx == 0 {
153
+ prevKeys = [][]byte {is .codec .IndexPrefix (uint32 (table .GetID ()), uint32 (indexToBackfill .GetID ()))}
154
+ } else {
155
+ // For later columns we are going to start with the previous sets of splits.
156
+ prevKeys = lastSplitPoints
157
+ }
158
+ // We don't know where later columns fall, so we will encode these
159
+ // against all the previous split points (sadly, this will have an exponential
160
+ // cost). Our limit on the number of split points will resample these if they
161
+ // become excessive.
162
+ for _ , prevKey := range prevKeys {
163
+ // Copy the base value before appending the next part of the key.
164
+ if colIdx > 0 {
165
+ tempKey := make ([]byte , len (prevKey ), cap (prevKey ))
166
+ copy (tempKey , prevKey )
167
+ prevKey = tempKey
168
+ }
169
+ newSplit , err := keyside .Encode (prevKey , bucket .UpperBound , encoding .Direction (indexToBackfill .GetKeyColumnDirection (colIdx )+ 1 ))
170
+ if err != nil {
171
+ return nil , err
172
+ }
173
+ splitPoints = appendAndShrinkSplitPoint (splitPoints , newSplit )
174
+ }
175
+ numInBucket = 0
176
+ continue
177
+ }
178
+ }
179
+ // Stop once enough partitions have been created. Or if no partitions exist,
180
+ // then there is insufficient data for an educated guess. As we process later
181
+ // columns, we end up creating all possible permutations of the previous split
182
+ // points we selected, which means the statistical likelihood of a valid split
183
+ // point getting selected only gets lower.
184
+ if len (splitPoints ) >= nSplits || len (splitPoints ) == 0 {
185
+ break
186
+ }
187
+ }
188
+ // Always emit a split point at the start of the index span if
189
+ // we generated any split points above
190
+ if len (splitPoints ) > 0 {
191
+ splitPoints = append (splitPoints , is .codec .IndexPrefix (uint32 (table .GetID ()), uint32 (indexToBackfill .GetID ())))
192
+ log .Infof (ctx , "generated %d split points from statistics for tableId=%d index=%d" , len (splitPoints ), table .GetID (), indexToBackfill .GetID ())
193
+ }
194
+ return splitPoints , nil
47
195
}
48
196
49
197
// MaybeSplitIndexSpans implements the scexec.IndexSpanSplitter interface.
@@ -121,6 +269,13 @@ func (is *indexSplitAndScatter) MaybeSplitIndexSpans(
121
269
splitPoints = append (splitPoints , newStartKey )
122
270
}
123
271
272
+ if len (splitPoints ) == 0 {
273
+ splitPoints , err = is .getSplitPointsWithStats (ctx , table , indexToBackfill , nSplits )
274
+ if err != nil {
275
+ log .Warningf (ctx , "unable to get split points for stats for tableID=%d index=%d due to %v" , tableID , indexToBackfill .GetID (), err )
276
+ }
277
+ }
278
+
124
279
if len (splitPoints ) == 0 {
125
280
// If we can't sample splits from another index, just add one split.
126
281
log .Infof (ctx , "making a single split point in tableId=%d index=%d" , tableID , indexToBackfill .GetID ())
@@ -130,6 +285,10 @@ func (is *indexSplitAndScatter) MaybeSplitIndexSpans(
130
285
if err != nil {
131
286
return err
132
287
}
288
+ // Execute the testing knob before adding a split.
289
+ if is .testingKnobs .BeforeIndexSplitAndScatter != nil {
290
+ is .testingKnobs .BeforeIndexSplitAndScatter ([][]byte {splitKey })
291
+ }
133
292
// We split without scattering here because there is only one split point,
134
293
// so scattering wouldn't spread that much load.
135
294
return is .db .AdminSplit (ctx , splitKey , expirationTime )
@@ -143,6 +302,10 @@ func (is *indexSplitAndScatter) MaybeSplitIndexSpans(
143
302
if step < 1 {
144
303
step = 1
145
304
}
305
+ // Execute the testing knob before the split and scatter.
306
+ if is .testingKnobs .BeforeIndexSplitAndScatter != nil {
307
+ is .testingKnobs .BeforeIndexSplitAndScatter (splitPoints )
308
+ }
146
309
for i := 0 ; i < nSplits ; i ++ {
147
310
// Evenly space out the ranges that we select from the ranges that are
148
311
// returned.
0 commit comments