github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/compile/debugTools.go (about) 1 // Copyright 2021 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package compile 16 17 import ( 18 "fmt" 19 "strings" 20 21 "github.com/matrixorigin/matrixone/pkg/sql/colexec/connector" 22 "github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch" 23 "github.com/matrixorigin/matrixone/pkg/vm" 24 "github.com/matrixorigin/matrixone/pkg/vm/process" 25 ) 26 27 var debugInstructionNames = map[vm.OpType]string{ 28 vm.Top: "top", 29 vm.Join: "join", 30 vm.Semi: "semi", 31 vm.RightSemi: "right semi", 32 vm.RightAnti: "right anti", 33 vm.Left: "left", 34 vm.Right: "right", 35 vm.Limit: "limit", 36 vm.Merge: "merge", 37 vm.Order: "order", 38 vm.Group: "group", 39 vm.Output: "output", 40 vm.Offset: "offset", 41 vm.Product: "product", 42 vm.Restrict: "restrict", 43 vm.Dispatch: "dispatch", 44 vm.Shuffle: "shuffle", 45 vm.Connector: "connect", 46 vm.Projection: "projection", 47 vm.Anti: "anti", 48 vm.Single: "single", 49 vm.Mark: "mark", 50 vm.LoopJoin: "loop join", 51 vm.LoopLeft: "loop left", 52 vm.LoopSemi: "loop semi", 53 vm.LoopAnti: "loop anti", 54 vm.LoopSingle: "loop single", 55 vm.LoopMark: "loop mark", 56 vm.MergeTop: "merge top", 57 vm.MergeLimit: "merge limit", 58 vm.MergeOrder: "merge order", 59 vm.MergeGroup: "merge group", 60 vm.MergeOffset: "merge offset", 61 vm.MergeRecursive: "merge recursive", 62 vm.MergeCTE: "merge cte", 63 vm.Partition: "partition", 64 vm.Deletion: "delete", 65 vm.Insert: "insert", 66 vm.PreInsert: "pre insert", 67 vm.PreInsertUnique: "pre insert uk", 68 vm.PreInsertSecondaryIndex: "pre insert 2nd", 69 vm.External: "external", 70 vm.Source: "source", 71 vm.Minus: "minus", 72 vm.Intersect: "intersect", 73 vm.IntersectAll: "intersect all", 74 vm.HashBuild: "hash build", 75 vm.MergeDelete: "merge delete", 76 vm.LockOp: "lockop", 77 vm.MergeBlock: "merge block", 78 vm.FuzzyFilter: "fuzzy filter", 79 vm.Sample: "sample", 80 vm.Window: "window", 81 vm.TimeWin: "timewin", 82 vm.Fill: "fill", 83 vm.TableScan: "tablescan", 84 vm.ValueScan: "valuescan", 85 vm.TableFunction: "tablefunction", 86 } 87 88 var debugMagicNames = map[magicType]string{ 89 Merge: "Merge", 90 Normal: "Normal", 91 Remote: "Remote", 92 Parallel: "Parallel", 93 CreateDatabase: "CreateDatabase", 94 CreateTable: "CreateTable", 95 CreateIndex: "CreateIndex", 96 DropDatabase: "DropDatabase", 97 DropTable: "DropTable", 98 DropIndex: "DropIndex", 99 MergeDelete: "MergeDelete", 100 MergeInsert: "MergeInsert", 101 } 102 103 var _ = DebugShowScopes 104 105 // DebugShowScopes show information of a scope structure. 106 func DebugShowScopes(ss []*Scope) string { 107 var generateReceiverMap func(*Scope, map[*process.WaitRegister]int) 108 generateReceiverMap = func(s *Scope, mp map[*process.WaitRegister]int) { 109 for i := range s.PreScopes { 110 generateReceiverMap(s.PreScopes[i], mp) 111 } 112 if s.Proc == nil { 113 return 114 } 115 for i := range s.Proc.Reg.MergeReceivers { 116 mp[s.Proc.Reg.MergeReceivers[i]] = len(mp) 117 } 118 } 119 120 receiverMap := make(map[*process.WaitRegister]int) 121 for i := range ss { 122 generateReceiverMap(ss[i], receiverMap) 123 } 124 125 return debugShowScopes(ss, 0, receiverMap) 126 } 127 128 func debugShowScopes(ss []*Scope, gap int, rmp map[*process.WaitRegister]int) string { 129 // new line and start with n space 130 gapNextLine := func() string { 131 str := "\n" 132 for i := 0; i < gap; i++ { 133 str += " " 134 } 135 return str 136 } 137 138 // return n space 139 addGap := func() string { 140 str := "" 141 for i := 0; i < gap; i++ { 142 str += " " 143 } 144 return str 145 } 146 147 // get receiver id string 148 getReceiverStr := func(s *Scope, rs []*process.WaitRegister) string { 149 str := "[" 150 for i := range rs { 151 remote := "" 152 for _, u := range s.RemoteReceivRegInfos { 153 if u.Idx == i { 154 remote = fmt.Sprintf("(%s)", u.Uuid) 155 break 156 } 157 } 158 if i != 0 { 159 str += ", " 160 } 161 if id, ok := rmp[rs[i]]; ok { 162 str += fmt.Sprintf("%d%s", id, remote) 163 } else { 164 str += "unknown" 165 } 166 } 167 str += "]" 168 return str 169 } 170 171 // convert magic to its string name 172 magicShow := func(magic magicType) string { 173 name, ok := debugMagicNames[magic] 174 if ok { 175 return name 176 } 177 return "unknown" 178 } 179 180 // explain the datasource 181 showDataSource := func(source *Source) string { 182 if source == nil { 183 return "nil" 184 } 185 s := fmt.Sprintf("%s.%s%s", source.SchemaName, source.RelationName, source.Attributes) 186 return strings.TrimLeft(s, ".") 187 } 188 189 // explain the operator information 190 showInstruction := func(instruction vm.Instruction, mp map[*process.WaitRegister]int) string { 191 id := instruction.Op 192 name, ok := debugInstructionNames[id] 193 if ok { 194 str := name 195 if id == vm.Connector { 196 var receiver = "unknown" 197 arg := instruction.Arg.(*connector.Argument) 198 if receiverId, okk := mp[arg.Reg]; okk { 199 receiver = fmt.Sprintf("%d", receiverId) 200 } 201 str += fmt.Sprintf(" to MergeReceiver %s", receiver) 202 } 203 if id == vm.Dispatch { 204 arg := instruction.Arg.(*dispatch.Argument) 205 chs := "" 206 for i := range arg.LocalRegs { 207 if i != 0 { 208 chs += ", " 209 } 210 if receiverId, okk := mp[arg.LocalRegs[i]]; okk { 211 chs += fmt.Sprintf("%d", receiverId) 212 } else { 213 chs += "unknown" 214 } 215 } 216 switch arg.FuncId { 217 case dispatch.ShuffleToAllFunc: 218 str += fmt.Sprintf(" shuffle to all of MergeReceiver [%s].", chs) 219 case dispatch.SendToAllFunc, dispatch.SendToAllLocalFunc: 220 str += fmt.Sprintf(" to all of MergeReceiver [%s].", chs) 221 case dispatch.SendToAnyLocalFunc: 222 str += fmt.Sprintf(" to any of MergeReceiver [%s].", chs) 223 default: 224 str += fmt.Sprintf(" unknow type dispatch [%s].", chs) 225 } 226 227 if len(arg.RemoteRegs) != 0 { 228 remoteChs := "" 229 for i, reg := range arg.RemoteRegs { 230 if i != 0 { 231 remoteChs += ", " 232 } 233 remoteChs += fmt.Sprintf("[addr: %s, uuid %s]", reg.NodeAddr, reg.Uuid) 234 } 235 str += fmt.Sprintf(" cross-cn receiver info: %s", remoteChs) 236 } 237 } 238 return str 239 } 240 return "unknown" 241 } 242 243 var result string 244 for i := range ss { 245 str := addGap() 246 receiverStr := "nil" 247 if ss[i].Proc != nil { 248 receiverStr = getReceiverStr(ss[i], ss[i].Proc.Reg.MergeReceivers) 249 } 250 str += fmt.Sprintf("Scope %d (Magic: %s, Receiver: %s): [", i+1, magicShow(ss[i].Magic), receiverStr) 251 for j, instruction := range ss[i].Instructions { 252 if j != 0 { 253 str += " -> " 254 } 255 str += showInstruction(instruction, rmp) 256 } 257 str += "]" 258 if ss[i].DataSource != nil { 259 str += gapNextLine() 260 str += fmt.Sprintf("DataSource: %s,", showDataSource(ss[i].DataSource)) 261 } 262 if len(ss[i].PreScopes) > 0 { 263 str += gapNextLine() 264 str += " PreScopes: {" 265 str += debugShowScopes(ss[i].PreScopes, gap+2, rmp) 266 str += gapNextLine() 267 str += "}" 268 } 269 result += "\n" 270 result += str 271 } 272 return result 273 }