|
@@ -6,6 +6,7 @@ import (
|
|
|
"github.com/emqx/kuiper/common/plugin_manager"
|
|
|
"github.com/emqx/kuiper/xstream/api"
|
|
|
"math"
|
|
|
+ "reflect"
|
|
|
"sort"
|
|
|
"strings"
|
|
|
"time"
|
|
@@ -270,6 +271,14 @@ type FieldRef struct {
|
|
|
func (fr *FieldRef) expr() {}
|
|
|
func (fr *FieldRef) node() {}
|
|
|
|
|
|
+type MetaRef struct {
|
|
|
+ StreamName StreamName
|
|
|
+ Name string
|
|
|
+}
|
|
|
+
|
|
|
+func (fr *MetaRef) expr() {}
|
|
|
+func (fr *MetaRef) node() {}
|
|
|
+
|
|
|
// The stream AST tree
|
|
|
type Options map[string]string
|
|
|
|
|
@@ -287,6 +296,9 @@ type StreamStmt struct {
|
|
|
|
|
|
func (ss *StreamStmt) node() {}
|
|
|
func (ss *StreamStmt) Stmt() {}
|
|
|
+func (ss *StreamStmt) isSchemaless() bool {
|
|
|
+ return ss.StreamFields == nil
|
|
|
+}
|
|
|
|
|
|
type FieldType interface {
|
|
|
fieldType()
|
|
@@ -456,6 +468,7 @@ func (fn walkFuncVisitor) Visit(n Node) Visitor { fn(n); return fn }
|
|
|
type Valuer interface {
|
|
|
// Value returns the value and existence flag for a given key.
|
|
|
Value(key string) (interface{}, bool)
|
|
|
+ Meta(key string) (interface{}, bool)
|
|
|
}
|
|
|
|
|
|
// CallValuer implements the Call method for evaluating function calls.
|
|
@@ -499,6 +512,10 @@ func (wv *WildcardValuer) Value(key string) (interface{}, bool) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+func (wv *WildcardValuer) Meta(key string) (interface{}, bool) {
|
|
|
+ return nil, false
|
|
|
+}
|
|
|
+
|
|
|
/**********************************
|
|
|
** Various Data Types for SQL transformation
|
|
|
*/
|
|
@@ -524,12 +541,26 @@ func (m Message) Value(key string) (interface{}, bool) {
|
|
|
return nil, false
|
|
|
}
|
|
|
|
|
|
+func (m Message) Meta(key string) (interface{}, bool) {
|
|
|
+ return m.Value(key)
|
|
|
+}
|
|
|
+
|
|
|
type Event interface {
|
|
|
GetTimestamp() int64
|
|
|
IsWatermark() bool
|
|
|
}
|
|
|
|
|
|
-type Metadata map[string]interface{}
|
|
|
+type Metadata Message
|
|
|
+
|
|
|
+func (m Metadata) Value(key string) (interface{}, bool) {
|
|
|
+ msg := Message(m)
|
|
|
+ return msg.Value(key)
|
|
|
+}
|
|
|
+
|
|
|
+func (m Metadata) Meta(key string) (interface{}, bool) {
|
|
|
+ msg := Message(m)
|
|
|
+ return msg.Meta(key)
|
|
|
+}
|
|
|
|
|
|
type Tuple struct {
|
|
|
Emitter string
|
|
@@ -538,26 +569,12 @@ type Tuple struct {
|
|
|
Metadata Metadata
|
|
|
}
|
|
|
|
|
|
-// Value returns the value for a key in the Message.
|
|
|
-func (m Metadata) Value(key string) (interface{}, bool) {
|
|
|
- key = strings.ToLower(key)
|
|
|
- if keys := strings.Split(key, "."); len(keys) == 1 {
|
|
|
- v, ok := m[key]
|
|
|
- return v, ok
|
|
|
- } else if len(keys) == 2 {
|
|
|
- v, ok := m[keys[1]]
|
|
|
- return v, ok
|
|
|
- }
|
|
|
- common.Log.Println("Invalid key: " + key + ", expect source.field or field.")
|
|
|
- return nil, false
|
|
|
+func (t *Tuple) Value(key string) (interface{}, bool) {
|
|
|
+ return t.Message.Value(key)
|
|
|
}
|
|
|
|
|
|
-func (t *Tuple) Value(key string) (interface{}, bool) {
|
|
|
- if v, ok := t.Message.Value(key); ok {
|
|
|
- return v, ok
|
|
|
- } else {
|
|
|
- return t.Metadata.Value(key)
|
|
|
- }
|
|
|
+func (t *Tuple) Meta(key string) (interface{}, bool) {
|
|
|
+ return t.Metadata.Value(key)
|
|
|
}
|
|
|
|
|
|
func (t *Tuple) All(stream string) (interface{}, bool) {
|
|
@@ -671,14 +688,26 @@ func (jt *JoinTuple) AddTuples(tuples []Tuple) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (jt *JoinTuple) Value(key string) (interface{}, bool) {
|
|
|
+func getTupleValue(tuple Tuple, t string, key string) (interface{}, bool) {
|
|
|
+ switch t {
|
|
|
+ case "value":
|
|
|
+ return tuple.Value(key)
|
|
|
+ case "meta":
|
|
|
+ return tuple.Meta(key)
|
|
|
+ default:
|
|
|
+ common.Log.Errorf("cannot get tuple for type %s", t)
|
|
|
+ return nil, false
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+func (jt *JoinTuple) doGetValue(t string, key string) (interface{}, bool) {
|
|
|
keys := strings.Split(key, ".")
|
|
|
tuples := jt.Tuples
|
|
|
switch len(keys) {
|
|
|
case 1:
|
|
|
if len(tuples) > 1 {
|
|
|
for _, tuple := range tuples { //TODO support key without modifier?
|
|
|
- v, ok := tuple.Message[key]
|
|
|
+ v, ok := getTupleValue(tuple, t, key)
|
|
|
if ok {
|
|
|
return v, ok
|
|
|
}
|
|
@@ -686,16 +715,14 @@ func (jt *JoinTuple) Value(key string) (interface{}, bool) {
|
|
|
common.Log.Infoln("Wrong key: ", key, ", not found")
|
|
|
return nil, false
|
|
|
} else {
|
|
|
- v, ok := tuples[0].Message[key]
|
|
|
- return v, ok
|
|
|
+ return getTupleValue(tuples[0], t, key)
|
|
|
}
|
|
|
case 2:
|
|
|
emitter, key := keys[0], keys[1]
|
|
|
//TODO should use hash here
|
|
|
for _, tuple := range tuples {
|
|
|
if tuple.Emitter == emitter {
|
|
|
- v, ok := tuple.Message[key]
|
|
|
- return v, ok
|
|
|
+ return getTupleValue(tuple, t, key)
|
|
|
}
|
|
|
}
|
|
|
return nil, false
|
|
@@ -705,6 +732,14 @@ func (jt *JoinTuple) Value(key string) (interface{}, bool) {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+func (jt *JoinTuple) Value(key string) (interface{}, bool) {
|
|
|
+ return jt.doGetValue("value", key)
|
|
|
+}
|
|
|
+
|
|
|
+func (jt *JoinTuple) Meta(key string) (interface{}, bool) {
|
|
|
+ return jt.doGetValue("meta", key)
|
|
|
+}
|
|
|
+
|
|
|
func (jt *JoinTuple) All(stream string) (interface{}, bool) {
|
|
|
if stream != "" {
|
|
|
for _, t := range jt.Tuples {
|
|
@@ -766,6 +801,7 @@ type SortingData interface {
|
|
|
type MultiSorter struct {
|
|
|
SortingData
|
|
|
fields SortFields
|
|
|
+ values []map[string]interface{}
|
|
|
}
|
|
|
|
|
|
// OrderedBy returns a Sorter that sorts using the less functions, in order.
|
|
@@ -783,31 +819,104 @@ func OrderedBy(fields SortFields) *MultiSorter {
|
|
|
// -1, 0, 1 and reduce the number of calls for greater efficiency: an
|
|
|
// exercise for the reader.
|
|
|
func (ms *MultiSorter) Less(i, j int) bool {
|
|
|
- p, q := ms.SortingData.Index(i), ms.SortingData.Index(j)
|
|
|
- vep, veq := &ValuerEval{Valuer: MultiValuer(p, &FunctionValuer{})}, &ValuerEval{Valuer: MultiValuer(q, &FunctionValuer{})}
|
|
|
+ p, q := ms.values[i], ms.values[j]
|
|
|
+ v := &ValuerEval{Valuer: MultiValuer(&FunctionValuer{})}
|
|
|
for _, field := range ms.fields {
|
|
|
- vp, ok := vep.Valuer.Value(field.Name)
|
|
|
- if !ok {
|
|
|
- return !field.Ascending
|
|
|
- }
|
|
|
- vq, ok := veq.Valuer.Value(field.Name)
|
|
|
- if !ok {
|
|
|
- return !field.Ascending
|
|
|
+ n := field.Name
|
|
|
+ vp, _ := p[n]
|
|
|
+ vq, _ := q[n]
|
|
|
+ if vp == nil && vq != nil {
|
|
|
+ return false
|
|
|
+ } else if vp != nil && vq == nil {
|
|
|
+ ms.valueSwap(true, i, j)
|
|
|
+ return true
|
|
|
+ } else if vp == nil && vq == nil {
|
|
|
+ return false
|
|
|
}
|
|
|
switch {
|
|
|
- case vep.simpleDataEval(vp, vq, LT):
|
|
|
+ case v.simpleDataEval(vp, vq, LT):
|
|
|
+ ms.valueSwap(field.Ascending, i, j)
|
|
|
return field.Ascending
|
|
|
- case veq.simpleDataEval(vq, vp, LT):
|
|
|
+ case v.simpleDataEval(vq, vp, LT):
|
|
|
+ ms.valueSwap(!field.Ascending, i, j)
|
|
|
return !field.Ascending
|
|
|
}
|
|
|
}
|
|
|
return false
|
|
|
}
|
|
|
|
|
|
+func (ms *MultiSorter) valueSwap(s bool, i, j int) {
|
|
|
+ if s {
|
|
|
+ ms.values[i], ms.values[j] = ms.values[j], ms.values[i]
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
// Sort sorts the argument slice according to the less functions passed to OrderedBy.
|
|
|
-func (ms *MultiSorter) Sort(data SortingData) {
|
|
|
+func (ms *MultiSorter) Sort(data SortingData) error {
|
|
|
ms.SortingData = data
|
|
|
+ types := make([]string, len(ms.fields))
|
|
|
+ ms.values = make([]map[string]interface{}, data.Len())
|
|
|
+ //load and validate data
|
|
|
+ for i := 0; i < data.Len(); i++ {
|
|
|
+ ms.values[i] = make(map[string]interface{})
|
|
|
+ p := data.Index(i)
|
|
|
+ vep := &ValuerEval{Valuer: MultiValuer(p, &FunctionValuer{})}
|
|
|
+ for j, field := range ms.fields {
|
|
|
+ n := field.Name
|
|
|
+ vp, _ := vep.Valuer.Value(n)
|
|
|
+ if err, ok := vp.(error); ok {
|
|
|
+ return err
|
|
|
+ } else {
|
|
|
+ if types[j] == "" && vp != nil {
|
|
|
+ types[j] = fmt.Sprintf("%T", vp)
|
|
|
+ }
|
|
|
+ if err := validate(types[j], vp); err != nil {
|
|
|
+ return err
|
|
|
+ } else {
|
|
|
+ ms.values[i][n] = vp
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
sort.Sort(ms)
|
|
|
+ return nil
|
|
|
+}
|
|
|
+
|
|
|
+func validate(t string, v interface{}) error {
|
|
|
+ if v == nil || t == "" {
|
|
|
+ return nil
|
|
|
+ }
|
|
|
+ vt := fmt.Sprintf("%T", v)
|
|
|
+ switch t {
|
|
|
+ case "int", "int64", "float64", "uint64":
|
|
|
+ if vt == "int" || vt == "int64" || vt == "float64" || vt == "uint64" {
|
|
|
+ return nil
|
|
|
+ } else {
|
|
|
+ return fmt.Errorf("incompatible types for comparison: %s and %s", t, vt)
|
|
|
+ }
|
|
|
+ case "bool":
|
|
|
+ if vt == "bool" {
|
|
|
+ return nil
|
|
|
+ } else {
|
|
|
+ return fmt.Errorf("incompatible types for comparison: %s and %s", t, vt)
|
|
|
+ }
|
|
|
+ case "string":
|
|
|
+ if vt == "string" {
|
|
|
+ return nil
|
|
|
+ } else {
|
|
|
+ return fmt.Errorf("incompatible types for comparison: %s and %s", t, vt)
|
|
|
+ }
|
|
|
+ case "time.Time":
|
|
|
+ _, err := common.InterfaceToTime(v, "")
|
|
|
+ if err != nil {
|
|
|
+ return fmt.Errorf("incompatible types for comparison: %s and %s", t, vt)
|
|
|
+ } else {
|
|
|
+ return nil
|
|
|
+ }
|
|
|
+ default:
|
|
|
+ return fmt.Errorf("incompatible types for comparison: %s and %s", t, vt)
|
|
|
+ }
|
|
|
+ return nil
|
|
|
}
|
|
|
|
|
|
type EvalResultMessage struct {
|
|
@@ -850,13 +959,22 @@ func (a multiValuer) Value(key string) (interface{}, bool) {
|
|
|
return nil, false
|
|
|
}
|
|
|
|
|
|
+func (a multiValuer) Meta(key string) (interface{}, bool) {
|
|
|
+ for _, valuer := range a {
|
|
|
+ if v, ok := valuer.Meta(key); ok {
|
|
|
+ return v, true
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return nil, false
|
|
|
+}
|
|
|
+
|
|
|
func (a multiValuer) Call(name string, args []interface{}) (interface{}, bool) {
|
|
|
for _, valuer := range a {
|
|
|
if valuer, ok := valuer.(CallValuer); ok {
|
|
|
if v, ok := valuer.Call(name, args); ok {
|
|
|
return v, true
|
|
|
} else {
|
|
|
- common.Log.Println(fmt.Sprintf("Found error \"%s\" when call func %s.\n", v, name))
|
|
|
+ return fmt.Errorf("call func %s error: %v", name, v), false
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -864,33 +982,26 @@ func (a multiValuer) Call(name string, args []interface{}) (interface{}, bool) {
|
|
|
}
|
|
|
|
|
|
type multiAggregateValuer struct {
|
|
|
- data AggregateData
|
|
|
- valuers []Valuer
|
|
|
+ data AggregateData
|
|
|
+ multiValuer
|
|
|
}
|
|
|
|
|
|
func MultiAggregateValuer(data AggregateData, valuers ...Valuer) Valuer {
|
|
|
return &multiAggregateValuer{
|
|
|
- data: data,
|
|
|
- valuers: valuers,
|
|
|
+ data: data,
|
|
|
+ multiValuer: valuers,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-func (a *multiAggregateValuer) Value(key string) (interface{}, bool) {
|
|
|
- for _, valuer := range a.valuers {
|
|
|
- if v, ok := valuer.Value(key); ok {
|
|
|
- return v, true
|
|
|
- }
|
|
|
- }
|
|
|
- return nil, false
|
|
|
-}
|
|
|
-
|
|
|
//The args is [][] for aggregation
|
|
|
func (a *multiAggregateValuer) Call(name string, args []interface{}) (interface{}, bool) {
|
|
|
var singleArgs []interface{} = nil
|
|
|
- for _, valuer := range a.valuers {
|
|
|
+ for _, valuer := range a.multiValuer {
|
|
|
if a, ok := valuer.(AggregateCallValuer); ok {
|
|
|
if v, ok := a.Call(name, args); ok {
|
|
|
return v, true
|
|
|
+ } else {
|
|
|
+ return fmt.Errorf("call func %s error: %v", name, v), false
|
|
|
}
|
|
|
} else if c, ok := valuer.(CallValuer); ok {
|
|
|
if singleArgs == nil {
|
|
@@ -960,6 +1071,9 @@ func (v *ValuerEval) Eval(expr Expr) interface{} {
|
|
|
} else {
|
|
|
for i := range expr.Args {
|
|
|
args[i] = v.Eval(expr.Args[i])
|
|
|
+ if _, ok := args[i].(error); ok {
|
|
|
+ return args[i]
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -976,6 +1090,15 @@ func (v *ValuerEval) Eval(expr Expr) interface{} {
|
|
|
val, _ := v.Valuer.Value(string(expr.StreamName) + "." + expr.Name)
|
|
|
return val
|
|
|
}
|
|
|
+ case *MetaRef:
|
|
|
+ if expr.StreamName == "" {
|
|
|
+ val, _ := v.Valuer.Meta(expr.Name)
|
|
|
+ return val
|
|
|
+ } else {
|
|
|
+ //The field specified with stream source
|
|
|
+ val, _ := v.Valuer.Meta(string(expr.StreamName) + "." + expr.Name)
|
|
|
+ return val
|
|
|
+ }
|
|
|
case *Wildcard:
|
|
|
val, _ := v.Valuer.Value("")
|
|
|
return val
|
|
@@ -989,100 +1112,113 @@ func (v *ValuerEval) evalBinaryExpr(expr *BinaryExpr) interface{} {
|
|
|
switch val := lhs.(type) {
|
|
|
case map[string]interface{}:
|
|
|
return v.evalJsonExpr(val, expr.OP, expr.RHS)
|
|
|
- case []interface{}:
|
|
|
+ case []interface{}, []map[string]interface{}:
|
|
|
return v.evalJsonExpr(val, expr.OP, expr.RHS)
|
|
|
+ case error:
|
|
|
+ return val
|
|
|
}
|
|
|
|
|
|
rhs := v.Eval(expr.RHS)
|
|
|
- if lhs == nil && rhs != nil {
|
|
|
- // When the LHS is nil and the RHS is a boolean, implicitly cast the
|
|
|
- // nil to false.
|
|
|
- if _, ok := rhs.(bool); ok {
|
|
|
- lhs = false
|
|
|
- }
|
|
|
- } else if lhs != nil && rhs == nil {
|
|
|
- // Implicit cast of the RHS nil to false when the LHS is a boolean.
|
|
|
- if _, ok := lhs.(bool); ok {
|
|
|
- rhs = false
|
|
|
- }
|
|
|
+ if _, ok := rhs.(error); ok {
|
|
|
+ return rhs
|
|
|
}
|
|
|
return v.simpleDataEval(lhs, rhs, expr.OP)
|
|
|
}
|
|
|
|
|
|
func (v *ValuerEval) evalJsonExpr(result interface{}, op Token, expr Expr) interface{} {
|
|
|
- if val, ok := result.(map[string]interface{}); ok {
|
|
|
+ switch val := result.(type) {
|
|
|
+ case map[string]interface{}:
|
|
|
switch op {
|
|
|
case ARROW:
|
|
|
- if exp, ok := expr.(*FieldRef); ok {
|
|
|
+ switch e := expr.(type) {
|
|
|
+ case *FieldRef, *MetaRef:
|
|
|
ve := &ValuerEval{Valuer: Message(val)}
|
|
|
- return ve.Eval(exp)
|
|
|
- } else {
|
|
|
- fmt.Printf("The right expression is not a field reference node.\n")
|
|
|
- return nil
|
|
|
+ return ve.Eval(e)
|
|
|
+ default:
|
|
|
+ return fmt.Errorf("the right expression is not a field reference node")
|
|
|
}
|
|
|
default:
|
|
|
- fmt.Printf("%v is an invalid operation.\n", op)
|
|
|
- return nil
|
|
|
+ return fmt.Errorf("%v is an invalid operation for %T", op, val)
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- if val, ok := result.([]interface{}); ok {
|
|
|
+ case []interface{}, []map[string]interface{}:
|
|
|
switch op {
|
|
|
case SUBSET:
|
|
|
+ val := reflect.ValueOf(result)
|
|
|
ber := v.Eval(expr)
|
|
|
if berVal, ok1 := ber.(*BracketEvalResult); ok1 {
|
|
|
if berVal.isIndex() {
|
|
|
- if berVal.Start >= len(val) {
|
|
|
- fmt.Printf("Out of index: %d of %d.\n", berVal.Start, len(val))
|
|
|
- return nil
|
|
|
+ if berVal.Start >= val.Len() {
|
|
|
+ return fmt.Errorf("out of index: %d of %d", berVal.Start, val.Len())
|
|
|
}
|
|
|
- return val[berVal.Start]
|
|
|
+ return val.Index(berVal.Start).Interface()
|
|
|
} else {
|
|
|
- if berVal.Start >= len(val) {
|
|
|
- fmt.Printf("Start value is out of index: %d of %d.\n", berVal.Start, len(val))
|
|
|
- return nil
|
|
|
+ if berVal.Start >= val.Len() {
|
|
|
+ return fmt.Errorf("start value is out of index: %d of %d", berVal.Start, val.Len())
|
|
|
}
|
|
|
|
|
|
- if berVal.End >= len(val) {
|
|
|
- fmt.Printf("End value is out of index: %d of %d.\n", berVal.End, len(val))
|
|
|
- return nil
|
|
|
+ if berVal.End >= val.Len() {
|
|
|
+ return fmt.Errorf("end value is out of index: %d of %d", berVal.End, val.Len())
|
|
|
}
|
|
|
- return val[berVal.Start:berVal.End]
|
|
|
+ return val.Slice(berVal.Start, berVal.End).Interface()
|
|
|
}
|
|
|
} else {
|
|
|
- fmt.Printf("Invalid evaluation result - %v.\n", berVal)
|
|
|
- return nil
|
|
|
+ return fmt.Errorf("invalid evaluation result - %v", berVal)
|
|
|
}
|
|
|
default:
|
|
|
- fmt.Printf("%v is an invalid operation.\n", op)
|
|
|
- return nil
|
|
|
+ return fmt.Errorf("%v is an invalid operation for %T", op, val)
|
|
|
}
|
|
|
}
|
|
|
return nil
|
|
|
}
|
|
|
|
|
|
+//lhs and rhs are non-nil
|
|
|
func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{} {
|
|
|
+ if lhs == nil || rhs == nil {
|
|
|
+ switch op {
|
|
|
+ case EQ, LTE, GTE:
|
|
|
+ if lhs == nil && rhs == nil {
|
|
|
+ return true
|
|
|
+ } else {
|
|
|
+ return false
|
|
|
+ }
|
|
|
+ case NEQ:
|
|
|
+ if lhs == nil && rhs == nil {
|
|
|
+ return false
|
|
|
+ } else {
|
|
|
+ return true
|
|
|
+ }
|
|
|
+ case LT, GT:
|
|
|
+ return false
|
|
|
+ default:
|
|
|
+ return nil
|
|
|
+ }
|
|
|
+ }
|
|
|
lhs = convertNum(lhs)
|
|
|
rhs = convertNum(rhs)
|
|
|
// Evaluate if both sides are simple types.
|
|
|
switch lhs := lhs.(type) {
|
|
|
case bool:
|
|
|
rhs, ok := rhs.(bool)
|
|
|
+ if !ok {
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
+ }
|
|
|
switch op {
|
|
|
case AND:
|
|
|
- return ok && (lhs && rhs)
|
|
|
+ return lhs && rhs
|
|
|
case OR:
|
|
|
- return ok && (lhs || rhs)
|
|
|
+ return lhs || rhs
|
|
|
case BITWISE_AND:
|
|
|
- return ok && (lhs && rhs)
|
|
|
+ return lhs && rhs
|
|
|
case BITWISE_OR:
|
|
|
- return ok && (lhs || rhs)
|
|
|
+ return lhs || rhs
|
|
|
case BITWISE_XOR:
|
|
|
- return ok && (lhs != rhs)
|
|
|
+ return lhs != rhs
|
|
|
case EQ:
|
|
|
- return ok && (lhs == rhs)
|
|
|
+ return lhs == rhs
|
|
|
case NEQ:
|
|
|
- return ok && (lhs != rhs)
|
|
|
+ return lhs != rhs
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case float64:
|
|
|
// Try the rhs as a float64, int64, or uint64
|
|
@@ -1095,48 +1231,41 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
rhsf, ok = float64(val), true
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+ if !ok {
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
+ }
|
|
|
rhs := rhsf
|
|
|
switch op {
|
|
|
case EQ:
|
|
|
- return ok && (lhs == rhs)
|
|
|
+ return lhs == rhs
|
|
|
case NEQ:
|
|
|
- return ok && (lhs != rhs)
|
|
|
+ return lhs != rhs
|
|
|
case LT:
|
|
|
- return ok && (lhs < rhs)
|
|
|
+ return lhs < rhs
|
|
|
case LTE:
|
|
|
- return ok && (lhs <= rhs)
|
|
|
+ return lhs <= rhs
|
|
|
case GT:
|
|
|
- return ok && (lhs > rhs)
|
|
|
+ return lhs > rhs
|
|
|
case GTE:
|
|
|
- return ok && (lhs >= rhs)
|
|
|
+ return lhs >= rhs
|
|
|
case ADD:
|
|
|
- if !ok {
|
|
|
- return nil
|
|
|
- }
|
|
|
return lhs + rhs
|
|
|
case SUB:
|
|
|
- if !ok {
|
|
|
- return nil
|
|
|
- }
|
|
|
return lhs - rhs
|
|
|
case MUL:
|
|
|
- if !ok {
|
|
|
- return nil
|
|
|
- }
|
|
|
return lhs * rhs
|
|
|
case DIV:
|
|
|
- if !ok {
|
|
|
- return nil
|
|
|
- } else if rhs == 0 {
|
|
|
- return float64(0)
|
|
|
+ if rhs == 0 {
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / rhs
|
|
|
case MOD:
|
|
|
- if !ok {
|
|
|
- return nil
|
|
|
+ if rhs == 0 {
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return math.Mod(lhs, rhs)
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case int64:
|
|
|
// Try as a float64 to see if a float cast is required.
|
|
@@ -1164,11 +1293,16 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs * rhs
|
|
|
case DIV:
|
|
|
if rhs == 0 {
|
|
|
- return float64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / rhs
|
|
|
case MOD:
|
|
|
+ if rhs == 0 {
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
+ }
|
|
|
return math.Mod(lhs, rhs)
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case int64:
|
|
|
switch op {
|
|
@@ -1193,18 +1327,18 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
case DIV:
|
|
|
if v.IntegerFloatDivision {
|
|
|
if rhs == 0 {
|
|
|
- return float64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return float64(lhs) / float64(rhs)
|
|
|
}
|
|
|
|
|
|
if rhs == 0 {
|
|
|
- return int64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / rhs
|
|
|
case MOD:
|
|
|
if rhs == 0 {
|
|
|
- return int64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs % rhs
|
|
|
case BITWISE_AND:
|
|
@@ -1213,6 +1347,8 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs | rhs
|
|
|
case BITWISE_XOR:
|
|
|
return lhs ^ rhs
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case uint64:
|
|
|
switch op {
|
|
@@ -1248,12 +1384,12 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return uint64(lhs) * rhs
|
|
|
case DIV:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return uint64(lhs) / rhs
|
|
|
case MOD:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return uint64(lhs) % rhs
|
|
|
case BITWISE_AND:
|
|
@@ -1262,7 +1398,11 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return uint64(lhs) | rhs
|
|
|
case BITWISE_XOR:
|
|
|
return uint64(lhs) ^ rhs
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case uint64:
|
|
|
// Try as a float64 to see if a float cast is required.
|
|
@@ -1290,11 +1430,16 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs * rhs
|
|
|
case DIV:
|
|
|
if rhs == 0 {
|
|
|
- return float64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / rhs
|
|
|
case MOD:
|
|
|
+ if rhs == 0 {
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
+ }
|
|
|
return math.Mod(lhs, rhs)
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case int64:
|
|
|
switch op {
|
|
@@ -1330,12 +1475,12 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs * uint64(rhs)
|
|
|
case DIV:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / uint64(rhs)
|
|
|
case MOD:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs % uint64(rhs)
|
|
|
case BITWISE_AND:
|
|
@@ -1344,6 +1489,8 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs | uint64(rhs)
|
|
|
case BITWISE_XOR:
|
|
|
return lhs ^ uint64(rhs)
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case uint64:
|
|
|
switch op {
|
|
@@ -1367,12 +1514,12 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs * rhs
|
|
|
case DIV:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs / rhs
|
|
|
case MOD:
|
|
|
if rhs == 0 {
|
|
|
- return uint64(0)
|
|
|
+ return fmt.Errorf("divided by zero")
|
|
|
}
|
|
|
return lhs % rhs
|
|
|
case BITWISE_AND:
|
|
@@ -1381,51 +1528,37 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs | rhs
|
|
|
case BITWISE_XOR:
|
|
|
return lhs ^ rhs
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case string:
|
|
|
+ rhss, ok := rhs.(string)
|
|
|
+ if !ok {
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
+ }
|
|
|
switch op {
|
|
|
case EQ:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs == rhs
|
|
|
+ return lhs == rhss
|
|
|
case NEQ:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs != rhs
|
|
|
+ return lhs != rhss
|
|
|
case LT:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs < rhs
|
|
|
+ return lhs < rhss
|
|
|
case LTE:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs <= rhs
|
|
|
+ return lhs <= rhss
|
|
|
case GT:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs > rhs
|
|
|
+ return lhs > rhss
|
|
|
case GTE:
|
|
|
- rhs, ok := rhs.(string)
|
|
|
- if !ok {
|
|
|
- return false
|
|
|
- }
|
|
|
- return lhs >= rhs
|
|
|
+ return lhs >= rhss
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
case time.Time:
|
|
|
rt, err := common.InterfaceToTime(rhs, "")
|
|
|
if err != nil {
|
|
|
- return false
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
switch op {
|
|
|
case EQ:
|
|
@@ -1440,16 +1573,18 @@ func (v *ValuerEval) simpleDataEval(lhs, rhs interface{}, op Token) interface{}
|
|
|
return lhs.After(rt)
|
|
|
case GTE:
|
|
|
return lhs.After(rt) || lhs.Equal(rt)
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
+ default:
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
}
|
|
|
|
|
|
- // The types were not comparable. If our operation was an equality operation,
|
|
|
- // return false instead of true.
|
|
|
- switch op {
|
|
|
- case EQ, NEQ, LT, LTE, GT, GTE:
|
|
|
- return false
|
|
|
- }
|
|
|
- return nil
|
|
|
+ return invalidOpError(lhs, op, rhs)
|
|
|
+}
|
|
|
+
|
|
|
+func invalidOpError(lhs interface{}, op Token, rhs interface{}) error {
|
|
|
+ return fmt.Errorf("invalid operation %[1]T(%[1]v) %s %[3]T(%[3]v)", lhs, tokens[op], rhs)
|
|
|
}
|
|
|
|
|
|
func convertNum(para interface{}) interface{} {
|