Skip to content

Commit

Permalink
Merge branch 'master' into insert-ignore-non-exist
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao committed Oct 23, 2019
2 parents b593624 + a9c92e4 commit e5c36eb
Show file tree
Hide file tree
Showing 36 changed files with 900 additions and 355 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ export PATH := $(path_to_add):$(PATH)
GO := GO111MODULE=on go
GOBUILD := $(GO) build $(BUILD_FLAG) -tags codes -trimpath
GOBUILDCOVERAGE := GOPATH=$(GOPATH) cd tidb-server; $(GO) test -coverpkg="../..." -c .
GOTEST := $(GO) test -p 4
GOTEST := $(GO) test -p 8
OVERALLS := GO111MODULE=on overalls

ARCH := "`uname -s`"
Expand Down
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -404,7 +404,7 @@ var defaultConf = Config{
Reporter: OpenTracingReporter{},
},
TiKVClient: TiKVClient{
GrpcConnectionCount: 16,
GrpcConnectionCount: 4,
GrpcKeepAliveTime: 10,
GrpcKeepAliveTimeout: 3,
CommitTimeout: "41s",
Expand Down
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -259,7 +259,7 @@ local-agent-host-port = ""

[tikv-client]
# Max gRPC connections that will be established with each tikv-server.
grpc-connection-count = 16
grpc-connection-count = 4

# After a duration of this time in seconds if the client doesn't see any activity it pings
# the server to see if the transport is still alive.
Expand Down
52 changes: 23 additions & 29 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1499,11 +1499,22 @@ func (d *ddl) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (err err
if err = checkTooLongTable(ident.Name); err != nil {
return err
}
viewInfo, cols := buildViewInfoWithTableColumns(ctx, s)
viewInfo, err := buildViewInfo(ctx, s)
if err != nil {
return err
}

colObjects := make([]interface{}, 0, len(viewInfo.Cols))
for _, col := range viewInfo.Cols {
colObjects = append(colObjects, col)
cols := make([]*table.Column, len(s.Cols))
colObjects := make([]interface{}, 0, len(s.Cols))

for i, v := range s.Cols {
cols[i] = table.ToColumn(&model.ColumnInfo{
Name: v,
ID: int64(i),
Offset: i,
State: model.StatePublic,
})
colObjects = append(colObjects, v)
}

if err = checkTooLongColumn(colObjects); err != nil {
Expand Down Expand Up @@ -1542,33 +1553,16 @@ func (d *ddl) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (err err
return d.callHookOnChanged(err)
}

func buildViewInfoWithTableColumns(ctx sessionctx.Context, s *ast.CreateViewStmt) (*model.ViewInfo, []*table.Column) {
viewInfo := &model.ViewInfo{Definer: s.Definer, Algorithm: s.Algorithm,
Security: s.Security, SelectStmt: s.Select.Text(), CheckOption: s.CheckOption, Cols: s.SchemaCols}
var tableColumns = make([]*table.Column, len(s.SchemaCols))
if s.Cols == nil {
for i, v := range s.SchemaCols {
tableColumns[i] = table.ToColumn(&model.ColumnInfo{
Name: v,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Version: model.CurrLatestColumnInfoVersion,
})
}
} else {
for i, v := range s.Cols {
tableColumns[i] = table.ToColumn(&model.ColumnInfo{
Name: v,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Version: model.CurrLatestColumnInfoVersion,
})
}
func buildViewInfo(ctx sessionctx.Context, s *ast.CreateViewStmt) (*model.ViewInfo, error) {
// Always Use `format.RestoreNameBackQuotes` to restore `SELECT` statement despite the `ANSI_QUOTES` SQL Mode is enabled or not.
restoreFlag := format.RestoreStringSingleQuotes | format.RestoreKeyWordUppercase | format.RestoreNameBackQuotes
var sb strings.Builder
if err := s.Select.Restore(format.NewRestoreCtx(restoreFlag, &sb)); err != nil {
return nil, err
}

return viewInfo, tableColumns
return &model.ViewInfo{Definer: s.Definer, Algorithm: s.Algorithm,
Security: s.Security, SelectStmt: sb.String(), CheckOption: s.CheckOption, Cols: nil}, nil
}

func checkPartitionByHash(ctx sessionctx.Context, pi *model.PartitionInfo, s *ast.CreateTableStmt, cols []*table.Column, tbInfo *model.TableInfo) error {
Expand Down
4 changes: 2 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3810,9 +3810,9 @@ func (s *testSuite) TestSelectView(c *C) {
tk.MustExec("drop table view_t;")
tk.MustExec("create table view_t(c int,d int)")
err := tk.ExecToErr("select * from view1")
c.Assert(err.Error(), Equals, plannercore.ErrViewInvalid.GenWithStackByArgs("test", "view1").Error())
c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'test.view_t.a' in 'field list'")
err = tk.ExecToErr("select * from view2")
c.Assert(err.Error(), Equals, plannercore.ErrViewInvalid.GenWithStackByArgs("test", "view2").Error())
c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'test.view_t.a' in 'field list'")
err = tk.ExecToErr("select * from view3")
c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'a' in 'field list'")
tk.MustExec("drop table view_t;")
Expand Down
16 changes: 13 additions & 3 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -447,16 +447,26 @@ func (s *testSuite2) TestShowCreateTable(c *C) {
tk.MustExec("create table t1(a int,b int)")
tk.MustExec("drop view if exists v1")
tk.MustExec("create or replace definer=`root`@`127.0.0.1` view v1 as select * from t1")
tk.MustQuery("show create table v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`a`, `b`) AS select * from t1 "))
tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`a`, `b`) AS select * from t1 "))
tk.MustQuery("show create table v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`a`, `b`) AS SELECT `test`.`t1`.`a`,`test`.`t1`.`b` FROM `test`.`t1` "))
tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`a`, `b`) AS SELECT `test`.`t1`.`a`,`test`.`t1`.`b` FROM `test`.`t1` "))
tk.MustExec("drop view v1")
tk.MustExec("drop table t1")

tk.MustExec("drop view if exists v")
tk.MustExec("create or replace definer=`root`@`127.0.0.1` view v as select JSON_MERGE('{}', '{}') as col;")
tk.MustQuery("show create view v").Check(testutil.RowsWithSep("|", "v|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v` (`col`) AS select JSON_MERGE('{}', '{}') as col; "))
tk.MustQuery("show create view v").Check(testutil.RowsWithSep("|", "v|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v` (`col`) AS SELECT JSON_MERGE('{}', '{}') AS `col` "))
tk.MustExec("drop view if exists v")

tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1(a int,b int)")
tk.MustExec("create or replace definer=`root`@`127.0.0.1` view v1 as select avg(a),t1.* from t1 group by a")
tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`avg(a)`, `a`, `b`) AS SELECT AVG(`a`),`test`.`t1`.`a`,`test`.`t1`.`b` FROM `test`.`t1` GROUP BY `a` "))
tk.MustExec("drop view v1")
tk.MustExec("create or replace definer=`root`@`127.0.0.1` view v1 as select a+b, t1.* , a as c from t1")
tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`a+b`, `a`, `b`, `c`) AS SELECT `a`+`b`,`test`.`t1`.`a`,`test`.`t1`.`b`,`a` AS `c` FROM `test`.`t1` "))
tk.MustExec("drop table t1")
tk.MustExec("drop view v1")

// For issue #9211
tk.MustExec("create table t(c int, b int as (c + 1))ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;")
tk.MustQuery("show create table `t`").Check(testutil.RowsWithSep("|",
Expand Down
18 changes: 16 additions & 2 deletions expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,15 @@ func (g *dataStrGener) gen() interface{} {
return fmt.Sprintf("%d:%d:%d", hour, minute, second)
}

// constStrGener always returns the given string
type constStrGener struct {
s string
}

func (g *constStrGener) gen() interface{} {
return g.s
}

type randDurInt struct{}

func (g *randDurInt) gen() interface{} {
Expand Down Expand Up @@ -472,11 +481,16 @@ type vecExprBenchCase struct {
type vecExprBenchCases map[string][]vecExprBenchCase

func fillColumn(eType types.EvalType, chk *chunk.Chunk, colIdx int, testCase vecExprBenchCase) {
batchSize := 1024
var gen dataGenerator
if len(testCase.geners) > colIdx && testCase.geners[colIdx] != nil {
gen = testCase.geners[colIdx]
} else {
}
fillColumnWithGener(eType, chk, colIdx, gen)
}

func fillColumnWithGener(eType types.EvalType, chk *chunk.Chunk, colIdx int, gen dataGenerator) {
batchSize := 1024
if gen == nil {
gen = &defaultGener{0.2, eType}
}

Expand Down
5 changes: 5 additions & 0 deletions expression/builtin_arithmetic.go
Original file line number Diff line number Diff line change
Expand Up @@ -670,13 +670,15 @@ func (c *arithmeticIntDivideFunctionClass) getFunction(ctx sessionctx.Context, a
bf.tp.Flag |= mysql.UnsignedFlag
}
sig := &builtinArithmeticIntDivideIntSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_IntDivideInt)
return sig, nil
}
bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDecimal, types.ETDecimal)
if mysql.HasUnsignedFlag(lhsTp.Flag) || mysql.HasUnsignedFlag(rhsTp.Flag) {
bf.tp.Flag |= mysql.UnsignedFlag
}
sig := &builtinArithmeticIntDivideDecimalSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_IntDivideDecimal)
return sig, nil
}

Expand Down Expand Up @@ -834,6 +836,7 @@ func (c *arithmeticModFunctionClass) getFunction(ctx sessionctx.Context, args []
bf.tp.Flag |= mysql.UnsignedFlag
}
sig := &builtinArithmeticModRealSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_ModReal)
return sig, nil
} else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal {
bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal)
Expand All @@ -842,13 +845,15 @@ func (c *arithmeticModFunctionClass) getFunction(ctx sessionctx.Context, args []
bf.tp.Flag |= mysql.UnsignedFlag
}
sig := &builtinArithmeticModDecimalSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_ModDecimal)
return sig, nil
} else {
bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt)
if mysql.HasUnsignedFlag(lhsTp.Flag) {
bf.tp.Flag |= mysql.UnsignedFlag
}
sig := &builtinArithmeticModIntSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_ModInt)
return sig, nil
}
}
Expand Down
15 changes: 15 additions & 0 deletions expression/builtin_arithmetic_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tidb/util/testutil"
"github.com/pingcap/tipb/go-tipb"
)

func (s *testEvaluatorSuite) TestSetFlenDecimal4RealOrDecimal(c *C) {
Expand Down Expand Up @@ -376,6 +377,12 @@ func (s *testEvaluatorSuite) TestArithmeticDivide(c *C) {
sig, err := funcs[ast.Div].getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(tc.args...)))
c.Assert(err, IsNil)
c.Assert(sig, NotNil)
switch sig.(type) {
case *builtinArithmeticIntDivideIntSig:
c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_IntDivideInt)
case *builtinArithmeticIntDivideDecimalSig:
c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_IntDivideDecimal)
}
val, err := evalBuiltinFunc(sig, chunk.Row{})
c.Assert(err, IsNil)
c.Assert(val, testutil.DatumEquals, types.NewDatum(tc.expect))
Expand Down Expand Up @@ -601,6 +608,14 @@ func (s *testEvaluatorSuite) TestArithmeticMod(c *C) {
c.Assert(err, IsNil)
c.Assert(sig, NotNil)
val, err := evalBuiltinFunc(sig, chunk.Row{})
switch sig.(type) {
case *builtinArithmeticModRealSig:
c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModReal)
case *builtinArithmeticModIntSig:
c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModInt)
case *builtinArithmeticModDecimalSig:
c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModDecimal)
}
c.Assert(err, IsNil)
c.Assert(val, testutil.DatumEquals, types.NewDatum(tc.expect))
}
Expand Down
33 changes: 31 additions & 2 deletions expression/builtin_arithmetic_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -583,9 +583,38 @@ func (b *builtinArithmeticPlusDecimalSig) vecEvalDecimal(input *chunk.Chunk, res
}

func (b *builtinArithmeticMultiplyIntUnsignedSig) vectorized() bool {
return false
return true
}

func (b *builtinArithmeticMultiplyIntUnsignedSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
if err := b.args[0].VecEvalInt(b.ctx, input, result); err != nil {
return err
}
n := input.NumRows()
buf, err := b.bufAllocator.get(types.ETInt, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf)

if err := b.args[1].VecEvalInt(b.ctx, input, buf); err != nil {
return err
}

x := result.Uint64s()
y := buf.Uint64s()
result.MergeNulls(buf)
var res uint64
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}

res = x[i] * y[i]
if x[i] != 0 && res/x[i] != y[i] {
return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String()))
}
x[i] = res
}
return nil
}
6 changes: 6 additions & 0 deletions expression/builtin_arithmetic_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,12 @@ var vecBuiltinArithmeticCases = map[string][]vecExprBenchCase{
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal, types.ETReal}},
{retEvalType: types.ETDecimal, childrenTypes: []types.EvalType{types.ETDecimal, types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: []dataGenerator{&rangeInt64Gener{-10000, 10000}, &rangeInt64Gener{-10000, 10000}}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeInt24, Flag: mysql.UnsignedFlag}, {Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag}},
geners: []dataGenerator{
&rangeInt64Gener{begin: 0, end: 10000},
&rangeInt64Gener{begin: 0, end: 10000},
},
},
},
ast.Round: {},
ast.And: {},
Expand Down
38 changes: 36 additions & 2 deletions expression/builtin_cast_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,11 +310,45 @@ func (b *builtinCastDurationAsIntSig) vecEvalInt(input *chunk.Chunk, result *chu
}

func (b *builtinCastIntAsTimeSig) vectorized() bool {
return false
return true
}

func (b *builtinCastIntAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()
buf, err := b.bufAllocator.get(types.ETInt, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf)
if err := b.args[0].VecEvalInt(b.ctx, input, buf); err != nil {
return err
}

result.ResizeTime(n, false)
result.MergeNulls(buf)
times := result.Times()
i64s := buf.Int64s()
stmt := b.ctx.GetSessionVars().StmtCtx
fsp := int8(b.tp.Decimal)
for i := 0; i < n; i++ {
if buf.IsNull(i) {
continue
}
tm, err := types.ParseTimeFromNum(stmt, i64s[i], b.tp.Tp, fsp)
if err != nil {
if err = handleInvalidTimeError(b.ctx, err); err != nil {
return err
}
result.SetNull(i, true)
continue
}
times[i] = tm
if b.tp.Tp == mysql.TypeDate {
// Truncate hh:mm:ss part if the type is Date.
times[i].Time = types.FromDate(tm.Time.Year(), tm.Time.Month(), tm.Time.Day(), 0, 0, 0, 0)
}
}
return nil
}

func (b *builtinCastRealAsJSONSig) vectorized() bool {
Expand Down
1 change: 1 addition & 0 deletions expression/builtin_cast_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{
{retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson}},
{retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&jsonStringGener{}}},
{retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETInt}},
},
}

Expand Down
Loading

0 comments on commit e5c36eb

Please sign in to comment.