expression: remove mockKVClient (#6299)
This commit is contained in:
@ -20,15 +20,12 @@ import (
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/ast"
|
||||
"github.com/pingcap/tidb/expression"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/parser"
|
||||
"github.com/pingcap/tidb/sessionctx"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/mock"
|
||||
tipb "github.com/pingcap/tipb/go-tipb"
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
var _ = Suite(&testEvaluatorSuite{})
|
||||
@ -38,57 +35,6 @@ func TestT(t *testing.T) {
|
||||
TestingT(t)
|
||||
}
|
||||
|
||||
// mockKvClient is mocked from tikv.CopClient to avoid circular dependency.
|
||||
type mockKvClient struct {
|
||||
}
|
||||
|
||||
// IsRequestTypeSupported implements the kv.Client interface..
|
||||
func (c *mockKvClient) IsRequestTypeSupported(reqType, subType int64) bool {
|
||||
switch reqType {
|
||||
case kv.ReqTypeSelect, kv.ReqTypeIndex:
|
||||
switch subType {
|
||||
case kv.ReqSubTypeGroupBy, kv.ReqSubTypeBasic, kv.ReqSubTypeTopN:
|
||||
return true
|
||||
default:
|
||||
return c.supportExpr(tipb.ExprType(subType))
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// Send implements the kv.Client interface..
|
||||
func (c *mockKvClient) Send(ctx context.Context, req *kv.Request) kv.Response {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mockKvClient) supportExpr(exprType tipb.ExprType) bool {
|
||||
switch exprType {
|
||||
case tipb.ExprType_Null, tipb.ExprType_Int64, tipb.ExprType_Uint64, tipb.ExprType_String, tipb.ExprType_Bytes,
|
||||
tipb.ExprType_MysqlDuration, tipb.ExprType_MysqlTime, tipb.ExprType_MysqlDecimal,
|
||||
tipb.ExprType_ColumnRef,
|
||||
tipb.ExprType_And, tipb.ExprType_Or,
|
||||
tipb.ExprType_LT, tipb.ExprType_LE, tipb.ExprType_EQ, tipb.ExprType_NE,
|
||||
tipb.ExprType_GE, tipb.ExprType_GT, tipb.ExprType_NullEQ,
|
||||
tipb.ExprType_In, tipb.ExprType_ValueList,
|
||||
tipb.ExprType_Like, tipb.ExprType_Not:
|
||||
return true
|
||||
case tipb.ExprType_Plus, tipb.ExprType_Div:
|
||||
return true
|
||||
case tipb.ExprType_Case, tipb.ExprType_If:
|
||||
return true
|
||||
case tipb.ExprType_Count, tipb.ExprType_First, tipb.ExprType_Max, tipb.ExprType_Min, tipb.ExprType_Sum, tipb.ExprType_Avg:
|
||||
return true
|
||||
case tipb.ExprType_JsonType, tipb.ExprType_JsonExtract, tipb.ExprType_JsonUnquote, tipb.ExprType_JsonValid,
|
||||
tipb.ExprType_JsonObject, tipb.ExprType_JsonArray, tipb.ExprType_JsonMerge, tipb.ExprType_JsonSet,
|
||||
tipb.ExprType_JsonInsert, tipb.ExprType_JsonReplace, tipb.ExprType_JsonRemove, tipb.ExprType_JsonContains:
|
||||
return false
|
||||
case kv.ReqSubTypeDesc:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
type dataGen4Expr2PbTest struct {
|
||||
}
|
||||
|
||||
@ -115,7 +61,7 @@ func (s *testEvaluatorSuite) TearDownSuite(c *C) {
|
||||
|
||||
func (s *testEvaluatorSuite) TestAggFunc2Pb(c *C) {
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.AggFuncSum, ast.AggFuncCount, ast.AggFuncAvg, ast.AggFuncGroupConcat, ast.AggFuncMax, ast.AggFuncMin, ast.AggFuncFirstRow}
|
||||
@ -132,13 +78,13 @@ func (s *testEvaluatorSuite) TestAggFunc2Pb(c *C) {
|
||||
}
|
||||
|
||||
jsons := []string{
|
||||
"{\"tp\":3002,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3003,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3002,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
"{\"tp\":3001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
"{\"tp\":3003,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
"null",
|
||||
"{\"tp\":3005,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3004,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3006,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3005,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
"{\"tp\":3004,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
"{\"tp\":3006,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}",
|
||||
}
|
||||
for i, funcName := range funcNames {
|
||||
aggFunc := &AggFuncDesc{
|
||||
|
||||
@ -18,69 +18,13 @@ import (
|
||||
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/ast"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/mysql"
|
||||
"github.com/pingcap/tidb/sessionctx/stmtctx"
|
||||
"github.com/pingcap/tidb/types"
|
||||
"github.com/pingcap/tidb/util/charset"
|
||||
"github.com/pingcap/tidb/util/mock"
|
||||
tipb "github.com/pingcap/tipb/go-tipb"
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
// mockKvClient is mocked from tikv.CopClient to avoid circular dependency.
|
||||
type mockKvClient struct {
|
||||
}
|
||||
|
||||
// IsRequestTypeSupported implements the kv.Client interface..
|
||||
func (c *mockKvClient) IsRequestTypeSupported(reqType, subType int64) bool {
|
||||
switch reqType {
|
||||
case kv.ReqTypeSelect, kv.ReqTypeIndex:
|
||||
switch subType {
|
||||
case kv.ReqSubTypeGroupBy, kv.ReqSubTypeBasic, kv.ReqSubTypeTopN:
|
||||
return true
|
||||
default:
|
||||
return c.supportExpr(tipb.ExprType(subType))
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// Send implements the kv.Client interface..
|
||||
func (c *mockKvClient) Send(ctx context.Context, req *kv.Request) kv.Response {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *mockKvClient) supportExpr(exprType tipb.ExprType) bool {
|
||||
switch exprType {
|
||||
case tipb.ExprType_Null, tipb.ExprType_Int64, tipb.ExprType_Uint64, tipb.ExprType_String, tipb.ExprType_Bytes,
|
||||
tipb.ExprType_MysqlDuration, tipb.ExprType_MysqlTime, tipb.ExprType_MysqlDecimal,
|
||||
tipb.ExprType_ColumnRef,
|
||||
tipb.ExprType_And, tipb.ExprType_Or,
|
||||
tipb.ExprType_LT, tipb.ExprType_LE, tipb.ExprType_EQ, tipb.ExprType_NE,
|
||||
tipb.ExprType_GE, tipb.ExprType_GT, tipb.ExprType_NullEQ,
|
||||
tipb.ExprType_In, tipb.ExprType_ValueList,
|
||||
tipb.ExprType_Like, tipb.ExprType_Not:
|
||||
return true
|
||||
case tipb.ExprType_Plus, tipb.ExprType_Div:
|
||||
return true
|
||||
case tipb.ExprType_Case, tipb.ExprType_If:
|
||||
return true
|
||||
case tipb.ExprType_Count, tipb.ExprType_First, tipb.ExprType_Max, tipb.ExprType_Min, tipb.ExprType_Sum, tipb.ExprType_Avg:
|
||||
return true
|
||||
case tipb.ExprType_JsonType, tipb.ExprType_JsonExtract, tipb.ExprType_JsonUnquote, tipb.ExprType_JsonValid,
|
||||
tipb.ExprType_JsonObject, tipb.ExprType_JsonArray, tipb.ExprType_JsonMerge, tipb.ExprType_JsonSet,
|
||||
tipb.ExprType_JsonInsert, tipb.ExprType_JsonReplace, tipb.ExprType_JsonRemove, tipb.ExprType_JsonContains:
|
||||
return false
|
||||
case tipb.ExprType_DateFormat:
|
||||
return true
|
||||
case kv.ReqSubTypeDesc:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
type dataGen4Expr2PbTest struct {
|
||||
}
|
||||
|
||||
@ -96,7 +40,7 @@ func (s *testEvaluatorSuite) TestConstant2Pb(c *C) {
|
||||
c.Skip("constant pb has changed")
|
||||
var constExprs []Expression
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
|
||||
// can be transformed
|
||||
constValue := new(Constant)
|
||||
@ -181,7 +125,7 @@ func (s *testEvaluatorSuite) TestConstant2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestColumn2Pb(c *C) {
|
||||
var colExprs []Expression
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
colExprs = append(colExprs, dg.genColumn(mysql.TypeBit, 1))
|
||||
@ -229,37 +173,37 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) {
|
||||
c.Assert(len(remained), Equals, 0)
|
||||
js, err := json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0}],\"sig\":0}")
|
||||
c.Assert(string(js), Equals, "{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0,\"field_type\":{\"tp\":4,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0,\"field_type\":{\"tp\":6,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0,\"field_type\":{\"tp\":7,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0,\"field_type\":{\"tp\":9,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0,\"field_type\":{\"tp\":10,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0,\"field_type\":{\"tp\":11,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0,\"field_type\":{\"tp\":13,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0,\"field_type\":{\"tp\":245,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0,\"field_type\":{\"tp\":246,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0,\"field_type\":{\"tp\":249,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0,\"field_type\":{\"tp\":250,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0,\"field_type\":{\"tp\":251,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0,\"field_type\":{\"tp\":252,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}")
|
||||
pbExprs = ExpressionsToPBList(sc, colExprs, client)
|
||||
jsons := []string{
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0,\"field_type\":{\"tp\":4,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0,\"field_type\":{\"tp\":6,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0,\"field_type\":{\"tp\":7,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0,\"field_type\":{\"tp\":9,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0,\"field_type\":{\"tp\":10,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0,\"field_type\":{\"tp\":11,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0,\"field_type\":{\"tp\":13,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0,\"field_type\":{\"tp\":245,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0,\"field_type\":{\"tp\":246,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0,\"field_type\":{\"tp\":249,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0,\"field_type\":{\"tp\":250,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0,\"field_type\":{\"tp\":251,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0,\"field_type\":{\"tp\":252,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
}
|
||||
for i, pbExpr := range pbExprs {
|
||||
c.Assert(pbExprs, NotNil)
|
||||
js, err := json.Marshal(pbExpr)
|
||||
js, err = json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, jsons[i])
|
||||
c.Assert(string(js), Equals, jsons[i], Commentf("%v\n", i))
|
||||
}
|
||||
|
||||
for _, expr := range colExprs {
|
||||
@ -267,20 +211,18 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) {
|
||||
expr.(*Column).Index = 0
|
||||
}
|
||||
pbExpr, pushed, remained = ExpressionsToPB(sc, colExprs, client)
|
||||
c.Assert(pbExpr, IsNil)
|
||||
c.Assert(len(pushed), Equals, 0)
|
||||
c.Assert(len(remained), Equals, len(colExprs))
|
||||
|
||||
pbExprs = ExpressionsToPBList(sc, colExprs, client)
|
||||
for _, pbExpr := range pbExprs {
|
||||
c.Assert(pbExpr, IsNil)
|
||||
}
|
||||
c.Assert(pbExpr, NotNil)
|
||||
js, err = json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":4,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":6,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":7,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":9,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":10,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":11,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":13,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":245,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":246,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":249,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":250,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":251,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":252,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":0}")
|
||||
c.Assert(len(pushed), Equals, len(colExprs))
|
||||
c.Assert(len(remained), Equals, 0)
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestCompareFunc2Pb(c *C) {
|
||||
var compareExprs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.LT, ast.LE, ast.GT, ast.GE, ast.EQ, ast.NE, ast.NullEQ}
|
||||
@ -296,18 +238,18 @@ func (s *testEvaluatorSuite) TestCompareFunc2Pb(c *C) {
|
||||
c.Assert(len(remained), Equals, 0)
|
||||
js, err := json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0},{\"tp\":2002,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0},{\"tp\":2006,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0},{\"tp\":2005,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0},{\"tp\":2003,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0},{\"tp\":2004,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0},{\"tp\":2007,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}],\"sig\":0}")
|
||||
c.Assert(string(js), Equals, "{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":2301,\"children\":[{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":110,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":120,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":130,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":140,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":150,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0},{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":160,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}],\"sig\":0}")
|
||||
|
||||
pbExprs := ExpressionsToPBList(sc, compareExprs, client)
|
||||
c.Assert(len(pbExprs), Equals, len(compareExprs))
|
||||
jsons := []string{
|
||||
"{\"tp\":2001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2002,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2006,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2005,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2003,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2004,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2007,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":110,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":120,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":130,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":140,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":150,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":160,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
}
|
||||
for i, pbExpr := range pbExprs {
|
||||
c.Assert(pbExprs, NotNil)
|
||||
@ -320,7 +262,7 @@ func (s *testEvaluatorSuite) TestCompareFunc2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) {
|
||||
var likeFuncs []Expression
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
|
||||
retTp := types.NewFieldType(mysql.TypeString)
|
||||
retTp.Charset = charset.CharsetUTF8
|
||||
@ -352,7 +294,7 @@ func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) {
|
||||
var arithmeticalFuncs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Mod, ast.IntDiv}
|
||||
@ -368,26 +310,28 @@ func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) {
|
||||
}
|
||||
|
||||
jsons := make(map[string]string)
|
||||
jsons[ast.Plus] = "{\"tp\":2201,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}"
|
||||
jsons[ast.Div] = "{\"tp\":2204,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}"
|
||||
jsons[ast.Plus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":200,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}"
|
||||
jsons[ast.Minus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":204,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}"
|
||||
jsons[ast.Mul] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":208,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}"
|
||||
jsons[ast.Div] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":211,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":31,\"collate\":63,\"charset\":\"binary\"}}"
|
||||
|
||||
pbExprs := ExpressionsToPBList(sc, arithmeticalFuncs, client)
|
||||
for i, pbExpr := range pbExprs {
|
||||
switch funcNames[i] {
|
||||
case ast.Plus, ast.Div:
|
||||
case ast.Mod, ast.IntDiv:
|
||||
c.Assert(pbExpr, IsNil, Commentf("%v\n", funcNames[i]))
|
||||
default:
|
||||
c.Assert(pbExpr, NotNil)
|
||||
js, err := json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, jsons[funcNames[i]])
|
||||
default:
|
||||
c.Assert(pbExpr, IsNil)
|
||||
c.Assert(string(js), Equals, jsons[funcNames[i]], Commentf("%v\n", funcNames[i]))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestDateFunc2Pb(c *C) {
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
fc, err := NewFunction(
|
||||
mock.NewContext(),
|
||||
@ -401,13 +345,13 @@ func (s *testEvaluatorSuite) TestDateFunc2Pb(c *C) {
|
||||
c.Assert(pbExprs[0], NotNil)
|
||||
js, err := json.Marshal(pbExprs[0])
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"tp\":6001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}")
|
||||
c.Assert(string(js), Equals, "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":6001,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":0,\"decimal\":-1,\"collate\":83,\"charset\":\"utf8\"}}")
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestLogicalFunc2Pb(c *C) {
|
||||
var logicalFuncs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.LogicAnd, ast.LogicOr, ast.LogicXor, ast.UnaryNot}
|
||||
@ -428,10 +372,10 @@ func (s *testEvaluatorSuite) TestLogicalFunc2Pb(c *C) {
|
||||
|
||||
pbExprs := ExpressionsToPBList(sc, logicalFuncs, client)
|
||||
jsons := []string{
|
||||
"{\"tp\":2301,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":2302,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":3101,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":3102,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"null",
|
||||
"{\"tp\":1001,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":3104,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
}
|
||||
for i, pbExpr := range pbExprs {
|
||||
js, err := json.Marshal(pbExpr)
|
||||
@ -443,7 +387,7 @@ func (s *testEvaluatorSuite) TestLogicalFunc2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestBitwiseFunc2Pb(c *C) {
|
||||
var bitwiseFuncs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.And, ast.Or, ast.Xor, ast.LeftShift, ast.RightShift, ast.BitNeg}
|
||||
@ -473,7 +417,7 @@ func (s *testEvaluatorSuite) TestBitwiseFunc2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) {
|
||||
var controlFuncs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{
|
||||
@ -499,9 +443,9 @@ func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) {
|
||||
|
||||
pbExprs := ExpressionsToPBList(sc, controlFuncs, client)
|
||||
jsons := []string{
|
||||
"{\"tp\":4007,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0}],\"sig\":0}",
|
||||
"{\"tp\":3301,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0}],\"sig\":0}",
|
||||
"null",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":0,\"collate\":83,\"charset\":\"\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":4101,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
"null",
|
||||
}
|
||||
for i, pbExpr := range pbExprs {
|
||||
@ -514,7 +458,7 @@ func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) {
|
||||
func (s *testEvaluatorSuite) TestOtherFunc2Pb(c *C) {
|
||||
var otherFuncs = make([]Expression, 0)
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
|
||||
funcNames := []string{ast.Coalesce, ast.IsNull}
|
||||
@ -530,49 +474,53 @@ func (s *testEvaluatorSuite) TestOtherFunc2Pb(c *C) {
|
||||
}
|
||||
|
||||
pbExprs := ExpressionsToPBList(sc, otherFuncs, client)
|
||||
for _, pbExpr := range pbExprs {
|
||||
jsons := map[string]string{
|
||||
ast.Coalesce: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":4201,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}",
|
||||
ast.IsNull: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}}],\"sig\":3116,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
|
||||
}
|
||||
for i, pbExpr := range pbExprs {
|
||||
js, err := json.Marshal(pbExpr)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "null")
|
||||
c.Assert(string(js), Equals, jsons[funcNames[i]])
|
||||
}
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestGroupByItem2Pb(c *C) {
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
item := dg.genColumn(mysql.TypeDouble, 0)
|
||||
pbByItem := GroupByItemToPB(sc, client, item)
|
||||
js, err := json.Marshal(pbByItem)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "null")
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},\"desc\":false}")
|
||||
|
||||
item = dg.genColumn(mysql.TypeDouble, 1)
|
||||
pbByItem = GroupByItemToPB(sc, client, item)
|
||||
js, err = json.Marshal(pbByItem)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},\"desc\":false}")
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},\"desc\":false}")
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestSortByItem2Pb(c *C) {
|
||||
sc := new(stmtctx.StatementContext)
|
||||
client := new(mockKvClient)
|
||||
client := new(mock.Client)
|
||||
dg := new(dataGen4Expr2PbTest)
|
||||
item := dg.genColumn(mysql.TypeDouble, 0)
|
||||
pbByItem := SortByItemToPB(sc, client, item, false)
|
||||
js, err := json.Marshal(pbByItem)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "null")
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},\"desc\":false}")
|
||||
|
||||
item = dg.genColumn(mysql.TypeDouble, 1)
|
||||
pbByItem = SortByItemToPB(sc, client, item, false)
|
||||
js, err = json.Marshal(pbByItem)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},\"desc\":false}")
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},\"desc\":false}")
|
||||
|
||||
item = dg.genColumn(mysql.TypeDouble, 1)
|
||||
pbByItem = SortByItemToPB(sc, client, item, true)
|
||||
js, err = json.Marshal(pbByItem)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0},\"desc\":true}")
|
||||
c.Assert(string(js), Equals, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":83,\"charset\":\"\"}},\"desc\":true}")
|
||||
}
|
||||
|
||||
@ -40,6 +40,10 @@ func (c *Client) IsRequestTypeSupported(reqType, subType int64) bool {
|
||||
default:
|
||||
return c.supportExpr(tipb.ExprType(subType))
|
||||
}
|
||||
case kv.ReqTypeDAG:
|
||||
return c.supportExpr(tipb.ExprType(subType))
|
||||
case kv.ReqTypeAnalyze:
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user