github.com/matrixorigin/matrixone@v1.2.0/pkg/sql/colexec/hashbuild/build.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 hashbuild 16 17 import ( 18 "bytes" 19 "runtime" 20 21 "github.com/matrixorigin/matrixone/pkg/common/hashmap" 22 "github.com/matrixorigin/matrixone/pkg/container/batch" 23 "github.com/matrixorigin/matrixone/pkg/container/types" 24 "github.com/matrixorigin/matrixone/pkg/container/vector" 25 "github.com/matrixorigin/matrixone/pkg/sql/colexec" 26 "github.com/matrixorigin/matrixone/pkg/vm" 27 "github.com/matrixorigin/matrixone/pkg/vm/process" 28 ) 29 30 const argName = "hash_build" 31 32 func (arg *Argument) String(buf *bytes.Buffer) { 33 buf.WriteString(argName) 34 buf.WriteString(": hash build ") 35 } 36 37 func (arg *Argument) Prepare(proc *process.Process) (err error) { 38 ap := arg 39 ap.ctr = new(container) 40 if len(proc.Reg.MergeReceivers) > 1 { 41 ap.ctr.InitReceiver(proc, true) 42 ap.ctr.isMerge = true 43 } else { 44 ap.ctr.InitReceiver(proc, false) 45 } 46 47 if ap.NeedHashMap { 48 ap.ctr.vecs = make([][]*vector.Vector, 0) 49 ctr := ap.ctr 50 ctr.executor = make([]colexec.ExpressionExecutor, len(ap.Conditions)) 51 ctr.keyWidth = 0 52 for i, expr := range ap.Conditions { 53 typ := expr.Typ 54 width := types.T(typ.Id).TypeLen() 55 // todo : for varlena type, always go strhashmap 56 if types.T(typ.Id).FixedLength() < 0 { 57 width = 128 58 } 59 ctr.keyWidth += width 60 ctr.executor[i], err = colexec.NewExpressionExecutor(proc, ap.Conditions[i]) 61 if err != nil { 62 return err 63 } 64 } 65 66 if ctr.keyWidth <= 8 { 67 if ctr.intHashMap, err = hashmap.NewIntHashMap(false, ap.Ibucket, ap.Nbucket, proc.Mp()); err != nil { 68 return err 69 } 70 } else { 71 if ctr.strHashMap, err = hashmap.NewStrMap(false, ap.Ibucket, ap.Nbucket, proc.Mp()); err != nil { 72 return err 73 } 74 } 75 76 } 77 78 ap.ctr.batches = make([]*batch.Batch, 0) 79 80 return nil 81 } 82 83 func (arg *Argument) Call(proc *process.Process) (vm.CallResult, error) { 84 if err, isCancel := vm.CancelCheck(proc); isCancel { 85 return vm.CancelResult, err 86 } 87 88 anal := proc.GetAnalyze(arg.GetIdx(), arg.GetParallelIdx(), arg.GetParallelMajor()) 89 anal.Start() 90 defer anal.Stop() 91 result := vm.NewCallResult() 92 ap := arg 93 ctr := ap.ctr 94 for { 95 switch ctr.state { 96 case BuildHashMap: 97 if err := ctr.build(ap, proc, anal, arg.GetIsFirst()); err != nil { 98 ctr.cleanHashMap() 99 return result, err 100 } 101 if ap.ctr.intHashMap != nil { 102 anal.Alloc(ap.ctr.intHashMap.Size()) 103 } else if ap.ctr.strHashMap != nil { 104 anal.Alloc(ap.ctr.strHashMap.Size()) 105 } 106 ctr.state = HandleRuntimeFilter 107 108 case HandleRuntimeFilter: 109 if err := ctr.handleRuntimeFilter(ap, proc); err != nil { 110 return result, err 111 } 112 ctr.state = SendHashMap 113 114 case SendHashMap: 115 result.Batch = batch.NewWithSize(0) 116 117 if ctr.inputBatchRowCount > 0 { 118 var jm *hashmap.JoinMap 119 if ap.NeedHashMap { 120 if ctr.keyWidth <= 8 { 121 jm = hashmap.NewJoinMap(ctr.multiSels, nil, ctr.intHashMap, nil, ctr.hasNull, ap.IsDup) 122 } else { 123 jm = hashmap.NewJoinMap(ctr.multiSels, nil, nil, ctr.strHashMap, ctr.hasNull, ap.IsDup) 124 } 125 jm.SetPushedRuntimeFilterIn(ctr.runtimeFilterIn) 126 result.Batch.AuxData = jm 127 } 128 ctr.intHashMap = nil 129 ctr.strHashMap = nil 130 ctr.multiSels = nil 131 } else { 132 ctr.cleanHashMap() 133 } 134 135 // this is just a dummy batch to indicate that the batch is must not empty. 136 // we should make sure this batch can be sent to the next join operator in other pipelines. 137 if result.Batch.IsEmpty() && ap.NeedHashMap { 138 result.Batch.AddRowCount(1) 139 } 140 141 ctr.state = SendBatch 142 return result, nil 143 case SendBatch: 144 if ctr.batchIdx >= len(ctr.batches) { 145 ctr.state = End 146 } else { 147 result.Batch = ctr.batches[ctr.batchIdx] 148 ctr.batchIdx++ 149 } 150 return result, nil 151 default: 152 result.Batch = nil 153 result.Status = vm.ExecStop 154 return result, nil 155 } 156 } 157 } 158 159 // make sure src is not empty 160 func (ctr *container) mergeIntoBatches(src *batch.Batch, proc *process.Process) error { 161 var err error 162 if src.RowCount() == colexec.DefaultBatchSize { 163 ctr.batches = append(ctr.batches, src) 164 return nil 165 } else { 166 offset := 0 167 appendRows := 0 168 length := src.RowCount() 169 for offset < length { 170 ctr.tmpBatch, appendRows, err = proc.AppendToFixedSizeFromOffset(ctr.tmpBatch, src, offset) 171 if err != nil { 172 return err 173 } 174 if ctr.tmpBatch.RowCount() == colexec.DefaultBatchSize { 175 ctr.batches = append(ctr.batches, ctr.tmpBatch) 176 ctr.tmpBatch = nil 177 } 178 offset += appendRows 179 } 180 proc.PutBatch(src) 181 } 182 return nil 183 } 184 185 func (ctr *container) collectBuildBatches(ap *Argument, proc *process.Process, anal process.Analyze, isFirst bool) error { 186 var err error 187 var currentBatch *batch.Batch 188 for { 189 if ap.ctr.isMerge { 190 currentBatch, _, err = ctr.ReceiveFromAllRegs(anal) 191 } else { 192 currentBatch, _, err = ctr.ReceiveFromSingleReg(0, anal) 193 } 194 if err != nil { 195 return err 196 } 197 if currentBatch == nil { 198 break 199 } 200 if currentBatch.IsEmpty() { 201 proc.PutBatch(currentBatch) 202 continue 203 } 204 anal.Input(currentBatch, isFirst) 205 anal.Alloc(int64(currentBatch.Size())) 206 ctr.inputBatchRowCount += currentBatch.RowCount() 207 err = ctr.mergeIntoBatches(currentBatch, proc) 208 if err != nil { 209 return err 210 } 211 } 212 if ctr.tmpBatch != nil && ctr.tmpBatch.RowCount() > 0 { 213 ctr.batches = append(ctr.batches, ctr.tmpBatch) 214 ctr.tmpBatch = nil 215 } 216 return nil 217 } 218 219 func (ctr *container) buildHashmap(ap *Argument, proc *process.Process) error { 220 if len(ctr.batches) == 0 || !ap.NeedHashMap { 221 return nil 222 } 223 var err error 224 if err = ctr.evalJoinCondition(proc); err != nil { 225 return err 226 } 227 228 var itr hashmap.Iterator 229 if ctr.keyWidth <= 8 { 230 itr = ctr.intHashMap.NewIterator() 231 } else { 232 itr = ctr.strHashMap.NewIterator() 233 } 234 235 if ap.HashOnPK { 236 // if hash on primary key, prealloc hashmap size to the count of batch 237 if ctr.keyWidth <= 8 { 238 err = ctr.intHashMap.PreAlloc(uint64(ctr.inputBatchRowCount), proc.Mp()) 239 } else { 240 err = ctr.strHashMap.PreAlloc(uint64(ctr.inputBatchRowCount), proc.Mp()) 241 } 242 if err != nil { 243 return err 244 } 245 } else { 246 if ap.NeedAllocateSels { 247 ctr.multiSels = make([][]int32, ctr.inputBatchRowCount) 248 } 249 } 250 251 var ( 252 cardinality uint64 253 sels []int32 254 ) 255 256 for i := 0; i < ctr.inputBatchRowCount; i += hashmap.UnitLimit { 257 if i%(hashmap.UnitLimit*32) == 0 { 258 runtime.Gosched() 259 } 260 n := ctr.inputBatchRowCount - i 261 if n > hashmap.UnitLimit { 262 n = hashmap.UnitLimit 263 } 264 265 // if not hash on primary key, estimate the hashmap size after 8192 rows 266 //preAlloc to improve performance and reduce memory reAlloc 267 if !ap.HashOnPK && ctr.inputBatchRowCount > hashmap.HashMapSizeThreshHold && i == hashmap.HashMapSizeEstimate { 268 if ctr.keyWidth <= 8 { 269 groupCount := ctr.intHashMap.GroupCount() 270 rate := float64(groupCount) / float64(i) 271 hashmapCount := uint64(float64(ctr.inputBatchRowCount) * rate) 272 if hashmapCount > groupCount { 273 err = ctr.intHashMap.PreAlloc(hashmapCount-groupCount, proc.Mp()) 274 } 275 } else { 276 groupCount := ctr.strHashMap.GroupCount() 277 rate := float64(groupCount) / float64(i) 278 hashmapCount := uint64(float64(ctr.inputBatchRowCount) * rate) 279 if hashmapCount > groupCount { 280 err = ctr.strHashMap.PreAlloc(hashmapCount-groupCount, proc.Mp()) 281 } 282 } 283 if err != nil { 284 return err 285 } 286 } 287 288 vecIdx1 := i / colexec.DefaultBatchSize 289 vecIdx2 := i % colexec.DefaultBatchSize 290 vals, zvals, err := itr.Insert(vecIdx2, n, ctr.vecs[vecIdx1]) 291 if err != nil { 292 return err 293 } 294 for k, v := range vals[:n] { 295 if zvals[k] == 0 { 296 ctr.hasNull = true 297 continue 298 } 299 if v == 0 { 300 continue 301 } 302 ai := int64(v) - 1 303 304 if !ap.HashOnPK && ap.NeedAllocateSels { 305 if ctr.multiSels[ai] == nil { 306 ctr.multiSels[ai] = make([]int32, 0) 307 } 308 ctr.multiSels[ai] = append(ctr.multiSels[ai], int32(i+k)) 309 } 310 } 311 312 if ap.RuntimeFilterSpec != nil { 313 if len(ap.ctr.uniqueJoinKeys) == 0 { 314 ap.ctr.uniqueJoinKeys = make([]*vector.Vector, len(ctr.executor)) 315 for j, vec := range ctr.vecs[vecIdx1] { 316 ap.ctr.uniqueJoinKeys[j] = proc.GetVector(*vec.GetType()) 317 } 318 } 319 320 if ap.HashOnPK { 321 for j, vec := range ctr.vecs[vecIdx1] { 322 err = ap.ctr.uniqueJoinKeys[j].UnionBatch(vec, int64(vecIdx2), n, nil, proc.Mp()) 323 if err != nil { 324 return err 325 } 326 } 327 } else { 328 if sels == nil { 329 sels = make([]int32, hashmap.UnitLimit) 330 } 331 332 sels = sels[:0] 333 for j, v := range vals[:n] { 334 if v > cardinality { 335 sels = append(sels, int32(i+j)) 336 cardinality = v 337 } 338 } 339 340 for j, vec := range ctr.vecs[vecIdx1] { 341 for _, sel := range sels { 342 _, idx2 := sel/colexec.DefaultBatchSize, sel%colexec.DefaultBatchSize 343 err = ap.ctr.uniqueJoinKeys[j].UnionOne(vec, int64(idx2), proc.Mp()) 344 if err != nil { 345 return err 346 } 347 } 348 } 349 } 350 } 351 } 352 353 return nil 354 } 355 356 func (ctr *container) build(ap *Argument, proc *process.Process, anal process.Analyze, isFirst bool) error { 357 err := ctr.collectBuildBatches(ap, proc, anal, isFirst) 358 if err != nil { 359 return err 360 } 361 err = ctr.buildHashmap(ap, proc) 362 if err != nil { 363 return err 364 } 365 if !ap.NeedMergedBatch { 366 // if do not need merged batch, free it now to save memory 367 for i := range ctr.batches { 368 proc.PutBatch(ctr.batches[i]) 369 } 370 ctr.batches = nil 371 } 372 return nil 373 } 374 375 func (ctr *container) handleRuntimeFilter(ap *Argument, proc *process.Process) error { 376 if ap.RuntimeFilterSpec == nil { 377 return nil 378 } 379 380 var runtimeFilter process.RuntimeFilterMessage 381 runtimeFilter.Tag = ap.RuntimeFilterSpec.Tag 382 383 if ap.RuntimeFilterSpec.Expr == nil { 384 runtimeFilter.Typ = process.RuntimeFilter_PASS 385 proc.SendRuntimeFilter(runtimeFilter, ap.RuntimeFilterSpec) 386 return nil 387 } else if ctr.inputBatchRowCount == 0 || len(ctr.uniqueJoinKeys) == 0 || ctr.uniqueJoinKeys[0].Length() == 0 { 388 runtimeFilter.Typ = process.RuntimeFilter_DROP 389 proc.SendRuntimeFilter(runtimeFilter, ap.RuntimeFilterSpec) 390 return nil 391 } 392 393 var hashmapCount uint64 394 if ctr.keyWidth <= 8 { 395 hashmapCount = ctr.intHashMap.GroupCount() 396 } else { 397 hashmapCount = ctr.strHashMap.GroupCount() 398 } 399 400 inFilterCardLimit := ap.RuntimeFilterSpec.UpperLimit 401 //inFilterCardLimit := plan.GetInFilterCardLimit() 402 //bloomFilterCardLimit := int64(plan.BloomFilterCardLimit) 403 //v, ok = runtime.ProcessLevelRuntime().GetGlobalVariables("runtime_filter_limit_bloom_filter") 404 //if ok { 405 // bloomFilterCardLimit = v.(int64) 406 //} 407 408 vec := ctr.uniqueJoinKeys[0] 409 410 defer func() { 411 vec.Free(proc.Mp()) 412 ctr.uniqueJoinKeys = nil 413 }() 414 415 if hashmapCount > uint64(inFilterCardLimit) { 416 runtimeFilter.Typ = process.RuntimeFilter_PASS 417 proc.SendRuntimeFilter(runtimeFilter, ap.RuntimeFilterSpec) 418 return nil 419 } else { 420 // Composite primary key 421 if ap.RuntimeFilterSpec.Expr.GetF() != nil { 422 bat := batch.NewWithSize(len(ctr.uniqueJoinKeys)) 423 bat.SetRowCount(vec.Length()) 424 copy(bat.Vecs, ctr.uniqueJoinKeys) 425 426 newVec, err := colexec.EvalExpressionOnce(proc, ap.RuntimeFilterSpec.Expr, []*batch.Batch{bat}) 427 if err != nil { 428 return err 429 } 430 431 for i := range ctr.uniqueJoinKeys { 432 ctr.uniqueJoinKeys[i].Free(proc.Mp()) 433 } 434 435 vec = newVec 436 } 437 438 vec.InplaceSort() 439 data, err := vec.MarshalBinary() 440 if err != nil { 441 return err 442 } 443 444 runtimeFilter.Typ = process.RuntimeFilter_IN 445 runtimeFilter.Card = int32(vec.Length()) 446 runtimeFilter.Data = data 447 proc.SendRuntimeFilter(runtimeFilter, ap.RuntimeFilterSpec) 448 ctr.runtimeFilterIn = true 449 } 450 return nil 451 } 452 453 func (ctr *container) evalJoinCondition(proc *process.Process) error { 454 for idx1 := range ctr.batches { 455 tmpVes := make([]*vector.Vector, len(ctr.executor)) 456 ctr.vecs = append(ctr.vecs, tmpVes) 457 for idx2 := range ctr.executor { 458 vec, err := ctr.executor[idx2].Eval(proc, []*batch.Batch{ctr.batches[idx1]}) 459 if err != nil { 460 return err 461 } 462 ctr.vecs[idx1][idx2] = vec 463 } 464 } 465 return nil 466 }