github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/pipeline/pipeline.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 pipeline 16 17 import ( 18 "bytes" 19 20 "github.com/matrixorigin/matrixone/pkg/container/batch" 21 "github.com/matrixorigin/matrixone/pkg/vm" 22 "github.com/matrixorigin/matrixone/pkg/vm/engine" 23 "github.com/matrixorigin/matrixone/pkg/vm/process" 24 ) 25 26 func New(attrs []string, ins vm.Instructions, reg *process.WaitRegister) *Pipeline { 27 return &Pipeline{ 28 reg: reg, 29 instructions: ins, 30 attrs: attrs, 31 } 32 } 33 34 func NewMerge(ins vm.Instructions, reg *process.WaitRegister) *Pipeline { 35 return &Pipeline{ 36 reg: reg, 37 instructions: ins, 38 } 39 } 40 41 func (p *Pipeline) String() string { 42 var buf bytes.Buffer 43 44 vm.String(p.instructions, &buf) 45 return buf.String() 46 } 47 48 func (p *Pipeline) Run(r engine.Reader, proc *process.Process) (end bool, err error) { 49 var bat *batch.Batch 50 // used to handle some push-down request 51 if p.reg != nil { 52 select { 53 case <-p.reg.Ctx.Done(): 54 case <-p.reg.Ch: 55 } 56 } 57 if err = vm.Prepare(p.instructions, proc); err != nil { 58 p.cleanup(proc, true) 59 return false, err 60 } 61 62 for { 63 select { 64 case <-proc.Ctx.Done(): 65 proc.SetInputBatch(nil) 66 return true, nil 67 default: 68 } 69 // read data from storage engine 70 if bat, err = r.Read(proc.Ctx, p.attrs, nil, proc.Mp()); err != nil { 71 p.cleanup(proc, true) 72 return false, err 73 } 74 if bat != nil { 75 bat.Cnt = 1 76 77 analyzeIdx := p.instructions[0].Idx 78 a := proc.GetAnalyze(analyzeIdx) 79 a.S3IOByte(bat) 80 a.Alloc(int64(bat.Size())) 81 } 82 83 proc.SetInputBatch(bat) 84 end, err = vm.Run(p.instructions, proc) 85 if err != nil { 86 p.cleanup(proc, true) 87 return end, err 88 } 89 if end { 90 // end is true means pipeline successfully completed 91 p.cleanup(proc, false) 92 return end, nil 93 } 94 } 95 } 96 97 func (p *Pipeline) ConstRun(bat *batch.Batch, proc *process.Process) (end bool, err error) { 98 // used to handle some push-down request 99 if p.reg != nil { 100 select { 101 case <-p.reg.Ctx.Done(): 102 case <-p.reg.Ch: 103 } 104 } 105 106 if err = vm.Prepare(p.instructions, proc); err != nil { 107 p.cleanup(proc, true) 108 return false, err 109 } 110 bat.Cnt = 1 111 pipelineInputBatches := []*batch.Batch{bat, nil} 112 for { 113 for i := range pipelineInputBatches { 114 proc.SetInputBatch(pipelineInputBatches[i]) 115 end, err = vm.Run(p.instructions, proc) 116 if err != nil { 117 p.cleanup(proc, true) 118 return end, err 119 } 120 if end { 121 p.cleanup(proc, false) 122 return end, nil 123 } 124 } 125 } 126 } 127 128 func (p *Pipeline) MergeRun(proc *process.Process) (end bool, err error) { 129 // used to handle some push-down request 130 if p.reg != nil { 131 select { 132 case <-p.reg.Ctx.Done(): 133 case <-p.reg.Ch: 134 } 135 } 136 137 if err = vm.Prepare(p.instructions, proc); err != nil { 138 proc.Cancel() 139 p.cleanup(proc, true) 140 return false, err 141 } 142 for { 143 end, err = vm.Run(p.instructions, proc) 144 if err != nil { 145 proc.Cancel() 146 p.cleanup(proc, true) 147 return end, err 148 } 149 if end { 150 proc.Cancel() 151 p.cleanup(proc, false) 152 return end, nil 153 } 154 } 155 }