diff --git a/pkg/sql/distsql/api.pb.go b/pkg/sql/distsql/api.pb.go index 0acf4b40239e..21f238245d70 100644 --- a/pkg/sql/distsql/api.pb.go +++ b/pkg/sql/distsql/api.pb.go @@ -29,6 +29,7 @@ TableReaderSpec JoinReaderSpec SorterSpec + EvaluatorSpec ProcessorCoreUnion ProcessorSpec FlowSpec diff --git a/pkg/sql/distsql/evaluator.go b/pkg/sql/distsql/evaluator.go new file mode 100644 index 000000000000..49fdefd11269 --- /dev/null +++ b/pkg/sql/distsql/evaluator.go @@ -0,0 +1,135 @@ +// Copyright 2016 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. +// +// Author: Irfan Sharif (irfansharif@cockroachlabs.com) + +package distsql + +import ( + "sync" + + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "golang.org/x/net/context" +) + +type evaluator struct { + input RowSource + output RowReceiver + ctx context.Context + exprs []exprHelper + render []parser.TypedExpr + + // Buffer to store intermediate results when evaluating expressions per row + // to avoid reallocation. + tuple parser.DTuple + rowAlloc sqlbase.EncDatumRowAlloc +} + +func newEvaluator( + flowCtx *FlowCtx, spec *EvaluatorSpec, input RowSource, output RowReceiver, +) (*evaluator, error) { + ev := &evaluator{ + input: input, + output: output, + ctx: log.WithLogTag(flowCtx.Context, "Evaluator", nil), + exprs: make([]exprHelper, len(spec.Exprs)), + render: make([]parser.TypedExpr, len(spec.Exprs)), + tuple: make(parser.DTuple, len(spec.Exprs)), + } + + for i, expr := range spec.Exprs { + err := ev.exprs[i].init(expr, spec.Types, flowCtx.evalCtx) + if err != nil { + return nil, err + } + } + + // Loop over the expressions in our expression set and extract out fully + // typed expressions, this will later be evaluated for each input row to + // construct our output row. + for i := range ev.exprs { + typedExpr, err := (&ev.exprs[i]).expr.TypeCheck(nil, parser.NoTypePreference) + if err != nil { + return nil, err + } + ev.render[i] = typedExpr + } + + return ev, nil +} + +// Run is part of the processor interface. +func (ev *evaluator) Run(wg *sync.WaitGroup) { + if wg != nil { + defer wg.Done() + } + + ctx, span := tracing.ChildSpan(ev.ctx, "evaluator") + defer tracing.FinishSpan(span) + + if log.V(2) { + log.Infof(ctx, "starting evaluator process") + defer log.Infof(ctx, "exiting evaluator") + } + + for { + row, err := ev.input.NextRow() + if err != nil || row == nil { + ev.output.Close(err) + return + } + + outRow, err := ev.eval(row) + if err != nil { + ev.output.Close(err) + return + } + + if log.V(3) { + log.Infof(ctx, "pushing %s\n", outRow) + } + // Push the row to the output RowReceiver; stop if they don't need more + // rows. + if !ev.output.PushRow(outRow) { + if log.V(2) { + log.Infof(ctx, "no more rows required") + } + ev.output.Close(nil) + return + } + } +} + +func (ev *evaluator) eval(row sqlbase.EncDatumRow) (sqlbase.EncDatumRow, error) { + for i := range ev.exprs { + datum, err := (&ev.exprs[i]).eval(row) + if err != nil { + return nil, err + } + ev.tuple[i] = datum + } + + outRow := ev.rowAlloc.AllocRow(len(ev.tuple)) + for i, datum := range ev.tuple { + encDatum, err := sqlbase.DatumToEncDatum(datum) + if err != nil { + return nil, err + } + outRow[i] = encDatum + } + return outRow, nil +} diff --git a/pkg/sql/distsql/evaluator_test.go b/pkg/sql/distsql/evaluator_test.go new file mode 100644 index 000000000000..2667d90a3d57 --- /dev/null +++ b/pkg/sql/distsql/evaluator_test.go @@ -0,0 +1,157 @@ +// Copyright 2016 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. +// +// Author: Irfan Sharif (irfansharif@cockroachlabs.com) + +package distsql + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + + "golang.org/x/net/context" +) + +func TestEvaluator(t *testing.T) { + defer leaktest.AfterTest(t)() + + v := [15]sqlbase.EncDatum{} + for i := range v { + v[i].SetDatum(sqlbase.ColumnType_INT, parser.NewDInt(parser.DInt(i))) + } + + dTrue, _ := parser.ParseDBool("true") + dFalse, _ := parser.ParseDBool("false") + + b := [2]sqlbase.EncDatum{} + b[0].SetDatum(sqlbase.ColumnType_BOOL, dTrue) + b[1].SetDatum(sqlbase.ColumnType_BOOL, dFalse) + + testCases := []struct { + spec EvaluatorSpec + input sqlbase.EncDatumRows + expected sqlbase.EncDatumRows + }{ + { + spec: EvaluatorSpec{ + Types: []sqlbase.ColumnType_Kind{sqlbase.ColumnType_INT, sqlbase.ColumnType_INT}, + Exprs: []Expression{{Expr: "$1"}, {Expr: "((($0)))"}}, + }, + input: sqlbase.EncDatumRows{ + {v[1], v[2]}, + {v[3], v[4]}, + {v[6], v[2]}, + {v[7], v[2]}, + {v[8], v[4]}, + }, + expected: sqlbase.EncDatumRows{ + {v[2], v[1]}, + {v[4], v[3]}, + {v[2], v[6]}, + {v[2], v[7]}, + {v[4], v[8]}, + }, + }, { + spec: EvaluatorSpec{ + Types: []sqlbase.ColumnType_Kind{sqlbase.ColumnType_INT, sqlbase.ColumnType_INT}, + Exprs: []Expression{ + {Expr: "$0 + $1"}, + {Expr: "$0 - $1"}, + {Expr: "$0 >= 8"}, + }, + }, + input: sqlbase.EncDatumRows{ + {v[10], v[0]}, + {v[9], v[1]}, + {v[8], v[2]}, + {v[7], v[3]}, + {v[6], v[4]}, + }, + expected: sqlbase.EncDatumRows{ + {v[10], v[10], b[0]}, + {v[10], v[8], b[0]}, + {v[10], v[6], b[0]}, + {v[10], v[4], b[1]}, + {v[10], v[2], b[1]}, + }, + }, { + spec: EvaluatorSpec{ + Types: []sqlbase.ColumnType_Kind{sqlbase.ColumnType_BOOL, sqlbase.ColumnType_BOOL}, + Exprs: []Expression{ + {Expr: "$0 AND $0"}, + {Expr: "$0 AND $1"}, + {Expr: "NOT $0"}, + }, + }, + input: sqlbase.EncDatumRows{ + {b[0], b[1]}, + }, + expected: sqlbase.EncDatumRows{ + {b[0], b[1], b[1]}, + }, + }, + { + spec: EvaluatorSpec{ + Types: []sqlbase.ColumnType_Kind{sqlbase.ColumnType_INT, sqlbase.ColumnType_INT}, + Exprs: []Expression{{Expr: "1"}}, + }, + input: sqlbase.EncDatumRows{ + {v[1], v[2]}, + {v[3], v[4]}, + {v[6], v[2]}, + {v[7], v[2]}, + {v[8], v[4]}, + }, + expected: sqlbase.EncDatumRows{ + {v[1]}, + {v[1]}, + {v[1]}, + {v[1]}, + {v[1]}, + }, + }, + } + + for _, c := range testCases { + es := c.spec + + in := &RowBuffer{rows: c.input} + out := &RowBuffer{} + + flowCtx := FlowCtx{ + Context: context.Background(), + evalCtx: &parser.EvalContext{}, + } + + ev, err := newEvaluator(&flowCtx, &es, in, out) + if err != nil { + t.Fatal(err) + } + + ev.Run(nil) + if out.err != nil { + t.Fatal(out.err) + } + if !out.closed { + t.Fatalf("output RowReceiver not closed") + } + + if result := out.rows.String(); result != c.expected.String() { + t.Errorf("invalid results: %s, expected %s'", result, c.expected.String()) + } + } +} diff --git a/pkg/sql/distsql/expr.go b/pkg/sql/distsql/expr.go index f23e8b4ad05a..7f57d5a9869f 100644 --- a/pkg/sql/distsql/expr.go +++ b/pkg/sql/distsql/expr.go @@ -13,6 +13,7 @@ // permissions and limitations under the License. // // Author: Radu Berinde (radu@cockroachlabs.com) +// Author: Irfan Sharif (irfansharif@cockroachlabs.com) package distsql @@ -27,8 +28,8 @@ import ( "github.com/pkg/errors" ) -// valArgsConvert is a parser.Visitor that converts Placeholders ($0, $1, etc.) to -// IndexedVars. +// valArgsConvert is a parser.Visitor that converts Placeholders ($0, $1, etc.) +// to IndexedVars. type valArgsConvert struct { h *parser.IndexedVarHelper err error @@ -82,8 +83,8 @@ type exprHelper struct { noCopy util.NoCopy expr parser.TypedExpr - // vars is used to generate IndexedVars that are "backed" by - // the values in `row`. + // vars is used to generate IndexedVars that are "backed" by the values in + // `row`. vars parser.IndexedVarHelper evalCtx *parser.EvalContext @@ -142,3 +143,31 @@ func (eh *exprHelper) evalFilter(row sqlbase.EncDatumRow) (bool, error) { eh.row = row return sqlbase.RunFilter(eh.expr, eh.evalCtx) } + +// Given a row, eval evaluates the wrapped expression and returns the +// resulting datum needed for rendering for eg. given a row (1, 2, 3, 4, 5): +// '$1' would return '2' +// '$1 + $4' would return '7' +// '$0' would return '1' +// '$1 + 10' would return '12' +func (eh *exprHelper) eval(row sqlbase.EncDatumRow) (parser.Datum, error) { + eh.row = row + + // TODO(irfansharif): eval here is very permissive, if expr is of type + // *parser.FuncExpr for example expr.Eval doesn't make sense therefore is + // explicitly tested for. There may very well be other expression types + // where the same holds true but are not yet checked for. The set of + // verified parser expressions are: + // ComparisonExpr, FuncExpr, AndExpr, BinaryExpr, NotExpr, OrExpr, + // ParenExpr, UnaryExpr. + // + // The list of unverified parser expressions are: + // IsOfTypeExpr, AnnotateTypeExpr, CaseExpr, CastExpr, CoalesceExpr, + // ExistsExpr, IfExpr, NullIfExpr. + switch eh.expr.(type) { + case *parser.FuncExpr: + return nil, errors.Errorf("aggregate functions not allowed") + default: + return eh.expr.Eval(eh.evalCtx) + } +} diff --git a/pkg/sql/distsql/flow.go b/pkg/sql/distsql/flow.go index bb7f48314fa9..176577e087e9 100644 --- a/pkg/sql/distsql/flow.go +++ b/pkg/sql/distsql/flow.go @@ -13,6 +13,7 @@ // permissions and limitations under the License. // // Author: Radu Berinde (radu@cockroachlabs.com) +// Author: Irfan Sharif (irfansharif@cockroachlabs.com) package distsql @@ -207,6 +208,12 @@ func (f *Flow) makeProcessor(ps *ProcessorSpec, inputs []RowSource) (processor, } return newSorter(&f.FlowCtx, ps.Core.Sorter, inputs[0], outputs[0]), nil } + if ps.Core.Evaluator != nil { + if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { + return nil, err + } + return newEvaluator(&f.FlowCtx, ps.Core.Evaluator, inputs[0], outputs[0]) + } return nil, errors.Errorf("unsupported processor %s", ps) } diff --git a/pkg/sql/distsql/processors.pb.go b/pkg/sql/distsql/processors.pb.go index 61fd64557b60..66a8f093b573 100644 --- a/pkg/sql/distsql/processors.pb.go +++ b/pkg/sql/distsql/processors.pb.go @@ -114,17 +114,39 @@ func (m *SorterSpec) String() string { return proto.CompactTextString func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{4} } +// EvaluatorSpec is the specification for an "evaluator", a fully +// programmable no-grouping aggregator. It runs a 'program' on each individual +// row and is restricted to operating on one row of data at a time. +// The 'program' is a set of expressions evaluated in order, the output +// schema therefore consists of the results of evaluating each of these +// expressions on the input row. +// +// TODO(irfansharif): Add support for an optional output filter expression. +// The filter expression would reference the columns in the row via $0, $1, +// etc., possibly optimizing if filtering on expressions common to the +// 'program'. +type EvaluatorSpec struct { + Types []cockroach_sql_sqlbase1.ColumnType_Kind `protobuf:"varint,1,rep,name=types,enum=cockroach.sql.sqlbase.ColumnType_Kind" json:"types,omitempty"` + Exprs []Expression `protobuf:"bytes,2,rep,name=exprs" json:"exprs"` +} + +func (m *EvaluatorSpec) Reset() { *m = EvaluatorSpec{} } +func (m *EvaluatorSpec) String() string { return proto.CompactTextString(m) } +func (*EvaluatorSpec) ProtoMessage() {} +func (*EvaluatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{5} } + type ProcessorCoreUnion struct { Noop *NoopCoreSpec `protobuf:"bytes,1,opt,name=noop" json:"noop,omitempty"` TableReader *TableReaderSpec `protobuf:"bytes,2,opt,name=tableReader" json:"tableReader,omitempty"` JoinReader *JoinReaderSpec `protobuf:"bytes,3,opt,name=joinReader" json:"joinReader,omitempty"` Sorter *SorterSpec `protobuf:"bytes,4,opt,name=sorter" json:"sorter,omitempty"` + Evaluator *EvaluatorSpec `protobuf:"bytes,5,opt,name=evaluator" json:"evaluator,omitempty"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } func (m *ProcessorCoreUnion) String() string { return proto.CompactTextString(m) } func (*ProcessorCoreUnion) ProtoMessage() {} -func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{5} } +func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{6} } type ProcessorSpec struct { // In most cases, there is one input. @@ -137,7 +159,7 @@ type ProcessorSpec struct { func (m *ProcessorSpec) Reset() { *m = ProcessorSpec{} } func (m *ProcessorSpec) String() string { return proto.CompactTextString(m) } func (*ProcessorSpec) ProtoMessage() {} -func (*ProcessorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{6} } +func (*ProcessorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{7} } // FlowSpec describes a "flow" which is a subgraph of a distributed SQL // computation consisting of processors and streams. @@ -149,7 +171,7 @@ type FlowSpec struct { func (m *FlowSpec) Reset() { *m = FlowSpec{} } func (m *FlowSpec) String() string { return proto.CompactTextString(m) } func (*FlowSpec) ProtoMessage() {} -func (*FlowSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{7} } +func (*FlowSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{8} } func init() { proto.RegisterType((*NoopCoreSpec)(nil), "cockroach.sql.distsql.NoopCoreSpec") @@ -157,6 +179,7 @@ func init() { proto.RegisterType((*TableReaderSpec)(nil), "cockroach.sql.distsql.TableReaderSpec") proto.RegisterType((*JoinReaderSpec)(nil), "cockroach.sql.distsql.JoinReaderSpec") proto.RegisterType((*SorterSpec)(nil), "cockroach.sql.distsql.SorterSpec") + proto.RegisterType((*EvaluatorSpec)(nil), "cockroach.sql.distsql.EvaluatorSpec") proto.RegisterType((*ProcessorCoreUnion)(nil), "cockroach.sql.distsql.ProcessorCoreUnion") proto.RegisterType((*ProcessorSpec)(nil), "cockroach.sql.distsql.ProcessorSpec") proto.RegisterType((*FlowSpec)(nil), "cockroach.sql.distsql.FlowSpec") @@ -371,6 +394,43 @@ func (m *SorterSpec) MarshalTo(data []byte) (int, error) { return i, nil } +func (m *EvaluatorSpec) Marshal() (data []byte, err error) { + size := m.Size() + data = make([]byte, size) + n, err := m.MarshalTo(data) + if err != nil { + return nil, err + } + return data[:n], nil +} + +func (m *EvaluatorSpec) MarshalTo(data []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Types) > 0 { + for _, num := range m.Types { + data[i] = 0x8 + i++ + i = encodeVarintProcessors(data, i, uint64(num)) + } + } + if len(m.Exprs) > 0 { + for _, msg := range m.Exprs { + data[i] = 0x12 + i++ + i = encodeVarintProcessors(data, i, uint64(msg.Size())) + n, err := msg.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + func (m *ProcessorCoreUnion) Marshal() (data []byte, err error) { size := m.Size() data = make([]byte, size) @@ -426,6 +486,16 @@ func (m *ProcessorCoreUnion) MarshalTo(data []byte) (int, error) { } i += n14 } + if m.Evaluator != nil { + data[i] = 0x2a + i++ + i = encodeVarintProcessors(data, i, uint64(m.Evaluator.Size())) + n15, err := m.Evaluator.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n15 + } return i, nil } @@ -459,11 +529,11 @@ func (m *ProcessorSpec) MarshalTo(data []byte) (int, error) { data[i] = 0x12 i++ i = encodeVarintProcessors(data, i, uint64(m.Core.Size())) - n15, err := m.Core.MarshalTo(data[i:]) + n16, err := m.Core.MarshalTo(data[i:]) if err != nil { return 0, err } - i += n15 + i += n16 if len(m.Output) > 0 { for _, msg := range m.Output { data[i] = 0x1a @@ -497,11 +567,11 @@ func (m *FlowSpec) MarshalTo(data []byte) (int, error) { data[i] = 0xa i++ i = encodeVarintProcessors(data, i, uint64(m.FlowID.Size())) - n16, err := m.FlowID.MarshalTo(data[i:]) + n17, err := m.FlowID.MarshalTo(data[i:]) if err != nil { return 0, err } - i += n16 + i += n17 if len(m.Processors) > 0 { for _, msg := range m.Processors { data[i] = 0x12 @@ -613,6 +683,23 @@ func (m *SorterSpec) Size() (n int) { return n } +func (m *EvaluatorSpec) Size() (n int) { + var l int + _ = l + if len(m.Types) > 0 { + for _, e := range m.Types { + n += 1 + sovProcessors(uint64(e)) + } + } + if len(m.Exprs) > 0 { + for _, e := range m.Exprs { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + return n +} + func (m *ProcessorCoreUnion) Size() (n int) { var l int _ = l @@ -632,6 +719,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.Sorter.Size() n += 1 + l + sovProcessors(uint64(l)) } + if m.Evaluator != nil { + l = m.Evaluator.Size() + n += 1 + l + sovProcessors(uint64(l)) + } return n } @@ -695,6 +786,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.Sorter != nil { return this.Sorter } + if this.Evaluator != nil { + return this.Evaluator + } return nil } @@ -708,6 +802,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.JoinReader = vt case *SorterSpec: this.Sorter = vt + case *EvaluatorSpec: + this.Evaluator = vt default: return false } @@ -1432,6 +1528,107 @@ func (m *SorterSpec) Unmarshal(data []byte) error { } return nil } +func (m *EvaluatorSpec) Unmarshal(data []byte) error { + l := len(data) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: EvaluatorSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EvaluatorSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType) + } + var v cockroach_sql_sqlbase1.ColumnType_Kind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + v |= (cockroach_sql_sqlbase1.ColumnType_Kind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Types = append(m.Types, v) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Exprs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Exprs = append(m.Exprs, Expression{}) + if err := m.Exprs[len(m.Exprs)-1].Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessors(data[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ProcessorCoreUnion) Unmarshal(data []byte) error { l := len(data) iNdEx := 0 @@ -1593,6 +1790,39 @@ func (m *ProcessorCoreUnion) Unmarshal(data []byte) error { return err } iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Evaluator", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Evaluator == nil { + m.Evaluator = &EvaluatorSpec{} + } + if err := m.Evaluator.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(data[iNdEx:]) @@ -1977,53 +2207,57 @@ func init() { } var fileDescriptorProcessors = []byte{ - // 757 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xbc, 0x55, 0xcd, 0x6e, 0xd3, 0x4c, - 0x14, 0xed, 0xc4, 0xce, 0x4f, 0xa7, 0x4d, 0x5b, 0x8d, 0xbe, 0x4f, 0x58, 0x59, 0x24, 0xa9, 0x5b, - 0x20, 0x45, 0xc2, 0x51, 0xbb, 0x41, 0xb0, 0x01, 0xa5, 0x2d, 0x22, 0x55, 0x29, 0xc8, 0x81, 0x0d, - 0x9b, 0xc8, 0xb5, 0xa7, 0xa9, 0xa9, 0xe3, 0x71, 0x67, 0xc6, 0x34, 0x3c, 0x02, 0x3b, 0xc4, 0x0b, - 0xc0, 0x03, 0xf0, 0x20, 0x59, 0xb2, 0x44, 0x48, 0x54, 0x90, 0xbe, 0x08, 0x9a, 0xf1, 0x38, 0x3f, - 0xa5, 0xae, 0x58, 0x20, 0x56, 0x1d, 0xdd, 0x39, 0xe7, 0xf4, 0xdc, 0x73, 0xef, 0x38, 0xf0, 0x8e, - 0x4b, 0xdc, 0x13, 0x4a, 0x1c, 0xf7, 0xb8, 0x19, 0x9d, 0xf4, 0x9a, 0xec, 0x34, 0x68, 0x7a, 0x3e, - 0xe3, 0xe2, 0x6f, 0x44, 0x89, 0x8b, 0x19, 0x23, 0x94, 0x59, 0x11, 0x25, 0x9c, 0xa0, 0xff, 0xc7, - 0x58, 0x8b, 0x9d, 0x06, 0x96, 0xc2, 0x55, 0xea, 0xb3, 0x12, 0xf2, 0x14, 0x1d, 0x36, 0x3d, 0x87, - 0x3b, 0x09, 0xb1, 0x62, 0x5e, 0x8d, 0xc0, 0x94, 0x8e, 0xc5, 0x2b, 0x57, 0x18, 0x61, 0xa7, 0xc1, - 0xa1, 0xc3, 0x70, 0x93, 0x71, 0x1a, 0xbb, 0x3c, 0xa6, 0xd8, 0x53, 0xd8, 0xbb, 0xd9, 0x58, 0x1c, - 0xba, 0xc4, 0xc3, 0x5e, 0xd7, 0x73, 0x78, 0xdc, 0x57, 0xf0, 0xf5, 0xec, 0x1e, 0xa7, 0x4c, 0xfe, - 0xd7, 0x23, 0x3d, 0x22, 0x8f, 0x4d, 0x71, 0x4a, 0xaa, 0xe6, 0x12, 0x5c, 0x3c, 0x20, 0x24, 0xda, - 0x26, 0x14, 0x77, 0x22, 0xec, 0x9a, 0x3b, 0x70, 0xf9, 0x85, 0x73, 0x18, 0x60, 0x1b, 0x3b, 0x1e, - 0xa6, 0x9d, 0xc8, 0x09, 0xd1, 0x26, 0xd4, 0x59, 0xe4, 0x84, 0x06, 0xa8, 0x83, 0xc6, 0xc2, 0xd6, - 0x0d, 0x6b, 0x92, 0x92, 0x6a, 0xd4, 0x12, 0xb0, 0x96, 0x3e, 0x3c, 0xaf, 0xcd, 0xd9, 0x12, 0x6a, - 0x7e, 0xd0, 0x2e, 0xc9, 0x60, 0x17, 0xb5, 0x60, 0x9e, 0x8b, 0x92, 0xd2, 0xb9, 0x65, 0xcd, 0xa6, - 0xad, 0x1a, 0xb4, 0x24, 0x6d, 0x07, 0x33, 0x97, 0xfa, 0x11, 0x27, 0x54, 0xc9, 0x26, 0x54, 0xb4, - 0x0a, 0xe7, 0xfd, 0xd0, 0xc3, 0x83, 0xae, 0xef, 0x0d, 0x8c, 0x5c, 0x1d, 0x34, 0xca, 0xea, 0xbe, - 0x24, 0xcb, 0x6d, 0x6f, 0x80, 0xaa, 0xb0, 0x48, 0xf1, 0x1b, 0x4c, 0x19, 0x36, 0xb4, 0x3a, 0x68, - 0x94, 0x14, 0x20, 0x2d, 0x0a, 0x1b, 0xc2, 0x22, 0x33, 0xf4, 0xba, 0x76, 0x85, 0x0d, 0x15, 0x9c, - 0x75, 0x29, 0x84, 0xd4, 0x86, 0xa4, 0xa2, 0x87, 0xb0, 0x70, 0xe4, 0x07, 0x1c, 0x53, 0x23, 0x2f, - 0x7b, 0x59, 0xcd, 0x10, 0xd9, 0x1d, 0x44, 0x14, 0x33, 0xe6, 0x93, 0x94, 0xaf, 0x68, 0x68, 0x03, - 0x2e, 0x91, 0x98, 0x47, 0x31, 0xef, 0xba, 0x24, 0x88, 0xfb, 0x21, 0x33, 0x0a, 0x75, 0xad, 0x51, - 0x6e, 0xe5, 0x56, 0x80, 0x5d, 0x4e, 0x6e, 0xb6, 0x93, 0x0b, 0xb4, 0x06, 0x21, 0x23, 0x47, 0xbc, - 0x1b, 0xf8, 0x7d, 0x9f, 0x1b, 0xc5, 0x3a, 0x68, 0x68, 0x4a, 0x6c, 0x5e, 0xd4, 0xf7, 0x45, 0x59, - 0x80, 0x8e, 0x1d, 0xea, 0x29, 0x50, 0x69, 0x1a, 0x24, 0xea, 0x12, 0x64, 0x5e, 0x00, 0xb8, 0xb4, - 0x47, 0xfc, 0xf0, 0xdf, 0xcf, 0x64, 0x92, 0x97, 0xf6, 0xb7, 0xf2, 0xd2, 0x33, 0xf2, 0x32, 0x3f, - 0x03, 0x08, 0x3b, 0x84, 0x72, 0xd5, 0xe1, 0x01, 0x5c, 0x56, 0x4c, 0x42, 0x3d, 0x4c, 0xfd, 0xb0, - 0xa7, 0x7a, 0xad, 0x65, 0x78, 0x78, 0xa6, 0x60, 0xca, 0x81, 0xfa, 0xbf, 0x69, 0x15, 0x6d, 0x41, - 0x94, 0x0a, 0x75, 0xfb, 0x0e, 0x77, 0x8f, 0xbb, 0x01, 0x0e, 0x67, 0xda, 0x5e, 0x49, 0xef, 0x9f, - 0x8a, 0xeb, 0x7d, 0x1c, 0xa2, 0x0a, 0xcc, 0x27, 0x83, 0xd1, 0xa6, 0x06, 0x93, 0x94, 0xcc, 0x8f, - 0x39, 0x88, 0x9e, 0xa7, 0x1f, 0x22, 0xf1, 0x0a, 0x5f, 0x86, 0x3e, 0x09, 0xd1, 0x3d, 0xa8, 0x87, - 0x84, 0x44, 0xca, 0xeb, 0x5a, 0x86, 0xd7, 0xe9, 0x97, 0x6b, 0x4b, 0x02, 0x7a, 0x02, 0x17, 0xf8, - 0x64, 0x75, 0xa5, 0xb1, 0x3f, 0x5c, 0x72, 0xec, 0xda, 0xd3, 0x54, 0xb4, 0x0b, 0xe1, 0xeb, 0xf1, - 0xb6, 0xa8, 0xc1, 0xdd, 0xcc, 0x10, 0x9a, 0x5d, 0x2b, 0x7b, 0x8a, 0x88, 0xee, 0xc3, 0x02, 0x93, - 0xe3, 0x30, 0xf4, 0x6b, 0x67, 0x3f, 0x99, 0x99, 0xad, 0x08, 0x0f, 0xf4, 0xe1, 0xa7, 0x1a, 0x30, - 0xbf, 0x03, 0x58, 0x1e, 0x27, 0x24, 0x67, 0xfa, 0x08, 0xe6, 0xfd, 0x30, 0x8a, 0xb9, 0x01, 0xe4, - 0x13, 0x5e, 0xcf, 0x50, 0x6c, 0x0b, 0x4c, 0xe7, 0x6d, 0xe8, 0x0a, 0x52, 0x9a, 0xba, 0x24, 0xa2, - 0x6d, 0xa8, 0xbb, 0x84, 0x62, 0x15, 0xcf, 0x46, 0x86, 0xc0, 0xef, 0x73, 0x49, 0x3f, 0x72, 0x82, - 0x8c, 0x76, 0x61, 0x21, 0x59, 0x0e, 0x43, 0x93, 0x3e, 0x6e, 0x67, 0x6d, 0x94, 0x04, 0xd9, 0x24, - 0x56, 0xfd, 0xa5, 0xbb, 0x9d, 0x90, 0xcd, 0x77, 0x00, 0x96, 0x1e, 0x07, 0xe4, 0x4c, 0xb6, 0xb6, - 0x09, 0x8b, 0x47, 0x01, 0x39, 0xeb, 0xfa, 0x9e, 0x1c, 0xfd, 0x62, 0xcb, 0x10, 0xd8, 0x6f, 0xe7, - 0xb5, 0x82, 0x80, 0xb4, 0x77, 0x46, 0xe3, 0x93, 0x5d, 0x10, 0xc0, 0xb6, 0x87, 0xf6, 0x20, 0x9c, - 0xfc, 0x92, 0x19, 0xb9, 0x6b, 0x23, 0x99, 0xc9, 0x51, 0xf9, 0x98, 0x62, 0xb7, 0x56, 0x87, 0x3f, - 0xab, 0x73, 0xc3, 0x51, 0x15, 0x7c, 0x19, 0x55, 0xc1, 0xd7, 0x51, 0x15, 0xfc, 0x18, 0x55, 0xc1, - 0xfb, 0x8b, 0xea, 0xdc, 0xab, 0xa2, 0x92, 0xf8, 0x15, 0x00, 0x00, 0xff, 0xff, 0x88, 0x10, 0xa0, - 0xaf, 0x51, 0x07, 0x00, 0x00, + // 830 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xbc, 0x55, 0x41, 0x8f, 0xdb, 0x44, + 0x14, 0xde, 0x59, 0x3b, 0xd9, 0xdd, 0xd7, 0x26, 0xad, 0x46, 0x20, 0xac, 0x1c, 0x12, 0xaf, 0xbb, + 0x40, 0x8a, 0x84, 0xa3, 0xf6, 0x82, 0x40, 0x20, 0x50, 0x76, 0x83, 0x48, 0x29, 0x05, 0x39, 0xe5, + 0xc2, 0x25, 0xf2, 0xda, 0xb3, 0x59, 0x53, 0xc7, 0x33, 0x3b, 0x33, 0x6e, 0xb3, 0x3f, 0x01, 0x89, + 0x03, 0xe2, 0x17, 0xf0, 0x03, 0xf8, 0x21, 0x39, 0x72, 0x44, 0x48, 0x54, 0x90, 0xfd, 0x23, 0x68, + 0xc6, 0xe3, 0xc4, 0x29, 0x6b, 0xd8, 0x03, 0xe2, 0x94, 0xd1, 0x9b, 0xef, 0xfb, 0xf2, 0xde, 0xf7, + 0xde, 0x1b, 0xc3, 0x3b, 0x11, 0x8d, 0x9e, 0x71, 0x1a, 0x46, 0xe7, 0x03, 0xf6, 0x6c, 0x36, 0x10, + 0x17, 0xe9, 0x20, 0x4e, 0x84, 0x54, 0xbf, 0x8c, 0xd3, 0x88, 0x08, 0x41, 0xb9, 0xf0, 0x19, 0xa7, + 0x92, 0xe2, 0xd7, 0xd7, 0x58, 0x5f, 0x5c, 0xa4, 0xbe, 0xc1, 0x75, 0xdc, 0x6d, 0x09, 0x7d, 0x62, + 0xa7, 0x83, 0x38, 0x94, 0x61, 0x41, 0xec, 0x78, 0xd7, 0x23, 0x08, 0xe7, 0x6b, 0xf1, 0xce, 0x35, + 0x89, 0x88, 0x8b, 0xf4, 0x34, 0x14, 0x64, 0x20, 0x24, 0xcf, 0x23, 0x99, 0x73, 0x12, 0x1b, 0xec, + 0xbb, 0xf5, 0x58, 0x92, 0x45, 0x34, 0x26, 0xf1, 0x34, 0x0e, 0x65, 0x3e, 0x37, 0xf0, 0xa3, 0xfa, + 0x1a, 0x2b, 0x49, 0xbe, 0x36, 0xa3, 0x33, 0xaa, 0x8f, 0x03, 0x75, 0x2a, 0xa2, 0x5e, 0x1b, 0x6e, + 0x3f, 0xa1, 0x94, 0x1d, 0x53, 0x4e, 0x26, 0x8c, 0x44, 0xde, 0x09, 0xdc, 0x79, 0x1a, 0x9e, 0xa6, + 0x24, 0x20, 0x61, 0x4c, 0xf8, 0x84, 0x85, 0x19, 0x7e, 0x00, 0xb6, 0x60, 0x61, 0xe6, 0x20, 0x17, + 0xf5, 0x6f, 0x3d, 0x7c, 0xc3, 0xdf, 0xb8, 0x64, 0x0a, 0xf5, 0x15, 0x6c, 0x68, 0x2f, 0x5f, 0xf6, + 0x76, 0x02, 0x0d, 0xf5, 0x7e, 0xb4, 0x5e, 0x91, 0x21, 0x11, 0x1e, 0x42, 0x43, 0xaa, 0x90, 0xd1, + 0x79, 0xcb, 0xdf, 0x76, 0xdb, 0x14, 0xe8, 0x6b, 0xda, 0x09, 0x11, 0x11, 0x4f, 0x98, 0xa4, 0xdc, + 0xc8, 0x16, 0x54, 0x7c, 0x08, 0x07, 0x49, 0x16, 0x93, 0xc5, 0x34, 0x89, 0x17, 0xce, 0xae, 0x8b, + 0xfa, 0x2d, 0x73, 0xbf, 0xaf, 0xc3, 0xe3, 0x78, 0x81, 0xbb, 0xb0, 0xc7, 0xc9, 0x73, 0xc2, 0x05, + 0x71, 0x2c, 0x17, 0xf5, 0xf7, 0x0d, 0xa0, 0x0c, 0xaa, 0x34, 0x54, 0x8a, 0xc2, 0xb1, 0x5d, 0xeb, + 0x9a, 0x34, 0x8c, 0x71, 0xfe, 0x2b, 0x26, 0x94, 0x69, 0x68, 0x2a, 0xfe, 0x18, 0x9a, 0x67, 0x49, + 0x2a, 0x09, 0x77, 0x1a, 0xba, 0x96, 0xc3, 0x1a, 0x91, 0xd1, 0x82, 0x71, 0x22, 0x44, 0x42, 0x4b, + 0xbe, 0xa1, 0xe1, 0xfb, 0xd0, 0xa6, 0xb9, 0x64, 0xb9, 0x9c, 0x46, 0x34, 0xcd, 0xe7, 0x99, 0x70, + 0x9a, 0xae, 0xd5, 0x6f, 0x0d, 0x77, 0xef, 0xa2, 0xa0, 0x55, 0xdc, 0x1c, 0x17, 0x17, 0xf8, 0x1e, + 0x80, 0xa0, 0x67, 0x72, 0x9a, 0x26, 0xf3, 0x44, 0x3a, 0x7b, 0x2e, 0xea, 0x5b, 0x46, 0xec, 0x40, + 0xc5, 0x1f, 0xab, 0xb0, 0x02, 0x9d, 0x87, 0x3c, 0x36, 0xa0, 0xfd, 0x2a, 0x48, 0xc5, 0x35, 0xc8, + 0xbb, 0x42, 0xd0, 0x7e, 0x44, 0x93, 0xec, 0xff, 0xef, 0xc9, 0xc6, 0x2f, 0xeb, 0xbf, 0xf2, 0xcb, + 0xae, 0xf1, 0xcb, 0xfb, 0x19, 0x01, 0x4c, 0x28, 0x97, 0xa6, 0xc2, 0x27, 0x70, 0xc7, 0x30, 0x29, + 0x8f, 0x09, 0x4f, 0xb2, 0x99, 0xa9, 0xb5, 0x57, 0x93, 0xc3, 0x97, 0x06, 0x66, 0x32, 0x30, 0xff, + 0x5b, 0x46, 0xf1, 0x43, 0xc0, 0xa5, 0xd0, 0x74, 0x1e, 0xca, 0xe8, 0x7c, 0x9a, 0x92, 0x6c, 0xab, + 0xec, 0xbb, 0xe5, 0xfd, 0x17, 0xea, 0xfa, 0x31, 0xc9, 0x70, 0x07, 0x1a, 0x45, 0x63, 0xac, 0x4a, + 0x63, 0x8a, 0x90, 0xf7, 0x3d, 0x82, 0xd6, 0xe8, 0x79, 0x98, 0xe6, 0xa1, 0xa4, 0x45, 0xc6, 0x1f, + 0x42, 0x43, 0x5e, 0x32, 0x22, 0x1c, 0xe4, 0x5a, 0xfd, 0x76, 0x6d, 0x4f, 0x8a, 0x7a, 0x9f, 0x5e, + 0x32, 0xe2, 0x7f, 0x9e, 0x64, 0x71, 0x50, 0x90, 0xf0, 0x47, 0xd0, 0x20, 0x0b, 0xc6, 0x85, 0xb3, + 0xab, 0xc7, 0xfb, 0xc6, 0x4e, 0x17, 0x2c, 0xef, 0x6a, 0x17, 0xf0, 0x57, 0xe5, 0xbb, 0xa8, 0x1e, + 0x85, 0xaf, 0xb3, 0x84, 0x66, 0xf8, 0x3d, 0xb0, 0x33, 0x4a, 0x99, 0xb1, 0xee, 0x5e, 0x8d, 0x68, + 0xf5, 0x21, 0x09, 0x34, 0x01, 0x7f, 0x06, 0xb7, 0xe4, 0x66, 0x93, 0xb4, 0x4f, 0x37, 0xdc, 0x39, + 0x12, 0x05, 0x55, 0x2a, 0x1e, 0x01, 0x7c, 0xbb, 0x1e, 0x5e, 0x33, 0x47, 0x6f, 0xd6, 0x08, 0x6d, + 0x4f, 0x79, 0x50, 0x21, 0xe2, 0xf7, 0xa1, 0x29, 0xf4, 0x74, 0x38, 0xf6, 0x3f, 0x8e, 0xe2, 0x66, + 0x84, 0x02, 0x43, 0xc0, 0x43, 0x38, 0x20, 0x65, 0xa7, 0xcc, 0xe2, 0x1f, 0xd5, 0xd9, 0x5b, 0xed, + 0x68, 0xb0, 0xa1, 0x7d, 0x60, 0x2f, 0x7f, 0xea, 0x21, 0xef, 0x77, 0x04, 0xad, 0xb5, 0xcb, 0xba, + 0xe9, 0x9f, 0x40, 0x23, 0xc9, 0x58, 0x2e, 0x75, 0xd3, 0xeb, 0x75, 0xc7, 0x0a, 0x33, 0xb9, 0xcc, + 0x22, 0x45, 0x2a, 0x3b, 0xa7, 0x89, 0xf8, 0x18, 0xec, 0x88, 0x72, 0x62, 0x2c, 0xbe, 0x5f, 0x23, + 0xf0, 0xf7, 0xde, 0x96, 0xef, 0xb6, 0x22, 0xe3, 0x11, 0x34, 0x8b, 0x79, 0x77, 0x2c, 0x9d, 0xc7, + 0xdb, 0x75, 0x4b, 0xa2, 0x41, 0x01, 0xcd, 0x8d, 0x47, 0xe5, 0xba, 0x16, 0x64, 0xef, 0x3b, 0x04, + 0xfb, 0x9f, 0xa6, 0xf4, 0x85, 0x2e, 0xed, 0x01, 0xec, 0x9d, 0xa5, 0xf4, 0xc5, 0x34, 0x89, 0xf5, + 0xf8, 0xdc, 0x1e, 0x3a, 0x0a, 0xfb, 0xdb, 0xcb, 0x5e, 0x53, 0x41, 0xc6, 0x27, 0xab, 0xf5, 0x29, + 0x68, 0x2a, 0xe0, 0x38, 0xc6, 0x8f, 0x00, 0x36, 0x1f, 0x67, 0x33, 0xc9, 0x47, 0xff, 0x56, 0x51, + 0x25, 0x8f, 0x0a, 0x7b, 0x78, 0xb8, 0xfc, 0xb3, 0xbb, 0xb3, 0x5c, 0x75, 0xd1, 0x2f, 0xab, 0x2e, + 0xfa, 0x75, 0xd5, 0x45, 0x7f, 0xac, 0xba, 0xe8, 0x87, 0xab, 0xee, 0xce, 0x37, 0x7b, 0x46, 0xe2, + 0xaf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x16, 0xdc, 0x3d, 0xf7, 0x24, 0x08, 0x00, 0x00, } diff --git a/pkg/sql/distsql/processors.proto b/pkg/sql/distsql/processors.proto index 09d51d333705..d2e70e6adafb 100644 --- a/pkg/sql/distsql/processors.proto +++ b/pkg/sql/distsql/processors.proto @@ -13,6 +13,7 @@ // permissions and limitations under the License. // // Author: Radu Berinde (radu@cockroachlabs.com) +// Author: Irfan Sharif (irfansharif@cockroachlabs.com) // // Processor definitions for distributed SQL APIs. See // docs/RFCS/distributed_sql.md. @@ -110,6 +111,23 @@ message SorterSpec { optional int64 limit = 3 [(gogoproto.nullable) = false]; } +// EvaluatorSpec is the specification for an "evaluator", a fully +// programmable no-grouping aggregator. It runs a 'program' on each individual +// row and is restricted to operating on one row of data at a time. +// The 'program' is a set of expressions evaluated in order, the output +// schema therefore consists of the results of evaluating each of these +// expressions on the input row. +// +// TODO(irfansharif): Add support for an optional output filter expression. +// The filter expression would reference the columns in the row via $0, $1, +// etc., possibly optimizing if filtering on expressions common to the +// 'program'. +message EvaluatorSpec { + repeated sqlbase.ColumnType.Kind types = 1; + + repeated Expression exprs = 2 [(gogoproto.nullable) = false]; +} + message ProcessorCoreUnion { option (gogoproto.onlyone) = true; @@ -117,8 +135,9 @@ message ProcessorCoreUnion { optional TableReaderSpec tableReader = 2; optional JoinReaderSpec joinReader = 3; optional SorterSpec sorter = 4; + optional EvaluatorSpec evaluator = 5; // TODO(radu): other "processor core" types will go here. - // TODO(irfansharif): add aggregation, join, etc. from #7587 + // TODO(irfansharif): add aggregation, join, set operations, etc. from #7587 } message ProcessorSpec { diff --git a/pkg/sql/sqlbase/encoded_datum.go b/pkg/sql/sqlbase/encoded_datum.go index 000101462d0e..bd2552b7da27 100644 --- a/pkg/sql/sqlbase/encoded_datum.go +++ b/pkg/sql/sqlbase/encoded_datum.go @@ -19,6 +19,7 @@ package sqlbase import ( "bytes" "fmt" + "strings" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -224,6 +225,19 @@ func (r EncDatumRow) String() string { return b.String() } +// DatumToEncDatum converts a parser.Datum to an EncDatum. +func DatumToEncDatum(datum parser.Datum) (EncDatum, error) { + dType, ok := ColumnType_Kind_value[strings.ToUpper(datum.Type())] + if !ok { + return EncDatum{}, errors.Errorf( + "Unknown type %s, could not convert to EncDatum", datum.Type()) + } + return EncDatum{ + Type: (ColumnType_Kind)(dType), + Datum: datum, + }, nil +} + // Compare returns the relative ordering of two EncDatumRows according to a // ColumnOrdering: // -1 if the receiver comes before the rhs in the ordering,