Skip to content

Commit

Permalink
planbuilder: Pass through type information as well
Browse files Browse the repository at this point in the history
With the type information we can better decide if we need to print the
collation change or not, since it's only needed for textual columns.

This also fixes a number of cases where we used to use the weight_string
but we no longer need to since we now pass through the correct type
information in more cases.

Signed-off-by: Dirkjan Bussink <[email protected]>
  • Loading branch information
dbussink committed Jul 17, 2023
1 parent 8da8cdf commit cd02256
Show file tree
Hide file tree
Showing 26 changed files with 176 additions and 146 deletions.
4 changes: 3 additions & 1 deletion go/vt/vtgate/engine/aggregations.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ type AggregateParams struct {
// These are used only for distinct opcodes.
KeyCol int
WCol int
Type sqltypes.Type
CollationID collations.ID

Alias string `json:",omitempty"`
Expand All @@ -60,6 +61,7 @@ func NewAggregateParam(opcode AggregateOpcode, col int, alias string) *Aggregate
Col: col,
Alias: alias,
WCol: -1,
Type: -1,
}
if opcode.NeedsComparableValues() {
out.KeyCol = col
Expand All @@ -76,7 +78,7 @@ func (ap *AggregateParams) String() string {
if ap.WAssigned() {
keyCol = fmt.Sprintf("%s|%d", keyCol, ap.WCol)
}
if ap.CollationID != collations.Unknown {
if sqltypes.IsText(ap.Type) && ap.CollationID != collations.Unknown {
keyCol += " COLLATE " + ap.CollationID.Get().Name()
}
dispOrigOp := ""
Expand Down
8 changes: 4 additions & 4 deletions go/vt/vtgate/engine/cached_size.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

7 changes: 4 additions & 3 deletions go/vt/vtgate/engine/distinct.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ type (
CheckCol struct {
Col int
WsCol *int
Type sqltypes.Type
Collation collations.ID
}
probeTable struct {
Expand Down Expand Up @@ -274,15 +275,15 @@ func (cc CheckCol) SwitchToWeightString() CheckCol {
return CheckCol{
Col: *cc.WsCol,
WsCol: nil,
Type: sqltypes.VarBinary,
Collation: collations.CollationBinaryID,
}
}

func (cc CheckCol) String() string {
coll := cc.Collation.Get()
var collation string
if coll != nil {
collation = ": " + coll.Name()
if sqltypes.IsText(cc.Type) && cc.Collation != collations.Unknown {
collation = ": " + cc.Collation.Get().Name()
}

var column string
Expand Down
7 changes: 5 additions & 2 deletions go/vt/vtgate/engine/distinct_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,12 +65,12 @@ func TestDistinct(t *testing.T) {
expectedError: "text type with an unknown/unsupported collation cannot be hashed",
}, {
testName: "varchar columns with collations",
collations: []collations.ID{collations.ID(0x21)},
collations: []collations.ID{collations.CollationUtf8mb4ID},
inputs: r("myid", "varchar", "monkey", "horse", "Horse", "Monkey", "horses", "MONKEY"),
expectedResult: r("myid", "varchar", "monkey", "horse", "horses"),
}, {
testName: "mixed columns",
collations: []collations.ID{collations.ID(0x21), collations.Unknown},
collations: []collations.ID{collations.CollationUtf8mb4ID, collations.Unknown},
inputs: r("myid|id", "varchar|int64", "monkey|1", "horse|1", "Horse|1", "Monkey|1", "horses|1", "MONKEY|2"),
expectedResult: r("myid|id", "varchar|int64", "monkey|1", "horse|1", "horses|1", "MONKEY|2"),
}}
Expand All @@ -88,6 +88,7 @@ func TestDistinct(t *testing.T) {
}
checkCols = append(checkCols, CheckCol{
Col: i,
Type: tc.inputs.Fields[i].Type,
Collation: collID,
})
}
Expand Down Expand Up @@ -133,6 +134,7 @@ func TestWeightStringFallBack(t *testing.T) {
checkCols := []CheckCol{{
Col: 0,
WsCol: &offsetOne,
Type: -1,
Collation: collations.Unknown,
}}
input := r("myid|weightstring(myid)",
Expand All @@ -158,6 +160,7 @@ func TestWeightStringFallBack(t *testing.T) {
utils.MustMatch(t, []CheckCol{{
Col: 0,
WsCol: &offsetOne,
Type: -1,
Collation: collations.Unknown,
}}, distinct.CheckCols, "checkCols should not be updated")
}
2 changes: 2 additions & 0 deletions go/vt/vtgate/engine/memory_sort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,7 @@ func TestMemorySortStreamExecuteCollation(t *testing.T) {
ms := &MemorySort{
OrderBy: []OrderByParams{{
Col: 0,
Type: sqltypes.VarChar,
CollationID: collationID,
}},
Input: fp,
Expand Down Expand Up @@ -319,6 +320,7 @@ func TestMemorySortExecuteCollation(t *testing.T) {
ms := &MemorySort{
OrderBy: []OrderByParams{{
Col: 0,
Type: sqltypes.VarChar,
CollationID: collationID,
}},
Input: fp,
Expand Down
1 change: 1 addition & 0 deletions go/vt/vtgate/engine/merge_sort_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,7 @@ func TestMergeSortCollation(t *testing.T) {
collationID, _ := collations.Local().LookupID("utf8mb4_hu_0900_ai_ci")
orderBy := []OrderByParams{{
Col: 0,
Type: sqltypes.VarChar,
CollationID: collationID,
}}

Expand Down
3 changes: 2 additions & 1 deletion go/vt/vtgate/engine/ordered_aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ type GroupByParams struct {
WeightStringCol int
Expr sqlparser.Expr
FromGroupBy bool
Type sqltypes.Type
CollationID collations.ID
}

Expand All @@ -78,7 +79,7 @@ func (gbp GroupByParams) String() string {
out = fmt.Sprintf("(%d|%d)", gbp.KeyCol, gbp.WeightStringCol)
}

if gbp.CollationID != collations.Unknown {
if sqltypes.IsText(gbp.Type) && gbp.CollationID != collations.Unknown {
collation := gbp.CollationID.Get()
out += " COLLATE " + collation.Name()
}
Expand Down
2 changes: 1 addition & 1 deletion go/vt/vtgate/engine/projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func (p *Projection) evalFields(env *evalengine.ExpressionEnv, infields []*query
return nil, err
}
var cs collations.ID = collations.CollationBinaryID
if sqltypes.IsQuoted(q) && !sqltypes.IsBinary(q) {
if sqltypes.IsText(q) {
cs = vcursor.ConnCollation()
}

Expand Down
8 changes: 7 additions & 1 deletion go/vt/vtgate/engine/route.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,8 @@ type OrderByParams struct {
StarColFixedIndex int
// v3 specific boolean. Used to also add weight strings originating from GroupBys to the Group by clause
FromGroupBy bool
// Type for knowing if the collation is relevant
Type querypb.Type
// Collation ID for comparison using collation
CollationID collations.ID
}
Expand All @@ -140,7 +142,11 @@ func (obp OrderByParams) String() string {
} else {
val += " ASC"
}
if obp.CollationID != collations.Unknown {

if obp.Type != -1 && obp.CollationID == collations.Unknown {
panic("OrderByParams: collationID is unknown but type is not unknown")
}
if sqltypes.IsText(obp.Type) && obp.CollationID != collations.Unknown {
collation := obp.CollationID.Get()
val += " COLLATE " + collation.Name()
}
Expand Down
2 changes: 2 additions & 0 deletions go/vt/vtgate/engine/route_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1081,6 +1081,7 @@ func TestRouteSortCollation(t *testing.T) {

sel.OrderBy = []OrderByParams{{
Col: 0,
Type: sqltypes.VarChar,
CollationID: collationID,
}}

Expand Down Expand Up @@ -1147,6 +1148,7 @@ func TestRouteSortCollation(t *testing.T) {
t.Run("Error when Unknown Collation", func(t *testing.T) {
sel.OrderBy = []OrderByParams{{
Col: 0,
Type: -1,
CollationID: collations.Unknown,
}}

Expand Down
4 changes: 2 additions & 2 deletions go/vt/vtgate/evalengine/cached_size.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

26 changes: 12 additions & 14 deletions go/vt/vtgate/evalengine/eval_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,21 +35,19 @@ type EvalResult struct {
// collation and what the client expects the result to be in.
func (er EvalResult) Value(id collations.ID) sqltypes.Value {
str, ok := er.v.(*evalBytes)
if ok && !str.isBinary() {
if str.col.Collation == id {
return sqltypes.MakeTrusted(str.SQLType(), str.bytes)
}
dst, err := charset.Convert(nil, id.Get().Charset(), str.bytes, str.col.Collation.Get().Charset())
if err != nil {
// If we can't convert, we just return what we have, but it's going
// to be invalidly encoded. Should normally never happen as only utf8mb4
// is really supported for the connection character set anyway and all
// other charsets can be converted to utf8mb4.
return sqltypes.MakeTrusted(str.SQLType(), str.bytes)
}
return sqltypes.MakeTrusted(str.SQLType(), dst)
if !ok || str.isBinary() || str.col.Collation == collations.Unknown || str.col.Collation == id {
return evalToSQLValue(er.v)
}

dst, err := charset.Convert(nil, id.Get().Charset(), str.bytes, str.col.Collation.Get().Charset())
if err != nil {
// If we can't convert, we just return what we have, but it's going
// to be invalidly encoded. Should normally never happen as only utf8mb4
// is really supported for the connection character set anyway and all
// other charsets can be converted to utf8mb4.
return sqltypes.MakeTrusted(str.SQLType(), str.bytes)
}
return evalToSQLValue(er.v)
return sqltypes.MakeTrusted(str.SQLType(), dst)
}

func (er EvalResult) Collation() collations.ID {
Expand Down
4 changes: 2 additions & 2 deletions go/vt/vtgate/planbuilder/grouping.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ func planGroupBy(pb *primitiveBuilder, input logicalPlan, groupBy sqlparser.Grou
default:
return nil, vterrors.VT12001("in scatter query: only simple references are allowed")
}
node.groupByKeys = append(node.groupByKeys, &engine.GroupByParams{KeyCol: colNumber, WeightStringCol: -1, FromGroupBy: true})
node.groupByKeys = append(node.groupByKeys, &engine.GroupByParams{KeyCol: colNumber, WeightStringCol: -1, Type: -1, FromGroupBy: true})
}
// Append the distinct aggregate if any.
if node.extraDistinct != nil {
Expand Down Expand Up @@ -111,7 +111,7 @@ func planDistinct(input logicalPlan) (logicalPlan, error) {
if rc.column.Origin() == node {
return newDistinctV3(node), nil
}
node.groupByKeys = append(node.groupByKeys, &engine.GroupByParams{KeyCol: i, WeightStringCol: -1, FromGroupBy: false})
node.groupByKeys = append(node.groupByKeys, &engine.GroupByParams{KeyCol: i, WeightStringCol: -1, Type: -1, FromGroupBy: false})
}
newInput, err := planDistinct(node.input)
if err != nil {
Expand Down
28 changes: 20 additions & 8 deletions go/vt/vtgate/planbuilder/horizon_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package planbuilder
import (
"fmt"

"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
Expand Down Expand Up @@ -279,10 +280,12 @@ func (hp *horizonPlanning) planAggrUsingOA(
// here we are building up the grouping keys for the OA,
// but they are lacking the input offsets because we have yet to push the columns down
for _, expr := range grouping {
typ, col, _ := ctx.SemTable.TypeForExpr(expr.Inner)
oa.groupByKeys = append(oa.groupByKeys, &engine.GroupByParams{
Expr: expr.Inner,
FromGroupBy: true,
CollationID: ctx.SemTable.CollationForExpr(expr.Inner),
Type: typ,
CollationID: col,
})
}

Expand Down Expand Up @@ -474,12 +477,14 @@ func addColumnsToOA(
o := groupings[count]
count++
a := aggregationExprs[offset]
collID := ctx.SemTable.CollationForExpr(a.Func.GetArg())
aggr := engine.NewAggregateParam(a.OpCode, o.col, a.Alias)
aggr.KeyCol = o.col
aggr.WCol = o.wsCol
aggr.Original = a.Original
aggr.CollationID = collID
aggr.Type, aggr.CollationID, _ = ctx.SemTable.TypeForExpr(a.Func.GetArg())
if aggr.Type != -1 && aggr.CollationID == collations.Unknown {
panic("unexpected: aggregate function without collation")
}
oa.aggregates = append(oa.aggregates, aggr)
}
lastOffset := distinctOffsets[len(distinctOffsets)-1]
Expand Down Expand Up @@ -678,11 +683,13 @@ func planOrderByForRoute(ctx *plancontext.PlanningContext, orderExprs []ops.Orde
if err != nil {
return nil, err
}
typ, col, _ := ctx.SemTable.TypeForExpr(order.Inner.Expr)
plan.eroute.OrderBy = append(plan.eroute.OrderBy, engine.OrderByParams{
Col: offset,
WeightStringCol: weightStringOffset,
Desc: order.Inner.Direction == sqlparser.DescOrder,
CollationID: ctx.SemTable.CollationForExpr(order.Inner.Expr),
Type: typ,
CollationID: col,
})
}
return plan, nil
Expand Down Expand Up @@ -823,12 +830,13 @@ func createMemorySortPlanOnAggregation(ctx *plancontext.PlanningContext, plan *o
return nil, vterrors.VT13001(fmt.Sprintf("expected to find ORDER BY expression (%s) in orderedAggregate", sqlparser.String(order.Inner)))
}

collationID := ctx.SemTable.CollationForExpr(order.SimplifiedExpr)
typ, collationID, _ := ctx.SemTable.TypeForExpr(order.SimplifiedExpr)
ms.eMemorySort.OrderBy = append(ms.eMemorySort.OrderBy, engine.OrderByParams{
Col: offset,
WeightStringCol: woffset,
Desc: order.Inner.Direction == sqlparser.DescOrder,
StarColFixedIndex: offset,
Type: typ,
CollationID: collationID,
})
}
Expand Down Expand Up @@ -871,12 +879,14 @@ func (hp *horizonPlanning) createMemorySortPlan(ctx *plancontext.PlanningContext
if err != nil {
return nil, err
}
typ, col, _ := ctx.SemTable.TypeForExpr(order.Inner.Expr)
ms.eMemorySort.OrderBy = append(ms.eMemorySort.OrderBy, engine.OrderByParams{
Col: offset,
WeightStringCol: weightStringOffset,
Desc: order.Inner.Direction == sqlparser.DescOrder,
StarColFixedIndex: offset,
CollationID: ctx.SemTable.CollationForExpr(order.Inner.Expr),
Type: typ,
CollationID: col,
})
}
return ms, nil
Expand Down Expand Up @@ -941,7 +951,8 @@ func (hp *horizonPlanning) planDistinctOA(semTable *semantics.SemTable, currPlan
for _, aggrParam := range currPlan.aggregates {
if semTable.EqualsExpr(expr, aggrParam.Expr) {
found = true
oa.groupByKeys = append(oa.groupByKeys, &engine.GroupByParams{KeyCol: aggrParam.Col, WeightStringCol: -1, CollationID: semTable.CollationForExpr(expr)})
typ, col, _ := semTable.TypeForExpr(expr)
oa.groupByKeys = append(oa.groupByKeys, &engine.GroupByParams{KeyCol: aggrParam.Col, WeightStringCol: -1, Type: typ, CollationID: col})
break
}
}
Expand Down Expand Up @@ -973,7 +984,8 @@ func (hp *horizonPlanning) addDistinct(ctx *plancontext.PlanningContext, plan lo
inner = sqlparser.NewColName(aliasExpr.As.String())
ctx.SemTable.CopyDependencies(aliasExpr.Expr, inner)
}
grpParam := &engine.GroupByParams{KeyCol: index, WeightStringCol: -1, CollationID: ctx.SemTable.CollationForExpr(inner), Expr: inner}
typ, col, _ := ctx.SemTable.TypeForExpr(inner)
grpParam := &engine.GroupByParams{KeyCol: index, WeightStringCol: -1, Type: typ, CollationID: col, Expr: inner}
_, wOffset, err := wrapAndPushExpr(ctx, aliasExpr.Expr, aliasExpr.Expr, plan)
if err != nil {
return nil, err
Expand Down
1 change: 1 addition & 0 deletions go/vt/vtgate/planbuilder/memory_sort.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ func newMemorySort(plan logicalPlan, orderBy v3OrderBy) (*memorySort, error) {
Desc: order.Direction == sqlparser.DescOrder,
StarColFixedIndex: colNumber,
FromGroupBy: order.fromGroupBy,
Type: -1,
CollationID: collations.Unknown,
}
ms.eMemorySort.OrderBy = append(ms.eMemorySort.OrderBy, ob)
Expand Down
Loading

0 comments on commit cd02256

Please sign in to comment.