From 19bbe15e258fd39bf54eac17d30cf2229518cd4f Mon Sep 17 00:00:00 2001 From: huby2358 Date: Mon, 4 Nov 2024 11:37:10 +0800 Subject: [PATCH] refactor: valuescan --- pkg/frontend/computation_wrapper.go | 11 - pkg/frontend/mysql_cmd_executor.go | 2 - pkg/frontend/session.go | 37 +- pkg/frontend/types.go | 17 +- pkg/sql/colexec/evalExpression.go | 3 +- pkg/sql/colexec/value_scan/types.go | 59 ++- pkg/sql/colexec/value_scan/value_scan.go | 185 ++++---- pkg/sql/compile/compile.go | 16 +- pkg/sql/compile/compile2.go | 3 - pkg/sql/compile/operator.go | 44 +- pkg/sql/plan/bind_insert.go | 73 +--- pkg/sql/plan/build_constraint_util.go | 152 ++++--- pkg/sql/plan/build_insert.go | 198 +-------- pkg/sql/plan/deepcopy.go | 4 +- pkg/sql/plan/make.go | 245 +++++++++++ pkg/sql/plan/query_builder.go | 20 +- pkg/sql/plan/rule/constant_fold.go | 197 +++++++++ pkg/sql/plan/types.go | 21 +- pkg/sql/util/eval_expr_util.go | 511 ++++++++--------------- pkg/vm/process/process.go | 20 - pkg/vm/process/process2.go | 5 +- pkg/vm/process/types.go | 43 +- proto/plan.proto | 3 +- 23 files changed, 934 insertions(+), 935 deletions(-) diff --git a/pkg/frontend/computation_wrapper.go b/pkg/frontend/computation_wrapper.go index 147afdd9c66a..40ab8d5b824d 100644 --- a/pkg/frontend/computation_wrapper.go +++ b/pkg/frontend/computation_wrapper.go @@ -371,17 +371,6 @@ func initExecuteStmtParam(reqCtx context.Context, ses *Session, cwft *TxnComputa } } - // The default count is 1. Setting it to 2 ensures that memory will not be reclaimed. - // Convenient to reuse memory next time - if prepareStmt.InsertBat != nil { - cwft.proc.SetPrepareBatch(prepareStmt.InsertBat) - for i := 0; i < len(prepareStmt.exprList); i++ { - for j := range prepareStmt.exprList[i] { - prepareStmt.exprList[i][j].ResetForNextQuery() - } - } - cwft.proc.SetPrepareExprList(prepareStmt.exprList) - } numParams := len(preparePlan.ParamTypes) if prepareStmt.params != nil && prepareStmt.params.Length() > 0 { // use binary protocol if prepareStmt.params.Length() != numParams { diff --git a/pkg/frontend/mysql_cmd_executor.go b/pkg/frontend/mysql_cmd_executor.go index f6127db742f3..dfb44df2d32f 100644 --- a/pkg/frontend/mysql_cmd_executor.go +++ b/pkg/frontend/mysql_cmd_executor.go @@ -1117,7 +1117,6 @@ func createPrepareStmt( PrepareStmt: saveStmt, getFromSendLongData: make(map[int]struct{}), } - prepareStmt.InsertBat = ses.GetTxnCompileCtx().GetProcess().GetPrepareBatch() dcPrepare, ok := preparePlan.GetDcl().Control.(*plan.DataControl_Prepare) if ok { @@ -2852,7 +2851,6 @@ func doComQuery(ses *Session, execCtx *ExecCtx, input *UserInput) (retErr error) proc.ReplaceTopCtx(execCtx.reqCtx) pu := getPu(ses.GetService()) - proc.CopyValueScanBatch(ses.proc) proc.Base.Id = ses.getNextProcessId() proc.Base.Lim.Size = pu.SV.ProcessLimitationSize proc.Base.Lim.BatchRows = pu.SV.ProcessLimitationBatchRows diff --git a/pkg/frontend/session.go b/pkg/frontend/session.go index 8014f91d7966..289f806406e7 100644 --- a/pkg/frontend/session.go +++ b/pkg/frontend/session.go @@ -40,7 +40,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/query" "github.com/matrixorigin/matrixone/pkg/pb/status" "github.com/matrixorigin/matrixone/pkg/perfcounter" - "github.com/matrixorigin/matrixone/pkg/sql/colexec" "github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect/mysql" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" @@ -956,14 +955,7 @@ func (ses *Session) SetPrepareStmt(ctx context.Context, name string, prepareStmt } else { stmt.Close() } - if prepareStmt != nil && prepareStmt.PreparePlan != nil { - isInsertValues, exprList := checkPlanIsInsertValues(ses.proc, - prepareStmt.PreparePlan.GetDcl().GetPrepare().GetPlan()) - if isInsertValues { - prepareStmt.proc = ses.proc - prepareStmt.exprList = exprList - } - } + ses.prepareStmts[name] = prepareStmt return nil @@ -1686,33 +1678,6 @@ func (ses *Session) reset(prev *Session) error { return nil } -func checkPlanIsInsertValues(proc *process.Process, - p *plan.Plan) (bool, [][]colexec.ExpressionExecutor) { - qry := p.GetQuery() - if qry != nil { - for _, node := range qry.Nodes { - if node.NodeType == plan.Node_VALUE_SCAN && node.RowsetData != nil { - exprList := make([][]colexec.ExpressionExecutor, len(node.RowsetData.Cols)) - for i, col := range node.RowsetData.Cols { - exprList[i] = make([]colexec.ExpressionExecutor, 0, len(col.Data)) - for _, data := range col.Data { - if data.Pos >= 0 { - continue - } - expr, err := colexec.NewExpressionExecutor(proc, data.Expr) - if err != nil { - return false, nil - } - exprList[i] = append(exprList[i], expr) - } - } - return true, exprList - } - } - } - return false, nil -} - func commitAfterMigrate(ses *Session, err error) error { //if ses == nil { // logutil.Error("session is nil") diff --git a/pkg/frontend/types.go b/pkg/frontend/types.go index 90eea5a8b6ec..6e08ec929e3d 100644 --- a/pkg/frontend/types.go +++ b/pkg/frontend/types.go @@ -38,7 +38,6 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/perfcounter" - "github.com/matrixorigin/matrixone/pkg/sql/colexec" "github.com/matrixorigin/matrixone/pkg/sql/compile" "github.com/matrixorigin/matrixone/pkg/sql/models" "github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect/mysql" @@ -264,12 +263,8 @@ type PrepareStmt struct { ParamTypes []byte ColDefData [][]byte IsCloudNonuser bool - IsInsertValues bool - InsertBat *batch.Batch proc *process.Process - exprList [][]colexec.ExpressionExecutor - params *vector.Vector getFromSendLongData map[int]struct{} @@ -377,17 +372,7 @@ func (prepareStmt *PrepareStmt) Close() { if prepareStmt.params != nil { prepareStmt.params.Free(prepareStmt.proc.Mp()) } - if prepareStmt.InsertBat != nil { - prepareStmt.InsertBat.Clean(prepareStmt.proc.Mp()) - prepareStmt.InsertBat = nil - } - if prepareStmt.exprList != nil { - for _, exprs := range prepareStmt.exprList { - for _, expr := range exprs { - expr.Free() - } - } - } + if prepareStmt.compile != nil { prepareStmt.compile.FreeOperator() prepareStmt.compile.SetIsPrepare(false) diff --git a/pkg/sql/colexec/evalExpression.go b/pkg/sql/colexec/evalExpression.go index ca8e844d64f3..263c95a6b8f3 100644 --- a/pkg/sql/colexec/evalExpression.go +++ b/pkg/sql/colexec/evalExpression.go @@ -136,7 +136,8 @@ func NewExpressionExecutor(proc *process.Process, planExpr *plan.Expr) (Expressi return ce, nil case *plan.Expr_P: - return NewParamExpressionExecutor(proc.Mp(), int(t.P.Pos), types.T_text.ToType()), nil + typ := types.New(types.T(planExpr.Typ.Id), planExpr.Typ.Width, planExpr.Typ.Scale) + return NewParamExpressionExecutor(proc.Mp(), int(t.P.Pos), typ), nil case *plan.Expr_V: typ := types.New(types.T(planExpr.Typ.Id), planExpr.Typ.Width, planExpr.Typ.Scale) diff --git a/pkg/sql/colexec/value_scan/types.go b/pkg/sql/colexec/value_scan/types.go index 2594241cfc0c..548f2dac5569 100644 --- a/pkg/sql/colexec/value_scan/types.go +++ b/pkg/sql/colexec/value_scan/types.go @@ -17,8 +17,8 @@ package value_scan import ( "github.com/matrixorigin/matrixone/pkg/common/reuse" "github.com/matrixorigin/matrixone/pkg/container/batch" + plan2 "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec" - "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -34,11 +34,12 @@ type ValueScan struct { // this means all the batches were saved other place. // there is no need clean them after operator done. dataInProcess bool + ColCount int + NodeType plan2.Node_NodeType - Batchs []*batch.Batch - RowsetData *plan.RowsetData - ColCount int - Uuid []byte + Batchs []*batch.Batch + RowsetData *plan2.RowsetData + ExprExecLists [][]colexec.ExpressionExecutor } type container struct { @@ -59,6 +60,10 @@ func init() { ) } +func NewArgument() *ValueScan { + return reuse.Alloc[ValueScan](nil) +} + func NewValueScanFromProcess() *ValueScan { vs := getFromReusePool() vs.dataInProcess = true @@ -92,13 +97,43 @@ func (valueScan *ValueScan) Release() { func (valueScan *ValueScan) Reset(proc *process.Process, _ bool, _ error) { valueScan.runningCtx.nowIdx = 0 - valueScan.doBatchClean(proc) + // valueScan.doBatchClean(proc) + if valueScan.Batchs != nil { + valueScan.resetBatchs() + } + for i := 0; i < valueScan.ColCount; i++ { + exprExecList := valueScan.ExprExecLists[i] + for _, expr := range exprExecList { + expr.ResetForNextQuery() + } + } valueScan.ResetProjection(proc) } func (valueScan *ValueScan) Free(proc *process.Process, _ bool, _ error) { valueScan.FreeProjection(proc) - valueScan.doBatchClean(proc) + // valueScan.doBatchClean(proc) + if valueScan.Batchs != nil { + valueScan.cleanBatchs(proc) + } + for i := range valueScan.ExprExecLists { + exprExecList := valueScan.ExprExecLists[i] + for i, expr := range exprExecList { + if expr != nil { + expr.Free() + exprExecList[i] = nil + } + } + } +} + +func (valueScan *ValueScan) cleanBatchs(proc *process.Process) { + for _, bat := range valueScan.Batchs { + if bat != nil { + bat.Clean(proc.Mp()) + } + } + valueScan.Batchs = nil } func (valueScan *ValueScan) doBatchClean(proc *process.Process) { @@ -118,6 +153,16 @@ func (valueScan *ValueScan) doBatchClean(proc *process.Process) { valueScan.Batchs = nil } +func (valueScan *ValueScan) resetBatchs() { + for _, bat := range valueScan.Batchs { + if bat != nil { + for _, vec := range bat.Vecs { + vec.CleanOnlyData() + } + } + } +} + // TypeName implement the `reuse.ReusableObject` interface. func (valueScan ValueScan) TypeName() string { return thisOperator diff --git a/pkg/sql/colexec/value_scan/value_scan.go b/pkg/sql/colexec/value_scan/value_scan.go index d1704553e749..b7933f32ba41 100644 --- a/pkg/sql/colexec/value_scan/value_scan.go +++ b/pkg/sql/colexec/value_scan/value_scan.go @@ -16,17 +16,12 @@ package value_scan import ( "bytes" - "fmt" - util2 "github.com/matrixorigin/matrixone/pkg/common/util" - "github.com/google/uuid" - "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" - plan2 "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/colexec" - "github.com/matrixorigin/matrixone/pkg/sql/util" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -35,6 +30,74 @@ func (valueScan *ValueScan) String(buf *bytes.Buffer) { buf.WriteString(thisOperator + ": value_scan") } +func evalRowsetData(proc *process.Process, rowsetExpr []*plan.RowsetExpr, vec *vector.Vector, exprExecs []colexec.ExpressionExecutor, +) error { + bats := []*batch.Batch{batch.EmptyForConstFoldBatch} + for i, expr := range exprExecs { + val, err := expr.Eval(proc, bats, nil) + if err != nil { + return err + } + if err := vec.Copy(val, int64(rowsetExpr[i].RowPos), 0, proc.Mp()); err != nil { + return err + } + } + return nil +} + +func (valueScan *ValueScan) makeValueScanBatch(proc *process.Process) (err error) { + var exprList []colexec.ExpressionExecutor + + if valueScan.RowsetData == nil { // select 1,2 + bat := batch.NewWithSize(1) + bat.Vecs[0] = vector.NewConstNull(types.T_int64.ToType(), 1, proc.Mp()) + bat.SetRowCount(1) + valueScan.Batchs = append(valueScan.Batchs, bat) + valueScan.Batchs = append(valueScan.Batchs, nil) + return nil + } + + if valueScan.ExprExecLists == nil { + if err := valueScan.InitExprExecList(proc); err != nil { + return err + } + } + + // select * from (values row(1,1), row(2,2), row(3,3)) a; + bat := valueScan.Batchs[0] + + for i := 0; i < valueScan.ColCount; i++ { + exprList = valueScan.ExprExecLists[i] + if len(exprList) == 0 { + continue + } + vec := bat.Vecs[i] + if err := evalRowsetData(proc, valueScan.RowsetData.Cols[i].Data, vec, exprList); err != nil { + return err + } + } + + return nil +} + +func (valueScan *ValueScan) InitExprExecList(proc *process.Process) error { + exprExecLists := make([][]colexec.ExpressionExecutor, len(valueScan.RowsetData.Cols)) + for i, col := range valueScan.RowsetData.Cols { + var exprExecList []colexec.ExpressionExecutor + for _, data := range col.Data { + exprExecutor, err := colexec.NewExpressionExecutor(proc, data.Expr) + if err != nil { + return err + } + exprExecList = append(exprExecList, exprExecutor) + } + exprExecLists[i] = exprExecList + } + + valueScan.ExprExecLists = exprExecLists + return nil +} + func (valueScan *ValueScan) Prepare(proc *process.Process) error { if valueScan.OpAnalyzer == nil { valueScan.OpAnalyzer = process.NewAnalyzer(valueScan.GetIdx(), valueScan.IsFirst, valueScan.IsLast, "value_scan") @@ -47,10 +110,18 @@ func (valueScan *ValueScan) Prepare(proc *process.Process) error { return err } - if valueScan.dataInProcess && len(valueScan.Batchs) == 0 { - valueScan.Batchs = append(valueScan.Batchs, nil, nil) - valueScan.Batchs[0], err = valueScan.getReadOnlyBatchFromProcess(proc) + /* if valueScan.dataInProcess && len(valueScan.Batchs) == 0 { + valueScan.Batchs = append(valueScan.Batchs, nil, nil) + valueScan.Batchs[0], err = valueScan.getReadOnlyBatchFromProcess(proc) + } + return err */ + // if valueScan.NodeType == plan2.Node_VALUE_SCAN { + err = valueScan.makeValueScanBatch(proc) + if err != nil { + return err } + // } + return err } @@ -67,12 +138,12 @@ func (valueScan *ValueScan) Call(proc *process.Process) (vm.CallResult, error) { if valueScan.runningCtx.nowIdx < len(valueScan.Batchs) { result.Batch = valueScan.Batchs[valueScan.runningCtx.nowIdx] - if valueScan.runningCtx.nowIdx > 0 { + /* if valueScan.runningCtx.nowIdx > 0 { if !valueScan.dataInProcess { valueScan.Batchs[valueScan.runningCtx.nowIdx-1].Clean(proc.GetMPool()) valueScan.Batchs[valueScan.runningCtx.nowIdx-1] = nil } - } + } */ valueScan.runningCtx.nowIdx++ } @@ -82,95 +153,3 @@ func (valueScan *ValueScan) Call(proc *process.Process) (vm.CallResult, error) { return result, err } - -func (valueScan *ValueScan) getReadOnlyBatchFromProcess(proc *process.Process) (bat *batch.Batch, err error) { - // if this is a select without source table. - // for example, select 1. - if valueScan.RowsetData == nil { - return batch.EmptyForConstFoldBatch, nil - } - - // Do Type Check. - // this is an execute sql for prepared-stmt: execute s1 and s1 is `insert into t select 1.` - // this is direct value_scan. - if bat = proc.GetPrepareBatch(); bat == nil { - if bat = proc.GetValueScanBatch(uuid.UUID(valueScan.Uuid)); bat == nil { - return nil, moerr.NewInfo(proc.Ctx, fmt.Sprintf("makeValueScanBatch failed, node id: %s", uuid.UUID(valueScan.Uuid).String())) - } - } - - // the following codes were copied from the old makeValueScanBatch. - if colsData := valueScan.RowsetData.Cols; len(colsData) > 0 { - var exprExeces []colexec.ExpressionExecutor - var strParam vector.FunctionParameterWrapper[types.Varlena] - - exprs := proc.GetPrepareExprList() - if params := proc.GetPrepareParams(); params != nil { - strParam = vector.GenerateFunctionStrParameter(params) - } - - for i := 0; i < valueScan.ColCount; i++ { - if exprs != nil { - exprExeces = exprs.([][]colexec.ExpressionExecutor)[i] - } - if strParam != nil { - for _, row := range colsData[i].Data { - if row.Pos >= 0 { - str, isNull := strParam.GetStrValue(uint64(row.Pos - 1)) - if err = util.SetBytesToAnyVector( - proc, util2.UnsafeBytesToString(str), int(row.RowPos), isNull, bat.Vecs[i]); err != nil { - return nil, err - } - } - } - } - - if err = evalRowsetData(proc, colsData[i].Data, bat.Vecs[i], exprExeces); err != nil { - return nil, err - } - } - } - - return bat, nil -} - -func evalRowsetData( - proc *process.Process, - exprs []*plan2.RowsetExpr, vec *vector.Vector, exprExecs []colexec.ExpressionExecutor, -) error { - vec.ResetArea() - bats := []*batch.Batch{batch.EmptyForConstFoldBatch} - - if len(exprExecs) > 0 { - for i, expr := range exprExecs { - val, err := expr.Eval(proc, bats, nil) - if err != nil { - return err - } - if err := vec.Copy(val, int64(exprs[i].RowPos), 0, proc.Mp()); err != nil { - return err - } - } - } else { - for _, expr := range exprs { - if expr.Pos >= 0 { - continue - } - - executor, err := colexec.NewExpressionExecutor(proc, expr.Expr) - if err != nil { - return err - } - val, err := executor.Eval(proc, bats, nil) - if err == nil { - err = vec.Copy(val, int64(expr.RowPos), 0, proc.Mp()) - } - executor.Free() - - if err != nil { - return err - } - } - } - return nil -} diff --git a/pkg/sql/compile/compile.go b/pkg/sql/compile/compile.go index f3a3743343dd..8b8af3e27320 100644 --- a/pkg/sql/compile/compile.go +++ b/pkg/sql/compile/compile.go @@ -1574,7 +1574,10 @@ func (c *Compile) compileExternScan(n *plan.Node) ([]*Scope, error) { // for resolve this bug, I do a hack at method `ColumnExpressionExecutor.Eval` with `[hack-#002]` Flag. // // build a value scan from an EmptyTable with same table structure is the correct way. - op := constructValueScan() + op, err := constructValueScan(c.proc, nil) + if err != nil { + return nil, err + } op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) ret.setRootOperator(op) c.anal.isFirst = false @@ -1787,14 +1790,11 @@ func (c *Compile) compileValueScan(n *plan.Node) ([]*Scope, error) { ds.Proc = c.proc.NewNoContextChildProc(0) currentFirstFlag := c.anal.isFirst - op := constructValueScan() - op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) - if n.RowsetData != nil { - op.RowsetData = n.RowsetData - op.ColCount = len(n.TableDef.Cols) - op.Uuid = n.Uuid + op, err := constructValueScan(c.proc, n) + if err != nil { + return nil, err } - + op.SetAnalyzeControl(c.anal.curNodeIdx, currentFirstFlag) ds.setRootOperator(op) c.anal.isFirst = false diff --git a/pkg/sql/compile/compile2.go b/pkg/sql/compile/compile2.go index 2d8dc9df4dd7..5fa9e2cfce59 100644 --- a/pkg/sql/compile/compile2.go +++ b/pkg/sql/compile/compile2.go @@ -180,9 +180,6 @@ func (c *Compile) Run(_ uint64) (queryResult *util2.RunResult, err error) { if runC != c { runC.Release() } - c.proc.CleanValueScanBatchs() - c.proc.SetPrepareBatch(nil) - c.proc.SetPrepareExprList(nil) }() // update the top context with some trace information and values. diff --git a/pkg/sql/compile/operator.go b/pkg/sql/compile/operator.go index 4be61d224520..89ebeab1fbb1 100644 --- a/pkg/sql/compile/operator.go +++ b/pkg/sql/compile/operator.go @@ -24,6 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/pb/pipeline" "github.com/matrixorigin/matrixone/pkg/pb/plan" @@ -91,6 +92,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/sql/plan/function" + "github.com/matrixorigin/matrixone/pkg/sql/plan/rule" "github.com/matrixorigin/matrixone/pkg/vm" "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/message" @@ -1999,8 +2001,46 @@ func constructTableScan(n *plan.Node) *table_scan.TableScan { return table_scan.NewArgument().WithTypes(types) } -func constructValueScan() *value_scan.ValueScan { - return value_scan.NewValueScanFromProcess() +func constructValueScan(proc *process.Process, n *plan.Node) (*value_scan.ValueScan, error) { + //return value_scan.NewValueScanFromProcess() + op := value_scan.NewArgument() + if n == nil { + return op, nil + } + op.NodeType = n.NodeType + if n.RowsetData == nil { + return op, nil + } + + op.ColCount = len(n.TableDef.Cols) + op.Batchs = make([]*batch.Batch, 2) + op.Batchs[0] = batch.NewWithSize(len(n.RowsetData.Cols)) + op.Batchs[0].SetRowCount(len(n.RowsetData.Cols[0].Data)) + rowsetData := &plan.RowsetData{ + Cols: make([]*plan.ColData, op.ColCount), + } + for i := 0; i < op.ColCount; i++ { + rowsetData.Cols[i] = new(plan.ColData) + } + + for i, col := range n.RowsetData.Cols { + vec := vector.NewVec(plan2.MakeTypeByPlan2Type(n.TableDef.Cols[i].Typ)) + op.Batchs[0].Vecs[i] = vec + for j, rowsetExpr := range col.Data { + get, err := rule.GetConstantValue2(proc, rowsetExpr.Expr, vec) + if err != nil { + op.Batchs[0].Clean(proc.Mp()) + return nil, err + } + if !get { + rowsetExpr.RowPos = int32(j) + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, rowsetExpr) + } + } + } + op.RowsetData = rowsetData + + return op, nil } func extraJoinConditions(exprs []*plan.Expr) (*plan.Expr, []*plan.Expr) { diff --git a/pkg/sql/plan/bind_insert.go b/pkg/sql/plan/bind_insert.go index 1d556b567567..7b8b9862deeb 100644 --- a/pkg/sql/plan/bind_insert.go +++ b/pkg/sql/plan/bind_insert.go @@ -21,12 +21,9 @@ import ( "github.com/google/uuid" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" - "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" - "github.com/matrixorigin/matrixone/pkg/sql/util" ) func (builder *QueryBuilder) bindInsert(stmt *tree.Insert, bindCtx *BindContext) (int32, error) { @@ -753,13 +750,10 @@ func (builder *QueryBuilder) buildValueScan( Cols: make([]*plan.ColDef, colCount), } projectList := make([]*plan.Expr, colCount) - bat := batch.NewWithSize(len(colNames)) for i, colName := range colNames { col := tableDef.Cols[tableDef.Name2ColIndex[colName]] colTyp := makeTypeByPlan2Type(col.Typ) - vec := vector.NewVec(colTyp) - bat.Vecs[i] = vec targetTyp := &plan.Expr{ Typ: col.Typ, Expr: &plan.Expr_T{ @@ -768,10 +762,6 @@ func (builder *QueryBuilder) buildValueScan( } var defExpr *plan.Expr if isAllDefault { - if err := vector.AppendMultiBytes(vec, nil, true, len(stmt.Rows), proc.Mp()); err != nil { - bat.Clean(proc.Mp()) - return 0, err - } defExpr, err := getDefaultExpr(builder.GetContext(), col) if err != nil { return 0, err @@ -781,65 +771,40 @@ func (builder *QueryBuilder) buildValueScan( return 0, err } for j := range stmt.Rows { - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - Pos: -1, - RowPos: int32(j), - Expr: defExpr, - }) + rowsetData.Cols[i].Data[j] = &plan.RowsetExpr{ + Expr: defExpr, + } } } else { binder := NewDefaultBinder(builder.GetContext(), nil, nil, col.Typ, nil) binder.builder = builder - for j, r := range stmt.Rows { + for _, r := range stmt.Rows { if nv, ok := r[i].(*tree.NumVal); ok { - canInsert, err := util.SetInsertValue(proc, nv, vec) + canInsert, expr, err := MakeInsertValueExpr(proc, nv, &colTyp) if err != nil { - bat.Clean(proc.Mp()) return 0, err } if canInsert { + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + Expr: expr, + }) continue } } - if err := vector.AppendBytes(vec, nil, true, proc.Mp()); err != nil { - bat.Clean(proc.Mp()) - return 0, err - } if _, ok := r[i].(*tree.DefaultVal); ok { defExpr, err = getDefaultExpr(builder.GetContext(), col) if err != nil { - bat.Clean(proc.Mp()) return 0, err } - } else if nv, ok := r[i].(*tree.ParamExpr); ok { - if !builder.isPrepareStatement { - bat.Clean(proc.Mp()) - return 0, moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") - } - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - RowPos: int32(j), - Pos: int32(nv.Offset), - Expr: &plan.Expr{ - Typ: constTextType, - Expr: &plan.Expr_P{ - P: &plan.ParamRef{ - Pos: int32(nv.Offset), - }, - }, - }, - }) - continue } else { defExpr, err = binder.BindExpr(r[i], 0, true) if err != nil { - bat.Clean(proc.Mp()) return 0, err } if col.Typ.Id == int32(types.T_enum) { defExpr, err = funcCastForEnumType(builder.GetContext(), defExpr, col.Typ) if err != nil { - bat.Clean(proc.Mp()) return 0, err } } @@ -848,22 +813,8 @@ func (builder *QueryBuilder) buildValueScan( if err != nil { return 0, err } - if nv, ok := r[i].(*tree.ParamExpr); ok { - if !builder.isPrepareStatement { - bat.Clean(proc.Mp()) - return 0, moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") - } - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - RowPos: int32(j), - Pos: int32(nv.Offset), - Expr: defExpr, - }) - continue - } rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - Pos: -1, - RowPos: int32(j), - Expr: defExpr, + Expr: defExpr, }) } } @@ -885,7 +836,6 @@ func (builder *QueryBuilder) buildValueScan( projectList[i] = expr } - bat.SetRowCount(len(stmt.Rows)) rowsetData.RowCount = int32(len(stmt.Rows)) nodeId, _ := uuid.NewV7() scanNode := &plan.Node{ @@ -895,11 +845,6 @@ func (builder *QueryBuilder) buildValueScan( BindingTags: []int32{lastTag}, Uuid: nodeId[:], } - if builder.isPrepareStatement { - proc.SetPrepareBatch(bat) - } else { - proc.SetValueScanBatch(nodeId, bat) - } nodeID := builder.appendNode(scanNode, bindCtx) if err = builder.addBinding(nodeID, tree.AliasClause{Alias: "_valuescan"}, bindCtx); err != nil { return 0, err diff --git a/pkg/sql/plan/build_constraint_util.go b/pkg/sql/plan/build_constraint_util.go index 7b04f9321636..31a766c2346f 100644 --- a/pkg/sql/plan/build_constraint_util.go +++ b/pkg/sql/plan/build_constraint_util.go @@ -22,14 +22,13 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/config" - "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" "github.com/matrixorigin/matrixone/pkg/sql/plan/function" "github.com/matrixorigin/matrixone/pkg/sql/util" + "github.com/matrixorigin/matrixone/pkg/vm/process" ) const ( @@ -960,6 +959,85 @@ func forceCastExpr(ctx context.Context, expr *Expr, targetType Type) (*Expr, err }, nil } +func MakeInsertValueExpr(proc *process.Process, numVal *tree.NumVal, colType *types.Type) (bool, *plan.Expr, error) { + if numVal.ValType == tree.P_null || numVal.ValType == tree.P_nulltext { + return true, makePlan2NullConstExprWithType(), nil + } + switch colType.Oid { + case types.T_bool: + canInsert, num, err := util.SetInsertValueBool(proc, numVal) + return canInsert, MakePlan2BoolConstExprWithType(num), err + + case types.T_bit: + canInsert, num, err := util.SetInsertValueBit(proc, numVal, colType) + return canInsert, MakePlan2Uint64ConstExprWithType(num), err + case types.T_int8: + canInsert, num, err := util.SetInsertValueNumber[int8](proc, numVal, colType) + return canInsert, MakePlan2Int8ConstExprWithType(num), err + case types.T_int16: + canInsert, num, err := util.SetInsertValueNumber[int16](proc, numVal, colType) + return canInsert, MakePlan2Int16ConstExprWithType(num), err + case types.T_int32: + canInsert, num, err := util.SetInsertValueNumber[int32](proc, numVal, colType) + return canInsert, MakePlan2Int32ConstExprWithType(num), err + case types.T_int64: + canInsert, num, err := util.SetInsertValueNumber[int64](proc, numVal, colType) + return canInsert, MakePlan2Int64ConstExprWithType(num), err + case types.T_uint8: + canInsert, num, err := util.SetInsertValueNumber[uint8](proc, numVal, colType) + return canInsert, MakePlan2Uint8ConstExprWithType(num), err + case types.T_uint16: + canInsert, num, err := util.SetInsertValueNumber[uint16](proc, numVal, colType) + return canInsert, MakePlan2Uint16ConstExprWithType(num), err + case types.T_uint32: + canInsert, num, err := util.SetInsertValueNumber[uint32](proc, numVal, colType) + return canInsert, MakePlan2Uint32ConstExprWithType(num), err + case types.T_uint64: + canInsert, num, err := util.SetInsertValueNumber[uint64](proc, numVal, colType) + return canInsert, MakePlan2Uint64ConstExprWithType(num), err + case types.T_float32: + canInsert, num, err := util.SetInsertValueNumber[float32](proc, numVal, colType) + return canInsert, MakePlan2Float32ConstExprWithType(num), err + case types.T_float64: + canInsert, num, err := util.SetInsertValueNumber[float64](proc, numVal, colType) + return canInsert, MakePlan2Float64ConstExprWithType(num), err + case types.T_decimal64: + canInsert, num, err := util.SetInsertValueDecimal64(proc, numVal, colType) + planType := MakePlan2TypeValue(colType) + return canInsert, MakePlan2Decimal64ExprWithType(num, &planType), err + case types.T_decimal128: + canInsert, num, err := util.SetInsertValueDecimal128(proc, numVal, colType) + planType := MakePlan2TypeValue(colType) + return canInsert, MakePlan2Decimal128ExprWithType(num, &planType), err + case types.T_char, types.T_varchar, types.T_blob, types.T_binary, types.T_varbinary, types.T_text, types.T_datalink, + types.T_array_float32, types.T_array_float64: + canInsert, num, err := util.SetInsertValueString(proc, numVal, colType) + return canInsert, MakePlan2StringConstExprWithType(string(num)), err + case types.T_json: + canInsert, num, err := util.SetInsertValueJSON(proc, numVal) + return canInsert, MakePlan2JsonConstExprWithType(string(num)), err + /* case types.T_uuid: + canInsert, num, err := setInsertValueUuid(proc, numVal) */ + case types.T_time: + canInsert, num, err := util.SetInsertValueTime(proc, numVal, colType) + return canInsert, MakePlan2TimeConstExprWithType(int64(num)), err + case types.T_date: + canInsert, num, err := util.SetInsertValueDate(proc, numVal, colType) + return canInsert, MakePlan2DateConstExprWithType(int32(num)), err + case types.T_datetime: + canInsert, num, err := util.SetInsertValueDateTime(proc, numVal, colType) + return canInsert, MakePlan2DateTimeConstExprWithType(int64(num)), err + case types.T_timestamp: + canInsert, num, err := util.SetInsertValueTimeStamp(proc, numVal, colType) + return canInsert, MakePlan2TimestampConstExprWithType(int64(num)), err + + case types.T_enum: + return false, nil, nil + } + + return false, nil, nil +} + func buildValueScan( isAllDefault bool, info *dmlSelectInfo, @@ -988,13 +1066,10 @@ func buildValueScan( Cols: make([]*plan.ColDef, colCount), } projectList := make([]*Expr, colCount) - bat := batch.NewWithSize(len(updateColumns)) for i, colName := range updateColumns { col := tableDef.Cols[colToIdx[colName]] colTyp := makeTypeByPlan2Type(col.Typ) - vec := vector.NewVec(colTyp) - bat.Vecs[i] = vec targetTyp := &plan.Expr{ Typ: col.Typ, Expr: &plan.Expr_T{ @@ -1011,70 +1086,42 @@ func buildValueScan( if err != nil { return err } + rowsetData.Cols[i].Data = make([]*plan.RowsetExpr, len(slt.Rows)) for j := range slt.Rows { - if err := vector.AppendBytes(vec, nil, true, proc.Mp()); err != nil { - bat.Clean(proc.Mp()) - return err + rowsetData.Cols[i].Data[j] = &plan.RowsetExpr{ + Expr: defExpr, } - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - Pos: -1, - RowPos: int32(j), - Expr: defExpr, - }) } } else { binder := NewDefaultBinder(builder.GetContext(), nil, nil, col.Typ, nil) binder.builder = builder - for j, r := range slt.Rows { + for _, r := range slt.Rows { if nv, ok := r[i].(*tree.NumVal); ok { - canInsert, err := util.SetInsertValue(proc, nv, vec) + canInsert, expr, err := MakeInsertValueExpr(proc, nv, &colTyp) if err != nil { - bat.Clean(proc.Mp()) return err } if canInsert { + rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ + Expr: expr, + }) continue } } - if err := vector.AppendBytes(vec, nil, true, proc.Mp()); err != nil { - bat.Clean(proc.Mp()) - return err - } if _, ok := r[i].(*tree.DefaultVal); ok { defExpr, err = getDefaultExpr(builder.GetContext(), col) if err != nil { - bat.Clean(proc.Mp()) return err } - } else if nv, ok := r[i].(*tree.ParamExpr); ok { - if !builder.isPrepareStatement { - bat.Clean(proc.Mp()) - return moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") - } - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - RowPos: int32(j), - Pos: int32(nv.Offset), - Expr: &plan.Expr{ - Typ: constTextType, - Expr: &plan.Expr_P{ - P: &plan.ParamRef{ - Pos: int32(nv.Offset), - }, - }, - }, - }) - continue } else { defExpr, err = binder.BindExpr(r[i], 0, true) if err != nil { - bat.Clean(proc.Mp()) return err } if col.Typ.Id == int32(types.T_enum) { defExpr, err = funcCastForEnumType(builder.GetContext(), defExpr, col.Typ) if err != nil { - bat.Clean(proc.Mp()) return err } } @@ -1083,22 +1130,8 @@ func buildValueScan( if err != nil { return err } - if nv, ok := r[i].(*tree.ParamExpr); ok { - if !builder.isPrepareStatement { - bat.Clean(proc.Mp()) - return moerr.NewInvalidInput(builder.GetContext(), "only prepare statement can use ? expr") - } - rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - RowPos: int32(j), - Pos: int32(nv.Offset), - Expr: defExpr, - }) - continue - } rowsetData.Cols[i].Data = append(rowsetData.Cols[i].Data, &plan.RowsetExpr{ - Pos: -1, - RowPos: int32(j), - Expr: defExpr, + Expr: defExpr, }) } } @@ -1160,7 +1193,6 @@ func buildValueScan( } } } - bat.SetRowCount(len(slt.Rows)) rowsetData.RowCount = int32(len(slt.Rows)) nodeId, _ := uuid.NewV7() scanNode := &plan.Node{ @@ -1171,11 +1203,7 @@ func buildValueScan( Uuid: nodeId[:], OnUpdateExprs: onUpdateExprs, } - if builder.isPrepareStatement { - proc.SetPrepareBatch(bat) - } else { - proc.SetValueScanBatch(nodeId, bat) - } + info.rootId = builder.appendNode(scanNode, bindCtx) if err = builder.addBinding(info.rootId, tree.AliasClause{Alias: "_valuescan"}, bindCtx); err != nil { return err diff --git a/pkg/sql/plan/build_insert.go b/pkg/sql/plan/build_insert.go index c06e1b14895e..ae76d0015c79 100644 --- a/pkg/sql/plan/build_insert.go +++ b/pkg/sql/plan/build_insert.go @@ -19,18 +19,13 @@ import ( "strings" "time" - "github.com/google/uuid" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/config" "github.com/matrixorigin/matrixone/pkg/container/batch" - "github.com/matrixorigin/matrixone/pkg/container/nulls" "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" - "github.com/matrixorigin/matrixone/pkg/sql/plan/function" - "github.com/matrixorigin/matrixone/pkg/sql/plan/rule" "github.com/matrixorigin/matrixone/pkg/sql/util" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" ) @@ -344,10 +339,11 @@ func buildInsert(stmt *tree.Insert, ctx CompilerContext, isReplace bool, isPrepa // The function returns the list of insert columns and an error, if any. func getInsertColsFromStmt(ctx context.Context, stmt *tree.Insert, tableDef *TableDef) ([]string, error) { var insertColsName []string - colToIdx := make(map[string]int) + colToIdx := make(map[string]int32) for i, col := range tableDef.Cols { - colToIdx[col.Name] = i + colToIdx[col.Name] = int32(i) } + tableDef.Name2ColIndex = colToIdx if stmt.Columns == nil { for _, col := range tableDef.Cols { if col.Name != catalog.FakePrimaryKeyColName { @@ -413,39 +409,7 @@ func canUsePkFilter(builder *QueryBuilder, ctx CompilerContext, stmt *tree.Inser } if used4UniqueIndex { - // verify that all cols that make up the unique index exist and no value is null - uSet := make(map[string]bool) - for _, n := range uniqueIndexDef.Parts { - uSet[n] = true - } - uCnt := len(uSet) - - var bat *batch.Batch - proc := ctx.GetProcess() - node := builder.qry.Nodes[0] - if node.Uuid == nil { - return false // TODO(jensenojs): issue14726 - } - - if builder.isPrepareStatement { - bat = proc.GetPrepareBatch() - } else { - bat = proc.GetValueScanBatch(uuid.UUID(node.Uuid)) - } - - for i, n := range insertColsName { - if _, ok := uSet[n]; ok { - uCnt-- - uniqueVec := bat.Vecs[i] - if nulls.Any(uniqueVec.GetNulls()) { - // has at least one values is null, then can not use pk filter, break conditon 5 - return false - } - } - } - if uCnt != 0 { - return false // at least one column that make up the unique index is NOT exist , break condtion 5 - } + return false } else { // check for auto increment primary key pkPos, pkTyp := getPkPos(tableDef, true) @@ -480,55 +444,10 @@ func canUsePkFilter(builder *QueryBuilder, ctx CompilerContext, stmt *tree.Inser } if autoIncIdx != -1 { - var bat *batch.Batch - proc := ctx.GetProcess() - node := builder.qry.Nodes[0] - if node.Uuid == nil { - return false // TODO(jensenojs): issue14726 - } - if builder.isPrepareStatement { - bat = proc.GetPrepareBatch() - } else { - bat = proc.GetValueScanBatch(uuid.UUID(node.Uuid)) - } - autoPkVec := bat.Vecs[autoIncIdx] - if nulls.Any(autoPkVec.GetNulls()) { - // has at least one values is null, then can not use pk filter, break conditon 2 - return false - } - } - } else if pkTyp.AutoIncr { // single auto incr primary key - var bat *batch.Batch - - autoIncIdx := -1 - for i, name := range insertColsName { - if tableDef.Pkey.PkeyColName == name { - autoIncIdx = i - break - } - } - - if autoIncIdx == -1 { - // have no auto pk col in values, break condition 2 return false - } else { - proc := ctx.GetProcess() - node := builder.qry.Nodes[0] - if node.Uuid == nil { - return false // TODO(jensenojs): issue14726 - } - if builder.isPrepareStatement { - bat = proc.GetPrepareBatch() - } else { - bat = proc.GetValueScanBatch(uuid.UUID(node.Uuid)) - } - - autoPkVec := bat.Vecs[autoIncIdx] - if nulls.Any(autoPkVec.GetNulls()) { - // has at least one values is null, then can not use pk filter, break conditon 2 - return false - } } + } else if pkTyp.AutoIncr { // single auto incr primary key + return false } } @@ -649,26 +568,14 @@ func newLocationMap(tableDef *TableDef, uniqueIndexDef *IndexDef) *locationMap { } func getPkValueExpr(builder *QueryBuilder, ctx CompilerContext, tableDef *TableDef, lmap *locationMap, insertColsNameFromStmt []string) (pkFilterExprs []*Expr, err error) { - var bat *batch.Batch var pkLocationInfo orderAndIdx var ok bool - var colTyp Type + var col *ColDef proc := ctx.GetProcess() node := builder.qry.Nodes[0] isCompound := len(lmap.m) > 1 forUniqueHiddenTable := lmap.isUnique - if builder.isPrepareStatement { - bat = proc.GetPrepareBatch() - } else { - bat = proc.GetValueScanBatch(uuid.UUID(node.Uuid)) - } - rowsCount := bat.RowCount() - - if rowsCount > 1 && len(bat.Vecs) > 0 && bat.Vecs[0].AllNull() { - return nil, nil - } - // insert pk col with default value, skip build pk filter expr insertColMap := make(map[string]bool) for _, name := range insertColsNameFromStmt { @@ -683,65 +590,25 @@ func getPkValueExpr(builder *QueryBuilder, ctx CompilerContext, tableDef *TableD // colExprs will store the constant value expressions (or UUID value) for each primary key column by the order in insert value SQL // that is, the key part of pkPosInValues, more info see the comment of func getPkOrderInValues colExprs := make([][]*Expr, len(lmap.m)) + rowsCount := len(node.RowsetData.Cols[0].Data) // If the expression is nil, it creates a constant expression with either the UUID value or a constant value. for idx, name := range insertColsNameFromStmt { - var varcharTyp Type if pkLocationInfo, ok = lmap.m[name]; !ok { continue } + col = tableDef.Cols[tableDef.Name2ColIndex[name]] valExprs := make([]*Expr, rowsCount) - rowTyp := bat.Vecs[idx].GetType() - colTyp = makePlan2Type(rowTyp) - - if rowTyp.Oid == types.T_uuid { - typ := types.T_varchar.ToType() - varcharTyp = MakePlan2Type(&typ) - } - for _, data := range node.RowsetData.Cols[idx].Data { + for i, data := range node.RowsetData.Cols[idx].Data { rowExpr := DeepCopyExpr(data.Expr) - e, err := forceCastExpr(builder.GetContext(), rowExpr, colTyp) + e, err := forceCastExpr(builder.GetContext(), rowExpr, col.Typ) if err != nil { return nil, err } - valExprs[data.RowPos] = e - } - - for i := 0; i < rowsCount; i++ { - if valExprs[i] == nil { - // handles UUID types specifically by creating a VARCHAR type and casting the UUID to a string. - if bat.Vecs[idx].GetType().Oid == types.T_uuid { - // we have not uuid type in plan.Const. so use string & cast string to uuid - val := vector.MustFixedColWithTypeCheck[types.Uuid](bat.Vecs[idx])[i] - constExpr := &plan.Expr{ - Typ: varcharTyp, - Expr: &plan.Expr_Lit{ - Lit: &plan.Literal{ - Value: &plan.Literal_Sval{ - Sval: val.String(), - }, - }, - }, - } - valExprs[i], err = appendCastBeforeExpr(proc.Ctx, constExpr, colTyp, false) - if err != nil { - return nil, err - } - } else { - constExpr := rule.GetConstantValue(bat.Vecs[idx], true, uint64(i)) - if constExpr == nil { - return nil, err - } - valExprs[i] = &plan.Expr{ - Typ: colTyp, - Expr: &plan.Expr_Lit{ - Lit: constExpr, - }, - } - } - } + valExprs[i] = e } + colExprs[pkLocationInfo.order] = valExprs } @@ -758,7 +625,7 @@ func getPkValueExpr(builder *QueryBuilder, ctx CompilerContext, tableDef *TableD } pkExpr := &plan.Expr{ - Typ: colTyp, + Typ: col.Typ, Expr: &plan.Expr_Col{ Col: &ColRef{ ColPos: 0, @@ -821,7 +688,7 @@ func getPkValueExpr(builder *QueryBuilder, ctx CompilerContext, tableDef *TableD pkExpr, serialExpr, }) - } else if rowsCount <= 3 { + } else { inArgs := make([]*plan.Expr, rowsCount) for i := range inArgs { serialArgs := make([]*plan.Expr, len(colExprs)) @@ -842,41 +709,6 @@ func getPkValueExpr(builder *QueryBuilder, ctx CompilerContext, tableDef *TableD }, }, }) - } else { - names := make([]string, len(lmap.m)) - for n, p := range lmap.m { - names[p.order] = n - } - bat.Attrs = insertColsNameFromStmt - toSerialBatch := bat.GetSubBatch(names) - // serialize - // __cpkey__ in (serial(a1,b1,c1,d1),serial(a2,b2,c2,d2),xxx) - // processing composite primary key - vec, err := function.RunFunctionDirectly(proc, function.SerialFunctionEncodeID, - toSerialBatch.Vecs, - toSerialBatch.RowCount()) - if err != nil { - return nil, err - } - vecLength := vec.Length() - vec.InplaceSortAndCompact() - data, err := vec.MarshalBinary() - vec.Free(proc.Mp()) - if err != nil { - return nil, err - } - filterExpr, _ = BindFuncExprImplByPlanExpr(builder.GetContext(), "in", []*Expr{ - pkExpr, - { - Typ: pkExpr.Typ, - Expr: &plan.Expr_Vec{ - Vec: &plan.LiteralVec{ - Len: int32(vecLength), - Data: data, - }, - }, - }, - }) } } diff --git a/pkg/sql/plan/deepcopy.go b/pkg/sql/plan/deepcopy.go index 324ad1dd195c..f53355815fc9 100644 --- a/pkg/sql/plan/deepcopy.go +++ b/pkg/sql/plan/deepcopy.go @@ -549,9 +549,7 @@ func DeepCopyColData(col *plan.ColData) *plan.ColData { } for i, e := range col.Data { newCol.Data[i] = &plan.RowsetExpr{ - Pos: e.Pos, - RowPos: e.RowPos, - Expr: DeepCopyExpr(e.Expr), + Expr: DeepCopyExpr(e.Expr), } } diff --git a/pkg/sql/plan/make.go b/pkg/sql/plan/make.go index e457067bf8df..6199722b1473 100644 --- a/pkg/sql/plan/make.go +++ b/pkg/sql/plan/make.go @@ -150,6 +150,33 @@ func makePlan2BoolConstExprWithType(v bool) *plan.Expr { } } +func makePlan2Int8ConstExpr(v int8) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_I8Val{ + I8Val: int32(v), + }, + }} +} + +func makePlan2Int16ConstExpr(v int16) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_I16Val{ + I16Val: int32(v), + }, + }} +} + +func makePlan2Int32ConstExpr(v int32) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_I32Val{ + I32Val: v, + }, + }} +} + func makePlan2Int64ConstExpr(v int64) *plan.Expr_Lit { return &plan.Expr_Lit{Lit: &plan.Literal{ Isnull: false, @@ -159,9 +186,127 @@ func makePlan2Int64ConstExpr(v int64) *plan.Expr_Lit { }} } +func makePlan2TimeConstExpr(v int64) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Timeval{ + Timeval: v, + }, + }} +} + +func makePlan2DateConstExpr(v int32) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Dateval{ + Dateval: v, + }, + }} +} + +func makePlan2DateTimeConstExpr(v int64) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Datetimeval{ + Datetimeval: v, + }, + }} +} + +func makePlan2TimestampConstExpr(v int64) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Timestampval{ + Timestampval: v, + }, + }} +} + +var MakePlan2BoolConstExprWithType = makePlan2BoolConstExprWithType +var MakePlan2Int8ConstExprWithType = makePlan2Int8ConstExprWithType +var MakePlan2Int16ConstExprWithType = makePlan2Int16ConstExprWithType +var MakePlan2Int32ConstExprWithType = makePlan2Int32ConstExprWithType var MakePlan2Int64ConstExprWithType = makePlan2Int64ConstExprWithType +var MakePlan2Uint8ConstExprWithType = makePlan2Uint8ConstExprWithType +var MakePlan2Uint16ConstExprWithType = makePlan2Uint16ConstExprWithType +var MakePlan2Uint32ConstExprWithType = makePlan2Uint32ConstExprWithType var MakePlan2Uint64ConstExprWithType = makePlan2Uint64ConstExprWithType +var MakePlan2TimeConstExprWithType = makePlan2TimeConstExprWithType +var MakePlan2DateConstExprWithType = makePlan2DateConstExprWithType +var MakePlan2DateTimeConstExprWithType = makePlan2DateTimeConstExprWithType +var MakePlan2TimestampConstExprWithType = makePlan2TimestampConstExprWithType + +func makePlan2Int8ConstExprWithType(v int8) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Int8ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_int8), + NotNullable: true, + }, + } +} + +func makePlan2Int16ConstExprWithType(v int16) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Int16ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_int16), + NotNullable: true, + }, + } +} + +func makePlan2Int32ConstExprWithType(v int32) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Int32ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_int32), + NotNullable: true, + }, + } +} + +func makePlan2TimeConstExprWithType(v int64) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2TimeConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_time), + NotNullable: true, + }, + } +} + +func makePlan2DateConstExprWithType(v int32) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2DateConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_date), + NotNullable: true, + }, + } +} + +func makePlan2DateTimeConstExprWithType(v int64) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2DateTimeConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_datetime), + NotNullable: true, + }, + } +} + +func makePlan2TimestampConstExprWithType(v int64) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2TimestampConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_timestamp), + NotNullable: true, + }, + } +} + func makePlan2Int64ConstExprWithType(v int64) *plan.Expr { return &plan.Expr{ Expr: makePlan2Int64ConstExpr(v), @@ -236,6 +381,33 @@ func makePlan2Int64VecExprWithType(mp *mpool.MPool, vals ...int64) *plan.Expr { } } +func makePlan2Uint8ConstExpr(v uint8) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_U8Val{ + U8Val: uint32(v), + }, + }} +} + +func makePlan2Uint16ConstExpr(v uint16) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_U16Val{ + U16Val: uint32(v), + }, + }} +} + +func makePlan2Uint32ConstExpr(v uint32) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_U32Val{ + U32Val: v, + }, + }} +} + func makePlan2Uint64ConstExpr(v uint64) *plan.Expr_Lit { return &plan.Expr_Lit{Lit: &plan.Literal{ Isnull: false, @@ -255,6 +427,45 @@ func makePlan2Uint64ConstExprWithType(v uint64) *plan.Expr { } } +func makePlan2Uint8ConstExprWithType(v uint8) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Uint8ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_uint8), + NotNullable: true, + }, + } +} + +func makePlan2Uint16ConstExprWithType(v uint16) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Uint16ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_uint16), + NotNullable: true, + }, + } +} + +func makePlan2Uint32ConstExprWithType(v uint32) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Uint32ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_uint32), + NotNullable: true, + }, + } +} + +func makePlan2Float32ConstExpr(v float32) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Fval{ + Fval: v, + }, + }} +} + func makePlan2Float64ConstExpr(v float64) *plan.Expr_Lit { return &plan.Expr_Lit{Lit: &plan.Literal{ Isnull: false, @@ -264,6 +475,7 @@ func makePlan2Float64ConstExpr(v float64) *plan.Expr_Lit { }} } +var MakePlan2Float32ConstExprWithType = makePlan2Float32ConstExprWithType var MakePlan2Float64ConstExprWithType = makePlan2Float64ConstExprWithType func makePlan2Float64ConstExprWithType(v float64) *plan.Expr { @@ -276,6 +488,25 @@ func makePlan2Float64ConstExprWithType(v float64) *plan.Expr { } } +func makePlan2Float32ConstExprWithType(v float32) *plan.Expr { + return &plan.Expr{ + Expr: makePlan2Float32ConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_float32), + NotNullable: true, + }, + } +} + +func makePlan2JsonConstExpr(v string) *plan.Expr_Lit { + return &plan.Expr_Lit{Lit: &plan.Literal{ + Isnull: false, + Value: &plan.Literal_Jsonval{ + Jsonval: v, + }, + }} +} + func makePlan2StringConstExpr(v string, isBin ...bool) *plan.Expr_Lit { c := &plan.Expr_Lit{Lit: &plan.Literal{ Isnull: false, @@ -290,6 +521,19 @@ func makePlan2StringConstExpr(v string, isBin ...bool) *plan.Expr_Lit { } var MakePlan2StringConstExprWithType = makePlan2StringConstExprWithType +var MakePlan2JsonConstExprWithType = makePlan2JsonConstExprWithType + +func makePlan2JsonConstExprWithType(v string) *plan.Expr { + width := int32(utf8.RuneCountInString(v)) + return &plan.Expr{ + Expr: makePlan2JsonConstExpr(v), + Typ: plan.Type{ + Id: int32(types.T_json), + NotNullable: true, + Width: width, + }, + } +} func makePlan2StringConstExprWithType(v string, isBin ...bool) *plan.Expr { width := int32(utf8.RuneCountInString(v)) @@ -435,6 +679,7 @@ func makePlan2TypeValue(typ *types.Type) plan.Type { } var MakeTypeByPlan2Type = makeTypeByPlan2Type +var MakePlan2TypeValue = makePlan2TypeValue func makeTypeByPlan2Type(typ plan.Type) types.Type { oid := types.T(typ.Id) diff --git a/pkg/sql/plan/query_builder.go b/pkg/sql/plan/query_builder.go index d33aa3472b86..0a18b440255e 100644 --- a/pkg/sql/plan/query_builder.go +++ b/pkg/sql/plan/query_builder.go @@ -26,9 +26,7 @@ import ( "github.com/google/uuid" "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" - "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" - "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" @@ -2560,12 +2558,10 @@ func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isR var timeWindowGroup *plan.Expr if clause == nil { - proc := builder.compCtx.GetProcess() rowCount := len(valuesClause.Rows) if len(valuesClause.Rows) == 0 { return 0, moerr.NewInternalError(builder.GetContext(), "values statement have not rows") } - bat := batch.NewWithSize(len(valuesClause.Rows[0])) strTyp := plan.Type{ Id: int32(types.T_text), NotNullable: false, @@ -2595,14 +2591,8 @@ func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isR } ctx.binder = NewWhereBinder(builder, ctx) for i := 0; i < colCount; i++ { - vec := vector.NewVec(types.T_text.ToType()) - bat.Vecs[i] = vec rowSetData.Cols[i] = &plan.ColData{} for j := 0; j < rowCount; j++ { - if err := vector.AppendBytes(vec, nil, true, proc.Mp()); err != nil { - bat.Clean(proc.Mp()) - return 0, err - } planExpr, err := ctx.binder.BindExpr(valuesClause.Rows[j][i], 0, true) if err != nil { return 0, err @@ -2612,9 +2602,7 @@ func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isR return 0, err } rowSetData.Cols[i].Data = append(rowSetData.Cols[i].Data, &plan.RowsetExpr{ - RowPos: int32(j), - Expr: planExpr, - Pos: -1, + Expr: planExpr, }) } @@ -2630,7 +2618,6 @@ func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isR Typ: strTyp, } } - bat.SetRowCount(rowCount) nodeUUID, _ := uuid.NewV7() nodeID = builder.appendNode(&plan.Node{ NodeType: plan.Node_VALUE_SCAN, @@ -2640,11 +2627,6 @@ func (builder *QueryBuilder) bindSelect(stmt *tree.Select, ctx *BindContext, isR Uuid: nodeUUID[:], NotCacheable: true, }, ctx) - if builder.isPrepareStatement { - proc.SetPrepareBatch(bat) - } else { - proc.SetValueScanBatch(nodeUUID, bat) - } if err = builder.addBinding(nodeID, tree.AliasClause{Alias: "_valuescan"}, ctx); err != nil { return 0, err diff --git a/pkg/sql/plan/rule/constant_fold.go b/pkg/sql/plan/rule/constant_fold.go index 5efe2a20b3a9..fdf00b59a33e 100644 --- a/pkg/sql/plan/rule/constant_fold.go +++ b/pkg/sql/plan/rule/constant_fold.go @@ -412,6 +412,203 @@ func GetConstantValue(vec *vector.Vector, transAll bool, row uint64) *plan.Liter } } +func GetConstantValue2(proc *process.Process, expr *plan.Expr, vec *vector.Vector) (get bool, err error) { + if cExpr, ok := expr.Expr.(*plan.Expr_Lit); ok { + if cExpr.Lit.Isnull { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return true, err + } + switch vec.GetType().Oid { + case types.T_bool: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Bval); ok { + val := val.Bval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_bit: + if val, ok := cExpr.Lit.Value.(*plan.Literal_U64Val); ok { + val := val.U64Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_int8: + if val, ok := cExpr.Lit.Value.(*plan.Literal_I8Val); ok { + val := int8(val.I8Val) + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_int16: + if val, ok := cExpr.Lit.Value.(*plan.Literal_I16Val); ok { + val := int16(val.I16Val) + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_int32: + if val, ok := cExpr.Lit.Value.(*plan.Literal_I32Val); ok { + val := val.I32Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_int64: + if val, ok := cExpr.Lit.Value.(*plan.Literal_I64Val); ok { + val := val.I64Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_uint8: + if val, ok := cExpr.Lit.Value.(*plan.Literal_U8Val); ok { + val := uint8(val.U8Val) + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_uint16: + if val, ok := cExpr.Lit.Value.(*plan.Literal_U16Val); ok { + val := uint16(val.U16Val) + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_uint32: + if val, ok := cExpr.Lit.Value.(*plan.Literal_U32Val); ok { + val := val.U32Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_uint64: + if val, ok := cExpr.Lit.Value.(*plan.Literal_U64Val); ok { + val := val.U64Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_float32: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Fval); ok { + val := val.Fval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_float64: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Dval); ok { + val := val.Dval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_varchar, types.T_char, types.T_binary, types.T_varbinary, types.T_text, + types.T_blob, types.T_datalink, types.T_json: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Sval); ok { + val := val.Sval + err = vector.AppendBytes(vec, []byte(val), false, proc.Mp()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, false, proc.Mp()) + return false, err + } + case types.T_timestamp: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Timestampval); ok { + val := val.Timestampval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_date: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Dval); ok { + val := val.Dval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_time: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Dval); ok { + val := val.Dval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_datetime: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Datetimeval); ok { + val := val.Datetimeval + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return true, err + } + case types.T_enum: + if val, ok := cExpr.Lit.Value.(*plan.Literal_EnumVal); ok { + val := val.EnumVal + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_decimal64: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Decimal64Val); ok { + val := val.Decimal64Val.A + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + case types.T_decimal128: + if val, ok := cExpr.Lit.Value.(*plan.Literal_Decimal128Val); ok { + val := val.Decimal128Val + err = vector.AppendFixed(vec, val, false, proc.GetMPool()) + return true, err + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } + default: + return false, nil + } + } else { + err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return false, err + } +} + func IsConstant(e *plan.Expr, varAndParamIsConst bool) bool { switch ef := e.Expr.(type) { case *plan.Expr_Lit, *plan.Expr_T, *plan.Expr_Vec: diff --git a/pkg/sql/plan/types.go b/pkg/sql/plan/types.go index 1a637622df9b..257af5c11765 100644 --- a/pkg/sql/plan/types.go +++ b/pkg/sql/plan/types.go @@ -181,9 +181,10 @@ type QueryBuilder struct { isForUpdate bool // if it's a query plan for update isRestore bool isSkipResolveTableDef bool + skipStats bool + + deleteNode map[uint64]int32 //delete node in this query. key is tableId, value is the nodeId of sinkScan node in the delete plan - deleteNode map[uint64]int32 //delete node in this query. key is tableId, value is the nodeId of sinkScan node in the delete plan - skipStats bool optimizerHints *OptimizerHints } @@ -233,9 +234,14 @@ type BindContext struct { recSelect bool finalSelect bool unionSelect bool - recRecursiveScanNodeId int32 isTryBindingCTE bool sliding bool + isDistinct bool + isCorrelated bool + hasSingleRow bool + forceWindows bool + isGroupingSet bool + recRecursiveScanNodeId int32 cteName string headings []string @@ -275,18 +281,12 @@ type BindContext struct { // for join tables bindingTree *BindingTreeNode - isDistinct bool - isCorrelated bool - hasSingleRow bool - parent *BindContext leftChild *BindContext rightChild *BindContext defaultDatabase string - forceWindows bool - // sample function related. sampleFunc SampleFuncCtx @@ -299,8 +299,7 @@ type BindContext struct { // lower is sys var lower_case_table_names lower int64 - isGroupingSet bool - groupingFlag []bool + groupingFlag []bool } type NameTuple struct { diff --git a/pkg/sql/util/eval_expr_util.go b/pkg/sql/util/eval_expr_util.go index 9755edd242c3..b11996d0a59b 100644 --- a/pkg/sql/util/eval_expr_util.go +++ b/pkg/sql/util/eval_expr_util.go @@ -112,16 +112,14 @@ func SetAnyToStringVector(proc *process.Process, val any, vec *vector.Vector, id } } -func SetBytesToAnyVector(proc *process.Process, val string, row int, - isNull bool, vec *vector.Vector) error { +func SetBytesToAnyVector(ctx context.Context, val string, row int, + isNull bool, vec *vector.Vector, proc *process.Process) error { if isNull { vec.GetNulls().Set(uint64(row)) return nil } else { vec.GetNulls().Unset(uint64(row)) } - - ctx := proc.Ctx switch vec.GetType().Oid { case types.T_bool: v, err := types.ParseBool(val) @@ -159,7 +157,7 @@ func SetBytesToAnyVector(proc *process.Process, val string, row int, if err != nil { return moerr.NewOutOfRangef(ctx, "int64", "value '%v'", val) } - return vector.SetFixedAtNoTypeCheck(vec, row, v) + return vector.SetFixedAtNoTypeCheck(vec, row, int64(v)) case types.T_uint8: v, err := strconv.ParseUint(val, 0, 8) if err != nil { @@ -183,7 +181,7 @@ func SetBytesToAnyVector(proc *process.Process, val string, row int, if err != nil { return moerr.NewOutOfRangef(ctx, "uint64", "value '%v'", val) } - return vector.SetFixedAtNoTypeCheck(vec, row, v) + return vector.SetFixedAtNoTypeCheck(vec, row, uint64(v)) case types.T_float32: v, err := strconv.ParseFloat(val, 32) if err != nil { @@ -195,7 +193,7 @@ func SetBytesToAnyVector(proc *process.Process, val string, row int, if err != nil { return moerr.NewOutOfRangef(ctx, "float64", "value '%v'", val) } - return vector.SetFixedAtNoTypeCheck(vec, row, v) + return vector.SetFixedAtNoTypeCheck(vec, row, float64(v)) case types.T_decimal64: v, err := types.ParseDecimal64(val, vec.GetType().Width, vec.GetType().Scale) if err != nil { @@ -263,86 +261,40 @@ func SetBytesToAnyVector(proc *process.Process, val string, row int, } } -func SetInsertValue(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (bool, error) { - switch vec.GetType().Oid { - case types.T_bool: - return setInsertValueBool(proc, numVal, vec) - case types.T_bit: - return setInsertValueBit(proc, numVal, vec) - case types.T_int8: - return setInsertValueNumber[int8](proc, numVal, vec) - case types.T_int16: - return setInsertValueNumber[int16](proc, numVal, vec) - case types.T_int32: - return setInsertValueNumber[int32](proc, numVal, vec) - case types.T_int64: - return setInsertValueNumber[int64](proc, numVal, vec) - case types.T_uint8: - return setInsertValueNumber[uint8](proc, numVal, vec) - case types.T_uint16: - return setInsertValueNumber[uint16](proc, numVal, vec) - case types.T_uint32: - return setInsertValueNumber[uint32](proc, numVal, vec) - case types.T_uint64: - return setInsertValueNumber[uint64](proc, numVal, vec) - case types.T_float32: - return setInsertValueNumber[float32](proc, numVal, vec) - case types.T_float64: - return setInsertValueNumber[float64](proc, numVal, vec) - case types.T_decimal64: - return setInsertValueDecimal64(proc, numVal, vec) - case types.T_decimal128: - return setInsertValueDecimal128(proc, numVal, vec) - case types.T_char, types.T_varchar, types.T_blob, types.T_binary, types.T_varbinary, types.T_text, types.T_datalink, - types.T_array_float32, types.T_array_float64: - return setInsertValueString(proc, numVal, vec) - case types.T_json: - return setInsertValueJSON(proc, numVal, vec) - case types.T_uuid: - return setInsertValueUuid(proc, numVal, vec) - case types.T_time: - return setInsertValueTime(proc, numVal, vec) - case types.T_date: - return setInsertValueDate(proc, numVal, vec) - case types.T_datetime: - return setInsertValueDateTime(proc, numVal, vec) - case types.T_timestamp: - return setInsertValueTimeStamp(proc, numVal, vec) - case types.T_enum: - return false, nil - } - - return false, nil -} - -func setInsertValueTimeStamp(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueTimeStamp(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Timestamp, err error) { canInsert = true - appendIntegerTimeStamp := func(val int64) error { + appendIntegerTimeStamp := func(val int64) (types.Timestamp, error) { if val < 0 || uint64(val) > 32536771199 { - return vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) + return 0, nil } result := types.UnixToTimestamp(val) - return vector.AppendFixed(vec, result, false, proc.Mp()) + return result, nil } switch numVal.ValType { case tree.P_null: - err = vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) + return true, 0, nil case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + } + if res, err = appendIntegerTimeStamp(val); err != nil { + return false, res, err } - err = appendIntegerTimeStamp(val) + return true, res, err case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = appendIntegerTimeStamp(int64(val)) + if res, err = appendIntegerTimeStamp(int64(val)); err != nil { + return false, res, err + } + return true, res, err case tree.P_decimal: canInsert = false @@ -353,50 +305,50 @@ func setInsertValueTimeStamp(proc *process.Process, numVal *tree.NumVal, vec *ve case tree.P_hexnum: var val uint64 if val, err = HexToInt(numVal.String()); err != nil { - return false, err + return false, res, err + } + if res, err = appendIntegerTimeStamp(int64(val)); err != nil { + return false, res, err } - err = appendIntegerTimeStamp(int64(val)) + return true, res, err case tree.P_char: s := numVal.String() if len(s) == 0 { - err = vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) + return true, 0, err } else { - typ := vec.GetType() - var val types.Timestamp zone := time.Local if proc != nil { zone = proc.GetSessionInfo().TimeZone } - val, err = types.ParseTimestamp(zone, s, typ.Scale) + res, err = types.ParseTimestamp(zone, s, typ.Scale) if err != nil { - return + return false, res, err } - err = vector.AppendFixed(vec, val, false, proc.Mp()) + return true, res, err } case tree.P_bit: var val uint64 if val, err = BinaryToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendIntegerTimeStamp(int64(val)) + if res, err = appendIntegerTimeStamp(int64(val)); err != nil { + return false, res, err + } + return true, res, err case tree.P_nulltext: - err = vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) + return true, 0, nil default: canInsert = false } return } -func setInsertValueDateTime(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueDateTime(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Datetime, err error) { canInsert = true - switch numVal.ValType { - case tree.P_null: - err = vector.AppendFixed[types.Datetime](vec, 0, true, proc.GetMPool()) - case tree.P_int64: canInsert = false @@ -414,16 +366,11 @@ func setInsertValueDateTime(proc *process.Process, numVal *tree.NumVal, vec *vec case tree.P_char: s := numVal.String() - if len(s) == 0 { - err = vector.AppendFixed[types.Datetime](vec, 0, true, proc.GetMPool()) - } else { - typ := vec.GetType() - var val types.Datetime - val, err = types.ParseDatetime(s, typ.Scale) + if len(s) > 0 { + res, err = types.ParseDatetime(s, typ.Scale) if err != nil { - return + return false, res, err } - err = vector.AppendFixed(vec, val, false, proc.Mp()) } case tree.P_bool: @@ -432,46 +379,39 @@ func setInsertValueDateTime(proc *process.Process, numVal *tree.NumVal, vec *vec canInsert = false case tree.P_bit: canInsert = false - case tree.P_nulltext: - err = vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) default: canInsert = false } return } -func setInsertValueTime(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueTime(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Time, err error) { canInsert = true - appendIntegerTime := func(val int64) error { - typ := vec.GetType() + appendIntegerTime := func(val int64) (types.Time, error) { + var res types.Time if val < types.MinInputIntTime || val > types.MaxInputIntTime { - return moerr.NewOutOfRangef(proc.Ctx, "time", "value %d", val) + return res, moerr.NewOutOfRangef(proc.Ctx, "time", "value %d", val) } - result, err := types.ParseInt64ToTime(val, typ.Scale) - if err != nil { - return err - } - return vector.AppendFixed(vec, result, false, proc.Mp()) + return types.ParseInt64ToTime(val, typ.Scale) } switch numVal.ValType { - case tree.P_null: - err = vector.AppendFixed[types.Time](vec, 0, true, proc.GetMPool()) - case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) } - err = appendIntegerTime(val) + res, err = appendIntegerTime(val) + return true, res, err case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = appendIntegerTime(int64(val)) + res, err = appendIntegerTime(int64(val)) + return true, res, err case tree.P_decimal: canInsert = false @@ -482,33 +422,33 @@ func setInsertValueTime(proc *process.Process, numVal *tree.NumVal, vec *vector. case tree.P_hexnum: var val uint64 if val, err = HexToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendIntegerTime(int64(val)) + res, err = appendIntegerTime(int64(val)) + return true, res, err case tree.P_char: s := numVal.String() if len(s) == 0 { - err = vector.AppendFixed[types.Time](vec, 0, true, proc.GetMPool()) + res = 0 } else { - typ := vec.GetType() - var val types.Time - val, err = types.ParseTime(s, typ.Scale) + res, err = types.ParseTime(s, typ.Scale) if err != nil { - return + return false, res, err } - err = vector.AppendFixed(vec, val, false, proc.Mp()) } + return true, res, err case tree.P_bit: var val uint64 if val, err = BinaryToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendIntegerTime(int64(val)) + res, err = appendIntegerTime(int64(val)) + return true, res, err case tree.P_nulltext: - err = vector.AppendFixed[types.Time](vec, 0, true, proc.GetMPool()) + return true, 0, nil default: canInsert = false @@ -516,13 +456,10 @@ func setInsertValueTime(proc *process.Process, numVal *tree.NumVal, vec *vector. return } -func setInsertValueDate(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueDate(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Date, err error) { canInsert = true switch numVal.ValType { - case tree.P_null: - err = vector.AppendFixed[types.Date](vec, 0, true, proc.GetMPool()) - case tree.P_int64: canInsert = false @@ -540,15 +477,11 @@ func setInsertValueDate(proc *process.Process, numVal *tree.NumVal, vec *vector. case tree.P_char: s := numVal.String() - var val types.Date - if len(s) == 0 { - err = vector.AppendFixed[types.Date](vec, 0, true, proc.GetMPool()) - } else { - val, err = types.ParseDateCast(s) + if len(s) > 0 { + res, err = types.ParseDateCast(s) if err != nil { - return + return false, res, err } - err = vector.AppendFixed(vec, val, false, proc.Mp()) } case tree.P_bool: @@ -557,82 +490,28 @@ func setInsertValueDate(proc *process.Process, numVal *tree.NumVal, vec *vector. canInsert = false case tree.P_bit: canInsert = false - case tree.P_nulltext: - err = vector.AppendFixed[types.Time](vec, 0, true, proc.GetMPool()) default: canInsert = false } return } -func setInsertValueUuid(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueBool(proc *process.Process, numVal *tree.NumVal) (canInsert bool, num bool, err error) { canInsert = true - switch numVal.ValType { - case tree.P_null: - err = vector.AppendFixed[types.Uuid](vec, types.Uuid{}, true, proc.GetMPool()) - - case tree.P_int64: - canInsert = false - - case tree.P_uint64: - canInsert = false - - case tree.P_decimal: - canInsert = false - - case tree.P_float64: - canInsert = false - - case tree.P_hexnum: - canInsert = false - - case tree.P_char: - s := numVal.String() - var val types.Uuid - val, err = types.ParseUuid(s) - if err != nil { - return - } - err = vector.AppendFixed(vec, val, false, proc.Mp()) - - case tree.P_bool: - canInsert = false - case tree.P_ScoreBinary: - canInsert = false - case tree.P_bit: - canInsert = false - case tree.P_nulltext: - err = vector.AppendFixed[types.Timestamp](vec, 0, true, proc.GetMPool()) - default: - canInsert = false - } - return -} - -func setInsertValueBool(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { - canInsert = true - switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - - case tree.P_bool: - val := numVal.Bool() - err = vector.AppendFixed[bool](vec, val, false, proc.Mp()) - case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) } - err = vector.AppendFixed[bool](vec, val == 1, false, proc.Mp()) + num = val == 1 case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = vector.AppendFixed[bool](vec, val == 1, false, proc.Mp()) + num = val == 1 case tree.P_decimal: canInsert = false @@ -647,24 +526,22 @@ func setInsertValueBool(proc *process.Process, numVal *tree.NumVal, vec *vector. case tree.P_char: originStr := numVal.String() if len(originStr) == 4 && strings.ToLower(originStr) == "true" { - err = vector.AppendFixed[bool](vec, true, false, proc.Mp()) + num = true } else { - err = vector.AppendFixed[bool](vec, false, false, proc.Mp()) + num = false } case tree.P_nulltext: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) default: canInsert = false } return } -func setInsertValueString(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueString(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, val []byte, err error) { canInsert = true checkStrLen := func(s string) ([]byte, error) { - typ := vec.GetType() destLen := int(typ.Width) if typ.Oid != types.T_text && typ.Oid != types.T_datalink && typ.Oid != types.T_binary && destLen != 0 && !typ.Oid.IsArrayRelate() { if utf8.RuneCountInString(s) > destLen { @@ -725,9 +602,6 @@ func setInsertValueString(proc *process.Process, numVal *tree.NumVal, vec *vecto } switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - case tree.P_bool: var s string if numVal.Bool() { @@ -735,40 +609,31 @@ func setInsertValueString(proc *process.Process, numVal *tree.NumVal, vec *vecto } else { s = "0" } - var val []byte - val, err = checkStrLen(s) - if err != nil { - return + if val, err = checkStrLen(s); err != nil { + canInsert = false } - err = vector.AppendBytes(vec, val, false, proc.Mp()) + return case tree.P_int64, tree.P_uint64, tree.P_char, tree.P_decimal, tree.P_float64: s := numVal.String() - var val []byte - val, err = checkStrLen(s) - if err != nil { - return + if val, err = checkStrLen(s); err != nil { + canInsert = false } - err = vector.AppendBytes(vec, val, false, proc.Mp()) + return case tree.P_hexnum: s := numVal.String()[2:] - var val []byte if val, err = hex.DecodeString(s); err != nil { - return + canInsert = false } - err = vector.AppendBytes(vec, val, false, proc.Mp()) + return case tree.P_bit: s := numVal.String()[2:] - var val []byte if val, err = DecodeBinaryString(s); err != nil { - return + canInsert = false } - err = vector.AppendBytes(vec, val, false, proc.Mp()) - - case tree.P_nulltext: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) + return default: canInsert = false @@ -776,24 +641,20 @@ func setInsertValueString(proc *process.Process, numVal *tree.NumVal, vec *vecto return } -func setInsertValueJSON(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueJSON(proc *process.Process, numVal *tree.NumVal) (canInsert bool, val []byte, err error) { canInsert = true switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) default: var json bytejson.ByteJson originStr := numVal.String() json, err = types.ParseStringToByteJson(originStr) if err != nil { - return false, err + return false, nil, err } - var val []byte val, err = types.EncodeJson(json) if err != nil { - return false, err + return false, nil, err } - err = vector.AppendBytes(vec, val, false, proc.Mp()) } return } @@ -817,86 +678,76 @@ func checkOverFlow[T1, T2 constraints.Integer | constraints.Float](ctx context.C return nil } -func setInsertValueNumber[T constraints.Integer | constraints.Float](proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueNumber[T constraints.Integer | constraints.Float](proc *process.Process, numVal *tree.NumVal, colType *types.Type) (canInsert bool, num T, err error) { canInsert = true + var n nulls.Nulls switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - case tree.P_bool: val := numVal.Bool() if val { - err = vector.AppendFixed(vec, T(1), false, proc.Mp()) + num = T(1) } else { - err = vector.AppendFixed(vec, T(0), false, proc.Mp()) + num = T(0) } - vec.GetType() case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, num, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) } - err = checkOverFlow[int64, T](proc.Ctx, vec.GetType(), val, vec.GetNulls()) + err = checkOverFlow[int64, T](proc.Ctx, colType, val, &n) if err != nil { - return false, err + return false, num, err } - err = vector.AppendFixed(vec, T(val), false, proc.Mp()) + num = T(val) case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, num, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = checkOverFlow[uint64, T](proc.Ctx, vec.GetType(), val, vec.GetNulls()) + err = checkOverFlow[uint64, T](proc.Ctx, colType, val, &n) if err != nil { - return false, err + return false, num, err } - err = vector.AppendFixed(vec, T(val), false, proc.Mp()) + num = T(val) case tree.P_float64: val, ok := numVal.Float64() if canInsert = ok; canInsert { - var v T - if err = checkOverFlow[float64, T](proc.Ctx, vec.GetType(), val, - vec.GetNulls()); err != nil { - return false, err + if err = checkOverFlow[float64, T](proc.Ctx, colType, val, + &n); err != nil { + return false, num, err } - if vec.GetType().Scale < 0 || vec.GetType().Width == 0 { - v = T(val) + if colType.Scale < 0 || colType.Width == 0 { + num = T(val) } else { - v, err = floatNumToFixFloat[T](val, numVal.String(), vec.GetType()) + num, err = floatNumToFixFloat[T](val, numVal.String(), colType) if err != nil { - return false, err + return false, num, err } } - if err = vector.AppendFixed(vec, v, false, proc.Mp()); err != nil { - return false, err - } } case tree.P_hexnum: var val uint64 if val, err = HexToInt(numVal.String()); err != nil { - return false, err + return false, num, err } - if err = checkOverFlow[uint64, T](proc.Ctx, vec.GetType(), val, vec.GetNulls()); err != nil { - return false, err + if err = checkOverFlow[uint64, T](proc.Ctx, colType, val, &n); err != nil { + return false, num, err } - err = vector.AppendFixed(vec, T(val), false, proc.Mp()) + num = T(val) case tree.P_bit: var val uint64 if val, err = BinaryToInt(numVal.String()); err != nil { - return false, err + return false, num, err } - if err = checkOverFlow[uint64, T](proc.Ctx, vec.GetType(), val, vec.GetNulls()); err != nil { - return false, err + if err = checkOverFlow[uint64, T](proc.Ctx, colType, val, &n); err != nil { + return false, num, err } - err = vector.AppendFixed(vec, T(val), false, proc.Mp()) - - case tree.P_nulltext: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) + num = T(val) default: canInsert = false @@ -904,60 +755,52 @@ func setInsertValueNumber[T constraints.Integer | constraints.Float](proc *proce return } -func setInsertValueDecimal64(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueDecimal64(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Decimal64, err error) { canInsert = true - appendWithStr := func(str string) error { - typ := vec.GetType() - result, err := types.ParseDecimal64(str, typ.Width, typ.Scale) - if err != nil { - return err - } - return vector.AppendFixed(vec, result, false, proc.Mp()) + appendWithStr := func(str string) (types.Decimal64, error) { + return types.ParseDecimal64(str, typ.Width, typ.Scale) } - appendWithUnSigned := func(v uint64) error { - typ := vec.GetType() - result, _ := types.Decimal64(v).Scale(typ.Scale) - return vector.AppendFixed(vec, result, false, proc.Mp()) + appendWithUnSigned := func(v uint64) (types.Decimal64, error) { + return types.Decimal64(v).Scale(typ.Scale) } switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) } - err = appendWithUnSigned(uint64(val)) + res, err = appendWithUnSigned(uint64(val)) + return case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = appendWithUnSigned(uint64(val)) + res, err = appendWithUnSigned(uint64(val)) + return case tree.P_decimal, tree.P_char, tree.P_float64: originStr := numVal.String() - err = appendWithStr(originStr) + res, err = appendWithStr(originStr) + return case tree.P_hexnum: var val uint64 if val, err = HexToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendWithUnSigned(val) + res, err = appendWithUnSigned(val) + return case tree.P_bit: var val uint64 if val, err = BinaryToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendWithUnSigned(val) - - case tree.P_nulltext: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) + res, err = appendWithUnSigned(val) + return default: canInsert = false @@ -965,61 +808,52 @@ func setInsertValueDecimal64(proc *process.Process, numVal *tree.NumVal, vec *ve return } -func setInsertValueDecimal128(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueDecimal128(proc *process.Process, numVal *tree.NumVal, typ *types.Type) (canInsert bool, res types.Decimal128, err error) { canInsert = true - appendWithStr := func(str string) error { - typ := vec.GetType() - result, err := types.ParseDecimal128(str, typ.Width, typ.Scale) - if err != nil { - return err - } - return vector.AppendFixed(vec, result, false, proc.Mp()) + appendWithStr := func(str string) (types.Decimal128, error) { + return types.ParseDecimal128(str, typ.Width, typ.Scale) } - appendWithUnSigned := func(v uint64) error { - typ := vec.GetType() + appendWithUnSigned := func(v uint64) (types.Decimal128, error) { result := types.Decimal128{B0_63: v, B64_127: 0} - result, _ = result.Scale(typ.Scale) - return vector.AppendFixed(vec, result, false, proc.Mp()) + return result.Scale(typ.Scale) } switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - case tree.P_int64: val, ok := numVal.Int64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid int value '%s'", numVal.String()) } - err = appendWithUnSigned(uint64(val)) - + res, err = appendWithUnSigned(uint64(val)) + return case tree.P_uint64: val, ok := numVal.Uint64() if !ok { - return false, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) + return false, res, moerr.NewInvalidInputf(proc.Ctx, "invalid uint value '%s'", numVal.String()) } - err = appendWithUnSigned(uint64(val)) + res, err = appendWithUnSigned(uint64(val)) + return case tree.P_decimal, tree.P_char, tree.P_float64: originStr := numVal.String() - err = appendWithStr(originStr) + res, err = appendWithStr(originStr) + return case tree.P_hexnum: var val uint64 if val, err = HexToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendWithUnSigned(val) + res, err = appendWithUnSigned(val) + return case tree.P_bit: var val uint64 if val, err = BinaryToInt(numVal.String()); err != nil { - return false, err + return false, res, err } - err = appendWithUnSigned(val) - - case tree.P_nulltext: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) + res, err = appendWithUnSigned(val) + return default: canInsert = false @@ -1046,21 +880,16 @@ func floatNumToFixFloat[T constraints.Float | constraints.Integer]( return T(v), nil } -func setInsertValueBit(proc *process.Process, numVal *tree.NumVal, vec *vector.Vector) (canInsert bool, err error) { +func SetInsertValueBit(proc *process.Process, numVal *tree.NumVal, colType *types.Type) (canInsert bool, val uint64, err error) { var ok bool canInsert = true - width := vec.GetType().Width + width := colType.Width switch numVal.ValType { - case tree.P_null: - err = vector.AppendBytes(vec, nil, true, proc.Mp()) - case tree.P_bool: - var val uint64 if numVal.Bool() { val = 1 } - err = vector.AppendFixed(vec, val, false, proc.Mp()) case tree.P_char: s := numVal.String() @@ -1069,7 +898,6 @@ func setInsertValueBit(proc *process.Process, numVal *tree.NumVal, vec *vector.V return } - var val uint64 for i := 0; i < len(s); i++ { val = (val << 8) | uint64(s[i]) } @@ -1077,38 +905,36 @@ func setInsertValueBit(proc *process.Process, numVal *tree.NumVal, vec *vector.V err = moerr.NewInvalidInputf(proc.Ctx, "data too long, type width = %d, val = %b", width, val) return } - err = vector.AppendFixed(vec, val, false, proc.Mp()) case tree.P_float64: - var val float64 - if val, ok = numVal.Float64(); !ok { + var num float64 + if num, ok = numVal.Float64(); !ok { err = moerr.NewInvalidInputf(proc.Ctx, "invalid float value '%s'", numVal.String()) return - } else if val < 0 { + } else if num < 0 { err = moerr.NewInvalidInputf(proc.Ctx, "unsupported negative value %v", val) return - } else if uint64(math.Round(val)) > uint64(1< uint64(1< uint64(1< uint64(1< uint64(1< uint64(1< uint64(1<= proc.Base.prepareParams.Length() { return nil, moerr.NewInternalErrorf(proc.Ctx, "get prepare params error, index %d not exists", i) diff --git a/proto/plan.proto b/proto/plan.proto index 4204532a9f6f..0134f051f5cc 100644 --- a/proto/plan.proto +++ b/proto/plan.proto @@ -536,8 +536,7 @@ message Stats { message RowsetExpr { int32 row_pos = 1; - int32 pos = 2; - Expr expr = 3; + Expr expr = 2; } message ColData {