diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index cbd24fa98fed..578ae77bc24c 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -193,7 +193,7 @@ func supportedNatively(core *execinfrapb.ProcessorCoreUnion) error { case core.Aggregator != nil: for _, agg := range core.Aggregator.Aggregations { if agg.FilterColIdx != nil { - return errors.Newf("filtering aggregation not supported") + return errFilteringAggregation } } return nil @@ -206,13 +206,13 @@ func supportedNatively(core *execinfrapb.ProcessorCoreUnion) error { case core.HashJoiner != nil: if !core.HashJoiner.OnExpr.Empty() && core.HashJoiner.Type != descpb.InnerJoin { - return errors.Newf("can't plan vectorized non-inner hash joins with ON expressions") + return errNonInnerHashJoinWithOnExpr } return nil case core.MergeJoiner != nil: if !core.MergeJoiner.OnExpr.Empty() && core.MergeJoiner.Type != descpb.InnerJoin { - return errors.Errorf("can't plan non-inner merge join with ON expressions") + return errNonInnerMergeJoinWithOnExpr } return nil @@ -222,11 +222,11 @@ func supportedNatively(core *execinfrapb.ProcessorCoreUnion) error { case core.Windower != nil: for _, wf := range core.Windower.WindowFns { if wf.FilterColIdx != tree.NoColumnIdx { - return errors.Newf("window functions with FILTER clause are not supported") + return errWindowFunctionFilterClause } if wf.Func.AggregateFunc != nil { if !colexecagg.IsAggOptimized(*wf.Func.AggregateFunc) { - return errors.Newf("default aggregate window functions not supported") + return errDefaultAggregateWindowFunction } } } @@ -257,6 +257,11 @@ var ( errExperimentalWrappingProhibited = errors.New("wrapping for non-JoinReader and non-LocalPlanNode cores is prohibited in vectorize=experimental_always") errWrappedCast = errors.New("mismatched types in NewColOperator and unsupported casts") errLookupJoinUnsupported = errors.New("lookup join reader is unsupported in vectorized") + errFilteringAggregation = errors.New("filtering aggregation not supported") + errNonInnerHashJoinWithOnExpr = errors.New("can't plan vectorized non-inner hash joins with ON expressions") + errNonInnerMergeJoinWithOnExpr = errors.New("can't plan vectorized non-inner merge joins with ON expressions") + errWindowFunctionFilterClause = errors.New("window functions with FILTER clause are not supported") + errDefaultAggregateWindowFunction = errors.New("default aggregate window functions not supported") ) func canWrap(mode sessiondatapb.VectorizeExecMode, core *execinfrapb.ProcessorCoreUnion) error { @@ -1664,6 +1669,8 @@ var renderWrappingRenderCountThreshold = settings.RegisterIntSetting( settings.NonNegativeInt, ) +var errFallbackToRenderWrapping = errors.New("falling back to wrapping a row-by-row processor due to many renders and low estimated row count") + // planPostProcessSpec plans the post processing stage specified in post on top // of r.Op. func (r *postProcessResult) planPostProcessSpec( @@ -1700,10 +1707,7 @@ func (r *postProcessResult) planPostProcessSpec( for _, expr := range exprs { tree.WalkExpr(&v, expr) if v.renderCount >= renderCountThreshold { - return errors.Newf( - "falling back to wrapping a row-by-row processor for at least "+ - "%d renders, estimated row count = %d", v.renderCount, estimatedRowCount, - ) + return errFallbackToRenderWrapping } } } @@ -2366,6 +2370,8 @@ func planProjectionOperators( } } +var errMixedTypeUnsupported = errors.New("dates and timestamp(tz) not supported in mixed-type expressions in the vectorized engine") + func checkSupportedProjectionExpr(left, right tree.TypedExpr) error { leftTyp := left.ResolvedType() rightTyp := right.ResolvedType() @@ -2381,19 +2387,20 @@ func checkSupportedProjectionExpr(left, right tree.TypedExpr) error { for _, t := range []*types.T{leftTyp, rightTyp} { switch t.Family() { case types.DateFamily, types.TimestampFamily, types.TimestampTZFamily: - return errors.New("dates and timestamp(tz) not supported in mixed-type expressions in the vectorized engine") + return errMixedTypeUnsupported } } return nil } +var errBinaryExprWithDatums = errors.New("datum-backed arguments on both sides and not datum-backed output of a binary expression is currently not supported") + func checkSupportedBinaryExpr(left, right tree.TypedExpr, outputType *types.T) error { leftDatumBacked := typeconv.TypeFamilyToCanonicalTypeFamily(left.ResolvedType().Family()) == typeconv.DatumVecCanonicalTypeFamily rightDatumBacked := typeconv.TypeFamilyToCanonicalTypeFamily(right.ResolvedType().Family()) == typeconv.DatumVecCanonicalTypeFamily outputDatumBacked := typeconv.TypeFamilyToCanonicalTypeFamily(outputType.Family()) == typeconv.DatumVecCanonicalTypeFamily if (leftDatumBacked && rightDatumBacked) && !outputDatumBacked { - return errors.New("datum-backed arguments on both sides and not datum-backed " + - "output of a binary expression is currently not supported") + return errBinaryExprWithDatums } return nil }