github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sqlmodel/reduce.go (about) 1 // Copyright 2022 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 sqlmodel 15 16 import ( 17 "strings" 18 19 "github.com/pingcap/tiflow/dm/pkg/log" 20 "go.uber.org/zap" 21 ) 22 23 // HasNotNullUniqueIdx returns true when the target table structure has PK or UK 24 // whose columns are all NOT NULL. 25 func (r *RowChange) HasNotNullUniqueIdx() bool { 26 return r.UniqueNotNullIdx() != nil 27 } 28 29 // IdentityValues returns the two group of values that can be used to identify 30 // the row. That is to say, if two row changes has same IdentityValues, they are 31 // changes of the same row. We can use this property to only replicate latest 32 // changes of one row. 33 // We always use same index for same table structure to get IdentityValues. 34 // two groups returned are from preValues and postValues. 35 func (r *RowChange) IdentityValues() ([]interface{}, []interface{}) { 36 r.lazyInitWhereHandle() 37 38 indexInfo := r.whereHandle.UniqueNotNullIdx 39 if indexInfo == nil { 40 return r.preValues, r.postValues 41 } 42 43 pre := make([]interface{}, 0, len(indexInfo.Columns)) 44 post := make([]interface{}, 0, len(indexInfo.Columns)) 45 46 for _, column := range indexInfo.Columns { 47 if r.preValues != nil { 48 pre = append(pre, r.preValues[column.Offset]) 49 } 50 if r.postValues != nil { 51 post = append(post, r.postValues[column.Offset]) 52 } 53 } 54 return pre, post 55 } 56 57 // RowIdentity returns the identity of this row change, caller should 58 // call IsIdentityUpdated/SplitUpdate before calling this method to 59 // make sure it's not updating the identity itself. 60 // we extract identity from preValues for update/delete, postValues for insert. 61 // if there's no primary key, return all values. 62 func (r *RowChange) RowIdentity() []interface{} { 63 r.lazyInitWhereHandle() 64 65 targetVals := r.preValues 66 if r.tp == RowChangeInsert { 67 targetVals = r.postValues 68 } 69 70 indexInfo := r.whereHandle.UniqueNotNullIdx 71 if indexInfo == nil { 72 return targetVals 73 } 74 75 identityVals := make([]interface{}, 0, len(indexInfo.Columns)) 76 for _, column := range indexInfo.Columns { 77 identityVals = append(identityVals, targetVals[column.Offset]) 78 } 79 return identityVals 80 } 81 82 // RowStrIdentity returns the identity of the row change as string slice 83 func (r *RowChange) RowStrIdentity() []string { 84 identity := r.RowIdentity() 85 identifyStr := make([]string, len(identity)) 86 for i := range identity { 87 identifyStr[i] = ColValAsStr(identity[i]) 88 } 89 return identifyStr 90 } 91 92 // IsIdentityUpdated returns true when the row is updated by the same values. 93 func (r *RowChange) IsIdentityUpdated() bool { 94 if r.tp != RowChangeUpdate { 95 return false 96 } 97 98 r.lazyInitWhereHandle() 99 pre, post := r.IdentityValues() 100 if len(pre) != len(post) { 101 // should not happen 102 return true 103 } 104 for i := range pre { 105 if pre[i] != post[i] { 106 return true 107 } 108 } 109 return false 110 } 111 112 // genKey gens key by values e.g. "a.1.b". 113 func genKey(values []interface{}) string { 114 builder := new(strings.Builder) 115 for i, v := range values { 116 if i != 0 { 117 builder.WriteString(".") 118 } 119 builder.WriteString(ColValAsStr(v)) 120 } 121 122 return builder.String() 123 } 124 125 // IdentityKey returns a string generated by IdentityValues. 126 // If RowChange.IsIdentityUpdated, the behaviour is undefined. 127 func (r *RowChange) IdentityKey() string { 128 pre, post := r.IdentityValues() 129 if len(pre) != 0 { 130 return genKey(pre) 131 } 132 return genKey(post) 133 } 134 135 // Reduce will merge two row changes of same row into one row changes, 136 // e.g., INSERT{1} + UPDATE{1 -> 2} -> INSERT{2}. Receiver will be changed 137 // in-place. 138 func (r *RowChange) Reduce(preRowChange *RowChange) { 139 if r.IdentityKey() != preRowChange.IdentityKey() { 140 log.L().DPanic("reduce row change failed, identity key not match", 141 zap.String("preID", preRowChange.IdentityKey()), 142 zap.String("curID", r.IdentityKey())) 143 return 144 } 145 146 // special handle INSERT + DELETE -> DELETE 147 if r.tp == RowChangeDelete && preRowChange.tp == RowChangeInsert { 148 return 149 } 150 151 r.preValues = preRowChange.preValues 152 r.calculateType() 153 } 154 155 // SplitUpdate will split current RowChangeUpdate into two RowChangeDelete and 156 // RowChangeInsert one. The behaviour is undefined for other types of RowChange. 157 func (r *RowChange) SplitUpdate() (*RowChange, *RowChange) { 158 if r.tp != RowChangeUpdate { 159 log.L().DPanic("SplitUpdate should only be called on RowChangeUpdate", 160 zap.Stringer("rowChange", r)) 161 return nil, nil 162 } 163 164 pre := &RowChange{ 165 sourceTable: r.sourceTable, 166 targetTable: r.targetTable, 167 preValues: r.preValues, 168 sourceTableInfo: r.sourceTableInfo, 169 targetTableInfo: r.targetTableInfo, 170 tiSessionCtx: r.tiSessionCtx, 171 tp: RowChangeDelete, 172 whereHandle: r.whereHandle, 173 } 174 post := &RowChange{ 175 sourceTable: r.sourceTable, 176 targetTable: r.targetTable, 177 postValues: r.postValues, 178 sourceTableInfo: r.sourceTableInfo, 179 targetTableInfo: r.targetTableInfo, 180 tiSessionCtx: r.tiSessionCtx, 181 tp: RowChangeInsert, 182 whereHandle: r.whereHandle, 183 } 184 185 return pre, post 186 }