Skip to content
This repository was archived by the owner on Jan 28, 2021. It is now read-only.

Commit 7413160

Browse files
committed
sql: change TransformExpressions signature to return (sql.Node, error) and make Expressioners nodes immutable
Signed-off-by: Manuel Carmona <[email protected]>
1 parent 10425c6 commit 7413160

File tree

9 files changed

+36
-46
lines changed

9 files changed

+36
-46
lines changed

sql/analyzer/rules.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -824,7 +824,7 @@ func resolveGetFieldIndexes(ctx *sql.Context, a *Analyzer, node sql.Node) (sql.N
824824
return n, nil
825825
}
826826

827-
err := expressioner.TransformExpressions(func(e sql.Expression) (sql.Expression, error) {
827+
return expressioner.TransformExpressions(func(e sql.Expression) (sql.Expression, error) {
828828
a.Log("transforming expression of type: %T", e)
829829
if !e.Resolved() {
830830
return e, nil
@@ -866,7 +866,5 @@ func resolveGetFieldIndexes(ctx *sql.Context, a *Analyzer, node sql.Node) (sql.N
866866
a.Log("fixed expression %T for %T", field, n)
867867
return fixedField, nil
868868
})
869-
870-
return n, err
871869
})
872870
}

sql/core.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -113,7 +113,7 @@ type Expressioner interface {
113113
// TransformExprssions applies for each expression in this node
114114
// the expression's TransformUp method with the given function, and
115115
// replace those expressions with the resultant expressions.
116-
TransformExpressions(TransformExprFunc) error
116+
TransformExpressions(TransformExprFunc) (Node, error)
117117
}
118118

119119
// Table represents a SQL table.

sql/plan/filter.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -73,14 +73,13 @@ func (p *Filter) Expressions() []sql.Expression {
7373
}
7474

7575
// TransformExpressions implements the Expressioner interface.
76-
func (p *Filter) TransformExpressions(f sql.TransformExprFunc) error {
76+
func (p *Filter) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
7777
expression, err := p.Expression.TransformUp(f)
7878
if err != nil {
79-
return err
79+
return nil, err
8080
}
8181

82-
p.Expression = expression
83-
return nil
82+
return NewFilter(expression, p.Child), nil
8483
}
8584

8685
// FilterIter is an iterator that filters another iterator and skips rows that

sql/plan/group_by.go

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -138,20 +138,18 @@ func (p *GroupBy) Expressions() []sql.Expression {
138138
}
139139

140140
// TransformExpressions implements the Expressioner interface.
141-
func (p *GroupBy) TransformExpressions(f sql.TransformExprFunc) error {
141+
func (p *GroupBy) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
142142
agg, err := transformExpressionsUp(f, p.Aggregate)
143143
if err != nil {
144-
return err
144+
return nil, err
145145
}
146146

147147
group, err := transformExpressionsUp(f, p.Grouping)
148148
if err != nil {
149-
return err
149+
return nil, err
150150
}
151151

152-
p.Aggregate = agg
153-
p.Grouping = group
154-
return nil
152+
return NewGroupBy(agg, group, p.Child), nil
155153
}
156154

157155
type groupByIter struct {

sql/plan/innerjoin.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -119,12 +119,11 @@ func (j *InnerJoin) Expressions() []sql.Expression {
119119
}
120120

121121
// TransformExpressions implements the Expressioner interface.
122-
func (j *InnerJoin) TransformExpressions(f sql.TransformExprFunc) error {
122+
func (j *InnerJoin) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
123123
cond, err := j.Cond.TransformUp(f)
124124
if err != nil {
125-
return err
125+
return nil, err
126126
}
127127

128-
j.Cond = cond
129-
return nil
128+
return NewInnerJoin(j.Left, j.Right, cond), nil
130129
}

sql/plan/project.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -110,14 +110,13 @@ func (p *Project) Expressions() []sql.Expression {
110110
}
111111

112112
// TransformExpressions implements the Expressioner interface.
113-
func (p *Project) TransformExpressions(f sql.TransformExprFunc) error {
113+
func (p *Project) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
114114
projects, err := transformExpressionsUp(f, p.Projections)
115115
if err != nil {
116-
return err
116+
return nil, err
117117
}
118118

119-
p.Projections = projects
120-
return nil
119+
return NewProject(projects, p.Child), nil
121120
}
122121

123122
type iter struct {

sql/plan/pushdown.go

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -169,18 +169,20 @@ func (t *PushdownProjectionAndFiltersTable) Expressions() []sql.Expression {
169169
}
170170

171171
// TransformExpressions implements the Expressioner interface.
172-
func (t *PushdownProjectionAndFiltersTable) TransformExpressions(f sql.TransformExprFunc) error {
172+
func (t *PushdownProjectionAndFiltersTable) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
173173
cols, err := transformExpressionsUp(f, t.Columns)
174174
if err != nil {
175-
return err
175+
return nil, err
176176
}
177177

178178
filters, err := transformExpressionsUp(f, t.Filters)
179179
if err != nil {
180-
return err
180+
return nil, err
181181
}
182182

183-
t.Columns = cols
184-
t.Filters = filters
185-
return nil
183+
return NewPushdownProjectionAndFiltersTable(
184+
cols,
185+
filters,
186+
t.PushdownProjectionAndFiltersTable,
187+
), nil
186188
}

sql/plan/sort.go

Lines changed: 10 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -142,25 +142,21 @@ func (s *Sort) Expressions() []sql.Expression {
142142
}
143143

144144
// TransformExpressions implements the Expressioner interface.
145-
func (s *Sort) TransformExpressions(f sql.TransformExprFunc) error {
146-
var cols = make([]sql.Expression, len(s.SortFields))
147-
for i, sortField := range s.SortFields {
148-
transformed, err := sortField.Column.TransformUp(f)
145+
func (s *Sort) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
146+
var sortFields = make([]SortField, len(s.SortFields))
147+
for i, field := range s.SortFields {
148+
transformed, err := field.Column.TransformUp(f)
149149
if err != nil {
150-
return err
150+
return nil, err
151151
}
152-
cols[i] = transformed
153-
}
154-
155-
for i, f := range s.SortFields {
156-
s.SortFields[i] = SortField{
157-
Column: cols[i],
158-
Order: f.Order,
159-
NullOrdering: f.NullOrdering,
152+
sortFields[i] = SortField{
153+
Column: transformed,
154+
Order: field.Order,
155+
NullOrdering: field.NullOrdering,
160156
}
161157
}
162158

163-
return nil
159+
return NewSort(sortFields, s.Child), nil
164160
}
165161

166162
type sortIter struct {

sql/plan/values.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -109,16 +109,15 @@ func (p *Values) Expressions() []sql.Expression {
109109
}
110110

111111
// TransformExpressions implements the Expressioner interface.
112-
func (p *Values) TransformExpressions(f sql.TransformExprFunc) error {
112+
func (p *Values) TransformExpressions(f sql.TransformExprFunc) (sql.Node, error) {
113113
tuples := [][]sql.Expression{}
114114
for _, tuple := range p.ExpressionTuples {
115115
transformed, err := transformExpressionsUp(f, tuple)
116116
if err != nil {
117-
return err
117+
return nil, err
118118
}
119119
tuples = append(tuples, transformed)
120120
}
121121

122-
p.ExpressionTuples = tuples
123-
return nil
122+
return NewValues(tuples), nil
124123
}

0 commit comments

Comments
 (0)