github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/backend/sql2kv.go (about) 1 // Copyright 2019 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package backend 15 16 import ( 17 "math/rand" 18 "sort" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/parser/model" 22 "github.com/pingcap/parser/mysql" 23 "github.com/pingcap/tidb/expression" 24 "github.com/pingcap/tidb/meta/autoid" 25 "github.com/pingcap/tidb/sessionctx/variable" 26 "github.com/pingcap/tidb/table" 27 "github.com/pingcap/tidb/table/tables" 28 "github.com/pingcap/tidb/tablecodec" 29 "github.com/pingcap/tidb/types" 30 "github.com/pingcap/tidb/util/chunk" 31 "go.uber.org/zap" 32 "go.uber.org/zap/zapcore" 33 34 // Import tidb/planner/core to initialize expression.RewriteAstExpr 35 _ "github.com/pingcap/tidb/planner/core" 36 37 "github.com/pingcap/tidb-lightning/lightning/common" 38 "github.com/pingcap/tidb-lightning/lightning/log" 39 "github.com/pingcap/tidb-lightning/lightning/metric" 40 "github.com/pingcap/tidb-lightning/lightning/verification" 41 ) 42 43 var extraHandleColumnInfo = model.NewExtraHandleColInfo() 44 45 type genCol struct { 46 index int 47 expr expression.Expression 48 } 49 50 type tableKVEncoder struct { 51 tbl table.Table 52 se *session 53 recordCache []types.Datum 54 genCols []genCol 55 // auto random bits value for this chunk 56 autoRandomHeaderBits int64 57 } 58 59 func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) { 60 metric.KvEncoderCounter.WithLabelValues("open").Inc() 61 meta := tbl.Meta() 62 cols := tbl.Cols() 63 se := newSession(options) 64 // Set CommonAddRecordCtx to session to reuse the slices and BufStore in AddRecord 65 recordCtx := tables.NewCommonAddRecordCtx(len(cols)) 66 tables.SetAddRecordCtx(se, recordCtx) 67 68 var autoRandomBits int64 69 if meta.PKIsHandle && meta.ContainsAutoRandomBits() { 70 for _, col := range cols { 71 if mysql.HasPriKeyFlag(col.Flag) { 72 incrementalBits := autoRandomIncrementBits(col, int(meta.AutoRandomBits)) 73 autoRandomBits = rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<meta.AutoRandomBits) << incrementalBits 74 break 75 } 76 } 77 } 78 79 // collect expressions for evaluating stored generated columns 80 genCols, err := collectGeneratedColumns(se, meta, cols) 81 if err != nil { 82 return nil, errors.Annotate(err, "failed to parse generated column expressions") 83 } 84 85 return &tableKVEncoder{ 86 tbl: tbl, 87 se: se, 88 genCols: genCols, 89 autoRandomHeaderBits: autoRandomBits, 90 }, nil 91 } 92 93 func autoRandomIncrementBits(col *table.Column, randomBits int) int { 94 typeBitsLength := mysql.DefaultLengthOfMysqlTypes[col.Tp] * 8 95 incrementalBits := typeBitsLength - randomBits 96 hasSignBit := !mysql.HasUnsignedFlag(col.Flag) 97 if hasSignBit { 98 incrementalBits -= 1 99 } 100 return incrementalBits 101 } 102 103 // collectGeneratedColumns collects all expressions required to evaluate the 104 // results of all stored generated columns. The returning slice is in evaluation 105 // order. 106 func collectGeneratedColumns(se *session, meta *model.TableInfo, cols []*table.Column) ([]genCol, error) { 107 maxGenColOffset := -1 108 for _, col := range cols { 109 if col.GeneratedStored && col.Offset > maxGenColOffset { 110 maxGenColOffset = col.Offset 111 } 112 } 113 114 if maxGenColOffset < 0 { 115 return nil, nil 116 } 117 118 // the expression rewriter requires a non-nil TxnCtx. 119 se.vars.TxnCtx = new(variable.TransactionContext) 120 defer func() { 121 se.vars.TxnCtx = nil 122 }() 123 124 // not using TableInfo2SchemaAndNames to avoid parsing all virtual generated columns again. 125 exprColumns := make([]*expression.Column, 0, len(cols)) 126 names := make(types.NameSlice, 0, len(cols)) 127 for i, col := range cols { 128 names = append(names, &types.FieldName{ 129 OrigTblName: meta.Name, 130 OrigColName: col.Name, 131 TblName: meta.Name, 132 ColName: col.Name, 133 }) 134 exprColumns = append(exprColumns, &expression.Column{ 135 RetType: col.FieldType.Clone(), 136 ID: col.ID, 137 UniqueID: int64(i), 138 Index: col.Offset, 139 OrigName: names[i].String(), 140 IsHidden: col.Hidden, 141 }) 142 } 143 schema := expression.NewSchema(exprColumns...) 144 145 // as long as we have a stored generated column, all columns it referred to must be evaluated as well. 146 // for simplicity we just evaluate all generated columns (virtual or not) before the last stored one. 147 var genCols []genCol 148 for i, col := range cols { 149 if col.GeneratedExpr != nil && col.Offset <= maxGenColOffset { 150 expr, err := expression.RewriteAstExpr(se, col.GeneratedExpr, schema, names) 151 if err != nil { 152 return nil, err 153 } 154 genCols = append(genCols, genCol{ 155 index: i, 156 expr: expr, 157 }) 158 } 159 } 160 161 // order the result by column offset so they match the evaluation order. 162 sort.Slice(genCols, func(i, j int) bool { 163 return cols[genCols[i].index].Offset < cols[genCols[j].index].Offset 164 }) 165 return genCols, nil 166 } 167 168 func (kvcodec *tableKVEncoder) Close() { 169 metric.KvEncoderCounter.WithLabelValues("closed").Inc() 170 } 171 172 type rowArrayMarshaler []types.Datum 173 174 var kindStr = [...]string{ 175 types.KindNull: "null", 176 types.KindInt64: "int64", 177 types.KindUint64: "uint64", 178 types.KindFloat32: "float32", 179 types.KindFloat64: "float64", 180 types.KindString: "string", 181 types.KindBytes: "bytes", 182 types.KindBinaryLiteral: "binary", 183 types.KindMysqlDecimal: "decimal", 184 types.KindMysqlDuration: "duration", 185 types.KindMysqlEnum: "enum", 186 types.KindMysqlBit: "bit", 187 types.KindMysqlSet: "set", 188 types.KindMysqlTime: "time", 189 types.KindInterface: "interface", 190 types.KindMinNotNull: "min", 191 types.KindMaxValue: "max", 192 types.KindRaw: "raw", 193 types.KindMysqlJSON: "json", 194 } 195 196 // MarshalLogArray implements the zapcore.ArrayMarshaler interface 197 func (row rowArrayMarshaler) MarshalLogArray(encoder zapcore.ArrayEncoder) error { 198 for _, datum := range row { 199 kind := datum.Kind() 200 var str string 201 var err error 202 switch kind { 203 case types.KindNull: 204 str = "NULL" 205 case types.KindMinNotNull: 206 str = "-inf" 207 case types.KindMaxValue: 208 str = "+inf" 209 default: 210 str, err = datum.ToString() 211 if err != nil { 212 return err 213 } 214 } 215 encoder.AppendObject(zapcore.ObjectMarshalerFunc(func(enc zapcore.ObjectEncoder) error { 216 enc.AddString("kind", kindStr[kind]) 217 enc.AddString("val", log.RedactString(str)) 218 return nil 219 })) 220 } 221 return nil 222 } 223 224 func logKVConvertFailed(logger log.Logger, row []types.Datum, j int, colInfo *model.ColumnInfo, err error) error { 225 var original types.Datum 226 if 0 <= j && j < len(row) { 227 original = row[j] 228 row = row[j : j+1] 229 } 230 231 logger.Error("kv convert failed", 232 zap.Array("original", rowArrayMarshaler(row)), 233 zap.Int("originalCol", j), 234 zap.String("colName", colInfo.Name.O), 235 zap.Stringer("colType", &colInfo.FieldType), 236 log.ShortError(err), 237 ) 238 239 log.L().Error("failed to covert kv value", log.ZapRedactReflect("origVal", original.GetValue()), 240 zap.Stringer("fieldType", &colInfo.FieldType), zap.String("column", colInfo.Name.O), 241 zap.Int("columnID", j+1)) 242 return errors.Annotatef( 243 err, 244 "failed to cast value as %s for column `%s` (#%d)", &colInfo.FieldType, colInfo.Name.O, j+1, 245 ) 246 } 247 248 func logEvalGenExprFailed(logger log.Logger, row []types.Datum, colInfo *model.ColumnInfo, err error) error { 249 logger.Error("kv convert failed: cannot evaluate generated column expression", 250 zap.Array("original", rowArrayMarshaler(row)), 251 zap.String("colName", colInfo.Name.O), 252 log.ShortError(err), 253 ) 254 255 return errors.Annotatef( 256 err, 257 "failed to evaluate generated column expression for column `%s`", 258 colInfo.Name.O, 259 ) 260 } 261 262 type kvPairs []common.KvPair 263 264 // MakeRowsFromKvPairs converts a KvPair slice into a Rows instance. This is 265 // mainly used for testing only. The resulting Rows instance should only be used 266 // for the importer backend. 267 func MakeRowsFromKvPairs(pairs []common.KvPair) Rows { 268 return kvPairs(pairs) 269 } 270 271 // MakeRowFromKvPairs converts a KvPair slice into a Row instance. This is 272 // mainly used for testing only. The resulting Row instance should only be used 273 // for the importer backend. 274 func MakeRowFromKvPairs(pairs []common.KvPair) Row { 275 return kvPairs(pairs) 276 } 277 278 // Encode a row of data into KV pairs. 279 // 280 // See comments in `(*TableRestore).initializeColumns` for the meaning of the 281 // `columnPermutation` parameter. 282 func (kvcodec *tableKVEncoder) Encode( 283 logger log.Logger, 284 row []types.Datum, 285 rowID int64, 286 columnPermutation []int, 287 ) (Row, error) { 288 cols := kvcodec.tbl.Cols() 289 290 var value types.Datum 291 var err error 292 var record []types.Datum 293 294 if kvcodec.recordCache != nil { 295 record = kvcodec.recordCache 296 } else { 297 record = make([]types.Datum, 0, len(cols)+1) 298 } 299 300 isAutoRandom := kvcodec.tbl.Meta().PKIsHandle && kvcodec.tbl.Meta().ContainsAutoRandomBits() 301 for i, col := range cols { 302 j := columnPermutation[i] 303 isAutoIncCol := mysql.HasAutoIncrementFlag(col.Flag) 304 isPk := mysql.HasPriKeyFlag(col.Flag) 305 if j >= 0 && j < len(row) { 306 value, err = table.CastValue(kvcodec.se, row[j], col.ToInfo(), false, false) 307 if err == nil { 308 err = col.HandleBadNull(&value, kvcodec.se.vars.StmtCtx) 309 } 310 } else if isAutoIncCol { 311 // we still need a conversion, e.g. to catch overflow with a TINYINT column. 312 value, err = table.CastValue(kvcodec.se, types.NewIntDatum(rowID), col.ToInfo(), false, false) 313 } else if isAutoRandom && isPk { 314 var val types.Datum 315 if mysql.HasUnsignedFlag(col.Flag) { 316 val = types.NewUintDatum(uint64(kvcodec.autoRandomHeaderBits | rowID)) 317 } else { 318 val = types.NewIntDatum(kvcodec.autoRandomHeaderBits | rowID) 319 } 320 value, err = table.CastValue(kvcodec.se, val, col.ToInfo(), false, false) 321 } else if col.IsGenerated() { 322 // inject some dummy value for gen col so that MutRowFromDatums below sees a real value instead of nil. 323 // if MutRowFromDatums sees a nil it won't initialize the underlying storage and cause SetDatum to panic. 324 value = types.GetMinValue(&col.FieldType) 325 } else { 326 value, err = table.GetColDefaultValue(kvcodec.se, col.ToInfo()) 327 } 328 if err != nil { 329 return nil, logKVConvertFailed(logger, row, j, col.ToInfo(), err) 330 } 331 332 record = append(record, value) 333 334 if isAutoRandom && isPk { 335 incrementalBits := autoRandomIncrementBits(col, int(kvcodec.tbl.Meta().AutoRandomBits)) 336 kvcodec.tbl.RebaseAutoID(kvcodec.se, value.GetInt64()&((1<<incrementalBits)-1), false, autoid.AutoRandomType) 337 } 338 if isAutoIncCol { 339 kvcodec.tbl.RebaseAutoID(kvcodec.se, value.GetInt64(), false, autoid.AutoIncrementType) 340 } 341 } 342 343 if common.TableHasAutoRowID(kvcodec.tbl.Meta()) { 344 j := columnPermutation[len(cols)] 345 if j >= 0 && j < len(row) { 346 value, err = table.CastValue(kvcodec.se, row[j], extraHandleColumnInfo, false, false) 347 } else { 348 value, err = types.NewIntDatum(rowID), nil 349 } 350 if err != nil { 351 return nil, logKVConvertFailed(logger, row, j, extraHandleColumnInfo, err) 352 } 353 record = append(record, value) 354 kvcodec.tbl.RebaseAutoID(kvcodec.se, value.GetInt64(), false, autoid.RowIDAllocType) 355 } 356 357 if len(kvcodec.genCols) > 0 { 358 mutRow := chunk.MutRowFromDatums(record) 359 for _, gc := range kvcodec.genCols { 360 col := cols[gc.index].ToInfo() 361 evaluated, err := gc.expr.Eval(mutRow.ToRow()) 362 if err != nil { 363 return nil, logEvalGenExprFailed(logger, row, col, err) 364 } 365 value, err := table.CastValue(kvcodec.se, evaluated, col, false, false) 366 if err != nil { 367 return nil, logEvalGenExprFailed(logger, row, col, err) 368 } 369 mutRow.SetDatum(gc.index, value) 370 record[gc.index] = value 371 } 372 } 373 374 _, err = kvcodec.tbl.AddRecord(kvcodec.se, record) 375 if err != nil { 376 logger.Error("kv encode failed", 377 zap.Array("originalRow", rowArrayMarshaler(row)), 378 zap.Array("convertedRow", rowArrayMarshaler(record)), 379 log.ShortError(err), 380 ) 381 return nil, errors.Trace(err) 382 } 383 pairs := kvcodec.se.takeKvPairs() 384 kvcodec.recordCache = record[:0] 385 return kvPairs(pairs), nil 386 } 387 388 func (kvs kvPairs) ClassifyAndAppend( 389 data *Rows, 390 dataChecksum *verification.KVChecksum, 391 indices *Rows, 392 indexChecksum *verification.KVChecksum, 393 ) { 394 dataKVs := (*data).(kvPairs) 395 indexKVs := (*indices).(kvPairs) 396 397 for _, kv := range kvs { 398 if kv.Key[tablecodec.TableSplitKeyLen+1] == 'r' { 399 dataKVs = append(dataKVs, kv) 400 dataChecksum.UpdateOne(kv) 401 } else { 402 indexKVs = append(indexKVs, kv) 403 indexChecksum.UpdateOne(kv) 404 } 405 } 406 407 *data = dataKVs 408 *indices = indexKVs 409 } 410 411 func (totalKVs kvPairs) SplitIntoChunks(splitSize int) []Rows { 412 if len(totalKVs) == 0 { 413 return nil 414 } 415 416 res := make([]Rows, 0, 1) 417 i := 0 418 cumSize := 0 419 420 for j, pair := range totalKVs { 421 size := len(pair.Key) + len(pair.Val) 422 if i < j && cumSize+size > splitSize { 423 res = append(res, kvPairs(totalKVs[i:j])) 424 i = j 425 cumSize = 0 426 } 427 cumSize += size 428 } 429 430 return append(res, kvPairs(totalKVs[i:])) 431 } 432 433 func (kvs kvPairs) Clear() Rows { 434 return kvPairs(kvs[:0]) 435 }