From 8823f12d324b0d5cc4b0750df5482d0eba521d2e Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 8 Oct 2018 17:38:17 +0800 Subject: [PATCH 001/509] session: set Sleep state for process info (#7826) (#7839) --- server/tidb_test.go | 14 ++++++++++++++ session/session.go | 3 +++ 2 files changed, 17 insertions(+) diff --git a/server/tidb_test.go b/server/tidb_test.go index db9c56cd77541..af00149189542 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -526,3 +526,17 @@ func (ts *TidbTestSuite) TestSumAvg(c *C) { c.Parallel() runTestSumAvg(c) } + +func (ts *TidbTestSuite) TestShowProcess(c *C) { + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) + c.Assert(err, IsNil) + ctx := context.Background() + results, err := qctx.Execute(ctx, "select 1") + c.Assert(err, IsNil) + pi := qctx.ShowProcess() + c.Assert(pi.Command, Equals, "Query") + results[0].Close() + pi = qctx.ShowProcess() + c.Assert(pi.Command, Equals, "Sleep") + qctx.Close() +} diff --git a/session/session.go b/session/session.go index cf44975b59e15..7169e25a6206d 100644 --- a/session/session.go +++ b/session/session.go @@ -712,6 +712,9 @@ func (s *session) SetProcessInfo(sql string) { State: s.Status(), Info: sql, } + if sql == "" { + pi.Command = "Sleep" + } if s.sessionVars.User != nil { pi.User = s.sessionVars.User.Username pi.Host = s.sessionVars.User.Hostname From e6025cb844952488e8896bdf796faea1ea1ba4a4 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 11 Oct 2018 16:51:06 +0800 Subject: [PATCH 002/509] executor: refine the precision for avg (#7860) (#7874) --- executor/aggfuncs/func_avg.go | 25 ++++++++++++++++++++++++- executor/builder.go | 21 +++++++++++++++++++++ expression/aggregation/descriptor.go | 19 ++++++++++++------- expression/typeinfer_test.go | 4 ++-- 4 files changed, 59 insertions(+), 10 deletions(-) diff --git a/executor/aggfuncs/func_avg.go b/executor/aggfuncs/func_avg.go index 139d60845273f..f917c60e1d044 100644 --- a/executor/aggfuncs/func_avg.go +++ b/executor/aggfuncs/func_avg.go @@ -14,6 +14,8 @@ package aggfuncs import ( + "github.com/cznic/mathutil" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -56,7 +58,19 @@ func (e *baseAvgDecimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Par finalResult := new(types.MyDecimal) err := types.DecimalDiv(&p.sum, decimalCount, finalResult, types.DivFracIncr) if err != nil { - return errors.Trace(err) + return err + } + // Make the decimal be the result of type inferring. + frac := e.args[0].GetType().Decimal + if len(e.args) == 2 { + frac = e.args[1].GetType().Decimal + } + if frac == -1 { + frac = mysql.MaxDecimalScale + } + err = finalResult.Round(finalResult, mathutil.Min(frac, mysql.MaxDecimalScale), types.ModeHalfEven) + if err != nil { + return err } chk.AppendMyDecimal(e.ordinal, finalResult) return nil @@ -195,6 +209,15 @@ func (e *avgOriginal4DistinctDecimal) AppendFinalResult2Chunk(sctx sessionctx.Co if err != nil { return errors.Trace(err) } + // Make the decimal be the result of type inferring. + frac := e.args[0].GetType().Decimal + if frac == -1 { + frac = mysql.MaxDecimalScale + } + err = finalResult.Round(finalResult, mathutil.Min(frac, mysql.MaxDecimalScale), types.ModeHalfEven) + if err != nil { + return err + } chk.AppendMyDecimal(e.ordinal, finalResult) return nil } diff --git a/executor/builder.go b/executor/builder.go index 1f9ef0e0cb96a..b16d39eee04fa 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -887,6 +887,27 @@ func (b *executorBuilder) wrapCastForAggArgs(funcs []*aggregation.AggFuncDesc) { } for i := range f.Args { f.Args[i] = castFunc(b.ctx, f.Args[i]) + if f.Name != ast.AggFuncAvg && f.Name != ast.AggFuncSum { + continue + } + // After wrapping cast on the argument, flen etc. may not the same + // as the type of the aggregation function. The following part set + // the type of the argument exactly as the type of the aggregation + // function. + // Note: If the `Tp` of argument is the same as the `Tp` of the + // aggregation function, it will not wrap cast function on it + // internally. The reason of the special handling for `Column` is + // that the `RetType` of `Column` refers to the `infoschema`, so we + // need to set a new variable for it to avoid modifying the + // definition in `infoschema`. + if col, ok := f.Args[i].(*expression.Column); ok { + col.RetType = types.NewFieldType(col.RetType.Tp) + } + // originTp is used when the the `Tp` of column is TypeFloat32 while + // the type of the aggregation function is TypeFloat64. + originTp := f.Args[i].GetType().Tp + *(f.Args[i].GetType()) = *(f.RetTp) + f.Args[i].GetType().Tp = originTp } } } diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 7cbf71b46b5b0..7c47ec85c0d64 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -291,17 +291,21 @@ func (a *AggFuncDesc) typeInfer4Count(ctx sessionctx.Context) { // Because child returns integer or decimal type. func (a *AggFuncDesc) typeInfer4Sum(ctx sessionctx.Context) { switch a.Args[0].GetType().Tp { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeNewDecimal: + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + a.RetTp = types.NewFieldType(mysql.TypeNewDecimal) + a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxDecimalWidth, 0 + case mysql.TypeNewDecimal: a.RetTp = types.NewFieldType(mysql.TypeNewDecimal) a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxDecimalWidth, a.Args[0].GetType().Decimal if a.RetTp.Decimal < 0 || a.RetTp.Decimal > mysql.MaxDecimalScale { a.RetTp.Decimal = mysql.MaxDecimalScale } - // TODO: a.Args[0] = expression.WrapWithCastAsDecimal(ctx, a.Args[0]) - default: + case mysql.TypeDouble, mysql.TypeFloat: a.RetTp = types.NewFieldType(mysql.TypeDouble) a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, a.Args[0].GetType().Decimal - //TODO: a.Args[0] = expression.WrapWithCastAsReal(ctx, a.Args[0]) + default: + a.RetTp = types.NewFieldType(mysql.TypeDouble) + a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, types.UnspecifiedLength } types.SetBinChsClnFlag(a.RetTp) } @@ -318,11 +322,12 @@ func (a *AggFuncDesc) typeInfer4Avg(ctx sessionctx.Context) { a.RetTp.Decimal = mathutil.Min(a.Args[0].GetType().Decimal+types.DivFracIncr, mysql.MaxDecimalScale) } a.RetTp.Flen = mysql.MaxDecimalWidth - // TODO: a.Args[0] = expression.WrapWithCastAsDecimal(ctx, a.Args[0]) - default: + case mysql.TypeDouble, mysql.TypeFloat: a.RetTp = types.NewFieldType(mysql.TypeDouble) a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, a.Args[0].GetType().Decimal - // TODO: a.Args[0] = expression.WrapWithCastAsReal(ctx, a.Args[0]) + default: + a.RetTp = types.NewFieldType(mysql.TypeDouble) + a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, types.UnspecifiedLength } types.SetBinChsClnFlag(a.RetTp) } diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 142c0c0c9bf41..bb86b140d5caf 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -822,14 +822,14 @@ func (s *testInferTypeSuite) createTestCase4Aggregations() []typeInferTestCase { {"sum(c_decimal)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 3}, {"sum(1.0)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 1}, {"sum(1.2e2)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, - {"sum(c_char)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, + {"sum(c_char)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"avg(c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 4}, {"avg(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"avg(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"avg(c_decimal)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 7}, {"avg(1.0)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 5}, {"avg(1.2e2)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, - {"avg(c_char)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, + {"avg(c_char)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"group_concat(c_int_d)", mysql.TypeVarString, charset.CharsetUTF8, 0, mysql.MaxBlobWidth, 0}, } } From 4323e840daac470cfbb69da0006098617ada3e63 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 11 Oct 2018 17:57:12 +0800 Subject: [PATCH 003/509] domain: fix memory leak for stats (#7864) (#7873) --- domain/domain.go | 18 ++++++++++++++++++ executor/executor_test.go | 1 + infoschema/tables_test.go | 1 + planner/core/cbo_test.go | 1 + server/statistics_handler_test.go | 1 + session/session.go | 5 +++-- statistics/handle_test.go | 1 + 7 files changed, 26 insertions(+), 2 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 865214c5972c3..a4caf5baa7af7 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -24,6 +24,7 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/ngaut/pools" + "github.com/ngaut/sync2" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -54,6 +55,7 @@ type Domain struct { privHandle *privileges.Handle statsHandle unsafe.Pointer statsLease time.Duration + statsUpdating sync2.AtomicInt32 ddl ddl.DDL info *InfoSyncer m sync.Mutex @@ -664,6 +666,20 @@ func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) { atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statistics.NewHandle(ctx, do.statsLease))) } +// StatsUpdating checks if the stats worker is updating. +func (do *Domain) StatsUpdating() bool { + return do.statsUpdating.Get() > 0 +} + +// SetStatsUpdating sets the value of stats updating. +func (do *Domain) SetStatsUpdating(val bool) { + if val { + do.statsUpdating.Set(1) + } else { + do.statsUpdating.Set(0) + } +} + // RunAutoAnalyze indicates if this TiDB server starts auto analyze worker and can run auto analyze job. var RunAutoAnalyze = true @@ -680,6 +696,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error { } owner := do.newStatsOwner() do.wg.Add(1) + do.SetStatsUpdating(true) go do.updateStatsWorker(ctx, owner) if RunAutoAnalyze { do.wg.Add(1) @@ -729,6 +746,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) log.Info("[stats] init stats info takes ", time.Now().Sub(t)) } defer func() { + do.SetStatsUpdating(false) recoverInDomain("updateStatsWorker", false) do.wg.Done() }() diff --git a/executor/executor_test.go b/executor/executor_test.go index d9ccc3b18cb5d..7a221de7de7db 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -113,6 +113,7 @@ func (s *testSuite) SetUpSuite(c *C) { } d, err := session.BootstrapSession(s.store) c.Assert(err, IsNil) + d.SetStatsUpdating(true) s.domain = d } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index befc73d93c7c2..919f7b9a9e0b7 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -93,6 +93,7 @@ func (s *testSuite) TestDataForTableStatsField(c *C) { session.SetStatsLease(0) do, err := session.BootstrapSession(store) c.Assert(err, IsNil) + do.SetStatsUpdating(true) defer do.Close() oldExpiryTime := infoschema.TableStatsCacheExpiry infoschema.TableStatsCacheExpiry = 0 diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 47582450a59b7..b1ec657000890 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -693,6 +693,7 @@ func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { session.SetSchemaLease(0) session.SetStatsLease(0) dom, err := session.BootstrapSession(store) + dom.SetStatsUpdating(true) return store, dom, errors.Trace(err) } diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index 33f1ab700a4ca..ce67e3ac6f179 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -49,6 +49,7 @@ func (ds *testDumpStatsSuite) startServer(c *C) { session.SetStatsLease(0) ds.domain, err = session.BootstrapSession(ds.store) c.Assert(err, IsNil) + ds.domain.SetStatsUpdating(true) tidbdrv := NewTiDBDriver(ds.store) cfg := config.NewConfig() diff --git a/session/session.go b/session/session.go index 7169e25a6206d..89280de515300 100644 --- a/session/session.go +++ b/session/session.go @@ -1065,8 +1065,9 @@ func CreateSession(store kv.Storage) (Session, error) { } privilege.BindPrivilegeManager(s, pm) - // Add statsUpdateHandle. - if do.StatsHandle() != nil { + // Add stats collector, and it will be freed by background stats worker + // which periodically updates stats using the collected data. + if do.StatsHandle() != nil && do.StatsUpdating() { s.statsCollector = do.StatsHandle().NewSessionStatsCollector() } diff --git a/statistics/handle_test.go b/statistics/handle_test.go index 07ab5f4decaba..6cfdca998e0d1 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -445,5 +445,6 @@ func newStoreWithBootstrap(statsLease time.Duration) (kv.Storage, *domain.Domain session.SetStatsLease(statsLease) domain.RunAutoAnalyze = false do, err := session.BootstrapSession(store) + do.SetStatsUpdating(true) return store, do, errors.Trace(err) } From 1db4288996c2ffdd0485c75c52cdaab4fc44e5bf Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 11 Oct 2018 19:31:45 +0800 Subject: [PATCH 004/509] stats: fix combined index low-bound check (#7814) (#7856) --- statistics/histogram.go | 22 ++++++++++++++++++++-- statistics/selectivity_test.go | 12 ++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index a7730d07d975d..f56072ffb29f1 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -682,9 +682,8 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { if hg.Bounds == nil { return true } - len := hg.Bounds.NumRows() return chunk.Compare(hg.Bounds.GetRow(0), 0, &val) > 0 || - chunk.Compare(hg.Bounds.GetRow(len-1), 0, &val) < 0 + chunk.Compare(hg.Bounds.GetRow(hg.Bounds.NumRows()-1), 0, &val) < 0 } // ErrorRate is the error rate of estimate row count by bucket and cm sketch. @@ -857,3 +856,22 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range } return totalCount, nil } + +func (idx *Index) outOfRange(val types.Datum) bool { + if idx.Bounds == nil { + return true + } + withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || + matchPrefix(idx.Bounds.GetRow(0), 0, &val) + withInHighBound := chunk.Compare(idx.Bounds.GetRow(idx.Bounds.NumRows()-1), 0, &val) >= 0 + return !withInLowBoundOrPrefixMatch || !withInHighBound +} + +// matchPrefix checks whether ad is the prefix of value +func matchPrefix(row chunk.Row, colIdx int, ad *types.Datum) bool { + switch ad.Kind() { + case types.KindString, types.KindBytes, types.KindBinaryLiteral, types.KindMysqlBit: + return strings.HasPrefix(row.GetString(colIdx), ad.GetString()) + } + return false +} diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 09221d6615240..ed24d0b97f326 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -229,6 +229,18 @@ func (s *testSelectivitySuite) TestDiscreteDistribution(c *C) { "└─IndexScan_8 0.00 cop table:t, index:a, b, range:[\"tw\" -inf,\"tw\" 0), keep order:false")) } +func (s *testSelectivitySuite) TestSelectCombinedLowBound(c *C) { + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(id int auto_increment, kid int, pid int, primary key(id), key(kid, pid))") + testKit.MustExec("insert into t (kid, pid) values (1,2), (1,3), (1,4),(1, 11), (1, 12), (1, 13), (1, 14), (2, 2), (2, 3), (2, 4)") + testKit.MustExec("analyze table t") + testKit.MustQuery("explain select * from t where kid = 1").Check(testkit.Rows( + "IndexReader_9 7.00 root index:IndexScan_8", + "└─IndexScan_8 7.00 cop table:t, index:kid, pid, range:[1,1], keep order:false")) +} + func getRange(start, end int64) []*ranger.Range { ran := &ranger.Range{ LowVal: []types.Datum{types.NewIntDatum(start)}, From fbdcf63e5243a81cce0799a06b6672fe37aa515a Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Thu, 11 Oct 2018 20:15:55 +0800 Subject: [PATCH 005/509] plan: exclude IsNull from constant propagation(cherry-pick #7835) (#7851) --- cmd/explaintest/r/explain_easy.result | 9 +++++++++ cmd/explaintest/t/explain_easy.test | 1 + expression/constant_propagation.go | 3 +++ expression/function_traits.go | 5 +++++ 4 files changed, 18 insertions(+) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 3ff9b16527833..2aa7615570d84 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -347,6 +347,15 @@ TableDual_5 0.00 root rows:0 explain select * from t where b = 1 and b = 2; id count task operator info TableDual_5 0.00 root rows:0 +explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null; +id count task operator info +Projection_7 12.50 root t1.a, t1.b, t2.a, t2.b +└─HashRightJoin_9 12.50 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)] + ├─TableReader_12 10.00 root data:Selection_11 + │ └─Selection_11 10.00 cop isnull(t2.b) + │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo + └─TableReader_14 10000.00 root data:TableScan_13 + └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; create table t(a bigint primary key); explain select * from t where a = 1 and a = 2; diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index 0a2d7ef884389..391e0039ad5d4 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -66,6 +66,7 @@ explain select * from t where b in (1, 2) and b in (1, 3); explain select * from t where a = 1 and a = 1; explain select * from t where a = 1 and a = 2; explain select * from t where b = 1 and b = 2; +explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null; drop table if exists t; create table t(a bigint primary key); diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index c0d7a0fd10ddf..2919c5dc359c8 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -188,6 +188,9 @@ func (s *propagateConstantSolver) tryToReplaceCond(src *Column, tgt *Column, con if _, ok := unFoldableFunctions[sf.FuncName.L]; ok { return false, true, cond } + if _, ok := inequalFunctions[sf.FuncName.L]; ok { + return false, true, cond + } for idx, expr := range sf.GetArgs() { if src.Equal(nil, expr) { replaced = true diff --git a/expression/function_traits.go b/expression/function_traits.go index db5356f042b80..9aa6952416eb1 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -50,3 +50,8 @@ var unFoldableFunctions = map[string]struct{}{ ast.GetVar: {}, ast.GetParam: {}, } + +// inequalFunctions stores functions which cannot be propagated from column equal condition. +var inequalFunctions = map[string]struct{}{ + ast.IsNull: {}, +} From 3950070e656a3a069fde4c3ecd44644b0ffdf39f Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 11 Oct 2018 20:43:57 +0800 Subject: [PATCH 006/509] util: refine chunk.SwapColumn to rebuild the column reference (#7841) (#7849) --- util/chunk/chunk.go | 40 +++++++++++++++++++++++++++++++++- util/chunk/chunk_test.go | 46 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 85 insertions(+), 1 deletion(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index a5f2cddc57fa1..d09150f1ab84b 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -137,9 +137,47 @@ func (c *Chunk) MakeRef(srcColIdx, dstColIdx int) { c.columns[dstColIdx] = c.columns[srcColIdx] } -// SwapColumn swaps column "c.columns[colIdx]" with column "other.columns[otherIdx]". +// SwapColumn swaps column "c.columns[colIdx]" with column +// "other.columns[otherIdx]". If there exists columns refer to the column to be +// swapped, we need to re-build the reference. func (c *Chunk) SwapColumn(colIdx int, other *Chunk, otherIdx int) { + // Find the leftmost column of the reference which is the actual column to + // be swapped. + for i := 0; i < colIdx; i++ { + if c.columns[i] == c.columns[colIdx] { + colIdx = i + } + } + for i := 0; i < otherIdx; i++ { + if other.columns[i] == other.columns[otherIdx] { + otherIdx = i + } + } + + // Find the columns which refer to the actual column to be swapped. + refColsIdx := make([]int, 0, len(c.columns)-colIdx) + for i := colIdx; i < len(c.columns); i++ { + if c.columns[i] == c.columns[colIdx] { + refColsIdx = append(refColsIdx, i) + } + } + refColsIdx4Other := make([]int, 0, len(other.columns)-otherIdx) + for i := otherIdx; i < len(other.columns); i++ { + if other.columns[i] == other.columns[otherIdx] { + refColsIdx4Other = append(refColsIdx4Other, i) + } + } + + // Swap columns from two chunks. c.columns[colIdx], other.columns[otherIdx] = other.columns[otherIdx], c.columns[colIdx] + + // Rebuild the reference. + for _, i := range refColsIdx { + c.MakeRef(colIdx, i) + } + for _, i := range refColsIdx4Other { + other.MakeRef(otherIdx, i) + } } // SwapColumns swaps columns with another Chunk. diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index f5158ca4b4da2..799a6e703ca64 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -471,6 +471,52 @@ func (s *testChunkSuite) TestChunkMemoryUsage(c *check.C) { c.Assert(memUsage, check.Equals, int64(expectedUsage)) } +func (s *testChunkSuite) TestSwapColumn(c *check.C) { + fieldTypes := make([]*types.FieldType, 0, 2) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeFloat}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeFloat}) + fieldTypes = append(fieldTypes, &types.FieldType{Tp: mysql.TypeFloat}) + + // chk1: column1 refers to column0 + chk1 := NewChunkWithCapacity(fieldTypes, 1) + chk1.AppendFloat64(0, 1) + chk1.MakeRef(0, 1) + chk1.AppendFloat64(2, 3) + + // chk2: column1 refers to column0 + chk2 := NewChunkWithCapacity(fieldTypes, 1) + chk2.AppendFloat64(0, 1) + chk2.MakeRef(0, 1) + chk2.AppendFloat64(2, 3) + + c.Assert(chk1.columns[0] == chk1.columns[1], check.IsTrue) + c.Assert(chk2.columns[0] == chk2.columns[1], check.IsTrue) + + checkRef := func() { + c.Assert(chk1.columns[0] == chk1.columns[1], check.IsTrue) + c.Assert(chk1.columns[0] == chk2.columns[0], check.IsFalse) + c.Assert(chk2.columns[0] == chk2.columns[1], check.IsTrue) + } + + chk1.SwapColumn(0, chk2, 0) + checkRef() + + chk1.SwapColumn(0, chk2, 1) + checkRef() + + chk2.SwapColumn(1, chk2, 0) + checkRef() + + chk2.SwapColumn(1, chk2, 1) + checkRef() + + chk2.SwapColumn(1, chk2, 2) + checkRef() + + chk2.SwapColumn(2, chk2, 0) + checkRef() +} + func BenchmarkAppendInt(b *testing.B) { b.ReportAllocs() chk := newChunk(8) From deea24f8df499a20806f7979d9265ddfdfde2d38 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 12 Oct 2018 08:25:02 +0800 Subject: [PATCH 007/509] store/tikv,executor: redesign the latch scheduler (#7711) (#7859) --- executor/write_test.go | 3 +- store/tikv/latch/latch.go | 206 ++++++++++++++++++----------- store/tikv/latch/latch_test.go | 57 +++++++- store/tikv/latch/scheduler.go | 38 ++++-- store/tikv/latch/scheduler_test.go | 2 +- store/tikv/txn.go | 3 - 6 files changed, 211 insertions(+), 98 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index eb444a305b7a1..033798c754a82 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1815,8 +1815,7 @@ func (s *testBypassSuite) TestBypassLatch(c *C) { // txn1 and txn2 data range do not overlap, but using latches result in txn conflict. fn() - _, err = tk1.Exec("commit") - c.Assert(err, NotNil) + tk1.MustExec("commit") tk1.MustExec("truncate table t") fn() diff --git a/store/tikv/latch/latch.go b/store/tikv/latch/latch.go index a8ca718947490..f6f57a460b106 100644 --- a/store/tikv/latch/latch.go +++ b/store/tikv/latch/latch.go @@ -14,6 +14,7 @@ package latch import ( + "bytes" "math/bits" "sort" "sync" @@ -22,32 +23,26 @@ import ( "github.com/spaolacci/murmur3" ) -// latch stores a key's waiting transactions information. -type latch struct { - // Whether there is any transaction in waitingQueue except head. - hasMoreWaiting bool - // The startTS of the transaction which is the head of waiting transactions. - waitingQueueHead uint64 - maxCommitTS uint64 - sync.Mutex -} +type node struct { + slotID int + key []byte + maxCommitTS uint64 + value *Lock -func (l *latch) isEmpty() bool { - return l.waitingQueueHead == 0 && !l.hasMoreWaiting + next *node } -func (l *latch) free() { - l.waitingQueueHead = 0 -} - -func (l *latch) refreshCommitTS(commitTS uint64) { - l.Lock() - defer l.Unlock() - l.maxCommitTS = mathutil.MaxUint64(commitTS, l.maxCommitTS) +// latch stores a key's waiting transactions information. +type latch struct { + queue *node + count int + waiting []*Lock + sync.Mutex } // Lock is the locks' information required for a transaction. type Lock struct { + keys [][]byte // The slot IDs of the latches(keys) that a startTS must acquire before being able to processed. requiredSlots []int // The number of latches that the transaction has acquired. For status is stale, it include the @@ -96,9 +91,20 @@ func (l *Lock) SetCommitTS(commitTS uint64) { // but conceptually a latch is a queue, and a slot is an index to the queue type Latches struct { slots []latch - // The waiting queue for each slot(slotID => slice of Lock). - waitingQueues map[int][]*Lock - sync.RWMutex +} + +type bytesSlice [][]byte + +func (s bytesSlice) Len() int { + return len(s) +} + +func (s bytesSlice) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +func (s bytesSlice) Less(i, j int) bool { + return bytes.Compare(s[i], s[j]) < 0 } // NewLatches create a Latches with fixed length, @@ -107,14 +113,15 @@ func NewLatches(size uint) *Latches { powerOfTwoSize := 1 << uint32(bits.Len32(uint32(size-1))) slots := make([]latch, powerOfTwoSize) return &Latches{ - slots: slots, - waitingQueues: make(map[int][]*Lock), + slots: slots, } } // genLock generates Lock for the transaction with startTS and keys. func (latches *Latches) genLock(startTS uint64, keys [][]byte) *Lock { + sort.Sort(bytesSlice(keys)) return &Lock{ + keys: keys, requiredSlots: latches.genSlotIDs(keys), acquiredCount: 0, startTS: startTS, @@ -126,17 +133,7 @@ func (latches *Latches) genSlotIDs(keys [][]byte) []int { for _, key := range keys { slots = append(slots, latches.slotID(key)) } - sort.Ints(slots) - if len(slots) <= 1 { - return slots - } - dedup := slots[:1] - for i := 1; i < len(slots); i++ { - if slots[i] != slots[i-1] { - dedup = append(dedup, slots[i]) - } - } - return dedup + return slots } // slotID return slotID for current key. @@ -150,8 +147,7 @@ func (latches *Latches) acquire(lock *Lock) acquireResult { return acquireStale } for lock.acquiredCount < len(lock.requiredSlots) { - slotID := lock.requiredSlots[lock.acquiredCount] - status := latches.acquireSlot(slotID, lock) + status := latches.acquireSlot(lock) if status != acquireSuccess { return status } @@ -161,75 +157,129 @@ func (latches *Latches) acquire(lock *Lock) acquireResult { // release releases all latches owned by the `lock` and returns the wakeup list. // Preconditions: the caller must ensure the transaction's status is not locked. -func (latches *Latches) release(lock *Lock, commitTS uint64, wakeupList []*Lock) []*Lock { +func (latches *Latches) release(lock *Lock, wakeupList []*Lock) []*Lock { wakeupList = wakeupList[:0] - for i := 0; i < lock.acquiredCount; i++ { - slotID := lock.requiredSlots[i] - if nextLock := latches.releaseSlot(slotID, commitTS); nextLock != nil { + for lock.acquiredCount > 0 { + if nextLock := latches.releaseSlot(lock); nextLock != nil { wakeupList = append(wakeupList, nextLock) } } return wakeupList } -// refreshCommitTS refreshes commitTS for keys. -func (latches *Latches) refreshCommitTS(keys [][]byte, commitTS uint64) { - slotIDs := latches.genSlotIDs(keys) - for _, slotID := range slotIDs { - latches.slots[slotID].refreshCommitTS(commitTS) - } -} - -func (latches *Latches) releaseSlot(slotID int, commitTS uint64) (nextLock *Lock) { +func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) { + key := lock.keys[lock.acquiredCount-1] + slotID := lock.requiredSlots[lock.acquiredCount-1] latch := &latches.slots[slotID] + lock.acquiredCount-- latch.Lock() defer latch.Unlock() - latch.maxCommitTS = mathutil.MaxUint64(latch.maxCommitTS, commitTS) - if !latch.hasMoreWaiting { - latch.free() + + find := findNode(latch.queue, key) + if find.value != lock { + panic("releaseSlot wrong") + } + find.maxCommitTS = mathutil.MaxUint64(find.maxCommitTS, lock.commitTS) + find.value = nil + if len(latch.waiting) == 0 { return nil } - nextLock, latch.hasMoreWaiting = latches.popFromWaitingQueue(slotID) - latch.waitingQueueHead = nextLock.startTS - nextLock.acquiredCount++ - if latch.maxCommitTS > nextLock.startTS { - nextLock.isStale = true + + var idx int + for idx = 0; idx < len(latch.waiting); idx++ { + waiting := latch.waiting[idx] + if bytes.Compare(waiting.keys[waiting.acquiredCount], key) == 0 { + break + } } - return nextLock -} + // Wake up the first one in waiting queue. + if idx < len(latch.waiting) { + nextLock = latch.waiting[idx] + // Delete element latch.waiting[idx] from the array. + copy(latch.waiting[idx:], latch.waiting[idx+1:]) + latch.waiting[len(latch.waiting)-1] = nil + latch.waiting = latch.waiting[:len(latch.waiting)-1] -func (latches *Latches) popFromWaitingQueue(slotID int) (front *Lock, hasMoreWaiting bool) { - latches.Lock() - defer latches.Unlock() - waiting := latches.waitingQueues[slotID] - front = waiting[0] - if len(waiting) == 1 { - delete(latches.waitingQueues, slotID) - } else { - latches.waitingQueues[slotID] = waiting[1:] - hasMoreWaiting = true + if find.maxCommitTS > nextLock.startTS { + nextLock.isStale = true + } } + return } -func (latches *Latches) acquireSlot(slotID int, lock *Lock) acquireResult { +func (latches *Latches) acquireSlot(lock *Lock) acquireResult { + key := lock.keys[lock.acquiredCount] + slotID := lock.requiredSlots[lock.acquiredCount] latch := &latches.slots[slotID] latch.Lock() defer latch.Unlock() - if latch.maxCommitTS > lock.startTS { + + // Try to recycle to limit the memory usage. + if latch.count >= latchListCount { + latch.recycle(lock.startTS) + } + + find := findNode(latch.queue, key) + if find == nil { + tmp := &node{ + slotID: slotID, + key: key, + value: lock, + } + tmp.next = latch.queue + latch.queue = tmp + latch.count++ + + lock.acquiredCount++ + return acquireSuccess + } + + if find.maxCommitTS > lock.startTS { lock.isStale = true return acquireStale } - if latch.isEmpty() { - latch.waitingQueueHead = lock.startTS + if find.value == nil { + find.value = lock lock.acquiredCount++ return acquireSuccess } + // Push the current transaction into waitingQueue. - latch.hasMoreWaiting = true - latches.Lock() - defer latches.Unlock() - latches.waitingQueues[slotID] = append(latches.waitingQueues[slotID], lock) + latch.waiting = append(latch.waiting, lock) return acquireLocked } + +// recycle is not thread safe, the latch should acquire its lock before executing this function. +func (l *latch) recycle(currentTS uint64) { + fakeHead := node{next: l.queue} + prev := &fakeHead + for curr := prev.next; curr != nil; curr = curr.next { + if tsoSub(currentTS, curr.maxCommitTS) >= expireDuration && curr.value == nil { + l.count-- + prev.next = curr.next + } else { + prev = curr + } + } + l.queue = fakeHead.next +} + +func (latches *Latches) recycle(currentTS uint64) { + for i := 0; i < len(latches.slots); i++ { + latch := &latches.slots[i] + latch.Lock() + latch.recycle(currentTS) + latch.Unlock() + } +} + +func findNode(list *node, key []byte) *node { + for n := list; n != nil; n = n.next { + if bytes.Compare(n.key, key) == 0 { + return n + } + } + return nil +} diff --git a/store/tikv/latch/latch_test.go b/store/tikv/latch/latch_test.go index 17178dd5c7850..951a9e3de1802 100644 --- a/store/tikv/latch/latch_test.go +++ b/store/tikv/latch/latch_test.go @@ -16,8 +16,10 @@ package latch import ( "sync/atomic" "testing" + "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/store/tikv/oracle" ) func TestT(t *testing.T) { @@ -48,7 +50,7 @@ func getTso() uint64 { func (s *testLatchSuite) TestWakeUp(c *C) { keysA := [][]byte{ - []byte("a"), []byte("b"), []byte("c"), []byte("c")} + []byte("a"), []byte("b"), []byte("c")} _, lockA := s.newLock(keysA) keysB := [][]byte{[]byte("d"), []byte("e"), []byte("a"), []byte("c")} @@ -65,7 +67,8 @@ func (s *testLatchSuite) TestWakeUp(c *C) { // A release lock, and get wakeup list. commitTSA := getTso() wakeupList := make([]*Lock, 0) - wakeupList = s.latches.release(lockA, commitTSA, wakeupList) + lockA.SetCommitTS(commitTSA) + wakeupList = s.latches.release(lockA, wakeupList) c.Assert(wakeupList[0].startTS, Equals, startTSB) // B acquire failed since startTSB has stale for some keys. @@ -73,7 +76,7 @@ func (s *testLatchSuite) TestWakeUp(c *C) { c.Assert(result, Equals, acquireStale) // B release lock since it received a stale. - wakeupList = s.latches.release(lockB, 0, wakeupList) + wakeupList = s.latches.release(lockB, wakeupList) c.Assert(wakeupList, HasLen, 0) // B restart:get a new startTS. @@ -85,7 +88,7 @@ func (s *testLatchSuite) TestWakeUp(c *C) { func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) { keys := [][]byte{ - []byte("a"), []byte("b"), []byte("c"), []byte("c")} + []byte("a"), []byte("b"), []byte("c")} _, lockA := s.newLock(keys) startTSB, lockB := s.newLock(keys) // acquire lockA success @@ -94,11 +97,53 @@ func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) { // release lockA commitTSA := getTso() wakeupList := make([]*Lock, 0) - s.latches.release(lockA, commitTSA, wakeupList) + lockA.SetCommitTS(commitTSA) + s.latches.release(lockA, wakeupList) c.Assert(commitTSA, Greater, startTSB) // acquire lockB first time, should be failed with stale since commitTSA > startTSB result = s.latches.acquire(lockB) c.Assert(result, Equals, acquireStale) - s.latches.release(lockB, 0, wakeupList) + s.latches.release(lockB, wakeupList) +} + +func (s *testLatchSuite) TestRecycle(c *C) { + latches := NewLatches(8) + now := time.Now() + startTS := oracle.ComposeTS(oracle.GetPhysical(now), 0) + lock := latches.genLock(startTS, [][]byte{ + []byte("a"), []byte("b"), + }) + lock1 := latches.genLock(startTS, [][]byte{ + []byte("b"), []byte("c"), + }) + c.Assert(latches.acquire(lock), Equals, acquireSuccess) + c.Assert(latches.acquire(lock1), Equals, acquireLocked) + lock.SetCommitTS(startTS + 1) + var wakeupList []*Lock + latches.release(lock, wakeupList) + + lock2 := latches.genLock(startTS+3, [][]byte{ + []byte("b"), []byte("c"), + }) + c.Assert(latches.acquire(lock2), Equals, acquireSuccess) + wakeupList = wakeupList[:0] + latches.release(lock2, wakeupList) + + allEmpty := true + for i := 0; i < len(latches.slots); i++ { + latch := &latches.slots[i] + if latch.queue != nil { + allEmpty = false + } + } + c.Assert(allEmpty, IsFalse) + + currentTS := oracle.ComposeTS(oracle.GetPhysical(now.Add(expireDuration)), 3) + latches.recycle(currentTS) + + for i := 0; i < len(latches.slots); i++ { + latch := &latches.slots[i] + c.Assert(latch.queue, IsNil) + } } diff --git a/store/tikv/latch/scheduler.go b/store/tikv/latch/scheduler.go index f3ffad7a77d9f..5f4e84cd7760d 100644 --- a/store/tikv/latch/scheduler.go +++ b/store/tikv/latch/scheduler.go @@ -15,15 +15,19 @@ package latch import ( "sync" + "time" + + "github.com/pingcap/tidb/store/tikv/oracle" ) const lockChanSize = 100 // LatchesScheduler is used to schedule latches for transactions. type LatchesScheduler struct { - latches *Latches - unlockCh chan *Lock - closed bool + latches *Latches + unlockCh chan *Lock + closed bool + lastRecycleTime uint64 sync.RWMutex } @@ -40,13 +44,31 @@ func NewScheduler(size uint) *LatchesScheduler { return scheduler } +// A transaction can last for at most 10 minutes, see also gcworker. +const expireDuration = 10 * time.Minute +const checkInterval = 5 * time.Minute +const checkCounter = 50000 +const latchListCount = 5 + func (scheduler *LatchesScheduler) run() { + var counter int wakeupList := make([]*Lock, 0) for lock := range scheduler.unlockCh { - wakeupList = scheduler.latches.release(lock, lock.commitTS, wakeupList) + wakeupList = scheduler.latches.release(lock, wakeupList) if len(wakeupList) > 0 { scheduler.wakeup(wakeupList) } + + if lock.commitTS > lock.startTS { + currentTS := lock.commitTS + elapsed := tsoSub(currentTS, scheduler.lastRecycleTime) + if elapsed > checkInterval && counter > checkCounter { + go scheduler.latches.recycle(lock.commitTS) + scheduler.lastRecycleTime = currentTS + counter = 0 + } + } + counter++ } } @@ -92,8 +114,8 @@ func (scheduler *LatchesScheduler) UnLock(lock *Lock) { } } -// RefreshCommitTS refreshes commitTS for keys. It could be used for the transaction not retryable, -// which would do 2PC directly and wouldn't get a lock. -func (scheduler *LatchesScheduler) RefreshCommitTS(keys [][]byte, commitTS uint64) { - scheduler.latches.refreshCommitTS(keys, commitTS) +func tsoSub(ts1, ts2 uint64) time.Duration { + t1 := oracle.GetTimeFromTS(ts1) + t2 := oracle.GetTimeFromTS(ts2) + return t1.Sub(t2) } diff --git a/store/tikv/latch/scheduler_test.go b/store/tikv/latch/scheduler_test.go index d57737fb9512d..fc2d5b1514d0f 100644 --- a/store/tikv/latch/scheduler_test.go +++ b/store/tikv/latch/scheduler_test.go @@ -29,7 +29,7 @@ func (s *testSchedulerSuite) SetUpTest(c *C) { func (s *testSchedulerSuite) TestWithConcurrency(c *C) { txns := [][][]byte{ - {[]byte("a"), []byte("a"), []byte("b"), []byte("c")}, + {[]byte("a"), []byte("b"), []byte("c")}, {[]byte("a"), []byte("d"), []byte("e"), []byte("f")}, {[]byte("e"), []byte("f"), []byte("g"), []byte("h")}, } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index b1edef7da5b97..18b7e1a17d31f 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -199,9 +199,6 @@ func (txn *tikvTxn) Commit(ctx context.Context) error { // When bypassLatch flag is true, commit directly. if bypassLatch { err = committer.executeAndWriteFinishBinlog(ctx) - if err == nil { - txn.store.txnLatches.RefreshCommitTS(committer.keys, committer.commitTS) - } log.Debug("[kv]", connID, " txnLatches enabled while txn not retryable, 2pc directly:", err) return errors.Trace(err) } From a84cce1bcabfdb931a3ebec226d739233645a91b Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 15 Oct 2018 13:13:43 +0800 Subject: [PATCH 008/509] executor: remove some useless code and avoid some redundancy check (#7639) (#7881) --- executor/builder.go | 18 ++-- executor/delete.go | 9 -- executor/executor_test.go | 6 ++ executor/insert.go | 5 - executor/insert_common.go | 222 +++++++++++++++++++------------------- executor/load_data.go | 2 +- executor/replace.go | 5 - executor/write.go | 14 +-- executor/write_test.go | 16 ++- table/column.go | 17 +++ table/column_test.go | 2 +- types/datum.go | 9 ++ 12 files changed, 172 insertions(+), 153 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index b16d39eee04fa..e7f63af8b8371 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -538,15 +538,15 @@ func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { baseExec.initCap = chunk.ZeroCapacity ivs := &InsertValues{ - baseExecutor: baseExec, - Table: v.Table, - Columns: v.Columns, - Lists: v.Lists, - SetList: v.SetList, - GenColumns: v.GenCols.Columns, - GenExprs: v.GenCols.Exprs, - needFillDefaultValues: v.NeedFillDefaultValue, - SelectExec: selectExec, + baseExecutor: baseExec, + Table: v.Table, + Columns: v.Columns, + Lists: v.Lists, + SetList: v.SetList, + GenColumns: v.GenCols.Columns, + GenExprs: v.GenCols.Exprs, + hasRefCols: v.NeedFillDefaultValue, + SelectExec: selectExec, } if v.IsReplace { diff --git a/executor/delete.go b/executor/delete.go index 59c9d1eee5fc7..026a1975666d8 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -39,20 +39,11 @@ type DeleteExec struct { // `delete from t as t1, t as t2`, the same table has two alias, we have to identify a table // by its alias instead of ID. tblMap map[int64][]*ast.TableName - - finished bool } // Next implements the Executor Next interface. func (e *DeleteExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - if e.finished { - return nil - } - defer func() { - e.finished = true - }() - if e.IsMultiTable { return errors.Trace(e.deleteMultiTablesByChunk(ctx)) } diff --git a/executor/executor_test.go b/executor/executor_test.go index 7a221de7de7db..3412567e0a53b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1451,6 +1451,12 @@ func (s *testSuite) TestGeneratedColumnRead(c *C) { result = tk.MustQuery(`SELECT * FROM test_gc_read WHERE d = 12`) result.Check(testkit.Rows(`3 4 7 12`)) + tk.MustExec(`INSERT INTO test_gc_read set a = 4, b = d + 1`) + result = tk.MustQuery(`SELECT * FROM test_gc_read ORDER BY a`) + result.Check(testkit.Rows(`0 `, `1 2 3 2`, `3 4 7 12`, + `4 `, `8 8 16 64`)) + tk.MustExec(`DELETE FROM test_gc_read where a = 4`) + // Test on-conditions on virtual/stored generated columns. tk.MustExec(`CREATE TABLE test_gc_help(a int primary key, b int, c int, d int)`) tk.MustExec(`INSERT INTO test_gc_help(a, b, c, d) SELECT * FROM test_gc_read`) diff --git a/executor/insert.go b/executor/insert.go index 6828a4d4e4990..932a2c09ce326 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -31,7 +31,6 @@ type InsertExec struct { *InsertValues OnDuplicate []*expression.Assignment Priority mysql.PriorityEnum - finished bool } func (e *InsertExec) exec(rows [][]types.Datum) error { @@ -68,7 +67,6 @@ func (e *InsertExec) exec(rows [][]types.Datum) error { } } } - e.finished = true return nil } @@ -131,9 +129,6 @@ func (e *InsertExec) batchUpdateDupRows(newRows [][]types.Datum) error { // Next implements Exec Next interface. func (e *InsertExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - if e.finished { - return nil - } cols, err := e.getColumns(e.Table.Cols()) if err != nil { return errors.Trace(err) diff --git a/executor/insert_common.go b/executor/insert_common.go index 628bead17cfbe..2517420943feb 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -34,11 +34,11 @@ type InsertValues struct { baseExecutor batchChecker - rowCount uint64 - maxRowsInBatch uint64 - lastInsertID uint64 - needFillDefaultValues bool - hasExtraHandle bool + rowCount uint64 + maxRowsInBatch uint64 + lastInsertID uint64 + hasRefCols bool + hasExtraHandle bool SelectExec Executor @@ -135,7 +135,7 @@ func (e *InsertValues) lazilyInitColDefaultValBuf() (ok bool) { return false } -func (e *InsertValues) fillValueList() error { +func (e *InsertValues) processSetList() error { if len(e.SetList) > 0 { if len(e.Lists) > 0 { return errors.Errorf("INSERT INTO %s: set type should not use values", e.Table) @@ -149,19 +149,21 @@ func (e *InsertValues) fillValueList() error { return nil } +// insertRows processes `insert|replace into values ()` or `insert|replace into set x=y` func (e *InsertValues) insertRows(cols []*table.Column, exec func(rows [][]types.Datum) error) (err error) { - // process `insert|replace ... set x=y...` - if err = e.fillValueList(); err != nil { + // For `insert|replace into set x=y`, process the set list here. + if err = e.processSetList(); err != nil { return errors.Trace(err) } - rows := make([][]types.Datum, len(e.Lists)) + rows := make([][]types.Datum, 0, len(e.Lists)) for i, list := range e.Lists { e.rowCount++ - rows[i], err = e.getRow(cols, list, i) + row, err := e.evalRow(cols, list, i) if err != nil { return errors.Trace(err) } + rows = append(rows, row) } return errors.Trace(exec(rows)) } @@ -185,9 +187,9 @@ func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int return e.filterErr(err) } -// getRow eval the insert statement. Because the value of column may calculated based on other column, -// it use fillDefaultValues to init the empty row before eval expressions when needFillDefaultValues is true. -func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression, rowIdx int) ([]types.Datum, error) { +// evalRow evaluates a to-be-inserted row. The value of the column may base on another column, +// so we use setValueForRefColumn to fill the empty row some default values when needFillDefaultValues is true. +func (e *InsertValues) evalRow(cols []*table.Column, list []expression.Expression, rowIdx int) ([]types.Datum, error) { rowLen := len(e.Table.Cols()) if e.hasExtraHandle { rowLen++ @@ -195,8 +197,9 @@ func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression row := make([]types.Datum, rowLen) hasValue := make([]bool, rowLen) - if e.needFillDefaultValues { - if err := e.fillDefaultValues(row, hasValue); err != nil { + // For statements like `insert into t set a = b + 1`. + if e.hasRefCols { + if err := e.setValueForRefColumn(row, hasValue); err != nil { return nil, errors.Trace(err) } } @@ -215,34 +218,36 @@ func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression row[offset], hasValue[offset] = val1, true } - return e.fillGenColData(cols, len(list), hasValue, row) + return e.fillRow(row, hasValue) } -// fillDefaultValues fills a row followed by these rules: +// setValueForRefColumn set some default values for the row to eval the row value with other columns, +// it follows these rules: // 1. for nullable and no default value column, use NULL. // 2. for nullable and have default value column, use it's default value. // 3. for not null column, use zero value even in strict mode. // 4. for auto_increment column, use zero value. // 5. for generated column, use NULL. -func (e *InsertValues) fillDefaultValues(row []types.Datum, hasValue []bool) error { +func (e *InsertValues) setValueForRefColumn(row []types.Datum, hasValue []bool) error { for i, c := range e.Table.Cols() { - var err error - if c.IsGenerated() { - continue - } else if mysql.HasAutoIncrementFlag(c.Flag) { - row[i] = table.GetZeroValue(c.ToInfo()) - } else { - row[i], err = e.getColDefaultValue(i, c) - hasValue[c.Offset] = true - if table.ErrNoDefaultValue.Equal(err) { - row[i] = table.GetZeroValue(c.ToInfo()) - hasValue[c.Offset] = false - } else if e.filterErr(err) != nil { - return errors.Trace(err) + d, err := e.getColDefaultValue(i, c) + if err == nil { + row[i] = d + if !mysql.HasAutoIncrementFlag(c.Flag) { + // It is an interesting behavior in MySQL. + // If the value of auto ID is not explicit, MySQL use 0 value for auto ID when it is + // evaluated by another column, but it should be used once only. + // When we fill it as an auto ID column, it should be set as it used to be. + // So just keep `hasValue` false for auto ID, and the others set true. + hasValue[c.Offset] = true } + } else if table.ErrNoDefaultValue.Equal(err) { + row[i] = table.GetZeroValue(c.ToInfo()) + hasValue[c.Offset] = false + } else if e.filterErr(err) != nil { + return errors.Trace(err) } } - return nil } @@ -270,7 +275,7 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C for innerChunkRow := iter.Begin(); innerChunkRow != iter.End(); innerChunkRow = iter.Next() { innerRow := types.CopyRow(innerChunkRow.GetDatumRow(fields)) e.rowCount++ - row, err := e.fillRowData(cols, innerRow) + row, err := e.getRow(cols, innerRow) if err != nil { return errors.Trace(err) } @@ -297,7 +302,10 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C return nil } -func (e *InsertValues) fillRowData(cols []*table.Column, vals []types.Datum) ([]types.Datum, error) { +// getRow gets the row which from `insert into select from` or `load data`. +// The input values from these two statements are datums instead of +// expressions which are used in `insert into set x=y`. +func (e *InsertValues) getRow(cols []*table.Column, vals []types.Datum) ([]types.Datum, error) { row := make([]types.Datum, len(e.Table.Cols())) hasValue := make([]bool, len(e.Table.Cols())) for i, v := range vals { @@ -311,32 +319,7 @@ func (e *InsertValues) fillRowData(cols []*table.Column, vals []types.Datum) ([] hasValue[offset] = true } - return e.fillGenColData(cols, len(vals), hasValue, row) -} - -func (e *InsertValues) fillGenColData(cols []*table.Column, valLen int, hasValue []bool, row []types.Datum) ([]types.Datum, error) { - err := e.initDefaultValues(row, hasValue) - if err != nil { - return nil, errors.Trace(err) - } - for i, expr := range e.GenExprs { - var val types.Datum - val, err = expr.Eval(chunk.MutRowFromDatums(row).ToRow()) - if e.filterErr(err) != nil { - return nil, errors.Trace(err) - } - val, err = table.CastValue(e.ctx, val, cols[valLen+i].ToInfo()) - if err != nil { - return nil, errors.Trace(err) - } - offset := cols[valLen+i].Offset - row[offset] = val - } - - if err = table.CheckNotNull(e.Table.Cols(), row); err != nil { - return nil, errors.Trace(err) - } - return row, nil + return e.fillRow(row, hasValue) } func (e *InsertValues) filterErr(err error) error { @@ -351,6 +334,7 @@ func (e *InsertValues) filterErr(err error) error { return nil } +// getColDefaultValue gets the column default value. func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.Datum, err error) { if e.colDefaultVals != nil && e.colDefaultVals[idx].valid { return e.colDefaultVals[idx].val, nil @@ -368,81 +352,100 @@ func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.D return defaultVal, nil } -// initDefaultValues fills generated columns, auto_increment column and empty column. +// fillColValue fills the column value if it is not set in the insert statement. +func (e *InsertValues) fillColValue(datum types.Datum, idx int, column *table.Column, hasValue bool) (types.Datum, + error) { + if mysql.HasAutoIncrementFlag(column.Flag) { + d, err := e.adjustAutoIncrementDatum(datum, hasValue, column) + if err != nil { + return types.Datum{}, errors.Trace(err) + } + return d, nil + } + if !hasValue { + d, err := e.getColDefaultValue(idx, column) + if e.filterErr(err) != nil { + return types.Datum{}, errors.Trace(err) + } + return d, nil + } + return datum, nil +} + +// fillRow fills generated columns, auto_increment column and empty column. // For NOT NULL column, it will return error or use zero value based on sql_mode. -func (e *InsertValues) initDefaultValues(row []types.Datum, hasValue []bool) error { +func (e *InsertValues) fillRow(row []types.Datum, hasValue []bool) ([]types.Datum, error) { + gIdx := 0 for i, c := range e.Table.Cols() { - if mysql.HasAutoIncrementFlag(c.Flag) || c.IsGenerated() { - // Just leave generated column as null. It will be calculated later - // but before we check whether the column can be null or not. - if !hasValue[i] { - row[i].SetNull() - } - // Adjust the value if this column has auto increment flag. - if mysql.HasAutoIncrementFlag(c.Flag) { - if err := e.adjustAutoIncrementDatum(row, i, c); err != nil { - return errors.Trace(err) - } + var err error + // Get the default value for all no value columns, the auto increment column is different from the others. + row[i], err = e.fillColValue(row[i], i, c, hasValue[i]) + if err != nil { + return nil, errors.Trace(err) + } + + // Evaluate the generated columns. + if c.IsGenerated() { + var val types.Datum + val, err = e.GenExprs[gIdx].Eval(chunk.MutRowFromDatums(row).ToRow()) + gIdx++ + if e.filterErr(err) != nil { + return nil, errors.Trace(err) } - } else { - if !hasValue[i] || (mysql.HasNotNullFlag(c.Flag) && row[i]. - IsNull() && e.ctx.GetSessionVars().StmtCtx.BadNullAsWarning) { - var err error - row[i], err = e.getColDefaultValue(i, c) - if e.filterErr(err) != nil { - return errors.Trace(err) - } + row[i], err = table.CastValue(e.ctx, val, c.ToInfo()) + if err != nil { + return nil, errors.Trace(err) } } + + // Handle the bad null error. + if row[i], err = c.HandleBadNull(row[i], e.ctx.GetSessionVars().StmtCtx); err != nil { + return nil, errors.Trace(err) + } } - return nil + return row, nil } -func (e *InsertValues) adjustAutoIncrementDatum(row []types.Datum, i int, c *table.Column) error { +func (e *InsertValues) adjustAutoIncrementDatum(d types.Datum, hasValue bool, c *table.Column) (types.Datum, error) { retryInfo := e.ctx.GetSessionVars().RetryInfo if retryInfo.Retrying { id, err := retryInfo.GetCurrAutoIncrementID() if err != nil { - return errors.Trace(err) + return types.Datum{}, errors.Trace(err) } - if mysql.HasUnsignedFlag(c.Flag) { - row[i].SetUint64(uint64(id)) - } else { - row[i].SetInt64(id) - } - return nil + d.SetAutoID(id, c.Flag) + return d, nil } var err error var recordID int64 - if !row[i].IsNull() { - recordID, err = row[i].ToInt64(e.ctx.GetSessionVars().StmtCtx) + if !hasValue { + d.SetNull() + } + if !d.IsNull() { + recordID, err = d.ToInt64(e.ctx.GetSessionVars().StmtCtx) if e.filterErr(err) != nil { - return errors.Trace(err) + return types.Datum{}, errors.Trace(err) } } // Use the value if it's not null and not 0. if recordID != 0 { err = e.Table.RebaseAutoID(e.ctx, recordID, true) if err != nil { - return errors.Trace(err) + return types.Datum{}, errors.Trace(err) } e.ctx.GetSessionVars().InsertID = uint64(recordID) - if mysql.HasUnsignedFlag(c.Flag) { - row[i].SetUint64(uint64(recordID)) - } else { - row[i].SetInt64(recordID) - } retryInfo.AddAutoIncrementID(recordID) - return nil + d.SetAutoID(recordID, c.Flag) + return d, nil } // Change NULL to auto id. // Change value 0 to auto id, if NoAutoValueOnZero SQL mode is not set. - if row[i].IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 { + if d.IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 { recordID, err = e.Table.AllocAutoID(e.ctx) if e.filterErr(err) != nil { - return errors.Trace(err) + return types.Datum{}, errors.Trace(err) } // It's compatible with mysql. So it sets last insert id to the first row. if e.rowCount == 1 { @@ -450,20 +453,15 @@ func (e *InsertValues) adjustAutoIncrementDatum(row []types.Datum, i int, c *tab } } - if mysql.HasUnsignedFlag(c.Flag) { - row[i].SetUint64(uint64(recordID)) - } else { - row[i].SetInt64(recordID) - } + d.SetAutoID(recordID, c.Flag) retryInfo.AddAutoIncrementID(recordID) - // the value of row[i] is adjusted by autoid, so we need to cast it again. - casted, err := table.CastValue(e.ctx, row[i], c.ToInfo()) + // the value of d is adjusted by auto ID, so we need to cast it again. + casted, err := table.CastValue(e.ctx, d, c.ToInfo()) if err != nil { - return errors.Trace(err) + return types.Datum{}, errors.Trace(err) } - row[i] = casted - return nil + return casted, nil } func (e *InsertValues) handleWarning(err error, logInfo string) { diff --git a/executor/load_data.go b/executor/load_data.go index 652ad68d1aa09..d6ee3724a3a74 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -274,7 +274,7 @@ func (e *LoadDataInfo) colsToRow(cols []field) []types.Datum { e.row[i].SetString(string(cols[i].str)) } } - row, err := e.fillRowData(e.columns, e.row) + row, err := e.getRow(e.columns, e.row) if err != nil { e.handleWarning(err, fmt.Sprintf("Load Data: insert data:%v failed:%v", e.row, errors.ErrorStack(err))) diff --git a/executor/replace.go b/executor/replace.go index 6f416c9ea0f9b..dbe79cfde4817 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -27,7 +27,6 @@ import ( type ReplaceExec struct { *InsertValues Priority int - finished bool } // Close implements the Executor Close interface. @@ -173,16 +172,12 @@ func (e *ReplaceExec) exec(newRows [][]types.Datum) error { return errors.Trace(err) } } - e.finished = true return nil } // Next implements the Executor Next interface. func (e *ReplaceExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - if e.finished { - return nil - } cols, err := e.getColumns(e.Table.Cols()) if err != nil { return errors.Trace(err) diff --git a/executor/write.go b/executor/write.go index dc4c993182720..de34f67bce23b 100644 --- a/executor/write.go +++ b/executor/write.go @@ -68,17 +68,11 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } } - // 2. Check null. + // 2. Handle the bad null error. for i, col := range t.Cols() { - if err := col.CheckNotNull(newData[i]); err != nil { - if sc.BadNullAsWarning { - newData[i], err = table.GetColDefaultValue(ctx, col.ToInfo()) - if err != nil { - return false, false, 0, errors.Trace(err) - } - } else { - return false, false, 0, errors.Trace(err) - } + var err error + if newData[i], err = col.HandleBadNull(newData[i], sc); err != nil { + return false, false, 0, errors.Trace(err) } } diff --git a/executor/write_test.go b/executor/write_test.go index 033798c754a82..98a91cf58d455 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -635,11 +635,16 @@ commit;` tk.MustQuery(`SELECT * FROM t1;`).Check(testkit.Rows("1")) testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT UNIQUE); + CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); INSERT t1 VALUES (1, 1);` tk.MustExec(testSQL) tk.MustExec(`INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2;`) tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 1", "2 2")) + _, err := tk.Exec(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) + c.Assert(err, NotNil) + tk.MustExec(`INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) + tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) } func (s *testSuite) TestInsertIgnoreOnDup(c *C) { @@ -1056,6 +1061,15 @@ func (s *testSuite) TestUpdate(c *C) { tk.MustExec("update (select * from t) t set c1 = 1111111") + // test update ignore for bad null error + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (i int not null default 10)`) + tk.MustExec("insert into t values (1)") + tk.MustExec("update ignore t set i = null;") + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) + tk.MustQuery("select * from t").Check(testkit.Rows("0")) + // issue 7237, update subquery table should be forbidden tk.MustExec("drop table t") tk.MustExec("create table t (k int, v int)") diff --git a/table/column.go b/table/column.go index c2b408943f5c5..8e6bb6925a13d 100644 --- a/table/column.go +++ b/table/column.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/charset" @@ -291,6 +292,19 @@ func (c *Column) CheckNotNull(data types.Datum) error { return nil } +// HandleBadNull handles the bad null error. +// If BadNullAsWarning is true, it will append the error as a warning, else return the error. +func (c *Column) HandleBadNull(d types.Datum, sc *stmtctx.StatementContext) (types.Datum, error) { + if err := c.CheckNotNull(d); err != nil { + if sc.BadNullAsWarning { + sc.AppendWarning(err) + return GetZeroValue(c.ToInfo()), nil + } + return types.Datum{}, errors.Trace(err) + } + return d, nil +} + // IsPKHandleColumn checks if the column is primary key handle column. func (c *Column) IsPKHandleColumn(tbInfo *model.TableInfo) bool { return mysql.HasPriKeyFlag(c.Flag) && tbInfo.PKIsHandle @@ -348,6 +362,9 @@ func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo) (t } if mysql.HasAutoIncrementFlag(col.Flag) { // Auto increment column doesn't has default value and we should not return error. + return GetZeroValue(col), nil + } + if col.IsGenerated() { return types.Datum{}, nil } sc := ctx.GetSessionVars().StmtCtx diff --git a/table/column_test.go b/table/column_test.go index bf17187a4d9ab..4bf3bc104c0c3 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -338,7 +338,7 @@ func (t *testTableSuite) TestGetDefaultValue(c *C) { }, }, true, - types.Datum{}, + types.NewIntDatum(0), nil, }, } diff --git a/types/datum.go b/types/datum.go index 641cd889d32d9..d670520ae06dd 100644 --- a/types/datum.go +++ b/types/datum.go @@ -323,6 +323,15 @@ func (d *Datum) GetRaw() []byte { return d.b } +// SetAutoID set the auto increment ID according to its int flag. +func (d *Datum) SetAutoID(id int64, flag uint) { + if mysql.HasUnsignedFlag(flag) { + d.SetUint64(uint64(id)) + } else { + d.SetInt64(id) + } +} + // GetValue gets the value of the datum of any kind. func (d *Datum) GetValue() interface{} { switch d.k { From 8fae90e2b9e406a98c1366340f1c863470042c61 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 15 Oct 2018 18:06:06 +0800 Subject: [PATCH 009/509] expression: make sysdate unfoldable (#7838) (#7895) --- cmd/explaintest/r/select.result | 4 ++++ cmd/explaintest/t/select.test | 2 ++ expression/function_traits.go | 1 + expression/function_traits_test.go | 26 ++++++++++++++++++++++++++ 4 files changed, 33 insertions(+) create mode 100644 expression/function_traits_test.go diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index accca3152896f..de524a7de28b0 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -328,3 +328,7 @@ Point_Get_1 1.00 root table:t, handle:1 desc select * from t where a = '1'; id count task operator info Point_Get_1 1.00 root table:t, handle:1 +desc select sysdate(), sleep(1), sysdate(); +id count task operator info +Projection_3 1.00 root sysdate(), sleep(1), sysdate() +└─TableDual_4 1.00 root rows:1 diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index 783b520e31dd0..d062a7e047a4d 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -163,3 +163,5 @@ drop table if exists t; create table t(a bigint primary key, b bigint); desc select * from t where a = 1; desc select * from t where a = '1'; + +desc select sysdate(), sleep(1), sysdate(); diff --git a/expression/function_traits.go b/expression/function_traits.go index 9aa6952416eb1..ae3f4b05ea888 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -40,6 +40,7 @@ var UnCacheableFunctions = map[string]struct{}{ // unFoldableFunctions stores functions which can not be folded duration constant folding stage. var unFoldableFunctions = map[string]struct{}{ + ast.Sysdate: {}, ast.FoundRows: {}, ast.Rand: {}, ast.UUID: {}, diff --git a/expression/function_traits_test.go b/expression/function_traits_test.go new file mode 100644 index 0000000000000..13982d9e85907 --- /dev/null +++ b/expression/function_traits_test.go @@ -0,0 +1,26 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package expression + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/util/testleak" +) + +func (s *testEvaluatorSuite) TestUnfoldableFuncs(c *C) { + defer testleak.AfterTest(c)() + _, ok := unFoldableFunctions[ast.Sysdate] + c.Assert(ok, IsTrue) +} From fd5d6660f48d3e6e5a2d58b2da1fa8350214fec8 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 15 Oct 2018 20:08:17 +0800 Subject: [PATCH 010/509] expression: fix painc on substring_index (#7806) (#7897) --- expression/builtin_string.go | 5 +++++ expression/integration_test.go | 1 + 2 files changed, 6 insertions(+) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index f20c7fe27a377..fd429d893a354 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -1210,6 +1210,11 @@ func (b *builtinSubstringIndexSig) evalString(row chunk.Row) (d string, isNull b } else { // If count is negative, everything to the right of the final delimiter (counting from the right) is returned. count = -count + if count < 0 { + // -count overflows max int64, returns an empty string. + return "", false, nil + } + if count < end { start = end - count } diff --git a/expression/integration_test.go b/expression/integration_test.go index 089732337147e..cdb8063583d94 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -736,6 +736,7 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { result.Check(testkit.Rows("www.pingcap 12345 45 2017 01:01")) result = tk.MustQuery(`select substring_index('www.pingcap.com', '.', 0), substring_index('www.pingcap.com', '.', 100), substring_index('www.pingcap.com', '.', -100)`) result.Check(testkit.Rows(" www.pingcap.com www.pingcap.com")) + tk.MustQuery(`select substring_index('xyz', 'abc', 9223372036854775808)`).Check(testkit.Rows(``)) result = tk.MustQuery(`select substring_index('www.pingcap.com', 'd', 1), substring_index('www.pingcap.com', '', 1), substring_index('', '.', 1)`) result.Check(testutil.RowsWithSep(",", "www.pingcap.com,,")) result = tk.MustQuery(`select substring_index(null, '.', 1), substring_index('www.pingcap.com', null, 1), substring_index('www.pingcap.com', '.', null)`) From 62a6be9e558fd1fe8ac6e48714ae61ee935341ab Mon Sep 17 00:00:00 2001 From: disksing Date: Tue, 16 Oct 2018 14:57:18 +0800 Subject: [PATCH 011/509] *: udpate pd client vendor (#7905) --- Gopkg.lock | 9 +++++---- store/mockstore/mocktikv/mock.go | 2 +- store/mockstore/mocktikv/pd.go | 2 +- store/mockstore/tikv.go | 2 +- store/tikv/gcworker/gc_worker.go | 2 +- store/tikv/interface.go | 2 +- store/tikv/kv.go | 2 +- store/tikv/lock_resolver.go | 2 +- store/tikv/oracle/oracles/pd.go | 4 ++-- store/tikv/pd_codec.go | 2 +- store/tikv/rawkv.go | 2 +- store/tikv/region_cache.go | 2 +- store/tikv/store_test.go | 2 +- store/tikv/test_util.go | 2 +- .../pingcap/pd/{pd-client => client}/client.go | 4 +++- .../pingcap/pd/{pd-client => client}/metrics.go | 0 16 files changed, 22 insertions(+), 19 deletions(-) rename vendor/github.com/pingcap/pd/{pd-client => client}/client.go (99%) rename vendor/github.com/pingcap/pd/{pd-client => client}/metrics.go (100%) diff --git a/Gopkg.lock b/Gopkg.lock index 6c299c8305fcc..9f90a5ba400d1 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -291,11 +291,11 @@ [[projects]] branch = "master" - digest = "1:2c64fe28351dc49da088194d80faceb34186e080a60c649694d094b2ccccacf5" + digest = "1:8f576565a8479071ca1951bf678519b1c32480aad8dec2379a64858c31d9e6a9" name = "github.com/pingcap/pd" - packages = ["pd-client"] + packages = ["client"] pruneopts = "NUT" - revision = "b5992e356ff447a6fdaa450b3f5ea33afbd11680" + revision = "eb892dda1e33a0b76191d39894ad4a806f313f6e" [[projects]] branch = "master" @@ -532,6 +532,7 @@ "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing", "github.com/grpc-ecosystem/go-grpc-prometheus", "github.com/ngaut/pools", + "github.com/ngaut/sync2", "github.com/opentracing/basictracer-go", "github.com/opentracing/opentracing-go", "github.com/pingcap/check", @@ -547,7 +548,7 @@ "github.com/pingcap/kvproto/pkg/kvrpcpb", "github.com/pingcap/kvproto/pkg/metapb", "github.com/pingcap/kvproto/pkg/tikvpb", - "github.com/pingcap/pd/pd-client", + "github.com/pingcap/pd/client", "github.com/pingcap/tipb/go-binlog", "github.com/pingcap/tipb/go-tipb", "github.com/pkg/errors", diff --git a/store/mockstore/mocktikv/mock.go b/store/mockstore/mocktikv/mock.go index 0d43376a6883d..1b4e08334a54f 100644 --- a/store/mockstore/mocktikv/mock.go +++ b/store/mockstore/mocktikv/mock.go @@ -14,7 +14,7 @@ package mocktikv import ( - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pkg/errors" ) diff --git a/store/mockstore/mocktikv/pd.go b/store/mockstore/mocktikv/pd.go index bb99ce0e2f488..e47e05c55963e 100644 --- a/store/mockstore/mocktikv/pd.go +++ b/store/mockstore/mocktikv/pd.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "golang.org/x/net/context" ) diff --git a/store/mockstore/tikv.go b/store/mockstore/tikv.go index 796aa74e2ba58..7afbe7726bd23 100644 --- a/store/mockstore/tikv.go +++ b/store/mockstore/tikv.go @@ -17,7 +17,7 @@ import ( "net/url" "strings" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/mocktikv" diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index fc18742e04ead..bf15962e867e3 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" diff --git a/store/tikv/interface.go b/store/tikv/interface.go index 88ed848c73b2b..8c111218b41b0 100644 --- a/store/tikv/interface.go +++ b/store/tikv/interface.go @@ -16,7 +16,7 @@ package tikv import ( "time" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 84d6747f3a344..fc834d1e530a3 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -24,7 +24,7 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/grpc-ecosystem/go-grpc-prometheus" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index ba8ab179b85c8..a85415398dde6 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" diff --git a/store/tikv/oracle/oracles/pd.go b/store/tikv/oracle/oracles/pd.go index d02748c3528ef..a8e7db8d8a02f 100644 --- a/store/tikv/oracle/oracles/pd.go +++ b/store/tikv/oracle/oracles/pd.go @@ -17,7 +17,7 @@ import ( "sync/atomic" "time" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pkg/errors" @@ -37,7 +37,7 @@ type pdOracle struct { } // NewPdOracle create an Oracle that uses a pd client source. -// Refer https://github.com/pingcap/pd/blob/master/pd-client/client.go for more details. +// Refer https://github.com/pingcap/pd/blob/master/client/client.go for more details. // PdOracle mantains `lastTS` to store the last timestamp got from PD server. If // `GetTimestamp()` is not called after `updateInterval`, it will be called by // itself to keep up with the timestamp on PD server. diff --git a/store/tikv/pd_codec.go b/store/tikv/pd_codec.go index e009a6c17e8d7..1e871c4a223e2 100644 --- a/store/tikv/pd_codec.go +++ b/store/tikv/pd_codec.go @@ -15,7 +15,7 @@ package tikv import ( "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" "golang.org/x/net/context" diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 44eeef9cb95f1..adbd12b656ed7 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 9bf21762a0622..381af34c5c6bf 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -22,7 +22,7 @@ import ( "github.com/google/btree" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/metrics" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/store/tikv/store_test.go b/store/tikv/store_test.go index 2fe3af12b194d..00735eee64a3f 100644 --- a/store/tikv/store_test.go +++ b/store/tikv/store_test.go @@ -20,7 +20,7 @@ import ( . "github.com/pingcap/check" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockoracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" diff --git a/store/tikv/test_util.go b/store/tikv/test_util.go index 060ea68a626c0..d4efcbbd0bc3e 100644 --- a/store/tikv/test_util.go +++ b/store/tikv/test_util.go @@ -14,7 +14,7 @@ package tikv import ( - "github.com/pingcap/pd/pd-client" + "github.com/pingcap/pd/client" "github.com/pingcap/tidb/kv" "github.com/pkg/errors" "github.com/twinj/uuid" diff --git a/vendor/github.com/pingcap/pd/pd-client/client.go b/vendor/github.com/pingcap/pd/client/client.go similarity index 99% rename from vendor/github.com/pingcap/pd/pd-client/client.go rename to vendor/github.com/pingcap/pd/client/client.go index 31d039558f609..1f0c643d1c972 100644 --- a/vendor/github.com/pingcap/pd/pd-client/client.go +++ b/vendor/github.com/pingcap/pd/client/client.go @@ -163,7 +163,9 @@ func (c *client) initClusterID() error { defer cancel() for i := 0; i < maxInitClusterRetries; i++ { for _, u := range c.urls { - members, err := c.getMembers(ctx, u) + timeoutCtx, timeoutCancel := context.WithTimeout(ctx, pdTimeout) + members, err := c.getMembers(timeoutCtx, u) + timeoutCancel() if err != nil || members.GetHeader() == nil { log.Errorf("[pd] failed to get cluster id: %v", err) continue diff --git a/vendor/github.com/pingcap/pd/pd-client/metrics.go b/vendor/github.com/pingcap/pd/client/metrics.go similarity index 100% rename from vendor/github.com/pingcap/pd/pd-client/metrics.go rename to vendor/github.com/pingcap/pd/client/metrics.go From c91290fd8ddc0ca02abe79a776b330c8502ceab1 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 17 Oct 2018 16:47:23 +0800 Subject: [PATCH 012/509] stats: fix panic caused by empty histogram (#7912) (#7928) --- statistics/histogram.go | 4 ++-- statistics/selectivity_test.go | 12 ++++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/statistics/histogram.go b/statistics/histogram.go index f56072ffb29f1..ca52273b20e1a 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -679,7 +679,7 @@ func (hg *Histogram) AvgCountPerValue(totalCount int64) float64 { } func (hg *Histogram) outOfRange(val types.Datum) bool { - if hg.Bounds == nil { + if hg.Len() == 0 { return true } return chunk.Compare(hg.Bounds.GetRow(0), 0, &val) > 0 || @@ -858,7 +858,7 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range } func (idx *Index) outOfRange(val types.Datum) bool { - if idx.Bounds == nil { + if idx.Histogram.Len() == 0 { return true } withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index ed24d0b97f326..77225890c2fca 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -292,6 +292,18 @@ func (s *testSelectivitySuite) TestEstimationForUnknownValues(c *C) { count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(9, 30)) c.Assert(err, IsNil) c.Assert(count, Equals, 2.2) + + testKit.MustExec("truncate table t") + testKit.MustExec("insert into t values (null, null)") + testKit.MustExec("analyze table t") + table, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + statsTbl = h.GetTableStats(table.Meta()) + + colID = table.Meta().Columns[0].ID + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(1, 30)) + c.Assert(err, IsNil) + c.Assert(count, Equals, 0.0) } func BenchmarkSelectivity(b *testing.B) { From 52d5ee27435e2d16c7fd9c86487974e9abd047ee Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 18 Oct 2018 13:16:38 +0800 Subject: [PATCH 013/509] *: make `explain` support `explain anaylze` (#7827)(#7888) (#7925) --- ast/misc.go | 5 +-- executor/adapter.go | 4 +-- executor/aggregate.go | 10 +++++- executor/builder.go | 15 ++++---- executor/distsql.go | 17 +++++++-- executor/executor.go | 35 ++++++++++++++++++ executor/explain.go | 46 ++++++++++++++++++++++-- executor/index_lookup_join.go | 5 +++ executor/join.go | 9 +++++ executor/merge_join.go | 6 ++++ executor/projection.go | 6 ++++ executor/sort.go | 9 +++++ executor/table_reader.go | 6 ++++ executor/union_scan.go | 5 +++ parser/parser.y | 8 +++++ planner/core/cbo_test.go | 30 ++++++++++++++++ planner/core/common_plans.go | 64 ++++++++++++++++++++++++++++++--- planner/core/planbuilder.go | 26 +++----------- sessionctx/stmtctx/stmtctx.go | 13 +++---- util/execdetails/execdetails.go | 49 +++++++++++++++++++++++++ 20 files changed, 320 insertions(+), 48 deletions(-) diff --git a/ast/misc.go b/ast/misc.go index 98ae76ca53ac0..620a9abf34c12 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -118,8 +118,9 @@ func (n *TraceStmt) Accept(v Visitor) (Node, bool) { type ExplainStmt struct { stmtNode - Stmt StmtNode - Format string + Stmt StmtNode + Format string + Analyze bool } // Accept implements Node Accept interface. diff --git a/executor/adapter.go b/executor/adapter.go index e877dd6e126e7..4849b8012997f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -365,12 +365,12 @@ func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) { if sessVars.InRestrictedSQL { internal = "[INTERNAL] " } + execDetail := sessVars.StmtCtx.GetExecDetails() if costTime < threshold { logutil.SlowQueryLogger.Debugf( "[QUERY] %vcost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", - internal, costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) + internal, costTime, execDetail, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) } else { - execDetail := sessVars.StmtCtx.GetExecDetails() logutil.SlowQueryLogger.Warnf( "[SLOW_QUERY] %vcost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", internal, costTime, execDetail, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) diff --git a/executor/aggregate.go b/executor/aggregate.go index d28d5a4c404f7..6c28ef688b90b 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -15,6 +15,7 @@ package executor import ( "sync" + "time" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" @@ -501,6 +502,10 @@ func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGro // Next implements the Executor Next interface. func (e *HashAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if e.isUnparallelExec { return errors.Trace(e.unparallelExec(ctx, chk)) @@ -756,8 +761,11 @@ func (e *StreamAggExec) Close() error { // Next implements the Executor Next interface. func (e *StreamAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() - for !e.executed && chk.NumRows() < e.maxChunkSize { err := e.consumeOneGroup(ctx, chk) if err != nil { diff --git a/executor/builder.go b/executor/builder.go index e7f63af8b8371..1c62d4cbf5702 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" @@ -448,13 +449,12 @@ func (b *executorBuilder) buildLimit(v *plannercore.PhysicalLimit) Executor { func (b *executorBuilder) buildPrepare(v *plannercore.Prepare) Executor { base := newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()) base.initCap = chunk.ZeroCapacity - e := &PrepareExec{ + return &PrepareExec{ baseExecutor: base, is: b.is, name: v.Name, sqlText: v.SQLText, } - return e } func (b *executorBuilder) buildExecute(v *plannercore.Execute) Executor { @@ -659,14 +659,15 @@ func (b *executorBuilder) buildTrace(v *plannercore.Trace) Executor { // buildExplain builds a explain executor. `e.rows` collects final result to `ExplainExec`. func (b *executorBuilder) buildExplain(v *plannercore.Explain) Executor { - e := &ExplainExec{ + explainExec := &ExplainExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + explain: v, } - e.rows = make([][]string, 0, len(v.Rows)) - for _, row := range v.Rows { - e.rows = append(e.rows, row) + if v.Analyze { + b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl() + explainExec.analyzeExec = b.build(v.ExecPlan) } - return e + return explainExec } func (b *executorBuilder) buildUnionScanExec(v *plannercore.PhysicalUnionScan) Executor { diff --git a/executor/distsql.go b/executor/distsql.go index d262d8b57bbb0..8f7beae3cc28e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -19,6 +19,7 @@ import ( "sort" "sync" "sync/atomic" + "time" "unsafe" "github.com/pingcap/tidb/distsql" @@ -243,6 +244,10 @@ func (e *IndexReaderExecutor) Close() error { // Next implements the Executor Next interface. func (e *IndexReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } err := e.result.Next(ctx, chk) if err != nil { e.feedback.Invalidate() @@ -474,7 +479,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha } func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { - tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, &TableReaderExecutor{ + tableReaderExec := &TableReaderExecutor{ baseExecutor: newBaseExecutor(e.ctx, e.schema, e.id+"_tableReader"), table: e.table, physicalTableID: e.physicalTableID, @@ -483,7 +488,11 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []in feedback: statistics.NewQueryFeedback(0, nil, 0, false), corColInFilter: e.corColInTblSide, plans: e.tblPlans, - }, handles) + } + // We assign `nil` to `runtimeStats` to forbidden `TableWorker` driven `IndexLookupExecutor`'s runtime stats collecting, + // because TableWorker information isn't showing in explain result now. + tableReaderExec.runtimeStats = nil + tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles) if err != nil { log.Error(err) return nil, errors.Trace(err) @@ -512,6 +521,10 @@ func (e *IndexLookUpExecutor) Close() error { // Next implements Exec Next interface. func (e *IndexLookUpExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() for { resultTask, err := e.getResultTask() diff --git a/executor/executor.go b/executor/executor.go index b6e2580a0d659..fe543c4d7eebf 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,6 +18,7 @@ import ( "runtime" "sync" "sync/atomic" + "time" "github.com/cznic/mathutil" "github.com/pingcap/tidb/ast" @@ -35,12 +36,14 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/execdetails" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) var ( + _ Executor = &baseExecutor{} _ Executor = &CheckTableExec{} _ Executor = &HashAggExec{} _ Executor = &LimitExec{} @@ -71,6 +74,7 @@ type baseExecutor struct { maxChunkSize int children []Executor retFieldTypes []*types.FieldType + runtimeStats *execdetails.RuntimeStats } // Open initializes children recursively and "childrenResults" according to children's schemas. @@ -127,6 +131,9 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id strin initCap: ctx.GetSessionVars().MaxChunkSize, maxChunkSize: ctx.GetSessionVars().MaxChunkSize, } + if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { + e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.Get(e.id) + } if schema != nil { cols := schema.Columns e.retFieldTypes = make([]*types.FieldType, len(cols)) @@ -168,6 +175,10 @@ type CancelDDLJobsExec struct { // Next implements the Executor Next interface. func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(e.maxChunkSize) if e.cursor >= len(e.jobIDs) { return nil @@ -610,6 +621,10 @@ type LimitExec struct { // Next implements the Executor Next interface. func (e *LimitExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if e.cursor >= e.end { return nil @@ -729,6 +744,10 @@ func (e *TableDualExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. func (e *TableDualExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if e.numReturned >= e.numDualRows { return nil @@ -780,6 +799,10 @@ func (e *SelectionExec) Close() error { // Next implements the Executor Next interface. func (e *SelectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(e.maxChunkSize) if !e.batched { @@ -855,6 +878,10 @@ type TableScanExec struct { // Next implements the Executor Next interface. func (e *TableScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(e.maxChunkSize) if e.isVirtualTable { return errors.Trace(e.nextChunk4InfoSchema(ctx, chk)) @@ -955,6 +982,10 @@ func (e *MaxOneRowExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. func (e *MaxOneRowExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if e.evaluated { return nil @@ -1097,6 +1128,10 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { // Next implements the Executor Next interface. func (e *UnionExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(e.maxChunkSize) if !e.initialized { e.initialize(ctx) diff --git a/executor/explain.go b/executor/explain.go index afc3f871e0883..019e07a655744 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -15,6 +15,7 @@ package executor import ( "github.com/cznic/mathutil" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/util/chunk" "golang.org/x/net/context" ) @@ -23,18 +24,39 @@ import ( type ExplainExec struct { baseExecutor - rows [][]string - cursor int + explain *core.Explain + analyzeExec Executor + rows [][]string + cursor int +} + +// Open implements the Executor Open interface. +func (e *ExplainExec) Open(ctx context.Context) error { + if e.analyzeExec != nil { + return e.analyzeExec.Open(ctx) + } + return nil } // Close implements the Executor Close interface. func (e *ExplainExec) Close() error { + if e.analyzeExec != nil { + e.analyzeExec.Close() + } e.rows = nil return nil } // Next implements the Executor Next interface. func (e *ExplainExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.rows == nil { + var err error + e.rows, err = e.generateExplainInfo(ctx) + if err != nil { + return err + } + } + chk.GrowAndReset(e.maxChunkSize) if e.cursor >= len(e.rows) { return nil @@ -49,3 +71,23 @@ func (e *ExplainExec) Next(ctx context.Context, chk *chunk.Chunk) error { e.cursor += numCurRows return nil } + +func (e *ExplainExec) generateExplainInfo(ctx context.Context) ([][]string, error) { + if e.analyzeExec != nil { + chk := e.analyzeExec.newFirstChunk() + for { + err := e.analyzeExec.Next(ctx, chk) + if err != nil { + return nil, err + } + if chk.NumRows() == 0 { + break + } + } + } + e.explain.RenderResult() + if e.analyzeExec != nil { + e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = nil + } + return e.explain.Rows, nil +} diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a0bc75994bc92..bc05f56d971fd 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -18,6 +18,7 @@ import ( "runtime" "sort" "sync" + "time" "unsafe" "github.com/pingcap/tidb/expression" @@ -189,6 +190,10 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork // Next implements the Executor interface. func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() e.joinResult.Reset() for { diff --git a/executor/join.go b/executor/join.go index a979790fe2af7..205127e64ee55 100644 --- a/executor/join.go +++ b/executor/join.go @@ -16,6 +16,7 @@ package executor import ( "sync" "sync/atomic" + "time" "unsafe" "github.com/pingcap/tidb/expression" @@ -483,6 +484,10 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu // step 1. fetch data from inner child and build a hash table; // step 2. fetch data from outer child in a background goroutine and probe the hash table in multiple join workers. func (e *HashJoinExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } if !e.prepared { e.innerFinished = make(chan error, 1) go util.WithRecovery(func() { e.fetchInnerAndBuildHashTable(ctx) }, e.finishFetchInnerAndBuildHashTable) @@ -696,6 +701,10 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error { // Next implements the Executor interface. func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { diff --git a/executor/merge_join.go b/executor/merge_join.go index 56f2102e763e2..4bbfeaac26e0d 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -14,6 +14,8 @@ package executor import ( + "time" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/chunk" @@ -261,6 +263,10 @@ func (e *MergeJoinExec) prepare(ctx context.Context, chk *chunk.Chunk) error { // Next implements the Executor Next interface. func (e *MergeJoinExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if !e.prepared { if err := e.prepare(ctx, chk); err != nil { diff --git a/executor/projection.go b/executor/projection.go index 168ce32f39914..dce2709f1271b 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -14,6 +14,8 @@ package executor import ( + "time" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" @@ -139,6 +141,10 @@ func (e *ProjectionExec) Open(ctx context.Context) error { // +------------------------------+ +----------------------+ // func (e *ProjectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(e.maxChunkSize) if e.isUnparallelExec() { return errors.Trace(e.unParallelExecute(ctx, chk)) diff --git a/executor/sort.go b/executor/sort.go index 95b9ecac29d38..83fa22618798e 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -16,6 +16,7 @@ package executor import ( "container/heap" "sort" + "time" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" @@ -73,6 +74,10 @@ func (e *SortExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. func (e *SortExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if !e.fetched { err := e.fetchRowChunks(ctx) @@ -296,6 +301,10 @@ func (e *TopNExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. func (e *TopNExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.Reset() if !e.fetched { e.totalLimit = int(e.limit.Offset + e.limit.Count) diff --git a/executor/table_reader.go b/executor/table_reader.go index ea9de3ed2b6f9..a941e3706bee8 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -14,6 +14,8 @@ package executor import ( + "time" + "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/model" plannercore "github.com/pingcap/tidb/planner/core" @@ -98,6 +100,10 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. func (e *TableReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { + if e.runtimeStats != nil { + start := time.Now() + defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } if err := e.resultHandler.nextChunk(ctx, chk); err != nil { e.feedback.Invalidate() return err diff --git a/executor/union_scan.go b/executor/union_scan.go index 3c1682936d71d..1ac6689c1b58d 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -15,6 +15,7 @@ package executor import ( "sort" + "time" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/model" @@ -125,6 +126,10 @@ func (us *UnionScanExec) Open(ctx context.Context) error { // Next implements the Executor Next interface. func (us *UnionScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { + if us.runtimeStats != nil { + start := time.Now() + defer func() { us.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }() + } chk.GrowAndReset(us.maxChunkSize) mutableRow := chunk.MutRowFromTypes(us.retTypes()) for i, batchSize := 0, chk.Capacity(); i < batchSize; i++ { diff --git a/parser/parser.y b/parser/parser.y index 786c4be6ae505..a7eb8136dee5c 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -2312,6 +2312,14 @@ ExplainStmt: Format: $4, } } +| ExplainSym "ANALYZE" ExplainableStmt + { + $$ = &ast.ExplainStmt { + Stmt: $3, + Format: "row", + Analyze: true, + } + } LengthNum: NUM diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index b1ec657000890..9382b5aa500fa 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -15,6 +15,7 @@ package core_test import ( "fmt" + "strings" "testing" . "github.com/pingcap/check" @@ -36,6 +37,35 @@ var _ = Suite(&testAnalyzeSuite{}) type testAnalyzeSuite struct { } +func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + tk.MustExec("create table t1(a int, b int, c int, key idx(a, b))") + tk.MustExec("create table t2(a int, b int)") + tk.MustExec("insert into t1 values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4)") + tk.MustExec("insert into t2 values (2, 22), (3, 33), (5, 55)") + tk.MustExec("analyze table t1, t2") + rs := tk.MustQuery("explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1") + c.Assert(len(rs.Rows()), Equals, 10) + for _, row := range rs.Rows() { + c.Assert(len(row), Equals, 5) + taskType := row[2].(string) + if taskType != "cop" { + execInfo := row[4].(string) + c.Assert(strings.Contains(execInfo, "time"), Equals, true) + c.Assert(strings.Contains(execInfo, "loops"), Equals, true) + c.Assert(strings.Contains(execInfo, "rows"), Equals, true) + } + } +} + // TestCBOWithoutAnalyze tests the plan with stats that only have count info. func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) { defer testleak.AfterTest(c)() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index f06459ffbbf7b..d629d611cc70c 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -17,12 +17,14 @@ import ( "bytes" "fmt" "strconv" + "strings" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/auth" @@ -421,11 +423,55 @@ type Explain struct { StmtPlan Plan Rows [][]string explainedPlans map[int]bool + Format string + Analyze bool + ExecStmt ast.StmtNode + ExecPlan Plan +} + +// prepareSchema prepares explain's result schema. +func (e *Explain) prepareSchema() error { + switch strings.ToLower(e.Format) { + case ast.ExplainFormatROW: + retFields := []string{"id", "count", "task", "operator info"} + if e.Analyze { + retFields = append(retFields, "execution info") + } + schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...) + for _, fieldName := range retFields { + schema.Append(buildColumn("", fieldName, mysql.TypeString, mysql.MaxBlobWidth)) + } + e.SetSchema(schema) + case ast.ExplainFormatDOT: + retFields := []string{"dot contents"} + schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...) + for _, fieldName := range retFields { + schema.Append(buildColumn("", fieldName, mysql.TypeString, mysql.MaxBlobWidth)) + } + e.SetSchema(schema) + default: + return errors.Errorf("explain format '%s' is not supported now", e.Format) + } + return nil +} + +// RenderResult renders the explain result as specified format. +func (e *Explain) RenderResult() error { + switch strings.ToLower(e.Format) { + case ast.ExplainFormatROW: + e.explainedPlans = map[int]bool{} + e.explainPlanInRowFormat(e.StmtPlan.(PhysicalPlan), "root", "", true) + case ast.ExplainFormatDOT: + e.prepareDotInfo(e.StmtPlan.(PhysicalPlan)) + default: + return errors.Errorf("explain format '%s' is not supported now", e.Format) + } + return nil } // explainPlanInRowFormat generates explain information for root-tasks. -func (e *Explain) explainPlanInRowFormat(p PhysicalPlan, TaskType, indent string, isLastChild bool) { - e.prepareOperatorInfo(p, TaskType, indent, isLastChild) +func (e *Explain) explainPlanInRowFormat(p PhysicalPlan, taskType, indent string, isLastChild bool) { + e.prepareOperatorInfo(p, taskType, indent, isLastChild) e.explainedPlans[p.ID()] = true // For every child we create a new sub-tree rooted by it. @@ -434,7 +480,7 @@ func (e *Explain) explainPlanInRowFormat(p PhysicalPlan, TaskType, indent string if e.explainedPlans[child.ID()] { continue } - e.explainPlanInRowFormat(child.(PhysicalPlan), TaskType, childIndent, i == len(p.Children())-1) + e.explainPlanInRowFormat(child.(PhysicalPlan), taskType, childIndent, i == len(p.Children())-1) } switch copPlan := p.(type) { @@ -450,10 +496,18 @@ func (e *Explain) explainPlanInRowFormat(p PhysicalPlan, TaskType, indent string // prepareOperatorInfo generates the following information for every plan: // operator id, task type, operator info, and the estemated row count. -func (e *Explain) prepareOperatorInfo(p PhysicalPlan, TaskType string, indent string, isLastChild bool) { +func (e *Explain) prepareOperatorInfo(p PhysicalPlan, taskType string, indent string, isLastChild bool) { operatorInfo := p.ExplainInfo() count := string(strconv.AppendFloat([]byte{}, p.statsInfo().RowCount, 'f', 2, 64)) - row := []string{e.prettyIdentifier(p.ExplainID(), indent, isLastChild), count, TaskType, operatorInfo} + row := []string{e.prettyIdentifier(p.ExplainID(), indent, isLastChild), count, taskType, operatorInfo} + if e.Analyze { + runtimeStatsColl := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl + if taskType == "cop" { + row = append(row, "") //TODO: wait collect resp from tikv + } else { + row = append(row, runtimeStatsColl.Get(p.ExplainID()).String()) + } + } e.Rows = append(e.Rows, row) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 33c48c11abe8f..f218ecf944a7d 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1445,27 +1445,11 @@ func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) (Plan, error) { return nil, ErrUnsupportedType.GenWithStackByArgs(targetPlan) } } - p := &Explain{StmtPlan: pp} - switch strings.ToLower(explain.Format) { - case ast.ExplainFormatROW: - retFields := []string{"id", "count", "task", "operator info"} - schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...) - for _, fieldName := range retFields { - schema.Append(buildColumn("", fieldName, mysql.TypeString, mysql.MaxBlobWidth)) - } - p.SetSchema(schema) - p.explainedPlans = map[int]bool{} - p.explainPlanInRowFormat(p.StmtPlan.(PhysicalPlan), "root", "", true) - case ast.ExplainFormatDOT: - retFields := []string{"dot contents"} - schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...) - for _, fieldName := range retFields { - schema.Append(buildColumn("", fieldName, mysql.TypeString, mysql.MaxBlobWidth)) - } - p.SetSchema(schema) - p.prepareDotInfo(p.StmtPlan.(PhysicalPlan)) - default: - return nil, errors.Errorf("explain format '%s' is not supported now", explain.Format) + p := &Explain{StmtPlan: pp, Analyze: explain.Analyze, Format: explain.Format, ExecStmt: explain.Stmt, ExecPlan: targetPlan} + p.ctx = b.ctx + err = p.prepareSchema() + if err != nil { + return nil, err } return p, nil } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 2fc9adfd49712..e59d2da7194cb 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -73,12 +73,13 @@ type StatementContext struct { } // Copied from SessionVars.TimeZone. - TimeZone *time.Location - Priority mysql.PriorityEnum - NotFillCache bool - MemTracker *memory.Tracker - TableIDs []int64 - IndexIDs []int64 + TimeZone *time.Location + Priority mysql.PriorityEnum + NotFillCache bool + MemTracker *memory.Tracker + RuntimeStatsColl *execdetails.RuntimeStatsColl + TableIDs []int64 + IndexIDs []int64 } // AddAffectedRows adds affected rows. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index a73c5b2cd60b5..ad384c51e0a7b 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -16,6 +16,8 @@ package execdetails import ( "fmt" "strings" + "sync" + "sync/atomic" "time" ) @@ -52,3 +54,50 @@ func (d ExecDetails) String() string { } return strings.Join(parts, " ") } + +// RuntimeStatsColl collects executors's execution info. +type RuntimeStatsColl struct { + mu sync.Mutex + stats map[string]*RuntimeStats +} + +// RuntimeStats collects one executor's execution info. +type RuntimeStats struct { + // executor's Next() called times. + loop int32 + // executor consume time. + consume int64 + // executor return row count. + rows int64 +} + +// NewRuntimeStatsColl creates new executor collector. +func NewRuntimeStatsColl() *RuntimeStatsColl { + return &RuntimeStatsColl{stats: make(map[string]*RuntimeStats)} +} + +// Get gets execStat for a executor. +func (e *RuntimeStatsColl) Get(planID string) *RuntimeStats { + e.mu.Lock() + defer e.mu.Unlock() + runtimeStats, exists := e.stats[planID] + if !exists { + runtimeStats = &RuntimeStats{} + e.stats[planID] = runtimeStats + } + return runtimeStats +} + +// Record records executor's execution. +func (e *RuntimeStats) Record(d time.Duration, rowNum int) { + atomic.AddInt32(&e.loop, 1) + atomic.AddInt64(&e.consume, int64(d)) + atomic.AddInt64(&e.rows, int64(rowNum)) +} + +func (e *RuntimeStats) String() string { + if e == nil { + return "" + } + return fmt.Sprintf("time:%v, loops:%d, rows:%d", time.Duration(e.consume), e.loop, e.rows) +} From 4021862c8297946fcfd0fd4c2f5c0f4fd7918d82 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 18 Oct 2018 15:19:18 +0800 Subject: [PATCH 014/509] stats: fix histogram boundaries overflow error (#7883) (#7944) --- statistics/feedback.go | 104 +++++++++++++++++++++++++++++------- statistics/feedback_test.go | 2 +- statistics/update.go | 4 -- statistics/update_test.go | 71 +++++++++++++++++++++++- types/etc_test.go | 2 +- types/helper.go | 5 +- 6 files changed, 159 insertions(+), 29 deletions(-) diff --git a/statistics/feedback.go b/statistics/feedback.go index 6477aa57e42ad..81eccc6787e88 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -243,12 +243,71 @@ type BucketFeedback struct { upper *types.Datum // The upper bound of the new bucket. } +// outOfRange checks if the `val` is between `min` and `max`. +func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, error) { + result, err := val.CompareDatum(sc, min) + if err != nil { + return 0, err + } + if result < 0 { + return result, nil + } + result, err = val.CompareDatum(sc, max) + if err != nil { + return 0, err + } + if result > 0 { + return result, nil + } + return 0, nil +} + +// adjustFeedbackBoundaries adjust the feedback boundaries according to the `min` and `max`. +// If the feedback has no intersection with `min` and `max`, we could just skip this feedback. +func (f *feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) { + result, err := outOfRange(sc, min, max, f.lower) + if err != nil { + return false, err + } + if result > 0 { + return true, nil + } + if result < 0 { + f.lower = min + } + result, err = outOfRange(sc, min, max, f.upper) + if err != nil { + return false, err + } + if result < 0 { + return true, nil + } + if result > 0 { + f.upper = max + } + return false, nil +} + // buildBucketFeedback build the feedback for each bucket from the histogram feedback. func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) { bktID2FB := make(map[int]*BucketFeedback) + if len(feedback.feedback) == 0 { + return bktID2FB, 0 + } total := 0 - for _, ran := range feedback.feedback { - idx, _ := h.Bounds.LowerBound(0, ran.lower) + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + kind := feedback.feedback[0].lower.Kind() + min, max := getMinValue(kind, h.tp), getMaxValue(kind, h.tp) + for _, fb := range feedback.feedback { + skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) + if err != nil { + log.Debugf("adjust feedback boundaries failed, err: %v", errors.ErrorStack(err)) + continue + } + if skip { + continue + } + idx, _ := h.Bounds.LowerBound(0, fb.lower) bktIdx := 0 // The last bucket also stores the feedback that falls outside the upper bound. if idx >= h.Bounds.NumRows()-2 { @@ -256,7 +315,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } else { bktIdx = idx / 2 // Make sure that this feedback lies within the bucket. - if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, ran.upper) < 0 { + if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, fb.upper) < 0 { continue } } @@ -266,23 +325,23 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)} bktID2FB[bktIdx] = bkt } - bkt.feedback = append(bkt.feedback, ran) + bkt.feedback = append(bkt.feedback, fb) // Update the bound if necessary. - res, err := bkt.lower.CompareDatum(nil, ran.lower) + res, err := bkt.lower.CompareDatum(nil, fb.lower) if err != nil { - log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, ran.lower, errors.ErrorStack(err)) + log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, fb.lower, errors.ErrorStack(err)) continue } if res > 0 { - bkt.lower = ran.lower + bkt.lower = fb.lower } - res, err = bkt.upper.CompareDatum(nil, ran.upper) + res, err = bkt.upper.CompareDatum(nil, fb.upper) if err != nil { - log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, ran.upper, errors.ErrorStack(err)) + log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, fb.upper, errors.ErrorStack(err)) continue } if res < 0 { - bkt.upper = ran.upper + bkt.upper = fb.upper } } return bktID2FB, total @@ -528,7 +587,12 @@ func splitBuckets(h *Histogram, feedback *QueryFeedback) ([]bucket, []bool, int6 func UpdateHistogram(h *Histogram, feedback *QueryFeedback) *Histogram { buckets, isNewBuckets, totalCount := splitBuckets(h, feedback) buckets = mergeBuckets(buckets, isNewBuckets, float64(totalCount)) - return buildNewHistogram(h, buckets) + hist := buildNewHistogram(h, buckets) + // Update the NDV of primary key column. + if feedback.tp == pkType { + hist.NDV = int64(hist.totalRowCount()) + } + return hist } // UpdateCMSketch updates the CMSketch by feedback. @@ -1077,13 +1141,13 @@ func supportColumnType(k byte) bool { func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) { switch k { case types.KindInt64: - max.SetInt64(math.MaxInt64) + max.SetInt64(types.SignedUpperBound[ft.Tp]) case types.KindUint64: - max.SetUint64(math.MaxUint64) + max.SetUint64(types.UnsignedUpperBound[ft.Tp]) case types.KindFloat32: - max.SetFloat32(math.MaxFloat32) + max.SetFloat32(float32(types.GetMaxFloat(ft.Flen, ft.Decimal))) case types.KindFloat64: - max.SetFloat64(math.MaxFloat64) + max.SetFloat64(types.GetMaxFloat(ft.Flen, ft.Decimal)) case types.KindString, types.KindBytes: val := types.MaxValueDatum() bytes, err := codec.EncodeKey(nil, nil, val) @@ -1093,7 +1157,7 @@ func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) { } max.SetBytes(bytes) case types.KindMysqlDecimal: - max.SetMysqlDecimal(types.NewMaxOrMinDec(false, mysql.MaxDecimalWidth, 0)) + max.SetMysqlDecimal(types.NewMaxOrMinDec(false, ft.Flen, ft.Decimal)) case types.KindMysqlDuration: max.SetMysqlDuration(types.Duration{Duration: math.MaxInt64}) case types.KindMysqlTime: @@ -1109,13 +1173,13 @@ func getMaxValue(k byte, ft *types.FieldType) (max types.Datum) { func getMinValue(k byte, ft *types.FieldType) (min types.Datum) { switch k { case types.KindInt64: - min.SetInt64(math.MinInt64) + min.SetInt64(types.SignedLowerBound[ft.Tp]) case types.KindUint64: min.SetUint64(0) case types.KindFloat32: - min.SetFloat32(-math.MaxFloat32) + min.SetFloat32(float32(-types.GetMaxFloat(ft.Flen, ft.Decimal))) case types.KindFloat64: - min.SetFloat64(-math.MaxFloat64) + min.SetFloat64(-types.GetMaxFloat(ft.Flen, ft.Decimal)) case types.KindString, types.KindBytes: val := types.MinNotNullDatum() bytes, err := codec.EncodeKey(nil, nil, val) @@ -1125,7 +1189,7 @@ func getMinValue(k byte, ft *types.FieldType) (min types.Datum) { } min.SetBytes(bytes) case types.KindMysqlDecimal: - min.SetMysqlDecimal(types.NewMaxOrMinDec(true, mysql.MaxDecimalWidth, 0)) + min.SetMysqlDecimal(types.NewMaxOrMinDec(true, ft.Flen, ft.Decimal)) case types.KindMysqlDuration: min.SetMysqlDuration(types.Duration{Duration: math.MinInt64}) case types.KindMysqlTime: diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 9b0907a2b6a0e..f5b83b44ee663 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -70,7 +70,7 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { defaultBucketCount = 7 defer func() { defaultBucketCount = originBucketCount }() c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals, - "column:0 ndv:0 totColSize:0\n"+ + "column:0 ndv:10057 totColSize:0\n"+ "num: 10000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ "num: 8 lower_bound: 2 upper_bound: 7 repeats: 0\n"+ "num: 11 lower_bound: 8 upper_bound: 19 repeats: 0\n"+ diff --git a/statistics/update.go b/statistics/update.go index 5b710579bb1b2..dc0bebd5a48d3 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -558,10 +558,6 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err)) } } - // Update the NDV of primary key column. - if table.Meta().PKIsHandle && q.tp == pkType { - hist.NDV = int64(hist.totalRowCount()) - } err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms) return errors.Trace(err) } diff --git a/statistics/update_test.go b/statistics/update_test.go index 0affb367df228..710be1cb946c4 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -594,7 +594,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { { // test primary key feedback sql: "select * from t where t.a <= 5", - hist: "column:1 ndv:3 totColSize:0\n" + + hist: "column:1 ndv:4 totColSize:0\n" + "num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" + "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n" + "num: 2 lower_bound: 3 upper_bound: 5 repeats: 0", @@ -1118,3 +1118,72 @@ func (s *testStatsUpdateSuite) TestIndexQueryFeedback(c *C) { c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, t.eqCount) } } + +func (s *testStatsUpdateSuite) TestFeedbackRanges(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + h := s.do.StatsHandle() + oriProbability := statistics.FeedbackProbability + oriNumber := statistics.MaxNumberOfRanges + defer func() { + statistics.FeedbackProbability = oriProbability + statistics.MaxNumberOfRanges = oriNumber + }() + statistics.FeedbackProbability = 1 + + testKit.MustExec("use test") + testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") + for i := 0; i < 20; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + testKit.MustExec("analyze table t with 3 buckets") + for i := 30; i < 40; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + tests := []struct { + sql string + hist string + colID int64 + }{ + { + sql: "select * from t where a <= 50 or (a > 130 and a < 140)", + hist: "column:1 ndv:30 totColSize:0\n" + + "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" + + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + + "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0", + colID: 1, + }, + { + sql: "select * from t where a >= 10", + hist: "column:1 ndv:30 totColSize:0\n" + + "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0\n" + + "num: 8 lower_bound: 8 upper_bound: 15 repeats: 0\n" + + "num: 14 lower_bound: 16 upper_bound: 127 repeats: 0", + colID: 1, + }, + { + sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))", + hist: "column:2 ndv:20 totColSize:20\n" + + "num: 7 lower_bound: -128 upper_bound: 6 repeats: 0\n" + + "num: 7 lower_bound: 7 upper_bound: 13 repeats: 1\n" + + "num: 6 lower_bound: 14 upper_bound: 19 repeats: 1", + colID: 2, + }, + } + is := s.do.InfoSchema() + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + for i, t := range tests { + testKit.MustQuery(t.sql) + c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsFeedbackToKV(), IsNil) + c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) + c.Assert(err, IsNil) + h.Update(is) + tblInfo := table.Meta() + tbl := h.GetTableStats(tblInfo) + c.Assert(tbl.Columns[t.colID].ToString(0), Equals, tests[i].hist) + } +} diff --git a/types/etc_test.go b/types/etc_test.go index 4a5e161f2e32e..60d272bff2866 100644 --- a/types/etc_test.go +++ b/types/etc_test.go @@ -122,7 +122,7 @@ func (s *testTypeEtcSuite) TestMaxFloat(c *C) { } for _, t := range tbl { - f := getMaxFloat(t.Flen, t.Decimal) + f := GetMaxFloat(t.Flen, t.Decimal) c.Assert(f, Equals, t.Expect) } } diff --git a/types/helper.go b/types/helper.go index 385921f479b9d..ec95d824d2f1e 100644 --- a/types/helper.go +++ b/types/helper.go @@ -59,7 +59,8 @@ func Truncate(f float64, dec int) float64 { return math.Trunc(tmp) / shift } -func getMaxFloat(flen int, decimal int) float64 { +// GetMaxFloat gets the max float for given flen and decimal. +func GetMaxFloat(flen int, decimal int) float64 { intPartLen := flen - decimal f := math.Pow10(intPartLen) f -= math.Pow10(-decimal) @@ -74,7 +75,7 @@ func TruncateFloat(f float64, flen int, decimal int) (float64, error) { return 0, ErrOverflow.GenWithStackByArgs("DOUBLE", "") } - maxF := getMaxFloat(flen, decimal) + maxF := GetMaxFloat(flen, decimal) if !math.IsInf(f, 0) { f = Round(f, decimal) From d9137e24ff9bff3599ad95da84928fd8e0e7497e Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 18 Oct 2018 16:16:49 +0800 Subject: [PATCH 015/509] executor: fix a bug in point get (#7934) (#7943) --- executor/point_get.go | 46 +++++++++++++++++++++++++++----------- executor/point_get_test.go | 5 ++++- 2 files changed, 37 insertions(+), 14 deletions(-) diff --git a/executor/point_get.go b/executor/point_get.go index cd150178843da..3d41181675bfc 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -135,40 +135,60 @@ func (e *PointGetExecutor) get(key kv.Key) (val []byte, err error) { } func (e *PointGetExecutor) decodeRowValToChunk(rowVal []byte, chk *chunk.Chunk) error { - colIDs := make(map[int64]int, e.schema.Len()) - for i, col := range e.schema.Columns { - colIDs[col.ID] = i + // One column could be filled for multi-times in the schema. e.g. select b, b, c, c from t where a = 1. + // We need to set the positions in the schema for the same column. + colID2DecodedPos := make(map[int64]int, e.schema.Len()) + decodedPos2SchemaPos := make([][]int, 0, e.schema.Len()) + for schemaPos, col := range e.schema.Columns { + if decodedPos, ok := colID2DecodedPos[col.ID]; !ok { + colID2DecodedPos[col.ID] = len(colID2DecodedPos) + decodedPos2SchemaPos = append(decodedPos2SchemaPos, []int{schemaPos}) + } else { + decodedPos2SchemaPos[decodedPos] = append(decodedPos2SchemaPos[decodedPos], schemaPos) + } } - colVals, err := tablecodec.CutRowNew(rowVal, colIDs) + decodedVals, err := tablecodec.CutRowNew(rowVal, colID2DecodedPos) if err != nil { return errors.Trace(err) } - if colVals == nil { - colVals = make([][]byte, len(colIDs)) + if decodedVals == nil { + decodedVals = make([][]byte, len(colID2DecodedPos)) } decoder := codec.NewDecoder(chk, e.ctx.GetSessionVars().Location()) - for id, offset := range colIDs { - if e.tblInfo.PKIsHandle && mysql.HasPriKeyFlag(e.schema.Columns[offset].RetType.Flag) { - chk.AppendInt64(offset, e.handle) + for id, decodedPos := range colID2DecodedPos { + schemaPoses := decodedPos2SchemaPos[decodedPos] + firstPos := schemaPoses[0] + if e.tblInfo.PKIsHandle && mysql.HasPriKeyFlag(e.schema.Columns[firstPos].RetType.Flag) { + chk.AppendInt64(firstPos, e.handle) + // Fill other positions. + for i := 1; i < len(schemaPoses); i++ { + chk.MakeRef(firstPos, schemaPoses[i]) + } continue } + // ExtraHandleID is added when building plan, we can make sure that there's only one column's ID is this. if id == model.ExtraHandleID { - chk.AppendInt64(offset, e.handle) + chk.AppendInt64(firstPos, e.handle) continue } - if len(colVals[offset]) == 0 { + if len(decodedVals[decodedPos]) == 0 { + // This branch only entered for updating and deleting. It won't have one column in multiple positions. colInfo := getColInfoByID(e.tblInfo, id) d, err1 := table.GetColOriginDefaultValue(e.ctx, colInfo) if err1 != nil { return errors.Trace(err1) } - chk.AppendDatum(offset, &d) + chk.AppendDatum(firstPos, &d) continue } - _, err = decoder.DecodeOne(colVals[offset], offset, e.schema.Columns[offset].RetType) + _, err = decoder.DecodeOne(decodedVals[decodedPos], firstPos, e.schema.Columns[firstPos].RetType) if err != nil { return errors.Trace(err) } + // Fill other positions. + for i := 1; i < len(schemaPoses); i++ { + chk.MakeRef(firstPos, schemaPoses[i]) + } } return nil } diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 5d0716618c850..65f8786ea7f71 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -38,7 +38,7 @@ func (s *testSuite) TestPointGet(c *C) { tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a bigint primary key, b bigint, c bigint);`) - tk.MustExec(`insert into t values(1, NULL, NULL), (2, NULL, 2), (3, 3, NULL), (4, 4, 4);`) + tk.MustExec(`insert into t values(1, NULL, NULL), (2, NULL, 2), (3, 3, NULL), (4, 4, 4), (5, 6, 7);`) tk.MustQuery(`select * from t where a = 1;`).Check(testkit.Rows( `1 `, )) @@ -51,4 +51,7 @@ func (s *testSuite) TestPointGet(c *C) { tk.MustQuery(`select * from t where a = 4;`).Check(testkit.Rows( `4 4 4`, )) + tk.MustQuery(`select a, a, b, a, b, c, b, c, c from t where a = 5;`).Check(testkit.Rows( + `5 5 6 5 6 7 6 7 7`, + )) } From 0f587c31a27371fff7b36753b3ffe085791c6e64 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 18 Oct 2018 17:03:30 +0800 Subject: [PATCH 016/509] planner, executor: refine ColumnPrune for LogicalUnionAll (#7930) (#7941) --- executor/executor_test.go | 15 +++++++++++++++ planner/core/rule_column_pruning.go | 9 +++++++++ 2 files changed, 24 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index 3412567e0a53b..1ae0306ad0911 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1021,6 +1021,21 @@ func (s *testSuite) TestUnion(c *C) { tk.MustQuery("select 1 union select 1 union all select 1").Check(testkit.Rows("1", "1")) tk.MustQuery("select 1 union all select 1 union select 1").Check(testkit.Rows("1")) + + tk.MustExec("drop table if exists t1, t2") + tk.MustExec(`create table t1(a bigint, b bigint);`) + tk.MustExec(`create table t2(a bigint, b bigint);`) + tk.MustExec(`insert into t1 values(1, 1);`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t1 select * from t1;`) + tk.MustExec(`insert into t2 values(1, 1);`) + tk.MustExec(`set @@tidb_max_chunk_size=2;`) + tk.MustQuery(`select count(*) from (select t1.a, t1.b from t1 left join t2 on t1.a=t2.a union all select t1.a, t1.a from t1 left join t2 on t1.a=t2.a) tmp;`).Check(testkit.Rows("128")) + tk.MustQuery(`select tmp.a, count(*) from (select t1.a, t1.b from t1 left join t2 on t1.a=t2.a union all select t1.a, t1.a from t1 left join t2 on t1.a=t2.a) tmp;`).Check(testkit.Rows("1 128")) } func (s *testSuite) TestIn(c *C) { diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 933c93b206900..afa552019588e 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -129,6 +129,15 @@ func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) { // PruneColumns implements LogicalPlan interface. func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column) { + used := getUsedList(parentUsedCols, p.schema) + hasBeenUsed := false + for i := range used { + hasBeenUsed = hasBeenUsed || used[i] + } + if !hasBeenUsed { + parentUsedCols = make([]*expression.Column, len(p.schema.Columns)) + copy(parentUsedCols, p.schema.Columns) + } for _, child := range p.Children() { child.PruneColumns(parentUsedCols) } From 3e3b90580f588eb0b3c72b11f635e6ec9b2384db Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Thu, 18 Oct 2018 18:25:26 +0800 Subject: [PATCH 017/509] expression: maintain `DeferredExpr` in aggressive constant folding. (#7926) --- expression/constant_fold.go | 6 ++++++ expression/constant_test.go | 28 ++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/expression/constant_fold.go b/expression/constant_fold.go index 0553b2c8c0c2a..0f7e85fa18ba9 100644 --- a/expression/constant_fold.go +++ b/expression/constant_fold.go @@ -118,9 +118,15 @@ func foldConstant(expr Expression) (Expression, bool) { return expr, isDeferredConst } if value.IsNull() { + if isDeferredConst { + return &Constant{Value: value, RetType: x.RetType, DeferredExpr: x}, true + } return &Constant{Value: value, RetType: x.RetType}, false } if isTrue, err := value.ToBool(sc); err == nil && isTrue == 0 { + if isDeferredConst { + return &Constant{Value: value, RetType: x.RetType, DeferredExpr: x}, true + } return &Constant{Value: value, RetType: x.RetType}, false } return expr, isDeferredConst diff --git a/expression/constant_test.go b/expression/constant_test.go index a2f4376afeeb2..bda66eb8b3f46 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -202,3 +202,31 @@ func (*testExpressionSuite) TestConstantFolding(c *C) { c.Assert(newConds.String(), Equals, tt.result, Commentf("different for expr %s", tt.condition)) } } + +func (*testExpressionSuite) TestDeferredExprNullConstantFold(c *C) { + defer testleak.AfterTest(c)() + nullConst := &Constant{ + Value: types.NewDatum(nil), + RetType: types.NewFieldType(mysql.TypeTiny), + DeferredExpr: Null, + } + tests := []struct { + condition Expression + deferred string + }{ + { + condition: newFunction(ast.LT, newColumn(0), nullConst), + deferred: "lt(test.t.0, )", + }, + } + for _, tt := range tests { + comment := Commentf("different for expr %s", tt.condition) + sf, ok := tt.condition.(*ScalarFunction) + c.Assert(ok, IsTrue, comment) + sf.GetCtx().GetSessionVars().StmtCtx.InNullRejectCheck = true + newConds := FoldConstant(tt.condition) + newConst, ok := newConds.(*Constant) + c.Assert(ok, IsTrue, comment) + c.Assert(newConst.DeferredExpr.String(), Equals, tt.deferred, comment) + } +} From e48149e00d6b38ef642100e949411ed09654618f Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 19 Oct 2018 18:14:57 +0800 Subject: [PATCH 018/509] executor: add the slow log for commit (#7951) (#7964) --- executor/adapter.go | 18 ++++++++++-------- session/session.go | 6 ++++++ 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 4849b8012997f..3b9d4cad470f5 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -117,7 +117,7 @@ func (a *recordSet) NewChunk() *chunk.Chunk { func (a *recordSet) Close() error { err := a.executor.Close() - a.stmt.logSlowQuery(a.txnStartTS, a.lastErr == nil) + a.stmt.LogSlowQuery(a.txnStartTS, a.lastErr == nil) if a.processinfo != nil { a.processinfo.SetProcessInfo("") } @@ -139,8 +139,9 @@ type ExecStmt struct { StmtNode ast.StmtNode - Ctx sessionctx.Context - startTime time.Time + Ctx sessionctx.Context + // StartTime stands for the starting time when executing the statement. + StartTime time.Time isPreparedStmt bool } @@ -186,7 +187,7 @@ func (a *ExecStmt) RebuildPlan() (int64, error) { // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns // result, execution is done after this function returns, in the returned ast.RecordSet Next method. func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { - a.startTime = time.Now() + a.StartTime = time.Now() sctx := a.Ctx if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { oriStats, _ := sctx.GetSessionVars().GetSystemVar(variable.TiDBBuildStatsConcurrency) @@ -267,7 +268,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co if sctx.Txn() != nil { txnTS = sctx.Txn().StartTS() } - a.logSlowQuery(txnTS, err == nil) + a.LogSlowQuery(txnTS, err == nil) }() err = e.Next(ctx, e.newFirstChunk()) @@ -333,13 +334,14 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { // QueryReplacer replaces new line and tab for grep result including query string. var QueryReplacer = strings.NewReplacer("\r", " ", "\n", " ", "\t", " ") -func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) { +// LogSlowQuery is used to print the slow query in the log files. +func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { level := log.GetLevel() if level < log.WarnLevel { return } cfg := config.GetGlobalConfig() - costTime := time.Since(a.startTime) + costTime := time.Since(a.StartTime) threshold := time.Duration(cfg.Log.SlowThreshold) * time.Millisecond if costTime < threshold && level < log.DebugLevel { return @@ -389,7 +391,7 @@ func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) { } domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{ SQL: sql, - Start: a.startTime, + Start: a.StartTime, Duration: costTime, Detail: sessVars.StmtCtx.GetExecDetails(), Succ: succ, diff --git a/session/session.go b/session/session.go index 89280de515300..20ab58c722808 100644 --- a/session/session.go +++ b/session/session.go @@ -375,7 +375,13 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { } func (s *session) CommitTxn(ctx context.Context) error { + stmt := executor.ExecStmt{ + Text: "commit", + Ctx: s, + StartTime: time.Now(), + } err := s.doCommitWithRetry(ctx) + stmt.LogSlowQuery(s.sessionVars.TxnCtx.StartTS, err == nil) label := metrics.LblOK if err != nil { label = metrics.LblError From 60364fe6ad328bc9aecfe58afbe85018f148f24f Mon Sep 17 00:00:00 2001 From: crazycs Date: Sat, 20 Oct 2018 23:09:42 +0800 Subject: [PATCH 019/509] ddl: fix invailid ddl job panic (#7940) (#7958) --- ddl/ddl_worker.go | 2 +- ddl/ddl_worker_test.go | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 6afeea59937ff..4237a2f56c42b 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -510,7 +510,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, default: // Invalid job, cancel it. job.State = model.JobStateCancelled - err = errInvalidDDLJob.GenWithStack("invalid ddl job %v", job) + err = errInvalidDDLJob.GenWithStack("invalid ddl job type: %v", job.Type) } // Save errors in job, so that others can know errors happened. diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index b6038bd5ce9ae..8aa62e5b684ba 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -136,6 +136,24 @@ func (s *testDDLSuite) TestTableError(c *C) { } +func (s *testDDLSuite) TestInvalidDDLJob(c *C) { + store := testCreateStore(c, "test_invalid_ddl_job_type_error") + defer store.Close() + d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + defer d.Stop() + ctx := testNewContext(d) + + job := &model.Job{ + SchemaID: 0, + TableID: 0, + Type: model.ActionNone, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{}, + } + err := d.doDDLJob(ctx, job) + c.Assert(err.Error(), Equals, "[ddl:3]invalid ddl job type: none") +} + func (s *testDDLSuite) TestForeignKeyError(c *C) { store := testCreateStore(c, "test_foreign_key_error") defer store.Close() From f6d68e6809aab01678fb924abac964a8701ee9af Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Oct 2018 15:17:00 +0800 Subject: [PATCH 020/509] domain: close slow query channel after closing session pool (#7847) (#7972) --- domain/domain.go | 8 +++++++- domain/topn_slow_query.go | 9 +++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/domain/domain.go b/domain/domain.go index a4caf5baa7af7..8061751fc852b 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -337,6 +337,12 @@ func (do *Domain) Reload() error { // LogSlowQuery keeps topN recent slow queries in domain. func (do *Domain) LogSlowQuery(query *SlowQueryInfo) { + do.slowQuery.mu.RLock() + defer do.slowQuery.mu.RUnlock() + if do.slowQuery.mu.closed { + return + } + select { case do.slowQuery.ch <- query: default: @@ -459,8 +465,8 @@ func (do *Domain) Close() { if do.etcdClient != nil { terror.Log(errors.Trace(do.etcdClient.Close())) } - do.slowQuery.Close() do.sysSessionPool.Close() + do.slowQuery.Close() do.wg.Wait() log.Info("[domain] close") } diff --git a/domain/topn_slow_query.go b/domain/topn_slow_query.go index 52251d34475f2..f8ae69c0be67e 100644 --- a/domain/topn_slow_query.go +++ b/domain/topn_slow_query.go @@ -115,6 +115,11 @@ type topNSlowQueries struct { period time.Duration ch chan *SlowQueryInfo msgCh chan *showSlowMessage + + mu struct { + sync.RWMutex + closed bool + } } func newTopNSlowQueries(topN int, period time.Duration, queueSize int) *topNSlowQueries { @@ -196,6 +201,10 @@ func (q *topNSlowQueries) QueryTop(count int, kind ast.ShowSlowKind) []*SlowQuer } func (q *topNSlowQueries) Close() { + q.mu.Lock() + q.mu.closed = true + q.mu.Unlock() + close(q.ch) } From f5d985275767a71c17f1a56452bb1d94cd734522 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Oct 2018 15:26:53 +0800 Subject: [PATCH 021/509] domain: close slow query channel after closing session pool (#7847) (#7972) From f4c18b681911a734285717f0fc1c30b4cd480512 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Oct 2018 15:27:37 +0800 Subject: [PATCH 022/509] domain: close slow query channel after closing session pool (#7847) (#7972) From 07a9d528ddd14b248df2cd0ee7be1c936148ba59 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Oct 2018 15:28:18 +0800 Subject: [PATCH 023/509] domain: close slow query channel after closing session pool (#7847) (#7972) From 7cb4fd83a06ac406cab1ef7f124307971903a0da Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 22 Oct 2018 22:47:17 +0800 Subject: [PATCH 024/509] admin: fix admin check table compare bug (#7818) (#7975) --- executor/admin_test.go | 8 ++++++++ executor/executor_test.go | 1 - types/datum_test.go | 38 ++++++++++++++++++++++++++++++++++++++ util/admin/admin.go | 20 +++++++++++++++++--- 4 files changed, 63 insertions(+), 4 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index c3f078257285f..0a1282c582388 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -498,6 +498,14 @@ func (s *testSuite) TestAdminCheckTable(c *C) { tk.MustExec(`ALTER TABLE t1 ADD INDEX idx5 (c5)`) tk.MustExec(`ALTER TABLE t1 ADD INDEX idx6 (c6)`) tk.MustExec(`admin check table t1`) + + // Test add index on decimal column. + tk.MustExec(`drop table if exists td1;`) + tk.MustExec(`CREATE TABLE td1 (c2 INT NULL DEFAULT '70');`) + tk.MustExec(`INSERT INTO td1 SET c2 = '5';`) + tk.MustExec(`ALTER TABLE td1 ADD COLUMN c4 DECIMAL(12,8) NULL DEFAULT '213.41598062';`) + tk.MustExec(`ALTER TABLE td1 ADD INDEX id2 (c4) ;`) + tk.MustExec(`ADMIN CHECK TABLE td1;`) } func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) { diff --git a/executor/executor_test.go b/executor/executor_test.go index 1ae0306ad0911..ccc48b19cbebd 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -273,7 +273,6 @@ func (s *testSuite) TestAdmin(c *C) { tk.MustExec("ALTER TABLE t1 ADD COLUMN c4 bit(10) default 127;") tk.MustExec("ALTER TABLE t1 ADD INDEX idx3 (c4);") tk.MustExec("admin check table t1;") - } func (s *testSuite) fillData(tk *testkit.TestKit, table string) { diff --git a/types/datum_test.go b/types/datum_test.go index b6a3d0bfb49fb..fc925ee7ecc5b 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -14,6 +14,8 @@ package types import ( + "reflect" + "testing" "time" . "github.com/pingcap/check" @@ -362,3 +364,39 @@ func (ts *testDatumSuite) TestCopyDatum(c *C) { } } } + +func prepareCompareDatums() ([]Datum, []Datum) { + vals := make([]Datum, 0, 5) + vals = append(vals, NewIntDatum(1)) + vals = append(vals, NewFloat64Datum(1.23)) + vals = append(vals, NewStringDatum("abcde")) + vals = append(vals, NewDecimalDatum(NewDecFromStringForTest("1.2345"))) + vals = append(vals, NewTimeDatum(Time{Time: FromGoTime(time.Date(2018, 3, 8, 16, 1, 0, 315313000, time.UTC)), Fsp: 6, Type: mysql.TypeTimestamp})) + + vals1 := make([]Datum, 0, 5) + vals1 = append(vals1, NewIntDatum(1)) + vals1 = append(vals1, NewFloat64Datum(1.23)) + vals1 = append(vals1, NewStringDatum("abcde")) + vals1 = append(vals1, NewDecimalDatum(NewDecFromStringForTest("1.2345"))) + vals1 = append(vals1, NewTimeDatum(Time{Time: FromGoTime(time.Date(2018, 3, 8, 16, 1, 0, 315313000, time.UTC)), Fsp: 6, Type: mysql.TypeTimestamp})) + return vals, vals1 +} + +func BenchmarkCompareDatum(b *testing.B) { + vals, vals1 := prepareCompareDatums() + sc := new(stmtctx.StatementContext) + b.ResetTimer() + for i := 0; i < b.N; i++ { + for j, v := range vals { + v.CompareDatum(sc, &vals1[j]) + } + } +} + +func BenchmarkCompareDatumByReflect(b *testing.B) { + vals, vals1 := prepareCompareDatums() + b.ResetTimer() + for i := 0; i < b.N; i++ { + reflect.DeepEqual(vals, vals1) + } +} diff --git a/util/admin/admin.go b/util/admin/admin.go index d71434e7bcf3b..325d09df98fc4 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -16,7 +16,6 @@ package admin import ( "fmt" "io" - "reflect" "sort" "github.com/pingcap/tidb/expression" @@ -353,6 +352,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table if err != nil { return errors.Trace(err) } + sc := sessCtx.GetSessionVars().StmtCtx for { vals1, h, err := it.Next() if terror.ErrorEqual(err, io.EOF) { @@ -375,7 +375,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table return errors.Trace(err) } adjustDatumKind(vals1, vals2) - if !reflect.DeepEqual(vals1, vals2) { + if !compareDatumSlice(sc, vals1, vals2) { record1 := &RecordData{Handle: h, Values: vals1} record2 := &RecordData{Handle: h, Values: vals2} return ErrDataInConsistent.GenWithStack("index:%#v != record:%#v", record1, record2) @@ -385,6 +385,19 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table return nil } +func compareDatumSlice(sc *stmtctx.StatementContext, val1s, val2s []types.Datum) bool { + if len(val1s) != len(val2s) { + return false + } + for i, v := range val1s { + res, err := v.CompareDatum(sc, &val2s[i]) + if err != nil || res != 0 { + return false + } + } + return true +} + // CheckRecordAndIndex is exported for testing. func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index, genExprs map[string]expression.Expression) error { sc := sessCtx.GetSessionVars().StmtCtx @@ -503,6 +516,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. } startKey := t.RecordKey(0) + sc := sessCtx.GetSessionVars().StmtCtx filterFunc := func(h int64, vals []types.Datum, cols []*table.Column) (bool, error) { vals2, ok := m[h] if !ok { @@ -514,7 +528,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. return true, nil } - if !reflect.DeepEqual(vals, vals2) { + if !compareDatumSlice(sc, vals, vals2) { record1 := &RecordData{Handle: h, Values: vals2} record2 := &RecordData{Handle: h, Values: vals} return false, ErrDataInConsistent.GenWithStack("data:%#v != record:%#v", record1, record2) From 65f77f7de6d57fcd558bd07882cbfd2707dd516e Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 23 Oct 2018 09:49:01 +0800 Subject: [PATCH 025/509] domain: close slow query channel after closing session pool (#7847) (#7972) From c2c7d3d0ebc79e753e8b7a2a32af46a277445c36 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 23 Oct 2018 11:36:27 +0800 Subject: [PATCH 026/509] stats: limit the length of sample values (#7931) (#7982) --- executor/analyze_test.go | 21 +++++++++++++++++++++ statistics/builder.go | 12 ++++++------ statistics/sample.go | 8 +++++++- 3 files changed, 34 insertions(+), 7 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index ee8ede97710c8..2fc3fdf7dd924 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -15,9 +15,12 @@ package executor_test import ( "fmt" + "strings" + . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testkit" @@ -84,3 +87,21 @@ func (s *testSuite) TestAnalyzeParameters(c *C) { tbl = s.domain.StatsHandle().GetTableStats(tableInfo) c.Assert(tbl.Columns[1].Len(), Equals, 4) } + +func (s *testSuite) TestAnalyzeTooLongColumns(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a json)") + value := fmt.Sprintf(`{"x":"%s"}`, strings.Repeat("x", mysql.MaxFieldVarCharLength)) + tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) + + tk.MustExec("analyze table t") + is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := table.Meta() + tbl := s.domain.StatsHandle().GetTableStats(tableInfo) + c.Assert(tbl.Columns[1].Len(), Equals, 0) + c.Assert(tbl.Columns[1].TotColSize, Equals, int64(65559)) +} diff --git a/statistics/builder.go b/statistics/builder.go index 616716693e59a..4a820240cbf00 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -96,8 +96,12 @@ func (b *SortedBuilder) Iterate(data types.Datum) error { // BuildColumn builds histogram from samples for column. func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) { count := collector.Count - if count == 0 { - return &Histogram{ID: id, NullCount: collector.NullCount}, nil + ndv := collector.FMSketch.NDV() + if ndv > count { + ndv = count + } + if count == 0 || len(collector.Samples) == 0 { + return NewHistogram(id, ndv, collector.NullCount, 0, tp, 0, collector.TotalSize), nil } sc := ctx.GetSessionVars().StmtCtx samples := collector.Samples @@ -105,10 +109,6 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample if err != nil { return nil, errors.Trace(err) } - ndv := collector.FMSketch.NDV() - if ndv > count { - ndv = count - } hg := NewHistogram(id, ndv, collector.NullCount, 0, tp, int(numBuckets), collector.TotalSize) sampleNum := int64(len(samples)) diff --git a/statistics/sample.go b/statistics/sample.go index 6b841fab0bd30..6b58fb5916c38 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -18,6 +18,7 @@ import ( "math/rand" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" @@ -73,6 +74,8 @@ func SampleCollectorToProto(c *SampleCollector) *tipb.SampleCollector { return collector } +const maxSampleValueLength = mysql.MaxFieldVarCharLength / 2 + // SampleCollectorFromProto converts SampleCollector from its protobuf representation. func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector { s := &SampleCollector{ @@ -85,7 +88,10 @@ func SampleCollectorFromProto(collector *tipb.SampleCollector) *SampleCollector } s.CMSketch = CMSketchFromProto(collector.CmSketch) for _, val := range collector.Samples { - s.Samples = append(s.Samples, types.NewBytesDatum(val)) + // When store the histogram bucket boundaries to kv, we need to limit the length of the value. + if len(val) <= maxSampleValueLength { + s.Samples = append(s.Samples, types.NewBytesDatum(val)) + } } return s } From b47f6a04b31df51f4511ff1cd022bfb4bd2cdf20 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 23 Oct 2018 13:19:23 +0800 Subject: [PATCH 027/509] executor: fix panic when limit is too large (#7936) (#8002) --- executor/executor_test.go | 4 ++++ executor/sort.go | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index ccc48b19cbebd..039e18cdf8d23 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -674,6 +674,10 @@ func (s *testSuite) TestSelectOrderBy(c *C) { r = tk.MustQuery("select * from select_order_test order by name, id limit 1 offset 100;") r.Check(testkit.Rows()) + // Test limit exceeds int range. + r = tk.MustQuery("select id from select_order_test order by name, id limit 18446744073709551615;") + r.Check(testkit.Rows("1", "2")) + // Test multiple field r = tk.MustQuery("select id, name from select_order_test where id = 1 group by id, name limit 1 offset 0;") r.Check(testkit.Rows("1 hello")) diff --git a/executor/sort.go b/executor/sort.go index 83fa22618798e..6dcb74361d8d9 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -227,7 +227,7 @@ func (e *SortExec) keyChunksLess(i, j int) bool { type TopNExec struct { SortExec limit *plannercore.PhysicalLimit - totalLimit int + totalLimit uint64 chkHeap *topNChunkHeap } @@ -307,7 +307,7 @@ func (e *TopNExec) Next(ctx context.Context, chk *chunk.Chunk) error { } chk.Reset() if !e.fetched { - e.totalLimit = int(e.limit.Offset + e.limit.Count) + e.totalLimit = e.limit.Offset + e.limit.Count e.Idx = int(e.limit.Offset) err := e.loadChunksUntilTotalLimit(ctx) if err != nil { @@ -335,7 +335,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.rowChunks = chunk.NewList(e.retTypes(), e.initCap, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel("rowChunks") - for e.rowChunks.Len() < e.totalLimit { + for uint64(e.rowChunks.Len()) < e.totalLimit { srcChk := e.children[0].newFirstChunk() err := e.children[0].Next(ctx, srcChk) if err != nil { @@ -363,7 +363,7 @@ const topNCompactionFactor = 4 func (e *TopNExec) executeTopN(ctx context.Context) error { heap.Init(e.chkHeap) - for len(e.rowPtrs) > e.totalLimit { + for uint64(len(e.rowPtrs)) > e.totalLimit { // The number of rows we loaded may exceeds total limit, remove greatest rows by Pop. heap.Pop(e.chkHeap) } From 59d6a936ae831f43aafc7c95fea497c7f9277822 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 23 Oct 2018 14:22:16 +0800 Subject: [PATCH 028/509] store/tikv: log more information when other err occurs (#7948) (#8006) --- store/tikv/coprocessor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 8db253229aa69..02d89a82dd4df 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -730,7 +730,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, resp *copRespo } if otherErr := resp.pbResp.GetOtherError(); otherErr != "" { err := errors.Errorf("other error: %s", otherErr) - log.Warnf("coprocessor err: %v", err) + log.Warnf("txn_start_ts:%d region_id:%d store_addr:%s, coprocessor err: %v", worker.req.StartTs, task.region.id, task.storeAddr, err) return nil, errors.Trace(err) } // When the request is using streaming API, the `Range` is not nil. From 096d2b24efcce67d2dcc90991a6dbc4c0ad2a80e Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Tue, 23 Oct 2018 14:32:37 +0800 Subject: [PATCH 029/509] parser: fix bug empty string in "ESCAPED BY" subclause of "FIELDS" cause panic (#7880) (#8005) --- parser/parser.y | 6 +++++- parser/parser_test.go | 1 + 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/parser/parser.y b/parser/parser.y index a7eb8136dee5c..a94c9259c9e58 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -7013,10 +7013,14 @@ Fields: }else if len(str) != 0 { enclosed = str[0] } + var escaped byte + if len(escape) > 0 { + escaped = escape[0] + } $$ = &ast.FieldsClause{ Terminated: $2.(string), Enclosed: enclosed, - Escaped: escape[0], + Escaped: escaped, } } diff --git a/parser/parser_test.go b/parser/parser_test.go index eb36ab0ab2884..0f790c2d0f55b 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -394,6 +394,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' (a,b) ignore 1 lines", false}, {"load data local infile '/tmp/t.csv' into table t lines starting by 'ab' terminated by 'xy' ignore 1 lines", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' escaped by '*' ignore 1 lines (a,b)", true}, + {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' escaped by ''", true}, // select for update {"SELECT * from t for update", true}, From 4741e6b74104c83db44d4c0f0dbe3a444d964637 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Tue, 23 Oct 2018 15:18:43 +0800 Subject: [PATCH 030/509] executor, planner: clone proj schema for different children in buildProj4Union (#7999) (#8007) --- executor/executor_test.go | 5 +++++ planner/core/logical_plan_builder.go | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 039e18cdf8d23..03ef8c4ec310f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1039,6 +1039,11 @@ func (s *testSuite) TestUnion(c *C) { tk.MustExec(`set @@tidb_max_chunk_size=2;`) tk.MustQuery(`select count(*) from (select t1.a, t1.b from t1 left join t2 on t1.a=t2.a union all select t1.a, t1.a from t1 left join t2 on t1.a=t2.a) tmp;`).Check(testkit.Rows("128")) tk.MustQuery(`select tmp.a, count(*) from (select t1.a, t1.b from t1 left join t2 on t1.a=t2.a union all select t1.a, t1.a from t1 left join t2 on t1.a=t2.a) tmp;`).Check(testkit.Rows("1 128")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t value(1 ,2)") + tk.MustQuery("select a, b from (select a, 0 as d, b from t union all select a, 0 as d, b from t) test;").Check(testkit.Rows("1 2", "1 2")) } func (s *testSuite) TestIn(c *C) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 35653f2db8a8f..6f95472269905 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -678,7 +678,7 @@ func (b *planBuilder) buildProjection4Union(u *LogicalUnionAll) { } b.optFlag |= flagEliminateProjection proj := LogicalProjection{Exprs: exprs}.init(b.ctx) - proj.SetSchema(expression.NewSchema(unionCols...)) + proj.SetSchema(u.schema.Clone()) proj.SetChildren(child) u.children[childID] = proj } From 6cb942b4b1de76f1c879b910b27cbdcf2a02c13a Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Tue, 23 Oct 2018 15:37:15 +0800 Subject: [PATCH 031/509] =?UTF-8?q?types:=20fix=20bug=20which=20Float=20ty?= =?UTF-8?q?pe=20is=20not=20effective=20in=20AddDate=20&=20SubDate=E2=80=A6?= =?UTF-8?q?=20(#8009)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- expression/builtin_time.go | 170 ++++++++++++++++++++++++++++++++- expression/integration_test.go | 3 +- parser/parser_test.go | 2 + types/time.go | 81 ++++++++-------- 4 files changed, 212 insertions(+), 44 deletions(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 556234ff46400..f9c6a9425b81b 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -216,15 +216,19 @@ var ( _ builtinFunc = &builtinAddDateStringDecimalSig{} _ builtinFunc = &builtinAddDateIntStringSig{} _ builtinFunc = &builtinAddDateIntIntSig{} + _ builtinFunc = &builtinAddDateIntDecimalSig{} _ builtinFunc = &builtinAddDateDatetimeStringSig{} _ builtinFunc = &builtinAddDateDatetimeIntSig{} + _ builtinFunc = &builtinAddDateDatetimeDecimalSig{} _ builtinFunc = &builtinSubDateStringStringSig{} _ builtinFunc = &builtinSubDateStringIntSig{} _ builtinFunc = &builtinSubDateStringDecimalSig{} _ builtinFunc = &builtinSubDateIntStringSig{} _ builtinFunc = &builtinSubDateIntIntSig{} + _ builtinFunc = &builtinSubDateIntDecimalSig{} _ builtinFunc = &builtinSubDateDatetimeStringSig{} _ builtinFunc = &builtinSubDateDatetimeIntSig{} + _ builtinFunc = &builtinSubDateDatetimeDecimalSig{} ) func convertTimeToMysqlTime(t time.Time, fsp int) (types.Time, error) { @@ -2589,8 +2593,14 @@ func (du *baseDateArithmitical) getIntervalFromDecimal(ctx sessionctx.Context, a interval = "00:" + interval case "SECOND_MICROSECOND": /* keep interval as original decimal */ + case "SECOND", "MICROSECOND": + args[1] = WrapWithCastAsReal(ctx, args[1]) + interval, isNull, err = args[1].EvalString(ctx, row) + if isNull || err != nil { + return "", true, errors.Trace(err) + } default: - // YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND, MICROSECOND + // YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE args[1] = WrapWithCastAsInt(ctx, args[1]) interval, isNull, err = args[1].EvalString(ctx, row) if isNull || err != nil { @@ -2620,7 +2630,8 @@ func (du *baseDateArithmitical) add(ctx sessionctx.Context, date types.Time, int return types.Time{}, true, errors.Trace(err) } - goTime = goTime.Add(dur) + duration := time.Duration(dur) + goTime = goTime.Add(duration) goTime = goTime.AddDate(int(year), int(month), int(day)) if goTime.Nanosecond() == 0 { @@ -2645,7 +2656,8 @@ func (du *baseDateArithmitical) sub(ctx sessionctx.Context, date types.Time, int return types.Time{}, true, errors.Trace(err) } - goTime = goTime.Add(dur) + duration := time.Duration(dur) + goTime = goTime.Add(duration) goTime = goTime.AddDate(int(year), int(month), int(day)) if goTime.Nanosecond() == 0 { @@ -2707,6 +2719,11 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres baseBuiltinFunc: bf, baseDateArithmitical: newDateArighmeticalUtil(), } + case dateEvalTp == types.ETInt && intervalEvalTp == types.ETDecimal: + sig = &builtinAddDateIntDecimalSig{ + baseBuiltinFunc: bf, + baseDateArithmitical: newDateArighmeticalUtil(), + } case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETString: sig = &builtinAddDateDatetimeStringSig{ baseBuiltinFunc: bf, @@ -2717,6 +2734,11 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres baseBuiltinFunc: bf, baseDateArithmitical: newDateArighmeticalUtil(), } + case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETDecimal: + sig = &builtinAddDateDatetimeDecimalSig{ + baseBuiltinFunc: bf, + baseDateArithmitical: newDateArighmeticalUtil(), + } } return sig, nil } @@ -2886,6 +2908,39 @@ func (b *builtinAddDateIntIntSig) evalTime(row chunk.Row) (types.Time, bool, err return result, isNull || err != nil, errors.Trace(err) } +type builtinAddDateIntDecimalSig struct { + baseBuiltinFunc + baseDateArithmitical +} + +func (b *builtinAddDateIntDecimalSig) Clone() builtinFunc { + newSig := &builtinAddDateIntDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalTime evals ADDDATE(date,INTERVAL expr unit). +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate +func (b *builtinAddDateIntDecimalSig) evalTime(row chunk.Row) (types.Time, bool, error) { + unit, isNull, err := b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + date, isNull, err := b.getDateFromInt(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + interval, isNull, err := b.getIntervalFromDecimal(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + result, isNull, err := b.add(b.ctx, date, interval, unit) + return result, isNull || err != nil, errors.Trace(err) +} + type builtinAddDateDatetimeStringSig struct { baseBuiltinFunc baseDateArithmitical @@ -2952,6 +3007,39 @@ func (b *builtinAddDateDatetimeIntSig) evalTime(row chunk.Row) (types.Time, bool return result, isNull || err != nil, errors.Trace(err) } +type builtinAddDateDatetimeDecimalSig struct { + baseBuiltinFunc + baseDateArithmitical +} + +func (b *builtinAddDateDatetimeDecimalSig) Clone() builtinFunc { + newSig := &builtinAddDateDatetimeDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalTime evals ADDDATE(date,INTERVAL expr unit). +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate +func (b *builtinAddDateDatetimeDecimalSig) evalTime(row chunk.Row) (types.Time, bool, error) { + unit, isNull, err := b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + date, isNull, err := b.getDateFromDatetime(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + interval, isNull, err := b.getIntervalFromDecimal(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + result, isNull, err := b.add(b.ctx, date, interval, unit) + return result, isNull || err != nil, errors.Trace(err) +} + type subDateFunctionClass struct { baseFunctionClass } @@ -3001,6 +3089,11 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres baseBuiltinFunc: bf, baseDateArithmitical: newDateArighmeticalUtil(), } + case dateEvalTp == types.ETInt && intervalEvalTp == types.ETDecimal: + sig = &builtinSubDateIntDecimalSig{ + baseBuiltinFunc: bf, + baseDateArithmitical: newDateArighmeticalUtil(), + } case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETString: sig = &builtinSubDateDatetimeStringSig{ baseBuiltinFunc: bf, @@ -3011,6 +3104,11 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres baseBuiltinFunc: bf, baseDateArithmitical: newDateArighmeticalUtil(), } + case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETDecimal: + sig = &builtinSubDateDatetimeDecimalSig{ + baseBuiltinFunc: bf, + baseDateArithmitical: newDateArighmeticalUtil(), + } } return sig, nil } @@ -3183,6 +3281,39 @@ type builtinSubDateDatetimeStringSig struct { baseDateArithmitical } +type builtinSubDateIntDecimalSig struct { + baseBuiltinFunc + baseDateArithmitical +} + +func (b *builtinSubDateIntDecimalSig) Clone() builtinFunc { + newSig := &builtinSubDateIntDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalTime evals SUBDATE(date,INTERVAL expr unit). +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate +func (b *builtinSubDateIntDecimalSig) evalTime(row chunk.Row) (types.Time, bool, error) { + unit, isNull, err := b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + date, isNull, err := b.getDateFromInt(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + interval, isNull, err := b.getIntervalFromDecimal(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + result, isNull, err := b.sub(b.ctx, date, interval, unit) + return result, isNull || err != nil, errors.Trace(err) +} + func (b *builtinSubDateDatetimeStringSig) Clone() builtinFunc { newSig := &builtinSubDateDatetimeStringSig{baseDateArithmitical: b.baseDateArithmitical} newSig.cloneFrom(&b.baseBuiltinFunc) @@ -3244,6 +3375,39 @@ func (b *builtinSubDateDatetimeIntSig) evalTime(row chunk.Row) (types.Time, bool return result, isNull || err != nil, errors.Trace(err) } +type builtinSubDateDatetimeDecimalSig struct { + baseBuiltinFunc + baseDateArithmitical +} + +func (b *builtinSubDateDatetimeDecimalSig) Clone() builtinFunc { + newSig := &builtinSubDateDatetimeDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalTime evals SUBDATE(date,INTERVAL expr unit). +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate +func (b *builtinSubDateDatetimeDecimalSig) evalTime(row chunk.Row) (types.Time, bool, error) { + unit, isNull, err := b.args[2].EvalString(b.ctx, row) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + date, isNull, err := b.getDateFromDatetime(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + interval, isNull, err := b.getIntervalFromDecimal(b.ctx, b.args, row, unit) + if isNull || err != nil { + return types.Time{}, true, errors.Trace(err) + } + + result, isNull, err := b.sub(b.ctx, date, interval, unit) + return result, isNull || err != nil, errors.Trace(err) +} + type timestampDiffFunctionClass struct { baseFunctionClass } diff --git a/expression/integration_test.go b/expression/integration_test.go index cdb8063583d94..2fc1dca204dbf 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1798,8 +1798,9 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { {"\"2011-01-01 00:00:00\"", "10.10", "DAY", "2011-01-11 00:00:00", "2010-12-22 00:00:00"}, {"\"2011-01-01 00:00:00\"", "10.10", "HOUR", "2011-01-01 10:00:00", "2010-12-31 14:00:00"}, {"\"2011-01-01 00:00:00\"", "10.10", "MINUTE", "2011-01-01 00:10:00", "2010-12-31 23:50:00"}, - {"\"2011-01-01 00:00:00\"", "10.10", "SECOND", "2011-01-01 00:00:10", "2010-12-31 23:59:50"}, + {"\"2011-01-01 00:00:00\"", "10.10", "SECOND", "2011-01-01 00:00:10.100000", "2010-12-31 23:59:49.900000"}, {"\"2011-01-01 00:00:00\"", "10.10", "MICROSECOND", "2011-01-01 00:00:00.000010", "2010-12-31 23:59:59.999990"}, + {"\"2011-01-01 00:00:00\"", "10.90", "MICROSECOND", "2011-01-01 00:00:00.000011", "2010-12-31 23:59:59.999989"}, {"\"2009-01-01\"", "6/4", "HOUR_MINUTE", "2009-01-04 12:20:00", "2008-12-28 11:40:00"}, {"\"2009-01-01\"", "6/0", "HOUR_MINUTE", "", ""}, diff --git a/parser/parser_test.go b/parser/parser_test.go index 0f790c2d0f55b..15370afa33d93 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -872,6 +872,8 @@ func (s *testParserSuite) TestBuiltin(c *C) { {"select now(6)", true}, {"select sysdate(), sysdate(6)", true}, {"SELECT time('01:02:03');", true}, + {"SELECT time('01:02:03.1')", true}, + {"SELECT time('20.1')", true}, {"SELECT TIMEDIFF('2000:01:01 00:00:00', '2000:01:01 00:00:00.000001');", true}, {"SELECT TIMESTAMPDIFF(MONTH,'2003-02-01','2003-05-01');", true}, {"SELECT TIMESTAMPDIFF(YEAR,'2002-05-01','2001-01-01');", true}, diff --git a/types/time.go b/types/time.go index 0e560fd70cc44..6c5a77f5bd479 100644 --- a/types/time.go +++ b/types/time.go @@ -153,7 +153,8 @@ var ( func FromGoTime(t gotime.Time) MysqlTime { year, month, day := t.Date() hour, minute, second := t.Clock() - microsecond := t.Nanosecond() / 1000 + // Nanosecond plus 500 then divided 1000 means rounding to microseconds. + microsecond := (t.Nanosecond() + 500) / 1000 return FromDate(year, int(month), day, hour, minute, second, microsecond) } @@ -1506,22 +1507,22 @@ func ExtractDurationNum(d *Duration, unit string) (int64, error) { } } -func extractSingleTimeValue(unit string, format string) (int64, int64, int64, gotime.Duration, error) { - iv, err := strconv.ParseInt(format, 10, 64) +func extractSingleTimeValue(unit string, format string) (int64, int64, int64, float64, error) { + fv, err := strconv.ParseFloat(format, 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } + iv := int64(fv + 0.5) - v := gotime.Duration(iv) switch strings.ToUpper(unit) { case "MICROSECOND": - return 0, 0, 0, v * gotime.Microsecond, nil + return 0, 0, 0, fv * float64(gotime.Microsecond), nil case "SECOND": - return 0, 0, 0, v * gotime.Second, nil + return 0, 0, 0, fv * float64(gotime.Second), nil case "MINUTE": - return 0, 0, 0, v * gotime.Minute, nil + return 0, 0, 0, float64(iv * int64(gotime.Minute)), nil case "HOUR": - return 0, 0, 0, v * gotime.Hour, nil + return 0, 0, 0, float64(iv * int64(gotime.Hour)), nil case "DAY": return 0, 0, iv, 0, nil case "WEEK": @@ -1538,33 +1539,33 @@ func extractSingleTimeValue(unit string, format string) (int64, int64, int64, go } // extractSecondMicrosecond extracts second and microsecond from a string and its format is `SS.FFFFFF`. -func extractSecondMicrosecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractSecondMicrosecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ".") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - seconds, err := strconv.ParseInt(fields[0], 10, 64) + seconds, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - microseconds, err := strconv.ParseInt(alignFrac(fields[1], MaxFsp), 10, 64) + microseconds, err := strconv.ParseFloat(alignFrac(fields[1], MaxFsp), 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - return 0, 0, 0, gotime.Duration(seconds)*gotime.Second + gotime.Duration(microseconds)*gotime.Microsecond, nil + return 0, 0, 0, seconds*float64(gotime.Second) + microseconds*float64(gotime.Microsecond), nil } // extractMinuteMicrosecond extracts minutes and microsecond from a string and its format is `MM:SS.FFFFFF`. -func extractMinuteMicrosecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractMinuteMicrosecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ":") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - minutes, err := strconv.ParseInt(fields[0], 10, 64) + minutes, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } @@ -1574,42 +1575,42 @@ func extractMinuteMicrosecond(format string) (int64, int64, int64, gotime.Durati return 0, 0, 0, 0, errors.Trace(err) } - return 0, 0, 0, gotime.Duration(minutes)*gotime.Minute + value, nil + return 0, 0, 0, minutes*float64(gotime.Minute) + value, nil } // extractMinuteSecond extracts minutes and second from a string and its format is `MM:SS`. -func extractMinuteSecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractMinuteSecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ":") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - minutes, err := strconv.ParseInt(fields[0], 10, 64) + minutes, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - seconds, err := strconv.ParseInt(fields[1], 10, 64) + seconds, err := strconv.ParseFloat(fields[1], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - return 0, 0, 0, gotime.Duration(minutes)*gotime.Minute + gotime.Duration(seconds)*gotime.Second, nil + return 0, 0, 0, minutes*float64(gotime.Minute) + seconds*float64(gotime.Second), nil } // extractHourMicrosecond extracts hour and microsecond from a string and its format is `HH:MM:SS.FFFFFF`. -func extractHourMicrosecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractHourMicrosecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ":") if len(fields) != 3 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - hours, err := strconv.ParseInt(fields[0], 10, 64) + hours, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - minutes, err := strconv.ParseInt(fields[1], 10, 64) + minutes, err := strconv.ParseFloat(fields[1], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } @@ -1619,56 +1620,56 @@ func extractHourMicrosecond(format string) (int64, int64, int64, gotime.Duration return 0, 0, 0, 0, errors.Trace(err) } - return 0, 0, 0, gotime.Duration(hours)*gotime.Hour + gotime.Duration(minutes)*gotime.Minute + value, nil + return 0, 0, 0, hours*float64(gotime.Hour) + minutes*float64(gotime.Minute) + value, nil } // extractHourSecond extracts hour and second from a string and its format is `HH:MM:SS`. -func extractHourSecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractHourSecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ":") if len(fields) != 3 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - hours, err := strconv.ParseInt(fields[0], 10, 64) + hours, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - minutes, err := strconv.ParseInt(fields[1], 10, 64) + minutes, err := strconv.ParseFloat(fields[1], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - seconds, err := strconv.ParseInt(fields[2], 10, 64) + seconds, err := strconv.ParseFloat(fields[2], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - return 0, 0, 0, gotime.Duration(hours)*gotime.Hour + gotime.Duration(minutes)*gotime.Minute + gotime.Duration(seconds)*gotime.Second, nil + return 0, 0, 0, hours*float64(gotime.Hour) + minutes*float64(gotime.Minute) + seconds*float64(gotime.Second), nil } // extractHourMinute extracts hour and minute from a string and its format is `HH:MM`. -func extractHourMinute(format string) (int64, int64, int64, gotime.Duration, error) { +func extractHourMinute(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, ":") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - hours, err := strconv.ParseInt(fields[0], 10, 64) + hours, err := strconv.ParseFloat(fields[0], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - minutes, err := strconv.ParseInt(fields[1], 10, 64) + minutes, err := strconv.ParseFloat(fields[1], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - return 0, 0, 0, gotime.Duration(hours)*gotime.Hour + gotime.Duration(minutes)*gotime.Minute, nil + return 0, 0, 0, hours*float64(gotime.Hour) + minutes*float64(gotime.Minute), nil } // extractDayMicrosecond extracts day and microsecond from a string and its format is `DD HH:MM:SS.FFFFFF`. -func extractDayMicrosecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractDayMicrosecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, " ") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) @@ -1688,7 +1689,7 @@ func extractDayMicrosecond(format string) (int64, int64, int64, gotime.Duration, } // extractDaySecond extracts day and hour from a string and its format is `DD HH:MM:SS`. -func extractDaySecond(format string) (int64, int64, int64, gotime.Duration, error) { +func extractDaySecond(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, " ") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) @@ -1708,7 +1709,7 @@ func extractDaySecond(format string) (int64, int64, int64, gotime.Duration, erro } // extractDayMinute extracts day and minute from a string and its format is `DD HH:MM`. -func extractDayMinute(format string) (int64, int64, int64, gotime.Duration, error) { +func extractDayMinute(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, " ") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) @@ -1728,7 +1729,7 @@ func extractDayMinute(format string) (int64, int64, int64, gotime.Duration, erro } // extractDayHour extracts day and hour from a string and its format is `DD HH`. -func extractDayHour(format string) (int64, int64, int64, gotime.Duration, error) { +func extractDayHour(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, " ") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) @@ -1739,16 +1740,16 @@ func extractDayHour(format string) (int64, int64, int64, gotime.Duration, error) return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - hours, err := strconv.ParseInt(fields[1], 10, 64) + hours, err := strconv.ParseFloat(fields[1], 64) if err != nil { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) } - return 0, 0, days, gotime.Duration(hours) * gotime.Hour, nil + return 0, 0, days, hours * float64(gotime.Hour), nil } // extractYearMonth extracts year and month from a string and its format is `YYYY-MM`. -func extractYearMonth(format string) (int64, int64, int64, gotime.Duration, error) { +func extractYearMonth(format string) (int64, int64, int64, float64, error) { fields := strings.Split(format, "-") if len(fields) != 2 { return 0, 0, 0, 0, ErrIncorrectDatetimeValue.GenWithStackByArgs(format) @@ -1768,7 +1769,7 @@ func extractYearMonth(format string) (int64, int64, int64, gotime.Duration, erro } // ExtractTimeValue extracts time value from time unit and format. -func ExtractTimeValue(unit string, format string) (int64, int64, int64, gotime.Duration, error) { +func ExtractTimeValue(unit string, format string) (int64, int64, int64, float64, error) { switch strings.ToUpper(unit) { case "MICROSECOND", "SECOND", "MINUTE", "HOUR", "DAY", "WEEK", "MONTH", "QUARTER", "YEAR": return extractSingleTimeValue(unit, format) From 500c9bb865b90a834bcd27b7b075c193556b9f37 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Tue, 23 Oct 2018 23:29:00 +0800 Subject: [PATCH 032/509] add changelog for 2.1.0 rc4 (#8020) (#8027) --- CHANGELOG.md | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3ca85b8598146..4181b0dfc0773 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,37 @@ # TiDB Changelog All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/pingcap/pd/blob/master/CHANGELOG.md). +## [2.1.0-rc.4] - 2018-10-23 +### SQL Optimizer +* Fix the issue that column pruning of `UnionAll` is incorrect in some cases [#7941](https://github.com/pingcap/tidb/pull/7941) +* Fix the issue that the result of the `UnionAll` operator is incorrect in some cases [#8007](https://github.com/pingcap/tidb/pull/8007) +### SQL Execution Engine +* Fix the precision issue of the `AVG` function [#7874](https://github.com/pingcap/tidb/pull/7874) +* Support using the `EXPLAIN ANALYZE` statement to check the runtime statistics including the execution time and the number of returned rows of each operator during the query execution process [#7925](https://github.com/pingcap/tidb/pull/7925) +* Fix the panic issue of the `PointGet` operator when a column of a table appears multiple times in the result set [#7943](https://github.com/pingcap/tidb/pull/7943) +* Fix the panic issue caused by too large values in the `Limit` subclause [#8002](https://github.com/pingcap/tidb/pull/8002) +* Fix the panic issue during the execution process of the `AddDate`/`SubDate` statement in some cases [#8009](https://github.com/pingcap/tidb/pull/8009) +### Statistics +* Fix the issue of judging the prefix of the histogram low-bound of the combined index as out of range [#7856](https://github.com/pingcap/tidb/pull/7856) +* Fix the memory leak issue caused by statistics collecting [#7873](https://github.com/pingcap/tidb/pull/7873) +* Fix the panic issue when the histogram is empty [#7928](https://github.com/pingcap/tidb/pull/7928) +* Fix the issue that the histogram bound is out of range when the statistics is being uploaded [#7944](https://github.com/pingcap/tidb/pull/7944) +* Limit the maximum length of values in the statistics sampling process [#7982](https://github.com/pingcap/tidb/pull/7982) +### Server +* Refactor Latch to avoid misjudgment of transaction conflicts and improve the execution performance of concurrent transactions [#7711](https://github.com/pingcap/tidb/pull/7711) +* Fix the panic issue caused by collecting slow queries in some cases [#7874](https://github.com/pingcap/tidb/pull/7847) +* Fix the panic issue when `ESCAPED BY` is an empty string in the `LOAD DATA` statement [#8005](https://github.com/pingcap/tidb/pull/8005) +* Complete the “coprocessor error” log information [#8006](https://github.com/pingcap/tidb/pull/8006) +### Compatibility +* Set the `Command` field of the `SHOW PROCESSLIST` result to `Sleep` when the query is empty [#7839](https://github.com/pingcap/tidb/pull/7839) +### Expressions +* Fix the constant folding issue of the `SYSDATE` function [#7895](https://github.com/pingcap/tidb/pull/7895) +* Fix the issue that `SUBSTRING_INDEX` panics in some cases [#7897](https://github.com/pingcap/tidb/pull/7897) +### DDL +* Fix the stack overflow issue caused by throwing the `invalid ddl job type` error [#7958](https://github.com/pingcap/tidb/pull/7958) +* Fix the issue that the result of `ADMIN CHECK TABLE` is incorrect in some cases [#7975](https://github.com/pingcap/tidb/pull/7975) + + ## [2.1.0-rc.2] - 2018-09-14 ### SQL Optimizer * Put forward a proposal of the next generation Planner [#7543](https://github.com/pingcap/tidb/pull/7543) @@ -15,7 +46,7 @@ All notable changes to this project will be documented in this file. See also [R * Optimize the performance of Hash aggregate operators [#7541](https://github.com/pingcap/tidb/pull/7541) * Optimize the performance of Join operators [#7493](https://github.com/pingcap/tidb/pull/7493), [#7433](https://github.com/pingcap/tidb/pull/7433) * Fix the issue that the result of `UPDATE JOIN` is incorrect when the Join order is changed [#7571](https://github.com/pingcap/tidb/pull/7571) -* Improve the performance of Chunk’s iterator [#7585](https://github.com/pingcap/tidb/pull/7585) +* Improve the performance of Chunk’s iterator [#7585](https://github.com/pingcap/tidb/pull/7585) ### Statistics * Fix the issue that the auto Analyze work repeatedly analyzes the statistics [#7550](https://github.com/pingcap/tidb/pull/7550) * Fix the statistics update error that occurs when there is no statistics change [#7530](https://github.com/pingcap/tidb/pull/7530) From 80e75845beca81d8d4f4939a5d4c2dd801bf303d Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 24 Oct 2018 17:39:46 +0800 Subject: [PATCH 033/509] stats: fix estimation for out of range point queries (#8015) (#8035) --- .../r/explain_complex_stats.result | 10 ++++----- cmd/explaintest/r/explain_easy_stats.result | 6 ++--- cmd/explaintest/r/tpch.result | 20 ++++++++--------- statistics/ddl_test.go | 2 +- statistics/histogram.go | 16 +++++++------- statistics/selectivity_test.go | 22 +++++++++++++++++-- statistics/table.go | 4 ++-- 7 files changed, 49 insertions(+), 31 deletions(-) diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index f00603e25c4ad..b0723109e8eb1 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -158,11 +158,11 @@ Projection_5 39.28 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st. └─TableScan_14 160.23 cop table:st, keep order:false explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info -Projection_9 428.55 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 -└─Limit_12 428.55 root offset:0, count:2000 - └─IndexJoin_18 428.55 root inner join, inner:IndexLookUp_17, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic - ├─TableReader_42 428.55 root data:Selection_41 - │ └─Selection_41 428.55 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592) +Projection_9 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 +└─Limit_12 428.32 root offset:0, count:2000 + └─IndexJoin_18 428.32 root inner join, inner:IndexLookUp_17, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic + ├─TableReader_42 428.32 root data:Selection_41 + │ └─Selection_41 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592) │ └─TableScan_40 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_17 970.00 root ├─IndexScan_14 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index b93c0e4b6f3fd..73cbf7877b685 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -47,10 +47,10 @@ explain select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id count task operator info Projection_6 2481.25 root test.t1.c1, test.t1.c2, test.t1.c3, test.t2.c1, test.t2.c2 └─MergeJoin_7 2481.25 root left outer join, left key:test.t1.c2, right key:test.t2.c1 - ├─IndexLookUp_17 1999.00 root - │ ├─Selection_16 1999.00 cop gt(test.t1.c1, 1) + ├─IndexLookUp_17 1998.00 root + │ ├─Selection_16 1998.00 cop gt(test.t1.c1, 1) │ │ └─IndexScan_14 1999.00 cop table:t1, index:c2, range:[NULL,+inf], keep order:true - │ └─TableScan_15 1999.00 cop table:t1, keep order:false + │ └─TableScan_15 1998.00 cop table:t1, keep order:false └─IndexLookUp_21 1985.00 root ├─IndexScan_19 1985.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true └─TableScan_20 1985.00 cop table:t2, keep order:false diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 1efe1d727b6ec..25c5be78f56b2 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -251,7 +251,7 @@ limit 10; id count task operator info Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderdate, tpch.orders.o_shippriority └─TopN_17 10.00 root 7_col_0:desc, tpch.orders.o_orderdate:asc, offset:0, count:10 - └─HashAgg_20 40256361.71 root group by:tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.orders.o_orderdate), firstrow(tpch.orders.o_shippriority), firstrow(tpch.lineitem.l_orderkey) + └─HashAgg_20 40227041.09 root group by:tpch.lineitem.l_orderkey, tpch.orders.o_orderdate, tpch.orders.o_shippriority, funcs:sum(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount))), firstrow(tpch.orders.o_orderdate), firstrow(tpch.orders.o_shippriority), firstrow(tpch.lineitem.l_orderkey) └─IndexJoin_26 91515927.49 root inner join, inner:IndexLookUp_25, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey ├─HashRightJoin_46 22592975.51 root inner join, inner:TableReader_52, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] │ ├─TableReader_52 1498236.00 root data:Selection_51 @@ -260,9 +260,9 @@ Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderd │ └─TableReader_49 36870000.00 root data:Selection_48 │ └─Selection_48 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) │ └─TableScan_47 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_25 163063881.42 root + └─IndexLookUp_25 162945114.27 root ├─IndexScan_22 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_24 163063881.42 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) + └─Selection_24 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_23 1.00 cop table:lineitem, keep order:false /* Q4 Order Priority Checking Query @@ -922,13 +922,13 @@ p_brand, p_type, p_size; id count task operator info -Sort_13 15.00 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc -└─Projection_14 15.00 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 - └─HashAgg_17 15.00 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) - └─HashLeftJoin_22 4022816.68 root anti semi join, inner:TableReader_46, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] - ├─IndexJoin_26 5028520.85 root inner join, inner:IndexReader_25, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey - │ ├─TableReader_41 1249969.60 root data:Selection_40 - │ │ └─Selection_40 1249969.60 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) +Sort_13 14.41 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_type:asc, tpch.part.p_size:asc +└─Projection_14 14.41 root tpch.part.p_brand, tpch.part.p_type, tpch.part.p_size, 9_col_0 + └─HashAgg_17 14.41 root group by:tpch.part.p_brand, tpch.part.p_size, tpch.part.p_type, funcs:count(distinct tpch.partsupp.ps_suppkey), firstrow(tpch.part.p_brand), firstrow(tpch.part.p_type), firstrow(tpch.part.p_size) + └─HashLeftJoin_22 3863988.24 root anti semi join, inner:TableReader_46, equal:[eq(tpch.partsupp.ps_suppkey, tpch.supplier.s_suppkey)] + ├─IndexJoin_26 4829985.30 root inner join, inner:IndexReader_25, outer key:tpch.part.p_partkey, inner key:tpch.partsupp.ps_partkey + │ ├─TableReader_41 1200618.43 root data:Selection_40 + │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_25 1.00 root index:IndexScan_24 │ └─IndexScan_24 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index a226eb97ae9e9..b5906e7224c18 100644 --- a/statistics/ddl_test.go +++ b/statistics/ddl_test.go @@ -137,7 +137,7 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { c.Assert(count, Equals, float64(2)) count, err = statsTbl.ColumnEqualRowCount(sc, types.NewIntDatum(1), tableInfo.Columns[3].ID) c.Assert(err, IsNil) - c.Assert(count, Equals, float64(2)) + c.Assert(count, Equals, float64(0)) testKit.MustExec("alter table t add column c4 datetime NOT NULL default CURRENT_TIMESTAMP") err = h.HandleDDLEvent(<-h.DDLEventCh()) diff --git a/statistics/histogram.go b/statistics/histogram.go index ca52273b20e1a..56ae7c104bf35 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -729,7 +729,7 @@ func (c *Column) String() string { return c.Histogram.ToString(0) } -func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum) (float64, error) { +func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, modifyCount int64) (float64, error) { if val.IsNull() { return float64(c.NullCount), nil } @@ -738,7 +738,7 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum) (f return 0.0, nil } if c.NDV > 0 && c.outOfRange(val) { - return c.totalRowCount() / (float64(c.NDV)), nil + return float64(modifyCount) / float64(c.NDV), nil } if c.CMSketch != nil { count, err := c.CMSketch.queryValue(sc, val) @@ -759,7 +759,7 @@ func (c *Column) getColumnRowCount(sc *stmtctx.StatementContext, ranges []*range // the point case. if !rg.LowExclude && !rg.HighExclude { var cnt float64 - cnt, err = c.equalRowCount(sc, rg.LowVal[0]) + cnt, err = c.equalRowCount(sc, rg.LowVal[0], modifyCount) if err != nil { return 0, errors.Trace(err) } @@ -773,14 +773,14 @@ func (c *Column) getColumnRowCount(sc *stmtctx.StatementContext, ranges []*range cnt += float64(modifyCount) / outOfRangeBetweenRate } if rg.LowExclude { - lowCnt, err := c.equalRowCount(sc, rg.LowVal[0]) + lowCnt, err := c.equalRowCount(sc, rg.LowVal[0], modifyCount) if err != nil { return 0, errors.Trace(err) } cnt -= lowCnt } if !rg.HighExclude { - highCnt, err := c.equalRowCount(sc, rg.HighVal[0]) + highCnt, err := c.equalRowCount(sc, rg.HighVal[0], modifyCount) if err != nil { return 0, errors.Trace(err) } @@ -809,10 +809,10 @@ func (idx *Index) String() string { return idx.Histogram.ToString(len(idx.Info.Columns)) } -func (idx *Index) equalRowCount(sc *stmtctx.StatementContext, b []byte) float64 { +func (idx *Index) equalRowCount(sc *stmtctx.StatementContext, b []byte, modifyCount int64) float64 { val := types.NewBytesDatum(b) if idx.NDV > 0 && idx.outOfRange(val) { - return idx.totalRowCount() / (float64(idx.NDV)) + return float64(modifyCount) / (float64(idx.NDV)) } if idx.CMSketch != nil { return float64(idx.CMSketch.QueryBytes(b)) @@ -834,7 +834,7 @@ func (idx *Index) getRowCount(sc *stmtctx.StatementContext, indexRanges []*range fullLen := len(indexRange.LowVal) == len(indexRange.HighVal) && len(indexRange.LowVal) == len(idx.Info.Columns) if fullLen && bytes.Equal(lb, rb) { if !indexRange.LowExclude && !indexRange.HighExclude { - totalCount += idx.equalRowCount(sc, lb) + totalCount += idx.equalRowCount(sc, lb, modifyCount) } continue } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 77225890c2fca..0a2a4e13c07a9 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -158,7 +158,7 @@ func (s *testSelectivitySuite) TestSelectivity(c *C) { }, { exprs: "a >= 1 and b > 1 and a < 2", - selectivity: 0.01817558299, + selectivity: 0.01783264746, }, { exprs: "a >= 1 and c > 1 and a < 2", @@ -174,7 +174,7 @@ func (s *testSelectivitySuite) TestSelectivity(c *C) { }, { exprs: "b > 1", - selectivity: 0.98148148148, + selectivity: 0.96296296296, }, { exprs: "a > 1 and b < 2 and c > 3 and d < 4 and e > 5", @@ -304,6 +304,24 @@ func (s *testSelectivitySuite) TestEstimationForUnknownValues(c *C) { count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(1, 30)) c.Assert(err, IsNil) c.Assert(count, Equals, 0.0) + + testKit.MustExec("drop table t") + testKit.MustExec("create table t(a int, b int, index idx(b))") + testKit.MustExec("insert into t values (1,1)") + testKit.MustExec("analyze table t") + table, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + statsTbl = h.GetTableStats(table.Meta()) + + colID = table.Meta().Columns[0].ID + count, err = statsTbl.GetRowCountByColumnRanges(sc, colID, getRange(2, 2)) + c.Assert(err, IsNil) + c.Assert(count, Equals, 0.0) + + idxID = table.Meta().Indices[0].ID + count, err = statsTbl.GetRowCountByIndexRanges(sc, idxID, getRange(2, 2)) + c.Assert(err, IsNil) + c.Assert(count, Equals, 0.0) } func BenchmarkSelectivity(b *testing.B) { diff --git a/statistics/table.go b/statistics/table.go index 3592b95fea1ef..1d05d1b284e6d 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -394,7 +394,7 @@ func (t *Table) ColumnEqualRowCount(sc *stmtctx.StatementContext, value types.Da return float64(t.Count) / pseudoEqualRate, nil } c := t.Columns[colID] - result, err := c.equalRowCount(sc, value) + result, err := c.equalRowCount(sc, value, t.ModifyCount) result *= c.getIncreaseFactor(t.Count) return result, errors.Trace(err) } @@ -551,7 +551,7 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idxID int64 // so we use heuristic methods to estimate the selectivity. if idx.NDV > 0 && len(ran.LowVal) == len(idx.Info.Columns) && rangePosition == len(ran.LowVal) { // for equality queries - selectivity = 1.0 / float64(idx.NDV) + selectivity = float64(coll.ModifyCount) / float64(idx.NDV) / idx.totalRowCount() } else { // for range queries selectivity = float64(coll.ModifyCount) / outOfRangeBetweenRate / idx.totalRowCount() From 73692d1dd03529ac7c1fb42c53d0b7bcf16e7d78 Mon Sep 17 00:00:00 2001 From: lysu Date: Wed, 24 Oct 2018 20:04:29 +0800 Subject: [PATCH 034/509] executor: improve wide table insert & update performance (#7935) (#8024) --- executor/builder.go | 13 +++++--- executor/insert.go | 10 ++---- executor/insert_common.go | 67 ++++++++++++++++++++++++++------------- executor/load_data.go | 7 ++-- executor/replace.go | 10 ++---- executor/update.go | 10 ++++-- 6 files changed, 70 insertions(+), 47 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 1c62d4cbf5702..8fb430f113633 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -548,6 +548,11 @@ func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { hasRefCols: v.NeedFillDefaultValue, SelectExec: selectExec, } + err := ivs.initInsertColumns() + if err != nil { + b.err = err + return nil + } if v.IsReplace { return b.buildReplace(ivs) @@ -572,17 +577,16 @@ func (b *executorBuilder) buildLoadData(v *plannercore.LoadData) Executor { GenColumns: v.GenCols.Columns, GenExprs: v.GenCols.Exprs, } - tableCols := tbl.Cols() - columns, err := insertVal.getColumns(tableCols) + err := insertVal.initInsertColumns() if err != nil { - b.err = errors.Trace(err) + b.err = err return nil } loadDataExec := &LoadDataExec{ baseExecutor: newBaseExecutor(b.ctx, nil, v.ExplainID()), IsLocal: v.IsLocal, loadDataInfo: &LoadDataInfo{ - row: make([]types.Datum, len(columns)), + row: make([]types.Datum, len(insertVal.insertColumns)), InsertValues: insertVal, Path: v.Path, Table: tbl, @@ -590,7 +594,6 @@ func (b *executorBuilder) buildLoadData(v *plannercore.LoadData) Executor { LinesInfo: v.LinesInfo, IgnoreLines: v.IgnoreLines, Ctx: b.ctx, - columns: columns, }, } diff --git a/executor/insert.go b/executor/insert.go index 932a2c09ce326..a62a2524c8003 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -129,15 +129,10 @@ func (e *InsertExec) batchUpdateDupRows(newRows [][]types.Datum) error { // Next implements Exec Next interface. func (e *InsertExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - cols, err := e.getColumns(e.Table.Cols()) - if err != nil { - return errors.Trace(err) - } - if len(e.children) > 0 && e.children[0] != nil { - return errors.Trace(e.insertRowsFromSelect(ctx, cols, e.exec)) + return e.insertRowsFromSelect(ctx, e.exec) } - return errors.Trace(e.insertRows(cols, e.exec)) + return e.insertRows(e.exec) } // Close implements the Executor Close interface. @@ -154,6 +149,7 @@ func (e *InsertExec) Open(ctx context.Context) error { if e.SelectExec != nil { return e.SelectExec.Open(ctx) } + e.initEvalBuffer() return nil } diff --git a/executor/insert_common.go b/executor/insert_common.go index 2517420943feb..72123ec508a40 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -50,9 +50,13 @@ type InsertValues struct { GenColumns []*ast.ColumnName GenExprs []expression.Expression + insertColumns []*table.Column + // colDefaultVals is used to store casted default value. // Because not every insert statement needs colDefaultVals, so we will init the buffer lazily. - colDefaultVals []defaultVal + colDefaultVals []defaultVal + evalBuffer chunk.MutRow + evalBufferTypes []*types.FieldType } type defaultVal struct { @@ -61,16 +65,18 @@ type defaultVal struct { valid bool } -// getColumns gets the explicitly specified columns of an insert statement. There are three cases: +// initInsertColumns sets the explicitly specified columns of an insert statement. There are three cases: // There are three types of insert statements: // 1 insert ... values(...) --> name type column // 2 insert ... set x=y... --> set type column // 3 insert ... (select ..) --> name type column // See https://dev.mysql.com/doc/refman/5.7/en/insert.html -func (e *InsertValues) getColumns(tableCols []*table.Column) ([]*table.Column, error) { +func (e *InsertValues) initInsertColumns() error { var cols []*table.Column var err error + tableCols := e.Table.Cols() + if len(e.SetList) > 0 { // Process `set` type column. columns := make([]string, 0, len(e.SetList)) @@ -82,10 +88,10 @@ func (e *InsertValues) getColumns(tableCols []*table.Column) ([]*table.Column, e } cols, err = table.FindCols(tableCols, columns, e.Table.Meta().PKIsHandle) if err != nil { - return nil, errors.Errorf("INSERT INTO %s: %s", e.Table.Meta().Name.O, err) + return errors.Errorf("INSERT INTO %s: %s", e.Table.Meta().Name.O, err) } if len(cols) == 0 { - return nil, errors.Errorf("INSERT INTO %s: empty column", e.Table.Meta().Name.O) + return errors.Errorf("INSERT INTO %s: empty column", e.Table.Meta().Name.O) } } else if len(e.Columns) > 0 { // Process `name` type column. @@ -98,7 +104,7 @@ func (e *InsertValues) getColumns(tableCols []*table.Column) ([]*table.Column, e } cols, err = table.FindCols(tableCols, columns, e.Table.Meta().PKIsHandle) if err != nil { - return nil, errors.Errorf("INSERT INTO %s: %s", e.Table.Meta().Name.O, err) + return errors.Errorf("INSERT INTO %s: %s", e.Table.Meta().Name.O, err) } } else { // If e.Columns are empty, use all columns instead. @@ -114,10 +120,25 @@ func (e *InsertValues) getColumns(tableCols []*table.Column) ([]*table.Column, e // Check column whether is specified only once. err = table.CheckOnce(cols) if err != nil { - return nil, errors.Trace(err) + return err } + e.insertColumns = cols + return nil +} - return cols, nil +func (e *InsertValues) initEvalBuffer() { + numCols := len(e.Table.Cols()) + if e.hasExtraHandle { + numCols++ + } + e.evalBufferTypes = make([]*types.FieldType, numCols) + for i, col := range e.Table.Cols() { + e.evalBufferTypes[i] = &col.FieldType + } + if e.hasExtraHandle { + e.evalBufferTypes[len(e.evalBufferTypes)-1] = types.NewFieldType(mysql.TypeLonglong) + } + e.evalBuffer = chunk.MutRowFromTypes(e.evalBufferTypes) } func (e *InsertValues) lazilyInitColDefaultValBuf() (ok bool) { @@ -150,7 +171,7 @@ func (e *InsertValues) processSetList() error { } // insertRows processes `insert|replace into values ()` or `insert|replace into set x=y` -func (e *InsertValues) insertRows(cols []*table.Column, exec func(rows [][]types.Datum) error) (err error) { +func (e *InsertValues) insertRows(exec func(rows [][]types.Datum) error) (err error) { // For `insert|replace into set x=y`, process the set list here. if err = e.processSetList(); err != nil { return errors.Trace(err) @@ -159,7 +180,7 @@ func (e *InsertValues) insertRows(cols []*table.Column, exec func(rows [][]types rows := make([][]types.Datum, 0, len(e.Lists)) for i, list := range e.Lists { e.rowCount++ - row, err := e.evalRow(cols, list, i) + row, err := e.evalRow(list, i) if err != nil { return errors.Trace(err) } @@ -189,7 +210,7 @@ func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int // evalRow evaluates a to-be-inserted row. The value of the column may base on another column, // so we use setValueForRefColumn to fill the empty row some default values when needFillDefaultValues is true. -func (e *InsertValues) evalRow(cols []*table.Column, list []expression.Expression, rowIdx int) ([]types.Datum, error) { +func (e *InsertValues) evalRow(list []expression.Expression, rowIdx int) ([]types.Datum, error) { rowLen := len(e.Table.Cols()) if e.hasExtraHandle { rowLen++ @@ -204,18 +225,20 @@ func (e *InsertValues) evalRow(cols []*table.Column, list []expression.Expressio } } + e.evalBuffer.SetDatums(row...) for i, expr := range list { - val, err := expr.Eval(chunk.MutRowFromDatums(row).ToRow()) - if err = e.handleErr(cols[i], &val, rowIdx, err); err != nil { + val, err := expr.Eval(e.evalBuffer.ToRow()) + if err = e.handleErr(e.insertColumns[i], &val, rowIdx, err); err != nil { return nil, errors.Trace(err) } - val1, err := table.CastValue(e.ctx, val, cols[i].ToInfo()) - if err = e.handleErr(cols[i], &val, rowIdx, err); err != nil { + val1, err := table.CastValue(e.ctx, val, e.insertColumns[i].ToInfo()) + if err = e.handleErr(e.insertColumns[i], &val, rowIdx, err); err != nil { return nil, errors.Trace(err) } - offset := cols[i].Offset - row[offset], hasValue[offset] = val1, true + offset := e.insertColumns[i].Offset + row[offset], hasValue[offset] = *val1.Copy(), true + e.evalBuffer.SetDatum(offset, val1) } return e.fillRow(row, hasValue) @@ -251,7 +274,7 @@ func (e *InsertValues) setValueForRefColumn(row []types.Datum, hasValue []bool) return nil } -func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.Column, exec func(rows [][]types.Datum) error) error { +func (e *InsertValues) insertRowsFromSelect(ctx context.Context, exec func(rows [][]types.Datum) error) error { // process `insert|replace into ... select ... from ...` selectExec := e.children[0] fields := selectExec.retTypes() @@ -275,7 +298,7 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C for innerChunkRow := iter.Begin(); innerChunkRow != iter.End(); innerChunkRow = iter.Next() { innerRow := types.CopyRow(innerChunkRow.GetDatumRow(fields)) e.rowCount++ - row, err := e.getRow(cols, innerRow) + row, err := e.getRow(innerRow) if err != nil { return errors.Trace(err) } @@ -305,16 +328,16 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C // getRow gets the row which from `insert into select from` or `load data`. // The input values from these two statements are datums instead of // expressions which are used in `insert into set x=y`. -func (e *InsertValues) getRow(cols []*table.Column, vals []types.Datum) ([]types.Datum, error) { +func (e *InsertValues) getRow(vals []types.Datum) ([]types.Datum, error) { row := make([]types.Datum, len(e.Table.Cols())) hasValue := make([]bool, len(e.Table.Cols())) for i, v := range vals { - casted, err := table.CastValue(e.ctx, v, cols[i].ToInfo()) + casted, err := table.CastValue(e.ctx, v, e.insertColumns[i].ToInfo()) if e.filterErr(err) != nil { return nil, errors.Trace(err) } - offset := cols[i].Offset + offset := e.insertColumns[i].Offset row[offset] = casted hasValue[offset] = true } diff --git a/executor/load_data.go b/executor/load_data.go index d6ee3724a3a74..83171cbbd8589 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -44,7 +44,6 @@ func NewLoadDataInfo(ctx sessionctx.Context, row []types.Datum, tbl table.Table, InsertValues: insertVal, Table: tbl, Ctx: ctx, - columns: cols, } } @@ -81,6 +80,9 @@ func (e *LoadDataExec) Close() error { // Open implements the Executor Open interface. func (e *LoadDataExec) Open(ctx context.Context) error { + if e.loadDataInfo.insertColumns != nil { + e.loadDataInfo.initEvalBuffer() + } return nil } @@ -95,7 +97,6 @@ type LoadDataInfo struct { LinesInfo *ast.LinesClause IgnoreLines uint64 Ctx sessionctx.Context - columns []*table.Column } // SetMaxRowsInBatch sets the max number of rows to insert in a batch. @@ -274,7 +275,7 @@ func (e *LoadDataInfo) colsToRow(cols []field) []types.Datum { e.row[i].SetString(string(cols[i].str)) } } - row, err := e.getRow(e.columns, e.row) + row, err := e.getRow(e.row) if err != nil { e.handleWarning(err, fmt.Sprintf("Load Data: insert data:%v failed:%v", e.row, errors.ErrorStack(err))) diff --git a/executor/replace.go b/executor/replace.go index dbe79cfde4817..bb06f2fec0455 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -42,6 +42,7 @@ func (e *ReplaceExec) Open(ctx context.Context) error { if e.SelectExec != nil { return e.SelectExec.Open(ctx) } + e.initEvalBuffer() return nil } @@ -178,13 +179,8 @@ func (e *ReplaceExec) exec(newRows [][]types.Datum) error { // Next implements the Executor Next interface. func (e *ReplaceExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - cols, err := e.getColumns(e.Table.Cols()) - if err != nil { - return errors.Trace(err) - } - if len(e.children) > 0 && e.children[0] != nil { - return errors.Trace(e.insertRowsFromSelect(ctx, cols, e.exec)) + return e.insertRowsFromSelect(ctx, e.exec) } - return errors.Trace(e.insertRows(cols, e.exec)) + return e.insertRows(e.exec) } diff --git a/executor/update.go b/executor/update.go index de60b92ab8a01..dc04ef5f505f3 100644 --- a/executor/update.go +++ b/executor/update.go @@ -42,6 +42,7 @@ type UpdateExec struct { // columns2Handle stores relationship between column ordinal to its table handle. // the columns ordinals is present in ordinal range format, @see executor.cols2Handle columns2Handle cols2HandleSlice + evalBuffer chunk.MutRow } func (e *UpdateExec) exec(schema *expression.Schema) ([]types.Datum, error) { @@ -141,6 +142,7 @@ func (e *UpdateExec) fetchChunkRows(ctx context.Context) error { fields := e.children[0].retTypes() globalRowIdx := 0 chk := e.children[0].newFirstChunk() + e.evalBuffer = chunk.MutRowFromTypes(fields) for { err := e.children[0].Next(ctx, chk) if err != nil { @@ -181,17 +183,19 @@ func (e *UpdateExec) handleErr(colName model.CIStr, rowIdx int, err error) error func (e *UpdateExec) composeNewRow(rowIdx int, oldRow []types.Datum) ([]types.Datum, error) { newRowData := types.CopyRow(oldRow) + e.evalBuffer.SetDatums(newRowData...) for _, assign := range e.OrderedList { handleIdx, handleFound := e.columns2Handle.findHandle(int32(assign.Col.Index)) if handleFound && e.canNotUpdate(oldRow[handleIdx]) { continue } - val, err := assign.Expr.Eval(chunk.MutRowFromDatums(newRowData).ToRow()) + val, err := assign.Expr.Eval(e.evalBuffer.ToRow()) if err1 := e.handleErr(assign.Col.ColName, rowIdx, err); err1 != nil { - return nil, errors.Trace(err1) + return nil, err1 } - newRowData[assign.Col.Index] = val + newRowData[assign.Col.Index] = *val.Copy() + e.evalBuffer.SetDatum(assign.Col.Index, val) } return newRowData, nil } From 7fb086f5df9eb6bb16c03a2f02ec00ad903296ff Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Thu, 25 Oct 2018 20:47:14 +0800 Subject: [PATCH 035/509] executor: print arguments in execute statement in log files (#7684) (#8053) --- ast/misc.go | 8 +++ ddl/db_change_test.go | 2 +- executor/adapter.go | 5 +- executor/executor.go | 110 +++++++++++++++++++++++++++++ executor/prepared.go | 122 ++++----------------------------- expression/builtin_other.go | 3 +- planner/core/common_plans.go | 20 ++---- session/session.go | 7 +- session/session_test.go | 4 +- sessionctx/variable/session.go | 30 ++++++-- 10 files changed, 172 insertions(+), 139 deletions(-) diff --git a/ast/misc.go b/ast/misc.go index 620a9abf34c12..c4493a24ccd66 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -184,6 +184,14 @@ func (n *DeallocateStmt) Accept(v Visitor) (Node, bool) { return v.Leave(n) } +// Prepared represents a prepared statement. +type Prepared struct { + Stmt StmtNode + Params []*ParamMarkerExpr + SchemaVersion int64 + UseCache bool +} + // ExecuteStmt is a statement to execute PreparedStmt. // See https://dev.mysql.com/doc/refman/5.7/en/execute.html type ExecuteStmt struct { diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index b8550c4bcdd83..79837b9765ce9 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -296,7 +296,7 @@ func (t *testExecInfo) compileSQL(idx int) (err error) { ctx := context.TODO() se.PrepareTxnCtx(ctx) sctx := se.(sessionctx.Context) - if err = executor.ResetStmtCtx(sctx, c.rawStmt); err != nil { + if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil { return errors.Trace(err) } c.stmt, err = compiler.Compile(ctx, c.rawStmt) diff --git a/executor/adapter.go b/executor/adapter.go index 3b9d4cad470f5..0f7c73d9a83ee 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -323,7 +323,6 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { if err != nil { return nil, errors.Trace(err) } - a.Text = executorExec.stmt.Text() a.isPreparedStmt = true a.Plan = executorExec.plan e = executorExec.stmtExec @@ -350,9 +349,9 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { if len(sql) > int(cfg.Log.QueryLogMaxLen) { sql = fmt.Sprintf("%.*q(len:%d)", cfg.Log.QueryLogMaxLen, sql, len(a.Text)) } - sql = QueryReplacer.Replace(sql) - sessVars := a.Ctx.GetSessionVars() + sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() + connID := sessVars.ConnectionID currentDB := sessVars.CurrentDB var tableIDs, indexIDs string diff --git a/executor/executor.go b/executor/executor.go index fe543c4d7eebf..2ea2381db9ec8 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -22,6 +22,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" @@ -30,6 +31,8 @@ import ( "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" @@ -37,6 +40,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/memory" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -1161,3 +1165,109 @@ func (e *UnionExec) Close() error { e.resourcePools = nil return errors.Trace(e.baseExecutor.Close()) } + +// ResetContextOfStmt resets the StmtContext and session variables. +// Before every execution, we must clear statement context. +func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { + vars := ctx.GetSessionVars() + sc := new(stmtctx.StatementContext) + sc.TimeZone = vars.Location() + sc.MemTracker = memory.NewTracker(s.Text(), vars.MemQuotaQuery) + switch config.GetGlobalConfig().OOMAction { + case config.OOMActionCancel: + sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{}) + case config.OOMActionLog: + sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{}) + default: + sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{}) + } + + if execStmt, ok := s.(*ast.ExecuteStmt); ok { + s, err = getPreparedStmt(execStmt, vars) + } + // TODO: Many same bool variables here. + // We should set only two variables ( + // IgnoreErr and StrictSQLMode) to avoid setting the same bool variables and + // pushing them down to TiKV as flags. + switch stmt := s.(type) { + case *ast.UpdateStmt: + sc.InUpdateOrDeleteStmt = true + sc.DupKeyAsWarning = stmt.IgnoreErr + sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr + sc.Priority = stmt.Priority + case *ast.DeleteStmt: + sc.InUpdateOrDeleteStmt = true + sc.DupKeyAsWarning = stmt.IgnoreErr + sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr + sc.Priority = stmt.Priority + case *ast.InsertStmt: + sc.InInsertStmt = true + sc.DupKeyAsWarning = stmt.IgnoreErr + sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr + sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr + sc.Priority = stmt.Priority + case *ast.CreateTableStmt, *ast.AlterTableStmt: + // Make sure the sql_mode is strict when checking column default value. + case *ast.LoadDataStmt: + sc.DupKeyAsWarning = true + sc.BadNullAsWarning = true + sc.TruncateAsWarning = !vars.StrictSQLMode + case *ast.SelectStmt: + sc.InSelectStmt = true + + // see https://dev.mysql.com/doc/refman/5.7/en/sql-mode.html#sql-mode-strict + // said "For statements such as SELECT that do not change data, invalid values + // generate a warning in strict mode, not an error." + // and https://dev.mysql.com/doc/refman/5.7/en/out-of-range-and-overflow.html + sc.OverflowAsWarning = true + + // Return warning for truncate error in selection. + sc.TruncateAsWarning = true + sc.IgnoreZeroInDate = true + if opts := stmt.SelectStmtOpts; opts != nil { + sc.Priority = opts.Priority + sc.NotFillCache = !opts.SQLCache + } + sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode() + case *ast.ShowStmt: + sc.IgnoreTruncate = true + sc.IgnoreZeroInDate = true + if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors { + sc.InShowWarning = true + sc.SetWarnings(vars.StmtCtx.GetWarnings()) + } + default: + sc.IgnoreTruncate = true + sc.IgnoreZeroInDate = true + } + vars.PreparedParams = vars.PreparedParams[:0] + if !vars.InRestrictedSQL { + if priority := mysql.PriorityEnum(atomic.LoadInt32(&variable.ForcePriority)); priority != mysql.NoPriority { + sc.Priority = priority + } + } + if vars.LastInsertID > 0 { + vars.PrevLastInsertID = vars.LastInsertID + vars.LastInsertID = 0 + } + vars.ResetPrevAffectedRows() + err = vars.SetSystemVar("warning_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(false))) + if err != nil { + return errors.Trace(err) + } + err = vars.SetSystemVar("error_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(true))) + if err != nil { + return errors.Trace(err) + } + vars.InsertID = 0 + vars.StmtCtx = sc + return +} diff --git a/executor/prepared.go b/executor/prepared.go index 2e629c7971661..962d35db6ae3c 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -16,22 +16,16 @@ package executor import ( "math" "sort" - "sync/atomic" - "fmt" "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" @@ -152,7 +146,7 @@ func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error { for i := 0; i < e.ParamCount; i++ { sorter.markers[i].Order = i } - prepared := &plannercore.Prepared{ + prepared := &ast.Prepared{ Stmt: stmt, Params: sorter.markers, SchemaVersion: e.is.SchemaMetaVersion(), @@ -219,9 +213,6 @@ func (e *ExecuteExec) Build() error { return errors.Trace(b.err) } e.stmtExec = stmtExec - if err = ResetStmtCtx(e.ctx, e.stmt); err != nil { - return err - } CountStmtNode(e.stmt, e.ctx.GetSessionVars().InRestrictedSQL) logExpensiveQuery(e.stmt, e.plan) return nil @@ -249,6 +240,9 @@ func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error { // CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement. func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (ast.Statement, error) { execStmt := &ast.ExecuteStmt{ExecID: ID} + if err := ResetContextOfStmt(ctx, execStmt); err != nil { + return nil, err + } execStmt.UsingVars = make([]ast.ExprNode, len(args)) for i, val := range args { execStmt.UsingVars[i] = ast.NewValueExpr(val) @@ -265,110 +259,22 @@ func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...inter StmtNode: execStmt, Ctx: ctx, } - if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID].(*plannercore.Prepared); ok { + if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID]; ok { stmt.Text = prepared.Stmt.Text() } return stmt, nil } -// ResetStmtCtx resets the StmtContext. -// Before every execution, we must clear statement context. -func ResetStmtCtx(ctx sessionctx.Context, s ast.StmtNode) (err error) { - sessVars := ctx.GetSessionVars() - sc := new(stmtctx.StatementContext) - sc.TimeZone = sessVars.Location() - sc.MemTracker = memory.NewTracker(s.Text(), sessVars.MemQuotaQuery) - switch config.GetGlobalConfig().OOMAction { - case config.OOMActionCancel: - sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{}) - case config.OOMActionLog: - sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{}) - default: - sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{}) - } - - // TODO: Many same bool variables here. - // We should set only two variables ( - // IgnoreErr and StrictSQLMode) to avoid setting the same bool variables and - // pushing them down to TiKV as flags. - switch stmt := s.(type) { - case *ast.UpdateStmt: - sc.InUpdateOrDeleteStmt = true - sc.DupKeyAsWarning = stmt.IgnoreErr - sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.Priority = stmt.Priority - case *ast.DeleteStmt: - sc.InUpdateOrDeleteStmt = true - sc.DupKeyAsWarning = stmt.IgnoreErr - sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.Priority = stmt.Priority - case *ast.InsertStmt: - sc.InInsertStmt = true - sc.DupKeyAsWarning = stmt.IgnoreErr - sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr - sc.Priority = stmt.Priority - case *ast.CreateTableStmt, *ast.AlterTableStmt: - // Make sure the sql_mode is strict when checking column default value. - case *ast.LoadDataStmt: - sc.DupKeyAsWarning = true - sc.BadNullAsWarning = true - sc.TruncateAsWarning = !sessVars.StrictSQLMode - case *ast.SelectStmt: - sc.InSelectStmt = true - - // see https://dev.mysql.com/doc/refman/5.7/en/sql-mode.html#sql-mode-strict - // said "For statements such as SELECT that do not change data, invalid values - // generate a warning in strict mode, not an error." - // and https://dev.mysql.com/doc/refman/5.7/en/out-of-range-and-overflow.html - sc.OverflowAsWarning = true - - // Return warning for truncate error in selection. - sc.TruncateAsWarning = true - sc.IgnoreZeroInDate = true - if opts := stmt.SelectStmtOpts; opts != nil { - sc.Priority = opts.Priority - sc.NotFillCache = !opts.SQLCache +func getPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.StmtNode, error) { + execID := stmt.ExecID + ok := false + if stmt.Name != "" { + if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok { + return nil, plannercore.ErrStmtNotFound } - sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode() - case *ast.ShowStmt: - sc.IgnoreTruncate = true - sc.IgnoreZeroInDate = true - if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors { - sc.InShowWarning = true - sc.SetWarnings(sessVars.StmtCtx.GetWarnings()) - } - default: - sc.IgnoreTruncate = true - sc.IgnoreZeroInDate = true } - if !sessVars.InRestrictedSQL { - if priority := mysql.PriorityEnum(atomic.LoadInt32(&variable.ForcePriority)); priority != mysql.NoPriority { - sc.Priority = priority - } - } - if sessVars.LastInsertID > 0 { - sessVars.PrevLastInsertID = sessVars.LastInsertID - sessVars.LastInsertID = 0 - } - sessVars.ResetPrevAffectedRows() - err = sessVars.SetSystemVar("warning_count", fmt.Sprintf("%d", sessVars.StmtCtx.NumWarnings(false))) - if err != nil { - return errors.Trace(err) - } - err = sessVars.SetSystemVar("error_count", fmt.Sprintf("%d", sessVars.StmtCtx.NumWarnings(true))) - if err != nil { - return errors.Trace(err) + if prepared, ok := vars.PreparedStmts[execID]; ok { + return prepared.Stmt, nil } - sessVars.InsertID = 0 - sessVars.StmtCtx = sc - return + return nil, plannercore.ErrStmtNotFound } diff --git a/expression/builtin_other.go b/expression/builtin_other.go index fa8f9130f1c7d..8afc8e9c0cce6 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -771,8 +771,7 @@ func (b *builtinGetParamStringSig) evalString(row chunk.Row) (string, bool, erro } v := sessionVars.PreparedParams[idx] - dt := v.(types.Datum) - str, err := (&dt).ToString() + str, err := v.ToString() if err != nil { return "", true, nil } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index d629d611cc70c..6a4db7fe3795b 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -126,14 +126,6 @@ type Prepare struct { SQLText string } -// Prepared represents a prepared statement. -type Prepared struct { - Stmt ast.StmtNode - Params []*ast.ParamMarkerExpr - SchemaVersion int64 - UseCache bool -} - // Execute represents prepare plan. type Execute struct { baseSchemaProducer @@ -150,26 +142,22 @@ func (e *Execute) optimizePreparedPlan(ctx sessionctx.Context, is infoschema.Inf if e.Name != "" { e.ExecID = vars.PreparedStmtNameToID[e.Name] } - v := vars.PreparedStmts[e.ExecID] - if v == nil { + prepared, ok := vars.PreparedStmts[e.ExecID] + if !ok { return errors.Trace(ErrStmtNotFound) } - prepared := v.(*Prepared) if len(prepared.Params) != len(e.UsingVars) { return errors.Trace(ErrWrongParamCount) } - if cap(vars.PreparedParams) < len(e.UsingVars) { - vars.PreparedParams = make([]interface{}, len(e.UsingVars)) - } for i, usingVar := range e.UsingVars { val, err := usingVar.Eval(chunk.Row{}) if err != nil { return errors.Trace(err) } prepared.Params[i].SetDatum(val) - vars.PreparedParams[i] = val + vars.PreparedParams = append(vars.PreparedParams, val) } if prepared.SchemaVersion != is.SchemaMetaVersion() { // If the schema version has changed we need to preprocess it again, @@ -189,7 +177,7 @@ func (e *Execute) optimizePreparedPlan(ctx sessionctx.Context, is infoschema.Inf return nil } -func (e *Execute) getPhysicalPlan(ctx sessionctx.Context, is infoschema.InfoSchema, prepared *Prepared) (Plan, error) { +func (e *Execute) getPhysicalPlan(ctx sessionctx.Context, is infoschema.InfoSchema, prepared *ast.Prepared) (Plan, error) { var cacheKey kvcache.Key sessionVars := ctx.GetSessionVars() sessionVars.StmtCtx.UseCache = prepared.UseCache diff --git a/session/session.go b/session/session.go index 20ab58c722808..953d024eaba07 100644 --- a/session/session.go +++ b/session/session.go @@ -54,7 +54,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/timeutil" - binlog "github.com/pingcap/tipb/go-binlog" + "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -796,7 +796,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.Rec // Step2: Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). startTS = time.Now() // Some executions are done in compile stage, so we reset them before compile. - if err := executor.ResetStmtCtx(s, stmtNode); err != nil { + if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { return nil, errors.Trace(err) } stmt, err := compiler.Compile(ctx, stmtNode) @@ -1431,6 +1431,7 @@ func logStmt(node ast.StmtNode, vars *variable.SessionVars) { func logQuery(query string, vars *variable.SessionVars) { if atomic.LoadUint32(&variable.ProcessGeneralLog) != 0 && !vars.InRestrictedSQL { query = executor.QueryReplacer.Replace(query) - log.Infof("[GENERAL_LOG] con:%d user:%s schema_ver:%d start_ts:%d sql:%s", vars.ConnectionID, vars.User, vars.TxnCtx.SchemaVersion, vars.TxnCtx.StartTS, query) + log.Infof("[GENERAL_LOG] con:%d user:%s schema_ver:%d start_ts:%d sql:%s%s", + vars.ConnectionID, vars.User, vars.TxnCtx.SchemaVersion, vars.TxnCtx.StartTS, query, vars.GetExecuteArgumentsInfo()) } } diff --git a/session/session_test.go b/session/session_test.go index f844a532a8706..30f0f7a745eec 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" - binlog "github.com/pingcap/tipb/go-binlog" + "github.com/pingcap/tipb/go-binlog" "golang.org/x/net/context" "google.golang.org/grpc" ) @@ -423,7 +423,7 @@ func (s *testSessionSuite) TestRetryCleanTxn(c *C) { stmtNode, err := parser.New().ParseOneStmt("insert retrytxn values (2, 'a')", "", "") c.Assert(err, IsNil) stmt, _ := session.Compile(context.TODO(), tk.Se, stmtNode) - executor.ResetStmtCtx(tk.Se, stmtNode) + executor.ResetContextOfStmt(tk.Se, stmtNode) history.Add(0, stmt, tk.Se.GetSessionVars().StmtCtx) _, err = tk.Exec("commit") c.Assert(err, NotNil) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 40afea3b9a4d6..26c0fbefb031d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -15,11 +15,13 @@ package variable import ( "crypto/tls" + "fmt" "strings" "sync" "sync/atomic" "time" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" @@ -164,12 +166,12 @@ type SessionVars struct { // systems variables, don't modify it directly, use GetSystemVar/SetSystemVar method. systems map[string]string // PreparedStmts stores prepared statement. - PreparedStmts map[uint32]interface{} + PreparedStmts map[uint32]*ast.Prepared PreparedStmtNameToID map[string]uint32 // preparedStmtID is id of prepared statement. preparedStmtID uint32 // params for prepared statements - PreparedParams []interface{} + PreparedParams []types.Datum // retry information RetryInfo *RetryInfo @@ -301,9 +303,9 @@ func NewSessionVars() *SessionVars { vars := &SessionVars{ Users: make(map[string]string), systems: make(map[string]string), - PreparedStmts: make(map[uint32]interface{}), + PreparedStmts: make(map[uint32]*ast.Prepared), PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]interface{}, 10), + PreparedParams: make([]types.Datum, 0, 10), TxnCtx: &TransactionContext{}, KVVars: kv.NewVariables(), RetryInfo: &RetryInfo{}, @@ -444,6 +446,26 @@ func (s *SessionVars) ResetPrevAffectedRows() { } } +// GetExecuteArgumentsInfo gets the argument list as a string of execute statement. +func (s *SessionVars) GetExecuteArgumentsInfo() string { + if len(s.PreparedParams) == 0 { + return "" + } + args := make([]string, 0, len(s.PreparedParams)) + for _, v := range s.PreparedParams { + if v.IsNull() { + args = append(args, "") + } else { + str, err := v.ToString() + if err != nil { + terror.Log(err) + } + args = append(args, str) + } + } + return fmt.Sprintf(" [arguments: %s]", strings.Join(args, ", ")) +} + // GetSystemVar gets the string value of a system variable. func (s *SessionVars) GetSystemVar(name string) (string, bool) { val, ok := s.systems[name] From 1b8102a733ce6572b6821211b890da8f995a64db Mon Sep 17 00:00:00 2001 From: Jun-Seok Heo Date: Thu, 25 Oct 2018 22:12:24 +0900 Subject: [PATCH 036/509] planner: fix a panic of a prepared statement with IndexScan when using the prepared plan cache (#8055) --- planner/core/find_best_task.go | 6 +++++- planner/core/prepare_test.go | 26 ++++++++++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index f84a6578bf98d..915edff781b3f 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -424,7 +424,11 @@ func (is *PhysicalIndexScan) initSchema(id int, idx *model.IndexInfo, isDoubleRe for _, col := range idx.Columns { colFound := is.dataSourceSchema.FindColumnByName(col.Name.L) if colFound == nil { - colFound = &expression.Column{ColName: col.Name, UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID()} + colFound = &expression.Column{ + ColName: col.Name, + RetType: &is.Table.Columns[col.Offset].FieldType, + UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID(), + } } else { colFound = colFound.Clone().(*expression.Column) } diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 191780284504a..1a456fbc679b3 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -63,3 +63,29 @@ func (s *testPrepareSuite) TestPrepareCache(c *C) { tk.MustQuery("execute stmt5").Check(testkit.Rows("1", "2", "2", "3", "4", "5")) tk.MustQuery("execute stmt5").Check(testkit.Rows("1", "2", "2", "3", "4", "5")) } + +func (s *testPrepareSuite) TestPrepareCacheIndexScan(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + orgEnable := core.PreparedPlanCacheEnabled() + orgCapacity := core.PreparedPlanCacheCapacity + defer func() { + dom.Close() + store.Close() + core.SetPreparedPlanCache(orgEnable) + core.PreparedPlanCacheCapacity = orgCapacity + }() + core.SetPreparedPlanCache(true) + core.PreparedPlanCacheCapacity = 100 + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, primary key (a, b))") + tk.MustExec("insert into t values(1, 1, 2), (1, 2, 3), (1, 3, 3), (2, 1, 2), (2, 2, 3), (2, 3, 3)") + tk.MustExec(`prepare stmt1 from "select a, c from t where a = ? and c = ?"`) + tk.MustExec("set @a=1, @b=3") + // When executing one statement at the first time, we don't use cache, so we need to execute it at least twice to test the cache. + tk.MustQuery("execute stmt1 using @a, @b").Check(testkit.Rows("1 3", "1 3")) + tk.MustQuery("execute stmt1 using @a, @b").Check(testkit.Rows("1 3", "1 3")) +} From 5707a9b320c2b4b3d724a56920aad69228f332d1 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 26 Oct 2018 12:29:37 +0800 Subject: [PATCH 037/509] server: add log for binary execute statement (#7987) (#8063) --- server/conn_stmt.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 4bb163d5e5076..2e62b7803548b 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -174,12 +174,12 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e err = parseStmtArgs(args, stmt.BoundParams(), nullBitmaps, stmt.GetParamsType(), paramValues) if err != nil { - return errors.Trace(err) + return errors.Annotatef(err, "%s", cc.preparedStmt2String(stmtID)) } } rs, err := stmt.Execute(ctx, args...) if err != nil { - return errors.Trace(err) + return errors.Annotatef(err, "%s", cc.preparedStmt2String(stmtID)) } if rs == nil { return errors.Trace(cc.writeOK()) @@ -558,3 +558,11 @@ func (cc *clientConn) handleSetOption(data []byte) (err error) { return errors.Trace(cc.flush()) } + +func (cc *clientConn) preparedStmt2String(stmtID uint32) string { + sv := cc.ctx.GetSessionVars() + if prepared, ok := sv.PreparedStmts[stmtID]; ok { + return prepared.Stmt.Text() + sv.GetExecuteArgumentsInfo() + } + return fmt.Sprintf("prepared statement not found, ID: %d", stmtID) +} From 75192d7d64ad8f112f436fce14bc64d512bec613 Mon Sep 17 00:00:00 2001 From: yu34po Date: Fri, 26 Oct 2018 16:21:46 +0800 Subject: [PATCH 038/509] expression: refine built-in func truncate to support uint arg (#8000) (#8068) --- expression/builtin_math.go | 44 ++++++++++++++++++++++++++++++--- expression/builtin_math_test.go | 3 +++ expression/integration_test.go | 2 ++ 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 04c59608d3bbd..68ee63a974114 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -110,6 +110,7 @@ var ( _ builtinFunc = &builtinTruncateIntSig{} _ builtinFunc = &builtinTruncateRealSig{} _ builtinFunc = &builtinTruncateDecimalSig{} + _ builtinFunc = &builtinTruncateUintSig{} ) type absFunctionClass struct { @@ -1737,7 +1738,11 @@ func (c *truncateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre var sig builtinFunc switch argTp { case types.ETInt: - sig = &builtinTruncateIntSig{bf} + if mysql.HasUnsignedFlag(args[0].GetType().Flag) { + sig = &builtinTruncateUintSig{bf} + } else { + sig = &builtinTruncateIntSig{bf} + } case types.ETReal: sig = &builtinTruncateRealSig{bf} case types.ETDecimal: @@ -1826,6 +1831,39 @@ func (b *builtinTruncateIntSig) evalInt(row chunk.Row) (int64, bool, error) { return 0, isNull, errors.Trace(err) } - floatX := float64(x) - return int64(types.Truncate(floatX, int(d))), false, nil + if d >= 0 { + return x, false, nil + } + shift := int64(math.Pow10(int(-d))) + return x / shift * shift, false, nil +} + +func (b *builtinTruncateUintSig) Clone() builtinFunc { + newSig := &builtinTruncateUintSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +type builtinTruncateUintSig struct { + baseBuiltinFunc +} + +// evalInt evals a TRUNCATE(X,D). +// See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_truncate +func (b *builtinTruncateUintSig) evalInt(row chunk.Row) (int64, bool, error) { + x, isNull, err := b.args[0].EvalInt(b.ctx, row) + if isNull || err != nil { + return 0, isNull, errors.Trace(err) + } + uintx := uint64(x) + + d, isNull, err := b.args[1].EvalInt(b.ctx, row) + if isNull || err != nil { + return 0, isNull, errors.Trace(err) + } + if d >= 0 { + return x, false, nil + } + shift := uint64(math.Pow10(int(-d))) + return int64(uintx / shift * shift), false, nil } diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index a0c291603a5c3..db01a7864dfe5 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -485,6 +485,9 @@ func (s *testEvaluatorSuite) TestTruncate(c *C) { {[]interface{}{newDec("23.298"), -100}, newDec("0")}, {[]interface{}{newDec("23.298"), 100}, newDec("23.298")}, {[]interface{}{nil, 2}, nil}, + {[]interface{}{uint64(9223372036854775808), -10}, 9223372030000000000}, + {[]interface{}{9223372036854775807, -7}, 9223372036850000000}, + {[]interface{}{uint64(18446744073709551615), -10}, uint64(18446744070000000000)}, } Dtbl := tblToDtbl(tbl) diff --git a/expression/integration_test.go b/expression/integration_test.go index 2fc1dca204dbf..90d9a213506d9 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -477,6 +477,8 @@ func (s *testIntegrationSuite) TestMathBuiltin(c *C) { result.Check(testkit.Rows("100 123 123 120")) result = tk.MustQuery("SELECT truncate(123.456, -2), truncate(123.456, 2), truncate(123.456, 1), truncate(123.456, 3), truncate(1.23, 100), truncate(123456E-3, 2);") result.Check(testkit.Rows("100 123.45 123.4 123.456 1.230000000000000000000000000000 123.45")) + result = tk.MustQuery("SELECT truncate(9223372036854775807, -7), truncate(9223372036854775808, -10), truncate(cast(-1 as unsigned), -10);") + result.Check(testkit.Rows("9223372036850000000 9223372030000000000 18446744070000000000")) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a date, b datetime, c timestamp, d varchar(20));`) From 423e9b6220b0b51e496e3b75436842b7dbcb0b50 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 29 Oct 2018 19:06:35 +0800 Subject: [PATCH 039/509] =?UTF-8?q?*:=20fix=20the=20issue=20of=20executing?= =?UTF-8?q?=20DDL=20after=20executing=20SQL=20failure=20in=20txn=20?= =?UTF-8?q?=E2=80=A6=20(#8056)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/ddl.go | 5 ----- ddl/foreign_key_test.go | 4 +++- executor/ddl.go | 5 +++++ executor/ddl_test.go | 17 +++++++++++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index ce00d89adc829..768a2f3be3f7e 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -476,11 +476,6 @@ func (d *ddl) asyncNotifyWorker(jobTp model.ActionType) { } func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { - // For every DDL, we must commit current transaction. - if err := ctx.NewTxn(); err != nil { - return errors.Trace(err) - } - // Get a global job ID and put the DDL job in the queue. err := d.addDDLJob(ctx, job) if err != nil { diff --git a/ddl/foreign_key_test.go b/ddl/foreign_key_test.go index 7fdde31834477..75f5700a6b614 100644 --- a/ddl/foreign_key_test.go +++ b/ddl/foreign_key_test.go @@ -78,7 +78,9 @@ func (s *testForeighKeySuite) testCreateForeignKey(c *C, tblInfo *model.TableInf BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{fkInfo}, } - err := s.d.doDDLJob(s.ctx, job) + err := s.ctx.NewTxn() + c.Assert(err, IsNil) + err = s.d.doDDLJob(s.ctx, job) c.Assert(err, IsNil) return job } diff --git a/executor/ddl.go b/executor/ddl.go index d972c69a7b18e..34b4744385083 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -67,6 +67,11 @@ func (e *DDLExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { return nil } e.done = true + + // For each DDL, we should commit the previous transaction and create a new transaction. + if err = e.ctx.NewTxn(); err != nil { + return errors.Trace(err) + } defer func() { e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false }() switch x := e.stmt.(type) { diff --git a/executor/ddl_test.go b/executor/ddl_test.go index b91a57b4b1559..45884d1c01e64 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -46,6 +46,23 @@ func (s *testSuite) TestTruncateTable(c *C) { result.Check(nil) } +// TestInTxnExecDDLFail tests the following case: +// 1. Execute the SQL of "begin"; +// 2. A SQL that will fail to execute; +// 3. Execute DDL. +func (s *testSuite) TestInTxnExecDDLFail(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t (i int key);") + tk.MustExec("insert into t values (1);") + tk.MustExec("begin;") + tk.MustExec("insert into t values (1);") + _, err := tk.Exec("truncate table t;") + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'PRIMARY'") + result := tk.MustQuery("select count(*) from t") + result.Check(testkit.Rows("1")) +} + func (s *testSuite) TestCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From ebebc2a1730a45088a6b3a98a0a615a335894802 Mon Sep 17 00:00:00 2001 From: WangXiangUSTC Date: Mon, 29 Oct 2018 19:39:14 +0800 Subject: [PATCH 040/509] update pkg/errors && add pump client in vendor (#8093) --- Gopkg.lock | 19 +- Gopkg.toml | 6 +- vendor/github.com/pingcap/tidb-tools/LICENSE | 201 ++++++++ .../pingcap/tidb-tools/pkg/etcd/etcd.go | 236 +++++++++ .../pingcap/tidb-tools/pkg/utils/cpu.go | 37 ++ .../pingcap/tidb-tools/pkg/utils/errors.go | 30 ++ .../pingcap/tidb-tools/pkg/utils/printer.go | 36 ++ .../pingcap/tidb-tools/pkg/utils/security.go | 57 +++ .../pingcap/tidb-tools/pkg/utils/tso.go | 24 + .../pingcap/tidb-tools/pkg/utils/urls.go | 55 +++ .../tidb-tools/tidb-binlog/node/node.go | 83 ++++ .../tidb-tools/tidb-binlog/node/registry.go | 187 +++++++ .../tidb-binlog/pump_client/client.go | 455 ++++++++++++++++++ .../tidb-binlog/pump_client/pump.go | 120 +++++ .../tidb-binlog/pump_client/selector.go | 274 +++++++++++ vendor/github.com/pkg/errors/errors.go | 33 +- vendor/github.com/pkg/errors/group.go | 9 + vendor/github.com/pkg/errors/juju_adaptor.go | 28 +- vendor/github.com/pkg/errors/stack.go | 84 +++- 19 files changed, 1931 insertions(+), 43 deletions(-) create mode 100644 vendor/github.com/pingcap/tidb-tools/LICENSE create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/etcd/etcd.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/cpu.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/errors.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/printer.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/security.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/tso.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/urls.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/node.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/registry.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/pump.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/selector.go diff --git a/Gopkg.lock b/Gopkg.lock index 9f90a5ba400d1..0b1a9469d6c3e 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -297,6 +297,18 @@ pruneopts = "NUT" revision = "eb892dda1e33a0b76191d39894ad4a806f313f6e" +[[projects]] + digest = "1:ef1a3a4694f068c0f914a809fbfa48e0a44d50dc1154d72e85b15be01facda23" + name = "github.com/pingcap/tidb-tools" + packages = [ + "pkg/etcd", + "pkg/utils", + "tidb-binlog/node", + "tidb-binlog/pump_client", + ] + pruneopts = "NUT" + revision = "5db58e3b7e6613456551c40d011806a346b2f44a" + [[projects]] branch = "master" digest = "1:14d83225e335cfa449b1f74d03c03ed70db64bf6af83a6e04c43e6e0021d6b94" @@ -310,13 +322,13 @@ revision = "371b48b15d93924a3a5375019e42b4420bc13d17" [[projects]] - digest = "1:91fee14a873676b2118c54e08c2e04897ea5c141b55a9b9fb8cf0721458d0d85" + digest = "1:4b96ce0f151b10c52b1d6876c179e9a6d2329f98939c4cf8148daaf27f981413" name = "github.com/pkg/errors" packages = ["."] pruneopts = "NUT" - revision = "9316aeb006f59424c65ff505c217f90c43d6445d" + revision = "1176802fff62540cc87d289bd40c52a2d6b2ea16" source = "https://github.com/pingcap/errors.git" - version = "v0.9.0" + version = "v0.11.0" [[projects]] digest = "1:1ef3c4d6e78616bd3d1b5b7d8899febb9aa1b83d3373fbbdc2804408c7977b57" @@ -549,6 +561,7 @@ "github.com/pingcap/kvproto/pkg/metapb", "github.com/pingcap/kvproto/pkg/tikvpb", "github.com/pingcap/pd/client", + "github.com/pingcap/tidb-tools/tidb-binlog/pump_client", "github.com/pingcap/tipb/go-binlog", "github.com/pingcap/tipb/go-tipb", "github.com/pkg/errors", diff --git a/Gopkg.toml b/Gopkg.toml index cd91b8d52d6ec..3d9ff447b90dc 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -99,6 +99,10 @@ required = ["github.com/golang/protobuf/jsonpb"] [[constraint]] name = "github.com/pkg/errors" - version = "0.9.0" + version = "0.11.0" source = "https://github.com/pingcap/errors.git" + +[[constraint]] + name = "github.com/pingcap/tidb-tools" + revision = "5db58e3b7e6613456551c40d011806a346b2f44a" diff --git a/vendor/github.com/pingcap/tidb-tools/LICENSE b/vendor/github.com/pingcap/tidb-tools/LICENSE new file mode 100644 index 0000000000000..b67d9091009d1 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/LICENSE @@ -0,0 +1,201 @@ +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {} + + 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. diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/etcd/etcd.go b/vendor/github.com/pingcap/tidb-tools/pkg/etcd/etcd.go new file mode 100644 index 0000000000000..9919ad7da3069 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/etcd/etcd.go @@ -0,0 +1,236 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package etcd + +import ( + "crypto/tls" + "path" + "strings" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/pkg/errors" + "golang.org/x/net/context" +) + +// Node organizes the ectd query result as a Trie tree +type Node struct { + Value []byte + Childs map[string]*Node +} + +// Client is a wrapped etcd client that support some simple method +type Client struct { + client *clientv3.Client + rootPath string +} + +// NewClient returns a wrapped etcd client +func NewClient(cli *clientv3.Client, root string) *Client { + return &Client{ + client: cli, + rootPath: root, + } +} + +// NewClientFromCfg returns a wrapped etcd client +func NewClientFromCfg(endpoints []string, dialTimeout time.Duration, root string, security *tls.Config) (*Client, error) { + cli, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + DialTimeout: dialTimeout, + TLS: security, + }) + if err != nil { + return nil, errors.Trace(err) + } + + return &Client{ + client: cli, + rootPath: root, + }, nil +} + +// Close shutdowns the connection to etcd +func (e *Client) Close() error { + if err := e.client.Close(); err != nil { + return errors.Trace(err) + } + return nil +} + +// Create guarantees to set a key = value with some options(like ttl) +func (e *Client) Create(ctx context.Context, key string, val string, opts []clientv3.OpOption) error { + key = keyWithPrefix(e.rootPath, key) + txnResp, err := e.client.KV.Txn(ctx).If( + clientv3.Compare(clientv3.ModRevision(key), "=", 0), + ).Then( + clientv3.OpPut(key, val, opts...), + ).Commit() + if err != nil { + return errors.Trace(err) + } + + if !txnResp.Succeeded { + return errors.AlreadyExistsf("key %s in etcd", key) + } + + return nil +} + +// Get returns a key/value matchs the given key +func (e *Client) Get(ctx context.Context, key string) ([]byte, error) { + key = keyWithPrefix(e.rootPath, key) + resp, err := e.client.KV.Get(ctx, key) + if err != nil { + return nil, errors.Trace(err) + } + + if len(resp.Kvs) == 0 { + return nil, errors.NotFoundf("key %s in etcd", key) + } + + return resp.Kvs[0].Value, nil +} + +// Update updates a key/value. +// set ttl 0 to disable the Lease ttl feature +func (e *Client) Update(ctx context.Context, key string, val string, ttl int64) error { + key = keyWithPrefix(e.rootPath, key) + + var opts []clientv3.OpOption + if ttl > 0 { + lcr, err := e.client.Lease.Grant(ctx, ttl) + if err != nil { + return errors.Trace(err) + } + + opts = []clientv3.OpOption{clientv3.WithLease(lcr.ID)} + } + + txnResp, err := e.client.KV.Txn(ctx).If( + clientv3.Compare(clientv3.ModRevision(key), ">", 0), + ).Then( + clientv3.OpPut(key, val, opts...), + ).Commit() + if err != nil { + return errors.Trace(err) + } + + if !txnResp.Succeeded { + return errors.NotFoundf("key %s in etcd", key) + } + + return nil +} + +// UpdateOrCreate updates a key/value, if the key does not exist then create, or update +func (e *Client) UpdateOrCreate(ctx context.Context, key string, val string, ttl int64) error { + key = keyWithPrefix(e.rootPath, key) + + var opts []clientv3.OpOption + if ttl > 0 { + lcr, err := e.client.Lease.Grant(ctx, ttl) + if err != nil { + return errors.Trace(err) + } + + opts = []clientv3.OpOption{clientv3.WithLease(lcr.ID)} + } + + _, err := e.client.KV.Do(ctx, clientv3.OpPut(key, val, opts...)) + if err != nil { + return errors.Trace(err) + } + return nil +} + +// List returns the trie struct that constructed by the key/value with same prefix +func (e *Client) List(ctx context.Context, key string) (*Node, error) { + key = keyWithPrefix(e.rootPath, key) + if !strings.HasSuffix(key, "/") { + key += "/" + } + + resp, err := e.client.KV.Get(ctx, key, clientv3.WithPrefix()) + if err != nil { + return nil, errors.Trace(err) + } + + root := new(Node) + length := len(key) + for _, kv := range resp.Kvs { + key := string(kv.Key) + if len(key) <= length { + continue + } + + keyTail := key[length:] + tailNode := parseToDirTree(root, keyTail) + tailNode.Value = kv.Value + } + + return root, nil +} + +// Delete deletes the key/values with matching prefix or key +func (e *Client) Delete(ctx context.Context, key string, withPrefix bool) error { + key = keyWithPrefix(e.rootPath, key) + var opts []clientv3.OpOption + if withPrefix { + opts = []clientv3.OpOption{clientv3.WithPrefix()} + } + + _, err := e.client.KV.Delete(ctx, key, opts...) + if err != nil { + return errors.Trace(err) + } + + return nil +} + +// Watch watchs the events of key with prefix. +func (e *Client) Watch(ctx context.Context, prefix string) clientv3.WatchChan { + return e.client.Watch(ctx, prefix, clientv3.WithPrefix()) +} + +func parseToDirTree(root *Node, path string) *Node { + pathDirs := strings.Split(path, "/") + current := root + var next *Node + var ok bool + + for _, dir := range pathDirs { + if current.Childs == nil { + current.Childs = make(map[string]*Node) + } + + next, ok = current.Childs[dir] + if !ok { + current.Childs[dir] = new(Node) + next = current.Childs[dir] + } + + current = next + } + + return current +} + +func keyWithPrefix(prefix, key string) string { + if strings.HasPrefix(key, prefix) { + return key + } + + return path.Join(prefix, key) +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/cpu.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/cpu.go new file mode 100644 index 0000000000000..71e2a9d235ed0 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/cpu.go @@ -0,0 +1,37 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "syscall" + "time" +) + +var ( + lastInspectUnixNano int64 + lastCPUUsageTime int64 +) + +// GetCPUPercentage calculates CPU usage and returns percentage in float64(e.g. 2.5 means 2.5%). +// http://man7.org/linux/man-pages/man2/getrusage.2.html +func GetCPUPercentage() float64 { + var ru syscall.Rusage + syscall.Getrusage(syscall.RUSAGE_SELF, &ru) + usageTime := ru.Utime.Nano() + ru.Stime.Nano() + nowTime := time.Now().UnixNano() + perc := float64(usageTime-lastCPUUsageTime) / float64(nowTime-lastInspectUnixNano) * 100.0 + lastInspectUnixNano = nowTime + lastCPUUsageTime = usageTime + return perc +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/errors.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/errors.go new file mode 100644 index 0000000000000..cc55919be7620 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/errors.go @@ -0,0 +1,30 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "github.com/pkg/errors" +) + +//OriginError return original err +func OriginError(err error) error { + for { + e := errors.Cause(err) + if e == err { + break + } + err = e + } + return err +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/printer.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/printer.go new file mode 100644 index 0000000000000..94bac09a0a412 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/printer.go @@ -0,0 +1,36 @@ +// Copyright 2017 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "fmt" + "runtime" +) + +// Version information. +var ( + Version = "None" + BuildTS = "None" + GitHash = "None" +) + +// GetRawInfo do what its name tells +func GetRawInfo(app string) string { + info := "" + info += fmt.Sprintf("%s: v%s\n", app, Version) + info += fmt.Sprintf("Git Commit Hash: %s\n", GitHash) + info += fmt.Sprintf("UTC Build Time: %s\n", BuildTS) + info += fmt.Sprintf("Go Version: %s\n", runtime.Version()) + return info +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/security.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/security.go new file mode 100644 index 0000000000000..a330bf0de08e7 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/security.go @@ -0,0 +1,57 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "crypto/tls" + "crypto/x509" + "io/ioutil" + + "github.com/pkg/errors" +) + +// ToTLSConfig generates tls's config. +func ToTLSConfig(SSLCA, SSLCert, SSLKey string) (*tls.Config, error) { + var tlsConfig *tls.Config + if len(SSLCA) != 0 { + var certificates = make([]tls.Certificate, 0) + if len(SSLCert) != 0 && len(SSLKey) != 0 { + // Load the client certificates from disk + certificate, err := tls.LoadX509KeyPair(SSLCert, SSLKey) + if err != nil { + return nil, errors.Errorf("could not load client key pair: %s", err) + } + certificates = append(certificates, certificate) + } + + // Create a certificate pool from the certificate authority + certPool := x509.NewCertPool() + ca, err := ioutil.ReadFile(SSLCA) + if err != nil { + return nil, errors.Errorf("could not read ca certificate: %s", err) + } + + // Append the certificates from the CA + if !certPool.AppendCertsFromPEM(ca) { + return nil, errors.New("failed to append ca certs") + } + + tlsConfig = &tls.Config{ + Certificates: certificates, + RootCAs: certPool, + } + } + + return tlsConfig, nil +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/tso.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/tso.go new file mode 100644 index 0000000000000..221a896c3229c --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/tso.go @@ -0,0 +1,24 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "time" +) + +// TSOToRoughTime translates tso to rough time that used to display +func TSOToRoughTime(ts int64) time.Time { + t := time.Unix(ts>>18/1000, 0) + return t +} diff --git a/vendor/github.com/pingcap/tidb-tools/pkg/utils/urls.go b/vendor/github.com/pingcap/tidb-tools/pkg/utils/urls.go new file mode 100644 index 0000000000000..34651e68d0f19 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/pkg/utils/urls.go @@ -0,0 +1,55 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "net" + "net/url" + "strings" + + "github.com/pkg/errors" +) + +// ParseHostPortAddr returns a scheme://host:port or host:port list +func ParseHostPortAddr(s string) ([]string, error) { + strs := strings.Split(s, ",") + addrs := make([]string, 0, len(strs)) + + for _, str := range strs { + str = strings.TrimSpace(str) + + // str may looks like 127.0.0.1:8000 + if _, _, err := net.SplitHostPort(str); err == nil { + addrs = append(addrs, str) + continue + } + + u, err := url.Parse(str) + if err != nil { + return nil, errors.Errorf("parse url %s failed %v", str, err) + } + if u.Scheme != "http" && u.Scheme != "https" && u.Scheme != "unix" && u.Scheme != "unixs" { + return nil, errors.Errorf("URL scheme must be http, https, unix, or unixs: %s", str) + } + if _, _, err := net.SplitHostPort(u.Host); err != nil { + return nil, errors.Errorf(`URL address does not have the form "host:port": %s`, str) + } + if u.Path != "" { + return nil, errors.Errorf("URL must not contain a path: %s", str) + } + addrs = append(addrs, u.String()) + } + + return addrs, nil +} diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/node.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/node.go new file mode 100644 index 0000000000000..d1716842a6337 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/node.go @@ -0,0 +1,83 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package node + +var ( + // DefaultRootPath is the root path of the keys stored in etcd, the `v1` is the tidb-binlog's version. + DefaultRootPath = "/tidb-binlog/v1" + + // PumpNode is the name of pump. + PumpNode = "pump" + + // DrainerNode is the name of drainer. + DrainerNode = "drainer" + + // NodePrefix is the map (node => it's prefix in storage) + NodePrefix = map[string]string{ + PumpNode: "pumps", + DrainerNode: "drainers", + } +) + +const ( + // Online means the node can receive request. + Online = "online" + + // Pausing means the node is pausing. + Pausing = "pausing" + + // Paused means the node is already paused. + Paused = "paused" + + // Closing means the node is closing, and the state will be Offline when closed. + Closing = "closing" + + // Offline means the node is offline, and will not provide service. + Offline = "offline" +) + +// Label is key/value pairs that are attached to objects +type Label struct { + Labels map[string]string `json:"labels"` +} + +// Status describes the status information of a tidb-binlog node in etcd. +type Status struct { + // the id of node. + NodeID string `json:"nodeId"` + + // the host of pump or node. + Addr string `json:"host"` + + // the state of pump. + State string `json:"state"` + + // the node is alive or not. + IsAlive bool `json:"isAlive"` + + // the score of node, it is report by node, calculated by node's qps, disk usage and binlog's data size. + // if Score is less than 0, means this node is useless. Now only used for pump. + Score int64 `json:"score"` + + // the label of this node. Now only used for pump. + // pump client will only send to a pump which label is matched. + Label *Label `json:"label"` + + // for pump: max commit ts in pump + // for drainer: drainer has consume all binlog less than or equal MaxCommitTS + MaxCommitTS int64 `json:"maxCommitTS"` + + // UpdateTS is the last update ts of node's status. + UpdateTS int64 `json:"updateTS"` +} diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/registry.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/registry.go new file mode 100644 index 0000000000000..c1dc08794c96b --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/node/registry.go @@ -0,0 +1,187 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package node + +import ( + "encoding/json" + "path" + "strings" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/pingcap/tidb-tools/pkg/etcd" + "github.com/pkg/errors" + log "github.com/sirupsen/logrus" + "golang.org/x/net/context" +) + +// EtcdRegistry wraps the reactions with etcd +type EtcdRegistry struct { + client *etcd.Client + reqTimeout time.Duration +} + +// NewEtcdRegistry returns an EtcdRegistry client +func NewEtcdRegistry(cli *etcd.Client, reqTimeout time.Duration) *EtcdRegistry { + return &EtcdRegistry{ + client: cli, + reqTimeout: reqTimeout, + } +} + +// Close closes the etcd client +func (r *EtcdRegistry) Close() error { + err := r.client.Close() + return errors.Trace(err) +} + +func (r *EtcdRegistry) prefixed(p ...string) string { + return path.Join(p...) +} + +// Node returns the nodeStatus that matchs nodeID in the etcd +func (r *EtcdRegistry) Node(pctx context.Context, prefix, nodeID string) (*Status, error) { + ctx, cancel := context.WithTimeout(pctx, r.reqTimeout) + defer cancel() + + data, err := r.client.Get(ctx, r.prefixed(prefix, nodeID)) + if err != nil { + return nil, errors.Trace(err) + } + + status := &Status{} + if err = json.Unmarshal(data, &status); err != nil { + return nil, errors.Annotatef(err, "Invalid nodeID(%s)", nodeID) + } + return status, nil +} + +// Nodes retruns all the nodeStatuses in the etcd +func (r *EtcdRegistry) Nodes(pctx context.Context, prefix string) ([]*Status, error) { + ctx, cancel := context.WithTimeout(pctx, r.reqTimeout) + defer cancel() + + resp, err := r.client.List(ctx, r.prefixed(prefix)) + if err != nil { + return nil, errors.Trace(err) + } + status, err := NodesStatusFromEtcdNode(resp) + if err != nil { + return nil, errors.Trace(err) + } + return status, nil +} + +// UpdateNode update the node information. +func (r *EtcdRegistry) UpdateNode(pctx context.Context, prefix string, status *Status) error { + ctx, cancel := context.WithTimeout(pctx, r.reqTimeout) + defer cancel() + + if exists, err := r.checkNodeExists(ctx, prefix, status.NodeID); err != nil { + return errors.Trace(err) + } else if !exists { + // not found then create a new node + log.Infof("node %s dosen't exist, will create one", status.NodeID) + return r.createNode(ctx, prefix, status) + } else { + // found it, update status infomation of the node + return r.updateNode(ctx, prefix, status) + } +} + +func (r *EtcdRegistry) checkNodeExists(ctx context.Context, prefix, nodeID string) (bool, error) { + _, err := r.client.Get(ctx, r.prefixed(prefix, nodeID)) + if err != nil { + if errors.IsNotFound(err) { + return false, nil + } + return false, errors.Trace(err) + } + return true, nil +} + +func (r *EtcdRegistry) updateNode(ctx context.Context, prefix string, status *Status) error { + objstr, err := json.Marshal(status) + if err != nil { + return errors.Annotatef(err, "error marshal NodeStatus(%v)", status) + } + key := r.prefixed(prefix, status.NodeID) + err = r.client.Update(ctx, key, string(objstr), 0) + return errors.Trace(err) +} + +func (r *EtcdRegistry) createNode(ctx context.Context, prefix string, status *Status) error { + objstr, err := json.Marshal(status) + if err != nil { + return errors.Annotatef(err, "error marshal NodeStatus(%v)", status) + } + key := r.prefixed(prefix, status.NodeID) + err = r.client.Create(ctx, key, string(objstr), nil) + return errors.Trace(err) +} + +// WatchNode watchs node's event +func (r *EtcdRegistry) WatchNode(pctx context.Context, prefix string) clientv3.WatchChan { + return r.client.Watch(pctx, prefix) +} + +func nodeStatusFromEtcdNode(id string, node *etcd.Node) (*Status, error) { + status := &Status{} + + if err := json.Unmarshal(node.Value, &status); err != nil { + return nil, errors.Annotatef(err, "error unmarshal NodeStatus with nodeID(%s), node value(%s)", id, node.Value) + } + + return status, nil +} + +// NodesStatusFromEtcdNode returns nodes' status under root node. +func NodesStatusFromEtcdNode(root *etcd.Node) ([]*Status, error) { + var statuses []*Status + for id, n := range root.Childs { + status, err := nodeStatusFromEtcdNode(id, n) + if err != nil { + return nil, err + } + if status == nil { + continue + } + statuses = append(statuses, status) + } + return statuses, nil +} + +// AnalyzeNodeID returns nodeID by analyze key path. +func AnalyzeNodeID(key string) string { + // the key looks like: /tidb-binlog/v1/pumps/nodeID, or /tidb-binlog/pumps/nodeID for old binlog version. + paths := strings.Split(key, "/") + nodeIDOffset := 3 + + if len(paths) >= 2 { + // version string start with 'v' + if !strings.HasPrefix(paths[1], "v") { + nodeIDOffset = 2 + } + } else { + log.Errorf("can't get nodeID or node type from key %s", key) + return "" + } + + if len(paths) < nodeIDOffset+1 { + log.Errorf("can't get nodeID or node type from key %s", key) + return "" + } + + return paths[nodeIDOffset] +} diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go new file mode 100644 index 0000000000000..192ef1a5be7b1 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go @@ -0,0 +1,455 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package client + +import ( + "crypto/tls" + "encoding/json" + "path" + "strings" + "sync" + "time" + + "github.com/coreos/etcd/mvcc/mvccpb" + "github.com/pingcap/pd/client" + "github.com/pingcap/tidb-tools/pkg/etcd" + "github.com/pingcap/tidb-tools/pkg/utils" + "github.com/pingcap/tidb-tools/tidb-binlog/node" + pb "github.com/pingcap/tipb/go-binlog" + "github.com/pkg/errors" + log "github.com/sirupsen/logrus" + "golang.org/x/net/context" +) + +const ( + // DefaultEtcdTimeout is the default timeout config for etcd. + DefaultEtcdTimeout = 5 * time.Second + + // DefaultRetryTime is the default time of retry. + DefaultRetryTime = 20 + + // DefaultBinlogWriteTimeout is the default max time binlog can use to write to pump. + DefaultBinlogWriteTimeout = 15 * time.Second + + // CheckInterval is the default interval for check unavaliable pumps. + CheckInterval = 30 * time.Second + + // RetryInterval is the default interval of retrying to write binlog. + RetryInterval = 100 * time.Millisecond +) + +var ( + // Logger is ... + Logger = log.New() + + // ErrNoAvaliablePump means no avaliable pump to write binlog. + ErrNoAvaliablePump = errors.New("no avaliable pump to write binlog") +) + +// PumpInfos saves pumps' infomations in pumps client. +type PumpInfos struct { + sync.RWMutex + // Pumps saves the map of pump's nodeID and pump status. + Pumps map[string]*PumpStatus + + // AvliablePumps saves the whole avaliable pumps' status. + AvaliablePumps map[string]*PumpStatus + + // UnAvaliablePumps saves the unAvaliable pumps. + // And only pump with Online state in this map need check is it avaliable. + UnAvaliablePumps map[string]*PumpStatus +} + +// PumpsClient is the client of pumps. +type PumpsClient struct { + ctx context.Context + + cancel context.CancelFunc + + wg sync.WaitGroup + + // ClusterID is the cluster ID of this tidb cluster. + ClusterID uint64 + + // the registry of etcd. + EtcdRegistry *node.EtcdRegistry + + // Pumps saves the pumps' information. + Pumps *PumpInfos + + // Selector will select a suitable pump. + Selector PumpSelector + + // the max retry time if write binlog failed. + RetryTime int + + // BinlogWriteTimeout is the max time binlog can use to write to pump. + BinlogWriteTimeout time.Duration + + // Security is the security config + Security *tls.Config +} + +// NewPumpsClient returns a PumpsClient. +func NewPumpsClient(etcdURLs string, timeout time.Duration, securityOpt pd.SecurityOption) (*PumpsClient, error) { + // TODO: get strategy from etcd, and can update strategy in real-time. now use Range as default. + strategy := Range + selector := NewSelector(strategy) + + ectdEndpoints, err := utils.ParseHostPortAddr(etcdURLs) + if err != nil { + return nil, errors.Trace(err) + } + + // get clusterid + pdCli, err := pd.NewClient(ectdEndpoints, securityOpt) + if err != nil { + return nil, errors.Trace(err) + } + clusterID := pdCli.GetClusterID(context.Background()) + pdCli.Close() + + security, err := utils.ToTLSConfig(securityOpt.CAPath, securityOpt.CertPath, securityOpt.KeyPath) + if err != nil { + return nil, errors.Trace(err) + } + + rootPath := path.Join(node.DefaultRootPath, node.NodePrefix[node.PumpNode]) + cli, err := etcd.NewClientFromCfg(ectdEndpoints, DefaultEtcdTimeout, rootPath, security) + if err != nil { + return nil, errors.Trace(err) + } + + pumpInfos := &PumpInfos{ + Pumps: make(map[string]*PumpStatus), + AvaliablePumps: make(map[string]*PumpStatus), + UnAvaliablePumps: make(map[string]*PumpStatus), + } + + ctx, cancel := context.WithCancel(context.Background()) + newPumpsClient := &PumpsClient{ + ctx: ctx, + cancel: cancel, + ClusterID: clusterID, + EtcdRegistry: node.NewEtcdRegistry(cli, DefaultEtcdTimeout), + Pumps: pumpInfos, + Selector: selector, + RetryTime: DefaultRetryTime, + BinlogWriteTimeout: timeout, + Security: security, + } + + err = newPumpsClient.getPumpStatus(ctx) + if err != nil { + return nil, errors.Trace(err) + } + newPumpsClient.Selector.SetPumps(copyPumps(newPumpsClient.Pumps.AvaliablePumps)) + + newPumpsClient.wg.Add(2) + go newPumpsClient.watchStatus() + go newPumpsClient.detect() + + return newPumpsClient, nil +} + +// getPumpStatus retruns all the pumps status in the etcd. +func (c *PumpsClient) getPumpStatus(pctx context.Context) error { + nodesStatus, err := c.EtcdRegistry.Nodes(pctx, node.DefaultRootPath) + if err != nil { + return errors.Trace(err) + } + + for _, status := range nodesStatus { + Logger.Debugf("[pumps client] get pump %v from etcd", status) + c.addPump(NewPumpStatus(status, c.Security), false) + } + + return nil +} + +// WriteBinlog writes binlog to a situable pump. +func (c *PumpsClient) WriteBinlog(binlog *pb.Binlog) error { + pump := c.Selector.Select(binlog) + if pump == nil { + return ErrNoAvaliablePump + } + Logger.Debugf("[pumps client] write binlog choose pump %s", pump.NodeID) + + commitData, err := binlog.Marshal() + if err != nil { + return errors.Trace(err) + } + req := &pb.WriteBinlogReq{ClusterID: c.ClusterID, Payload: commitData} + + retryTime := 0 + startTime := time.Now() + var resp *pb.WriteBinlogResp + + for { + if pump == nil { + return ErrNoAvaliablePump + } + + resp, err = pump.writeBinlog(req, c.BinlogWriteTimeout) + if err == nil && resp.Errmsg != "" { + err = errors.New(resp.Errmsg) + } + if err == nil { + return nil + } + Logger.Errorf("[pumps client] write binlog error %v", err) + + if binlog.Tp == pb.BinlogType_Commit { + // only use one pump to write commit binlog, util write success or blocked for ten minutes. + if time.Since(startTime) > 10*time.Minute { + break + } + } else { + if !isRetryableError(err) { + // this kind of error is not retryable, return directly. + return err + } + + // every pump can retry 5 times, if retry 5 times and still failed, set this pump unavaliable, and choose a new pump. + if (retryTime+1)%5 == 0 { + c.setPumpAvaliable(pump, false) + pump = c.Selector.Next(binlog, retryTime/5+1) + Logger.Debugf("[pumps client] avaliable pumps: %v, write binlog choose pump %v", c.Pumps.AvaliablePumps, pump) + } + + retryTime++ + if retryTime > c.RetryTime { + break + } + } + + time.Sleep(RetryInterval) + } + + return err +} + +// setPumpAvaliable set pump's isAvaliable, and modify UnAvaliablePumps or AvaliablePumps. +func (c *PumpsClient) setPumpAvaliable(pump *PumpStatus, avaliable bool) { + pump.IsAvaliable = avaliable + if pump.IsAvaliable { + err := pump.createGrpcClient(c.Security) + if err != nil { + Logger.Errorf("[pumps client] create grpc client for pump %s failed, error: %v", pump.NodeID, err) + pump.IsAvaliable = false + return + } + + c.Pumps.Lock() + delete(c.Pumps.UnAvaliablePumps, pump.NodeID) + if _, ok := c.Pumps.Pumps[pump.NodeID]; ok { + c.Pumps.AvaliablePumps[pump.NodeID] = pump + } + c.Pumps.Unlock() + + } else { + c.Pumps.Lock() + delete(c.Pumps.AvaliablePumps, pump.NodeID) + if _, ok := c.Pumps.Pumps[pump.NodeID]; ok { + c.Pumps.UnAvaliablePumps[pump.NodeID] = pump + } + c.Pumps.Unlock() + } + + c.Pumps.RLock() + c.Selector.SetPumps(copyPumps(c.Pumps.AvaliablePumps)) + c.Pumps.RUnlock() +} + +// addPump add a new pump. +func (c *PumpsClient) addPump(pump *PumpStatus, updateSelector bool) { + c.Pumps.Lock() + + if pump.State == node.Online { + c.Pumps.AvaliablePumps[pump.NodeID] = pump + } else { + c.Pumps.UnAvaliablePumps[pump.NodeID] = pump + } + c.Pumps.Pumps[pump.NodeID] = pump + + if updateSelector { + c.Selector.SetPumps(copyPumps(c.Pumps.AvaliablePumps)) + } + + c.Pumps.Unlock() +} + +// updatePump update pump's status, and return whether pump's IsAvaliable should be changed. +func (c *PumpsClient) updatePump(status *node.Status) (pump *PumpStatus, avaliableChanged, avaliable bool) { + var ok bool + c.Pumps.Lock() + if pump, ok = c.Pumps.Pumps[status.NodeID]; ok { + if pump.Status.State != status.State { + if status.State == node.Online { + avaliableChanged = true + avaliable = true + } else if pump.Status.State == node.Online { + avaliableChanged = true + avaliable = false + } + } + pump.Status = *status + } + c.Pumps.Unlock() + + return +} + +// removePump removes a pump. +func (c *PumpsClient) removePump(nodeID string) { + c.Pumps.Lock() + if pump, ok := c.Pumps.Pumps[nodeID]; ok { + pump.closeGrpcClient() + } + delete(c.Pumps.Pumps, nodeID) + delete(c.Pumps.UnAvaliablePumps, nodeID) + delete(c.Pumps.AvaliablePumps, nodeID) + c.Selector.SetPumps(copyPumps(c.Pumps.AvaliablePumps)) + c.Pumps.Unlock() +} + +// exist returns true if pumps client has pump matched this nodeID. +func (c *PumpsClient) exist(nodeID string) bool { + c.Pumps.RLock() + _, ok := c.Pumps.Pumps[nodeID] + c.Pumps.RUnlock() + return ok +} + +// watchStatus watchs pump's status in etcd. +func (c *PumpsClient) watchStatus() { + defer c.wg.Done() + rootPath := path.Join(node.DefaultRootPath, node.NodePrefix[node.PumpNode]) + rch := c.EtcdRegistry.WatchNode(c.ctx, rootPath) + for { + select { + case <-c.ctx.Done(): + Logger.Info("[pumps client] watch status finished") + return + case wresp := <-rch: + for _, ev := range wresp.Events { + status := &node.Status{} + err := json.Unmarshal(ev.Kv.Value, &status) + if err != nil { + Logger.Errorf("[pumps client] unmarshal pump status %q failed", ev.Kv.Value) + continue + } + + switch ev.Type { + case mvccpb.PUT: + if !c.exist(status.NodeID) { + Logger.Infof("[pumps client] find a new pump %s", status.NodeID) + c.addPump(NewPumpStatus(status, c.Security), true) + continue + } + + pump, avaliableChanged, avaliable := c.updatePump(status) + if avaliableChanged { + Logger.Infof("[pumps client] pump %s's state is changed to %s", pump.Status.NodeID, status.State) + c.setPumpAvaliable(pump, avaliable) + } + + case mvccpb.DELETE: + // now will not delete pump node in fact, just for compatibility. + nodeID := node.AnalyzeNodeID(string(ev.Kv.Key)) + Logger.Infof("[pumps client] remove pump %s", nodeID) + c.removePump(nodeID) + } + } + } + } +} + +// detect send detect binlog to pumps with online state in UnAvaliablePumps, +func (c *PumpsClient) detect() { + defer c.wg.Done() + for { + select { + case <-c.ctx.Done(): + Logger.Infof("[pumps client] heartbeat finished") + return + default: + // send detect binlog to pump, if this pump can return response without error + // means this pump is avaliable. + needCheckPumps := make([]*PumpStatus, 0, len(c.Pumps.UnAvaliablePumps)) + checkPassPumps := make([]*PumpStatus, 0, 1) + req := &pb.WriteBinlogReq{ClusterID: c.ClusterID, Payload: nil} + c.Pumps.RLock() + for _, pump := range c.Pumps.UnAvaliablePumps { + if pump.Status.State == node.Online { + needCheckPumps = append(needCheckPumps, pump) + } + } + c.Pumps.RUnlock() + + for _, pump := range needCheckPumps { + err := pump.createGrpcClient(c.Security) + if err != nil { + Logger.Errorf("[pumps client] create grpc client for pump %s failed, error %v", pump.NodeID, errors.Trace(err)) + continue + } + if pump.Client == nil { + continue + } + + _, err = pump.writeBinlog(req, c.BinlogWriteTimeout) + if err == nil { + checkPassPumps = append(checkPassPumps, pump) + } else { + Logger.Errorf("[pumps client] write detect binlog to pump %s error %v", pump.NodeID, err) + } + } + + for _, pump := range checkPassPumps { + c.setPumpAvaliable(pump, true) + } + + time.Sleep(CheckInterval) + } + } +} + +// Close closes the PumpsClient. +func (c *PumpsClient) Close() { + Logger.Infof("[pumps client] is closing") + c.cancel() + c.wg.Wait() + Logger.Infof("[pumps client] is closed") +} + +func isRetryableError(err error) bool { + // ResourceExhausted is a error code in grpc. + // ResourceExhausted indicates some resource has been exhausted, perhaps + // a per-user quota, or perhaps the entire file system is out of space. + // https://github.com/grpc/grpc-go/blob/9cc4fdbde2304827ffdbc7896f49db40c5536600/codes/codes.go#L76 + if strings.Contains(err.Error(), "ResourceExhausted") { + return false + } + + return true +} + +func copyPumps(pumps map[string]*PumpStatus) []*PumpStatus { + ps := make([]*PumpStatus, 0, len(pumps)) + for _, pump := range pumps { + ps = append(ps, pump) + } + + return ps +} diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/pump.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/pump.go new file mode 100644 index 0000000000000..0a509b004729b --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/pump.go @@ -0,0 +1,120 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package client + +import ( + "crypto/tls" + "net" + "time" + + "github.com/pingcap/tidb-tools/tidb-binlog/node" + pb "github.com/pingcap/tipb/go-binlog" + "github.com/pkg/errors" + "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials" +) + +// PumpStatus saves pump's status. +type PumpStatus struct { + /* + Pump has these state: + Online: + only when pump's state is online that pumps client can write binlog to. + Pausing: + this pump is pausing, and can't provide write binlog service. And this state will turn into Paused when pump is quit. + Paused: + this pump is paused, and can't provide write binlog service. + Closing: + this pump is closing, and can't provide write binlog service. And this state will turn into Offline when pump is quit. + Offline: + this pump is offline, and can't provide write binlog service forever. + */ + node.Status + + // the pump is avaliable or not + IsAvaliable bool + + grpcConn *grpc.ClientConn + + // the client of this pump + Client pb.PumpClient +} + +// NewPumpStatus returns a new PumpStatus according to node's status. +func NewPumpStatus(status *node.Status, security *tls.Config) *PumpStatus { + pumpStatus := &PumpStatus{} + pumpStatus.Status = *status + pumpStatus.IsAvaliable = (status.State == node.Online) + + if status.State != node.Online { + return pumpStatus + } + + err := pumpStatus.createGrpcClient(security) + if err != nil { + Logger.Errorf("[pumps client] create grpc client for %s failed, error %v", status.NodeID, err) + pumpStatus.IsAvaliable = false + } + + return pumpStatus +} + +// createGrpcClient create grpc client for online pump. +func (p *PumpStatus) createGrpcClient(security *tls.Config) error { + // release the old connection, and create a new one + if p.grpcConn != nil { + p.grpcConn.Close() + } + + dialerOpt := grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) { + return net.DialTimeout("tcp", addr, timeout) + }) + Logger.Debugf("[pumps client] create gcpc client at %s", p.Addr) + var clientConn *grpc.ClientConn + var err error + if security != nil { + clientConn, err = grpc.Dial(p.Addr, dialerOpt, grpc.WithTransportCredentials(credentials.NewTLS(security))) + } else { + clientConn, err = grpc.Dial(p.Addr, dialerOpt, grpc.WithInsecure()) + } + if err != nil { + return err + } + + p.grpcConn = clientConn + p.Client = pb.NewPumpClient(clientConn) + + return nil +} + +// closeGrpcClient closes the pump's grpc connection. +func (p *PumpStatus) closeGrpcClient() { + if p.grpcConn != nil { + p.grpcConn.Close() + p.Client = nil + } +} + +func (p *PumpStatus) writeBinlog(req *pb.WriteBinlogReq, timeout time.Duration) (*pb.WriteBinlogResp, error) { + if p.Client == nil { + return nil, errors.Errorf("pump %s don't have avaliable pump client", p.NodeID) + } + + ctx, cancel := context.WithTimeout(context.Background(), timeout) + resp, err := p.Client.WriteBinlog(ctx, req) + cancel() + + return resp, err +} diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/selector.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/selector.go new file mode 100644 index 0000000000000..343c250e1cc67 --- /dev/null +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/selector.go @@ -0,0 +1,274 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package client + +import ( + "hash/fnv" + "strconv" + "sync" + + pb "github.com/pingcap/tipb/go-binlog" +) + +const ( + // Range means range algorithm. + Range = "range" + + // Hash means hash algorithm. + Hash = "hash" + + // Score means choose pump by it's score. + Score = "score" +) + +// PumpSelector selects pump for sending binlog. +type PumpSelector interface { + // SetPumps set pumps to be selected. + SetPumps([]*PumpStatus) + + // Select returns a situable pump. + Select(*pb.Binlog) *PumpStatus + + // returns the next pump. + Next(*pb.Binlog, int) *PumpStatus +} + +// HashSelector select a pump by hash. +type HashSelector struct { + sync.RWMutex + + // TsMap saves the map of start_ts with pump when send prepare binlog. + // And Commit binlog should send to the same pump. + TsMap map[int64]*PumpStatus + + // PumpMap saves the map of pump's node id with pump. + PumpMap map[string]*PumpStatus + + // the pumps to be selected. + Pumps []*PumpStatus +} + +// NewHashSelector returns a new HashSelector. +func NewHashSelector() PumpSelector { + return &HashSelector{ + TsMap: make(map[int64]*PumpStatus), + PumpMap: make(map[string]*PumpStatus), + Pumps: make([]*PumpStatus, 0, 10), + } +} + +// SetPumps implement PumpSelector.SetPumps. +func (h *HashSelector) SetPumps(pumps []*PumpStatus) { + h.Lock() + h.PumpMap = make(map[string]*PumpStatus) + h.Pumps = pumps + for _, pump := range pumps { + h.PumpMap[pump.NodeID] = pump + } + h.Unlock() +} + +// Select implement PumpSelector.Select. +func (h *HashSelector) Select(binlog *pb.Binlog) *PumpStatus { + // TODO: use status' label to match situale pump. + h.Lock() + defer h.Unlock() + + if pump, ok := h.TsMap[binlog.StartTs]; ok { + // binlog is commit binlog or rollback binlog, choose the same pump by start ts map. + delete(h.TsMap, binlog.StartTs) + return pump + } + + if len(h.Pumps) == 0 { + return nil + } + + if binlog.Tp == pb.BinlogType_Prewrite { + pump := h.Pumps[hashTs(binlog.StartTs)%len(h.Pumps)] + h.TsMap[binlog.StartTs] = pump + return pump + } + + // can't find pump in ts map, or unkow binlog type, choose a new one. + return h.Pumps[hashTs(binlog.StartTs)%len(h.Pumps)] +} + +// Next implement PumpSelector.Next. Only for Prewrite binlog. +func (h *HashSelector) Next(binlog *pb.Binlog, retryTime int) *PumpStatus { + h.Lock() + defer h.Unlock() + + if len(h.Pumps) == 0 { + return nil + } + + nextPump := h.Pumps[(hashTs(binlog.StartTs)+int(retryTime))%len(h.Pumps)] + if binlog.Tp == pb.BinlogType_Prewrite { + h.TsMap[binlog.StartTs] = nextPump + } + + return nextPump +} + +// RangeSelector select a pump by range. +type RangeSelector struct { + sync.RWMutex + + // Offset saves the offset in Pumps. + Offset int + + // TsMap saves the map of start_ts with pump when send prepare binlog. + // And Commit binlog should send to the same pump. + TsMap map[int64]*PumpStatus + + // PumpMap saves the map of pump's node id with pump. + PumpMap map[string]*PumpStatus + + // the pumps to be selected. + Pumps []*PumpStatus +} + +// NewRangeSelector returns a new ScoreSelector. +func NewRangeSelector() PumpSelector { + return &RangeSelector{ + Offset: 0, + TsMap: make(map[int64]*PumpStatus), + PumpMap: make(map[string]*PumpStatus), + Pumps: make([]*PumpStatus, 0, 10), + } +} + +// SetPumps implement PumpSelector.SetPumps. +func (r *RangeSelector) SetPumps(pumps []*PumpStatus) { + r.Lock() + r.PumpMap = make(map[string]*PumpStatus) + r.Pumps = pumps + for _, pump := range pumps { + r.PumpMap[pump.NodeID] = pump + } + r.Offset = 0 + r.Unlock() +} + +// Select implement PumpSelector.Select. +func (r *RangeSelector) Select(binlog *pb.Binlog) *PumpStatus { + // TODO: use status' label to match situale pump. + r.Lock() + defer func() { + if r.Offset == len(r.Pumps) { + r.Offset = 0 + } + r.Unlock() + }() + + if pump, ok := r.TsMap[binlog.StartTs]; ok { + // binlog is commit binlog or rollback binlog, choose the same pump by start ts map. + delete(r.TsMap, binlog.StartTs) + return pump + } + + if len(r.Pumps) == 0 { + return nil + } + + if r.Offset >= len(r.Pumps) { + r.Offset = 0 + } + + if binlog.Tp == pb.BinlogType_Prewrite { + pump := r.Pumps[r.Offset] + r.TsMap[binlog.StartTs] = pump + r.Offset++ + return pump + } + + // can't find pump in ts map, or the pump is not avaliable, choose a new one. + return r.Pumps[r.Offset] +} + +// Next implement PumpSelector.Next. Only for Prewrite binlog. +func (r *RangeSelector) Next(binlog *pb.Binlog, retryTime int) *PumpStatus { + r.Lock() + defer func() { + if len(r.Pumps) != 0 { + r.Offset = (r.Offset + 1) % len(r.Pumps) + } + r.Unlock() + }() + + if len(r.Pumps) == 0 { + return nil + } + + if r.Offset >= len(r.Pumps) { + r.Offset = 0 + } + + nextPump := r.Pumps[r.Offset] + if binlog.Tp == pb.BinlogType_Prewrite { + r.TsMap[binlog.StartTs] = nextPump + } + + return nextPump +} + +// ScoreSelector select a pump by pump's score. +type ScoreSelector struct{} + +// NewScoreSelector returns a new ScoreSelector. +func NewScoreSelector() PumpSelector { + return &ScoreSelector{} +} + +// SetPumps implement PumpSelector.SetPumps. +func (s *ScoreSelector) SetPumps(pumps []*PumpStatus) { + // TODO +} + +// Select implement PumpSelector.Select. +func (s *ScoreSelector) Select(binlog *pb.Binlog) *PumpStatus { + // TODO + return nil +} + +// Next implement PumpSelector.Next. Only for Prewrite binlog. +func (s *ScoreSelector) Next(binlog *pb.Binlog, retryTime int) *PumpStatus { + // TODO + return nil +} + +// NewSelector returns a PumpSelector according to the algorithm. +func NewSelector(algorithm string) PumpSelector { + var selector PumpSelector + switch algorithm { + case Range: + selector = NewRangeSelector() + case Hash: + selector = NewHashSelector() + case Score: + selector = NewScoreSelector() + default: + Logger.Warnf("unknow algorithm %s, use range as default", algorithm) + selector = NewRangeSelector() + } + + return selector +} + +func hashTs(ts int64) int { + h := fnv.New32a() + h.Write([]byte(strconv.FormatInt(ts, 10))) + return int(h.Sum32()) +} diff --git a/vendor/github.com/pkg/errors/errors.go b/vendor/github.com/pkg/errors/errors.go index 0168061fd5276..2e1d3f6289668 100644 --- a/vendor/github.com/pkg/errors/errors.go +++ b/vendor/github.com/pkg/errors/errors.go @@ -152,7 +152,8 @@ func (f *fundamental) Format(s fmt.State, verb rune) { // WithStack annotates err with a stack trace at the point WithStack was called. // If err is nil, WithStack returns nil. // -// Deprecated: use AddStack +// For most use cases this is deprecated and AddStack should be used (which will ensure just one stack trace). +// However, one may want to use this in some situations, for example to create a 2nd trace across a goroutine. func WithStack(err error) error { if err == nil { return nil @@ -173,22 +174,6 @@ func AddStack(err error) error { return WithStack(err) } -// GetStackTracer will return the first StackTracer in the causer chain. -// This function is used by AddStack to avoid creating redundant stack traces. -// -// You can also use the StackTracer interface on the returned error to get the stack trace. -func GetStackTracer(origErr error) StackTracer { - var stacked StackTracer - WalkDeep(origErr, func(err error) bool { - if stackTracer, ok := err.(StackTracer); ok { - stacked = stackTracer - return true - } - return false - }) - return stacked -} - type withStack struct { error *stack @@ -213,10 +198,11 @@ func (w *withStack) Format(s fmt.State, verb rune) { } // Wrap returns an error annotating err with a stack trace -// at the point Annotate is called, and the supplied message. -// If err is nil, Annotate returns nil. +// at the point Wrap is called, and the supplied message. +// If err is nil, Wrap returns nil. // -// Deprecated: use Annotate instead +// For most use cases this is deprecated in favor of Annotate. +// Annotate avoids creating duplicate stack traces. func Wrap(err error, message string) error { if err == nil { return nil @@ -234,10 +220,11 @@ func Wrap(err error, message string) error { } // Wrapf returns an error annotating err with a stack trace -// at the point Annotatef is call, and the format specifier. -// If err is nil, Annotatef returns nil. +// at the point Wrapf is call, and the format specifier. +// If err is nil, Wrapf returns nil. // -// Deprecated: use Annotatef instead +// For most use cases this is deprecated in favor of Annotatef. +// Annotatef avoids creating duplicate stack traces. func Wrapf(err error, format string, args ...interface{}) error { if err == nil { return nil diff --git a/vendor/github.com/pkg/errors/group.go b/vendor/github.com/pkg/errors/group.go index 003932c95e865..e5a969ab76f76 100644 --- a/vendor/github.com/pkg/errors/group.go +++ b/vendor/github.com/pkg/errors/group.go @@ -6,6 +6,15 @@ type ErrorGroup interface { Errors() []error } +// Errors uses the ErrorGroup interface to return a slice of errors. +// If the ErrorGroup interface is not implemented it returns an array containing just the given error. +func Errors(err error) []error { + if eg, ok := err.(ErrorGroup); ok { + return eg.Errors() + } + return []error{err} +} + // WalkDeep does a depth-first traversal of all errors. // Any ErrorGroup is traversed (after going deep). // The visitor function can return true to end the traversal early diff --git a/vendor/github.com/pkg/errors/juju_adaptor.go b/vendor/github.com/pkg/errors/juju_adaptor.go index 773a1970866c7..8bcfe2f37bd5b 100644 --- a/vendor/github.com/pkg/errors/juju_adaptor.go +++ b/vendor/github.com/pkg/errors/juju_adaptor.go @@ -2,16 +2,17 @@ package errors import ( "fmt" + "strings" ) // ==================== juju adaptor start ======================== -// Trace annotates err with a stack trace at the point WithStack was called. -// If err is nil or already contain stack trace return directly. +// Trace just calls AddStack. func Trace(err error) error { return AddStack(err) } +// Annotate adds a message and ensures there is a stack trace. func Annotate(err error, message string) error { if err == nil { return nil @@ -31,6 +32,7 @@ func Annotate(err error, message string) error { } } +// Annotatef adds a message and ensures there is a stack trace. func Annotatef(err error, format string, args ...interface{}) error { if err == nil { return nil @@ -51,6 +53,8 @@ func Annotatef(err error, format string, args ...interface{}) error { } // ErrorStack will format a stack trace if it is available, otherwise it will be Error() +// If the error is nil, the empty string is returned +// Note that this just calls fmt.Sprintf("%+v", err) func ErrorStack(err error) string { if err == nil { return "" @@ -58,6 +62,11 @@ func ErrorStack(err error) string { return fmt.Sprintf("%+v", err) } +// IsNotFound reports whether err was not found error. +func IsNotFound(err error) bool { + return strings.Contains(err.Error(), "not found") +} + // NotFoundf represents an error with not found message. func NotFoundf(format string, args ...interface{}) error { return Errorf(format+" not found", args...) @@ -73,4 +82,19 @@ func NotSupportedf(format string, args ...interface{}) error { return Errorf(format+" not supported", args...) } +// NotValidf represents an error with not valid message. +func NotValidf(format string, args ...interface{}) error { + return Errorf(format+" not valid", args...) +} + +// IsAlreadyExists reports whether err was already exists error. +func IsAlreadyExists(err error) bool { + return strings.Contains(err.Error(), "already exists") +} + +// AlreadyExistsf represents an error with already exists message. +func AlreadyExistsf(format string, args ...interface{}) error { + return Errorf(format+" already exists", args...) +} + // ==================== juju adaptor end ======================== diff --git a/vendor/github.com/pkg/errors/stack.go b/vendor/github.com/pkg/errors/stack.go index 6edd7e5699f79..bb1e6a84f339d 100644 --- a/vendor/github.com/pkg/errors/stack.go +++ b/vendor/github.com/pkg/errors/stack.go @@ -1,10 +1,12 @@ package errors import ( + "bytes" "fmt" "io" "path" "runtime" + "strconv" "strings" ) @@ -14,6 +16,22 @@ type StackTracer interface { StackTrace() StackTrace } +// GetStackTracer will return the first StackTracer in the causer chain. +// This function is used by AddStack to avoid creating redundant stack traces. +// +// You can also use the StackTracer interface on the returned error to get the stack trace. +func GetStackTracer(origErr error) StackTracer { + var stacked StackTracer + WalkDeep(origErr, func(err error) bool { + if stackTracer, ok := err.(StackTracer); ok { + stacked = stackTracer + return true + } + return false + }) + return stacked +} + // Frame represents a program counter inside a stack frame. type Frame uintptr @@ -56,6 +74,11 @@ func (f Frame) line() int { // GOPATH separated by \n\t (\n\t) // %+v equivalent to %+s:%d func (f Frame) Format(s fmt.State, verb rune) { + f.format(s, s, verb) +} + +// format allows stack trace printing calls to be made with a bytes.Buffer. +func (f Frame) format(w io.Writer, s fmt.State, verb rune) { switch verb { case 's': switch { @@ -63,23 +86,25 @@ func (f Frame) Format(s fmt.State, verb rune) { pc := f.pc() fn := runtime.FuncForPC(pc) if fn == nil { - io.WriteString(s, "unknown") + io.WriteString(w, "unknown") } else { file, _ := fn.FileLine(pc) - fmt.Fprintf(s, "%s\n\t%s", fn.Name(), file) + io.WriteString(w, fn.Name()) + io.WriteString(w, "\n\t") + io.WriteString(w, file) } default: - io.WriteString(s, path.Base(f.file())) + io.WriteString(w, path.Base(f.file())) } case 'd': - fmt.Fprintf(s, "%d", f.line()) + io.WriteString(w, strconv.Itoa(f.line())) case 'n': name := runtime.FuncForPC(f.pc()).Name() - io.WriteString(s, funcname(name)) + io.WriteString(w, funcname(name)) case 'v': - f.Format(s, 's') - io.WriteString(s, ":") - f.Format(s, 'd') + f.format(w, s, 's') + io.WriteString(w, ":") + f.format(w, s, 'd') } } @@ -95,23 +120,50 @@ type StackTrace []Frame // // %+v Prints filename, function, and line number for each Frame in the stack. func (st StackTrace) Format(s fmt.State, verb rune) { + var b bytes.Buffer switch verb { case 'v': switch { case s.Flag('+'): - for _, f := range st { - fmt.Fprintf(s, "\n%+v", f) + b.Grow(len(st) * stackMinLen) + for _, fr := range st { + b.WriteByte('\n') + fr.format(&b, s, verb) } case s.Flag('#'): - fmt.Fprintf(s, "%#v", []Frame(st)) + fmt.Fprintf(&b, "%#v", []Frame(st)) default: - fmt.Fprintf(s, "%v", []Frame(st)) + st.formatSlice(&b, s, verb) } case 's': - fmt.Fprintf(s, "%s", []Frame(st)) + st.formatSlice(&b, s, verb) } + io.Copy(s, &b) } +// formatSlice will format this StackTrace into the given buffer as a slice of +// Frame, only valid when called with '%s' or '%v'. +func (st StackTrace) formatSlice(b *bytes.Buffer, s fmt.State, verb rune) { + b.WriteByte('[') + if len(st) == 0 { + b.WriteByte(']') + return + } + + b.Grow(len(st) * (stackMinLen / 4)) + st[0].format(b, s, verb) + for _, fr := range st[1:] { + b.WriteByte(' ') + fr.format(b, s, verb) + } + b.WriteByte(']') +} + +// stackMinLen is a best-guess at the minimum length of a stack trace. It +// doesn't need to be exact, just give a good enough head start for the buffer +// to avoid the expensive early growth. +const stackMinLen = 96 + // stack represents a stack of program counters. type stack []uintptr @@ -120,10 +172,14 @@ func (s *stack) Format(st fmt.State, verb rune) { case 'v': switch { case st.Flag('+'): + var b bytes.Buffer + b.Grow(len(*s) * stackMinLen) for _, pc := range *s { f := Frame(pc) - fmt.Fprintf(st, "\n%+v", f) + b.WriteByte('\n') + f.format(&b, st, 'v') } + io.Copy(st, &b) } } } From 56b8c4a5eb7984ec63646d31df350e15bfc571fe Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 30 Oct 2018 11:38:52 +0800 Subject: [PATCH 041/509] infoschema: fix pseudo data for profiling table (#8092) (#8096) --- infoschema/tables.go | 4 ++-- infoschema/tables_test.go | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index d47bb0e007870..80e80a54144b7 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1170,8 +1170,8 @@ func dataForPseudoProfiling() [][]types.Datum { 0, // PAGE_FAULTS_MAJOR 0, // PAGE_FAULTS_MINOR 0, // SWAPS - 0, // SOURCE_FUNCTION - 0, // SOURCE_FILE + "", // SOURCE_FUNCTION + "", // SOURCE_FILE 0, // SOURCE_LINE ) rows = append(rows, row) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 919f7b9a9e0b7..0bdd327075f8a 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -176,3 +176,20 @@ func (s *testSuite) TestSchemataCharacterSet(c *C) { testkit.Rows("utf8mb4 utf8_bin")) } + +func (s *testSuite) TestProfiling(c *C) { + testleak.BeforeTest() + defer testleak.AfterTest(c)() + store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() + session.SetStatsLease(0) + do, err := session.BootstrapSession(store) + c.Assert(err, IsNil) + defer do.Close() + + tk := testkit.NewTestKit(c, store) + tk.MustQuery("select * from information_schema.profiling").Check(testkit.Rows()) + tk.MustExec("set @@profiling=1") + tk.MustQuery("select * from information_schema.profiling").Check(testkit.Rows("0 0 0 0 0 0 0 0 0 0 0 0 0 0 0")) +} From 3e6f687aa3f45cf3b6568335dca5cc31d013f712 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Tue, 30 Oct 2018 13:31:06 +0800 Subject: [PATCH 042/509] fix reassigned partition id in truncate table does not take effect (#8103) --- ddl/db_test.go | 26 ++++++++++++++++++++++++++ ddl/partition.go | 20 ++++++++++++++++++++ ddl/table.go | 14 +++++--------- 3 files changed, 51 insertions(+), 9 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index e7a4f3a687fa7..9b9c0d9a80bb9 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3008,6 +3008,32 @@ func (s *testDBSuite) TestTruncatePartitionAndDropTable(c *C) { hasOldPartitionData = checkPartitionDelRangeDone(c, s, partitionPrefix) c.Assert(hasOldPartitionData, IsFalse) s.testErrorCode(c, "select * from t4;", tmysql.ErrNoSuchTable) + + // Test truncate table partition reassigns new partitionIDs. + s.tk.MustExec("drop table if exists t5;") + s.tk.MustExec("set @@session.tidb_enable_table_partition=1;") + s.tk.MustExec(`create table t5( + id int, name varchar(50), + purchased date + ) + partition by range( year(purchased) ) ( + partition p0 values less than (1990), + partition p1 values less than (1995), + partition p2 values less than (2000), + partition p3 values less than (2005), + partition p4 values less than (2010), + partition p5 values less than (2015) + );`) + is = domain.GetDomain(ctx).InfoSchema() + oldTblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t5")) + c.Assert(err, IsNil) + oldPID = oldTblInfo.Meta().Partition.Definitions[0].ID + s.tk.MustExec("truncate table t5;") + is = domain.GetDomain(ctx).InfoSchema() + c.Assert(err, IsNil) + newTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t5")) + newPID := newTblInfo.Meta().Partition.Definitions[0].ID + c.Assert(oldPID != newPID, IsTrue) } func (s *testDBSuite) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { diff --git a/ddl/partition.go b/ddl/partition.go index b1110209c3e3b..9a44f88e80dc3 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -417,3 +417,23 @@ func isRangePartitionColUnsignedBigint(cols []*table.Column, pi *model.Partition } return false } + +// truncateTableByReassignPartitionIDs reassigns new partition ids. +func truncateTableByReassignPartitionIDs(t *meta.Meta, tblInfo *model.TableInfo) error { + newDefs := make([]model.PartitionDefinition, 0, len(tblInfo.Partition.Definitions)) + for _, def := range tblInfo.Partition.Definitions { + pid, err := t.GenGlobalID() + if err != nil { + return errors.Trace(err) + } + newDef := model.PartitionDefinition{ + ID: pid, + Name: def.Name, + LessThan: def.LessThan, + Comment: def.Comment, + } + newDefs = append(newDefs, newDef) + } + tblInfo.Partition.Definitions = newDefs + return nil +} diff --git a/ddl/table.go b/ddl/table.go index 930f871ff43d2..0bba885b07c53 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -209,18 +209,14 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, errors.Trace(err) } - // We use the new partition ID because all the old data is encoded with the old partition ID, it can not be accessed anymore. var oldPartitionIDs []int64 if tblInfo.GetPartitionInfo() != nil { oldPartitionIDs = getPartitionIDs(tblInfo) - for _, def := range tblInfo.Partition.Definitions { - var pid int64 - pid, err = t.GenGlobalID() - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - def.ID = pid + // We use the new partition ID because all the old data is encoded with the old partition ID, it can not be accessed anymore. + err = truncateTableByReassignPartitionIDs(t, tblInfo) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) } } From 26a3a3db88b6246324f17ceff323795719570f99 Mon Sep 17 00:00:00 2001 From: liyuzhou <2541781827@qq.com> Date: Tue, 30 Oct 2018 15:37:33 +0800 Subject: [PATCH 043/509] sessionctx, executor: add max_allowed_packet verification (#8090) (#8104) --- executor/set_test.go | 8 ++++++++ sessionctx/variable/varsutil.go | 3 +++ 2 files changed, 11 insertions(+) diff --git a/executor/set_test.go b/executor/set_test.go index 147bc22d16636..3ef60edfb8ce6 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -337,6 +337,14 @@ func (s *testSuite) TestValidateSetVar(c *C) { _, err = tk.Exec("set @@global.max_connections='hello'") c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + tk.MustExec("set @@global.max_allowed_packet=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_allowed_packet value: '-1'")) + result = tk.MustQuery("select @@global.max_allowed_packet;") + result.Check(testkit.Rows("1024")) + + _, err = tk.Exec("set @@global.max_allowed_packet='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + tk.MustExec("set @@global.max_connect_errors=18446744073709551615") tk.MustExec("set @@global.max_connect_errors=-1") diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 459b498b7352f..d956701cb358d 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -244,6 +244,9 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return checkUInt64SystemVar(name, value, 0, 2, vars) case InnodbLockWaitTimeout: return checkUInt64SystemVar(name, value, 1, 1073741824, vars) + // See "https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_max_allowed_packet" + case MaxAllowedPacket: + return checkUInt64SystemVar(name, value, 1024, 1073741824, vars) case MaxConnections: return checkUInt64SystemVar(name, value, 1, 100000, vars) case MaxConnectErrors: From 6999f640d60a372ce1d9c541a05511d56ac276c8 Mon Sep 17 00:00:00 2001 From: WangXiangUSTC Date: Tue, 30 Oct 2018 19:22:12 +0800 Subject: [PATCH 044/509] binlog: use pumps client to write binlog (#8098) --- config/config.go | 4 +- config/config.toml.example | 5 +- config/config_test.go | 4 +- ddl/ddl.go | 11 +- session/session.go | 4 +- session/session_test.go | 3 +- session/tidb.go | 29 ------ session/tidb_test.go | 17 ---- sessionctx/binloginfo/binloginfo.go | 122 +++++++++++++---------- sessionctx/binloginfo/binloginfo_test.go | 5 +- sessionctx/variable/session.go | 3 +- table/tables/tables_test.go | 2 +- tidb-server/main.go | 35 +++---- 13 files changed, 108 insertions(+), 136 deletions(-) diff --git a/config/config.go b/config/config.go index a6f3c51ba81b4..d4b4b0a003ed9 100644 --- a/config/config.go +++ b/config/config.go @@ -189,7 +189,7 @@ type PreparedPlanCache struct { // OpenTracing is the opentracing section of the config. type OpenTracing struct { - Enable bool `toml:"enable" json:"enbale"` + Enable bool `toml:"enable" json:"enable"` Sampler OpenTracingSampler `toml:"sampler" json:"sampler"` Reporter OpenTracingReporter `toml:"reporter" json:"reporter"` RPCMetrics bool `toml:"rpc-metrics" json:"rpc-metrics"` @@ -241,7 +241,7 @@ type TiKVClient struct { // Binlog is the config for binlog. type Binlog struct { - BinlogSocket string `toml:"binlog-socket" json:"binlog-socket"` + Enable bool `toml:"enable" json:"enable"` WriteTimeout string `toml:"write-timeout" json:"write-timeout"` // If IgnoreError is true, when writting binlog meets error, TiDB would // ignore the error. diff --git a/config/config.toml.example b/config/config.toml.example index bea0a99590eb6..13702b8e08063 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -236,9 +236,8 @@ enabled = false capacity = 10240000 [binlog] - -# Socket file to write binlog. -binlog-socket = "" +# enable to write binlog. +enable = false # WriteTimeout specifies how long it will wait for writing binlog to pump. write-timeout = "15s" diff --git a/config/config_test.go b/config/config_test.go index 5631254a52028..88f60fec4b920 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -33,7 +33,7 @@ func TestT(t *testing.T) { func (s *testConfigSuite) TestConfig(c *C) { conf := new(Config) - conf.Binlog.BinlogSocket = "/tmp/socket" + conf.Binlog.Enable = true conf.Binlog.IgnoreError = true conf.TiKVClient.CommitTimeout = "10s" @@ -52,7 +52,7 @@ commit-timeout="41s"`) c.Assert(conf.Load(configFile), IsNil) // Test that the original value will not be clear by load the config file that does not contain the option. - c.Assert(conf.Binlog.BinlogSocket, Equals, "/tmp/socket") + c.Assert(conf.Binlog.Enable, Equals, true) c.Assert(conf.TiKVClient.CommitTimeout, Equals, "41s") c.Assert(f.Close(), IsNil) diff --git a/ddl/ddl.go b/ddl/ddl.go index 768a2f3be3f7e..75ebad6f7b3dd 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -24,6 +24,7 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/ngaut/pools" + pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" @@ -225,7 +226,7 @@ type DDL interface { // GetTableMaxRowID gets the max row ID of a normal table or a partition. GetTableMaxRowID(startTS uint64, tbl table.PhysicalTable) (int64, bool, error) // SetBinlogClient sets the binlog client for DDL worker. It's exported for testing. - SetBinlogClient(interface{}) + SetBinlogClient(*pumpcli.PumpsClient) } // ddl is used to handle the statements that define the structure or schema of the database. @@ -246,8 +247,8 @@ type ddlCtx struct { schemaSyncer SchemaSyncer ddlJobDoneCh chan struct{} ddlEventCh chan<- *util.Event - lease time.Duration // lease is schema lease. - binlogCli interface{} // binlogCli is used for Binlog. + lease time.Duration // lease is schema lease. + binlogCli *pumpcli.PumpsClient // binlogCli is used for Binlog. // hook may be modified. mu struct { @@ -327,7 +328,7 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, ddlJobDoneCh: make(chan struct{}, 1), ownerManager: manager, schemaSyncer: syncer, - binlogCli: binloginfo.GetPumpClient(), + binlogCli: binloginfo.GetPumpsClient(), } ddlCtx.mu.hook = hook ddlCtx.mu.interceptor = &BaseInterceptor{} @@ -537,7 +538,7 @@ func (d *ddl) callHookOnChanged(err error) error { } // SetBinlogClient implements DDL.SetBinlogClient interface. -func (d *ddl) SetBinlogClient(binlogCli interface{}) { +func (d *ddl) SetBinlogClient(binlogCli *pumpcli.PumpsClient) { d.binlogCli = binlogCli } diff --git a/session/session.go b/session/session.go index 953d024eaba07..4e157e199f47e 100644 --- a/session/session.go +++ b/session/session.go @@ -292,7 +292,7 @@ func (s *session) doCommit(ctx context.Context) error { Tp: binlog.BinlogType_Prewrite, PrewriteValue: prewriteData, }, - Client: s.sessionVars.BinlogClient.(binlog.PumpClient), + Client: s.sessionVars.BinlogClient, } s.txn.SetOption(kv.BinlogInfo, info) } @@ -1187,7 +1187,7 @@ func createSession(store kv.Storage) (*session, error) { domain.BindDomain(s, dom) // session implements variable.GlobalVarAccessor. Bind it to ctx. s.sessionVars.GlobalVarsAccessor = s - s.sessionVars.BinlogClient = binloginfo.GetPumpClient() + s.sessionVars.BinlogClient = binloginfo.GetPumpsClient() s.txn.init() return s, nil } diff --git a/session/session_test.go b/session/session_test.go index 30f0f7a745eec..3a7aa1890da0f 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" @@ -122,7 +123,7 @@ func (s *testSessionSuite) TestForCoverage(c *C) { tk.MustExec("admin check table t") // Cover dirty table operations in StateTxn. - tk.Se.GetSessionVars().BinlogClient = &mockBinlogPump{} + tk.Se.GetSessionVars().BinlogClient = binloginfo.MockPumpsClient(&mockBinlogPump{}) tk.MustExec("begin") tk.MustExec("truncate table t") tk.MustExec("insert t values ()") diff --git a/session/tidb.go b/session/tidb.go index 1357c3e753f01..a312c6bb6ffa0 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -37,8 +37,6 @@ import ( "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials" ) type domainMap struct { @@ -266,33 +264,6 @@ func newStoreWithRetry(path string, maxRetries int) (kv.Storage, error) { return s, errors.Trace(err) } -// DialPumpClientWithRetry tries to dial to binlogSocket, -// if any error happens, it will try to re-dial, -// or return this error when timeout. -func DialPumpClientWithRetry(binlogSocket string, maxRetries int, dialerOpt grpc.DialOption) (*grpc.ClientConn, error) { - var clientCon *grpc.ClientConn - err := util.RunWithRetry(maxRetries, util.RetryInterval, func() (bool, error) { - log.Infof("setup binlog client") - var err error - tlsConfig, err := config.GetGlobalConfig().Security.ToTLSConfig() - if err != nil { - log.Infof("error happen when setting binlog client: %s", errors.ErrorStack(err)) - } - - if tlsConfig != nil { - clientCon, err = grpc.Dial(binlogSocket, grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)), dialerOpt) - } else { - clientCon, err = grpc.Dial(binlogSocket, grpc.WithInsecure(), dialerOpt) - } - - if err != nil { - log.Infof("error happen when setting binlog client: %s", errors.ErrorStack(err)) - } - return true, errors.Trace(err) - }) - return clientCon, errors.Trace(err) -} - var queryStmtTable = []string{"explain", "select", "show", "execute", "describe", "desc", "admin"} func trimSQL(sql string) string { diff --git a/session/tidb_test.go b/session/tidb_test.go index 1821351008d94..99eaa32723ef1 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -27,13 +27,11 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" "golang.org/x/net/context" - "google.golang.org/grpc" ) func TestT(t *testing.T) { @@ -125,21 +123,6 @@ func (s *testMainSuite) TestRetryOpenStore(c *C) { c.Assert(uint64(elapse), GreaterEqual, uint64(3*time.Second)) } -func (s *testMainSuite) TestRetryDialPumpClient(c *C) { - retryDialPumpClientMustFail := func(binlogSocket string, clientCon *grpc.ClientConn, maxRetries int, dialerOpt grpc.DialOption) (err error) { - return util.RunWithRetry(maxRetries, 10, func() (bool, error) { - // Assume that it'll always return an error. - return true, errors.New("must fail") - }) - } - begin := time.Now() - err := retryDialPumpClientMustFail("", nil, 3, nil) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "must fail") - elapse := time.Since(begin) - c.Assert(uint64(elapse), GreaterEqual, uint64(6*10*time.Millisecond)) -} - func (s *testMainSuite) TestSysSessionPoolGoroutineLeak(c *C) { store, dom := newStoreWithBootstrap(c, s.dbName+"goroutine_leak") defer dom.Close() diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index acd9ae82dc8d4..235614cede25d 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -14,12 +14,15 @@ package binloginfo import ( + "io/ioutil" "regexp" "strings" "sync" "sync/atomic" "time" + "github.com/pingcap/tidb-tools/tidb-binlog/node" + pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" @@ -27,49 +30,39 @@ import ( binlog "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" log "github.com/sirupsen/logrus" - "golang.org/x/net/context" "google.golang.org/grpc" ) func init() { grpc.EnableTracing = false + // don't need output pumps client's log + pumpcli.Logger.Out = ioutil.Discard } -var binlogWriteTimeout = 15 * time.Second - -// pumpClient is the gRPC client to write binlog, it is opened on server start and never close, +// pumpsClient is the client to write binlog, it is opened on server start and never close, // shared by all sessions. -var pumpClient binlog.PumpClient -var pumpClientLock sync.RWMutex +var pumpsClient *pumpcli.PumpsClient +var pumpsClientLock sync.RWMutex // BinlogInfo contains binlog data and binlog client. type BinlogInfo struct { Data *binlog.Binlog - Client binlog.PumpClient + Client *pumpcli.PumpsClient } -// GetPumpClient gets the pump client instance. -func GetPumpClient() binlog.PumpClient { - pumpClientLock.RLock() - client := pumpClient - pumpClientLock.RUnlock() +// GetPumpsClient gets the pumps client instance. +func GetPumpsClient() *pumpcli.PumpsClient { + pumpsClientLock.RLock() + client := pumpsClient + pumpsClientLock.RUnlock() return client } -// SetPumpClient sets the pump client instance. -func SetPumpClient(client binlog.PumpClient) { - pumpClientLock.Lock() - pumpClient = client - pumpClientLock.Unlock() -} - -// SetGRPCTimeout sets grpc timeout for writing binlog. -func SetGRPCTimeout(timeout time.Duration) { - if timeout < 300*time.Millisecond { - log.Warnf("set binlog grpc timeout %s ignored, use default value %s", timeout, binlogWriteTimeout) - return // Avoid invalid value - } - binlogWriteTimeout = timeout +// SetPumpsClient sets the pumps client instance. +func SetPumpsClient(client *pumpcli.PumpsClient) { + pumpsClientLock.Lock() + pumpsClient = client + pumpsClientLock.Unlock() } // GetPrewriteValue gets binlog prewrite value in the context. @@ -111,50 +104,39 @@ func (info *BinlogInfo) WriteBinlog(clusterID uint64) error { return nil } - commitData, err := info.Data.Marshal() - if err != nil { - return errors.Trace(err) - } - req := &binlog.WriteBinlogReq{ClusterID: clusterID, Payload: commitData} - - // Retry many times because we may raise CRITICAL error here. - for i := 0; i < 20; i++ { - var resp *binlog.WriteBinlogResp - ctx, cancel := context.WithTimeout(context.Background(), binlogWriteTimeout) - resp, err = info.Client.WriteBinlog(ctx, req) - cancel() - if err == nil && resp.Errmsg != "" { - err = errors.New(resp.Errmsg) - } - if err == nil { - return nil - } - if strings.Contains(err.Error(), "received message larger than max") { - // This kind of error is not critical and not retryable, return directly. - return errors.Errorf("binlog data is too large (%s)", err.Error()) - } - log.Errorf("write binlog error %v", err) - time.Sleep(time.Second) + if info.Client == nil { + return errors.New("pumps client is nil") } + // it will retry in PumpsClient if write binlog fail. + err := info.Client.WriteBinlog(info.Data) if err != nil { + log.Errorf("write binlog fail %v", errors.ErrorStack(err)) if atomic.LoadUint32(&ignoreError) == 1 { - log.Errorf("critical error, write binlog fail but error ignored: %s", errors.ErrorStack(err)) + log.Error("write binlog fail but error ignored") metrics.CriticalErrorCounter.Add(1) // If error happens once, we'll stop writing binlog. atomic.CompareAndSwapUint32(&skipBinlog, skip, skip+1) return nil } + + if strings.Contains(err.Error(), "received message larger than max") { + // This kind of error is not critical, return directly. + return errors.Errorf("binlog data is too large (%s)", err.Error()) + } + + return terror.ErrCritical.GenWithStackByArgs(err) } - return terror.ErrCritical.GenWithStackByArgs(err) + return nil } // SetDDLBinlog sets DDL binlog in the kv.Transaction. -func SetDDLBinlog(client interface{}, txn kv.Transaction, jobID int64, ddlQuery string) { +func SetDDLBinlog(client *pumpcli.PumpsClient, txn kv.Transaction, jobID int64, ddlQuery string) { if client == nil { return } + ddlQuery = addSpecialComment(ddlQuery) info := &BinlogInfo{ Data: &binlog.Binlog{ @@ -162,7 +144,7 @@ func SetDDLBinlog(client interface{}, txn kv.Transaction, jobID int64, ddlQuery DdlJobId: jobID, DdlQuery: []byte(ddlQuery), }, - Client: client.(binlog.PumpClient), + Client: client, } txn.SetOption(kv.BinlogInfo, info) } @@ -182,3 +164,35 @@ func addSpecialComment(ddlQuery string) string { } return ddlQuery[:loc[0]] + specialPrefix + ddlQuery[loc[0]:loc[1]] + ` */` + ddlQuery[loc[1]:] } + +// MockPumpsClient creates a PumpsClient, used for test. +func MockPumpsClient(client binlog.PumpClient) *pumpcli.PumpsClient { + nodeID := "pump-1" + pump := &pumpcli.PumpStatus{ + Status: node.Status{ + NodeID: nodeID, + State: node.Online, + }, + IsAvaliable: true, + Client: client, + } + + pumpInfos := &pumpcli.PumpInfos{ + Pumps: make(map[string]*pumpcli.PumpStatus), + AvaliablePumps: make(map[string]*pumpcli.PumpStatus), + UnAvaliablePumps: make(map[string]*pumpcli.PumpStatus), + } + pumpInfos.Pumps[nodeID] = pump + pumpInfos.AvaliablePumps[nodeID] = pump + + pCli := &pumpcli.PumpsClient{ + ClusterID: 1, + Pumps: pumpInfos, + Selector: pumpcli.NewSelector(pumpcli.Range), + RetryTime: 1, + BinlogWriteTimeout: 15 * time.Second, + } + pCli.Selector.SetPumps([]*pumpcli.PumpStatus{pump}) + + return pCli +} diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 8650a9afb1ede..a6f1e9cc1f9cf 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -22,6 +22,7 @@ import ( "time" . "github.com/pingcap/check" + pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -81,7 +82,7 @@ type testBinlogSuite struct { unixFile string serv *grpc.Server pump *mockBinlogPump - client binlog.PumpClient + client *pumpcli.PumpsClient ddl ddl.DDL } @@ -112,7 +113,7 @@ func (s *testBinlogSuite) SetUpSuite(c *C) { sessionDomain := domain.GetDomain(tk.Se.(sessionctx.Context)) s.ddl = sessionDomain.DDL() - s.client = binlog.NewPumpClient(clientCon) + s.client = binloginfo.MockPumpsClient(binlog.NewPumpClient(clientCon)) s.ddl.SetBinlogClient(s.client) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 26c0fbefb031d..94037bbef8242 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -21,6 +21,7 @@ import ( "sync/atomic" "time" + pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" @@ -237,7 +238,7 @@ type SessionVars struct { SnapshotInfoschema interface{} // BinlogClient is used to write binlog. - BinlogClient interface{} + BinlogClient *pumpcli.PumpsClient // GlobalVarsAccessor is used to set and get global variables. GlobalVarsAccessor GlobalVarAccessor diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 0eccc2d025d0b..0446a2d5c00d4 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -84,7 +84,7 @@ func (ts *testSuite) TestBasic(c *C) { c.Assert(autoid, Greater, int64(0)) ctx := ts.se - ctx.GetSessionVars().BinlogClient = binloginfo.GetPumpClient() + ctx.GetSessionVars().BinlogClient = binloginfo.GetPumpsClient() ctx.GetSessionVars().InRestrictedSQL = false rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc"), false) c.Assert(err, IsNil) diff --git a/tidb-server/main.go b/tidb-server/main.go index 44bddf593e4e0..f65ae2e7fa085 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -16,7 +16,6 @@ package main import ( "flag" "fmt" - "net" "os" "runtime" "strconv" @@ -24,6 +23,8 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/pd/client" + pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" @@ -41,18 +42,15 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/gcworker" "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/signal" "github.com/pingcap/tidb/util/systimemon" "github.com/pingcap/tidb/x-server" - "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" log "github.com/sirupsen/logrus" - "google.golang.org/grpc" ) // Flag Names @@ -65,7 +63,7 @@ const ( nmAdvertiseAddress = "advertise-address" nmPort = "P" nmSocket = "socket" - nmBinlogSocket = "binlog-socket" + nmEnableBinlog = "enable-binlog" nmRunDDL = "run-ddl" nmLogLevel = "L" nmLogFile = "log-file" @@ -92,7 +90,7 @@ var ( advertiseAddress = flag.String(nmAdvertiseAddress, "", "tidb server advertise IP") port = flag.String(nmPort, "4000", "tidb server port") socket = flag.String(nmSocket, "", "The socket file to use for connection.") - binlogSocket = flag.String(nmBinlogSocket, "", "socket file to write binlog") + enableBinlog = flagBoolean(nmEnableBinlog, false, "enable generate binlog") runDDL = flagBoolean(nmRunDDL, true, "run ddl worker on this tidb-server") ddlLease = flag.String(nmDdlLease, "45s", "schema lease duration, very dangerous to change only if you know what you do") tokenLimit = flag.Int(nmTokenLimit, 1000, "the limit of concurrent executed sessions") @@ -170,20 +168,23 @@ func createStoreAndDomain() { } func setupBinlogClient() { - if cfg.Binlog.BinlogSocket == "" { + if !cfg.Binlog.Enable { return } - dialerOpt := grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) { - return net.DialTimeout("unix", addr, timeout) - }) - clientConn, err := session.DialPumpClientWithRetry(cfg.Binlog.BinlogSocket, util.DefaultMaxRetries, dialerOpt) - terror.MustNil(err) + if cfg.Binlog.IgnoreError { binloginfo.SetIgnoreError(true) } - binloginfo.SetGRPCTimeout(parseDuration(cfg.Binlog.WriteTimeout)) - binloginfo.SetPumpClient(binlog.NewPumpClient(clientConn)) - log.Infof("created binlog client at %s, ignore error %v", cfg.Binlog.BinlogSocket, cfg.Binlog.IgnoreError) + + client, err := pumpcli.NewPumpsClient(cfg.Path, parseDuration(cfg.Binlog.WriteTimeout), pd.SecurityOption{ + CAPath: cfg.Security.ClusterSSLCA, + CertPath: cfg.Security.ClusterSSLCert, + KeyPath: cfg.Security.ClusterSSLKey, + }) + terror.MustNil(err) + + binloginfo.SetPumpsClient(client) + log.Infof("create pumps client success, ignore binlog error %v", cfg.Binlog.IgnoreError) } // Prometheus push. @@ -287,8 +288,8 @@ func overrideConfig() { if actualFlags[nmSocket] { cfg.Socket = *socket } - if actualFlags[nmBinlogSocket] { - cfg.Binlog.BinlogSocket = *binlogSocket + if actualFlags[nmEnableBinlog] { + cfg.Binlog.Enable = *enableBinlog } if actualFlags[nmRunDDL] { cfg.RunDDL = *runDDL From 01c9d82443bf0db71962c3061ae3d32a5ad44932 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 31 Oct 2018 15:28:00 +0800 Subject: [PATCH 045/509] expression: fix cast json to decimal bug. (#8030) (#8109) --- executor/executor_test.go | 15 ++++++++++++ expression/builtin_cast.go | 8 +++---- expression/builtin_cast_test.go | 42 +++++++++++++++++++++++++++++++++ types/convert.go | 15 ++++++++++++ types/convert_test.go | 21 +++++++++++++++++ 5 files changed, 97 insertions(+), 4 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 03ef8c4ec310f..a0f1375277584 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1332,6 +1332,21 @@ func (s *testSuite) TestJSON(c *C) { // check CAST AS JSON. result = tk.MustQuery(`select CAST('3' AS JSON), CAST('{}' AS JSON), CAST(null AS JSON)`) result.Check(testkit.Rows(`3 {} `)) + + // Check cast json to decimal. + tk.MustExec("drop table if exists test_json") + tk.MustExec("create table test_json ( a decimal(60,2) as (JSON_EXTRACT(b,'$.c')), b json );") + tk.MustExec(`insert into test_json (b) values + ('{"c": "1267.1"}'), + ('{"c": "1267.01"}'), + ('{"c": "1267.1234"}'), + ('{"c": "1267.3456"}'), + ('{"c": "1234567890123456789012345678901234567890123456789012345"}'), + ('{"c": "1234567890123456789012345678901234567890123456789012345.12345"}');`) + + tk.MustQuery("select a from test_json;").Check(testkit.Rows("1267.10", "1267.01", "1267.12", + "1267.35", "1234567890123456789012345678901234567890123456789012345.00", + "1234567890123456789012345678901234567890123456789012345.12")) } func (s *testSuite) TestMultiUpdate(c *C) { diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 719ca5a2e9614..b3e8a539b9f7f 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -1541,11 +1541,11 @@ func (b *builtinCastJSONAsDecimalSig) evalDecimal(row chunk.Row) (res *types.MyD return res, isNull, errors.Trace(err) } sc := b.ctx.GetSessionVars().StmtCtx - f64, err := types.ConvertJSONToFloat(sc, val) - if err == nil { - res = new(types.MyDecimal) - err = res.FromFloat64(f64) + res, err = types.ConvertJSONToDecimal(sc, val) + if err != nil { + return res, false, errors.Trace(err) } + res, err = types.ProduceDecWithSpecifiedTp(res, b.tp, sc) return res, false, errors.Trace(err) } diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index ab92b03021295..dd9d00daed014 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -1082,6 +1082,48 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { c.Assert(iRes, Equals, int64(0)) } +func (s *testEvaluatorSuite) TestCastJSONAsDecimalSig(c *C) { + ctx, sc := s.ctx, s.ctx.GetSessionVars().StmtCtx + originIgnoreTruncate := sc.IgnoreTruncate + sc.IgnoreTruncate = true + defer func() { + sc.IgnoreTruncate = originIgnoreTruncate + }() + + col := &Column{RetType: types.NewFieldType(mysql.TypeJSON), Index: 0} + decFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, []Expression{col}), false) + decFunc.tp = types.NewFieldType(mysql.TypeNewDecimal) + decFunc.tp.Flen = 60 + decFunc.tp.Decimal = 2 + sig := &builtinCastJSONAsDecimalSig{decFunc} + + var tests = []struct { + In string + Out *types.MyDecimal + }{ + {`{}`, types.NewDecFromStringForTest("0")}, + {`[]`, types.NewDecFromStringForTest("0")}, + {`3`, types.NewDecFromStringForTest("3")}, + {`-3`, types.NewDecFromStringForTest("-3")}, + {`4.5`, types.NewDecFromStringForTest("4.5")}, + {`"1234"`, types.NewDecFromStringForTest("1234")}, + // test truncate + {`"1234.1234"`, types.NewDecFromStringForTest("1234.12")}, + {`"1234.4567"`, types.NewDecFromStringForTest("1234.46")}, + // test big decimal + {`"1234567890123456789012345678901234567890123456789012345"`, types.NewDecFromStringForTest("1234567890123456789012345678901234567890123456789012345")}, + } + for _, tt := range tests { + j, err := json.ParseBinaryFromString(tt.In) + c.Assert(err, IsNil) + row := chunk.MutRowFromDatums([]types.Datum{types.NewDatum(j)}) + res, isNull, err := sig.evalDecimal(row.ToRow()) + c.Assert(isNull, Equals, false) + c.Assert(err, IsNil) + c.Assert(res.Compare(tt.Out), Equals, 0) + } +} + // TestWrapWithCastAsTypesClasses tests WrapWithCastAsInt/Real/String/Decimal. func (s *testEvaluatorSuite) TestWrapWithCastAsTypesClasses(c *C) { ctx := s.ctx diff --git a/types/convert.go b/types/convert.go index b8eb173c4b7ce..08ec314b927bf 100644 --- a/types/convert.go +++ b/types/convert.go @@ -378,6 +378,21 @@ func ConvertJSONToFloat(sc *stmtctx.StatementContext, j json.BinaryJSON) (float6 return 0, errors.New("Unknown type code in JSON") } +// ConvertJSONToDecimal casts JSON into decimal. +func ConvertJSONToDecimal(sc *stmtctx.StatementContext, j json.BinaryJSON) (*MyDecimal, error) { + res := new(MyDecimal) + if j.TypeCode != json.TypeCodeString { + f64, err := ConvertJSONToFloat(sc, j) + if err != nil { + return res, errors.Trace(err) + } + err = res.FromFloat64(f64) + return res, errors.Trace(err) + } + err := sc.HandleTruncate(res.FromString([]byte(j.GetString()))) + return res, errors.Trace(err) +} + // getValidFloatPrefix gets prefix of string which can be successfully parsed as float. func getValidFloatPrefix(sc *stmtctx.StatementContext, s string) (valid string, err error) { var ( diff --git a/types/convert_test.go b/types/convert_test.go index df35527d7e612..c9103e92c17cb 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -804,6 +804,27 @@ func (s *testTypeConvertSuite) TestConvertJSONToFloat(c *C) { } } +func (s *testTypeConvertSuite) TestConvertJSONToDecimal(c *C) { + var tests = []struct { + In string + Out *MyDecimal + }{ + {`{}`, NewDecFromStringForTest("0")}, + {`[]`, NewDecFromStringForTest("0")}, + {`3`, NewDecFromStringForTest("3")}, + {`-3`, NewDecFromStringForTest("-3")}, + {`4.5`, NewDecFromStringForTest("4.5")}, + {`"1234"`, NewDecFromStringForTest("1234")}, + {`"1234567890123456789012345678901234567890123456789012345"`, NewDecFromStringForTest("1234567890123456789012345678901234567890123456789012345")}, + } + for _, tt := range tests { + j, err := json.ParseBinaryFromString(tt.In) + c.Assert(err, IsNil) + casted, _ := ConvertJSONToDecimal(new(stmtctx.StatementContext), j) + c.Assert(casted.Compare(tt.Out), Equals, 0) + } +} + func (s *testTypeConvertSuite) TestNumberToDuration(c *C) { var testCases = []struct { number int64 From dcc5fc7b55d2b097f330ac54b464c5d823d0d4ae Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 31 Oct 2018 15:42:41 +0800 Subject: [PATCH 046/509] ddl, domain: make schema correct after canceling jobs (#7997) (#8057) --- ddl/ddl_test.go | 10 ++++ ddl/ddl_worker.go | 25 +++++----- ddl/fail_db_test.go | 116 ++++++++++++++++++++++++++++++++++++++++++++ ddl/table.go | 10 ++++ domain/domain.go | 5 ++ 5 files changed, 155 insertions(+), 11 deletions(-) diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index a353102726404..3968a7d04ffc2 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -39,6 +39,8 @@ import ( type DDLForTest interface { // SetHook sets the hook. SetHook(h Callback) + // GetHook gets the hook. + GetHook() Callback // SetInterceptoror sets the interceptor. SetInterceptoror(h Interceptor) } @@ -51,6 +53,14 @@ func (d *ddl) SetHook(h Callback) { d.mu.hook = h } +// GetHook implements DDL.GetHook interface. +func (d *ddl) GetHook() Callback { + d.mu.Lock() + defer d.mu.Unlock() + + return d.mu.hook +} + // SetInterceptoror implements DDL.SetInterceptoror interface. func (d *ddl) SetInterceptoror(i Interceptor) { d.mu.Lock() diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 4237a2f56c42b..128737f065d74 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -276,18 +276,20 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerFinishDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() - switch job.Type { - case model.ActionAddIndex: - if job.State != model.JobStateRollbackDone { - break + if !job.IsCancelled() { + switch job.Type { + case model.ActionAddIndex: + if job.State != model.JobStateRollbackDone { + break + } + // After rolling back an AddIndex operation, we need to use delete-range to delete the half-done index data. + err = w.deleteRange(job) + case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropTablePartition: + err = w.deleteRange(job) + } + if err != nil { + return errors.Trace(err) } - // After rolling back an AddIndex operation, we need to use delete-range to delete the half-done index data. - err = w.deleteRange(job) - case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropTablePartition: - err = w.deleteRange(job) - } - if err != nil { - return errors.Trace(err) } _, err = t.DeQueueDDLJob() @@ -380,6 +382,7 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { // and retry later if the job is not cancelled. schemaVer, runJobErr = w.runDDLJob(d, t, job) if job.IsCancelled() { + txn.Reset() err = w.finishDDLJob(t, job) return errors.Trace(err) } diff --git a/ddl/fail_db_test.go b/ddl/fail_db_test.go index 086f13af894eb..724d59bdcf621 100644 --- a/ddl/fail_db_test.go +++ b/ddl/fail_db_test.go @@ -15,15 +15,131 @@ package ddl_test import ( "fmt" + "time" gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) +var _ = Suite(&testFailDBSuite{}) + +type testFailDBSuite struct { + lease time.Duration + store kv.Storage + dom *domain.Domain + se session.Session + p *parser.Parser +} + +func (s *testFailDBSuite) SetUpSuite(c *C) { + testleak.BeforeTest() + s.lease = 200 * time.Millisecond + ddl.WaitTimeWhenErrorOccured = 1 * time.Microsecond + var err error + s.store, err = mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + session.SetSchemaLease(s.lease) + s.dom, err = session.BootstrapSession(s.store) + c.Assert(err, IsNil) + s.se, err = session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + s.p = parser.New() +} + +func (s *testFailDBSuite) TearDownSuite(c *C) { + s.se.Execute(context.Background(), "drop database if exists test_db_state") + s.se.Close() + s.dom.Close() + s.store.Close() + testleak.AfterTest(c)() +} + +// TestHalfwayCancelOperations tests the case that the schema is correct after the execution of operations are cancelled halfway. +func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { + gofail.Enable("github.com/pingcap/tidb/ddl/truncateTableErr", `return(true)`) + defer gofail.Disable("github.com/pingcap/tidb/ddl/truncateTableErr") + + // test for truncating table + _, err := s.se.Execute(context.Background(), "create database cancel_job_db") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "use cancel_job_db") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "create table t(a int)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "insert into t values(1)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "truncate table t") + c.Assert(err, NotNil) + // Make sure that the table's data has not been deleted. + rs, err := s.se.Execute(context.Background(), "select count(*) from t") + c.Assert(err, IsNil) + chk := rs[0].NewChunk() + err = rs[0].Next(context.Background(), chk) + c.Assert(err, IsNil) + c.Assert(chk.NumRows() == 0, IsFalse) + row := chk.GetRow(0) + c.Assert(row.Len(), Equals, 1) + c.Assert(row.GetInt64(0), DeepEquals, int64(1)) + c.Assert(rs[0].Close(), IsNil) + // Reload schema. + s.dom.ResetHandle(s.store) + err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil) + c.Assert(err, IsNil) + s.se, err = session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "use cancel_job_db") + c.Assert(err, IsNil) + // Test schema is correct. + _, err = s.se.Execute(context.Background(), "select * from t") + c.Assert(err, IsNil) + + // test for renaming table + gofail.Enable("github.com/pingcap/tidb/ddl/errRenameTable", `return(true)`) + defer gofail.Disable("github.com/pingcap/tidb/ddl/errRenameTable") + _, err = s.se.Execute(context.Background(), "create table tx(a int)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "insert into tx values(1)") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "rename table tx to ty") + c.Assert(err, NotNil) + // Make sure that the table's data has not been deleted. + rs, err = s.se.Execute(context.Background(), "select count(*) from tx") + c.Assert(err, IsNil) + chk = rs[0].NewChunk() + err = rs[0].Next(context.Background(), chk) + c.Assert(err, IsNil) + c.Assert(chk.NumRows() == 0, IsFalse) + row = chk.GetRow(0) + c.Assert(row.Len(), Equals, 1) + c.Assert(row.GetInt64(0), DeepEquals, int64(1)) + c.Assert(rs[0].Close(), IsNil) + // Reload schema. + s.dom.ResetHandle(s.store) + err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil) + c.Assert(err, IsNil) + s.se, err = session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "use cancel_job_db") + c.Assert(err, IsNil) + // Test schema is correct. + _, err = s.se.Execute(context.Background(), "select * from tx") + c.Assert(err, IsNil) + + // clean up + _, err = s.se.Execute(context.Background(), "drop database cancel_job_db") + c.Assert(err, IsNil) +} + // TestInitializeOffsetAndState tests the case that the column's offset and state don't be initialized in the file of ddl_api.go when // doing the operation of 'modify column'. func (s *testStateChangeSuite) TestInitializeOffsetAndState(c *C) { diff --git a/ddl/table.go b/ddl/table.go index 0bba885b07c53..cf7643f1a5ff4 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -208,6 +208,11 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // gofail: var truncateTableErr bool + // if truncateTableErr { + // job.State = model.JobStateCancelled + // return ver, errors.New("occur an error after dropping table.") + // } var oldPartitionIDs []int64 if tblInfo.GetPartitionInfo() != nil { @@ -334,6 +339,11 @@ func onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // gofail: var renameTableErr bool + // if renameTableErr { + // job.State = model.JobStateCancelled + // return ver, errors.New("occur an error after renaming table.") + // } tblInfo.Name = tableName err = t.CreateTable(newSchemaID, tblInfo) if err != nil { diff --git a/domain/domain.go b/domain/domain.go index 8061751fc852b..280112c45a76f 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -533,6 +533,11 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio } } +// ResetHandle resets the domain's infoschema handle. It is used for testing. +func (do *Domain) ResetHandle(store kv.Storage) { + do.infoHandle = infoschema.NewHandle(store) +} + // Init initializes a domain. func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.Resource, error)) error { if ebd, ok := do.store.(EtcdBackend); ok { From adb5733443d58a6cde6639df1b7760790829b381 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Thu, 1 Nov 2018 16:30:24 +0800 Subject: [PATCH 047/509] planner/core: fix index resolution on PhysicalIndexReader (#8118) (#8132) --- executor/executor_test.go | 17 +++++++++++++++++ planner/core/resolve_indices.go | 9 +-------- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index a0f1375277584..c6f5f9b688e32 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3229,3 +3229,20 @@ func (s *testSuite) TestCurrentTimestampValueSelection(c *C) { c.Assert(strings.Split(b, ".")[1], Equals, "00") c.Assert(len(strings.Split(d, ".")[1]), Equals, 3) } + +func (s *testSuite) TestRowID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(a varchar(10), b varchar(10), c varchar(1), index idx(a, b, c));`) + tk.MustExec(`insert into t values('a', 'b', 'c');`) + tk.MustExec(`insert into t values('a', 'b', 'c');`) + tk.MustQuery(`select b, _tidb_rowid from t use index(idx) where a = 'a';`).Check(testkit.Rows( + `b 1`, + `b 2`, + )) + tk.MustExec(`begin;`) + tk.MustExec(`select * from t for update`) + tk.MustQuery(`select distinct b from t use index(idx) where a = 'a';`).Check(testkit.Rows(`b`)) + tk.MustExec(`commit;`) +} diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index d11f3c49d453e..97ca5c1619500 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -15,7 +15,6 @@ package core import ( "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" ) // ResolveIndices implements Plan interface. @@ -107,13 +106,7 @@ func (p *PhysicalIndexReader) ResolveIndices() { p.physicalSchemaProducer.ResolveIndices() p.indexPlan.ResolveIndices() for i, col := range p.OutputColumns { - if col.ID != model.ExtraHandleID { - p.OutputColumns[i] = col.ResolveIndices(p.indexPlan.Schema()).(*expression.Column) - } else { - p.OutputColumns[i] = col.Clone().(*expression.Column) - // If this is extra handle, then it must be the tail. - p.OutputColumns[i].Index = len(p.OutputColumns) - 1 - } + p.OutputColumns[i] = col.ResolveIndices(p.indexPlan.Schema()).(*expression.Column) } } From dfd223e039d8edf83d0360c5caf14cc1e641f606 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 2 Nov 2018 15:08:35 +0800 Subject: [PATCH 048/509] expression, util: move disjoint set to util package(#7950) (#8148) --- expression/constant_propagation.go | 39 +++++------------------ util/disjointset/int_set.go | 42 ++++++++++++++++++++++++ util/disjointset/int_set_test.go | 51 ++++++++++++++++++++++++++++++ 3 files changed, 101 insertions(+), 31 deletions(-) create mode 100644 util/disjointset/int_set.go create mode 100644 util/disjointset/int_set_test.go diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 2919c5dc359c8..8bf71a42c4abf 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/disjointset" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) @@ -27,34 +28,11 @@ import ( // MaxPropagateColsCnt means the max number of columns that can participate propagation. var MaxPropagateColsCnt = 100 -type multiEqualSet struct { - parent []int -} - -func (m *multiEqualSet) init(l int) { - m.parent = make([]int, l) - for i := range m.parent { - m.parent[i] = i - } -} - -func (m *multiEqualSet) addRelation(a int, b int) { - m.parent[m.findRoot(a)] = m.findRoot(b) -} - -func (m *multiEqualSet) findRoot(a int) int { - if a == m.parent[a] { - return a - } - m.parent[a] = m.findRoot(m.parent[a]) - return m.parent[a] -} - type propagateConstantSolver struct { - colMapper map[string]int // colMapper maps column to its index - unionSet *multiEqualSet // unionSet stores the relations like col_i = col_j - eqList []*Constant // if eqList[i] != nil, it means col_i = eqList[i] - columns []*Column // columns stores all columns appearing in the conditions + colMapper map[string]int // colMapper maps column to its index + unionSet *disjointset.IntSet // unionSet stores the relations like col_i = col_j + eqList []*Constant // if eqList[i] != nil, it means col_i = eqList[i] + columns []*Column // columns stores all columns appearing in the conditions conditions []Expression ctx sessionctx.Context } @@ -106,8 +84,7 @@ func (s *propagateConstantSolver) propagateConstantEQ() { // We maintain a unionSet representing the equivalent for every two columns. func (s *propagateConstantSolver) propagateColumnEQ() { visited := make([]bool, len(s.conditions)) - s.unionSet = &multiEqualSet{} - s.unionSet.init(len(s.columns)) + s.unionSet = disjointset.NewIntSet(len(s.columns)) for i := range s.conditions { if fun, ok := s.conditions[i].(*ScalarFunction); ok && fun.FuncName.L == ast.EQ { lCol, lOk := fun.GetArgs()[0].(*Column) @@ -115,7 +92,7 @@ func (s *propagateConstantSolver) propagateColumnEQ() { if lOk && rOk { lID := s.getColID(lCol) rID := s.getColID(rCol) - s.unionSet.addRelation(lID, rID) + s.unionSet.Union(lID, rID) visited[i] = true } } @@ -125,7 +102,7 @@ func (s *propagateConstantSolver) propagateColumnEQ() { for i, coli := range s.columns { for j := i + 1; j < len(s.columns); j++ { // unionSet doesn't have iterate(), we use a two layer loop to iterate col_i = col_j relation - if s.unionSet.findRoot(i) != s.unionSet.findRoot(j) { + if s.unionSet.FindRoot(i) != s.unionSet.FindRoot(j) { continue } colj := s.columns[j] diff --git a/util/disjointset/int_set.go b/util/disjointset/int_set.go new file mode 100644 index 0000000000000..0881b4aa1b03f --- /dev/null +++ b/util/disjointset/int_set.go @@ -0,0 +1,42 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package disjointset + +// IntSet is the int disjoint set. +type IntSet struct { + parent []int +} + +// NewIntSet returns a new int disjoint set. +func NewIntSet(size int) *IntSet { + p := make([]int, size) + for i := range p { + p[i] = i + } + return &IntSet{parent: p} +} + +// Union unions two sets in int disjoint set. +func (m *IntSet) Union(a int, b int) { + m.parent[m.FindRoot(a)] = m.FindRoot(b) +} + +// FindRoot finds the representative element of the set that `a` belongs to. +func (m *IntSet) FindRoot(a int) int { + if a == m.parent[a] { + return a + } + m.parent[a] = m.FindRoot(m.parent[a]) + return m.parent[a] +} diff --git a/util/disjointset/int_set_test.go b/util/disjointset/int_set_test.go new file mode 100644 index 0000000000000..9ea0287cf61da --- /dev/null +++ b/util/disjointset/int_set_test.go @@ -0,0 +1,51 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package disjointset + +import ( + . "github.com/pingcap/check" + "testing" +) + +var _ = Suite(&testDisjointSetSuite{}) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +type testDisjointSetSuite struct { +} + +func (s *testDisjointSetSuite) TestIntDisjointSet(c *C) { + set := NewIntSet(10) + c.Assert(len(set.parent), Equals, 10) + for i := range set.parent { + c.Assert(set.parent[i], Equals, i) + } + set.Union(0, 1) + set.Union(1, 3) + set.Union(4, 2) + set.Union(2, 6) + set.Union(3, 5) + set.Union(7, 8) + set.Union(9, 6) + c.Assert(set.FindRoot(0), Equals, set.FindRoot(1)) + c.Assert(set.FindRoot(3), Equals, set.FindRoot(1)) + c.Assert(set.FindRoot(5), Equals, set.FindRoot(1)) + c.Assert(set.FindRoot(2), Equals, set.FindRoot(4)) + c.Assert(set.FindRoot(6), Equals, set.FindRoot(4)) + c.Assert(set.FindRoot(9), Equals, set.FindRoot(2)) + c.Assert(set.FindRoot(7), Equals, set.FindRoot(8)) +} From f48a51045b458be813b23cb8b5f0da14e3df30ed Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 2 Nov 2018 15:36:26 +0800 Subject: [PATCH 049/509] planner: refine PhysicalProjection.ResolveIndices during neighbouringProj(#8073) (#8152) --- executor/executor_test.go | 18 +++++++++++++++ planner/core/resolve_indices.go | 39 +++++++++++++++++++++++++++++++++ 2 files changed, 57 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index c6f5f9b688e32..f60689229c631 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1046,6 +1046,24 @@ func (s *testSuite) TestUnion(c *C) { tk.MustQuery("select a, b from (select a, 0 as d, b from t union all select a, 0 as d, b from t) test;").Check(testkit.Rows("1 2", "1 2")) } +func (s *testSuite) TestNeighbouringProj(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("create table t2(a int, b int)") + tk.MustExec("insert into t1 value(1, 1), (2, 2)") + tk.MustExec("insert into t2 value(1, 1), (2, 2)") + tk.MustQuery("select sum(c) from (select t1.a as a, t1.a as c, length(t1.b) from t1 union select a, b, b from t2) t;").Check(testkit.Rows("5")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint, b bigint, c bigint);") + tk.MustExec("insert into t values(1, 1, 1), (2, 2, 2), (3, 3, 3);") + rs := tk.MustQuery("select cast(count(a) as signed), a as another, a from t group by a order by cast(count(a) as signed), a limit 10;") + rs.Check(testkit.Rows("1 1 1", "1 2 2", "1 3 3")) +} + func (s *testSuite) TestIn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 97ca5c1619500..27562bc21838b 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -15,6 +15,7 @@ package core import ( "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/util/disjointset" ) // ResolveIndices implements Plan interface. @@ -23,6 +24,44 @@ func (p *PhysicalProjection) ResolveIndices() { for i, expr := range p.Exprs { p.Exprs[i] = expr.ResolveIndices(p.children[0].Schema()) } + childProj, isProj := p.children[0].(*PhysicalProjection) + if !isProj { + return + } + refine4NeighbourProj(p, childProj) +} + +// refine4NeighbourProj refines the index for p.Exprs whose type is *Column when +// there is two neighbouring Projections. +// This function is introduced because that different childProj.Expr may refer +// to the same index of childProj.Schema, so we need to keep this relation +// between the specified expressions in the parent Projection. +func refine4NeighbourProj(p, childProj *PhysicalProjection) { + inputIdx2OutputIdxes := make(map[int][]int) + for i, expr := range childProj.Exprs { + col, isCol := expr.(*expression.Column) + if !isCol { + continue + } + inputIdx2OutputIdxes[col.Index] = append(inputIdx2OutputIdxes[col.Index], i) + } + childSchemaUnionSet := disjointset.NewIntSet(childProj.schema.Len()) + for _, outputIdxes := range inputIdx2OutputIdxes { + if len(outputIdxes) <= 1 { + continue + } + for i := 1; i < len(outputIdxes); i++ { + childSchemaUnionSet.Union(outputIdxes[0], outputIdxes[i]) + } + } + + for _, expr := range p.Exprs { + col, isCol := expr.(*expression.Column) + if !isCol { + continue + } + col.Index = childSchemaUnionSet.FindRoot(col.Index) + } } // ResolveIndices implements Plan interface. From cc0949dd7d6d887afa0efb3ea58acdba3f45ef8d Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Fri, 2 Nov 2018 17:28:02 +0800 Subject: [PATCH 050/509] stats: fix selectivity estimation for primary key (#8134) (#8149) --- statistics/bootstrap.go | 2 +- statistics/dump.go | 1 + statistics/handle.go | 2 +- statistics/histogram.go | 5 +++-- statistics/selectivity.go | 2 +- statistics/selectivity_test.go | 17 +++++++++++++++++ statistics/table.go | 4 +++- 7 files changed, 27 insertions(+), 6 deletions(-) diff --git a/statistics/bootstrap.go b/statistics/bootstrap.go index 850ad1a043a71..c4322a03acc56 100644 --- a/statistics/bootstrap.go +++ b/statistics/bootstrap.go @@ -120,7 +120,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables stat continue } hist := NewHistogram(id, ndv, nullCount, version, &colInfo.FieldType, 0, totColSize) - table.Columns[hist.ID] = &Column{Histogram: *hist, Info: colInfo, Count: nullCount} + table.Columns[hist.ID] = &Column{Histogram: *hist, Info: colInfo, Count: nullCount, isHandle: tbl.Meta().PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag)} } } } diff --git a/statistics/dump.go b/statistics/dump.go index 70a04e87b21c8..7e1fedd83b93d 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -216,6 +216,7 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *J CMSketch: CMSketchFromProto(jsonCol.CMSketch), Info: colInfo, Count: count, + isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), } tbl.Columns[col.ID] = col } diff --git a/statistics/handle.go b/statistics/handle.go index b014d863c97e5..0a612500d1e0b 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -258,7 +258,7 @@ func (h *Handle) LoadNeededHistograms() error { if err != nil { return errors.Trace(err) } - tbl.Columns[c.ID] = &Column{Histogram: *hg, Info: c.Info, CMSketch: cms, Count: int64(hg.totalRowCount())} + tbl.Columns[c.ID] = &Column{Histogram: *hg, Info: c.Info, CMSketch: cms, Count: int64(hg.totalRowCount()), isHandle: c.isHandle} h.UpdateTableStats([]*Table{tbl}, nil) histogramNeededColumns.delete(col) } diff --git a/statistics/histogram.go b/statistics/histogram.go index 56ae7c104bf35..f0c304d4da365 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -720,8 +720,9 @@ func (e *ErrorRate) merge(rate *ErrorRate) { type Column struct { Histogram *CMSketch - Count int64 - Info *model.ColumnInfo + Count int64 + Info *model.ColumnInfo + isHandle bool ErrorRate } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index ef474455d98ea..f7e87a4913308 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -180,7 +180,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp return 0, errors.Trace(err) } sets = append(sets, &exprSet{tp: colType, ID: id, mask: maskCovered, ranges: ranges, numCols: 1}) - if mysql.HasPriKeyFlag(colInfo.Info.Flag) { + if colInfo.isHandle { sets[len(sets)-1].tp = pkType } } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 0a2a4e13c07a9..128f87ec55c92 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -324,6 +324,23 @@ func (s *testSelectivitySuite) TestEstimationForUnknownValues(c *C) { c.Assert(count, Equals, 0.0) } +func (s *testSelectivitySuite) TestPrimaryKeySelectivity(c *C) { + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a char(10) primary key, b int)") + testKit.MustQuery(`explain select * from t where a > "t"`).Check(testkit.Rows( + "IndexLookUp_10 3333.33 root ", + "├─IndexScan_8 3333.33 cop table:t, index:a, range:(\"t\",+inf], keep order:false, stats:pseudo", + "└─TableScan_9 3333.33 cop table:t, keep order:false, stats:pseudo")) + + testKit.MustExec("drop table t") + testKit.MustExec("create table t(a int primary key, b int)") + testKit.MustQuery(`explain select * from t where a > 1`).Check(testkit.Rows( + "TableReader_6 3333.33 root data:TableScan_5", + "└─TableScan_5 3333.33 cop table:t, range:(1,+inf], keep order:false, stats:pseudo")) +} + func BenchmarkSelectivity(b *testing.B) { c := &C{} s := &testSelectivitySuite{} diff --git a/statistics/table.go b/statistics/table.go index 1d05d1b284e6d..65c84a06499c6 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -188,6 +188,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * Info: colInfo, Count: count + nullCount, ErrorRate: errorRate, + isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), } break } @@ -206,6 +207,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * CMSketch: cms, Count: int64(hg.totalRowCount()), ErrorRate: errorRate, + isHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag), } break } @@ -610,7 +612,7 @@ func PseudoTable(tblInfo *model.TableInfo) *Table { } for _, col := range tblInfo.Columns { if col.State == model.StatePublic { - t.Columns[col.ID] = &Column{Info: col} + t.Columns[col.ID] = &Column{Info: col, isHandle: tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag)} } } for _, idx := range tblInfo.Indices { From 2f7d4b9ca25b65ffdd9088c2b36519563ec1003a Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Sat, 3 Nov 2018 20:35:50 +0800 Subject: [PATCH 051/509] Makefile: explicitly disable go module in release-2.1 branch (#8162) --- Makefile | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/Makefile b/Makefile index a3013863c404d..ad524f99eaa18 100644 --- a/Makefile +++ b/Makefile @@ -12,9 +12,9 @@ path_to_add := $(addsuffix /bin,$(subst :,/bin:,$(GOPATH))) export PATH := $(path_to_add):$(PATH) GO := go -GOBUILD := CGO_ENABLED=0 $(GO) build $(BUILD_FLAG) -GOTEST := CGO_ENABLED=1 $(GO) test -p 3 -OVERALLS := CGO_ENABLED=1 overalls +GOBUILD := GO111MODULE=off CGO_ENABLED=0 $(GO) build $(BUILD_FLAG) +GOTEST := GO111MODULE=off CGO_ENABLED=1 $(GO) test -p 3 +OVERALLS := GO111MODULE=off CGO_ENABLED=1 overalls GOVERALLS := goveralls ARCH := "`uname -s`" @@ -122,6 +122,7 @@ vet: clean: $(GO) clean -i ./... rm -rf *.out + rm -rf parser/parser.go todo: @grep -n ^[[:space:]]*_[[:space:]]*=[[:space:]][[:alpha:]][[:alnum:]]* */*.go parser/parser.y || true @@ -171,13 +172,13 @@ tikv_integration_test: parserlib $(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) -RACE_FLAG = +RACE_FLAG = ifeq ("$(WITH_RACE)", "1") RACE_FLAG = -race GOBUILD = GOPATH=$(GOPATH) CGO_ENABLED=1 $(GO) build endif -CHECK_FLAG = +CHECK_FLAG = ifeq ("$(WITH_CHECK)", "1") CHECK_FLAG = $(TEST_LDFLAGS) endif From 3be9756ec633395fd76b62b4af08e6353bc6d9e4 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 5 Nov 2018 16:00:12 +0800 Subject: [PATCH 052/509] expression: fix "values" function in non-insert statement (#8019) (#8169) --- expression/builtin_other.go | 21 +++++++++++++++++++++ expression/builtin_other_test.go | 26 ++++++++++++++++++++------ expression/integration_test.go | 10 ++++++++++ 3 files changed, 51 insertions(+), 6 deletions(-) diff --git a/expression/builtin_other.go b/expression/builtin_other.go index 8afc8e9c0cce6..7e07abeae88cc 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -512,6 +512,9 @@ func (b *builtinValuesIntSig) Clone() builtinFunc { // evalInt evals a builtinValuesIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesIntSig) evalInt(_ chunk.Row) (int64, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return 0, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return 0, true, errors.New("Session current insert values is nil") @@ -540,6 +543,9 @@ func (b *builtinValuesRealSig) Clone() builtinFunc { // evalReal evals a builtinValuesRealSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesRealSig) evalReal(_ chunk.Row) (float64, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return 0, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return 0, true, errors.New("Session current insert values is nil") @@ -568,6 +574,9 @@ func (b *builtinValuesDecimalSig) Clone() builtinFunc { // evalDecimal evals a builtinValuesDecimalSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesDecimalSig) evalDecimal(_ chunk.Row) (*types.MyDecimal, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return nil, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return nil, true, errors.New("Session current insert values is nil") @@ -596,6 +605,9 @@ func (b *builtinValuesStringSig) Clone() builtinFunc { // evalString evals a builtinValuesStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesStringSig) evalString(_ chunk.Row) (string, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return "", true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return "", true, errors.New("Session current insert values is nil") @@ -624,6 +636,9 @@ func (b *builtinValuesTimeSig) Clone() builtinFunc { // evalTime evals a builtinValuesTimeSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesTimeSig) evalTime(_ chunk.Row) (types.Time, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return types.Time{}, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return types.Time{}, true, errors.New("Session current insert values is nil") @@ -652,6 +667,9 @@ func (b *builtinValuesDurationSig) Clone() builtinFunc { // evalDuration evals a builtinValuesDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesDurationSig) evalDuration(_ chunk.Row) (types.Duration, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return types.Duration{}, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return types.Duration{}, true, errors.New("Session current insert values is nil") @@ -681,6 +699,9 @@ func (b *builtinValuesJSONSig) Clone() builtinFunc { // evalJSON evals a builtinValuesJSONSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesJSONSig) evalJSON(_ chunk.Row) (json.BinaryJSON, bool, error) { + if !b.ctx.GetSessionVars().StmtCtx.InInsertStmt { + return json.BinaryJSON{}, true, nil + } row := b.ctx.GetSessionVars().CurrInsertValues if row.IsEmpty() { return json.BinaryJSON{}, true, errors.New("Session current insert values is nil") diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index b0b3494cbed92..893dad92110f3 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -14,7 +14,6 @@ package expression import ( - "fmt" "math" "time" @@ -202,20 +201,35 @@ func (s *testEvaluatorSuite) TestGetVar(c *C) { func (s *testEvaluatorSuite) TestValues(c *C) { defer testleak.AfterTest(c)() + + origin := s.ctx.GetSessionVars().StmtCtx.InInsertStmt + s.ctx.GetSessionVars().StmtCtx.InInsertStmt = false + defer func() { + s.ctx.GetSessionVars().StmtCtx.InInsertStmt = origin + }() + fc := &valuesFunctionClass{baseFunctionClass{ast.Values, 0, 0}, 1, types.NewFieldType(mysql.TypeVarchar)} _, err := fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(""))) c.Assert(err, ErrorMatches, "*Incorrect parameter count in the call to native function 'values'") + sig, err := fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums())) c.Assert(err, IsNil) - _, err = evalBuiltinFunc(sig, chunk.Row{}) - c.Assert(err.Error(), Equals, "Session current insert values is nil") + + ret, err := evalBuiltinFunc(sig, chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(ret.IsNull(), IsTrue) + s.ctx.GetSessionVars().CurrInsertValues = chunk.MutRowFromDatums(types.MakeDatums("1")).ToRow() - _, err = evalBuiltinFunc(sig, chunk.Row{}) - c.Assert(err.Error(), Equals, fmt.Sprintf("Session current insert values len %d and column's offset %v don't match", 1, 1)) + ret, err = evalBuiltinFunc(sig, chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(ret.IsNull(), IsTrue) + currInsertValues := types.MakeDatums("1", "2") + s.ctx.GetSessionVars().StmtCtx.InInsertStmt = true s.ctx.GetSessionVars().CurrInsertValues = chunk.MutRowFromDatums(currInsertValues).ToRow() - ret, err := evalBuiltinFunc(sig, chunk.Row{}) + ret, err = evalBuiltinFunc(sig, chunk.Row{}) c.Assert(err, IsNil) + cmp, err := ret.CompareDatum(nil, &currInsertValues[1]) c.Assert(err, IsNil) c.Assert(cmp, Equals, 0) diff --git a/expression/integration_test.go b/expression/integration_test.go index 90d9a213506d9..f2ac973bbff5d 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -3604,3 +3604,13 @@ func (s *testIntegrationSuite) TestDecimalMul(c *C) { res := tk.MustQuery("select * from t;") res.Check(testkit.Rows("0.55125221922461136")) } + +func (s *testIntegrationSuite) TestValuesInNonInsertStmt(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a bigint, b double, c decimal, d varchar(20), e datetime, f time, g json);`) + tk.MustExec(`insert into t values(1, 1.1, 2.2, "abc", "2018-10-24", NOW(), "12");`) + res := tk.MustQuery(`select values(a), values(b), values(c), values(d), values(e), values(f), values(g) from t;`) + res.Check(testkit.Rows(` `)) +} From 15c5ee219bd67049332aa8a84b168ccc2178c506 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 5 Nov 2018 17:10:21 +0800 Subject: [PATCH 053/509] *: add a variable tidb_slow_log_threshold to set the slow log threshold dynamically (#8094) (#8173) --- config/config.go | 4 ++-- executor/adapter.go | 3 ++- executor/set_test.go | 7 +++++++ sessionctx/variable/session.go | 3 +++ sessionctx/variable/sysvar.go | 2 ++ sessionctx/variable/tidb_vars.go | 5 ++++- sessionctx/variable/varsutil.go | 4 +++- util/logutil/log.go | 2 ++ 8 files changed, 25 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index d4b4b0a003ed9..5b849a5fef110 100644 --- a/config/config.go +++ b/config/config.go @@ -87,7 +87,7 @@ type Log struct { File logutil.FileLogConfig `toml:"file" json:"file"` SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"` - SlowThreshold uint `toml:"slow-threshold" json:"slow-threshold"` + SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"` ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"` QueryLogMaxLen uint `toml:"query-log-max-len" json:"query-log-max-len"` } @@ -273,7 +273,7 @@ var defaultConf = Config{ LogRotate: true, MaxSize: logutil.DefaultLogMaxSize, }, - SlowThreshold: 300, + SlowThreshold: logutil.DefaultSlowThreshold, ExpensiveThreshold: 10000, QueryLogMaxLen: 2048, }, diff --git a/executor/adapter.go b/executor/adapter.go index 0f7c73d9a83ee..95e76acdaadb1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -17,6 +17,7 @@ import ( "fmt" "math" "strings" + "sync/atomic" "time" "github.com/pingcap/tidb/ast" @@ -341,7 +342,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { } cfg := config.GetGlobalConfig() costTime := time.Since(a.StartTime) - threshold := time.Duration(cfg.Log.SlowThreshold) * time.Millisecond + threshold := time.Duration(atomic.LoadUint64(&cfg.Log.SlowThreshold)) * time.Millisecond if costTime < threshold && level < log.DebugLevel { return } diff --git a/executor/set_test.go b/executor/set_test.go index 3ef60edfb8ce6..bd6d0f9dc64cb 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -234,6 +234,13 @@ func (s *testSuite) TestSetVar(c *C) { tk.MustQuery(`select @@tidb_force_priority;`).Check(testkit.Rows("NO_PRIORITY")) _, err = tk.Exec(`set global tidb_force_priority = ""`) c.Assert(err, NotNil) + + tk.MustExec("set tidb_slow_log_threshold = 0") + tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("0")) + tk.MustExec("set tidb_slow_log_threshold = 1") + tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("1")) + _, err = tk.Exec("set global tidb_slow_log_threshold = 0") + c.Assert(err, NotNil) } func (s *testSuite) TestSetCharset(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 94037bbef8242..89b1224109469 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" ) @@ -585,6 +586,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.MemQuotaNestedLoopApply = tidbOptInt64(val, DefTiDBMemQuotaNestedLoopApply) case TiDBGeneralLog: atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt32(val, DefTiDBGeneralLog))) + case TiDBSlowLogThreshold: + atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) case TiDBRetryLimit: s.RetryLimit = tidbOptInt64(val, DefTiDBRetryLimit) case TiDBDisableTxnAutoRetry: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index f3a25e17db457..2ab0c4d9fed9d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/charset" + "github.com/pingcap/tidb/util/logutil" ) // ScopeFlag is for system variable whether can be changed in global/session dynamically or not. @@ -660,6 +661,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBOptimizerSelectivityLevel, strconv.Itoa(DefTiDBOptimizerSelectivityLevel)}, /* The following variable is defined as session scope but is actually server scope. */ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, + {ScopeSession, TiDBSlowLogThreshold, strconv.Itoa(logutil.DefaultSlowThreshold)}, {ScopeSession, TiDBConfig, ""}, {ScopeGlobal | ScopeSession, TiDBDDLReorgWorkerCount, strconv.Itoa(DefTiDBDDLReorgWorkerCount)}, {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 3cb5314d07273..8b1ea4371c3d6 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -95,7 +95,10 @@ const ( // tidb_general_log is used to log every query in the server in info level. TiDBGeneralLog = "tidb_general_log" - // tidb_retry_limit is the maximun number of retries when committing a transaction. + // tidb_slow_log_threshold is used to set the slow log threshold in the server. + TiDBSlowLogThreshold = "tidb_slow_log_threshold" + + // tidb_retry_limit is the maximum number of retries when committing a transaction. TiDBRetryLimit = "tidb_retry_limit" // tidb_disable_txn_auto_retry disables transaction auto retry. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index d956701cb358d..e8766de4fccc4 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -85,6 +85,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return string(j), true, nil case TiDBForcePriority: return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], true, nil + case TiDBSlowLogThreshold: + return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), true, nil } sVal, ok := s.systems[key] if ok { @@ -325,7 +327,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaNestedLoopApply, - TiDBRetryLimit: + TiDBRetryLimit, TiDBSlowLogThreshold: _, err := strconv.ParseInt(value, 10, 64) if err != nil { return value, ErrWrongValueForVar.GenWithStackByArgs(name) diff --git a/util/logutil/log.go b/util/logutil/log.go index d9681cd56009b..610c43b91bcc1 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -33,6 +33,8 @@ const ( DefaultLogMaxSize = 300 // MB defaultLogFormat = "text" defaultLogLevel = log.InfoLevel + // DefaultSlowThreshold is the default slow log threshold in millisecond. + DefaultSlowThreshold = 300 ) // FileLogConfig serializes file log related config in toml/json. From 5bfe2fe1c3b6200e5ad3c4f1dca8c0f18fd7dad4 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Mon, 5 Nov 2018 18:14:30 +0800 Subject: [PATCH 054/509] *: avoid using columnEvaluator for the Projectin build by buildProjtion4Union (#8142) (#8165) --- executor/builder.go | 4 +-- executor/executor_test.go | 7 +++++ executor/projection.go | 4 +-- expression/evaluator.go | 42 +++++++++++++------------- planner/core/exhaust_physical_plans.go | 5 +-- planner/core/logical_plan_builder.go | 2 +- planner/core/logical_plans.go | 7 +++++ planner/core/physical_plans.go | 5 +-- 8 files changed, 46 insertions(+), 30 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 8fb430f113633..68af3a6bfd888 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -981,7 +981,7 @@ func (b *executorBuilder) buildProjBelowAgg(aggFuncs []*aggregation.AggFuncDesc, return &ProjectionExec{ baseExecutor: newBaseExecutor(b.ctx, expression.NewSchema(projSchemaCols...), projFromID, src), - evaluatorSuit: expression.NewEvaluatorSuit(projExprs), + evaluatorSuit: expression.NewEvaluatorSuite(projExprs, false), } } @@ -1112,7 +1112,7 @@ func (b *executorBuilder) buildProjection(v *plannercore.PhysicalProjection) Exe e := &ProjectionExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), childExec), numWorkers: b.ctx.GetSessionVars().ProjectionConcurrency, - evaluatorSuit: expression.NewEvaluatorSuit(v.Exprs), + evaluatorSuit: expression.NewEvaluatorSuite(v.Exprs, v.AvoidColumnEvaluator), calculateNoDelay: v.CalculateNoDelay, } diff --git a/executor/executor_test.go b/executor/executor_test.go index f60689229c631..311df7dcfffbf 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1044,6 +1044,13 @@ func (s *testSuite) TestUnion(c *C) { tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t value(1 ,2)") tk.MustQuery("select a, b from (select a, 0 as d, b from t union all select a, 0 as d, b from t) test;").Check(testkit.Rows("1 2", "1 2")) + + // #issue 8141 + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("insert into t1 value(1,2),(1,1),(2,2),(2,2),(3,2),(3,2)") + tk.MustExec("set @@tidb_max_chunk_size=2;") + tk.MustQuery("select count(*) from (select a as c, a as d from t1 union all select a, b from t1) t;").Check(testkit.Rows("12")) } func (s *testSuite) TestNeighbouringProj(c *C) { diff --git a/executor/projection.go b/executor/projection.go index dce2709f1271b..4b3506c999cba 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -49,7 +49,7 @@ type projectionOutput struct { type ProjectionExec struct { baseExecutor - evaluatorSuit *expression.EvaluatorSuit + evaluatorSuit *expression.EvaluatorSuite calculateNoDelay bool prepared bool @@ -295,7 +295,7 @@ func (f *projectionInputFetcher) run(ctx context.Context) { type projectionWorker struct { sctx sessionctx.Context - evaluatorSuit *expression.EvaluatorSuit + evaluatorSuit *expression.EvaluatorSuite globalFinishCh <-chan struct{} inputGiveBackCh chan<- *projectionInput diff --git a/expression/evaluator.go b/expression/evaluator.go index 84f2cc81627c8..d1ff7221b7d5c 100644 --- a/expression/evaluator.go +++ b/expression/evaluator.go @@ -64,36 +64,36 @@ func (e *defaultEvaluator) run(ctx sessionctx.Context, input, output *chunk.Chun return nil } -// EvaluatorSuit is responsible for the evaluation of a list of expressions. +// EvaluatorSuite is responsible for the evaluation of a list of expressions. // It separates them to "column" and "other" expressions and evaluates "other" // expressions before "column" expressions. -type EvaluatorSuit struct { +type EvaluatorSuite struct { *columnEvaluator // Evaluator for column expressions. *defaultEvaluator // Evaluator for other expressions. } -// NewEvaluatorSuit creates an EvaluatorSuit to evaluate all the exprs. -func NewEvaluatorSuit(exprs []Expression) *EvaluatorSuit { - e := &EvaluatorSuit{} +// NewEvaluatorSuite creates an EvaluatorSuite to evaluate all the exprs. +// avoidColumnEvaluator can be removed after column pool is supported. +func NewEvaluatorSuite(exprs []Expression, avoidColumnEvaluator bool) *EvaluatorSuite { + e := &EvaluatorSuite{} - for i, expr := range exprs { - switch x := expr.(type) { - case *Column: + for i := 0; i < len(exprs); i++ { + if col, isCol := exprs[i].(*Column); isCol && !avoidColumnEvaluator { if e.columnEvaluator == nil { e.columnEvaluator = &columnEvaluator{inputIdxToOutputIdxes: make(map[int][]int)} } - inputIdx, outputIdx := x.Index, i + inputIdx, outputIdx := col.Index, i e.columnEvaluator.inputIdxToOutputIdxes[inputIdx] = append(e.columnEvaluator.inputIdxToOutputIdxes[inputIdx], outputIdx) - default: - if e.defaultEvaluator == nil { - e.defaultEvaluator = &defaultEvaluator{ - outputIdxes: make([]int, 0, len(exprs)), - exprs: make([]Expression, 0, len(exprs)), - } + continue + } + if e.defaultEvaluator == nil { + e.defaultEvaluator = &defaultEvaluator{ + outputIdxes: make([]int, 0, len(exprs)), + exprs: make([]Expression, 0, len(exprs)), } - e.defaultEvaluator.exprs = append(e.defaultEvaluator.exprs, x) - e.defaultEvaluator.outputIdxes = append(e.defaultEvaluator.outputIdxes, i) } + e.defaultEvaluator.exprs = append(e.defaultEvaluator.exprs, exprs[i]) + e.defaultEvaluator.outputIdxes = append(e.defaultEvaluator.outputIdxes, i) } if e.defaultEvaluator != nil { @@ -102,14 +102,14 @@ func NewEvaluatorSuit(exprs []Expression) *EvaluatorSuit { return e } -// Vectorizable checks whether this EvaluatorSuit can use vectorizd execution mode. -func (e *EvaluatorSuit) Vectorizable() bool { +// Vectorizable checks whether this EvaluatorSuite can use vectorizd execution mode. +func (e *EvaluatorSuite) Vectorizable() bool { return e.defaultEvaluator == nil || e.defaultEvaluator.vectorizable } -// Run evaluates all the expressions hold by this EvaluatorSuit. +// Run evaluates all the expressions hold by this EvaluatorSuite. // NOTE: "defaultEvaluator" must be evaluated before "columnEvaluator". -func (e *EvaluatorSuit) Run(ctx sessionctx.Context, input, output *chunk.Chunk) error { +func (e *EvaluatorSuite) Run(ctx sessionctx.Context, input, output *chunk.Chunk) error { if e.defaultEvaluator != nil { err := e.defaultEvaluator.run(ctx, input, output) if err != nil { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index dc102b0c34d08..3f8c36f512c8b 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -692,8 +692,9 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty return nil } proj := PhysicalProjection{ - Exprs: p.Exprs, - CalculateNoDelay: p.calculateNoDelay, + Exprs: p.Exprs, + CalculateNoDelay: p.calculateNoDelay, + AvoidColumnEvaluator: p.avoidColumnEvaluator, }.init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), newProp) proj.SetSchema(p.schema) return []PhysicalPlan{proj} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 6f95472269905..46b60897c2d85 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -677,7 +677,7 @@ func (b *planBuilder) buildProjection4Union(u *LogicalUnionAll) { } } b.optFlag |= flagEliminateProjection - proj := LogicalProjection{Exprs: exprs}.init(b.ctx) + proj := LogicalProjection{Exprs: exprs, avoidColumnEvaluator: true}.init(b.ctx) proj.SetSchema(u.schema.Clone()) proj.SetChildren(child) u.children[childID] = proj diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 9caedb2b3cea1..99967e12e5f71 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -185,6 +185,13 @@ type LogicalProjection struct { // Currently it is "true" only when the current sql query is a "DO" statement. // See "https://dev.mysql.com/doc/refman/5.7/en/do.html" for more detail. calculateNoDelay bool + + // avoidColumnRef is a temporary variable which is ONLY used to avoid + // building columnEvaluator for the expressions of Projection which is + // built by buildProjection4Union. + // This can be removed after column pool being supported. + // Related issue: TiDB#8141(https://github.com/pingcap/tidb/issues/8141) + avoidColumnEvaluator bool } func (p *LogicalProjection) extractCorrelatedCols() []*expression.CorrelatedColumn { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 53e80699f78ef..48ceea5f48001 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -170,8 +170,9 @@ func (ts *PhysicalTableScan) IsPartition() (bool, int64) { type PhysicalProjection struct { physicalSchemaProducer - Exprs []expression.Expression - CalculateNoDelay bool + Exprs []expression.Expression + CalculateNoDelay bool + AvoidColumnEvaluator bool } // PhysicalTopN is the physical operator of topN. From de38a50ed410dab25495808690d31c1f5888b4ce Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 5 Nov 2018 18:30:47 +0800 Subject: [PATCH 055/509] executor: fix update float panic (#8045) (#8170) --- executor/update.go | 28 ++++++++++++++++++++++++---- executor/write_test.go | 6 ++++++ 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/executor/update.go b/executor/update.go index dc04ef5f505f3..9409dde0e161e 100644 --- a/executor/update.go +++ b/executor/update.go @@ -140,6 +140,17 @@ func (e *UpdateExec) Next(ctx context.Context, chk *chunk.Chunk) error { func (e *UpdateExec) fetchChunkRows(ctx context.Context) error { fields := e.children[0].retTypes() + schema := e.children[0].Schema() + colsInfo := make([]*table.Column, len(fields)) + for id, cols := range schema.TblID2Handle { + tbl := e.tblID2table[id] + for _, col := range cols { + offset := getTableOffset(schema, col) + for i, c := range tbl.WritableCols() { + colsInfo[offset+i] = c + } + } + } globalRowIdx := 0 chk := e.children[0].newFirstChunk() e.evalBuffer = chunk.MutRowFromTypes(fields) @@ -156,7 +167,7 @@ func (e *UpdateExec) fetchChunkRows(ctx context.Context) error { for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { chunkRow := chk.GetRow(rowIdx) datumRow := chunkRow.GetDatumRow(fields) - newRow, err1 := e.composeNewRow(globalRowIdx, datumRow) + newRow, err1 := e.composeNewRow(globalRowIdx, datumRow, colsInfo) if err1 != nil { return errors.Trace(err1) } @@ -181,7 +192,7 @@ func (e *UpdateExec) handleErr(colName model.CIStr, rowIdx int, err error) error return errors.Trace(err) } -func (e *UpdateExec) composeNewRow(rowIdx int, oldRow []types.Datum) ([]types.Datum, error) { +func (e *UpdateExec) composeNewRow(rowIdx int, oldRow []types.Datum, cols []*table.Column) ([]types.Datum, error) { newRowData := types.CopyRow(oldRow) e.evalBuffer.SetDatums(newRowData...) for _, assign := range e.OrderedList { @@ -190,10 +201,19 @@ func (e *UpdateExec) composeNewRow(rowIdx int, oldRow []types.Datum) ([]types.Da continue } val, err := assign.Expr.Eval(e.evalBuffer.ToRow()) + if err = e.handleErr(assign.Col.ColName, rowIdx, err); err != nil { + return nil, err + } - if err1 := e.handleErr(assign.Col.ColName, rowIdx, err); err1 != nil { - return nil, err1 + // info of `_tidb_rowid` column is nil. + // No need to cast `_tidb_rowid` column value. + if cols[assign.Col.Index] != nil { + val, err = table.CastValue(e.ctx, val, cols[assign.Col.Index].ColumnInfo) + if err = e.handleErr(assign.Col.ColName, rowIdx, err); err != nil { + return nil, errors.Trace(err) + } } + newRowData[assign.Col.Index] = *val.Copy() e.evalBuffer.SetDatum(assign.Col.Index, val) } diff --git a/executor/write_test.go b/executor/write_test.go index 98a91cf58d455..f4d9f0542b370 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1076,6 +1076,12 @@ func (s *testSuite) TestUpdate(c *C) { _, err = tk.Exec("update t, (select * from t) as b set b.k = t.k") c.Assert(err.Error(), Equals, "[planner:1288]The target table b of the UPDATE is not updatable") tk.MustExec("update t, (select * from t) as b set t.k = b.k") + + // issue 8045 + tk.MustExec("drop table if exists t1") + tk.MustExec(`CREATE TABLE t1 (c1 float)`) + tk.MustExec("INSERT INTO t1 SET c1 = 1") + tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") } func (s *testSuite) TestPartitionedTableUpdate(c *C) { From e5ce3e5531d3308ff2530588537e5ad3104854b9 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 5 Nov 2018 21:11:23 +0800 Subject: [PATCH 056/509] planner: fix issue#8135 (#8143) (#8180) --- executor/executor_test.go | 5 +++ planner/core/exhaust_physical_plans.go | 2 +- planner/core/find_best_task.go | 44 +++++--------------------- 3 files changed, 14 insertions(+), 37 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 311df7dcfffbf..1aa27ce095110 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3270,4 +3270,9 @@ func (s *testSuite) TestRowID(c *C) { tk.MustExec(`select * from t for update`) tk.MustQuery(`select distinct b from t use index(idx) where a = 'a';`).Check(testkit.Rows(`b`)) tk.MustExec(`commit;`) + + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(a varchar(5) primary key)`) + tk.MustExec(`insert into t values('a')`) + tk.MustQuery("select *, _tidb_rowid from t use index(`primary`) where _tidb_rowid=1").Check(testkit.Rows("a 1")) } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 3f8c36f512c8b..37d8dfee2b1b1 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -495,7 +495,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn cop := &copTask{ indexPlan: is, } - if !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) { + if !isCoveringIndex(ds.schema.Columns, is.Index.Columns, is.Table.PKIsHandle) { // On this way, it's double read case. ts := PhysicalTableScan{Columns: ds.Columns, Table: is.Table}.init(ds.ctx) ts.SetSchema(is.dataSourceSchema) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 915edff781b3f..50c0ae748e4bd 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -288,18 +288,18 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err return } -func isCoveringIndex(columns []*model.ColumnInfo, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { - for _, colInfo := range columns { - if pkIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) { +func isCoveringIndex(columns []*expression.Column, indexColumns []*model.IndexColumn, pkIsHandle bool) bool { + for _, col := range columns { + if pkIsHandle && mysql.HasPriKeyFlag(col.RetType.Flag) { continue } - if colInfo.ID == model.ExtraHandleID { + if col.ID == model.ExtraHandleID { continue } isIndexColumn := false for _, indexCol := range indexColumns { - isFullLen := indexCol.Length == types.UnspecifiedLength || indexCol.Length == colInfo.Flen - if colInfo.Name.L == indexCol.Name.L && isFullLen { + isFullLen := indexCol.Length == types.UnspecifiedLength || indexCol.Length == col.RetType.Flen + if col.ColName.L == indexCol.Name.L && isFullLen { isIndexColumn = true break } @@ -347,7 +347,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, path * } rowCount := path.countAfterAccess cop := &copTask{indexPlan: is} - if !isCoveringIndex(is.Columns, is.Index.Columns, is.Table.PKIsHandle) { + if !isCoveringIndex(ds.schema.Columns, is.Index.Columns, is.Table.PKIsHandle) { // If it's parent requires single read task, return max cost. if prop.TaskTp == property.CopSingleReadTaskType { return invalidTask, nil @@ -488,16 +488,9 @@ func matchIndicesProp(idxCols []*model.IndexColumn, propCols []*expression.Colum func splitIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, table *model.TableInfo) (indexConds, tableConds []expression.Expression) { - var pkName model.CIStr - if table.PKIsHandle { - pkInfo := table.GetPkColInfo() - if pkInfo != nil { - pkName = pkInfo.Name - } - } var indexConditions, tableConditions []expression.Expression for _, cond := range conditions { - if checkIndexCondition(cond, indexColumns, pkName) { + if isCoveringIndex(expression.ExtractColumns(cond), indexColumns, table.PKIsHandle) { indexConditions = append(indexConditions, cond) } else { tableConditions = append(tableConditions, cond) @@ -506,27 +499,6 @@ func splitIndexFilterConditions(conditions []expression.Expression, indexColumns return indexConditions, tableConditions } -// checkIndexCondition will check whether all columns of condition is index columns or primary key column. -func checkIndexCondition(condition expression.Expression, indexColumns []*model.IndexColumn, pkName model.CIStr) bool { - cols := expression.ExtractColumns(condition) - for _, col := range cols { - if pkName.L == col.ColName.L { - continue - } - isIndexColumn := false - for _, indCol := range indexColumns { - if col.ColName.L == indCol.Name.L && indCol.Length == types.UnspecifiedLength { - isIndexColumn = true - break - } - } - if !isIndexColumn { - return false - } - } - return true -} - // convertToTableScan converts the DataSource to table scan. func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, path *accessPath) (task task, err error) { // It will be handled in convertToIndexScan. From 89140e21d0abc4bbb2e8fa56d3288f1f2c8c9627 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 5 Nov 2018 21:54:10 +0800 Subject: [PATCH 057/509] stats: fix converting duration to timestamp (#8174) (#8182) --- statistics/gc.go | 3 +-- statistics/handle.go | 7 ++++++- statistics/handle_test.go | 21 +++++++++++++++------ 3 files changed, 22 insertions(+), 9 deletions(-) diff --git a/statistics/gc.go b/statistics/gc.go index 75efc03c1558a..cdcd91c979b28 100644 --- a/statistics/gc.go +++ b/statistics/gc.go @@ -19,7 +19,6 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" @@ -31,7 +30,7 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error // To make sure that all the deleted tables' schema and stats info have been acknowledged to all tidb, // we only garbage collect version before 10 lease. lease := mathutil.MaxInt64(int64(h.Lease), int64(ddlLease)) - offset := oracle.ComposeTS(10*lease, 0) + offset := DurationToTS(10 * time.Duration(lease)) if h.LastUpdateVersion() < offset { return nil } diff --git a/statistics/handle.go b/statistics/handle.go index 0a612500d1e0b..3b0160343d569 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -109,6 +109,11 @@ func (h *Handle) GetQueryFeedback() []*QueryFeedback { return h.feedback } +// DurationToTS converts duration to timestamp. +func DurationToTS(d time.Duration) uint64 { + return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) +} + // Update reads stats meta from store and updates the stats map. func (h *Handle) Update(is infoschema.InfoSchema) error { lastVersion := h.LastUpdateVersion() @@ -117,7 +122,7 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { // and A0 < B0 < B1 < A1. We will first read the stats of B, and update the lastVersion to B0, but we cannot read // the table stats of A0 if we read stats that greater than lastVersion which is B0. // We can read the stats if the diff between commit time and version is less than three lease. - offset := oracle.ComposeTS(3*int64(h.Lease), 0) + offset := DurationToTS(3 * h.Lease) if lastVersion >= offset { lastVersion = lastVersion - offset } else { diff --git a/statistics/handle_test.go b/statistics/handle_test.go index 6cfdca998e0d1..eb46e0758f023 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -238,6 +238,14 @@ func (s *testStatsCacheSuite) TestAvgColLen(c *C) { c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0) } +func (s *testStatsCacheSuite) TestDurationToTS(c *C) { + tests := []time.Duration{time.Millisecond, time.Second, time.Minute, time.Hour} + for _, t := range tests { + ts := statistics.DurationToTS(t) + c.Assert(oracle.ExtractPhysical(ts)*int64(time.Millisecond), Equals, int64(t)) + } +} + func (s *testStatsCacheSuite) TestVersion(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) @@ -249,11 +257,12 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) tableInfo1 := tbl1.Meta() - h := statistics.NewHandle(testKit.Se, 1) - testKit.MustExec("update mysql.stats_meta set version = 2 where table_id = ?", tableInfo1.ID) + h := statistics.NewHandle(testKit.Se, time.Millisecond) + unit := oracle.ComposeTS(1, 0) + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) h.Update(is) - c.Assert(h.LastUpdateVersion(), Equals, uint64(2)) + c.Assert(h.LastUpdateVersion(), Equals, 2*unit) statsTbl1 := h.GetTableStats(tableInfo1) c.Assert(statsTbl1.Pseudo, IsFalse) @@ -264,15 +273,15 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { c.Assert(err, IsNil) tableInfo2 := tbl2.Meta() // A smaller version write, and we can still read it. - testKit.MustExec("update mysql.stats_meta set version = 1 where table_id = ?", tableInfo2.ID) + testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", unit, tableInfo2.ID) h.Update(is) - c.Assert(h.LastUpdateVersion(), Equals, uint64(2)) + c.Assert(h.LastUpdateVersion(), Equals, 2*unit) statsTbl2 := h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Pseudo, IsFalse) testKit.MustExec("insert t1 values(1,2)") testKit.MustExec("analyze table t1") - offset := oracle.ComposeTS(3*int64(h.Lease), 0) + offset := 3 * unit testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", offset+4, tableInfo1.ID) h.Update(is) c.Assert(h.LastUpdateVersion(), Equals, offset+uint64(4)) From 49b7dc9c74b4121e674fae626ade26474df4fa81 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 6 Nov 2018 17:08:45 +0800 Subject: [PATCH 058/509] *: add a variable tidb_query_log_max_len to set the max length of the query string in the log dynamically (#8183) (#8200) --- config/config.go | 4 ++-- executor/adapter.go | 4 ++-- executor/set_test.go | 7 +++++++ sessionctx/variable/session.go | 2 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 3 +++ sessionctx/variable/varsutil.go | 6 +++++- util/logutil/log.go | 2 ++ 8 files changed, 24 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 5b849a5fef110..e568f92a1133f 100644 --- a/config/config.go +++ b/config/config.go @@ -89,7 +89,7 @@ type Log struct { SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"` SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"` ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"` - QueryLogMaxLen uint `toml:"query-log-max-len" json:"query-log-max-len"` + QueryLogMaxLen uint64 `toml:"query-log-max-len" json:"query-log-max-len"` } // Security is the security section of the config. @@ -275,7 +275,7 @@ var defaultConf = Config{ }, SlowThreshold: logutil.DefaultSlowThreshold, ExpensiveThreshold: 10000, - QueryLogMaxLen: 2048, + QueryLogMaxLen: logutil.DefaultQueryLogMaxLen, }, Status: Status{ ReportStatus: true, diff --git a/executor/adapter.go b/executor/adapter.go index 95e76acdaadb1..508ff1f9382f1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -347,8 +347,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { return } sql := a.Text - if len(sql) > int(cfg.Log.QueryLogMaxLen) { - sql = fmt.Sprintf("%.*q(len:%d)", cfg.Log.QueryLogMaxLen, sql, len(a.Text)) + if maxQueryLen := atomic.LoadUint64(&cfg.Log.QueryLogMaxLen); uint64(len(sql)) > maxQueryLen { + sql = fmt.Sprintf("%.*q(len:%d)", maxQueryLen, sql, len(a.Text)) } sessVars := a.Ctx.GetSessionVars() sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() diff --git a/executor/set_test.go b/executor/set_test.go index bd6d0f9dc64cb..dd9b2955e324d 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -241,6 +241,13 @@ func (s *testSuite) TestSetVar(c *C) { tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("1")) _, err = tk.Exec("set global tidb_slow_log_threshold = 0") c.Assert(err, NotNil) + + tk.MustExec("set tidb_query_log_max_len = 0") + tk.MustQuery("select @@session.tidb_query_log_max_len;").Check(testkit.Rows("0")) + tk.MustExec("set tidb_query_log_max_len = 20") + tk.MustQuery("select @@session.tidb_query_log_max_len;").Check(testkit.Rows("20")) + _, err = tk.Exec("set global tidb_query_log_max_len = 20") + c.Assert(err, NotNil) } func (s *testSuite) TestSetCharset(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 89b1224109469..4ac6b9b978df6 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -588,6 +588,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt32(val, DefTiDBGeneralLog))) case TiDBSlowLogThreshold: atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) + case TiDBQueryLogMaxLen: + atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) case TiDBRetryLimit: s.RetryLimit = tidbOptInt64(val, DefTiDBRetryLimit) case TiDBDisableTxnAutoRetry: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 2ab0c4d9fed9d..69b5a2839ac84 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -662,6 +662,7 @@ var defaultSysVars = []*SysVar{ /* The following variable is defined as session scope but is actually server scope. */ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, {ScopeSession, TiDBSlowLogThreshold, strconv.Itoa(logutil.DefaultSlowThreshold)}, + {ScopeSession, TiDBQueryLogMaxLen, strconv.Itoa(logutil.DefaultQueryLogMaxLen)}, {ScopeSession, TiDBConfig, ""}, {ScopeGlobal | ScopeSession, TiDBDDLReorgWorkerCount, strconv.Itoa(DefTiDBDDLReorgWorkerCount)}, {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 8b1ea4371c3d6..341b8df20e818 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -98,6 +98,9 @@ const ( // tidb_slow_log_threshold is used to set the slow log threshold in the server. TiDBSlowLogThreshold = "tidb_slow_log_threshold" + // tidb_query_log_max_len is used to set the max length of the query in the log. + TiDBQueryLogMaxLen = "tidb_query_log_max_len" + // tidb_retry_limit is the maximum number of retries when committing a transaction. TiDBRetryLimit = "tidb_retry_limit" diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index e8766de4fccc4..c13effeebd902 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -87,6 +87,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], true, nil case TiDBSlowLogThreshold: return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), true, nil + case TiDBQueryLogMaxLen: + return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), true, nil } sVal, ok := s.systems[key] if ok { @@ -327,7 +329,9 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaNestedLoopApply, - TiDBRetryLimit, TiDBSlowLogThreshold: + TiDBRetryLimit, + TiDBSlowLogThreshold, + TiDBQueryLogMaxLen: _, err := strconv.ParseInt(value, 10, 64) if err != nil { return value, ErrWrongValueForVar.GenWithStackByArgs(name) diff --git a/util/logutil/log.go b/util/logutil/log.go index 610c43b91bcc1..8c74aed9d0737 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -35,6 +35,8 @@ const ( defaultLogLevel = log.InfoLevel // DefaultSlowThreshold is the default slow log threshold in millisecond. DefaultSlowThreshold = 300 + // DefaultQueryLogMaxLen is the default max length of the query in the log. + DefaultQueryLogMaxLen = 2048 ) // FileLogConfig serializes file log related config in toml/json. From a89b6fe8d6a0e4f7b93381e7f6a54a4c257b508f Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 6 Nov 2018 17:27:41 +0800 Subject: [PATCH 059/509] executor: recovery panic in parallel hashagg and projection (#8185) (#8204) --- executor/aggregate.go | 17 +++++++++++++++++ executor/projection.go | 24 ++++++++++++++++++++++-- 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/executor/aggregate.go b/executor/aggregate.go index 6c28ef688b90b..3a4c2f588e5ca 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -23,10 +23,12 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/set" "github.com/pkg/errors" + log "github.com/sirupsen/logrus" "github.com/spaolacci/murmur3" "golang.org/x/net/context" ) @@ -314,9 +316,18 @@ func (w *HashAggPartialWorker) getChildInput() bool { return true } +func recoveryHashAgg(output chan *AfFinalResult, r interface{}) { + output <- &AfFinalResult{err: errors.Errorf("%v", r)} + buf := util.GetStack() + log.Errorf("panic in the recoverable goroutine: %v, stack trace:\n%s", r, buf) +} + func (w *HashAggPartialWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup, finalConcurrency int) { needShuffle, sc := false, ctx.GetSessionVars().StmtCtx defer func() { + if r := recover(); r != nil { + recoveryHashAgg(w.globalOutputCh, r) + } if needShuffle { w.shuffleIntermData(sc, finalConcurrency) } @@ -492,6 +503,9 @@ func (w *HashAggFinalWorker) receiveFinalResultHolder() (*chunk.Chunk, bool) { func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGroup) { defer func() { + if r := recover(); r != nil { + recoveryHashAgg(w.outputCh, r) + } waitGroup.Done() }() if err := w.consumeIntermData(ctx); err != nil { @@ -521,6 +535,9 @@ func (e *HashAggExec) fetchChildData(ctx context.Context) { err error ) defer func() { + if r := recover(); r != nil { + recoveryHashAgg(e.finalOutputCh, r) + } for i := range e.partialInputChs { close(e.partialInputChs[i]) } diff --git a/executor/projection.go b/executor/projection.go index 4b3506c999cba..90a5fcc72f840 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -18,8 +18,10 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -264,7 +266,11 @@ type projectionInputFetcher struct { // a. There is no more input from child. // b. "ProjectionExec" close the "globalFinishCh" func (f *projectionInputFetcher) run(ctx context.Context) { + var output *projectionOutput defer func() { + if r := recover(); r != nil { + recoveryProjection(output, r) + } close(f.globalOutputCh) }() @@ -275,7 +281,7 @@ func (f *projectionInputFetcher) run(ctx context.Context) { } targetWorker := input.targetWorker - output := readProjectionOutput(f.outputCh, f.globalFinishCh) + output = readProjectionOutput(f.outputCh, f.globalFinishCh) if output == nil { return } @@ -317,13 +323,19 @@ type projectionWorker struct { // It is finished and exited once: // a. "ProjectionExec" closes the "globalFinishCh". func (w *projectionWorker) run(ctx context.Context) { + var output *projectionOutput + defer func() { + if r := recover(); r != nil { + recoveryProjection(output, r) + } + }() for { input := readProjectionInput(w.inputCh, w.globalFinishCh) if input == nil { return } - output := readProjectionOutput(w.outputCh, w.globalFinishCh) + output = readProjectionOutput(w.outputCh, w.globalFinishCh) if output == nil { return } @@ -339,6 +351,14 @@ func (w *projectionWorker) run(ctx context.Context) { } } +func recoveryProjection(output *projectionOutput, r interface{}) { + if output != nil { + output.done <- errors.Errorf("%v", r) + } + buf := util.GetStack() + log.Errorf("panic in the recoverable goroutine: %v, stack trace:\n%s", r, buf) +} + func readProjectionInput(inputCh <-chan *projectionInput, finishCh <-chan struct{}) *projectionInput { select { case <-finishCh: From 12439c3dc5fbbd5c8592448aa6e50be507d8d109 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Wed, 7 Nov 2018 22:05:48 +0800 Subject: [PATCH 060/509] session: add transaction start timestamp to the log when retrying (#8091) (#8224) --- session/session.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/session/session.go b/session/session.go index 4e157e199f47e..5822fd28b910e 100644 --- a/session/session.go +++ b/session/session.go @@ -464,6 +464,7 @@ func (s *session) retry(ctx context.Context, maxCnt uint) error { nh := GetHistory(s) var err error var schemaVersion int64 + orgStartTS := s.GetSessionVars().TxnCtx.StartTS for { s.PrepareTxnCtx(ctx) s.sessionVars.RetryInfo.ResetOffset() @@ -493,6 +494,8 @@ func (s *session) retry(ctx context.Context, maxCnt uint) error { } s.StmtCommit() } + log.Warnf("con:%d retrying_txn_start_ts:%d original_txn_start_ts:(%d)", + connID, s.GetSessionVars().TxnCtx.StartTS, orgStartTS) if hook := ctx.Value("preCommitHook"); hook != nil { // For testing purpose. hook.(func())() From 6a5152c0c4ff8765acaffc908af29234d043fd68 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 8 Nov 2018 10:42:07 +0800 Subject: [PATCH 061/509] store/tikv/latch: fix a deadlock in the latch scheduler (#8186) (#8220) --- store/tikv/latch/latch.go | 19 +++++++- store/tikv/latch/latch_test.go | 3 ++ store/tikv/latch/scheduler.go | 7 ++- store/tikv/latch/scheduler_test.go | 73 +++++++++++++++++++++++------- 4 files changed, 79 insertions(+), 23 deletions(-) diff --git a/store/tikv/latch/latch.go b/store/tikv/latch/latch.go index f6f57a460b106..a62b6da8a09cb 100644 --- a/store/tikv/latch/latch.go +++ b/store/tikv/latch/latch.go @@ -18,8 +18,10 @@ import ( "math/bits" "sort" "sync" + "time" "github.com/cznic/mathutil" + log "github.com/sirupsen/logrus" "github.com/spaolacci/murmur3" ) @@ -181,6 +183,12 @@ func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) { } find.maxCommitTS = mathutil.MaxUint64(find.maxCommitTS, lock.commitTS) find.value = nil + // Make a copy of the key, so latch does not reference the transaction's memory. + // If we do not do it, transaction memory can't be recycle by GC and there will + // be a leak. + copyKey := make([]byte, len(find.key)) + copy(copyKey, find.key) + find.key = copyKey if len(latch.waiting) == 0 { return nil } @@ -201,6 +209,8 @@ func (latches *Latches) releaseSlot(lock *Lock) (nextLock *Lock) { latch.waiting = latch.waiting[:len(latch.waiting)-1] if find.maxCommitTS > nextLock.startTS { + find.value = nextLock + nextLock.acquiredCount++ nextLock.isStale = true } } @@ -252,27 +262,32 @@ func (latches *Latches) acquireSlot(lock *Lock) acquireResult { } // recycle is not thread safe, the latch should acquire its lock before executing this function. -func (l *latch) recycle(currentTS uint64) { +func (l *latch) recycle(currentTS uint64) int { + total := 0 fakeHead := node{next: l.queue} prev := &fakeHead for curr := prev.next; curr != nil; curr = curr.next { if tsoSub(currentTS, curr.maxCommitTS) >= expireDuration && curr.value == nil { l.count-- prev.next = curr.next + total++ } else { prev = curr } } l.queue = fakeHead.next + return total } func (latches *Latches) recycle(currentTS uint64) { + total := 0 for i := 0; i < len(latches.slots); i++ { latch := &latches.slots[i] latch.Lock() - latch.recycle(currentTS) + total += latch.recycle(currentTS) latch.Unlock() } + log.Debugf("recycle run at %v, recycle count = %d...\n", time.Now(), total) } func findNode(list *node, key []byte) *node { diff --git a/store/tikv/latch/latch_test.go b/store/tikv/latch/latch_test.go index 951a9e3de1802..4b10c118883a6 100644 --- a/store/tikv/latch/latch_test.go +++ b/store/tikv/latch/latch_test.go @@ -122,6 +122,9 @@ func (s *testLatchSuite) TestRecycle(c *C) { lock.SetCommitTS(startTS + 1) var wakeupList []*Lock latches.release(lock, wakeupList) + // Release lock will grant latch to lock1 automatically, + // so release lock1 is called here. + latches.release(lock1, wakeupList) lock2 := latches.genLock(startTS+3, [][]byte{ []byte("b"), []byte("c"), diff --git a/store/tikv/latch/scheduler.go b/store/tikv/latch/scheduler.go index 5f4e84cd7760d..cbf2cb8157dfa 100644 --- a/store/tikv/latch/scheduler.go +++ b/store/tikv/latch/scheduler.go @@ -44,9 +44,8 @@ func NewScheduler(size uint) *LatchesScheduler { return scheduler } -// A transaction can last for at most 10 minutes, see also gcworker. -const expireDuration = 10 * time.Minute -const checkInterval = 5 * time.Minute +const expireDuration = 2 * time.Minute +const checkInterval = 1 * time.Minute const checkCounter = 50000 const latchListCount = 5 @@ -62,7 +61,7 @@ func (scheduler *LatchesScheduler) run() { if lock.commitTS > lock.startTS { currentTS := lock.commitTS elapsed := tsoSub(currentTS, scheduler.lastRecycleTime) - if elapsed > checkInterval && counter > checkCounter { + if elapsed > checkInterval || counter > checkCounter { go scheduler.latches.recycle(lock.commitTS) scheduler.lastRecycleTime = currentTS counter = 0 diff --git a/store/tikv/latch/scheduler_test.go b/store/tikv/latch/scheduler_test.go index fc2d5b1514d0f..a2e88dde6b407 100644 --- a/store/tikv/latch/scheduler_test.go +++ b/store/tikv/latch/scheduler_test.go @@ -14,7 +14,10 @@ package latch import ( + "bytes" + "math/rand" "sync" + "time" . "github.com/pingcap/check" ) @@ -28,28 +31,64 @@ func (s *testSchedulerSuite) SetUpTest(c *C) { } func (s *testSchedulerSuite) TestWithConcurrency(c *C) { - txns := [][][]byte{ - {[]byte("a"), []byte("b"), []byte("c")}, - {[]byte("a"), []byte("d"), []byte("e"), []byte("f")}, - {[]byte("e"), []byte("f"), []byte("g"), []byte("h")}, - } - sched := NewScheduler(1024) + sched := NewScheduler(7) defer sched.Close() + rand.Seed(time.Now().Unix()) + ch := make(chan [][]byte, 100) + const workerCount = 10 var wg sync.WaitGroup - wg.Add(len(txns)) - for _, txn := range txns { - go func(txn [][]byte, wg *sync.WaitGroup) { - lock := sched.Lock(getTso(), txn) - defer sched.UnLock(lock) - if lock.IsStale() { - // Should restart the transaction or return error - } else { - lock.SetCommitTS(getTso()) - // Do 2pc + wg.Add(workerCount) + for i := 0; i < workerCount; i++ { + go func(ch <-chan [][]byte, wg *sync.WaitGroup) { + for txn := range ch { + lock := sched.Lock(getTso(), txn) + if lock.IsStale() { + // Should restart the transaction or return error + } else { + lock.SetCommitTS(getTso()) + // Do 2pc + } + sched.UnLock(lock) } wg.Done() - }(txn, &wg) + }(ch, &wg) + } + + for i := 0; i < 999; i++ { + ch <- generate() } + close(ch) + wg.Wait() } + +// generate generates something like: +// {[]byte("a"), []byte("b"), []byte("c")} +// {[]byte("a"), []byte("d"), []byte("e"), []byte("f")} +// {[]byte("e"), []byte("f"), []byte("g"), []byte("h")} +// The data should not repeat in the sequence. +func generate() [][]byte { + table := []byte{'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'} + ret := make([][]byte, 0, 5) + chance := []int{100, 60, 40, 20} + for i := 0; i < len(chance); i++ { + needMore := rand.Intn(100) < chance[i] + if needMore { + randBytes := []byte{table[rand.Intn(len(table))]} + if !contains(randBytes, ret) { + ret = append(ret, randBytes) + } + } + } + return ret +} + +func contains(x []byte, set [][]byte) bool { + for _, y := range set { + if bytes.Compare(x, y) == 0 { + return true + } + } + return false +} From 6f079070d0f44d66579e289066f162f16fce333d Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 8 Nov 2018 15:23:58 +0800 Subject: [PATCH 062/509] *: add a variable to control whether we can write _tidb_rowid (#8218) --- executor/insert_common.go | 3 +++ executor/rowid_test.go | 40 ++++++++++++++++++++++++++-- executor/update.go | 8 ++++-- sessionctx/variable/session.go | 6 +++++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 +++ sessionctx/variable/varsutil_test.go | 1 + 7 files changed, 59 insertions(+), 4 deletions(-) diff --git a/executor/insert_common.go b/executor/insert_common.go index 72123ec508a40..f3f2d413069d8 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -112,6 +112,9 @@ func (e *InsertValues) initInsertColumns() error { } for _, col := range cols { if col.Name.L == model.ExtraHandleName.L { + if !e.ctx.GetSessionVars().AllowWriteRowID { + return errors.Errorf("insert, update and replace statements for _tidb_rowid are not supported.") + } e.hasExtraHandle = true break } diff --git a/executor/rowid_test.go b/executor/rowid_test.go index 56a9ac155792f..0e7d8a3d101d8 100644 --- a/executor/rowid_test.go +++ b/executor/rowid_test.go @@ -19,8 +19,12 @@ import ( ) func (s *testSuite) TestExportRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") + tk := testkit.NewTestKitWithInit(c, s.store) + tk.Se.GetSessionVars().AllowWriteRowID = true + defer func() { + tk.Se.GetSessionVars().AllowWriteRowID = false + }() + tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int)") tk.MustExec("insert t values (1, 7), (1, 8), (1, 9)") @@ -49,4 +53,36 @@ func (s *testSuite) TestExportRowID(c *C) { c.Assert(err, NotNil) _, err = tk.Exec("delete from s where _tidb_rowid = 1") c.Assert(err, NotNil) + + // Make sure "AllowWriteRowID" is a session variable. + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use test") + _, err = tk1.Exec("insert into t (a, _tidb_rowid) values(10, 1);") + c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") +} + +func (s *testSuite) TestNotAllowWriteRowID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table tt(id binary(10), c int, primary key(id));") + tk.MustExec("insert tt values (1, 10);") + // select statement + tk.MustQuery("select *, _tidb_rowid from tt"). + Check(testkit.Rows("1\x00\x00\x00\x00\x00\x00\x00\x00\x00 10 1")) + // insert statement + _, err := tk.Exec("insert into tt (id, c, _tidb_rowid) values(30000,10,1);") + c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + // replace statement + _, err = tk.Exec("replace into tt (id, c, _tidb_rowid) values(30000,10,1);") + c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + // update statement + _, err = tk.Exec("update tt set id = 2, _tidb_rowid = 1 where _tidb_rowid = 1") + c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + tk.MustExec("update tt set id = 2 where _tidb_rowid = 1") + tk.MustExec("admin check table tt;") + tk.MustExec("drop table tt") + // There is currently no real support for inserting, updating, and replacing _tidb_rowid statements. + // After we support it, the following operations must be passed. + // tk.MustExec("insert into tt (id, c, _tidb_rowid) values(30000,10,1);") + // tk.MustExec("admin check table tt;") } diff --git a/executor/update.go b/executor/update.go index 9409dde0e161e..c4be098057a52 100644 --- a/executor/update.go +++ b/executor/update.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -46,7 +47,7 @@ type UpdateExec struct { } func (e *UpdateExec) exec(schema *expression.Schema) ([]types.Datum, error) { - assignFlag, err := e.getUpdateColumns(schema.Len()) + assignFlag, err := e.getUpdateColumns(e.ctx, schema.Len()) if err != nil { return nil, errors.Trace(err) } @@ -230,9 +231,12 @@ func (e *UpdateExec) Open(ctx context.Context) error { return e.SelectExec.Open(ctx) } -func (e *UpdateExec) getUpdateColumns(schemaLen int) ([]bool, error) { +func (e *UpdateExec) getUpdateColumns(ctx sessionctx.Context, schemaLen int) ([]bool, error) { assignFlag := make([]bool, schemaLen) for _, v := range e.OrderedList { + if !ctx.GetSessionVars().AllowWriteRowID && v.Col.ColName.L == model.ExtraHandleName.L { + return nil, errors.Errorf("insert, update and replace statements for _tidb_rowid are not supported.") + } idx := v.Col.Index assignFlag[idx] = true } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4ac6b9b978df6..ed7f136e7293e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -256,6 +256,10 @@ type SessionVars struct { // AllowInSubqueryUnFolding can be set to true to fold in subquery AllowInSubqueryUnFolding bool + // AllowWriteRowID can be set to false to forbid write data to _tidb_rowid. + // This variable is currently not recommended to be turned on. + AllowWriteRowID bool + // CurrInsertValues is used to record current ValuesExpr's values. // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values CurrInsertValues chunk.Row @@ -536,6 +540,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.AllowAggPushDown = TiDBOptOn(val) case TiDBOptInSubqUnFolding: s.AllowInSubqueryUnFolding = TiDBOptOn(val) + case TiDBOptWriteRowID: + s.AllowWriteRowID = TiDBOptOn(val) case TiDBIndexLookupConcurrency: s.IndexLookupConcurrency = tidbOptPositiveInt32(val, DefIndexLookupConcurrency) case TiDBIndexLookupJoinConcurrency: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 69b5a2839ac84..cc7a383e06d0c 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -622,6 +622,7 @@ var defaultSysVars = []*SysVar{ /* TiDB specific variables */ {ScopeSession, TiDBSnapshot, ""}, {ScopeSession, TiDBOptAggPushDown, boolToIntStr(DefOptAggPushDown)}, + {ScopeSession, TiDBOptWriteRowID, boolToIntStr(DefOptWriteRowID)}, {ScopeGlobal | ScopeSession, TiDBBuildStatsConcurrency, strconv.Itoa(DefBuildStatsConcurrency)}, {ScopeGlobal, TiDBAutoAnalyzeRatio, strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64)}, {ScopeGlobal, TiDBAutoAnalyzeStartTime, DefAutoAnalyzeStartTime}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 341b8df20e818..68b31d018c3a0 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -41,6 +41,9 @@ const ( // tidb_opt_agg_push_down is used to enable/disable the optimizer rule of aggregation push down. TiDBOptAggPushDown = "tidb_opt_agg_push_down" + // tidb_opt_write_row_id is used to enable/disable the operations of insert、replace and update to _tidb_rowid. + TiDBOptWriteRowID = "tidb_opt_write_row_id" + // Auto analyze will run if (table modify count)/(table row count) is greater than this value. TiDBAutoAnalyzeRatio = "tidb_auto_analyze_ratio" @@ -215,6 +218,7 @@ const ( DefSkipUTF8Check = false DefOptAggPushDown = false DefOptInSubqUnfolding = false + DefOptWriteRowID = false DefBatchInsert = false DefBatchDelete = false DefCurretTS = 0 diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 2c2a96a25ea4c..0b9e71d661a3d 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -77,6 +77,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaIndexLookupReader, Equals, int64(DefTiDBMemQuotaIndexLookupReader)) c.Assert(vars.MemQuotaIndexLookupJoin, Equals, int64(DefTiDBMemQuotaIndexLookupJoin)) c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) + c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) From 6f9fac4f98a8329da094dacf6e8d0f4f0b83a121 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 8 Nov 2018 19:49:54 +0800 Subject: [PATCH 063/509] store/tikv: refine backoff log (#8207) (#8239) --- session/session.go | 2 +- store/tikv/2pc.go | 3 ++- store/tikv/backoff.go | 9 ++++++++- store/tikv/coprocessor.go | 1 + store/tikv/scan.go | 2 +- store/tikv/snapshot.go | 6 ++++-- 6 files changed, 17 insertions(+), 6 deletions(-) diff --git a/session/session.go b/session/session.go index 5822fd28b910e..2a9a05acb28b9 100644 --- a/session/session.go +++ b/session/session.go @@ -1434,7 +1434,7 @@ func logStmt(node ast.StmtNode, vars *variable.SessionVars) { func logQuery(query string, vars *variable.SessionVars) { if atomic.LoadUint32(&variable.ProcessGeneralLog) != 0 && !vars.InRestrictedSQL { query = executor.QueryReplacer.Replace(query) - log.Infof("[GENERAL_LOG] con:%d user:%s schema_ver:%d start_ts:%d sql:%s%s", + log.Infof("[GENERAL_LOG] con:%d user:%s schema_ver:%d txn_start_ts:%d sql:%s%s", vars.ConnectionID, vars.User, vars.TxnCtx.SchemaVersion, vars.TxnCtx.StartTS, query, vars.GetExecuteArgumentsInfo()) } } diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 347490b64ebf1..345ae5de2996c 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -569,7 +569,8 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { if !committed && !undetermined { c.cleanWg.Add(1) go func() { - err := c.cleanupKeys(NewBackoffer(context.Background(), cleanupMaxBackoff).WithVars(c.txn.vars), c.keys) + cleanupKeysCtx := context.WithValue(context.Background(), txnStartKey, ctx.Value(txnStartKey)) + err := c.cleanupKeys(NewBackoffer(cleanupKeysCtx, cleanupMaxBackoff).WithVars(c.txn.vars), c.keys) if err != nil { metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback").Inc() log.Infof("con:%d 2PC cleanup err: %v, tid: %d", c.connID, err, c.startTS) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 76b5e32ec6c22..3fe5ae9b954c6 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -184,6 +184,9 @@ type Backoffer struct { vars *kv.Variables } +// txnStartKey is a key for transaction start_ts info in context.Context. +const txnStartKey = "_txn_start_key" + // NewBackoffer creates a Backoffer with maximum sleep time(in ms). func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { return &Backoffer{ @@ -225,7 +228,11 @@ func (b *Backoffer) Backoff(typ backoffType, err error) error { b.totalSleep += f(b.ctx) b.types = append(b.types, typ) - log.Debugf("%v, retry later(totalsleep %dms, maxsleep %dms)", err, b.totalSleep, b.maxSleep) + var startTs interface{} = "" + if ts := b.ctx.Value(txnStartKey); ts != nil { + startTs = ts + } + log.Debugf("%v, retry later(totalsleep %dms, maxsleep %dms), type: %s, txn_start_ts: %v", err, b.totalSleep, b.maxSleep, typ.String(), startTs) b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) if b.maxSleep > 0 && b.totalSleep >= b.maxSleep { diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 02d89a82dd4df..6638318851b34 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -80,6 +80,7 @@ func (c *CopClient) supportExpr(exprType tipb.ExprType) bool { // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { + ctx = context.WithValue(ctx, txnStartKey, req.StartTs) bo := NewBackoffer(ctx, copBuildTaskMaxBackoff).WithVars(vars) tasks, err := buildCopTasks(bo, c.store.regionCache, &copRanges{mid: req.KeyRanges}, req.Desc, req.Streaming) if err != nil { diff --git a/store/tikv/scan.go b/store/tikv/scan.go index ab0ba76bc0e60..d5c140f6a7e3f 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -74,7 +74,7 @@ func (s *Scanner) Value() []byte { // Next return next element. func (s *Scanner) Next() error { - bo := NewBackoffer(context.Background(), scannerNextMaxBackoff) + bo := NewBackoffer(context.WithValue(context.Background(), txnStartKey, s.snapshot.version.Ver), scannerNextMaxBackoff) if !s.valid { return errors.New("scanner iterator is invalid") } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 730db0273c193..c4c14e984e57d 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -73,7 +73,8 @@ func (s *tikvSnapshot) BatchGet(keys []kv.Key) (map[string][]byte, error) { // We want [][]byte instead of []kv.Key, use some magic to save memory. bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) - bo := NewBackoffer(context.Background(), batchGetMaxBackoff).WithVars(s.vars) + ctx := context.WithValue(context.Background(), txnStartKey, s.version.Ver) + bo := NewBackoffer(ctx, batchGetMaxBackoff).WithVars(s.vars) // Create a map to collect key-values from region servers. var mu sync.Mutex @@ -208,7 +209,8 @@ func (s *tikvSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, coll // Get gets the value for key k from snapshot. func (s *tikvSnapshot) Get(k kv.Key) ([]byte, error) { - val, err := s.get(NewBackoffer(context.Background(), getMaxBackoff), k) + ctx := context.WithValue(context.Background(), txnStartKey, s.version.Ver) + val, err := s.get(NewBackoffer(ctx, getMaxBackoff), k) if err != nil { return nil, errors.Trace(err) } From 86477491325d3e8907e32fb075b6e6b2bd8b696c Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 9 Nov 2018 10:45:20 +0800 Subject: [PATCH 064/509] planner: fix union statements order (#8214) (#8238) --- cmd/explaintest/r/explain_easy.result | 24 ++++++++++++------------ planner/core/logical_plan_builder.go | 3 ++- planner/core/logical_plan_test.go | 9 +++++++-- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 2aa7615570d84..29e6fa54c2455 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -152,18 +152,18 @@ TableReader_5 10000.00 root data:TableScan_4 explain select c1 from t2 union select c1 from t2 union all select c1 from t2; id count task operator info Union_17 26000.00 root -├─TableReader_20 10000.00 root data:TableScan_19 -│ └─TableScan_19 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo -└─HashAgg_24 16000.00 root group by:t2.c1, funcs:firstrow(join_agg_0) - └─Union_25 16000.00 root - ├─StreamAgg_38 8000.00 root group by:col_2, funcs:firstrow(col_0), firstrow(col_1) - │ └─IndexReader_39 8000.00 root index:StreamAgg_29 - │ └─StreamAgg_29 8000.00 cop group by:test.t2.c1, funcs:firstrow(test.t2.c1), firstrow(test.t2.c1) - │ └─IndexScan_37 10000.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true, stats:pseudo - └─StreamAgg_55 8000.00 root group by:col_2, funcs:firstrow(col_0), firstrow(col_1) - └─IndexReader_56 8000.00 root index:StreamAgg_46 - └─StreamAgg_46 8000.00 cop group by:test.t2.c1, funcs:firstrow(test.t2.c1), firstrow(test.t2.c1) - └─IndexScan_54 10000.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true, stats:pseudo +├─HashAgg_21 16000.00 root group by:t2.c1, funcs:firstrow(join_agg_0) +│ └─Union_22 16000.00 root +│ ├─StreamAgg_35 8000.00 root group by:col_2, funcs:firstrow(col_0), firstrow(col_1) +│ │ └─IndexReader_36 8000.00 root index:StreamAgg_26 +│ │ └─StreamAgg_26 8000.00 cop group by:test.t2.c1, funcs:firstrow(test.t2.c1), firstrow(test.t2.c1) +│ │ └─IndexScan_34 10000.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true, stats:pseudo +│ └─StreamAgg_52 8000.00 root group by:col_2, funcs:firstrow(col_0), firstrow(col_1) +│ └─IndexReader_53 8000.00 root index:StreamAgg_43 +│ └─StreamAgg_43 8000.00 cop group by:test.t2.c1, funcs:firstrow(test.t2.c1), firstrow(test.t2.c1) +│ └─IndexScan_51 10000.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true, stats:pseudo +└─TableReader_59 10000.00 root data:TableScan_58 + └─TableScan_58 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select c1 from t2 union all select c1 from t2 union select c1 from t2; id count task operator info HashAgg_18 24000.00 root group by:t2.c1, funcs:firstrow(join_agg_0) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 46b60897c2d85..24fa283513d0a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -694,7 +694,8 @@ func (b *planBuilder) buildUnion(union *ast.UnionStmt) (LogicalPlan, error) { if unionDistinctPlan != nil { unionDistinctPlan = b.buildDistinct(unionDistinctPlan, unionDistinctPlan.Schema().Len()) if len(allSelectPlans) > 0 { - allSelectPlans = append(allSelectPlans, unionDistinctPlan) + // Can't change the statements order in order to get the correct column info. + allSelectPlans = append([]LogicalPlan{unionDistinctPlan}, allSelectPlans...) } } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 569f97e45f199..8caf8dfeec1ec 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1798,7 +1798,7 @@ func (s *testPlanSuite) TestUnion(c *C) { }, { sql: "select a from t union select a from t union all select a from t", - best: "UnionAll{DataScan(t)->Projection->UnionAll{DataScan(t)->Projection->DataScan(t)->Projection}->Aggr(firstrow(t.a))->Projection}", + best: "UnionAll{UnionAll{DataScan(t)->Projection->DataScan(t)->Projection}->Aggr(firstrow(t.a))->Projection->DataScan(t)->Projection}", err: false, }, { @@ -1811,6 +1811,11 @@ func (s *testPlanSuite) TestUnion(c *C) { best: "", err: true, }, + { + sql: "select * from (select 1 as a union select 1 union all select 2) t order by a", + best: "UnionAll{UnionAll{Dual->Projection->Projection->Dual->Projection->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection->Projection}->Projection->Sort", + err: false, + }, } for i, tt := range tests { comment := Commentf("case:%v sql:%s", i, tt.sql) @@ -1825,7 +1830,7 @@ func (s *testPlanSuite) TestUnion(c *C) { plan, err := builder.build(stmt) if tt.err { c.Assert(err, NotNil) - return + continue } c.Assert(err, IsNil) p := plan.(LogicalPlan) From db7192ebfe55dcd84bce04ecc2a6d616f17a886c Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Sat, 10 Nov 2018 10:11:22 +0800 Subject: [PATCH 065/509] [release-2.1] vendor: update kvproto (#8246) --- Gopkg.lock | 7 +- Gopkg.toml | 2 +- .../pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go | 307 ++++++++++-------- 3 files changed, 184 insertions(+), 132 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 0b1a9469d6c3e..d5adffb288c28 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -273,8 +273,8 @@ revision = "8d44bfdf1030639ae7130922c95df12d6d4da3b6" [[projects]] - branch = "master" - digest = "1:84db0209caf3c48beaec43d15bd22ca6256d958b807c2ac626e2425e232e92c4" + branch = "release-2.1" + digest = "1:85aa2923e4dac3cef4dda68972f580cbc10dc1450f019f98a8b7e476f5eb2b07" name = "github.com/pingcap/kvproto" packages = [ "pkg/coprocessor", @@ -287,7 +287,7 @@ "pkg/tikvpb", ] pruneopts = "NUT" - revision = "529c652955d8fa74faf56f91b2f428d5779fd7d5" + revision = "8e3f33ac49297d7c93b61a955531191084a2f685" [[projects]] branch = "master" @@ -561,6 +561,7 @@ "github.com/pingcap/kvproto/pkg/metapb", "github.com/pingcap/kvproto/pkg/tikvpb", "github.com/pingcap/pd/client", + "github.com/pingcap/tidb-tools/tidb-binlog/node", "github.com/pingcap/tidb-tools/tidb-binlog/pump_client", "github.com/pingcap/tipb/go-binlog", "github.com/pingcap/tipb/go-tipb", diff --git a/Gopkg.toml b/Gopkg.toml index 3d9ff447b90dc..4ffa13e39d08d 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -74,7 +74,7 @@ required = ["github.com/golang/protobuf/jsonpb"] [[constraint]] name = "github.com/pingcap/kvproto" - branch = "master" + branch = "release-2.1" [[constraint]] name = "gopkg.in/natefinch/lumberjack.v2" diff --git a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go index 5b221822b95f9..a1d180d3638bd 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go @@ -556,6 +556,9 @@ type ScanRequest struct { Version uint64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"` KeyOnly bool `protobuf:"varint,5,opt,name=key_only,json=keyOnly,proto3" json:"key_only,omitempty"` Reverse bool `protobuf:"varint,6,opt,name=reverse,proto3" json:"reverse,omitempty"` + // For compatibility, when scanning forward, the range to scan is [start_key, end_key); and when scanning + // backward, the range is [end_key, start_key). + EndKey []byte `protobuf:"bytes,7,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` } func (m *ScanRequest) Reset() { *m = ScanRequest{} } @@ -605,6 +608,13 @@ func (m *ScanRequest) GetReverse() bool { return false } +func (m *ScanRequest) GetEndKey() []byte { + if m != nil { + return m.EndKey + } + return nil +} + type KvPair struct { Error *KeyError `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` @@ -2868,6 +2878,12 @@ func (m *ScanRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if len(m.EndKey) > 0 { + dAtA[i] = 0x3a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.EndKey))) + i += copy(dAtA[i:], m.EndKey) + } return i, nil } @@ -5338,6 +5354,10 @@ func (m *ScanRequest) Size() (n int) { if m.Reverse { n += 2 } + l = len(m.EndKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } return n } @@ -7947,6 +7967,37 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } } m.Reverse = bool(v != 0) + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EndKey", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.EndKey = append(m.EndKey[:0], dAtA[iNdEx:postIndex]...) + if m.EndKey == nil { + m.EndKey = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -15321,133 +15372,133 @@ var ( func init() { proto.RegisterFile("kvrpcpb.proto", fileDescriptorKvrpcpb) } var fileDescriptorKvrpcpb = []byte{ - // 2036 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xef, 0x6e, 0x1b, 0xc7, + // 2047 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x5f, 0x6f, 0x1b, 0xc7, 0x11, 0xf7, 0x1d, 0x8f, 0xe4, 0x71, 0x48, 0x51, 0xcc, 0x4a, 0xb6, 0x19, 0xbb, 0xb1, 0x95, 0x6b, - 0x0c, 0xcb, 0x2a, 0x2a, 0xa3, 0x4a, 0xd0, 0x4f, 0x45, 0x11, 0x58, 0x76, 0x1d, 0xc5, 0x72, 0x2d, - 0xac, 0x54, 0x17, 0x06, 0x9a, 0x32, 0xab, 0xe3, 0x8a, 0x3c, 0xf0, 0x78, 0x7b, 0xbe, 0x5b, 0x52, - 0x22, 0x8a, 0xa0, 0x28, 0x8a, 0x14, 0xc8, 0xc7, 0x16, 0x05, 0x5a, 0x14, 0xcd, 0x03, 0xf4, 0x05, - 0xfa, 0xa5, 0x2f, 0xd0, 0x8f, 0x7d, 0x84, 0xc2, 0x7d, 0x91, 0x62, 0xff, 0xdc, 0x1d, 0x8f, 0xa4, - 0x62, 0xf5, 0x4a, 0x29, 0x9f, 0x78, 0x3b, 0x33, 0xbb, 0x33, 0xbf, 0x99, 0xd9, 0xd9, 0x7f, 0x84, - 0x95, 0xc1, 0x38, 0x0a, 0xdd, 0xf0, 0x78, 0x3b, 0x8c, 0x18, 0x67, 0xa8, 0xaa, 0x9b, 0xb7, 0x1a, - 0x43, 0xca, 0x49, 0x42, 0xbe, 0xb5, 0x42, 0xa3, 0x88, 0x45, 0x69, 0x73, 0xbd, 0xc7, 0x7a, 0x4c, - 0x7e, 0x3e, 0x14, 0x5f, 0x8a, 0xea, 0x7c, 0x01, 0xf6, 0x3e, 0x73, 0x07, 0x7b, 0xc1, 0x09, 0x43, - 0xef, 0x43, 0x23, 0x8c, 0xbc, 0x21, 0x89, 0x26, 0x1d, 0x9f, 0xb9, 0x83, 0xb6, 0xb1, 0x61, 0x6c, - 0x36, 0x70, 0x5d, 0xd3, 0x84, 0x98, 0x10, 0x11, 0xac, 0xce, 0x98, 0x46, 0xb1, 0xc7, 0x82, 0xb6, - 0xb9, 0x61, 0x6c, 0x5a, 0xb8, 0x2e, 0x68, 0x2f, 0x15, 0x09, 0xb5, 0xa0, 0x34, 0xa0, 0x93, 0x76, - 0x49, 0x76, 0x16, 0x9f, 0xe8, 0x5d, 0xb0, 0x65, 0x27, 0xce, 0xfd, 0xb6, 0x25, 0x3b, 0x54, 0x45, - 0xfb, 0x88, 0xfb, 0xce, 0xd7, 0x06, 0xd8, 0xcf, 0xe8, 0xe4, 0x89, 0xb0, 0x14, 0x3d, 0x80, 0x8a, - 0xa0, 0xd3, 0xae, 0xd4, 0x5c, 0xdf, 0x79, 0x67, 0x3b, 0xc1, 0x99, 0x98, 0x88, 0xb5, 0x00, 0xfa, - 0x0e, 0xd4, 0x22, 0xca, 0xa3, 0x09, 0x39, 0xf6, 0xa9, 0x34, 0xa2, 0x86, 0x33, 0x02, 0x5a, 0x87, - 0x32, 0x39, 0x66, 0x11, 0x97, 0x46, 0xd4, 0xb0, 0x6a, 0xa0, 0x1d, 0xb0, 0x5d, 0x16, 0x9c, 0xf8, - 0x9e, 0xcb, 0xa5, 0x19, 0xf5, 0x9d, 0x1b, 0xa9, 0x82, 0x9f, 0x47, 0x1e, 0xa7, 0xbb, 0x9a, 0x8b, - 0x53, 0x39, 0xe7, 0x14, 0x56, 0x72, 0x2c, 0x81, 0x25, 0xe6, 0x24, 0xe2, 0x1d, 0x1e, 0x4b, 0x2b, - 0x2d, 0x5c, 0x95, 0xed, 0xa3, 0x18, 0xdd, 0x85, 0x7a, 0xd2, 0x4f, 0x70, 0x95, 0x6b, 0x20, 0x21, - 0x1d, 0xc5, 0x0b, 0x3c, 0xd3, 0x86, 0xaa, 0xf6, 0xae, 0xb4, 0xa8, 0x81, 0x93, 0xa6, 0xf3, 0x75, - 0x09, 0xaa, 0xbb, 0x2c, 0xe0, 0xf4, 0x8c, 0xa3, 0xdb, 0x02, 0x6c, 0xcf, 0x63, 0x41, 0xc7, 0xeb, - 0x6a, 0xa5, 0xb6, 0x22, 0xec, 0x75, 0xd1, 0x0f, 0xa1, 0xa1, 0x99, 0x34, 0x64, 0x6e, 0x5f, 0xaa, - 0xad, 0xef, 0xac, 0x6d, 0xeb, 0x54, 0xc0, 0x92, 0xf7, 0x44, 0xb0, 0x70, 0x3d, 0xca, 0x1a, 0x68, - 0x03, 0xac, 0x90, 0xd2, 0x48, 0x5a, 0x53, 0xdf, 0x69, 0x24, 0xf2, 0x07, 0x94, 0x46, 0x58, 0x72, - 0x10, 0x02, 0x8b, 0xd3, 0x68, 0xd8, 0x2e, 0x4b, 0x8d, 0xf2, 0x1b, 0x3d, 0x04, 0x3b, 0x8c, 0x3c, - 0x16, 0x79, 0x7c, 0xd2, 0xae, 0x6c, 0x18, 0x9b, 0xcd, 0x9d, 0xb5, 0xd4, 0x87, 0xbb, 0x6c, 0x38, - 0x24, 0x41, 0xf7, 0x20, 0xf2, 0x70, 0x2a, 0x84, 0x3e, 0x86, 0x55, 0x2f, 0x66, 0x3e, 0xe1, 0xc2, - 0x42, 0x9f, 0x8e, 0xa9, 0xdf, 0xae, 0xca, 0x7e, 0x37, 0xd3, 0x7e, 0x7b, 0x09, 0x7f, 0x5f, 0xb0, - 0x71, 0xd3, 0xcb, 0xb5, 0xd1, 0x07, 0xd0, 0x0c, 0x18, 0xef, 0x9c, 0x78, 0xbe, 0xdf, 0x71, 0x89, - 0xdb, 0xa7, 0x6d, 0x7b, 0xc3, 0xd8, 0xb4, 0x71, 0x23, 0x60, 0xfc, 0x27, 0x9e, 0xef, 0xef, 0x0a, - 0x9a, 0x8c, 0xcb, 0x24, 0x70, 0x3b, 0x3e, 0xeb, 0xb5, 0x6b, 0x92, 0x5f, 0x15, 0xed, 0x7d, 0xd6, - 0x13, 0x71, 0xe9, 0x93, 0xa0, 0xeb, 0xd3, 0x0e, 0xf7, 0x86, 0xb4, 0x0d, 0x92, 0x0b, 0x8a, 0x74, - 0xe4, 0x0d, 0xa9, 0x10, 0x88, 0x5d, 0x12, 0x74, 0xba, 0x94, 0x13, 0xcf, 0x6f, 0xd7, 0x95, 0x80, - 0x20, 0x3d, 0x96, 0x94, 0x4f, 0x2d, 0xdb, 0x6a, 0x95, 0x85, 0xfb, 0x48, 0xb7, 0xf3, 0x7a, 0xc4, - 0xa2, 0xd1, 0xd0, 0x79, 0x0c, 0xf0, 0x49, 0x36, 0xc2, 0x4d, 0xa8, 0x9e, 0x12, 0x8f, 0x77, 0x86, - 0x2a, 0x29, 0x4a, 0xb8, 0x22, 0x9a, 0xcf, 0x63, 0xf4, 0x1e, 0x40, 0x18, 0x31, 0x97, 0xc6, 0xb1, - 0xe0, 0x99, 0x92, 0x57, 0xd3, 0x94, 0xe7, 0xb1, 0xf3, 0x63, 0xb0, 0x0f, 0x5d, 0x12, 0xc8, 0xd9, - 0xb7, 0x0e, 0x65, 0xce, 0x38, 0xf1, 0xf5, 0x08, 0xaa, 0x21, 0x12, 0x5d, 0x8b, 0xd3, 0xee, 0x4c, - 0x7f, 0xda, 0x75, 0x7e, 0x6b, 0x00, 0x1c, 0xa6, 0x76, 0xa2, 0xfb, 0x50, 0x3e, 0x15, 0xd9, 0x3a, - 0x37, 0x7f, 0x12, 0x25, 0x58, 0xf1, 0xd1, 0x3d, 0xb0, 0xe4, 0x0c, 0x37, 0xcf, 0x93, 0x93, 0x6c, - 0x21, 0xd6, 0x25, 0x9c, 0xe8, 0x1c, 0x59, 0x24, 0x26, 0xd8, 0xce, 0x04, 0xea, 0x4f, 0xce, 0xa8, - 0xab, 0x8c, 0x88, 0xd1, 0x47, 0x79, 0x7f, 0x1b, 0x3a, 0x21, 0x93, 0xce, 0x99, 0xdb, 0x72, 0x41, - 0xf8, 0x28, 0x1f, 0x04, 0x73, 0xa6, 0x57, 0x86, 0x72, 0x3a, 0x32, 0x4e, 0x17, 0xe0, 0x29, 0xe5, - 0x98, 0xbe, 0x1e, 0xd1, 0x98, 0xa3, 0x2d, 0xa8, 0xba, 0x6a, 0xce, 0x68, 0xad, 0xad, 0xa9, 0xe4, - 0x94, 0x74, 0x9c, 0x08, 0x24, 0x93, 0xd1, 0xcc, 0x4d, 0xc6, 0xa4, 0xac, 0x95, 0xd4, 0xcc, 0xd6, - 0x4d, 0xe7, 0xd7, 0x50, 0x97, 0x5a, 0xe2, 0x90, 0x05, 0x31, 0x45, 0x3f, 0xc8, 0xa6, 0x9c, 0xa8, - 0x5b, 0x5a, 0x57, 0x73, 0x3b, 0xa9, 0xb7, 0xb2, 0x9a, 0xa5, 0xb3, 0x4d, 0x96, 0xb6, 0xfb, 0x50, - 0x56, 0xb2, 0xb3, 0x1e, 0x4f, 0x8a, 0x1f, 0x56, 0x7c, 0x91, 0x05, 0x63, 0xe2, 0x8f, 0xa8, 0xae, - 0x12, 0xaa, 0xe1, 0xfc, 0xc3, 0x80, 0xba, 0xf0, 0x40, 0x11, 0xa0, 0xb7, 0xa1, 0xa6, 0x2a, 0x56, - 0x06, 0x57, 0x95, 0xb0, 0x67, 0x74, 0x22, 0xd4, 0xf9, 0xde, 0xd0, 0x53, 0x95, 0x72, 0x05, 0xab, - 0xc6, 0xb4, 0x27, 0xac, 0x9c, 0x27, 0xc4, 0x34, 0x1b, 0xd0, 0x49, 0x87, 0x05, 0xfe, 0x44, 0xd6, - 0x05, 0x1b, 0x57, 0x07, 0x74, 0xf2, 0x22, 0xf0, 0xa5, 0xfb, 0x22, 0x2a, 0xe4, 0xa8, 0xac, 0x0c, - 0x36, 0x4e, 0x9a, 0xce, 0x2b, 0xa8, 0x3c, 0x1b, 0x1f, 0x10, 0x6f, 0xca, 0x0d, 0xc6, 0x5b, 0xdc, - 0x30, 0x1f, 0x9d, 0xc5, 0x8e, 0xe9, 0x43, 0x43, 0xf9, 0xa5, 0x78, 0x68, 0xee, 0x41, 0x39, 0x24, - 0x5e, 0x24, 0x66, 0x67, 0x69, 0xb3, 0xbe, 0xb3, 0x9a, 0xd9, 0x24, 0x6d, 0xc6, 0x8a, 0xeb, 0xbc, - 0x00, 0xfb, 0xf9, 0x88, 0xcb, 0xba, 0x84, 0x6e, 0x83, 0xc9, 0x42, 0x39, 0x76, 0x73, 0xa7, 0x9e, - 0xca, 0xbf, 0x08, 0xb1, 0xc9, 0xc2, 0x0b, 0x9b, 0xfe, 0xa5, 0x09, 0xab, 0x07, 0x11, 0x95, 0x13, - 0xb2, 0x48, 0x5c, 0x1f, 0x42, 0x6d, 0xa8, 0x0d, 0x4a, 0x6c, 0xcf, 0xfc, 0x99, 0x98, 0x8a, 0x33, - 0x99, 0xb9, 0xe5, 0xbd, 0x34, 0xbf, 0xbc, 0x7f, 0x17, 0x56, 0x54, 0xae, 0xe4, 0xc3, 0xdf, 0x90, - 0xc4, 0x97, 0x59, 0x0e, 0xa4, 0xcb, 0x79, 0x39, 0xb7, 0x9c, 0xa3, 0x1d, 0xb8, 0x1e, 0x0f, 0xbc, - 0xb0, 0xe3, 0xb2, 0x20, 0xe6, 0x11, 0xf1, 0x02, 0xde, 0x71, 0xfb, 0xd4, 0x1d, 0xe8, 0x8c, 0x58, - 0x13, 0xcc, 0xdd, 0x94, 0xb7, 0x2b, 0x58, 0x4e, 0x08, 0xad, 0xcc, 0x0d, 0xc5, 0xc3, 0xf8, 0x00, - 0x2a, 0x92, 0x3b, 0xef, 0x8b, 0x34, 0xb7, 0xb4, 0x80, 0xf3, 0x37, 0x03, 0x56, 0xc4, 0x62, 0xe5, - 0x15, 0x2a, 0x1c, 0x73, 0x3e, 0x32, 0x17, 0xf8, 0x08, 0x81, 0x35, 0xa0, 0x93, 0xb8, 0x5d, 0xda, - 0x28, 0x6d, 0x36, 0xb0, 0xfc, 0x46, 0xf7, 0xa0, 0xe9, 0x4a, 0xad, 0x33, 0xde, 0x5d, 0x51, 0x54, - 0xdd, 0xf5, 0x53, 0xcb, 0x2e, 0xb7, 0x2a, 0xb8, 0x72, 0xec, 0x05, 0x3e, 0xeb, 0x39, 0x3e, 0x34, - 0x13, 0x53, 0x2f, 0xbf, 0xfa, 0x38, 0x3d, 0x58, 0xd9, 0x1b, 0x86, 0x2c, 0x4a, 0x1d, 0x93, 0x4b, - 0x32, 0xe3, 0x02, 0x49, 0x36, 0x0f, 0xd2, 0x5c, 0x00, 0xd2, 0x79, 0x05, 0xcd, 0x44, 0x51, 0x71, - 0x58, 0xeb, 0xd3, 0xb0, 0x6a, 0x09, 0x86, 0x5f, 0xc1, 0xfa, 0x23, 0xc2, 0xdd, 0x3e, 0x66, 0xbe, - 0x7f, 0x4c, 0xdc, 0xc1, 0x55, 0xc6, 0xd8, 0x89, 0xe1, 0xfa, 0x8c, 0xf2, 0x2b, 0x88, 0x5a, 0x0c, - 0xcd, 0x5d, 0x9f, 0x92, 0x60, 0x14, 0x2e, 0x67, 0x21, 0x9c, 0x43, 0x5f, 0x9a, 0x47, 0xef, 0xfc, - 0xd1, 0x80, 0xd5, 0x54, 0xeb, 0x15, 0x2c, 0x8c, 0xf3, 0x89, 0x55, 0x5a, 0x94, 0x58, 0x03, 0x58, - 0x95, 0x01, 0x28, 0xb8, 0x2b, 0x48, 0x62, 0x6a, 0x4e, 0xcd, 0xdb, 0xf3, 0xf7, 0x05, 0x3e, 0xb4, - 0x32, 0x65, 0x97, 0xbe, 0x02, 0xfd, 0xde, 0x80, 0x55, 0xb1, 0xd8, 0x89, 0x4a, 0x5d, 0x04, 0xdb, - 0x5d, 0xa8, 0x0f, 0xc9, 0xd9, 0x4c, 0x4a, 0xc3, 0x90, 0x9c, 0x25, 0x09, 0x9d, 0xdb, 0x29, 0x94, - 0xce, 0xdb, 0x29, 0x58, 0x53, 0x3b, 0x05, 0xe7, 0x4f, 0x06, 0xb4, 0x32, 0x9b, 0xae, 0x20, 0x0d, - 0xee, 0x43, 0x59, 0x2c, 0x36, 0x6a, 0xd6, 0x2d, 0x3c, 0x22, 0x2a, 0xbe, 0xf3, 0x21, 0x54, 0x8f, - 0xce, 0xd4, 0xce, 0xba, 0x05, 0x25, 0x7e, 0x16, 0xe8, 0x93, 0x93, 0xf8, 0x44, 0x37, 0xa0, 0x12, - 0x73, 0xc2, 0x47, 0xc9, 0x29, 0x4d, 0xb7, 0xc4, 0x3e, 0x0b, 0x61, 0x1a, 0x33, 0x7f, 0x4c, 0x8b, - 0x7a, 0xf9, 0x42, 0xa5, 0xe3, 0x62, 0xc9, 0x8c, 0xbe, 0x0f, 0x35, 0x7e, 0x16, 0x74, 0xbc, 0xe0, - 0x84, 0xc5, 0x6d, 0x4b, 0x02, 0xce, 0x34, 0x6b, 0x74, 0xd8, 0xe6, 0xea, 0x23, 0x76, 0x5e, 0xc3, - 0x5a, 0xce, 0xf8, 0x2b, 0x28, 0x3d, 0x2f, 0xa1, 0xf6, 0x74, 0xb7, 0x88, 0x9b, 0xde, 0x03, 0x88, - 0xc9, 0x09, 0xed, 0x84, 0xcc, 0x0b, 0xb8, 0xf6, 0x51, 0x4d, 0x50, 0x0e, 0x04, 0xc1, 0xe9, 0x03, - 0x88, 0x71, 0xaf, 0x00, 0xc1, 0x67, 0xb0, 0x82, 0xc9, 0xe9, 0xd2, 0x0e, 0x11, 0x4d, 0x30, 0xdd, - 0x13, 0x7d, 0xef, 0x60, 0xba, 0x27, 0x0e, 0x83, 0x66, 0x32, 0xfc, 0x92, 0x17, 0xba, 0x73, 0xb6, - 0x95, 0xb1, 0xc4, 0x73, 0x30, 0x5a, 0x12, 0x9e, 0x85, 0x4a, 0x34, 0x4a, 0x2b, 0x45, 0xf9, 0x4a, - 0xa2, 0x94, 0x4a, 0x97, 0xbd, 0x9c, 0x9f, 0x02, 0xc2, 0xe4, 0x54, 0x96, 0xd9, 0x82, 0xa0, 0x2e, - 0x56, 0x5e, 0xe7, 0x22, 0xf7, 0x4b, 0x58, 0xcb, 0x29, 0x5e, 0x36, 0xb0, 0x6e, 0x06, 0x6c, 0x89, - 0x8b, 0xd5, 0x7c, 0xfe, 0xad, 0xe5, 0xb4, 0x5c, 0xfa, 0x2a, 0xf5, 0x39, 0xb4, 0x30, 0x39, 0x7d, - 0x4c, 0x7d, 0x5a, 0xec, 0x58, 0xf3, 0xf6, 0x29, 0xf5, 0x0b, 0x78, 0x67, 0x4a, 0xc3, 0xb2, 0xc3, - 0xd2, 0x83, 0xeb, 0x89, 0xc3, 0x8a, 0x83, 0xb8, 0x48, 0x64, 0x08, 0xdc, 0x98, 0x55, 0xb4, 0x6c, - 0x2c, 0x63, 0x40, 0x7a, 0x68, 0x12, 0xf4, 0xe8, 0xd2, 0x2f, 0x0f, 0x6e, 0x42, 0x95, 0x06, 0xdd, - 0xa9, 0xdd, 0x42, 0x85, 0x06, 0xdd, 0x67, 0x74, 0x22, 0xa6, 0x4e, 0x4e, 0xef, 0xb2, 0x71, 0x7d, - 0x65, 0xc8, 0x20, 0x7d, 0x1b, 0xd8, 0xe6, 0x4a, 0x9f, 0x0a, 0xe3, 0xa5, 0xc2, 0xfd, 0x8b, 0x21, - 0xcb, 0xeb, 0x15, 0x5e, 0x00, 0x4d, 0x5f, 0xf3, 0x58, 0xf9, 0x6b, 0x1e, 0x85, 0xbf, 0x9c, 0xe2, - 0xef, 0xc1, 0x6a, 0x6a, 0x5b, 0x71, 0xe0, 0xef, 0x43, 0x69, 0x30, 0x3e, 0xb7, 0xb4, 0x08, 0x9e, - 0xf3, 0xb1, 0x7c, 0x29, 0x90, 0x2e, 0xce, 0x43, 0x32, 0xce, 0x0f, 0x9d, 0x99, 0x4b, 0xcb, 0xbf, - 0x1b, 0x59, 0x31, 0x2c, 0xea, 0xcc, 0x07, 0x50, 0x89, 0x84, 0x09, 0x0b, 0xaf, 0x19, 0x54, 0xfc, - 0xb5, 0x80, 0xd8, 0xe2, 0x50, 0xe2, 0xf6, 0x3b, 0xd3, 0xfe, 0xad, 0x09, 0xca, 0xfe, 0xff, 0xea, - 0x63, 0x1f, 0xd6, 0xf3, 0x76, 0x5f, 0xaa, 0xa3, 0xbf, 0x34, 0xa0, 0xf6, 0x7c, 0xec, 0xba, 0xf2, - 0xe1, 0x03, 0xdd, 0x05, 0x8b, 0x4f, 0x42, 0xba, 0xe8, 0xb6, 0x4b, 0x32, 0x72, 0x2f, 0x22, 0x66, - 0xfe, 0x45, 0xe4, 0x36, 0xd4, 0xf4, 0x36, 0x97, 0xc7, 0x7a, 0x87, 0x6b, 0x2b, 0x82, 0x7a, 0x2e, - 0x89, 0xfb, 0x4c, 0x6c, 0x94, 0xe5, 0xfe, 0x42, 0xbd, 0x7f, 0x80, 0x24, 0xbd, 0x94, 0x3b, 0x99, - 0x1f, 0x29, 0x33, 0x64, 0xe3, 0x9b, 0xde, 0x5d, 0xd2, 0x2d, 0x8a, 0x39, 0xbd, 0x0f, 0xfa, 0x8d, - 0x01, 0xb6, 0xe8, 0x2e, 0xef, 0xb5, 0xfe, 0x1f, 0x10, 0x53, 0x6f, 0x34, 0xa5, 0xdc, 0x1b, 0xcd, - 0xdb, 0x11, 0x7c, 0xa5, 0x6d, 0x90, 0xa7, 0x90, 0xe4, 0xce, 0x7d, 0xf6, 0xea, 0x33, 0x31, 0x52, - 0xdf, 0xb9, 0x6f, 0x41, 0x45, 0xde, 0x85, 0x25, 0x31, 0x42, 0x39, 0x41, 0x19, 0x13, 0xac, 0x25, - 0x84, 0xac, 0x54, 0x9d, 0x9c, 0x86, 0xf2, 0xb2, 0xd2, 0x06, 0xac, 0x25, 0x9c, 0x43, 0x58, 0x13, - 0xc4, 0xa7, 0x94, 0x3f, 0x9a, 0x88, 0x54, 0x5d, 0xc6, 0xd2, 0xec, 0xfc, 0xce, 0x80, 0xf5, 0xfc, - 0xa8, 0xcb, 0xde, 0xe4, 0xde, 0x03, 0x4b, 0x1c, 0x7f, 0xe6, 0x9e, 0x20, 0x12, 0xb7, 0x62, 0xc9, - 0x76, 0x3e, 0x87, 0x9b, 0xa9, 0x1d, 0x87, 0x2a, 0x70, 0x45, 0x10, 0x9e, 0x9f, 0x06, 0xce, 0x5f, - 0x0d, 0x68, 0xcf, 0xab, 0x58, 0x36, 0xdc, 0xf9, 0x27, 0xc2, 0xc4, 0x01, 0xd6, 0x37, 0x3b, 0xe0, - 0x33, 0x40, 0x87, 0xa1, 0xef, 0x71, 0xf5, 0xde, 0x57, 0x74, 0x99, 0x10, 0x23, 0xe4, 0x96, 0x09, - 0x41, 0x10, 0xa5, 0xf3, 0x0f, 0x06, 0xac, 0xe5, 0xc6, 0x2f, 0x0e, 0xdc, 0x01, 0xcb, 0xa7, 0x27, - 0x5c, 0x1f, 0xcc, 0x9a, 0xf9, 0x87, 0x4a, 0x2c, 0x79, 0xe8, 0x03, 0x28, 0x47, 0x5e, 0xaf, 0xcf, - 0x75, 0xd8, 0x67, 0x85, 0x14, 0xd3, 0xf9, 0x02, 0xde, 0xfd, 0x59, 0x20, 0xce, 0x8c, 0x8f, 0x69, - 0xcc, 0x23, 0x36, 0xb9, 0xe2, 0x5d, 0x0e, 0x85, 0x5b, 0x8b, 0xd4, 0x2f, 0x39, 0x25, 0xb6, 0xbe, - 0x07, 0x90, 0xbd, 0xac, 0x22, 0x80, 0xca, 0x4f, 0x59, 0x34, 0x24, 0x7e, 0xeb, 0x1a, 0xaa, 0x42, - 0x69, 0x9f, 0x9d, 0xb6, 0x0c, 0x64, 0x83, 0xf5, 0x89, 0xd7, 0xeb, 0xb7, 0xcc, 0xad, 0x0d, 0x68, - 0xe6, 0x9f, 0x53, 0x51, 0x05, 0xcc, 0xc3, 0xbd, 0xd6, 0x35, 0xf1, 0x8b, 0x77, 0x5b, 0xc6, 0xd6, - 0x36, 0x98, 0x2f, 0x42, 0xd1, 0xf5, 0x60, 0xc4, 0xd5, 0x18, 0x8f, 0xa9, 0xaf, 0xc6, 0x10, 0x55, - 0xa8, 0x65, 0xa2, 0x06, 0xd8, 0xc9, 0xf5, 0x65, 0xab, 0xf4, 0x68, 0xeb, 0x9f, 0x6f, 0xee, 0x18, - 0xff, 0x7a, 0x73, 0xc7, 0xf8, 0xf7, 0x9b, 0x3b, 0xc6, 0x9f, 0xff, 0x73, 0xe7, 0x1a, 0xb4, 0x5d, - 0x36, 0xdc, 0x0e, 0xbd, 0xa0, 0xe7, 0x92, 0x70, 0x9b, 0x7b, 0x83, 0xf1, 0xf6, 0x60, 0x2c, 0xff, - 0x4c, 0x70, 0x5c, 0x91, 0x3f, 0x1f, 0xfe, 0x37, 0x00, 0x00, 0xff, 0xff, 0xc3, 0x3e, 0x6a, 0x3d, - 0xa0, 0x20, 0x00, 0x00, + 0x0c, 0xcb, 0x2a, 0x2a, 0xa3, 0x4a, 0xd0, 0xa7, 0xa2, 0x08, 0x2c, 0xb9, 0x8e, 0x62, 0xb9, 0x16, + 0x56, 0xaa, 0x0b, 0x03, 0x4d, 0x98, 0xd5, 0x71, 0x45, 0x1e, 0x78, 0xbc, 0x3d, 0xdf, 0xad, 0x28, + 0x11, 0x45, 0x50, 0x14, 0x45, 0x0a, 0xe4, 0xb1, 0x45, 0x81, 0x16, 0x45, 0xf3, 0x01, 0xfa, 0x05, + 0xfa, 0xd2, 0x2f, 0xd0, 0xc7, 0x3e, 0xf4, 0x03, 0x14, 0xee, 0x17, 0x29, 0xf6, 0xcf, 0xdd, 0xf1, + 0x48, 0xca, 0x56, 0xaf, 0x94, 0xf2, 0xc4, 0xdb, 0x99, 0xd9, 0x9d, 0xf9, 0xcd, 0xcc, 0xce, 0xfe, + 0x23, 0x2c, 0x0d, 0x46, 0x51, 0xe8, 0x86, 0x47, 0x9b, 0x61, 0xc4, 0x38, 0x43, 0x55, 0xdd, 0xbc, + 0xd5, 0x18, 0x52, 0x4e, 0x12, 0xf2, 0xad, 0x25, 0x1a, 0x45, 0x2c, 0x4a, 0x9b, 0xab, 0x3d, 0xd6, + 0x63, 0xf2, 0xf3, 0xa1, 0xf8, 0x52, 0x54, 0xe7, 0x4b, 0xb0, 0xf7, 0x98, 0x3b, 0xd8, 0x0d, 0x8e, + 0x19, 0x7a, 0x1f, 0x1a, 0x61, 0xe4, 0x0d, 0x49, 0x34, 0xee, 0xf8, 0xcc, 0x1d, 0xb4, 0x8d, 0x35, + 0x63, 0xbd, 0x81, 0xeb, 0x9a, 0x26, 0xc4, 0x84, 0x88, 0x60, 0x75, 0x46, 0x34, 0x8a, 0x3d, 0x16, + 0xb4, 0xcd, 0x35, 0x63, 0xdd, 0xc2, 0x75, 0x41, 0x7b, 0xa1, 0x48, 0xa8, 0x05, 0xa5, 0x01, 0x1d, + 0xb7, 0x4b, 0xb2, 0xb3, 0xf8, 0x44, 0xef, 0x82, 0x2d, 0x3b, 0x71, 0xee, 0xb7, 0x2d, 0xd9, 0xa1, + 0x2a, 0xda, 0x87, 0xdc, 0x77, 0xbe, 0x31, 0xc0, 0x7e, 0x4a, 0xc7, 0x8f, 0x85, 0xa5, 0xe8, 0x01, + 0x54, 0x04, 0x9d, 0x76, 0xa5, 0xe6, 0xfa, 0xd6, 0x3b, 0x9b, 0x09, 0xce, 0xc4, 0x44, 0xac, 0x05, + 0xd0, 0x77, 0xa0, 0x16, 0x51, 0x1e, 0x8d, 0xc9, 0x91, 0x4f, 0xa5, 0x11, 0x35, 0x9c, 0x11, 0xd0, + 0x2a, 0x94, 0xc9, 0x11, 0x8b, 0xb8, 0x34, 0xa2, 0x86, 0x55, 0x03, 0x6d, 0x81, 0xed, 0xb2, 0xe0, + 0xd8, 0xf7, 0x5c, 0x2e, 0xcd, 0xa8, 0x6f, 0xdd, 0x48, 0x15, 0xfc, 0x3c, 0xf2, 0x38, 0xdd, 0xd6, + 0x5c, 0x9c, 0xca, 0x39, 0xa7, 0xb0, 0x94, 0x63, 0x09, 0x2c, 0x31, 0x27, 0x11, 0xef, 0xf0, 0x58, + 0x5a, 0x69, 0xe1, 0xaa, 0x6c, 0x1f, 0xc6, 0xe8, 0x2e, 0xd4, 0x93, 0x7e, 0x82, 0xab, 0x5c, 0x03, + 0x09, 0xe9, 0x30, 0x9e, 0xe3, 0x99, 0x36, 0x54, 0xb5, 0x77, 0xa5, 0x45, 0x0d, 0x9c, 0x34, 0x9d, + 0x6f, 0x4a, 0x50, 0xdd, 0x66, 0x01, 0xa7, 0x67, 0x1c, 0xdd, 0x16, 0x60, 0x7b, 0x1e, 0x0b, 0x3a, + 0x5e, 0x57, 0x2b, 0xb5, 0x15, 0x61, 0xb7, 0x8b, 0x7e, 0x08, 0x0d, 0xcd, 0xa4, 0x21, 0x73, 0xfb, + 0x52, 0x6d, 0x7d, 0x6b, 0x65, 0x53, 0xa7, 0x02, 0x96, 0xbc, 0xc7, 0x82, 0x85, 0xeb, 0x51, 0xd6, + 0x40, 0x6b, 0x60, 0x85, 0x94, 0x46, 0xd2, 0x9a, 0xfa, 0x56, 0x23, 0x91, 0xdf, 0xa7, 0x34, 0xc2, + 0x92, 0x83, 0x10, 0x58, 0x9c, 0x46, 0xc3, 0x76, 0x59, 0x6a, 0x94, 0xdf, 0xe8, 0x21, 0xd8, 0x61, + 0xe4, 0xb1, 0xc8, 0xe3, 0xe3, 0x76, 0x65, 0xcd, 0x58, 0x6f, 0x6e, 0xad, 0xa4, 0x3e, 0xdc, 0x66, + 0xc3, 0x21, 0x09, 0xba, 0xfb, 0x91, 0x87, 0x53, 0x21, 0xf4, 0x31, 0x2c, 0x7b, 0x31, 0xf3, 0x09, + 0x17, 0x16, 0xfa, 0x74, 0x44, 0xfd, 0x76, 0x55, 0xf6, 0xbb, 0x99, 0xf6, 0xdb, 0x4d, 0xf8, 0x7b, + 0x82, 0x8d, 0x9b, 0x5e, 0xae, 0x8d, 0x3e, 0x80, 0x66, 0xc0, 0x78, 0xe7, 0xd8, 0xf3, 0xfd, 0x8e, + 0x4b, 0xdc, 0x3e, 0x6d, 0xdb, 0x6b, 0xc6, 0xba, 0x8d, 0x1b, 0x01, 0xe3, 0x3f, 0xf1, 0x7c, 0x7f, + 0x5b, 0xd0, 0x64, 0x5c, 0xc6, 0x81, 0xdb, 0xf1, 0x59, 0xaf, 0x5d, 0x93, 0xfc, 0xaa, 0x68, 0xef, + 0xb1, 0x9e, 0x88, 0x4b, 0x9f, 0x04, 0x5d, 0x9f, 0x76, 0xb8, 0x37, 0xa4, 0x6d, 0x90, 0x5c, 0x50, + 0xa4, 0x43, 0x6f, 0x48, 0x85, 0x40, 0xec, 0x92, 0xa0, 0xd3, 0xa5, 0x9c, 0x78, 0x7e, 0xbb, 0xae, + 0x04, 0x04, 0x69, 0x47, 0x52, 0x3e, 0xb5, 0x6c, 0xab, 0x55, 0x16, 0xee, 0x23, 0xdd, 0xce, 0xab, + 0x13, 0x16, 0x9d, 0x0c, 0x9d, 0x1d, 0x80, 0x4f, 0xb2, 0x11, 0x6e, 0x42, 0xf5, 0x94, 0x78, 0xbc, + 0x33, 0x54, 0x49, 0x51, 0xc2, 0x15, 0xd1, 0x7c, 0x16, 0xa3, 0xf7, 0x00, 0xc2, 0x88, 0xb9, 0x34, + 0x8e, 0x05, 0xcf, 0x94, 0xbc, 0x9a, 0xa6, 0x3c, 0x8b, 0x9d, 0x1f, 0x83, 0x7d, 0xe0, 0x92, 0x40, + 0xce, 0xbe, 0x55, 0x28, 0x73, 0xc6, 0x89, 0xaf, 0x47, 0x50, 0x0d, 0x91, 0xe8, 0x5a, 0x9c, 0x76, + 0xa7, 0xfa, 0xd3, 0xae, 0xf3, 0x1b, 0x03, 0xe0, 0x20, 0xb5, 0x13, 0xdd, 0x87, 0xf2, 0xa9, 0xc8, + 0xd6, 0x99, 0xf9, 0x93, 0x28, 0xc1, 0x8a, 0x8f, 0xee, 0x81, 0x25, 0x67, 0xb8, 0x79, 0x9e, 0x9c, + 0x64, 0x0b, 0xb1, 0x2e, 0xe1, 0x44, 0xe7, 0xc8, 0x3c, 0x31, 0xc1, 0x76, 0xc6, 0x50, 0x7f, 0x7c, + 0x46, 0x5d, 0x65, 0x44, 0x8c, 0x3e, 0xca, 0xfb, 0xdb, 0xd0, 0x09, 0x99, 0x74, 0xce, 0xdc, 0x96, + 0x0b, 0xc2, 0x47, 0xf9, 0x20, 0x98, 0x53, 0xbd, 0x32, 0x94, 0x93, 0x91, 0x71, 0xba, 0x00, 0x4f, + 0x28, 0xc7, 0xf4, 0xd5, 0x09, 0x8d, 0x39, 0xda, 0x80, 0xaa, 0xab, 0xe6, 0x8c, 0xd6, 0xda, 0x9a, + 0x48, 0x4e, 0x49, 0xc7, 0x89, 0x40, 0x32, 0x19, 0xcd, 0xdc, 0x64, 0x4c, 0xca, 0x5a, 0x49, 0xcd, + 0x6c, 0xdd, 0x74, 0x7e, 0x05, 0x75, 0xa9, 0x25, 0x0e, 0x59, 0x10, 0x53, 0xf4, 0x83, 0x6c, 0xca, + 0x89, 0xba, 0xa5, 0x75, 0x35, 0x37, 0x93, 0x7a, 0x2b, 0xab, 0x59, 0x3a, 0xdb, 0x64, 0x69, 0xbb, + 0x0f, 0x65, 0x25, 0x3b, 0xed, 0xf1, 0xa4, 0xf8, 0x61, 0xc5, 0x17, 0x59, 0x30, 0x22, 0xfe, 0x09, + 0xd5, 0x55, 0x42, 0x35, 0x9c, 0x7f, 0x19, 0x50, 0x17, 0x1e, 0x28, 0x02, 0xf4, 0x36, 0xd4, 0x54, + 0xc5, 0xca, 0xe0, 0xaa, 0x12, 0xf6, 0x94, 0x8e, 0x85, 0x3a, 0xdf, 0x1b, 0x7a, 0xaa, 0x52, 0x2e, + 0x61, 0xd5, 0x98, 0xf4, 0x84, 0x95, 0xf3, 0x84, 0x98, 0x66, 0x03, 0x3a, 0xee, 0xb0, 0xc0, 0x1f, + 0xcb, 0xba, 0x60, 0xe3, 0xea, 0x80, 0x8e, 0x9f, 0x07, 0xbe, 0x74, 0x5f, 0x44, 0x85, 0x1c, 0x95, + 0x95, 0xc1, 0xc6, 0x49, 0x53, 0xcc, 0x0e, 0x1a, 0x74, 0xa5, 0xfe, 0xaa, 0xd4, 0x5f, 0xa1, 0x41, + 0xf7, 0x29, 0x1d, 0x3b, 0x2f, 0xa1, 0xf2, 0x74, 0xb4, 0x4f, 0xbc, 0x09, 0xff, 0x18, 0x6f, 0xf1, + 0xcf, 0x6c, 0xd8, 0xe6, 0x7b, 0xac, 0x0f, 0x0d, 0xe5, 0xb0, 0xe2, 0x31, 0xbb, 0x07, 0xe5, 0x90, + 0x78, 0x91, 0x98, 0xb6, 0xa5, 0xf5, 0xfa, 0xd6, 0x72, 0x66, 0x93, 0xb4, 0x19, 0x2b, 0xae, 0xf3, + 0x1c, 0xec, 0x67, 0x27, 0x5c, 0x16, 0x2c, 0x74, 0x1b, 0x4c, 0x16, 0xca, 0xb1, 0x9b, 0x5b, 0xf5, + 0x54, 0xfe, 0x79, 0x88, 0x4d, 0x16, 0x5e, 0xd8, 0xf4, 0xaf, 0x4c, 0x58, 0xde, 0x8f, 0xa8, 0x9c, + 0xa9, 0x45, 0x02, 0xfe, 0x10, 0x6a, 0x43, 0x6d, 0x50, 0x62, 0x7b, 0xe6, 0xcf, 0xc4, 0x54, 0x9c, + 0xc9, 0xcc, 0xac, 0xfb, 0xa5, 0xd9, 0x75, 0xff, 0xbb, 0xb0, 0xa4, 0x92, 0x28, 0x9f, 0x17, 0x0d, + 0x49, 0x7c, 0x91, 0x25, 0x47, 0xba, 0xce, 0x97, 0x73, 0xeb, 0x3c, 0xda, 0x82, 0xeb, 0xf1, 0xc0, + 0x0b, 0x3b, 0x2e, 0x0b, 0x62, 0x1e, 0x11, 0x2f, 0xe0, 0x1d, 0xb7, 0x4f, 0xdd, 0x81, 0x4e, 0x95, + 0x15, 0xc1, 0xdc, 0x4e, 0x79, 0xdb, 0x82, 0xe5, 0x84, 0xd0, 0xca, 0xdc, 0x50, 0x3c, 0x8c, 0x0f, + 0xa0, 0x22, 0xb9, 0xb3, 0xbe, 0x48, 0x73, 0x4b, 0x0b, 0x38, 0x7f, 0x35, 0x60, 0x49, 0xac, 0x62, + 0x5e, 0xa1, 0x8a, 0x32, 0xe3, 0x23, 0x73, 0x8e, 0x8f, 0x10, 0x58, 0x03, 0x3a, 0x8e, 0xdb, 0xa5, + 0xb5, 0xd2, 0x7a, 0x03, 0xcb, 0x6f, 0x74, 0x0f, 0x9a, 0xae, 0xd4, 0x3a, 0xe5, 0xdd, 0x25, 0x45, + 0xd5, 0x5d, 0x3f, 0xb5, 0xec, 0x72, 0xab, 0x82, 0x2b, 0x47, 0x5e, 0xe0, 0xb3, 0x9e, 0xe3, 0x43, + 0x33, 0x31, 0xf5, 0xf2, 0xcb, 0x92, 0xd3, 0x83, 0xa5, 0xdd, 0x61, 0xc8, 0xa2, 0xd4, 0x31, 0xb9, + 0x24, 0x33, 0x2e, 0x90, 0x64, 0xb3, 0x20, 0xcd, 0x39, 0x20, 0x9d, 0x97, 0xd0, 0x4c, 0x14, 0x15, + 0x87, 0xb5, 0x3a, 0x09, 0xab, 0x96, 0x60, 0xf8, 0x25, 0xac, 0x3e, 0x22, 0xdc, 0xed, 0x63, 0xe6, + 0xfb, 0x47, 0xc4, 0x1d, 0x5c, 0x65, 0x8c, 0x9d, 0x18, 0xae, 0x4f, 0x29, 0xbf, 0x82, 0xa8, 0xc5, + 0xd0, 0xdc, 0xf6, 0x29, 0x09, 0x4e, 0xc2, 0xc5, 0xac, 0x90, 0x33, 0xe8, 0x4b, 0xb3, 0xe8, 0x9d, + 0x3f, 0x18, 0xb0, 0x9c, 0x6a, 0xbd, 0x82, 0x15, 0x73, 0x36, 0xb1, 0x4a, 0xf3, 0x12, 0x6b, 0x00, + 0xcb, 0x32, 0x00, 0x05, 0xb7, 0x0b, 0x49, 0x4c, 0xcd, 0x89, 0x79, 0x7b, 0xfe, 0x86, 0xc1, 0x87, + 0x56, 0xa6, 0xec, 0xd2, 0x57, 0xa0, 0xdf, 0x19, 0xb0, 0x2c, 0x16, 0x3b, 0x51, 0xa9, 0x8b, 0x60, + 0xbb, 0x0b, 0xf5, 0x21, 0x39, 0x9b, 0x4a, 0x69, 0x18, 0x92, 0xb3, 0x24, 0xa1, 0x73, 0x5b, 0x88, + 0xd2, 0x79, 0x5b, 0x08, 0x6b, 0x62, 0x0b, 0xe1, 0xfc, 0xd1, 0x80, 0x56, 0x66, 0xd3, 0x15, 0xa4, + 0xc1, 0x7d, 0x28, 0x8b, 0xc5, 0x46, 0xcd, 0xba, 0xb9, 0x67, 0x47, 0xc5, 0x77, 0x3e, 0x84, 0xea, + 0xe1, 0x99, 0xda, 0x72, 0xb7, 0xa0, 0xc4, 0xcf, 0x02, 0x7d, 0xa4, 0x12, 0x9f, 0xe8, 0x06, 0x54, + 0x62, 0x4e, 0xf8, 0x49, 0x72, 0x7c, 0xd3, 0x2d, 0xe7, 0xef, 0x06, 0x20, 0x4c, 0x63, 0xe6, 0x8f, + 0x68, 0x51, 0x2f, 0x5f, 0xa8, 0x74, 0x5c, 0x2c, 0x99, 0xd1, 0xf7, 0xa1, 0xc6, 0xcf, 0x82, 0x8e, + 0x17, 0x1c, 0xb3, 0xb8, 0x6d, 0x49, 0xc0, 0x99, 0x66, 0x8d, 0x0e, 0xdb, 0x5c, 0x7d, 0xc4, 0xce, + 0x2b, 0x58, 0xc9, 0x19, 0x7f, 0x05, 0xa5, 0xe7, 0x05, 0xd4, 0x9e, 0x6c, 0x17, 0x71, 0xd3, 0x7b, + 0x00, 0x31, 0x39, 0xa6, 0x9d, 0x90, 0x79, 0x01, 0xd7, 0x3e, 0xaa, 0x09, 0xca, 0xbe, 0x20, 0x38, + 0x7d, 0x00, 0x31, 0xee, 0x15, 0x20, 0xf8, 0x0c, 0x96, 0x30, 0x39, 0x5d, 0xd8, 0xe9, 0xa2, 0x09, + 0xa6, 0x7b, 0xac, 0x2f, 0x24, 0x4c, 0xf7, 0xd8, 0x61, 0xd0, 0x4c, 0x86, 0x5f, 0xf0, 0x42, 0x77, + 0xce, 0xb6, 0x32, 0x96, 0x78, 0xf6, 0x4f, 0x16, 0x84, 0x67, 0xae, 0x12, 0x8d, 0xd2, 0x4a, 0x51, + 0xbe, 0x94, 0x28, 0xa5, 0xd2, 0x45, 0x2f, 0xe7, 0xa7, 0x80, 0x30, 0x39, 0x95, 0x65, 0xb6, 0x20, + 0xa8, 0x8b, 0x95, 0xd7, 0x99, 0xc8, 0x7d, 0x0e, 0x2b, 0x39, 0xc5, 0x8b, 0x06, 0xd6, 0xcd, 0x80, + 0x2d, 0x70, 0xb1, 0x9a, 0xcd, 0xbf, 0x95, 0x9c, 0x96, 0x4b, 0x5f, 0xa5, 0xbe, 0x80, 0x16, 0x26, + 0xa7, 0x3b, 0xd4, 0xa7, 0xc5, 0x8e, 0x35, 0x6f, 0x9f, 0x52, 0xbf, 0x80, 0x77, 0x26, 0x34, 0x2c, + 0x3a, 0x2c, 0x3d, 0xb8, 0x9e, 0x38, 0xac, 0x38, 0x88, 0x8b, 0x44, 0x86, 0xc0, 0x8d, 0x69, 0x45, + 0x8b, 0xc6, 0x32, 0x02, 0xa4, 0x87, 0x26, 0x41, 0x8f, 0x2e, 0xfc, 0x56, 0x61, 0xe2, 0xc0, 0x5f, + 0xca, 0x1d, 0xf8, 0x3f, 0x87, 0x95, 0x9c, 0xde, 0x45, 0xe3, 0xfa, 0xda, 0x90, 0x41, 0xfa, 0x36, + 0xb0, 0xcd, 0x94, 0x3e, 0x15, 0xc6, 0x4b, 0x85, 0xfb, 0x67, 0x43, 0x96, 0xd7, 0x2b, 0xbc, 0x19, + 0x9a, 0xbc, 0xff, 0xb1, 0xf2, 0xf7, 0x3f, 0x0a, 0x7f, 0x39, 0xc5, 0xdf, 0x83, 0xe5, 0xd4, 0xb6, + 0xe2, 0xc0, 0xdf, 0x87, 0xd2, 0x60, 0x74, 0x6e, 0x69, 0x11, 0x3c, 0xe7, 0x63, 0xf9, 0x84, 0x20, + 0x5d, 0x9c, 0x87, 0x64, 0x9c, 0x1f, 0x3a, 0x33, 0x97, 0x96, 0x7f, 0x33, 0xb2, 0x62, 0x58, 0xd4, + 0x99, 0x0f, 0xa0, 0x12, 0x09, 0x13, 0xe6, 0x5e, 0x33, 0xa8, 0xf8, 0x6b, 0x01, 0xb1, 0xc5, 0xa1, + 0xc4, 0xed, 0x77, 0x26, 0xfd, 0x5b, 0x13, 0x94, 0xbd, 0xff, 0xd5, 0xc7, 0x3e, 0xac, 0xe6, 0xed, + 0xbe, 0x54, 0x47, 0x7f, 0x65, 0x40, 0xed, 0xd9, 0xc8, 0x75, 0xe5, 0x8b, 0x08, 0xba, 0x0b, 0x16, + 0x1f, 0x87, 0x74, 0xde, 0x6d, 0x97, 0x64, 0xe4, 0x9e, 0x4a, 0xcc, 0xfc, 0x53, 0xc9, 0x6d, 0xa8, + 0xe9, 0x6d, 0x2e, 0x8f, 0xf5, 0x0e, 0xd7, 0x56, 0x04, 0xf5, 0x8e, 0x12, 0xf7, 0x99, 0xd8, 0x28, + 0xcb, 0xfd, 0x85, 0x7a, 0x18, 0x01, 0x49, 0x7a, 0x21, 0x77, 0x32, 0x3f, 0x52, 0x66, 0xc8, 0xc6, + 0x9b, 0x1e, 0x64, 0xd2, 0x2d, 0x8a, 0x39, 0xb9, 0x0f, 0xfa, 0xb5, 0x01, 0xb6, 0xe8, 0x2e, 0xef, + 0xb5, 0xfe, 0x1f, 0x10, 0x13, 0x8f, 0x37, 0xa5, 0xdc, 0xe3, 0xcd, 0xdb, 0x11, 0x7c, 0xad, 0x6d, + 0x90, 0xa7, 0x90, 0xe4, 0x32, 0x7e, 0xfa, 0xea, 0x33, 0x31, 0x52, 0x5f, 0xc6, 0x6f, 0x40, 0x45, + 0xde, 0x85, 0x25, 0x31, 0x42, 0x39, 0x41, 0x19, 0x13, 0xac, 0x25, 0x84, 0xac, 0x54, 0x9d, 0x9c, + 0x86, 0xf2, 0xb2, 0xd2, 0x06, 0xac, 0x25, 0x9c, 0x03, 0x58, 0x11, 0xc4, 0x27, 0x94, 0x3f, 0x1a, + 0x8b, 0x54, 0x5d, 0xc4, 0xd2, 0xec, 0xfc, 0xd6, 0x80, 0xd5, 0xfc, 0xa8, 0x8b, 0xde, 0xe4, 0xde, + 0x03, 0x4b, 0x1c, 0x7f, 0x66, 0xde, 0x26, 0x12, 0xb7, 0x62, 0xc9, 0x76, 0xbe, 0x80, 0x9b, 0xa9, + 0x1d, 0x07, 0x2a, 0x70, 0x45, 0x10, 0x9e, 0x9f, 0x06, 0xce, 0x5f, 0x0c, 0x68, 0xcf, 0xaa, 0x58, + 0x34, 0xdc, 0xd9, 0xb7, 0xc3, 0xc4, 0x01, 0xd6, 0x9b, 0x1d, 0xf0, 0x19, 0xa0, 0x83, 0xd0, 0xf7, + 0xb8, 0x7a, 0x08, 0x2c, 0xba, 0x4c, 0x88, 0x11, 0x72, 0xcb, 0x84, 0x20, 0x88, 0xd2, 0xf9, 0x7b, + 0x03, 0x56, 0x72, 0xe3, 0x17, 0x07, 0xee, 0x80, 0xe5, 0xd3, 0x63, 0xae, 0x0f, 0x66, 0xcd, 0xfc, + 0x0b, 0x26, 0x96, 0x3c, 0xf4, 0x01, 0x94, 0x23, 0xaf, 0xd7, 0xe7, 0x3a, 0xec, 0xd3, 0x42, 0x8a, + 0xe9, 0x7c, 0x09, 0xef, 0xfe, 0x2c, 0x10, 0x67, 0xc6, 0x1d, 0x1a, 0xf3, 0x88, 0x8d, 0xaf, 0x78, + 0x97, 0x43, 0xe1, 0xd6, 0x3c, 0xf5, 0x0b, 0x4e, 0x89, 0x8d, 0xef, 0x01, 0x64, 0x4f, 0xae, 0x08, + 0xa0, 0xf2, 0x53, 0x16, 0x0d, 0x89, 0xdf, 0xba, 0x86, 0xaa, 0x50, 0xda, 0x63, 0xa7, 0x2d, 0x03, + 0xd9, 0x60, 0x7d, 0xe2, 0xf5, 0xfa, 0x2d, 0x73, 0x63, 0x0d, 0x9a, 0xf9, 0x77, 0x56, 0x54, 0x01, + 0xf3, 0x60, 0xb7, 0x75, 0x4d, 0xfc, 0xe2, 0xed, 0x96, 0xb1, 0xb1, 0x09, 0xe6, 0xf3, 0x50, 0x74, + 0xdd, 0x3f, 0xe1, 0x6a, 0x8c, 0x1d, 0xea, 0xab, 0x31, 0x44, 0x15, 0x6a, 0x99, 0xa8, 0x01, 0x76, + 0x72, 0x7d, 0xd9, 0x2a, 0x3d, 0xda, 0xf8, 0xc7, 0xeb, 0x3b, 0xc6, 0x3f, 0x5f, 0xdf, 0x31, 0xfe, + 0xfd, 0xfa, 0x8e, 0xf1, 0xa7, 0xff, 0xdc, 0xb9, 0x06, 0x6d, 0x97, 0x0d, 0x37, 0x43, 0x2f, 0xe8, + 0xb9, 0x24, 0xdc, 0xe4, 0xde, 0x60, 0xb4, 0x39, 0x18, 0xc9, 0x7f, 0x19, 0x1c, 0x55, 0xe4, 0xcf, + 0x87, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0x98, 0xac, 0xa6, 0xa9, 0xb9, 0x20, 0x00, 0x00, } From fba8feccfeff1219c6696c3b0d497d479f1065c5 Mon Sep 17 00:00:00 2001 From: WangXiangUSTC Date: Mon, 12 Nov 2018 00:51:44 +0800 Subject: [PATCH 066/509] update vendor for pump client (#8264) --- Gopkg.lock | 4 ++-- Gopkg.toml | 2 +- .../pingcap/tidb-tools/tidb-binlog/pump_client/client.go | 5 ++--- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index d5adffb288c28..68b4329b332ee 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -298,7 +298,7 @@ revision = "eb892dda1e33a0b76191d39894ad4a806f313f6e" [[projects]] - digest = "1:ef1a3a4694f068c0f914a809fbfa48e0a44d50dc1154d72e85b15be01facda23" + digest = "1:09802a4b5acb491d99be650e2c64115de817ff7b588ffd2cb016f3ca1f9c61aa" name = "github.com/pingcap/tidb-tools" packages = [ "pkg/etcd", @@ -307,7 +307,7 @@ "tidb-binlog/pump_client", ] pruneopts = "NUT" - revision = "5db58e3b7e6613456551c40d011806a346b2f44a" + revision = "8dd71127bcf09130ae032ab6877dd22f32b53e14" [[projects]] branch = "master" diff --git a/Gopkg.toml b/Gopkg.toml index 4ffa13e39d08d..b04b1587e98dd 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -105,4 +105,4 @@ required = ["github.com/golang/protobuf/jsonpb"] [[constraint]] name = "github.com/pingcap/tidb-tools" - revision = "5db58e3b7e6613456551c40d011806a346b2f44a" + revision = "8dd71127bcf09130ae032ab6877dd22f32b53e14" diff --git a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go index 192ef1a5be7b1..da7cf3d9fa106 100644 --- a/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go +++ b/vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client.go @@ -125,8 +125,7 @@ func NewPumpsClient(etcdURLs string, timeout time.Duration, securityOpt pd.Secur return nil, errors.Trace(err) } - rootPath := path.Join(node.DefaultRootPath, node.NodePrefix[node.PumpNode]) - cli, err := etcd.NewClientFromCfg(ectdEndpoints, DefaultEtcdTimeout, rootPath, security) + cli, err := etcd.NewClientFromCfg(ectdEndpoints, DefaultEtcdTimeout, node.DefaultRootPath, security) if err != nil { return nil, errors.Trace(err) } @@ -165,7 +164,7 @@ func NewPumpsClient(etcdURLs string, timeout time.Duration, securityOpt pd.Secur // getPumpStatus retruns all the pumps status in the etcd. func (c *PumpsClient) getPumpStatus(pctx context.Context) error { - nodesStatus, err := c.EtcdRegistry.Nodes(pctx, node.DefaultRootPath) + nodesStatus, err := c.EtcdRegistry.Nodes(pctx, node.NodePrefix[node.PumpNode]) if err != nil { return errors.Trace(err) } From 4beba41b74abacb2dfc10922aa18afe8a8586848 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 12 Nov 2018 10:40:38 +0800 Subject: [PATCH 067/509] *: spilt the internal SQLs in the transaction related metrics (#8255) (#8262) --- metrics/session.go | 8 ++++---- session/session.go | 31 ++++++++++++++++--------------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/metrics/session.go b/metrics/session.go index cb024694a8549..4bfc98f486f37 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -62,14 +62,14 @@ var ( Subsystem: "session", Name: "retry_error_total", Help: "Counter of session retry error.", - }, []string{LblType}) + }, []string{LblSQLType, LblType}) TransactionCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "session", Name: "transaction_total", Help: "Counter of transactions.", - }, []string{LblType}) + }, []string{LblSQLType, LblType}) SessionRestrictedSQLCounter = prometheus.NewCounter( prometheus.CounterOpts{ @@ -86,7 +86,7 @@ var ( Name: "transaction_statement_num", Help: "Buckated histogram of statements count in each transaction.", Buckets: prometheus.ExponentialBuckets(1, 2, 12), - }, []string{LblType}) + }, []string{LblSQLType, LblType}) TransactionDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -95,7 +95,7 @@ var ( Name: "transaction_duration_seconds", Help: "Bucketed histogram of a transaction execution duration, including retry.", Buckets: prometheus.ExponentialBuckets(0.001, 2, 16), // range 1ms ~ 64s - }, []string{LblType}) + }, []string{LblSQLType, LblType}) ) // Label constants. diff --git a/session/session.go b/session/session.go index 2a9a05acb28b9..76812690bb408 100644 --- a/session/session.go +++ b/session/session.go @@ -345,10 +345,11 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { err = s.retry(ctx, uint(maxRetryCount)) } } + label := s.getSQLLabel() counter := s.sessionVars.TxnCtx.StatementCount duration := time.Since(s.GetSessionVars().TxnCtx.CreateTime).Seconds() - metrics.StatementPerTransaction.WithLabelValues(metrics.RetLabel(err)).Observe(float64(counter)) - metrics.TransactionDuration.WithLabelValues(metrics.RetLabel(err)).Observe(float64(duration)) + metrics.StatementPerTransaction.WithLabelValues(label, metrics.RetLabel(err)).Observe(float64(counter)) + metrics.TransactionDuration.WithLabelValues(label, metrics.RetLabel(err)).Observe(float64(duration)) s.cleanRetryInfo() if isoLevelOneShot := &s.sessionVars.TxnIsolationLevelOneShot; isoLevelOneShot.State != 0 { @@ -386,7 +387,7 @@ func (s *session) CommitTxn(ctx context.Context) error { if err != nil { label = metrics.LblError } - metrics.TransactionCounter.WithLabelValues(label).Inc() + metrics.TransactionCounter.WithLabelValues(s.getSQLLabel(), label).Inc() return errors.Trace(err) } @@ -394,7 +395,7 @@ func (s *session) RollbackTxn(ctx context.Context) error { var err error if s.txn.Valid() { terror.Log(s.txn.Rollback()) - metrics.TransactionCounter.WithLabelValues(metrics.LblRollback).Inc() + metrics.TransactionCounter.WithLabelValues(s.getSQLLabel(), metrics.LblRollback).Inc() } s.cleanRetryInfo() s.txn.changeToInvalid() @@ -439,6 +440,13 @@ const sqlLogMaxLen = 1024 // SchemaChangedWithoutRetry is used for testing. var SchemaChangedWithoutRetry bool +func (s *session) getSQLLabel() string { + if s.sessionVars.InRestrictedSQL { + return metrics.LblInternal + } + return metrics.LblGeneral +} + func (s *session) isRetryableError(err error) bool { if SchemaChangedWithoutRetry { return kv.IsRetryableError(err) @@ -508,13 +516,13 @@ func (s *session) retry(ctx context.Context, maxCnt uint) error { } if !s.isRetryableError(err) { log.Warnf("con:%d session:%v, err:%v in retry", connID, s, err) - metrics.SessionRetryErrorCounter.WithLabelValues(metrics.LblUnretryable) + metrics.SessionRetryErrorCounter.WithLabelValues(s.getSQLLabel(), metrics.LblUnretryable) return errors.Trace(err) } retryCnt++ if retryCnt >= maxCnt { log.Warnf("con:%d Retry reached max count %d", connID, retryCnt) - metrics.SessionRetryErrorCounter.WithLabelValues(metrics.LblReachMax) + metrics.SessionRetryErrorCounter.WithLabelValues(s.getSQLLabel(), metrics.LblReachMax) return errors.Trace(err) } log.Warnf("con:%d retryable error: %v, txn: %v", connID, err, s.txn) @@ -748,11 +756,7 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode } return nil, errors.Trace(err) } - label := metrics.LblGeneral - if s.sessionVars.InRestrictedSQL { - label = metrics.LblInternal - } - metrics.SessionExecuteRunDuration.WithLabelValues(label).Observe(time.Since(startTime).Seconds()) + metrics.SessionExecuteRunDuration.WithLabelValues(s.getSQLLabel()).Observe(time.Since(startTime).Seconds()) if recordSet != nil { recordSets = append(recordSets, recordSet) @@ -786,10 +790,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.Rec log.Warnf("con:%d parse error:\n%v\n%s", connID, err, sql) return nil, errors.Trace(err) } - label := metrics.LblGeneral - if s.sessionVars.InRestrictedSQL { - label = metrics.LblInternal - } + label := s.getSQLLabel() metrics.SessionExecuteParseDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds()) compiler := executor.Compiler{Ctx: s} From a3e47ae5e24beb4a98ba0065590d6eab3ca6ae94 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Mon, 12 Nov 2018 11:36:25 +0800 Subject: [PATCH 068/509] tikvclient: Add endKey param to Scanner (#8178) (#8247) --- ddl/db_test.go | 10 +-- ddl/delete_range.go | 2 +- ddl/index.go | 2 +- kv/buffer_store.go | 16 ++-- kv/buffer_store_test.go | 2 +- kv/kv.go | 10 ++- kv/mem_buffer_test.go | 20 ++--- kv/memdb_buffer.go | 16 ++-- kv/mock.go | 12 +-- kv/mock_test.go | 4 +- kv/union_store.go | 8 +- kv/union_store_test.go | 18 ++--- session/txn.go | 16 ++-- store/mockstore/mocktikv/mock_tikv_test.go | 24 +++++- store/mockstore/mocktikv/rpc.go | 6 +- store/store_test.go | 29 ++++---- store/tikv/delete_range_test.go | 2 +- store/tikv/lock_test.go | 4 +- store/tikv/safepoint_test.go | 2 +- store/tikv/scan.go | 7 +- store/tikv/scan_mock_test.go | 10 ++- store/tikv/scan_test.go | 86 +++++++++++----------- store/tikv/snapshot.go | 10 +-- store/tikv/snapshot_test.go | 4 +- store/tikv/store_test.go | 2 +- store/tikv/ticlient_test.go | 4 +- store/tikv/txn.go | 10 +-- structure/hash.go | 2 +- table/tables/index.go | 6 +- table/tables/tables.go | 4 +- util/admin/admin.go | 2 +- util/prefix_helper.go | 4 +- 32 files changed, 192 insertions(+), 162 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 9b9c0d9a80bb9..0044961c6836b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1865,7 +1865,7 @@ func (s *testDBSuite) TestTruncateTable(c *C) { hasOldTableData := true for i := 0; i < waitForCleanDataRound; i++ { err = kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { - it, err1 := txn.Seek(tablePrefix) + it, err1 := txn.Iter(tablePrefix, nil) if err1 != nil { return err1 } @@ -2793,7 +2793,7 @@ func (s *testDBSuite) TestAlterTableDropPartition(c *C) { s.tk.MustExec("drop table if exists tr;") s.tk.MustExec(` create table tr( - id int, name varchar(50), + id int, name varchar(50), purchased date ) partition by range( year(purchased) ) ( @@ -2884,7 +2884,7 @@ func checkPartitionDelRangeDone(c *C, s *testDBSuite, partitionPrefix kv.Key) bo hasOldPartitionData := true for i := 0; i < waitForCleanDataRound; i++ { err := kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { - it, err := txn.Seek(partitionPrefix) + it, err := txn.Iter(partitionPrefix, nil) if err != nil { return err } @@ -2935,7 +2935,7 @@ func (s *testDBSuite) TestTruncatePartitionAndDropTable(c *C) { s.tk.MustExec("drop table if exists t3;") s.tk.MustExec("set @@session.tidb_enable_table_partition=1;") s.tk.MustExec(`create table t3( - id int, name varchar(50), + id int, name varchar(50), purchased date ) partition by range( year(purchased) ) ( @@ -2974,7 +2974,7 @@ func (s *testDBSuite) TestTruncatePartitionAndDropTable(c *C) { s.tk.MustExec("drop table if exists t4;") s.tk.MustExec("set @@session.tidb_enable_table_partition=1;") s.tk.MustExec(`create table t4( - id int, name varchar(50), + id int, name varchar(50), purchased date ) partition by range( year(purchased) ) ( diff --git a/ddl/delete_range.go b/ddl/delete_range.go index aa1bf30640ad3..fba35867f37b0 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -154,7 +154,7 @@ func (dr *delRange) doTask(ctx sessionctx.Context, r util.DelRangeTask) error { finish := true dr.keys = dr.keys[:0] err := kv.RunInNewTxn(dr.store, false, func(txn kv.Transaction) error { - iter, err := txn.Seek(oldStartKey) + iter, err := txn.Iter(oldStartKey, nil) if err != nil { return errors.Trace(err) } diff --git a/ddl/index.go b/ddl/index.go index b546cf256cb96..92dbd2723c5ad 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1192,7 +1192,7 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version return errors.Trace(err) } firstKey := t.RecordKey(seekHandle) - it, err := snap.Seek(firstKey) + it, err := snap.Iter(firstKey, nil) if err != nil { return errors.Trace(err) } diff --git a/kv/buffer_store.go b/kv/buffer_store.go index 5e798db5c8875..2317a0f324437 100644 --- a/kv/buffer_store.go +++ b/kv/buffer_store.go @@ -74,26 +74,26 @@ func (s *BufferStore) Get(k Key) ([]byte, error) { return val, nil } -// Seek implements the Retriever interface. -func (s *BufferStore) Seek(k Key) (Iterator, error) { - bufferIt, err := s.MemBuffer.Seek(k) +// Iter implements the Retriever interface. +func (s *BufferStore) Iter(k Key, upperBound Key) (Iterator, error) { + bufferIt, err := s.MemBuffer.Iter(k, upperBound) if err != nil { return nil, errors.Trace(err) } - retrieverIt, err := s.r.Seek(k) + retrieverIt, err := s.r.Iter(k, upperBound) if err != nil { return nil, errors.Trace(err) } return NewUnionIter(bufferIt, retrieverIt, false) } -// SeekReverse implements the Retriever interface. -func (s *BufferStore) SeekReverse(k Key) (Iterator, error) { - bufferIt, err := s.MemBuffer.SeekReverse(k) +// IterReverse implements the Retriever interface. +func (s *BufferStore) IterReverse(k Key) (Iterator, error) { + bufferIt, err := s.MemBuffer.IterReverse(k) if err != nil { return nil, errors.Trace(err) } - retrieverIt, err := s.r.SeekReverse(k) + retrieverIt, err := s.r.IterReverse(k) if err != nil { return nil, errors.Trace(err) } diff --git a/kv/buffer_store_test.go b/kv/buffer_store_test.go index af84a9294cd24..1af716011c2a4 100644 --- a/kv/buffer_store_test.go +++ b/kv/buffer_store_test.go @@ -53,7 +53,7 @@ func (s testBufferStoreSuite) TestSaveTo(c *C) { err := bs.SaveTo(mutator) c.Check(err, IsNil) - iter, err := mutator.Seek(nil) + iter, err := mutator.Iter(nil, nil) c.Check(err, IsNil) for iter.Valid() { cmp := bytes.Compare(iter.Key(), iter.Value()) diff --git a/kv/kv.go b/kv/kv.go index 1c9230ed65a6a..01ed6a8556eee 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -82,15 +82,17 @@ type Retriever interface { // Get gets the value for key k from kv store. // If corresponding kv pair does not exist, it returns nil and ErrNotExist. Get(k Key) ([]byte, error) - // Seek creates an Iterator positioned on the first entry that k <= entry's key. + // Iter creates an Iterator positioned on the first entry that k <= entry's key. // If such entry is not found, it returns an invalid Iterator with no error. + // It yields only keys that < upperBound. If upperBound is nil, it means the upperBound is unbounded. // The Iterator must be Closed after use. - Seek(k Key) (Iterator, error) + Iter(k Key, upperBound Key) (Iterator, error) - // SeekReverse creates a reversed Iterator positioned on the first entry which key is less than k. + // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. // The returned iterator will iterate from greater key to smaller key. // If k is nil, the returned iterator will be positioned at the last key. - SeekReverse(k Key) (Iterator, error) + // TODO: Add lower bound limit + IterReverse(k Key) (Iterator, error) } // Mutator is the interface wraps the basic Set and Delete methods. diff --git a/kv/mem_buffer_test.go b/kv/mem_buffer_test.go index 92807b528da2d..e7ec5a1f4f9e6 100644 --- a/kv/mem_buffer_test.go +++ b/kv/mem_buffer_test.go @@ -76,7 +76,7 @@ func valToStr(c *C, iter Iterator) string { func checkNewIterator(c *C, buffer MemBuffer) { for i := startIndex; i < testCount; i++ { val := encodeInt(i * indexStep) - iter, err := buffer.Seek(val) + iter, err := buffer.Iter(val, nil) c.Assert(err, IsNil) c.Assert([]byte(iter.Key()), BytesEquals, val) c.Assert(decodeInt([]byte(valToStr(c, iter))), Equals, i*indexStep) @@ -86,7 +86,7 @@ func checkNewIterator(c *C, buffer MemBuffer) { // Test iterator Next() for i := startIndex; i < testCount-1; i++ { val := encodeInt(i * indexStep) - iter, err := buffer.Seek(val) + iter, err := buffer.Iter(val, nil) c.Assert(err, IsNil) c.Assert([]byte(iter.Key()), BytesEquals, val) c.Assert(valToStr(c, iter), Equals, string(val)) @@ -102,7 +102,7 @@ func checkNewIterator(c *C, buffer MemBuffer) { } // Non exist and beyond maximum seek test - iter, err := buffer.Seek(encodeInt(testCount * indexStep)) + iter, err := buffer.Iter(encodeInt(testCount*indexStep), nil) c.Assert(err, IsNil) c.Assert(iter.Valid(), IsFalse) @@ -110,7 +110,7 @@ func checkNewIterator(c *C, buffer MemBuffer) { // it returns the smallest key that larger than the one we are seeking inBetween := encodeInt((testCount-1)*indexStep - 1) last := encodeInt((testCount - 1) * indexStep) - iter, err = buffer.Seek(inBetween) + iter, err = buffer.Iter(inBetween, nil) c.Assert(err, IsNil) c.Assert(iter.Valid(), IsTrue) c.Assert([]byte(iter.Key()), Not(BytesEquals), inBetween) @@ -140,7 +140,7 @@ func (s *testKVSuite) TestNewIterator(c *C) { defer testleak.AfterTest(c)() for _, buffer := range s.bs { // should be invalid - iter, err := buffer.Seek(nil) + iter, err := buffer.Iter(nil, nil) c.Assert(err, IsNil) c.Assert(iter.Valid(), IsFalse) @@ -155,7 +155,7 @@ func (s *testKVSuite) TestIterNextUntil(c *C) { buffer := NewMemDbBuffer(DefaultTxnMembufCap) insertData(c, buffer) - iter, err := buffer.Seek(nil) + iter, err := buffer.Iter(nil, nil) c.Assert(err, IsNil) err = NextUntil(iter, func(k Key) bool { @@ -168,7 +168,7 @@ func (s *testKVSuite) TestIterNextUntil(c *C) { func (s *testKVSuite) TestBasicNewIterator(c *C) { defer testleak.AfterTest(c)() for _, buffer := range s.bs { - it, err := buffer.Seek([]byte("2")) + it, err := buffer.Iter([]byte("2"), nil) c.Assert(err, IsNil) c.Assert(it.Valid(), IsFalse) } @@ -193,7 +193,7 @@ func (s *testKVSuite) TestNewIteratorMin(c *C) { } cnt := 0 - it, err := buffer.Seek(nil) + it, err := buffer.Iter(nil, nil) c.Assert(err, IsNil) for it.Valid() { cnt++ @@ -201,7 +201,7 @@ func (s *testKVSuite) TestNewIteratorMin(c *C) { } c.Assert(cnt, Equals, 6) - it, err = buffer.Seek([]byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000")) + it, err = buffer.Iter([]byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "DATA_test_main_db_tbl_tbl_test_record__00000000000000000001") } @@ -294,7 +294,7 @@ func benchIterator(b *testing.B, buffer MemBuffer) { } b.ResetTimer() for i := 0; i < b.N; i++ { - iter, err := buffer.Seek(nil) + iter, err := buffer.Iter(nil, nil) if err != nil { b.Error(err) } diff --git a/kv/memdb_buffer.go b/kv/memdb_buffer.go index a4aaf7496ce0f..9a57e2cfe16fc 100644 --- a/kv/memdb_buffer.go +++ b/kv/memdb_buffer.go @@ -50,14 +50,10 @@ func NewMemDbBuffer(cap int) MemBuffer { } } -// Seek creates an Iterator. -func (m *memDbBuffer) Seek(k Key) (Iterator, error) { - var i Iterator - if k == nil { - i = &memDbIter{iter: m.db.NewIterator(&util.Range{}), reverse: false} - } else { - i = &memDbIter{iter: m.db.NewIterator(&util.Range{Start: []byte(k)}), reverse: false} - } +// Iter creates an Iterator. +func (m *memDbBuffer) Iter(k Key, upperBound Key) (Iterator, error) { + i := &memDbIter{iter: m.db.NewIterator(&util.Range{Start: []byte(k), Limit: []byte(upperBound)}), reverse: false} + err := i.Next() if err != nil { return nil, errors.Trace(err) @@ -69,7 +65,7 @@ func (m *memDbBuffer) SetCap(cap int) { } -func (m *memDbBuffer) SeekReverse(k Key) (Iterator, error) { +func (m *memDbBuffer) IterReverse(k Key) (Iterator, error) { var i *memDbIter if k == nil { i = &memDbIter{iter: m.db.NewIterator(&util.Range{}), reverse: true} @@ -161,7 +157,7 @@ func (i *memDbIter) Close() { // WalkMemBuffer iterates all buffered kv pairs in memBuf func WalkMemBuffer(memBuf MemBuffer, f func(k Key, v []byte) error) error { - iter, err := memBuf.Seek(nil) + iter, err := memBuf.Iter(nil, nil) if err != nil { return errors.Trace(err) } diff --git a/kv/mock.go b/kv/mock.go index 9fb8a0559c33d..6fc5a1ed52508 100644 --- a/kv/mock.go +++ b/kv/mock.go @@ -68,11 +68,11 @@ func (t *mockTxn) Get(k Key) ([]byte, error) { return nil, nil } -func (t *mockTxn) Seek(k Key) (Iterator, error) { +func (t *mockTxn) Iter(k Key, upperBound Key) (Iterator, error) { return nil, nil } -func (t *mockTxn) SeekReverse(k Key) (Iterator, error) { +func (t *mockTxn) IterReverse(k Key) (Iterator, error) { return nil, nil } @@ -211,10 +211,10 @@ func (s *mockSnapshot) BatchGet(keys []Key) (map[string][]byte, error) { return m, nil } -func (s *mockSnapshot) Seek(k Key) (Iterator, error) { - return s.store.Seek(k) +func (s *mockSnapshot) Iter(k Key, upperBound Key) (Iterator, error) { + return s.store.Iter(k, upperBound) } -func (s *mockSnapshot) SeekReverse(k Key) (Iterator, error) { - return s.store.SeekReverse(k) +func (s *mockSnapshot) IterReverse(k Key) (Iterator, error) { + return s.store.IterReverse(k) } diff --git a/kv/mock_test.go b/kv/mock_test.go index 93dc59490b4a4..2c1db84cf8a97 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -46,8 +46,8 @@ func (s testMockSuite) TestInterface(c *C) { if transaction.IsReadOnly() { transaction.Get(Key("lock")) transaction.Set(Key("lock"), []byte{}) - transaction.Seek(Key("lock")) - transaction.SeekReverse(Key("lock")) + transaction.Iter(Key("lock"), nil) + transaction.IterReverse(Key("lock")) } transaction.Commit(context.Background()) diff --git a/kv/union_store.go b/kv/union_store.go index 6e3111f3ca240..82e3a96223334 100644 --- a/kv/union_store.go +++ b/kv/union_store.go @@ -127,18 +127,18 @@ func (lmb *lazyMemBuffer) Delete(k Key) error { return lmb.mb.Delete(k) } -func (lmb *lazyMemBuffer) Seek(k Key) (Iterator, error) { +func (lmb *lazyMemBuffer) Iter(k Key, upperBound Key) (Iterator, error) { if lmb.mb == nil { return invalidIterator{}, nil } - return lmb.mb.Seek(k) + return lmb.mb.Iter(k, upperBound) } -func (lmb *lazyMemBuffer) SeekReverse(k Key) (Iterator, error) { +func (lmb *lazyMemBuffer) IterReverse(k Key) (Iterator, error) { if lmb.mb == nil { return invalidIterator{}, nil } - return lmb.mb.SeekReverse(k) + return lmb.mb.IterReverse(k) } func (lmb *lazyMemBuffer) Size() int { diff --git a/kv/union_store_test.go b/kv/union_store_test.go index 6241e853582de..2b06b2f39d605 100644 --- a/kv/union_store_test.go +++ b/kv/union_store_test.go @@ -63,46 +63,46 @@ func (s *testUnionStoreSuite) TestSeek(c *C) { s.store.Set([]byte("2"), []byte("2")) s.store.Set([]byte("3"), []byte("3")) - iter, err := s.us.Seek(nil) + iter, err := s.us.Iter(nil, nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("1"), []byte("2"), []byte("3")}, [][]byte{[]byte("1"), []byte("2"), []byte("3")}) - iter, err = s.us.Seek([]byte("2")) + iter, err = s.us.Iter([]byte("2"), nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3")}, [][]byte{[]byte("2"), []byte("3")}) s.us.Set([]byte("4"), []byte("4")) - iter, err = s.us.Seek([]byte("2")) + iter, err = s.us.Iter([]byte("2"), nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3"), []byte("4")}, [][]byte{[]byte("2"), []byte("3"), []byte("4")}) s.us.Delete([]byte("3")) - iter, err = s.us.Seek([]byte("2")) + iter, err = s.us.Iter([]byte("2"), nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("4")}, [][]byte{[]byte("2"), []byte("4")}) } -func (s *testUnionStoreSuite) TestSeekReverse(c *C) { +func (s *testUnionStoreSuite) TestIterReverse(c *C) { defer testleak.AfterTest(c)() s.store.Set([]byte("1"), []byte("1")) s.store.Set([]byte("2"), []byte("2")) s.store.Set([]byte("3"), []byte("3")) - iter, err := s.us.SeekReverse(nil) + iter, err := s.us.IterReverse(nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("3"), []byte("2"), []byte("1")}, [][]byte{[]byte("3"), []byte("2"), []byte("1")}) - iter, err = s.us.SeekReverse([]byte("3")) + iter, err = s.us.IterReverse([]byte("3")) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1")}, [][]byte{[]byte("2"), []byte("1")}) s.us.Set([]byte("0"), []byte("0")) - iter, err = s.us.SeekReverse([]byte("3")) + iter, err = s.us.IterReverse([]byte("3")) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1"), []byte("0")}, [][]byte{[]byte("2"), []byte("1"), []byte("0")}) s.us.Delete([]byte("1")) - iter, err = s.us.SeekReverse([]byte("3")) + iter, err = s.us.IterReverse([]byte("3")) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("0")}, [][]byte{[]byte("2"), []byte("0")}) } diff --git a/session/txn.go b/session/txn.go index 5b37b63ad29f3..6e391335fc966 100644 --- a/session/txn.go +++ b/session/txn.go @@ -159,26 +159,26 @@ func (st *TxnState) Delete(k kv.Key) error { return st.buf.Delete(k) } -// Seek overrides the Transaction interface. -func (st *TxnState) Seek(k kv.Key) (kv.Iterator, error) { - bufferIt, err := st.buf.Seek(k) +// Iter overrides the Transaction interface. +func (st *TxnState) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { + bufferIt, err := st.buf.Iter(k, upperBound) if err != nil { return nil, errors.Trace(err) } - retrieverIt, err := st.Transaction.Seek(k) + retrieverIt, err := st.Transaction.Iter(k, upperBound) if err != nil { return nil, errors.Trace(err) } return kv.NewUnionIter(bufferIt, retrieverIt, false) } -// SeekReverse overrides the Transaction interface. -func (st *TxnState) SeekReverse(k kv.Key) (kv.Iterator, error) { - bufferIt, err := st.buf.SeekReverse(k) +// IterReverse overrides the Transaction interface. +func (st *TxnState) IterReverse(k kv.Key) (kv.Iterator, error) { + bufferIt, err := st.buf.IterReverse(k) if err != nil { return nil, errors.Trace(err) } - retrieverIt, err := st.Transaction.SeekReverse(k) + retrieverIt, err := st.Transaction.IterReverse(k) if err != nil { return nil, errors.Trace(err) } diff --git a/store/mockstore/mocktikv/mock_tikv_test.go b/store/mockstore/mocktikv/mock_tikv_test.go index 807a4267134ba..a4af86f037a28 100644 --- a/store/mockstore/mocktikv/mock_tikv_test.go +++ b/store/mockstore/mocktikv/mock_tikv_test.go @@ -119,7 +119,11 @@ func (s *testMockTiKVSuite) mustDeleteOK(c *C, key string, startTS, commitTS uin } func (s *testMockTiKVSuite) mustScanOK(c *C, start string, limit int, ts uint64, expect ...string) { - pairs := s.store.Scan([]byte(start), nil, limit, ts, kvrpcpb.IsolationLevel_SI) + s.mustRangeScanOK(c, start, "", limit, ts, expect...) +} + +func (s *testMockTiKVSuite) mustRangeScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) { + pairs := s.store.Scan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI) c.Assert(len(pairs)*2, Equals, len(expect)) for i := 0; i < len(pairs); i++ { c.Assert(pairs[i].Err, IsNil) @@ -129,7 +133,11 @@ func (s *testMockTiKVSuite) mustScanOK(c *C, start string, limit int, ts uint64, } func (s *testMockTiKVSuite) mustReverseScanOK(c *C, end string, limit int, ts uint64, expect ...string) { - pairs := s.store.ReverseScan(nil, []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI) + s.mustRangeReverseScanOK(c, "", end, limit, ts, expect...) +} + +func (s *testMockTiKVSuite) mustRangeReverseScanOK(c *C, start, end string, limit int, ts uint64, expect ...string) { + pairs := s.store.ReverseScan([]byte(start), []byte(end), limit, ts, kvrpcpb.IsolationLevel_SI) c.Assert(len(pairs)*2, Equals, len(expect)) for i := 0; i < len(pairs); i++ { c.Assert(pairs[i].Err, IsNil) @@ -249,6 +257,9 @@ func (s *testMockTiKVSuite) TestReverseScan(c *C) { s.mustReverseScanOK(c, "C\x00", 3, 10, "C", "C10", "A", "A10") s.mustReverseScanOK(c, "C\x00", 4, 10, "C", "C10", "A", "A10") s.mustReverseScanOK(c, "B", 1, 10, "A", "A10") + s.mustRangeReverseScanOK(c, "", "E", 5, 10, "C", "C10", "A", "A10") + s.mustRangeReverseScanOK(c, "", "C\x00", 5, 10, "C", "C10", "A", "A10") + s.mustRangeReverseScanOK(c, "A\x00", "C", 5, 10) } checkV10() @@ -260,6 +271,9 @@ func (s *testMockTiKVSuite) TestReverseScan(c *C) { s.mustReverseScanOK(c, "Z", 5, 20, "E", "E10", "D", "D20", "C", "C10", "B", "B20", "A", "A10") s.mustReverseScanOK(c, "C\x00", 5, 20, "C", "C10", "B", "B20", "A", "A10") s.mustReverseScanOK(c, "A\x00", 1, 20, "A", "A10") + s.mustRangeReverseScanOK(c, "B", "D", 5, 20, "C", "C10", "B", "B20") + s.mustRangeReverseScanOK(c, "B", "D\x00", 5, 20, "D", "D20", "C", "C10", "B", "B20") + s.mustRangeReverseScanOK(c, "B\x00", "D\x00", 5, 20, "D", "D20", "C", "C10") } checkV10() checkV20() @@ -308,6 +322,9 @@ func (s *testMockTiKVSuite) TestScan(c *C) { s.mustScanOK(c, "A\x00", 3, 10, "C", "C10", "E", "E10") s.mustScanOK(c, "C", 4, 10, "C", "C10", "E", "E10") s.mustScanOK(c, "F", 1, 10) + s.mustRangeScanOK(c, "", "E", 5, 10, "A", "A10", "C", "C10") + s.mustRangeScanOK(c, "", "C\x00", 5, 10, "A", "A10", "C", "C10") + s.mustRangeScanOK(c, "A\x00", "C", 5, 10) } checkV10() @@ -319,6 +336,9 @@ func (s *testMockTiKVSuite) TestScan(c *C) { s.mustScanOK(c, "", 5, 20, "A", "A10", "B", "B20", "C", "C10", "D", "D20", "E", "E10") s.mustScanOK(c, "C", 5, 20, "C", "C10", "D", "D20", "E", "E10") s.mustScanOK(c, "D\x00", 1, 20, "E", "E10") + s.mustRangeScanOK(c, "B", "D", 5, 20, "B", "B20", "C", "C10") + s.mustRangeScanOK(c, "B", "D\x00", 5, 20, "B", "B20", "C", "C10", "D", "D20") + s.mustRangeScanOK(c, "B\x00", "D\x00", 5, 20, "C", "C10", "D", "D20") } checkV10() checkV20() diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 93a24331d9d9d..8a51bce389ccb 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -229,7 +229,11 @@ func (h *rpcHandler) handleKvScan(req *kvrpcpb.ScanRequest) *kvrpcpb.ScanRespons if !h.checkKeyInRegion(req.GetStartKey()) { panic("KvScan: startKey not in region") } - pairs := h.mvccStore.Scan(req.GetStartKey(), h.endKey, int(req.GetLimit()), req.GetVersion(), h.isolationLevel) + endKey := h.endKey + if len(req.EndKey) > 0 && (len(endKey) == 0 || bytes.Compare(req.EndKey, endKey) < 0) { + endKey = req.EndKey + } + pairs := h.mvccStore.Scan(req.GetStartKey(), endKey, int(req.GetLimit()), req.GetVersion(), h.isolationLevel) return &kvrpcpb.ScanResponse{ Pairs: convertToPbPairs(pairs), } diff --git a/store/store_test.go b/store/store_test.go index b69f92dbb7014..95ef4c131b4e9 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -97,7 +97,7 @@ func valToStr(c *C, iter kv.Iterator) string { func checkSeek(c *C, txn kv.Transaction) { for i := startIndex; i < testCount; i++ { val := encodeInt(i * indexStep) - iter, err := txn.Seek(val) + iter, err := txn.Iter(val, nil) c.Assert(err, IsNil) c.Assert([]byte(iter.Key()), BytesEquals, val) c.Assert(decodeInt([]byte(valToStr(c, iter))), Equals, i*indexStep) @@ -107,7 +107,7 @@ func checkSeek(c *C, txn kv.Transaction) { // Test iterator Next() for i := startIndex; i < testCount-1; i++ { val := encodeInt(i * indexStep) - iter, err := txn.Seek(val) + iter, err := txn.Iter(val, nil) c.Assert(err, IsNil) c.Assert([]byte(iter.Key()), BytesEquals, val) c.Assert(valToStr(c, iter), Equals, string(val)) @@ -123,7 +123,7 @@ func checkSeek(c *C, txn kv.Transaction) { } // Non exist and beyond maximum seek test - iter, err := txn.Seek(encodeInt(testCount * indexStep)) + iter, err := txn.Iter(encodeInt(testCount*indexStep), nil) c.Assert(err, IsNil) c.Assert(iter.Valid(), IsFalse) @@ -131,7 +131,7 @@ func checkSeek(c *C, txn kv.Transaction) { // it returns the smallest key that larger than the one we are seeking inBetween := encodeInt((testCount-1)*indexStep - 1) last := encodeInt((testCount - 1) * indexStep) - iter, err = txn.Seek(inBetween) + iter, err = txn.Iter(inBetween, nil) c.Assert(err, IsNil) c.Assert(iter.Valid(), IsTrue) c.Assert([]byte(iter.Key()), Not(BytesEquals), inBetween) @@ -278,7 +278,7 @@ func (s *testKVSuite) TestDelete2(c *C) { txn, err = s.s.Begin() c.Assert(err, IsNil) - it, err := txn.Seek([]byte("DATA_test_tbl_department_record__0000000001_0003")) + it, err := txn.Iter([]byte("DATA_test_tbl_department_record__0000000001_0003"), nil) c.Assert(err, IsNil) for it.Valid() { err = txn.Delete([]byte(it.Key())) @@ -290,7 +290,7 @@ func (s *testKVSuite) TestDelete2(c *C) { txn, err = s.s.Begin() c.Assert(err, IsNil) - it, _ = txn.Seek([]byte("DATA_test_tbl_department_record__000000000")) + it, _ = txn.Iter([]byte("DATA_test_tbl_department_record__000000000"), nil) c.Assert(it.Valid(), IsFalse) txn.Commit(context.Background()) } @@ -312,7 +312,7 @@ func (s *testKVSuite) TestBasicSeek(c *C) { c.Assert(err, IsNil) defer txn.Commit(context.Background()) - it, err := txn.Seek([]byte("2")) + it, err := txn.Iter([]byte("2"), nil) c.Assert(err, IsNil) c.Assert(it.Valid(), Equals, false) txn.Delete([]byte("1")) @@ -333,30 +333,30 @@ func (s *testKVSuite) TestBasicTable(c *C) { err = txn.Set([]byte("1"), []byte("1")) c.Assert(err, IsNil) - it, err := txn.Seek([]byte("0")) + it, err := txn.Iter([]byte("0"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "1") err = txn.Set([]byte("0"), []byte("0")) c.Assert(err, IsNil) - it, err = txn.Seek([]byte("0")) + it, err = txn.Iter([]byte("0"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "0") err = txn.Delete([]byte("0")) c.Assert(err, IsNil) txn.Delete([]byte("1")) - it, err = txn.Seek([]byte("0")) + it, err = txn.Iter([]byte("0"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "2") err = txn.Delete([]byte("3")) c.Assert(err, IsNil) - it, err = txn.Seek([]byte("2")) + it, err = txn.Iter([]byte("2"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "2") - it, err = txn.Seek([]byte("3")) + it, err = txn.Iter([]byte("3"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "4") err = txn.Delete([]byte("2")) @@ -411,13 +411,14 @@ func (s *testKVSuite) TestSeekMin(c *C) { txn.Set([]byte(row.key), []byte(row.value)) } - it, err := txn.Seek(nil) + it, err := txn.Iter(nil, nil) + c.Assert(err, IsNil) for it.Valid() { fmt.Printf("%s, %s\n", it.Key(), it.Value()) it.Next() } - it, err = txn.Seek([]byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000")) + it, err = txn.Iter([]byte("DATA_test_main_db_tbl_tbl_test_record__00000000000000000000"), nil) c.Assert(err, IsNil) c.Assert(string(it.Key()), Equals, "DATA_test_main_db_tbl_tbl_test_record__00000000000000000001") diff --git a/store/tikv/delete_range_test.go b/store/tikv/delete_range_test.go index 7864eb0954878..ee0b3c6c73126 100644 --- a/store/tikv/delete_range_test.go +++ b/store/tikv/delete_range_test.go @@ -50,7 +50,7 @@ func (s *testDeleteRangeSuite) TearDownTest(c *C) { func (s *testDeleteRangeSuite) checkData(c *C, expectedData map[string]string) { txn, err := s.store.Begin() c.Assert(err, IsNil) - it, err := txn.Seek([]byte("a")) + it, err := txn.Iter([]byte("a"), nil) c.Assert(err, IsNil) // Scan all data and save into a map diff --git a/store/tikv/lock_test.go b/store/tikv/lock_test.go index 20bedba474f10..31e0c4597b121 100644 --- a/store/tikv/lock_test.go +++ b/store/tikv/lock_test.go @@ -116,7 +116,7 @@ func (s *testLockSuite) TestScanLockResolveWithSeek(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - iter, err := txn.Seek([]byte("a")) + iter, err := txn.Iter([]byte("a"), nil) c.Assert(err, IsNil) for ch := byte('a'); ch <= byte('z'); ch++ { c.Assert(iter.Valid(), IsTrue) @@ -133,7 +133,7 @@ func (s *testLockSuite) TestScanLockResolveWithSeekKeyOnly(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) txn.SetOption(kv.KeyOnly, true) - iter, err := txn.Seek([]byte("a")) + iter, err := txn.Iter([]byte("a"), nil) c.Assert(err, IsNil) for ch := byte('a'); ch <= byte('z'); ch++ { c.Assert(iter.Valid(), IsTrue) diff --git a/store/tikv/safepoint_test.go b/store/tikv/safepoint_test.go index 29b640a1d48fd..76e3449d9a95f 100644 --- a/store/tikv/safepoint_test.go +++ b/store/tikv/safepoint_test.go @@ -100,7 +100,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { s.waitUntilErrorPlugIn(txn3.startTS) - _, seekerr := txn3.Seek(encodeKey(s.prefix, "")) + _, seekerr := txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(seekerr, NotNil) isFallBehind = terror.ErrorEqual(errors.Cause(geterr2), ErrGCTooEarly) isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) diff --git a/store/tikv/scan.go b/store/tikv/scan.go index d5c140f6a7e3f..c3d5c555ba37b 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -30,10 +30,11 @@ type Scanner struct { cache []*pb.KvPair idx int nextStartKey []byte + endKey []byte eof bool } -func newScanner(snapshot *tikvSnapshot, startKey []byte, batchSize int) (*Scanner, error) { +func newScanner(snapshot *tikvSnapshot, startKey []byte, endKey []byte, batchSize int) (*Scanner, error) { // It must be > 1. Otherwise scanner won't skipFirst. if batchSize <= 1 { batchSize = scanBatchSize @@ -43,6 +44,7 @@ func newScanner(snapshot *tikvSnapshot, startKey []byte, batchSize int) (*Scanne batchSize: batchSize, valid: true, nextStartKey: startKey, + endKey: endKey, } err := scanner.Next() if kv.IsErrNotFound(err) { @@ -147,6 +149,7 @@ func (s *Scanner) getData(bo *Backoffer) error { Type: tikvrpc.CmdScan, Scan: &pb.ScanRequest{ StartKey: s.nextStartKey, + EndKey: s.endKey, Limit: uint32(s.batchSize), Version: s.startTS(), KeyOnly: s.snapshot.keyOnly, @@ -199,7 +202,7 @@ func (s *Scanner) getData(bo *Backoffer) error { // No more data in current Region. Next getData() starts // from current Region's endKey. s.nextStartKey = loc.EndKey - if len(loc.EndKey) == 0 { + if len(loc.EndKey) == 0 || (len(s.endKey) > 0 && kv.Key(s.nextStartKey).Cmp(kv.Key(s.endKey)) >= 0) { // Current Region is the last one. s.eof = true } diff --git a/store/tikv/scan_mock_test.go b/store/tikv/scan_mock_test.go index af453a985ff32..94b720314f2bd 100644 --- a/store/tikv/scan_mock_test.go +++ b/store/tikv/scan_mock_test.go @@ -41,11 +41,19 @@ func (s *testScanMockSuite) TestScanMultipleRegions(c *C) { txn, err = store.Begin() c.Assert(err, IsNil) snapshot := newTiKVSnapshot(store, kv.Version{Ver: txn.StartTS()}) - scanner, err := newScanner(snapshot, []byte("a"), 10) + scanner, err := newScanner(snapshot, []byte("a"), nil, 10) c.Assert(err, IsNil) for ch := byte('a'); ch <= byte('z'); ch++ { c.Assert([]byte{ch}, BytesEquals, []byte(scanner.Key())) c.Assert(scanner.Next(), IsNil) } c.Assert(scanner.Valid(), IsFalse) + + scanner, err = newScanner(snapshot, []byte("a"), []byte("i"), 10) + c.Assert(err, IsNil) + for ch := byte('a'); ch <= byte('h'); ch++ { + c.Assert([]byte{ch}, BytesEquals, []byte(scanner.Key())) + c.Assert(scanner.Next(), IsNil) + } + c.Assert(scanner.Valid(), IsFalse) } diff --git a/store/tikv/scan_test.go b/store/tikv/scan_test.go index e4897bc2de2b7..ff9184d0c0323 100644 --- a/store/tikv/scan_test.go +++ b/store/tikv/scan_test.go @@ -40,7 +40,7 @@ func (s *testScanSuite) SetUpSuite(c *C) { func (s *testScanSuite) TearDownSuite(c *C) { txn := s.beginTxn(c) - scanner, err := txn.Seek(encodeKey(s.prefix, "")) + scanner, err := txn.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) c.Assert(scanner, NotNil) for scanner.Valid() { @@ -62,7 +62,25 @@ func (s *testScanSuite) beginTxn(c *C) *tikvTxn { return txn.(*tikvTxn) } -func (s *testScanSuite) TestSeek(c *C) { +func (s *testScanSuite) TestScan(c *C) { + check := func(c *C, scan kv.Iterator, rowNum int, keyOnly bool) { + for i := 0; i < rowNum; i++ { + k := scan.Key() + c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) + if !keyOnly { + v := scan.Value() + c.Assert(v, BytesEquals, valueBytes(i)) + } + // Because newScan return first item without calling scan.Next() just like go-hbase, + // for-loop count will decrease 1. + if i < rowNum-1 { + scan.Next() + } + } + scan.Next() + c.Assert(scan.Valid(), IsFalse) + } + for _, rowNum := range s.rowNums { txn := s.beginTxn(c) for i := 0; i < rowNum; i++ { @@ -76,57 +94,35 @@ func (s *testScanSuite) TestSeek(c *C) { val, err := txn2.Get(encodeKey(s.prefix, s08d("key", 0))) c.Assert(err, IsNil) c.Assert(val, BytesEquals, valueBytes(0)) - scan, err := txn2.Seek(encodeKey(s.prefix, "")) + // Test scan without upperBound + scan, err := txn2.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) - - for i := 0; i < rowNum; i++ { - k := scan.Key() - c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) - v := scan.Value() - c.Assert(v, BytesEquals, valueBytes(i)) - // Because newScan return first item without calling scan.Next() just like go-hbase, - // for-loop count will decrease 1. - if i < rowNum-1 { - scan.Next() - } - } - scan.Next() - c.Assert(scan.Valid(), IsFalse) + check(c, scan, rowNum, false) + // Test scan with upperBound + upperBound := rowNum / 2 + scan, err = txn2.Iter(encodeKey(s.prefix, ""), encodeKey(s.prefix, s08d("key", upperBound))) + c.Assert(err, IsNil) + check(c, scan, upperBound, false) txn3 := s.beginTxn(c) txn3.SetOption(kv.KeyOnly, true) - scan, err = txn3.Seek(encodeKey(s.prefix, "")) + // Test scan without upper bound + scan, err = txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) - - for i := 0; i < rowNum; i++ { - k := scan.Key() - c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) - // Because newScan return first item without calling scan.Next() just like go-hbase, - // for-loop count will decrease 1. - if i < rowNum-1 { - scan.Next() - } - } - scan.Next() - c.Assert(scan.Valid(), IsFalse) + check(c, scan, rowNum, true) + // test scan with upper bound + scan, err = txn3.Iter(encodeKey(s.prefix, ""), encodeKey(s.prefix, s08d("key", upperBound))) + c.Assert(err, IsNil) + check(c, scan, upperBound, true) // Restore KeyOnly to false txn3.SetOption(kv.KeyOnly, false) - scan, err = txn3.Seek(encodeKey(s.prefix, "")) + scan, err = txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) - - for i := 0; i < rowNum; i++ { - k := scan.Key() - c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) - v := scan.Value() - c.Assert(v, BytesEquals, valueBytes(i)) - // Because newScan return first item without calling scan.Next() just like go-hbase, - // for-loop count will decrease 1. - if i < rowNum-1 { - scan.Next() - } - } - scan.Next() - c.Assert(scan.Valid(), IsFalse) + check(c, scan, rowNum, true) + // test scan with upper bound + scan, err = txn3.Iter(encodeKey(s.prefix, ""), encodeKey(s.prefix, s08d("key", upperBound))) + c.Assert(err, IsNil) + check(c, scan, upperBound, true) } } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index c4c14e984e57d..51eaa4a89b216 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -280,14 +280,14 @@ func (s *tikvSnapshot) get(bo *Backoffer, k kv.Key) ([]byte, error) { } } -// Seek return a list of key-value pair after `k`. -func (s *tikvSnapshot) Seek(k kv.Key) (kv.Iterator, error) { - scanner, err := newScanner(s, k, scanBatchSize) +// Iter return a list of key-value pair after `k`. +func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { + scanner, err := newScanner(s, k, upperBound, scanBatchSize) return scanner, errors.Trace(err) } -// SeekReverse creates a reversed Iterator positioned on the first entry which key is less than k. -func (s *tikvSnapshot) SeekReverse(k kv.Key) (kv.Iterator, error) { +// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. +func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { return nil, kv.ErrNotImplemented } diff --git a/store/tikv/snapshot_test.go b/store/tikv/snapshot_test.go index 4ea695f224e4b..61d5da2112863 100644 --- a/store/tikv/snapshot_test.go +++ b/store/tikv/snapshot_test.go @@ -42,7 +42,7 @@ func (s *testSnapshotSuite) SetUpSuite(c *C) { func (s *testSnapshotSuite) TearDownSuite(c *C) { txn := s.beginTxn(c) - scanner, err := txn.Seek(encodeKey(s.prefix, "")) + scanner, err := txn.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) c.Assert(scanner, NotNil) for scanner.Valid() { @@ -70,7 +70,7 @@ func (s *testSnapshotSuite) checkAll(keys []kv.Key, c *C) { m, err := snapshot.BatchGet(keys) c.Assert(err, IsNil) - scan, err := txn.Seek(encodeKey(s.prefix, "")) + scan, err := txn.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) cnt := 0 for scan.Valid() { diff --git a/store/tikv/store_test.go b/store/tikv/store_test.go index 00735eee64a3f..116f4b875748e 100644 --- a/store/tikv/store_test.go +++ b/store/tikv/store_test.go @@ -229,7 +229,7 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { // Cover Seek request. client.priority = pb.CommandPri_High txn.SetOption(kv.Priority, kv.PriorityHigh) - iter, err := txn.Seek([]byte("key")) + iter, err := txn.Iter([]byte("key"), nil) c.Assert(err, IsNil) for iter.Valid() { c.Assert(iter.Next(), IsNil) diff --git a/store/tikv/ticlient_test.go b/store/tikv/ticlient_test.go index 1b4ea24ad5c82..e67b9c7562c57 100644 --- a/store/tikv/ticlient_test.go +++ b/store/tikv/ticlient_test.go @@ -61,7 +61,7 @@ func clearStorage(store kv.Storage) error { if err != nil { return errors.Trace(err) } - iter, err := txn.Seek(nil) + iter, err := txn.Iter(nil, nil) if err != nil { return errors.Trace(err) } @@ -93,7 +93,7 @@ func (s *testTiclientSuite) SetUpSuite(c *C) { func (s *testTiclientSuite) TearDownSuite(c *C) { // Clean all data, or it may pollute other data. txn := s.beginTxn(c) - scanner, err := txn.Seek(encodeKey(s.prefix, "")) + scanner, err := txn.Iter(encodeKey(s.prefix, ""), nil) c.Assert(err, IsNil) c.Assert(scanner, NotNil) for scanner.Valid() { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 18b7e1a17d31f..1c4464bc49488 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -115,23 +115,23 @@ func (txn *tikvTxn) String() string { return fmt.Sprintf("%d", txn.StartTS()) } -func (txn *tikvTxn) Seek(k kv.Key) (kv.Iterator, error) { +func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { metrics.TiKVTxnCmdCounter.WithLabelValues("seek").Inc() start := time.Now() defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("seek").Observe(time.Since(start).Seconds()) }() - return txn.us.Seek(k) + return txn.us.Iter(k, upperBound) } -// SeekReverse creates a reversed Iterator positioned on the first entry which key is less than k. -func (txn *tikvTxn) SeekReverse(k kv.Key) (kv.Iterator, error) { +// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. +func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { metrics.TiKVTxnCmdCounter.WithLabelValues("seek_reverse").Inc() start := time.Now() defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("seek_reverse").Observe(time.Since(start).Seconds()) }() - return txn.us.SeekReverse(k) + return txn.us.IterReverse(k) } func (txn *tikvTxn) Delete(k kv.Key) error { diff --git a/structure/hash.go b/structure/hash.go index 8b04016c37bc1..bfa13118cce8c 100644 --- a/structure/hash.go +++ b/structure/hash.go @@ -238,7 +238,7 @@ func (t *TxStructure) HClear(key []byte) error { func (t *TxStructure) iterateHash(key []byte, fn func(k []byte, v []byte) error) error { dataPrefix := t.hashDataKeyPrefix(key) - it, err := t.reader.Seek(dataPrefix) + it, err := t.reader.Iter(dataPrefix, nil) if err != nil { return errors.Trace(err) } diff --git a/table/tables/index.go b/table/tables/index.go index e11993001bfac..c36d55c698cb1 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -240,7 +240,7 @@ func (c *index) Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues // Drop removes the KV index from store. func (c *index) Drop(rm kv.RetrieverMutator) error { - it, err := rm.Seek(c.prefix) + it, err := rm.Iter(c.prefix, nil) if err != nil { return errors.Trace(err) } @@ -270,7 +270,7 @@ func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues return nil, false, errors.Trace(err) } - it, err := r.Seek(key) + it, err := r.Iter(key, nil) if err != nil { return nil, false, errors.Trace(err) } @@ -284,7 +284,7 @@ func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues // SeekFirst returns an iterator which points to the first entry of the KV index. func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) { - it, err := r.Seek(c.prefix) + it, err := r.Iter(c.prefix, nil) if err != nil { return nil, errors.Trace(err) } diff --git a/table/tables/tables.go b/table/tables/tables.go index 8660f8625e27e..4a75d3f7e84d3 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -766,7 +766,7 @@ func (t *tableCommon) buildIndexForRow(ctx sessionctx.Context, rm kv.RetrieverMu // IterRecords implements table.Table IterRecords interface. func (t *tableCommon) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { - it, err := ctx.Txn().Seek(startKey) + it, err := ctx.Txn().Iter(startKey, nil) if err != nil { return errors.Trace(err) } @@ -896,7 +896,7 @@ func (t *tableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetS // Seek implements table.Table Seek interface. func (t *tableCommon) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { seekKey := tablecodec.EncodeRowKeyWithHandle(t.physicalTableID, h) - iter, err := ctx.Txn().Seek(seekKey) + iter, err := ctx.Txn().Iter(seekKey, nil) if !iter.Valid() || !iter.Key().HasPrefix(t.RecordPrefix()) { // No more records in the table, skip to the end. return 0, false, nil diff --git a/util/admin/admin.go b/util/admin/admin.go index 325d09df98fc4..a76d5f416e8d1 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -636,7 +636,7 @@ func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h // genExprs use to calculate generated column value. func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Table, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc, genExprs map[string]expression.Expression) error { - it, err := retriever.Seek(startKey) + it, err := retriever.Iter(startKey, nil) if err != nil { return errors.Trace(err) } diff --git a/util/prefix_helper.go b/util/prefix_helper.go index ccfd1ece9d633..a2c5d76dfd168 100644 --- a/util/prefix_helper.go +++ b/util/prefix_helper.go @@ -26,7 +26,7 @@ import ( // ScanMetaWithPrefix scans metadata with the prefix. func ScanMetaWithPrefix(retriever kv.Retriever, prefix kv.Key, filter func(kv.Key, []byte) bool) error { - iter, err := retriever.Seek(prefix) + iter, err := retriever.Iter(prefix, nil) if err != nil { return errors.Trace(err) } @@ -56,7 +56,7 @@ func ScanMetaWithPrefix(retriever kv.Retriever, prefix kv.Key, filter func(kv.Ke // DelKeyWithPrefix deletes keys with prefix. func DelKeyWithPrefix(rm kv.RetrieverMutator, prefix kv.Key) error { var keys []kv.Key - iter, err := rm.Seek(prefix) + iter, err := rm.Iter(prefix, nil) if err != nil { return errors.Trace(err) } From 21330da7a20177d490ae75e1689cfd988b7645f2 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Mon, 12 Nov 2018 15:23:59 +0800 Subject: [PATCH 069/509] *: Make use of the upperBound of ticlient's kv_scan interface to ensure no overbound scan will happen (#8081) (#8257) --- ddl/delete_range.go | 8 ++------ ddl/index.go | 22 ++++++++++++++++++---- ddl/reorg.go | 2 +- store/tikv/scan.go | 5 +++++ structure/hash.go | 2 +- table/tables/index.go | 8 +++++--- table/tables/tables.go | 6 +++--- util/admin/admin.go | 6 ++++-- util/prefix_helper.go | 4 ++-- 9 files changed, 41 insertions(+), 22 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index fba35867f37b0..4b12de2d2f419 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -14,7 +14,6 @@ package ddl import ( - "bytes" "encoding/hex" "fmt" "math" @@ -154,7 +153,7 @@ func (dr *delRange) doTask(ctx sessionctx.Context, r util.DelRangeTask) error { finish := true dr.keys = dr.keys[:0] err := kv.RunInNewTxn(dr.store, false, func(txn kv.Transaction) error { - iter, err := txn.Iter(oldStartKey, nil) + iter, err := txn.Iter(oldStartKey, r.EndKey) if err != nil { return errors.Trace(err) } @@ -164,10 +163,7 @@ func (dr *delRange) doTask(ctx sessionctx.Context, r util.DelRangeTask) error { if !iter.Valid() { break } - finish = bytes.Compare(iter.Key(), r.EndKey) >= 0 - if finish { - break - } + finish = false dr.keys = append(dr.keys, iter.Key().Clone()) newStartKey = iter.Key().Next() diff --git a/ddl/index.go b/ddl/index.go index 92dbd2723c5ad..5eedd76f3dbc0 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -619,7 +619,7 @@ func (w *addIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgInde // taskDone means that the added handle is out of taskRange.endHandle. taskDone := false oprStartTime := startTime - err := iterateSnapshotRows(w.sessCtx.GetStore(), w.priority, w.table, txn.StartTS(), taskRange.startHandle, + err := iterateSnapshotRows(w.sessCtx.GetStore(), w.priority, w.table, txn.StartTS(), taskRange.startHandle, taskRange.endHandle, taskRange.endIncluded, func(handle int64, recordKey kv.Key, rawRow []byte) (bool, error) { oprEndTime := time.Now() w.logSlowOperations(oprEndTime.Sub(oprStartTime), "iterateSnapshotRows in fetchRowColVals", 0) @@ -1183,7 +1183,7 @@ func allocateIndexID(tblInfo *model.TableInfo) int64 { // recordIterFunc is used for low-level record iteration. type recordIterFunc func(h int64, rowKey kv.Key, rawRecord []byte) (more bool, err error) -func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version uint64, seekHandle int64, fn recordIterFunc) error { +func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version uint64, startHandle int64, endHandle int64, endIncluded bool, fn recordIterFunc) error { ver := kv.Version{Ver: version} snap, err := store.GetSnapshot(ver) @@ -1191,8 +1191,22 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version if err != nil { return errors.Trace(err) } - firstKey := t.RecordKey(seekHandle) - it, err := snap.Iter(firstKey, nil) + firstKey := t.RecordKey(startHandle) + + // Calculate the exclusive upper bound + var upperBound kv.Key + if endIncluded { + if endHandle == math.MaxInt64 { + upperBound = t.RecordKey(endHandle).PrefixNext() + } else { + // PrefixNext is time costing. Try to avoid it if possible. + upperBound = t.RecordKey(endHandle + 1) + } + } else { + upperBound = t.RecordKey(endHandle) + } + + it, err := snap.Iter(firstKey, upperBound) if err != nil { return errors.Trace(err) } diff --git a/ddl/reorg.go b/ddl/reorg.go index 10cf2d85b9064..035146c9976b1 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -304,7 +304,7 @@ func getTableRange(d *ddlCtx, tbl table.PhysicalTable, snapshotVer uint64, prior startHandle = math.MinInt64 endHandle = math.MaxInt64 // Get the start handle of this partition. - err = iterateSnapshotRows(d.store, priority, tbl, snapshotVer, math.MinInt64, + err = iterateSnapshotRows(d.store, priority, tbl, snapshotVer, math.MinInt64, math.MaxInt64, true, func(h int64, rowKey kv.Key, rawRecord []byte) (bool, error) { startHandle = h return false, nil diff --git a/store/tikv/scan.go b/store/tikv/scan.go index c3d5c555ba37b..1ce05ecc6089d 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -98,6 +98,11 @@ func (s *Scanner) Next() error { } current := s.cache[s.idx] + if len(s.endKey) > 0 && kv.Key(current.Key).Cmp(kv.Key(s.endKey)) >= 0 { + s.eof = true + s.Close() + return nil + } // Try to resolve the lock if current.GetError() != nil { // 'current' would be modified if the lock being resolved diff --git a/structure/hash.go b/structure/hash.go index bfa13118cce8c..e0f7b32cdb726 100644 --- a/structure/hash.go +++ b/structure/hash.go @@ -238,7 +238,7 @@ func (t *TxStructure) HClear(key []byte) error { func (t *TxStructure) iterateHash(key []byte, fn func(k []byte, v []byte) error) error { dataPrefix := t.hashDataKeyPrefix(key) - it, err := t.reader.Iter(dataPrefix, nil) + it, err := t.reader.Iter(dataPrefix, dataPrefix.PrefixNext()) if err != nil { return errors.Trace(err) } diff --git a/table/tables/index.go b/table/tables/index.go index c36d55c698cb1..f4580d0e93978 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -240,7 +240,7 @@ func (c *index) Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues // Drop removes the KV index from store. func (c *index) Drop(rm kv.RetrieverMutator) error { - it, err := rm.Iter(c.prefix, nil) + it, err := rm.Iter(c.prefix, c.prefix.PrefixNext()) if err != nil { return errors.Trace(err) } @@ -270,7 +270,8 @@ func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues return nil, false, errors.Trace(err) } - it, err := r.Iter(key, nil) + upperBound := c.prefix.PrefixNext() + it, err := r.Iter(key, upperBound) if err != nil { return nil, false, errors.Trace(err) } @@ -284,7 +285,8 @@ func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues // SeekFirst returns an iterator which points to the first entry of the KV index. func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) { - it, err := r.Iter(c.prefix, nil) + upperBound := c.prefix.PrefixNext() + it, err := r.Iter(c.prefix, upperBound) if err != nil { return nil, errors.Trace(err) } diff --git a/table/tables/tables.go b/table/tables/tables.go index 4a75d3f7e84d3..c24dd25d7f1d3 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -766,7 +766,8 @@ func (t *tableCommon) buildIndexForRow(ctx sessionctx.Context, rm kv.RetrieverMu // IterRecords implements table.Table IterRecords interface. func (t *tableCommon) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { - it, err := ctx.Txn().Iter(startKey, nil) + prefix := t.RecordPrefix() + it, err := ctx.Txn().Iter(startKey, prefix.PrefixNext()) if err != nil { return errors.Trace(err) } @@ -782,7 +783,6 @@ func (t *tableCommon) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols for _, col := range cols { colMap[col.ID] = &col.FieldType } - prefix := t.RecordPrefix() defaultVals := make([]types.Datum, len(cols)) for it.Valid() && it.Key().HasPrefix(prefix) { // first kv pair is row lock information. @@ -896,7 +896,7 @@ func (t *tableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetS // Seek implements table.Table Seek interface. func (t *tableCommon) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { seekKey := tablecodec.EncodeRowKeyWithHandle(t.physicalTableID, h) - iter, err := ctx.Txn().Iter(seekKey, nil) + iter, err := ctx.Txn().Iter(seekKey, t.RecordPrefix().PrefixNext()) if !iter.Valid() || !iter.Key().HasPrefix(t.RecordPrefix()) { // No more records in the table, skip to the end. return 0, false, nil diff --git a/util/admin/admin.go b/util/admin/admin.go index a76d5f416e8d1..a137c41ab410f 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -636,7 +636,10 @@ func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h // genExprs use to calculate generated column value. func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Table, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc, genExprs map[string]expression.Expression) error { - it, err := retriever.Iter(startKey, nil) + prefix := t.RecordPrefix() + keyUpperBound := prefix.PrefixNext() + + it, err := retriever.Iter(startKey, keyUpperBound) if err != nil { return errors.Trace(err) } @@ -663,7 +666,6 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab } } - prefix := t.RecordPrefix() for it.Valid() && it.Key().HasPrefix(prefix) { // first kv pair is row lock information. // TODO: check valid lock diff --git a/util/prefix_helper.go b/util/prefix_helper.go index a2c5d76dfd168..15fe13491291b 100644 --- a/util/prefix_helper.go +++ b/util/prefix_helper.go @@ -26,7 +26,7 @@ import ( // ScanMetaWithPrefix scans metadata with the prefix. func ScanMetaWithPrefix(retriever kv.Retriever, prefix kv.Key, filter func(kv.Key, []byte) bool) error { - iter, err := retriever.Iter(prefix, nil) + iter, err := retriever.Iter(prefix, prefix.PrefixNext()) if err != nil { return errors.Trace(err) } @@ -56,7 +56,7 @@ func ScanMetaWithPrefix(retriever kv.Retriever, prefix kv.Key, filter func(kv.Ke // DelKeyWithPrefix deletes keys with prefix. func DelKeyWithPrefix(rm kv.RetrieverMutator, prefix kv.Key) error { var keys []kv.Key - iter, err := rm.Iter(prefix, nil) + iter, err := rm.Iter(prefix, prefix.PrefixNext()) if err != nil { return errors.Trace(err) } From 43049f473ff62e9fb420cd835afa63c28f8add52 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 12 Nov 2018 15:46:11 +0800 Subject: [PATCH 070/509] *: support for "admin show next_row_id" (#8268) --- ast/misc.go | 1 + executor/admin_test.go | 43 ++++++++++++++++++++++++++++++++++++ executor/aggregate_test.go | 2 +- executor/builder.go | 10 +++++++++ executor/executor.go | 38 +++++++++++++++++++++++++++++++ parser/misc.go | 1 + parser/parser.y | 10 ++++++++- parser/parser_test.go | 1 + planner/core/common_plans.go | 6 +++++ planner/core/planbuilder.go | 13 +++++++++++ 10 files changed, 123 insertions(+), 2 deletions(-) diff --git a/ast/misc.go b/ast/misc.go index c4493a24ccd66..e9e20b162e0f0 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -628,6 +628,7 @@ const ( AdminShowDDLJobQueries AdminChecksumTable AdminShowSlow + AdminShowNextRowID ) // HandleRange represents a range where handle value >= Begin and < End. diff --git a/executor/admin_test.go b/executor/admin_test.go index 0a1282c582388..fa3ed027cb605 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -18,6 +18,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -515,3 +516,45 @@ func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) { tk.MustExec("insert into t values(1, 1, 1), (9223372036854775807, 2, 2);") tk.MustExec("admin check index t idx;") } + +func (s *testSuite) TestAdminShowNextID(c *C) { + step := int64(10) + autoIDStep := autoid.GetStep() + autoid.SetStep(step) + defer autoid.SetStep(autoIDStep) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t(id int, c int)") + // Start handle is 1. + r := tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 1")) + // Row ID is step + 1. + tk.MustExec("insert into t values(1, 1)") + r = tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 11")) + // Row ID is original + step. + for i := 0; i < int(step); i++ { + tk.MustExec("insert into t values(10000, 1)") + } + r = tk.MustQuery("admin show t next_row_id") + r.Check(testkit.Rows("test t _tidb_rowid 21")) + + // test for a table with the primary key + tk.MustExec("create table tt(id int primary key auto_increment, c int)") + // Start handle is 1. + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test tt id 1")) + // After rebasing auto ID, row ID is 20 + step + 1. + tk.MustExec("insert into tt values(20, 1)") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test tt id 31")) + // test for renaming the table + tk.MustExec("create database test1") + tk.MustExec("rename table test.tt to test1.tt") + tk.MustExec("use test1") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test1 tt id 31")) + tk.MustExec("insert test1.tt values ()") + r = tk.MustQuery("admin show tt next_row_id") + r.Check(testkit.Rows("test1 tt id 41")) +} diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 9b2d861907bcf..f8a098097575c 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -239,7 +239,7 @@ func (s *testSuite) TestAggregation(c *C) { result = tk.MustQuery("select count(*) from information_schema.columns") // When adding new memory columns in information_schema, please update this variable. - columnCountOfAllInformationSchemaTables := "757" + columnCountOfAllInformationSchemaTables := "759" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") diff --git a/executor/builder.go b/executor/builder.go index 68af3a6bfd888..2af348a383579 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -112,6 +112,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildSelectLock(v) case *plannercore.CancelDDLJobs: return b.buildCancelDDLJobs(v) + case *plannercore.ShowNextRowID: + return b.buildShowNextRowID(v) case *plannercore.ShowDDL: return b.buildShowDDL(v) case *plannercore.ShowDDLJobs: @@ -185,6 +187,14 @@ func (b *executorBuilder) buildCancelDDLJobs(v *plannercore.CancelDDLJobs) Execu return e } +func (b *executorBuilder) buildShowNextRowID(v *plannercore.ShowNextRowID) Executor { + e := &ShowNextRowIDExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + tblName: v.TableName, + } + return e +} + func (b *executorBuilder) buildShowDDL(v *plannercore.ShowDDL) Executor { // We get DDLInfo here because for Executors that returns result set, // next will be called after transaction has been committed. diff --git a/executor/executor.go b/executor/executor.go index 2ea2381db9ec8..0d97743e7b93c 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -55,6 +55,7 @@ var ( _ Executor = &ProjectionExec{} _ Executor = &SelectionExec{} _ Executor = &SelectLockExec{} + _ Executor = &ShowNextRowIDExec{} _ Executor = &ShowDDLExec{} _ Executor = &ShowDDLJobsExec{} _ Executor = &ShowDDLJobQueriesExec{} @@ -200,6 +201,43 @@ func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } +// ShowNextRowIDExec represents a show the next row ID executor. +type ShowNextRowIDExec struct { + baseExecutor + tblName *ast.TableName + done bool +} + +// Next implements the Executor Next interface. +func (e *ShowNextRowIDExec) Next(ctx context.Context, chk *chunk.Chunk) error { + chk.Reset() + if e.done { + return nil + } + is := domain.GetDomain(e.ctx).InfoSchema() + tbl, err := is.TableByName(e.tblName.Schema, e.tblName.Name) + if err != nil { + return errors.Trace(err) + } + colName := model.ExtraHandleName + for _, col := range tbl.Meta().Columns { + if mysql.HasAutoIncrementFlag(col.Flag) { + colName = col.Name + break + } + } + nextGlobalID, err := tbl.Allocator(e.ctx).NextGlobalAutoID(tbl.Meta().ID) + if err != nil { + return errors.Trace(err) + } + chk.AppendString(0, e.tblName.Schema.O) + chk.AppendString(1, e.tblName.Name.O) + chk.AppendString(2, colName.O) + chk.AppendInt64(3, nextGlobalID) + e.done = true + return nil +} + // ShowDDLExec represents a show DDL executor. type ShowDDLExec struct { baseExecutor diff --git a/parser/misc.go b/parser/misc.go index 51c623016dd05..4190932fb75f7 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -355,6 +355,7 @@ var tokenMap = map[string]int{ "NAMES": names, "NATIONAL": national, "NATURAL": natural, + "NEXT_ROW_ID": nextRowID, "NO": no, "NO_WRITE_TO_BINLOG": noWriteToBinLog, "NONE": none, diff --git a/parser/parser.y b/parser/parser.y index a94c9259c9e58..6326d2cb30028 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -420,6 +420,7 @@ import ( extract "EXTRACT" getFormat "GET_FORMAT" groupConcat "GROUP_CONCAT" + nextRowID "NEXT_ROW_ID" inplace "INPLACE" internal "INTERNAL" min "MIN" @@ -2838,7 +2839,7 @@ TiDBKeyword: NotKeywordToken: "ADDDATE" | "BIT_AND" | "BIT_OR" | "BIT_XOR" | "CAST" | "COPY" | "COUNT" | "CURTIME" | "DATE_ADD" | "DATE_SUB" | "EXTRACT" | "GET_FORMAT" | "GROUP_CONCAT" | "INPLACE" | "INTERNAL" -|"MIN" | "MAX" | "MAX_EXECUTION_TIME" | "NOW" | "RECENT" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TOP" | "TRIM" +|"MIN" | "MAX" | "MAX_EXECUTION_TIME" | "NOW" | "RECENT" | "POSITION" | "SUBDATE" | "SUBSTRING" | "SUM" | "TIMESTAMPADD" | "TIMESTAMPDIFF" | "TOP" | "TRIM" | "NEXT_ROW_ID" /************************************************************************************ * @@ -5173,6 +5174,13 @@ AdminStmt: JobNumber: $5.(int64), } } +| "ADMIN" "SHOW" TableName "NEXT_ROW_ID" + { + $$ = &ast.AdminStmt{ + Tp: ast.AdminShowNextRowID, + Tables: []*ast.TableName{$3.(*ast.TableName)}, + } + } | "ADMIN" "CHECK" "TABLE" TableNameList { $$ = &ast.AdminStmt{ diff --git a/parser/parser_test.go b/parser/parser_test.go index 15370afa33d93..af7a8ba4f2bbc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -439,6 +439,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"admin checksum table t1, t2;", true}, {"admin cancel ddl jobs 1", true}, {"admin cancel ddl jobs 1, 2", true}, + {"admin show t1 next_row_id", true}, {"admin recover index t1 idx_a", true}, {"admin cleanup index t1 idx_a", true}, {"admin show slow top 3", true}, diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 6a4db7fe3795b..7f750c67cb260 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -60,6 +60,12 @@ type ShowDDLJobQueries struct { JobIDs []int64 } +// ShowNextRowID is for showing the next global row ID. +type ShowNextRowID struct { + baseSchemaProducer + TableName *ast.TableName +} + // CheckTable is used for checking table data, built from the 'admin check table' statement. type CheckTable struct { baseSchemaProducer diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f218ecf944a7d..3601e4b712532 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -476,6 +476,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { p := &ChecksumTable{Tables: as.Tables} p.SetSchema(buildChecksumTableSchema()) ret = p + case ast.AdminShowNextRowID: + p := &ShowNextRowID{TableName: as.Tables[0]} + p.SetSchema(buildShowNextRowID()) + ret = p case ast.AdminShowDDL: p := &ShowDDL{} p.SetSchema(buildShowDDLFields()) @@ -706,6 +710,15 @@ func (b *planBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { return b.buildAnalyzeTable(as), nil } +func buildShowNextRowID() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, 4)...) + schema.Append(buildColumn("", "DB_NAME", mysql.TypeVarchar, mysql.MaxDatabaseNameLength)) + schema.Append(buildColumn("", "TABLE_NAME", mysql.TypeVarchar, mysql.MaxTableNameLength)) + schema.Append(buildColumn("", "COLUMN_NAME", mysql.TypeVarchar, mysql.MaxColumnNameLength)) + schema.Append(buildColumn("", "NEXT_GLOBAL_ROW_ID", mysql.TypeLonglong, 4)) + return schema +} + func buildShowDDLFields() *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, 4)...) schema.Append(buildColumn("", "SCHEMA_VER", mysql.TypeLonglong, 4)) From a6d2c0e11eb5618cd1d8a74bfc18df3067a59edb Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 12 Nov 2018 17:11:08 +0800 Subject: [PATCH 071/509] add changelog for v2.1.0-rc.5 (#8272) (#8275) --- CHANGELOG.md | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4181b0dfc0773..083bc8c44f4b0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,38 @@ # TiDB Changelog All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/pingcap/pd/blob/master/CHANGELOG.md). +## [2.1.0-rc.5] - 2018-11-12 +### SQL Optimizer +* Fix the issue that `IndexReader` reads the wrong handle in some cases [#8132](https://github.com/pingcap/tidb/pull/8132) +* Fix the issue occurred while the `IndexScan Prepared` statement uses `Plan Cache` [#8055](https://github.com/pingcap/tidb/pull/8055) +* Fix the issue that the result of the `Union` statement is unstable [#8165](https://github.com/pingcap/tidb/pull/8165) +### SQL Execution Engine +* Improve the performance of TiDB on inserting or updating wide tables [#8024](https://github.com/pingcap/tidb/pull/8024) +* Support the unsigned `int` flag in the `Truncate` built-in function [#8068](https://github.com/pingcap/tidb/pull/8068) +* Fix the error occurred while converting JSON data to the decimal type [#8109](https://github.com/pingcap/tidb/pull/8109) +* Fix the error occurred when you `Update` the float type [#8170](https://github.com/pingcap/tidb/pull/8170) +### Statistics +* Fix the incorrect statistics issue during point queries in some cases [#8035](https://github.com/pingcap/tidb/pull/8035) +* Fix the selectivity estimation of statistics for primary key in some cases [#8149](https://github.com/pingcap/tidb/pull/8149) +* Fix the issue that the statistics of deleted tables are not cleared up for a long period of time [#8182](https://github.com/pingcap/tidb/pull/8182) +### Server +* Improve the readability of logs and make logs better + - [#8063](https://github.com/pingcap/tidb/pull/8063) + - [#8053](https://github.com/pingcap/tidb/pull/8053) + - [#8224](https://github.com/pingcap/tidb/pull/8224) +* Fix the error occurred when obtaining the table data of `infoschema.profiling` [#8096](https://github.com/pingcap/tidb/pull/8096) +* Replace the unix socket with the pumps client to write binlogs [#8098](https://github.com/pingcap/tidb/pull/8098) +* Add the threshold value for the `tidb_slow_log_threshold` environment variable, which dynamically sets the slow log [#8094](https://github.com/pingcap/tidb/pull/8094) +* Add the original length of a SQL statement truncated while the `tidb_query_log_max_len` environment variable dynamically sets logs [8200](https://github.com/pingcap/tidb/pull/8200) +* Add the `tidb_opt_write_row_id` environment variable to control whether to allow writing `_tidb_rowid` [#8218](https://github.com/pingcap/tidb/pull/8218) +* Add an upper bound to the `Scan` command of ticlient, to avoid overbound scan [#8081](https://github.com/pingcap/tidb/pull/8081), [#8247](https://github.com/pingcap/tidb/pull/8247) +### DDL +* Fix the issue that executing DDL statements in transactions encounters an error in some cases [#8056](https://github.com/pingcap/tidb/pull/8056) +* Fix the issue that executing `truncate table` in partition tables does not take effect [#8103](https://github.com/pingcap/tidb/pull/8103) +* Fix the issue that the DDL operation does not roll back correctly after being cancelled in some cases [#8057](https://github.com/pingcap/tidb/pull/8057) +* Add the `admin show next_row_id` command to return the next available row ID [#8268](https://github.com/pingcap/tidb/pull/8268) + + ## [2.1.0-rc.4] - 2018-10-23 ### SQL Optimizer * Fix the issue that column pruning of `UnionAll` is incorrect in some cases [#7941](https://github.com/pingcap/tidb/pull/7941) From 67b37c5f9f6fd17f3b1f6ff66bf5476b96d28f61 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Tue, 13 Nov 2018 13:56:41 +0800 Subject: [PATCH 072/509] planner: fix expression rewriter wrong compare logic (#8269) (#8289) --- planner/core/expression_rewriter.go | 33 ++++++------- planner/core/expression_rewriter_test.go | 60 ++++++++++++++++++++++++ 2 files changed, 74 insertions(+), 19 deletions(-) create mode 100644 planner/core/expression_rewriter_test.go diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 0f804f6d430e6..4d5ceed793b33 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -169,15 +169,11 @@ type expressionRewriter struct { } // 1. If op are EQ or NE or NullEQ, constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to (a0 op b0) and (a1 op b1) and (a2 op b2) -// 2. If op are LE or GE, constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to -// `IF( (a0 op b0) EQ 0, 0, -// IF ( (a1 op b1) EQ 0, 0, a2 op b2))` -// 3. If op are LT or GT, constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to +// 2. Else constructBinaryOpFunctions converts (a0,a1,a2) op (b0,b1,b2) to // `IF( a0 NE b0, a0 op b0, -// IF( a1 NE b1, -// a1 op b1, -// a2 op b2) -// )` +// IF ( isNull(a0 NE b0), Null, +// IF ( a1 NE b1, a1 op b1, +// IF ( isNull(a1 NE b1), Null, a2 op b2))))` func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, r expression.Expression, op string) (expression.Expression, error) { lLen, rLen := expression.GetRowLen(l), expression.GetRowLen(r) if lLen == 1 && rLen == 1 { @@ -198,15 +194,10 @@ func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, return expression.ComposeCNFCondition(er.ctx, funcs...), nil default: larg0, rarg0 := expression.GetFuncArg(l, 0), expression.GetFuncArg(r, 0) - var expr1, expr2, expr3 expression.Expression - if op == ast.LE || op == ast.GE { - expr1 = expression.NewFunctionInternal(er.ctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) - expr1 = expression.NewFunctionInternal(er.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), expr1, expression.Zero) - expr2 = expression.Zero - } else if op == ast.LT || op == ast.GT { - expr1 = expression.NewFunctionInternal(er.ctx, ast.NE, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) - expr2 = expression.NewFunctionInternal(er.ctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) - } + var expr1, expr2, expr3, expr4, expr5 expression.Expression + expr1 = expression.NewFunctionInternal(er.ctx, ast.NE, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) + expr2 = expression.NewFunctionInternal(er.ctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) + expr3 = expression.NewFunctionInternal(er.ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), expr1) var err error l, err = expression.PopRowFirstArg(er.ctx, l) if err != nil { @@ -216,11 +207,15 @@ func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, if err != nil { return nil, errors.Trace(err) } - expr3, err = er.constructBinaryOpFunction(l, r, op) + expr4, err = er.constructBinaryOpFunction(l, r, op) + if err != nil { + return nil, errors.Trace(err) + } + expr5, err = expression.NewFunction(er.ctx, ast.If, types.NewFieldType(mysql.TypeTiny), expr3, expression.Null, expr4) if err != nil { return nil, errors.Trace(err) } - return expression.NewFunction(er.ctx, ast.If, types.NewFieldType(mysql.TypeTiny), expr1, expr2, expr3) + return expression.NewFunction(er.ctx, ast.If, types.NewFieldType(mysql.TypeTiny), expr1, expr2, expr5) } } diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go new file mode 100644 index 0000000000000..bedd1328446ca --- /dev/null +++ b/planner/core/expression_rewriter_test.go @@ -0,0 +1,60 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" +) + +var _ = Suite(&testExpressionRewriterSuite{}) + +type testExpressionRewriterSuite struct { +} + +func (s *testExpressionRewriterSuite) TestIfNullEliminateColName(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int not null, b int not null)") + rs, err := tk.Exec("select ifnull(a,b) from t") + c.Assert(err, IsNil) + fields := rs.Fields() + c.Assert(fields[0].Column.Name.L, Equals, "ifnull(a,b)") +} + +func (s *testExpressionRewriterSuite) TestBinaryOpFunction(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t(a int, b int, c int);") + tk.MustExec("INSERT INTO t VALUES (1, 2, 3), (NULL, 2, 3 ), (1, NULL, 3),(1, 2, NULL),(NULL, 2, 3+1), (1, NULL, 3+1), (1, 2+1, NULL),(NULL, 2, 3-1), (1, NULL, 3-1), (1, 2-1, NULL)") + tk.MustQuery("SELECT * FROM t WHERE (a,b,c) <= (1,2,3) order by b").Check(testkit.Rows("1 1 ", "1 2 3")) + tk.MustQuery("SELECT * FROM t WHERE (a,b,c) > (1,2,3) order by b").Check(testkit.Rows("1 3 ")) +} From 03639c3a70a8563c05e70d128d54a57c96bf68d2 Mon Sep 17 00:00:00 2001 From: Tao Meng Date: Wed, 14 Nov 2018 10:43:32 +0800 Subject: [PATCH 073/509] planner: fix error caused by different length between inner and outer OrderByItems(#8273) (#8301) --- planner/core/exhaust_physical_plans.go | 3 +++ planner/core/physical_plan_test.go | 9 +++++++++ 2 files changed, 12 insertions(+) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 37d8dfee2b1b1..97232a2501fe6 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -733,6 +733,9 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { // Check if this prop's columns can match by items totally. func matchItems(p *property.PhysicalProperty, items []*ByItems) bool { + if len(items) < len(p.Cols) { + return false + } for i, col := range p.Cols { sortItem := items[i] if sortItem.Desc != p.Desc || !sortItem.Expr.Equal(nil, col) { diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 2a82e8bac589d..04e4d4fc55d30 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -188,6 +188,15 @@ func (s *testPlanSuite) TestDAGPlanBuilderSimpleCase(c *C) { sql: "select * from (select * from t use index() order by b) t left join t t1 on t.a=t1.a limit 10", best: "IndexJoin{TableReader(Table(t)->TopN([test.t.b],0,10))->TopN([test.t.b],0,10)->TableReader(Table(t))}(test.t.a,t1.a)->Limit", }, + // Test embedded ORDER BY which imposes on different number of columns than outer query. + { + sql: "select * from ((SELECT 1 a,3 b) UNION (SELECT 2,1) ORDER BY (SELECT 2)) t order by a,b", + best: "UnionAll{Dual->Projection->Dual->Projection}->HashAgg->Sort", + }, + { + sql: "select * from ((SELECT 1 a,6 b) UNION (SELECT 2,5) UNION (SELECT 2, 4) ORDER BY 1) t order by 1, 2", + best: "UnionAll{Dual->Projection->Dual->Projection->Dual->Projection}->HashAgg->Sort->Sort", + }, } for i, tt := range tests { comment := Commentf("case:%v sql:%s", i, tt.sql) From a5afbafba342a9e3cf897aa6484cc5f5020a3dea Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Wed, 14 Nov 2018 12:34:03 +0800 Subject: [PATCH 074/509] planner/core: make TIDB_INLJ to specify the inner table (#8243) (#8298) --- executor/index_lookup_join_test.go | 2 +- executor/join_test.go | 22 +++++++++++----------- planner/core/exhaust_physical_plans.go | 4 ++-- planner/core/logical_plan_builder.go | 11 +++++------ planner/core/logical_plans.go | 4 ++-- planner/core/physical_plan_test.go | 16 ++++++++-------- 6 files changed, 29 insertions(+), 30 deletions(-) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index ecca46773e1d9..f25792324db93 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -29,7 +29,7 @@ func (s *testSuite) TestIndexLookupJoinHang(c *C) { tk.Se.GetSessionVars().IndexJoinBatchSize = 1 tk.Se.GetSessionVars().IndexLookupJoinConcurrency = 1 - rs, err := tk.Exec("select /*+ TIDB_INLJ(o, i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0") + rs, err := tk.Exec("select /*+ TIDB_INLJ(i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0") c.Assert(err, IsNil) chk := rs.NewChunk() for i := 0; i < 5; i++ { diff --git a/executor/join_test.go b/executor/join_test.go index e79758db5ef96..681687d9c1373 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -145,11 +145,11 @@ func (s *testSuite) TestJoin(c *C) { // The physical plans of the two sql are tested at physical_plan_test.go tk.MustQuery("select /*+ TIDB_INLJ(t, t1) */ * from t join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4")) - tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Check(testkit.Rows("1 2 1 1", "1 3 1 1", "1 4 1 1", "3 4 3 3")) + tk.MustQuery("select /*+ TIDB_INLJ(t) */ * from t1 join t on t.a=t1.a and t.a < t1.b").Check(testkit.Rows("1 2 1 1", "1 3 1 1", "1 4 1 1", "3 4 3 3")) // Test single index reader. tk.MustQuery("select /*+ TIDB_INLJ(t, t1) */ t1.b from t1 join t on t.b=t1.b").Check(testkit.Rows("2", "3")) - tk.MustQuery("select /*+ TIDB_INLJ(t, t1) */ * from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) - tk.MustQuery("select /*+ TIDB_INLJ(t, t1) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000")) + tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) + tk.MustQuery("select /*+ TIDB_INLJ(t) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000")) // Test that two conflict hints will return error. _, err = tk.Exec("select /*+ TIDB_INLJ(t) TIDB_SMJ(t) */ * from t join t1 on t.a=t1.a") @@ -169,8 +169,8 @@ func (s *testSuite) TestJoin(c *C) { tk.MustExec("create table t1(a int, b int)") tk.MustExec("insert into t values(1, 3), (2, 2), (3, 1)") tk.MustExec("insert into t1 values(0, 0), (1, 2), (1, 3), (3, 4)") - tk.MustQuery("select /*+ TIDB_INLJ(t) */ * from t join t1 on t.a=t1.a order by t.b").Check(testkit.Rows("3 1 3 4", "1 3 1 2", "1 3 1 3")) - tk.MustQuery("select /*+ TIDB_INLJ(t1) */ t.a, t.b from t join t1 on t.a=t1.a where t1.b = 4 limit 1").Check(testkit.Rows("3 1")) + tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t join t1 on t.a=t1.a order by t.b").Check(testkit.Rows("3 1 3 4", "1 3 1 2", "1 3 1 3")) + tk.MustQuery("select /*+ TIDB_INLJ(t) */ t.a, t.b from t join t1 on t.a=t1.a where t1.b = 4 limit 1").Check(testkit.Rows("3 1")) tk.MustQuery("select /*+ TIDB_INLJ(t, t1) */ * from t right join t1 on t.a=t1.a order by t.b").Check(testkit.Rows(" 0 0", "3 1 3 4", "1 3 1 2", "1 3 1 3")) // join reorder will disorganize the resulting schema @@ -187,7 +187,7 @@ func (s *testSuite) TestJoin(c *C) { tk.MustExec("create table t1(a int)") tk.MustExec("insert into t values(1,2), (5,3), (6,4)") tk.MustExec("insert into t1 values(1), (2), (3)") - tk.MustQuery("select /*+ TIDB_INLJ(t1) */ t1.a from t1, t where t.a = 5 and t.b = t1.a").Check(testkit.Rows("3")) + tk.MustQuery("select /*+ TIDB_INLJ(t) */ t1.a from t1, t where t.a = 5 and t.b = t1.a").Check(testkit.Rows("3")) // test issue#4997 tk.MustExec("drop table if exists t1, t2") @@ -326,7 +326,7 @@ func (s *testSuite) TestJoinCast(c *C) { tk.MustExec("create index k1 on t1(c1)") tk.MustExec("insert into t values(0), (2)") tk.MustExec("insert into t1 values(0), (9)") - result = tk.MustQuery("select /*+ TIDB_INLJ(t) */ * from t left join t1 on t1.c1 = t.c1") + result = tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t left join t1 on t1.c1 = t.c1") result.Sort().Check(testkit.Rows("0.0 0.00", "2.0 ")) tk.MustExec("drop table if exists t") @@ -672,7 +672,7 @@ func (s *testSuite) TestSubquery(c *C) { tk.MustExec("CREATE TABLE t1(a int, b int default 0)") tk.MustExec("create index k1 on t1(a)") tk.MustExec("INSERT INTO t1 (a) values(1), (2), (3), (4), (5)") - result = tk.MustQuery("select (select /*+ TIDB_INLJ(x1) */ x2.a from t1 x1, t1 x2 where x1.a = t1.a and x1.a = x2.a) from t1") + result = tk.MustQuery("select (select /*+ TIDB_INLJ(x2) */ x2.a from t1 x1, t1 x2 where x1.a = t1.a and x1.a = x2.a) from t1") result.Check(testkit.Rows("1", "2", "3", "4", "5")) tk.MustExec("drop table if exists t1, t2") @@ -809,7 +809,7 @@ func (s *testSuite) TestIssue5255(c *C) { tk.MustExec("create table t2(a int primary key)") tk.MustExec("insert into t1 values(1, '2017-11-29', 2.2)") tk.MustExec("insert into t2 values(1)") - tk.MustQuery("select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a=t2.a").Check(testkit.Rows("1 2017-11-29 2.2 1")) + tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t1 join t2 on t1.a=t2.a").Check(testkit.Rows("1 2017-11-29 2.2 1")) } func (s *testSuite) TestIssue5278(c *C) { @@ -847,7 +847,7 @@ func (s *testSuite) TestIndexLookupJoin(c *C) { tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a decimal(6,2), index idx(a));`) tk.MustExec(`insert into t values(1.01), (2.02), (NULL);`) - tk.MustQuery(`select /*+ TIDB_INLJ(t1) */ t1.a from t t1 join t t2 on t1.a=t2.a order by t1.a;`).Check(testkit.Rows( + tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ t1.a from t t1 join t t2 on t1.a=t2.a order by t1.a;`).Check(testkit.Rows( `1.01`, `2.02`, )) @@ -856,7 +856,7 @@ func (s *testSuite) TestIndexLookupJoin(c *C) { tk.MustExec(`create table t(a bigint, b bigint, unique key idx1(a, b));`) tk.MustExec(`insert into t values(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6);`) tk.MustExec(`set @@tidb_max_chunk_size = 2;`) - tk.MustQuery(`select /*+ TIDB_INLJ(t1) */ * from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b + 4;`).Check(testkit.Rows( + tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b + 4;`).Check(testkit.Rows( `1 1 `, `1 2 `, `1 3 `, diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 97232a2501fe6..6a111bc5d63da 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -591,8 +591,8 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) ([]Phys return nil, false } plans := make([]PhysicalPlan, 0, 2) - leftOuter := (p.preferJoinType & preferLeftAsIndexOuter) > 0 - rightOuter := (p.preferJoinType & preferRightAsIndexOuter) > 0 + rightOuter := (p.preferJoinType & preferLeftAsIndexInner) > 0 + leftOuter := (p.preferJoinType & preferRightAsIndexInner) > 0 switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: join := p.getIndexJoinByOuterIdx(prop, 0) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 24fa283513d0a..a10cf7b82ec5e 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -255,15 +255,15 @@ func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) error { p.preferJoinType |= preferHashJoin } if hintInfo.ifPreferINLJ(lhsAlias) { - p.preferJoinType |= preferLeftAsIndexOuter + p.preferJoinType |= preferLeftAsIndexInner } if hintInfo.ifPreferINLJ(rhsAlias) { - p.preferJoinType |= preferRightAsIndexOuter + p.preferJoinType |= preferRightAsIndexInner } // If there're multiple join types and one of them is not index join hint, // then there is a conflict of join types. - if bits.OnesCount(p.preferJoinType) > 1 && (p.preferJoinType^preferRightAsIndexOuter^preferLeftAsIndexOuter) > 0 { + if bits.OnesCount(p.preferJoinType) > 1 && (p.preferJoinType^preferRightAsIndexInner^preferLeftAsIndexInner) > 0 { return errors.New("Join hints are conflict, you can only specify one type of join") } return nil @@ -2007,9 +2007,8 @@ func (b *planBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onConditio if b.TableHints().ifPreferHashJoin(outerAlias, innerAlias) { joinPlan.preferJoinType |= preferHashJoin } - // semi join's outer is always the left side. - if b.TableHints().ifPreferINLJ(outerAlias) { - joinPlan.preferJoinType = preferLeftAsIndexOuter + if b.TableHints().ifPreferINLJ(innerAlias) { + joinPlan.preferJoinType = preferRightAsIndexInner } // If there're multiple join hints, they're conflict. if bits.OnesCount(joinPlan.preferJoinType) > 1 { diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 99967e12e5f71..a87ddede8aff0 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -85,8 +85,8 @@ func (tp JoinType) String() string { } const ( - preferLeftAsIndexOuter = 1 << iota - preferRightAsIndexOuter + preferLeftAsIndexInner = 1 << iota + preferRightAsIndexInner preferHashJoin preferMergeJoin ) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 04e4d4fc55d30..58e47867914c9 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -372,7 +372,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, // Test Index Join + TableScan + Rotate. { - sql: "select /*+ TIDB_INLJ(t2) */ t1.a , t2.a from t t1, t t2 where t1.a = t2.c", + sql: "select /*+ TIDB_INLJ(t1) */ t1.a , t2.a from t t1, t t2 where t1.a = t2.c", best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t2.c,t1.a)->Projection", }, // Test Index Join + OuterJoin + TableScan. @@ -391,33 +391,33 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }, // Test Index Join failed. { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 right outer join t t2 on t1.a = t2.b", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 right outer join t t2 on t1.a = t2.b", best: "RightHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.b)", }, // Test Semi Join hint success. { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 where t1.a in (select a from t t2)", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 where t1.a in (select a from t t2)", best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", }, // Test Semi Join hint fail. { - sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 where t1.a in (select a from t t2)", + sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 where t1.a in (select a from t t2)", best: "MergeSemiJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", }, { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 join t t2 where t1.c=t2.c and t1.f=t2.f", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.c=t2.c and t1.f=t2.f", best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)", }, { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 join t t2 where t1.a = t2.a and t1.f=t2.f", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.a = t2.a and t1.f=t2.f", best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", }, { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 join t t2 where t1.f=t2.f and t1.a=t2.a", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.f=t2.f and t1.a=t2.a", best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", }, { - sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 join t t2 where t1.a=t2.a and t2.a in (1, 2)", + sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.a=t2.a and t2.a in (1, 2)", best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([in(t2.a, 1, 2)]))}(t1.a,t2.a)", }, } From 4ec85cd783e0a4efffe4edb0facedd226dade534 Mon Sep 17 00:00:00 2001 From: Andrew Date: Wed, 14 Nov 2018 16:00:44 +0800 Subject: [PATCH 075/509] planner, executor: eliminate extra columns introduced by OrderBy upon Union (#8290) (#8306) --- executor/executor_test.go | 9 +++++++++ planner/core/logical_plan_builder.go | 20 ++++++++++++++++++-- planner/core/logical_plan_test.go | 5 +++++ 3 files changed, 32 insertions(+), 2 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 1aa27ce095110..2f9f004a1518c 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1051,6 +1051,15 @@ func (s *testSuite) TestUnion(c *C) { tk.MustExec("insert into t1 value(1,2),(1,1),(2,2),(2,2),(3,2),(3,2)") tk.MustExec("set @@tidb_max_chunk_size=2;") tk.MustQuery("select count(*) from (select a as c, a as d from t1 union all select a, b from t1) t;").Check(testkit.Rows("12")) + + // #issue 8189 and #issue 8199 + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("CREATE TABLE t1 (a int not null, b char (10) not null)") + tk.MustExec("insert into t1 values(1,'a'),(2,'b'),(3,'c'),(3,'c')") + tk.MustExec("CREATE TABLE t2 (a int not null, b char (10) not null)") + tk.MustExec("insert into t2 values(1,'a'),(2,'b'),(3,'c'),(3,'c')") + tk.MustQuery("select a from t1 union select a from t1 order by (select a+1);").Check(testkit.Rows("1", "2", "3")) } func (s *testSuite) TestNeighbouringProj(c *C) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index a10cf7b82ec5e..42dec103e670c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -705,6 +705,8 @@ func (b *planBuilder) buildUnion(union *ast.UnionStmt) (LogicalPlan, error) { unionPlan = unionAllPlan } + oldLen := unionPlan.Schema().Len() + if union.OrderBy != nil { unionPlan, err = b.buildSort(unionPlan, union.OrderBy.Items, nil) if err != nil { @@ -718,6 +720,20 @@ func (b *planBuilder) buildUnion(union *ast.UnionStmt) (LogicalPlan, error) { return nil, errors.Trace(err) } } + + // Fix issue #8189 (https://github.com/pingcap/tidb/issues/8189). + // If there are extra expressions generated from `ORDER BY` clause, generate a `Projection` to remove them. + if oldLen != unionPlan.Schema().Len() { + proj := LogicalProjection{Exprs: expression.Column2Exprs(unionPlan.Schema().Columns[:oldLen])}.init(b.ctx) + proj.SetChildren(unionPlan) + schema := expression.NewSchema(unionPlan.Schema().Clone().Columns[:oldLen]...) + for _, col := range schema.Columns { + col.UniqueID = b.ctx.GetSessionVars().AllocPlanColumnID() + } + proj.SetSchema(schema) + return proj, nil + } + return unionPlan, nil } @@ -860,7 +876,7 @@ func (b *planBuilder) buildLimit(src LogicalPlan, limit *ast.Limit) (LogicalPlan return li, nil } -// colMatch(a,b) means that if a match b, e.g. t.a can match test.t.a but test.t.a can't match t.a. +// colMatch means that if a match b, e.g. t.a can match test.t.a but test.t.a can't match t.a. // Because column a want column from database test exactly. func colMatch(a *ast.ColumnName, b *ast.ColumnName) bool { if a.Schema.L == "" || a.Schema.L == b.Schema.L { @@ -917,7 +933,7 @@ func resolveFromSelectFields(v *ast.ColumnNameExpr, fields []*ast.SelectField, i return } -// AggregateFuncExtractor visits Expr tree. +// havingAndOrderbyExprResolver visits Expr tree. // It converts ColunmNameExpr to AggregateFuncExpr and collects AggregateFuncExpr. type havingAndOrderbyExprResolver struct { inAggFunc bool diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 8caf8dfeec1ec..91d093b7fa79b 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1816,6 +1816,11 @@ func (s *testPlanSuite) TestUnion(c *C) { best: "UnionAll{UnionAll{Dual->Projection->Projection->Dual->Projection->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection->Projection}->Projection->Sort", err: false, }, + { + sql: "select * from (select 1 as a union select 1 union all select 2) t order by (select a)", + best: "Apply{UnionAll{UnionAll{Dual->Projection->Projection->Dual->Projection->Projection}->Aggr(firstrow(a))->Projection->Dual->Projection->Projection}->Dual->Projection->MaxOneRow}->Sort->Projection", + err: false, + }, } for i, tt := range tests { comment := Commentf("case:%v sql:%s", i, tt.sql) From f0df5c6573fb34bc8ecf7417b95a74019e9d0c0b Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 14 Nov 2018 16:15:04 +0800 Subject: [PATCH 076/509] kv, store: remove `Rollback` in `RunInNewTxn` (#8250) (#8276) --- kv/txn.go | 2 -- store/tikv/2pc_fail_test.go | 5 +++++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/kv/txn.go b/kv/txn.go index 56221f45d3cfe..6a8710dbf51dd 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -63,8 +63,6 @@ func RunInNewTxn(store Storage, retryable bool, f func(txn Transaction) error) e } if retryable && IsRetryableError(err) { log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err) - err1 := txn.Rollback() - terror.Log(errors.Trace(err1)) BackOff(i) continue } diff --git a/store/tikv/2pc_fail_test.go b/store/tikv/2pc_fail_test.go index e840c94bed02b..5fe4b2b0d8b4e 100644 --- a/store/tikv/2pc_fail_test.go +++ b/store/tikv/2pc_fail_test.go @@ -16,6 +16,7 @@ package tikv import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/terror" "github.com/pkg/errors" "golang.org/x/net/context" @@ -33,6 +34,10 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { err = t1.Commit(context.Background()) c.Assert(err, NotNil) c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue, Commentf("%s", errors.ErrorStack(err))) + + // We don't need to call "Rollback" after "Commit" fails. + err = t1.Rollback() + c.Assert(err, Equals, kv.ErrInvalidTxn) } // TestFailCommitPrimaryRegionError tests RegionError is handled properly when From da65c8ee241bec379dde5c954888254a87f40610 Mon Sep 17 00:00:00 2001 From: kennytm Date: Thu, 15 Nov 2018 10:27:05 +0800 Subject: [PATCH 077/509] executor: properly escape backquotes in identifiers in SHOW CREATE TABLE (#8322) --- executor/show.go | 32 +++++++++++++++++++++++++------- executor/show_test.go | 29 +++++++++++++++++++++++++++++ 2 files changed, 54 insertions(+), 7 deletions(-) diff --git a/executor/show.go b/executor/show.go index 773e7e6d8e85b..72d20117ccdca 100644 --- a/executor/show.go +++ b/executor/show.go @@ -468,19 +468,35 @@ func getDefaultCollate(charsetName string) string { return "" } +// escape the identifier for pretty-printing. +// For instance, the identifier "foo `bar`" will become "`foo ``bar```". +// The sqlMode controls whether to escape with backquotes (`) or double quotes +// (`"`) depending on whether mysql.ModeANSIQuotes is enabled. +func escape(cis model.CIStr, sqlMode mysql.SQLMode) string { + var quote string + if sqlMode&mysql.ModeANSIQuotes != 0 { + quote = `"` + } else { + quote = "`" + } + return quote + strings.Replace(cis.O, quote, quote+quote, -1) + quote +} + func (e *ShowExec) fetchShowCreateTable() error { tb, err := e.getTable() if err != nil { return errors.Trace(err) } + sqlMode := e.ctx.GetSessionVars().SQLMode + // TODO: let the result more like MySQL. var buf bytes.Buffer - buf.WriteString(fmt.Sprintf("CREATE TABLE `%s` (\n", tb.Meta().Name.O)) + buf.WriteString(fmt.Sprintf("CREATE TABLE %s (\n", escape(tb.Meta().Name, sqlMode))) var pkCol *table.Column var hasAutoIncID bool for i, col := range tb.Cols() { - buf.WriteString(fmt.Sprintf(" `%s` %s", col.Name.O, col.GetTypeDesc())) + buf.WriteString(fmt.Sprintf(" %s %s", escape(col.Name, sqlMode), col.GetTypeDesc())) if col.IsGenerated() { // It's a generated column. buf.WriteString(fmt.Sprintf(" GENERATED ALWAYS AS (%s)", col.GeneratedExprString)) @@ -537,7 +553,7 @@ func (e *ShowExec) fetchShowCreateTable() error { if pkCol != nil { // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") - buf.WriteString(fmt.Sprintf(" PRIMARY KEY (`%s`)", pkCol.Name.O)) + buf.WriteString(fmt.Sprintf(" PRIMARY KEY (%s)", escape(pkCol.Name, sqlMode))) } if len(tb.Indices()) > 0 { @@ -555,14 +571,14 @@ func (e *ShowExec) fetchShowCreateTable() error { if idxInfo.Primary { buf.WriteString(" PRIMARY KEY ") } else if idxInfo.Unique { - buf.WriteString(fmt.Sprintf(" UNIQUE KEY `%s` ", idxInfo.Name.O)) + buf.WriteString(fmt.Sprintf(" UNIQUE KEY %s ", escape(idxInfo.Name, sqlMode))) } else { - buf.WriteString(fmt.Sprintf(" KEY `%s` ", idxInfo.Name.O)) + buf.WriteString(fmt.Sprintf(" KEY %s ", escape(idxInfo.Name, sqlMode))) } cols := make([]string, 0, len(idxInfo.Columns)) for _, c := range idxInfo.Columns { - colInfo := fmt.Sprintf("`%s`", c.Name.String()) + colInfo := escape(c.Name, sqlMode) if c.Length != types.UnspecifiedLength { colInfo = fmt.Sprintf("%s(%s)", colInfo, strconv.Itoa(c.Length)) } @@ -664,8 +680,10 @@ func (e *ShowExec) fetchShowCreateDatabase() error { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(e.DBName.O) } + sqlMode := e.ctx.GetSessionVars().SQLMode + var buf bytes.Buffer - fmt.Fprintf(&buf, "CREATE DATABASE `%s`", db.Name.O) + fmt.Fprintf(&buf, "CREATE DATABASE %s", escape(db.Name, sqlMode)) if s := db.Charset; len(s) > 0 { fmt.Fprintf(&buf, " /* !40100 DEFAULT CHARACTER SET %s */", s) } diff --git a/executor/show_test.go b/executor/show_test.go index 6d548ece3854a..21398c6c30ca6 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -667,3 +667,32 @@ func (s *testSuite) TestShowSlow(c *C) { tk.MustQuery(`admin show slow top internal 3`) tk.MustQuery(`admin show slow top all 3`) } + +func (s *testSuite) TestShowEscape(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists `t``abl\"e`") + tk.MustExec("create table `t``abl\"e`(`c``olum\"n` int(11) primary key)") + tk.MustQuery("show create table `t``abl\"e`").Check(testutil.RowsWithSep("|", + ""+ + "t`abl\"e CREATE TABLE `t``abl\"e` (\n"+ + " `c``olum\"n` int(11) NOT NULL,\n"+ + " PRIMARY KEY (`c``olum\"n`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin", + )) + + // ANSI_QUOTES will change the SHOW output + tk.MustExec("set @old_sql_mode=@@sql_mode") + tk.MustExec("set sql_mode=ansi_quotes") + tk.MustQuery("show create table \"t`abl\"\"e\"").Check(testutil.RowsWithSep("|", + ""+ + "t`abl\"e CREATE TABLE \"t`abl\"\"e\" (\n"+ + " \"c`olum\"\"n\" int(11) NOT NULL,\n"+ + " PRIMARY KEY (\"c`olum\"\"n\")\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin", + )) + + tk.MustExec("rename table \"t`abl\"\"e\" to t") + tk.MustExec("set sql_mode=@old_sql_mode") +} From d6482bfe6403a1b3f8915f3e5583ff6af8dc569b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 15 Nov 2018 12:09:14 +0800 Subject: [PATCH 078/509] session,executor: run statement without transaction (#8260) (#8325) --- executor/adapter.go | 16 +++--- executor/builder.go | 9 ++-- executor/errors.go | 2 + executor/executor.go | 4 -- executor/executor_test.go | 2 +- executor/index_lookup_join.go | 19 ++++++++ executor/simple.go | 2 + planner/core/logical_plan_builder.go | 2 +- session/session.go | 20 +++++--- session/session_test.go | 73 ++++++++++++++++++++++++---- session/tidb.go | 13 ++++- session/txn.go | 6 ++- sessionctx/context.go | 3 +- table/tables/tables.go | 2 +- util/mock/context.go | 28 +++++++---- 15 files changed, 156 insertions(+), 45 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 508ff1f9382f1..23abc549d1ef2 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -240,11 +240,15 @@ func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { return a.handleNoDelayExecutor(ctx, sctx, e, pi) } + var txnStartTS uint64 + if sctx.Txn(false).Valid() { + txnStartTS = sctx.Txn().StartTS() + } return &recordSet{ executor: e, stmt: a, processinfo: pi, - txnStartTS: sctx.Txn().StartTS(), + txnStartTS: txnStartTS, }, nil } @@ -266,7 +270,8 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co } terror.Log(errors.Trace(e.Close())) txnTS := uint64(0) - if sctx.Txn() != nil { + // Don't active pending txn here. + if sctx.Txn(false).Valid() { txnTS = sctx.Txn().StartTS() } a.LogSlowQuery(txnTS, err == nil) @@ -290,9 +295,6 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { if isPointGet { log.Debugf("con:%d InitTxnWithStartTS %s", ctx.GetSessionVars().ConnectionID, a.Text) err = ctx.InitTxnWithStartTS(math.MaxUint64) - } else { - log.Debugf("con:%d ActivePendingTxn %s", ctx.GetSessionVars().ConnectionID, a.Text) - err = ctx.ActivePendingTxn() } if err != nil { return nil, errors.Trace(err) @@ -408,7 +410,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { // IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions: // 1. ctx is auto commit tagged -// 2. txn is nil +// 2. txn is not valid // 2. plan is point get by pk or unique key func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannercore.Plan) bool { // check auto commit @@ -417,7 +419,7 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannerco } // check txn - if ctx.Txn() != nil { + if ctx.Txn(false).Valid() { return false } diff --git a/executor/builder.go b/executor/builder.go index 2af348a383579..69aa6dae7185c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -1146,8 +1147,6 @@ func (b *executorBuilder) buildTableDual(v *plannercore.PhysicalTableDual) Execu baseExecutor: base, numDualRows: v.RowCount, } - // Init the startTS for later use. - b.getStartTS() return e } @@ -1158,10 +1157,14 @@ func (b *executorBuilder) getStartTS() uint64 { } startTS := b.ctx.GetSessionVars().SnapshotTS - if startTS == 0 && b.ctx.Txn() != nil { + if startTS == 0 && b.ctx.Txn().Valid() { startTS = b.ctx.Txn().StartTS() } b.startTS = startTS + if b.startTS == 0 { + // The the code should never run here if there is no bug. + log.Error(errors.ErrorStack(errors.Trace(ErrGetStartTS))) + } return startTS } diff --git a/executor/errors.go b/executor/errors.go index ec8482ac56a44..e0aae2b46d2db 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -26,10 +26,12 @@ const ( codeResultIsEmpty codeErrBuildExec codeBatchInsertFail + codeGetStartTS ) // Error instances. var ( + ErrGetStartTS = terror.ClassExecutor.New(codeGetStartTS, "Can not get start ts") ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan") ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements") ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements") diff --git a/executor/executor.go b/executor/executor.go index 0d97743e7b93c..e3e412097d520 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -735,10 +735,6 @@ func init() { // but the plan package cannot import the executor package because of the dependency cycle. // So we assign a function implemented in the executor package to the plan package to avoid the dependency cycle. plannercore.EvalSubquery = func(p plannercore.PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) (rows [][]types.Datum, err error) { - err = sctx.ActivePendingTxn() - if err != nil { - return rows, errors.Trace(err) - } e := &executorBuilder{is: is, ctx: sctx} exec := e.build(p) if e.err != nil { diff --git a/executor/executor_test.go b/executor/executor_test.go index 2f9f004a1518c..4c3b01bcc9ca1 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2103,7 +2103,7 @@ func (s *testSuite) TestSelectForUpdate(c *C) { tk.MustExec("drop table if exists t, t1") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("create table t (c1 int, c2 int, c3 int)") tk.MustExec("insert t values (11, 2, 3)") tk.MustExec("insert t values (12, 2, 3)") diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index bc05f56d971fd..b42684c451f0b 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -129,6 +129,25 @@ type innerWorker struct { // Open implements the Executor interface. func (e *IndexLookUpJoin) Open(ctx context.Context) error { + // Be careful, very dirty hack in this line!!! + // IndexLookUpJoin need to rebuild executor (the dataReaderBuilder) during + // executing. However `executor.Next()` is lazy evaluation when the RecordSet + // result is drained. + // Lazy evaluation means the saved session context may change during executor's + // building and its running. + // A specific sequence for example: + // + // e := buildExecutor() // txn at build time + // recordSet := runStmt(e) + // session.CommitTxn() // txn closed + // recordSet.Next() + // e.dataReaderBuilder.Build() // txn is used again, which is already closed + // + // The trick here is `getStartTS` will cache start ts in the dataReaderBuilder, + // so even txn is destroyed later, the dataReaderBuilder could still use the + // cached start ts to construct DAG. + e.innerCtx.readerBuilder.getStartTS() + err := e.children[0].Open(ctx) if err != nil { return errors.Trace(err) diff --git a/executor/simple.go b/executor/simple.go index 7c3c1c5dc0030..aed2e196973f3 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -113,6 +113,8 @@ func (e *SimpleExec) executeBegin(s *ast.BeginStmt) error { // the transaction with COMMIT or ROLLBACK. The autocommit mode then // reverts to its previous state. e.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusInTrans, true) + // Call ctx.Txn() to active pending txn. + e.ctx.Txn() return nil } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 42dec103e670c..90280d4851757 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1883,7 +1883,7 @@ func (b *planBuilder) buildDataSource(tn *ast.TableName) (LogicalPlan, error) { // If this SQL is executed in a non-readonly transaction, we need a // "UnionScan" operator to read the modifications of former SQLs, which is // buffered in tidb-server memory. - if b.ctx.Txn() != nil && !b.ctx.Txn().IsReadOnly() { + if b.ctx.Txn(false).Valid() && !b.ctx.Txn(false).IsReadOnly() { us := LogicalUnionScan{}.init(b.ctx) us.SetChildren(ds) result = us diff --git a/session/session.go b/session/session.go index 76812690bb408..ca74bd47ee2d7 100644 --- a/session/session.go +++ b/session/session.go @@ -937,9 +937,20 @@ func (s *session) DropPreparedStmt(stmtID uint32) error { return nil } -func (s *session) Txn() kv.Transaction { - if !s.txn.Valid() { - return nil +func (s *session) Txn(opt ...bool) kv.Transaction { + if s.txn.pending() && len(opt) == 0 { + // Transaction is lazy intialized. + // PrepareTxnCtx is called to get a tso future, makes s.txn a pending txn, + // If Txn() is called later, wait for the future to get a valid txn. + txnCap := s.getMembufCap() + if err := s.txn.changePendingToValid(txnCap); err != nil { + s.txn.fail = errors.Trace(err) + } else { + s.sessionVars.TxnCtx.StartTS = s.txn.StartTS() + } + if !s.sessionVars.IsAutocommit() { + s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, true) + } } return &s.txn } @@ -1350,9 +1361,6 @@ func (s *session) PrepareTxnCtx(ctx context.Context) { SchemaVersion: is.SchemaMetaVersion(), CreateTime: time.Now(), } - if !s.sessionVars.IsAutocommit() { - s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, true) - } } // RefreshTxnCtx implements context.RefreshTxnCtx interface. diff --git a/session/session_test.go b/session/session_test.go index 3a7aa1890da0f..7be247bde8771 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -243,7 +243,7 @@ func (s *testSessionSuite) TestRowLock(c *C) { tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("create table t (c1 int, c2 int, c3 int)") tk.MustExec("insert t values (11, 2, 3)") tk.MustExec("insert t values (12, 2, 3)") @@ -301,6 +301,61 @@ func (s *testSessionSuite) TestAutocommit(c *C) { c.Assert(int(tk.Se.Status()&mysql.ServerStatusAutocommit), Equals, 0) tk.MustExec("set autocommit='On'") c.Assert(int(tk.Se.Status()&mysql.ServerStatusAutocommit), Greater, 0) + + // When autocommit is 0, transaction start ts should be the first *valid* + // statement, rather than *any* statement. + tk.MustExec("create table t (id int)") + tk.MustExec("set @@autocommit = 0") + tk.MustExec("rollback") + tk.MustExec("set @@autocommit = 0") + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.MustExec("insert into t select 1") + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + + // TODO: MySQL compatibility for setting global variable. + // tk.MustExec("begin") + // tk.MustExec("insert into t values (42)") + // tk.MustExec("set @@global.autocommit = 1") + // tk.MustExec("rollback") + // tk.MustQuery("select count(*) from t where id = 42").Check(testkit.Rows("0")) + // Even the transaction is rollbacked, the set statement succeed. + // tk.MustQuery("select @@global.autocommit").Rows("1") +} + +// TestTxnLazyInitialize tests that when autocommit = 0, not all statement starts +// a new transaction. +func (s *testSessionSuite) TestTxnLazyInitialize(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int)") + + tk.MustExec("set @@autocommit = 0") + c.Assert(tk.Se.Txn(false).Valid(), IsFalse) + tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) + tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) + + // Those statement should not start a new transaction automacally. + tk.MustQuery("select 1") + tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) + + tk.MustExec("set @@tidb_general_log = 0") + tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) + + tk.MustQuery("explain select * from t") + tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) + + // Begin statement should start a new transaction. + tk.MustExec("begin") + c.Assert(tk.Se.Txn(false).Valid(), IsTrue) + tk.MustExec("rollback") + + tk.MustExec("select * from t") + c.Assert(tk.Se.Txn(false).Valid(), IsTrue) + tk.MustExec("rollback") + + tk.MustExec("insert into t values (1)") + c.Assert(tk.Se.Txn(false).Valid(), IsTrue) + tk.MustExec("rollback") } func (s *testSessionSuite) TestGlobalVarAccessor(c *C) { @@ -428,7 +483,7 @@ func (s *testSessionSuite) TestRetryCleanTxn(c *C) { history.Add(0, stmt, tk.Se.GetSessionVars().StmtCtx) _, err = tk.Exec("commit") c.Assert(err, NotNil) - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) c.Assert(tk.Se.GetSessionVars().InTxn(), IsFalse) } @@ -507,11 +562,11 @@ func (s *testSessionSuite) TestInTrans(c *C) { tk.MustExec("insert t values ()") c.Assert(tk.Se.Txn().Valid(), IsTrue) tk.MustExec("drop table if exists t;") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("insert t values ()") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("commit") tk.MustExec("insert t values ()") @@ -521,11 +576,11 @@ func (s *testSessionSuite) TestInTrans(c *C) { tk.MustExec("insert t values ()") c.Assert(tk.Se.Txn().Valid(), IsTrue) tk.MustExec("commit") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("insert t values ()") c.Assert(tk.Se.Txn().Valid(), IsTrue) tk.MustExec("commit") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("set autocommit=1") tk.MustExec("drop table if exists t") @@ -535,7 +590,7 @@ func (s *testSessionSuite) TestInTrans(c *C) { tk.MustExec("insert t values ()") c.Assert(tk.Se.Txn().Valid(), IsTrue) tk.MustExec("rollback") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) } func (s *testSessionSuite) TestRetryPreparedStmt(c *C) { @@ -544,7 +599,7 @@ func (s *testSessionSuite) TestRetryPreparedStmt(c *C) { tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") - c.Assert(tk.Se.Txn(), IsNil) + c.Assert(tk.Se.Txn().Valid(), IsFalse) tk.MustExec("create table t (c1 int, c2 int, c3 int)") tk.MustExec("insert t values (11, 2, 3)") diff --git a/session/tidb.go b/session/tidb.go index a312c6bb6ffa0..9a8472cf83c9a 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -150,7 +150,7 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast if err == nil { GetHistory(sctx).Add(0, s, se.sessionVars.StmtCtx) } - if sctx.Txn() != nil { + if sctx.Txn(false).Valid() { if err != nil { sctx.StmtRollback() } else { @@ -177,6 +177,17 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast history.Count(), sctx.GetSessionVars().IsAutocommit()) } } + if se.txn.pending() { + // After run statement finish, txn state is still pending means the + // statement never need a Txn(), such as: + // + // set @@tidb_general_log = 1 + // set @@autocommit = 0 + // select 1 + // + // Reset txn state to invalid to dispose the pending start ts. + se.txn.changeToInvalid() + } return rs, errors.Trace(err) } diff --git a/session/txn.go b/session/txn.go index 6e391335fc966..300e5f6b47811 100644 --- a/session/txn.go +++ b/session/txn.go @@ -53,11 +53,15 @@ func (st *TxnState) init() { st.mutations = make(map[int64]*binlog.TableMutation) } -// Valid overrides Transaction interface. +// Valid implements the kv.Transaction interface. func (st *TxnState) Valid() bool { return st.Transaction != nil && st.Transaction.Valid() } +func (st *TxnState) pending() bool { + return st.Transaction == nil && st.txnFuture != nil +} + func (st *TxnState) validOrPending() bool { return st.txnFuture != nil || st.Valid() } diff --git a/sessionctx/context.go b/sessionctx/context.go index 4c8ee4cd43dc5..1daf9383e6eb3 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -34,7 +34,8 @@ type Context interface { NewTxn() error // Txn returns the current transaction which is created before executing a statement. - Txn() kv.Transaction + // The returned kv.Transaction is not nil, but maybe pending or invalid. + Txn(...bool) kv.Transaction // GetClient gets a kv.Client. GetClient() kv.Client diff --git a/table/tables/tables.go b/table/tables/tables.go index c24dd25d7f1d3..ff411bc98e14f 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1026,7 +1026,7 @@ func (ctx *ctxForPartitionExpr) NewTxn() error { } // Txn returns the current transaction which is created before executing a statement. -func (ctx *ctxForPartitionExpr) Txn() kv.Transaction { +func (ctx *ctxForPartitionExpr) Txn(...bool) kv.Transaction { panic("not support") } diff --git a/util/mock/context.go b/util/mock/context.go index 2eb7631e16769..0ddd2d02f4493 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -39,8 +39,8 @@ var _ sqlexec.SQLExecutor = (*Context)(nil) // Context represents mocked sessionctx.Context. type Context struct { values map[fmt.Stringer]interface{} - txn kv.Transaction // mock global variable - Store kv.Storage // mock global variable + txn wrapTxn // mock global variable + Store kv.Storage // mock global variable sessionVars *variable.SessionVars mux sync.Mutex // fix data race in ddl test. ctx context.Context @@ -49,6 +49,14 @@ type Context struct { pcache *kvcache.SimpleLRUCache } +type wrapTxn struct { + kv.Transaction +} + +func (txn *wrapTxn) Valid() bool { + return txn.Transaction != nil && txn.Transaction.Valid() +} + // Execute implements sqlexec.SQLExecutor Execute interface. func (c *Context) Execute(ctx context.Context, sql string) ([]ast.RecordSet, error) { return nil, errors.Errorf("Not Support.") @@ -81,8 +89,8 @@ func (c *Context) GetSessionVars() *variable.SessionVars { } // Txn implements sessionctx.Context Txn interface. -func (c *Context) Txn() kv.Transaction { - return c.txn +func (c *Context) Txn(...bool) kv.Transaction { + return &c.txn } // GetClient implements sessionctx.Context GetClient interface. @@ -122,7 +130,7 @@ func (c *Context) NewTxn() error { if c.Store == nil { return errors.New("store is not set") } - if c.txn != nil && c.txn.Valid() { + if c.txn.Valid() { err := c.txn.Commit(c.ctx) if err != nil { return errors.Trace(err) @@ -133,7 +141,7 @@ func (c *Context) NewTxn() error { if err != nil { return errors.Trace(err) } - c.txn = txn + c.txn.Transaction = txn return nil } @@ -144,7 +152,7 @@ func (c *Context) RefreshTxnCtx(ctx context.Context) error { // ActivePendingTxn implements the sessionctx.Context interface. func (c *Context) ActivePendingTxn() error { - if c.txn != nil { + if c.txn.Valid() { return nil } if c.Store != nil { @@ -152,14 +160,14 @@ func (c *Context) ActivePendingTxn() error { if err != nil { return errors.Trace(err) } - c.txn = txn + c.txn.Transaction = txn } return nil } // InitTxnWithStartTS implements the sessionctx.Context interface with startTS. func (c *Context) InitTxnWithStartTS(startTS uint64) error { - if c.txn != nil { + if c.txn.Valid() { return nil } if c.Store != nil { @@ -172,7 +180,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return errors.Trace(err) } txn.SetCap(membufCap) - c.txn = txn + c.txn.Transaction = txn } return nil } From b7f741a5662cadaae7bc7b3e2e9a1bfad4420abd Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 15 Nov 2018 13:14:49 +0800 Subject: [PATCH 079/509] variable: validate setting tx_isolation variable value (#8299) (#8314) --- domain/schema_validator_test.go | 10 +++++++--- executor/set_test.go | 31 +++++++++++++++++++++++++++++++ sessionctx/variable/session.go | 25 ++++++++++++++++++------- sessionctx/variable/varsutil.go | 7 +++++++ 4 files changed, 63 insertions(+), 10 deletions(-) diff --git a/domain/schema_validator_test.go b/domain/schema_validator_test.go index 2fc2ffd4955af..b6e04a8ece294 100644 --- a/domain/schema_validator_test.go +++ b/domain/schema_validator_test.go @@ -15,6 +15,7 @@ package domain import ( "math/rand" + "sync" "time" . "github.com/pingcap/check" @@ -34,7 +35,9 @@ func (*testSuite) TestSchemaValidator(c *C) { leaseGrantCh := make(chan leaseGrantItem) oracleCh := make(chan uint64) exit := make(chan struct{}) - go serverFunc(lease, leaseGrantCh, oracleCh, exit) + var wg sync.WaitGroup + wg.Add(1) + go serverFunc(lease, leaseGrantCh, oracleCh, exit, &wg) validator := NewSchemaValidator(lease).(*schemaValidator) @@ -95,7 +98,7 @@ func (*testSuite) TestSchemaValidator(c *C) { c.Assert(valid, Equals, ResultUnknown) close(exit) - time.Sleep(time.Millisecond) + wg.Wait() } func reload(validator SchemaValidator, leaseGrantCh chan leaseGrantItem, ids ...int64) int64 { @@ -107,7 +110,8 @@ func reload(validator SchemaValidator, leaseGrantCh chan leaseGrantItem, ids ... // serverFunc plays the role as a remote server, runs in a separate goroutine. // It can grant lease and provide timestamp oracle. // Caller should communicate with it through channel to mock network. -func serverFunc(lease time.Duration, requireLease chan leaseGrantItem, oracleCh chan uint64, exit chan struct{}) { +func serverFunc(lease time.Duration, requireLease chan leaseGrantItem, oracleCh chan uint64, exit chan struct{}, wg *sync.WaitGroup) { + defer wg.Done() var version int64 leaseTS := uint64(time.Now().UnixNano()) ticker := time.NewTicker(lease) diff --git a/executor/set_test.go b/executor/set_test.go index dd9b2955e324d..a9ecf17a286f1 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -510,4 +510,35 @@ func (s *testSuite) TestValidateSetVar(c *C) { tk.MustExec("set @@innodb_lock_wait_timeout = 1073741825") tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect innodb_lock_wait_timeout value: '1073741825'")) + + _, err = tk.Exec("set @@tx_isolation=''") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) + + _, err = tk.Exec("set global tx_isolation=''") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) + + _, err = tk.Exec("set @@transaction_isolation=''") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) + + _, err = tk.Exec("set global transaction_isolation=''") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) + + _, err = tk.Exec("set global tx_isolation='REPEATABLE-READ1'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) + + tk.MustExec("set @@tx_isolation='READ-COMMITTED'") + result = tk.MustQuery("select @@tx_isolation;") + result.Check(testkit.Rows("READ-COMMITTED")) + + tk.MustExec("set @@tx_isolation='read-COMMITTED'") + result = tk.MustQuery("select @@tx_isolation;") + result.Check(testkit.Rows("READ-COMMITTED")) + + tk.MustExec("set @@tx_isolation='REPEATABLE-READ'") + result = tk.MustQuery("select @@tx_isolation;") + result.Check(testkit.Rows("REPEATABLE-READ")) + + tk.MustExec("set @@tx_isolation='SERIALIZABLE'") + result = tk.MustQuery("select @@tx_isolation;") + result.Check(testkit.Rows("SERIALIZABLE")) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index ed7f136e7293e..385c7f1643418 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -619,13 +619,24 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { // special session variables. const ( - SQLModeVar = "sql_mode" - AutocommitVar = "autocommit" - CharacterSetResults = "character_set_results" - MaxAllowedPacket = "max_allowed_packet" - TimeZone = "time_zone" - TxnIsolation = "tx_isolation" - TxnIsolationOneShot = "tx_isolation_one_shot" + SQLModeVar = "sql_mode" + AutocommitVar = "autocommit" + CharacterSetResults = "character_set_results" + MaxAllowedPacket = "max_allowed_packet" + TimeZone = "time_zone" + TxnIsolation = "tx_isolation" + TransactionIsolation = "transaction_isolation" + TxnIsolationOneShot = "tx_isolation_one_shot" +) + +var ( + // TxIsolationNames are the valid values of the variable "tx_isolation" or "transaction_isolation". + TxIsolationNames = map[string]struct{}{ + "READ-UNCOMMITTED": {}, + "READ-COMMITTED": {}, + "REPEATABLE-READ": {}, + "SERIALIZABLE": {}, + } ) // TableDelta stands for the changed count for one table. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index c13effeebd902..215e82a5b142d 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -343,6 +343,13 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return "", errors.Trace(err) } return v, nil + case TxnIsolation, TransactionIsolation: + upVal := strings.ToUpper(value) + _, exists := TxIsolationNames[upVal] + if !exists { + return "", ErrWrongValueForVar.GenWithStackByArgs(name, value) + } + return upVal, nil } return value, nil } From 4ecd87a10c08859e43c23dcd2db46b75256f6af5 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 15 Nov 2018 19:57:06 +0800 Subject: [PATCH 080/509] util/execdetail: fix race 2.1 (#8331) --- executor/explain.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/executor/explain.go b/executor/explain.go index 019e07a655744..8fb9fb1e05db9 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -40,9 +40,6 @@ func (e *ExplainExec) Open(ctx context.Context) error { // Close implements the Executor Close interface. func (e *ExplainExec) Close() error { - if e.analyzeExec != nil { - e.analyzeExec.Close() - } e.rows = nil return nil } @@ -84,6 +81,7 @@ func (e *ExplainExec) generateExplainInfo(ctx context.Context) ([][]string, erro break } } + e.analyzeExec.Close() } e.explain.RenderResult() if e.analyzeExec != nil { From 9eb47a83147bdba3431f1d408f155e1e8613ef3a Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 15 Nov 2018 20:49:25 +0800 Subject: [PATCH 081/509] executor, cmd: refine HashJoinExec for specific JoinType if outer/inner is nil (#8296) (#8315) --- .gitignore | 1 + executor/join.go | 7 +++-- executor/join_test.go | 59 +++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 65 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index e01efc8b13a7a..ae407994cdc6b 100644 --- a/.gitignore +++ b/.gitignore @@ -15,5 +15,6 @@ y.output profile.coverprofile explain_test cmd/explaintest/explain-test.out +cmd/explaintest/explaintest_tidb-server _tools/ *.fail.go diff --git a/executor/join.go b/executor/join.go index 205127e64ee55..0ea7bd0a39271 100644 --- a/executor/join.go +++ b/executor/join.go @@ -211,8 +211,11 @@ func (e *HashJoinExec) fetchOuterChunks(ctx context.Context) { } return } - if !hasWaitedForInner { + if outerResult.NumRows() == 0 { + e.finished.Store(true) + return + } jobFinished, innerErr := e.wait4Inner() if innerErr != nil { e.joinResultCh <- &hashjoinWorkerResult{ @@ -241,7 +244,7 @@ func (e *HashJoinExec) wait4Inner() (finished bool, err error) { return false, errors.Trace(err) } } - if e.hashTable.Len() == 0 && e.joinType == plannercore.InnerJoin { + if e.hashTable.Len() == 0 && (e.joinType == plannercore.InnerJoin || e.joinType == plannercore.SemiJoin) { return true, nil } return false, nil diff --git a/executor/join_test.go b/executor/join_test.go index 681687d9c1373..b1e79dd88c1e5 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -921,3 +921,62 @@ func (s *testSuite) TestEmbeddedOuterJoin(c *C) { tk.MustQuery("select * from (t1 left join t2 on t1.a = t2.a) left join (t2 t3 left join t2 t4 on t3.a = t4.a) on t2.b = 1"). Check(testkit.Rows("1 1 ")) } + +func (s *testSuite) TestHashJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int);") + tk.MustExec("create table t2(a int, b int);") + tk.MustExec("insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5);") + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("5")) + tk.MustQuery("select count(*) from t2").Check(testkit.Rows("0")) + tk.MustExec("set @@tidb_max_chunk_size=1;") + result := tk.MustQuery("explain analyze select /*+ TIDB_HJ(t1, t2) */ * from t1 where exists (select a from t2 where t1.a = t2.a);") + // id count task operator info execution info + // HashLeftJoin_9 8000.00 root semi join, inner:TableReader_13, equal:[eq(test.t1.a, test.t2.a)] time:1.036712ms, loops:1, rows:0 + // ├─TableReader_11 10000.00 root data:TableScan_10 time:441.096µs, loops:1, rows:1 + // │ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo + // └─TableReader_13 10000.00 root data:TableScan_12 time:212.376µs, loops:1, rows:0 + // └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo + row := result.Rows() + c.Assert(len(row), Equals, 5) + outerExecInfo := row[1][4].(string) + c.Assert(outerExecInfo[len(outerExecInfo)-1:], Equals, "1") + innerExecInfo := row[3][4].(string) + c.Assert(innerExecInfo[len(innerExecInfo)-1:], Equals, "0") + + tk.MustExec("insert into t2 select * from t1;") + tk.MustExec("delete from t1;") + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("0")) + tk.MustQuery("select count(*) from t2").Check(testkit.Rows("5")) + result = tk.MustQuery("explain analyze select /*+ TIDB_HJ(t1, t2) */ * from t1 where not exists (select a from t2 where t1.a = t2.a);") + // id count task operator info execution info | + // Projection_8 4.00 root test.t1.a, test.t1.b time:193.08µs, loops:1, rows:0 | + // └─Selection_9 4.00 root not(6_aux_0) time:146.95µs, loops:1, rows:0 | + // └─HashLeftJoin_10 5.00 root left outer semi join, inner:TableReader_14, equal:[eq(test.t1.a, test.t2.a)] time:144.293µs, loops:1, rows:0 | + // ├─TableReader_12 5.00 root data:TableScan_11 time:26.27µs, loops:1, rows:0 | + // │ └─TableScan_11 5.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo | + // └─TableReader_14 5.00 root data:TableScan_13 time:0s, loops:0, rows:0 | + // └─TableScan_13 5.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo | + row = result.Rows() + c.Assert(len(row), Equals, 7) + outerExecInfo = row[3][4].(string) + c.Assert(outerExecInfo[len(outerExecInfo)-1:], Equals, "0") + innerExecInfo = row[5][4].(string) + c.Assert(innerExecInfo[len(innerExecInfo)-1:], LessEqual, "5") + + result = tk.MustQuery("explain analyze select /*+ TIDB_HJ(t1, t2) */ * from t1 left outer join t2 on t1.a = t2.a;") + // id count task operator info execution info + // HashLeftJoin_6 12500.00 root left outer join, inner:TableReader_10, equal:[eq(test.t1.a, test.t2.a)] time:502.553µs, loops:1, rows:0 + // ├─TableReader_8 10000.00 root data:TableScan_7 time:27.302µs, loops:1, rows:0 + // │ └─TableScan_7 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo + // └─TableReader_10 10000.00 root data:TableScan_9 time:0s, loops:0, rows:0 + // └─TableScan_9 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo + row = result.Rows() + c.Assert(len(row), Equals, 5) + outerExecInfo = row[1][4].(string) + c.Assert(outerExecInfo[len(outerExecInfo)-1:], Equals, "0") + innerExecInfo = row[3][4].(string) + c.Assert(innerExecInfo[len(innerExecInfo)-1:], LessEqual, "5") +} From c9a09438a1e8ffbefd49dc72e7760c1b741121e8 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 16 Nov 2018 11:59:20 +0800 Subject: [PATCH 082/509] session: add sql types for commit and retry logs (#8281) (#8326) --- session/session.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/session/session.go b/session/session.go index ca74bd47ee2d7..21aa1f722eb25 100644 --- a/session/session.go +++ b/session/session.go @@ -337,7 +337,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { // BatchInsert already commit the first batch 1000 rows, then it commit 1000-2000 and retry the statement, // Finally t1 will have more data than t2, with no errors return to user! if s.isRetryableError(err) && !s.sessionVars.BatchInsert && commitRetryLimit > 0 { - log.Warnf("con:%d retryable error: %v, txn: %v", s.sessionVars.ConnectionID, err, s.txn) + log.Warnf("[%s] con:%d retryable error: %v, txn: %v", s.getSQLLabel(), s.sessionVars.ConnectionID, err, s.txn) // Transactions will retry 2 ~ commitRetryLimit times. // We make larger transactions retry less times to prevent cluster resource outage. txnSizeRate := float64(txnSize) / float64(kv.TxnTotalSizeLimit) @@ -473,6 +473,7 @@ func (s *session) retry(ctx context.Context, maxCnt uint) error { var err error var schemaVersion int64 orgStartTS := s.GetSessionVars().TxnCtx.StartTS + label := s.getSQLLabel() for { s.PrepareTxnCtx(ctx) s.sessionVars.RetryInfo.ResetOffset() @@ -515,17 +516,17 @@ func (s *session) retry(ctx context.Context, maxCnt uint) error { } } if !s.isRetryableError(err) { - log.Warnf("con:%d session:%v, err:%v in retry", connID, s, err) - metrics.SessionRetryErrorCounter.WithLabelValues(s.getSQLLabel(), metrics.LblUnretryable) + log.Warnf("[%s] con:%d session:%v, err:%v in retry", label, connID, s, err) + metrics.SessionRetryErrorCounter.WithLabelValues(label, metrics.LblUnretryable) return errors.Trace(err) } retryCnt++ if retryCnt >= maxCnt { - log.Warnf("con:%d Retry reached max count %d", connID, retryCnt) - metrics.SessionRetryErrorCounter.WithLabelValues(s.getSQLLabel(), metrics.LblReachMax) + log.Warnf("[%s] con:%d Retry reached max count %d", label, connID, retryCnt) + metrics.SessionRetryErrorCounter.WithLabelValues(label, metrics.LblReachMax) return errors.Trace(err) } - log.Warnf("con:%d retryable error: %v, txn: %v", connID, err, s.txn) + log.Warnf("[%s] con:%d retryable error: %v, txn: %v", label, connID, err, s.txn) kv.BackOff(retryCnt) s.txn.changeToInvalid() s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, false) From 65be0df2d9675e3f76833da82e1d0823e05d4408 Mon Sep 17 00:00:00 2001 From: lysu Date: Fri, 16 Nov 2018 20:08:08 +0800 Subject: [PATCH 083/509] executor: fix union dual table order by but not sort (#8335) --- cmd/explaintest/r/explain_easy.result | 21 ++++++++++++++++ cmd/explaintest/t/explain_easy.test | 3 +++ executor/executor_test.go | 5 ++++ parser/parser.y | 36 +++++++++++++++------------ parser/parser_test.go | 33 ++++++++++++++++++++++++ 5 files changed, 82 insertions(+), 16 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 29e6fa54c2455..7b19308be756b 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -379,3 +379,24 @@ Projection_5 8000.00 root test.ta.a └─TableReader_9 10000.00 root data:TableScan_8 └─TableScan_8 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo rollback; +explain SELECT 0 AS a FROM dual UNION SELECT 1 AS a FROM dual ORDER BY a; +id count task operator info +Sort_13 2.00 root a:asc +└─HashAgg_17 2.00 root group by:a, funcs:firstrow(join_agg_0) + └─Union_18 2.00 root + ├─HashAgg_21 1.00 root group by:a, funcs:firstrow(a), firstrow(a) + │ └─Projection_22 1.00 root 0 + │ └─TableDual_23 1.00 root rows:1 + └─HashAgg_26 1.00 root group by:a, funcs:firstrow(a), firstrow(a) + └─Projection_27 1.00 root 1 + └─TableDual_28 1.00 root rows:1 +explain SELECT 0 AS a FROM dual UNION (SELECT 1 AS a FROM dual ORDER BY a) +id count task operator info +HashAgg_15 2.00 root group by:a, funcs:firstrow(join_agg_0) +└─Union_16 2.00 root + ├─HashAgg_19 1.00 root group by:a, funcs:firstrow(a), firstrow(a) + │ └─Projection_20 1.00 root 0 + │ └─TableDual_21 1.00 root rows:1 + └─StreamAgg_26 1.00 root group by:a, funcs:firstrow(a), firstrow(a) + └─Projection_31 1.00 root 1 + └─TableDual_32 1.00 root rows:1 diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index 391e0039ad5d4..eb7d36e0a7198 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -80,3 +80,6 @@ begin; insert tb values ('1'); explain select * from ta where a = 1; rollback; + +explain SELECT 0 AS a FROM dual UNION SELECT 1 AS a FROM dual ORDER BY a; +explain SELECT 0 AS a FROM dual UNION (SELECT 1 AS a FROM dual ORDER BY a) diff --git a/executor/executor_test.go b/executor/executor_test.go index 4c3b01bcc9ca1..7953f1123f295 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1060,6 +1060,11 @@ func (s *testSuite) TestUnion(c *C) { tk.MustExec("CREATE TABLE t2 (a int not null, b char (10) not null)") tk.MustExec("insert into t2 values(1,'a'),(2,'b'),(3,'c'),(3,'c')") tk.MustQuery("select a from t1 union select a from t1 order by (select a+1);").Check(testkit.Rows("1", "2", "3")) + + // #issue 8201 + for i := 0; i < 4; i++ { + tk.MustQuery("SELECT(SELECT 0 AS a FROM dual UNION SELECT 1 AS a FROM dual ORDER BY a ASC LIMIT 1) AS dev").Check(testkit.Rows("0")) + } } func (s *testSuite) TestNeighbouringProj(c *C) { diff --git a/parser/parser.y b/parser/parser.y index 6326d2cb30028..440e81ad404f3 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -719,7 +719,7 @@ import ( SelectStmtLimit "SELECT statement optional LIMIT clause" SelectStmtOpts "Select statement options" SelectStmtBasic "SELECT statement from constant value" - SelectStmtFromDual "SELECT statement from dual" + SelectStmtFromDualTable "SELECT statement from dual table" SelectStmtFromTable "SELECT statement from table" SelectStmtGroup "SELECT statement optional GROUP BY clause" ShowTargetFilterable "Show target that can be filtered by WHERE or LIKE" @@ -4240,22 +4240,18 @@ SelectStmtBasic: $$ = st } -SelectStmtFromDual: - SelectStmtBasic FromDual WhereClauseOptional OrderByOptional +SelectStmtFromDualTable: + SelectStmtBasic FromDual WhereClauseOptional { st := $1.(*ast.SelectStmt) lastField := st.Fields.Fields[len(st.Fields.Fields)-1] if lastField.Expr != nil && lastField.AsName.O == "" { - lastEnd := yyS[yypt-2].offset-1 + lastEnd := yyS[yypt-1].offset-1 lastField.SetText(parser.src[lastField.Offset:lastEnd]) } if $3 != nil { st.Where = $3.(ast.ExprNode) } - - if $4 != nil { - st.OrderBy = $4.(*ast.OrderByClause) - } } @@ -4316,13 +4312,16 @@ SelectStmt: } $$ = st } -| SelectStmtFromDual SelectStmtLimit SelectLockOpt +| SelectStmtFromDualTable OrderByOptional SelectStmtLimit SelectLockOpt { st := $1.(*ast.SelectStmt) - st.LockTp = $3.(ast.SelectLockType) if $2 != nil { - st.Limit = $2.(*ast.Limit) + st.OrderBy = $2.(*ast.OrderByClause) } + if $3 != nil { + st.Limit = $3.(*ast.Limit) + } + st.LockTp = $4.(ast.SelectLockType) $$ = st } | SelectStmtFromTable OrderByOptional SelectStmtLimit SelectLockOpt @@ -4784,19 +4783,24 @@ UnionStmt: } $$ = union } -| UnionClauseList "UNION" UnionOpt SelectStmtFromDual SelectStmtLimit SelectLockOpt +| UnionClauseList "UNION" UnionOpt SelectStmtFromDualTable OrderByOptional + SelectStmtLimit SelectLockOpt { st := $4.(*ast.SelectStmt) union := $1.(*ast.UnionStmt) st.IsAfterUnionDistinct = $3.(bool) lastSelect := union.SelectList.Selects[len(union.SelectList.Selects)-1] - endOffset := parser.endOffset(&yyS[yypt-4]) + endOffset := parser.endOffset(&yyS[yypt-5]) parser.setLastSelectFieldText(lastSelect, endOffset) union.SelectList.Selects = append(union.SelectList.Selects, st) if $5 != nil { - union.Limit = $5.(*ast.Limit) - } else { - st.LockTp = $6.(ast.SelectLockType) + union.OrderBy = $5.(*ast.OrderByClause) + } + if $6 != nil { + union.Limit = $6.(*ast.Limit) + } + if $5 == nil && $6 == nil { + st.LockTp = $7.(ast.SelectLockType) } $$ = union } diff --git a/parser/parser_test.go b/parser/parser_test.go index af7a8ba4f2bbc..ec819f97c09dc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2024,10 +2024,43 @@ func (s *testParserSuite) TestUnion(c *C) { {"select * from (select 1 union select 2) as a", true}, {"insert into t select c1 from t1 union select c2 from t2", true}, {"insert into t (c) select c1 from t1 union select c2 from t2", true}, + {"select 2 as a from dual union select 1 as b from dual order by a", true}, } s.RunTest(c, table) } +func (s *testParserSuite) TestUnionOrderBy(c *C) { + parser := New() + tests := []struct { + src string + hasOrderBy []bool + }{ + {"select 2 as a from dual union select 1 as b from dual order by a", []bool{false, false, true}}, + {"select 2 as a from dual union (select 1 as b from dual order by a)", []bool{false, true, false}}, + {"(select 2 as a from dual order by a) union select 1 as b from dual order by a", []bool{true, false, true}}, + {"select 1 a, 2 b from dual order by a", []bool{true}}, + {"select 1 a, 2 b from dual", []bool{false}}, + } + + for _, t := range tests { + stmt, err := parser.Parse(t.src, "", "") + c.Assert(err, IsNil) + us, ok := stmt[0].(*ast.UnionStmt) + if ok { + var i int + for _, s := range us.SelectList.Selects { + c.Assert(s.OrderBy != nil, Equals, t.hasOrderBy[i]) + i++ + } + c.Assert(us.OrderBy != nil, Equals, t.hasOrderBy[i]) + } + ss, ok := stmt[0].(*ast.SelectStmt) + if ok { + c.Assert(ss.OrderBy != nil, Equals, t.hasOrderBy[0]) + } + } +} + func (s *testParserSuite) TestLikeEscape(c *C) { defer testleak.AfterTest(c)() table := []testCase{ From 30241e040fca0bf3e8f1b78e5c1c281514ce125b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 21 Nov 2018 11:48:50 +0800 Subject: [PATCH 084/509] session: fix bootstrap error in ANSI sql_mode (#8353) (#8367) --- session/bootstrap_test.go | 27 +++++++++++++++++++++++++++ session/session.go | 2 +- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index cff66c5b1b9bb..49787fa5df0ab 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -259,6 +259,33 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { c.Assert(ver, Equals, int64(currentBootstrapVersion)) } +func (s *testBootstrapSuite) TestANSISQLMode(c *C) { + defer testleak.AfterTest(c)() + store, dom := newStoreWithBootstrap(c, s.dbName) + defer store.Close() + se := newSession(c, store, s.dbName) + mustExecSQL(c, se, "USE mysql;") + mustExecSQL(c, se, `set @@global.sql_mode="NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,ANSI"`) + mustExecSQL(c, se, `delete from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) + delete(storeBootstrapped, store.UUID()) + se.Close() + + // Do some clean up, BootstrapSession will not create a new domain otherwise. + dom.Close() + domap.Delete(store) + + // Set ANSI sql_mode and bootstrap again, to cover a bugfix. + // Once we have a SQL like that: + // select variable_value from mysql.tidb where variable_name = "system_tz" + // it fails to execute in the ANSI sql_mode, and makes TiDB cluster fail to bootstrap. + dom1, err := BootstrapSession(store) + c.Assert(err, IsNil) + defer dom1.Close() + se = newSession(c, store, s.dbName) + mustExecSQL(c, se, "select @@global.sql_mode") + se.Close() +} + func (s *testBootstrapSuite) TestOldPasswordUpgrade(c *C) { pwd := "abc" oldpwd := fmt.Sprintf("%X", auth.Sha1Hash([]byte(pwd))) diff --git a/session/session.go b/session/session.go index 21aa1f722eb25..744ce86e3b3d5 100644 --- a/session/session.go +++ b/session/session.go @@ -1098,7 +1098,7 @@ func CreateSession(store kv.Storage) (Session, error) { // loadSystemTZ loads systemTZ from mysql.tidb func loadSystemTZ(se *session) (string, error) { - sql := `select variable_value from mysql.tidb where variable_name = "system_tz"` + sql := `select variable_value from mysql.tidb where variable_name = 'system_tz'` rss, errLoad := se.Execute(context.Background(), sql) if errLoad != nil { return "", errLoad From 7aa0f64f174ebf0fa9c27df2771e0bbc5dfb9c99 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Thu, 22 Nov 2018 14:36:02 +0800 Subject: [PATCH 085/509] server: unix socket should verify user's authentication (#8381) (#8398) --- server/conn.go | 16 +++++++++------- server/server.go | 2 +- server/server_test.go | 3 +++ session/session.go | 7 +++++-- sessionctx/variable/tidb_vars.go | 1 + 5 files changed, 19 insertions(+), 10 deletions(-) diff --git a/server/conn.go b/server/conn.go index df2fa182ebe93..d2b49011ae6b7 100644 --- a/server/conn.go +++ b/server/conn.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/arena" "github.com/pingcap/tidb/util/auth" @@ -391,16 +392,17 @@ func (cc *clientConn) openSessionAndDoAuth(authData []byte) error { if err != nil { return errors.Trace(err) } - if !cc.server.skipAuth() { - // Do Auth. + host := variable.DefHostname + if !cc.server.isUnixSocket() { addr := cc.bufReadConn.RemoteAddr().String() - host, _, err1 := net.SplitHostPort(addr) - if err1 != nil { + // Do Auth. + host, _, err = net.SplitHostPort(addr) + if err != nil { return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, addr, "YES")) } - if !cc.ctx.Auth(&auth.UserIdentity{Username: cc.user, Hostname: host}, authData, cc.salt) { - return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, host, "YES")) - } + } + if !cc.ctx.Auth(&auth.UserIdentity{Username: cc.user, Hostname: host}, authData, cc.salt) { + return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, host, "YES")) } if cc.dbname != "" { err = cc.useDB(context.Background(), cc.dbname) diff --git a/server/server.go b/server/server.go index 9b8e0ecd10d48..add925488b65b 100644 --- a/server/server.go +++ b/server/server.go @@ -129,7 +129,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn { return cc } -func (s *Server) skipAuth() bool { +func (s *Server) isUnixSocket() bool { return s.cfg.Socket != "" } diff --git a/server/server_test.go b/server/server_test.go index f8544cae430cc..ace52e763d7cb 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -161,6 +161,9 @@ func (dbt *DBTest) mustQueryRows(query string, args ...interface{}) { func runTestRegression(c *C, overrider configOverrider, dbName string) { runTestsOnNewDB(c, overrider, dbName, func(dbt *DBTest) { + // Show the user + dbt.mustExec("select user()") + // Create Table dbt.mustExec("CREATE TABLE test (val TINYINT)") diff --git a/session/session.go b/session/session.go index 744ce86e3b3d5..79e5c4a67d949 100644 --- a/session/session.go +++ b/session/session.go @@ -1023,10 +1023,13 @@ func (s *session) GetSessionVars() *variable.SessionVars { func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []byte) bool { pm := privilege.GetPrivilegeManager(s) - // Check IP. + // Check IP or localhost. if pm.ConnectionVerification(user.Username, user.Hostname, authentication, salt) { s.sessionVars.User = user return true + } else if user.Hostname == variable.DefHostname { + log.Errorf("User connection verification failed %s", user) + return false } // Check Hostname. @@ -1046,7 +1049,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []by func getHostByIP(ip string) []string { if ip == "127.0.0.1" { - return []string{"localhost"} + return []string{variable.DefHostname} } addrs, err := net.LookupAddr(ip) terror.Log(errors.Trace(err)) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 68b31d018c3a0..85d9b7f44bc6f 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -204,6 +204,7 @@ const ( // Default TiDB system variable values. const ( + DefHostname = "localhost" DefIndexLookupConcurrency = 4 DefIndexLookupJoinConcurrency = 4 DefIndexSerialScanConcurrency = 1 From 6b2b801e074b2a8893d8fb11813aca3740c4cf62 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Thu, 22 Nov 2018 18:12:13 +0800 Subject: [PATCH 086/509] server,types: convert zero input correctly for column type `year` (#8397) --- executor/insert_test.go | 14 ++++++++++++++ server/util.go | 11 ++++++++++- server/util_test.go | 6 ++++++ types/datum.go | 10 ++++++---- types/time.go | 5 ++++- types/time_test.go | 25 ++++++++++++++++++++++++- 6 files changed, 64 insertions(+), 7 deletions(-) diff --git a/executor/insert_test.go b/executor/insert_test.go index 38bd7b2ad1a1b..0456096d9eb4b 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -160,3 +160,17 @@ func (s *testSuite) TestInsertDateTimeWithTimeZone(c *C) { `1 1970-01-01 09:20:34`, )) } + +func (s *testSuite) TestInsertZeroYear(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists t1;`) + tk.MustExec(`create table t1(a year(4));`) + tk.MustExec(`insert into t1 values(0000),(00),("0000"),("00");`) + tk.MustQuery(`select * from t1;`).Check(testkit.Rows( + `0`, + `0`, + `2000`, + `2000`, + )) +} diff --git a/server/util.go b/server/util.go index 352fc9e75c12e..463967b0aac19 100644 --- a/server/util.go +++ b/server/util.go @@ -285,9 +285,18 @@ func dumpTextRow(buffer []byte, columns []*ColumnInfo, row chunk.Row) ([]byte, e continue } switch col.Type { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeYear, mysql.TypeInt24, mysql.TypeLong: + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong: tmp = strconv.AppendInt(tmp[:0], row.GetInt64(i), 10) buffer = dumpLengthEncodedString(buffer, tmp) + case mysql.TypeYear: + year := row.GetInt64(i) + tmp = tmp[:0] + if year == 0 { + tmp = append(tmp, '0', '0', '0', '0') + } else { + tmp = strconv.AppendInt(tmp, year, 10) + } + buffer = dumpLengthEncodedString(buffer, tmp) case mysql.TypeLonglong: if mysql.HasUnsignedFlag(uint(columns[i].Flag)) { tmp = strconv.AppendUint(tmp[:0], row.GetUint64(i), 10) diff --git a/server/util_test.go b/server/util_test.go index 2ddd55fe70e66..46116cab842b1 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -153,6 +153,12 @@ func (s *testUtilSuite) TestDumpTextValue(c *C) { bs, err = dumpTextRow(nil, columns, chunk.MutRowFromDatums([]types.Datum{d}).ToRow()) c.Assert(err, IsNil) c.Assert(mustDecodeStr(c, bs), Equals, "1.23") + + year := types.NewIntDatum(0) + columns[0].Type = mysql.TypeYear + bs, err = dumpTextRow(nil, columns, chunk.MutRowFromDatums([]types.Datum{year}).ToRow()) + c.Assert(err, IsNil) + c.Assert(mustDecodeStr(c, bs), Equals, "0000") } func mustDecodeStr(c *C, b []byte) string { diff --git a/types/datum.go b/types/datum.go index d670520ae06dd..bf0f03878deff 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1162,9 +1162,10 @@ func ProduceDecWithSpecifiedTp(dec *MyDecimal, tp *FieldType, sc *stmtctx.Statem func (d *Datum) convertToMysqlYear(sc *stmtctx.StatementContext, target *FieldType) (Datum, error) { var ( - ret Datum - y int64 - err error + ret Datum + y int64 + err error + fromStr bool ) switch d.k { case KindString, KindBytes: @@ -1172,6 +1173,7 @@ func (d *Datum) convertToMysqlYear(sc *stmtctx.StatementContext, target *FieldTy if err != nil { return ret, errors.Trace(err) } + fromStr = true case KindMysqlTime: y = int64(d.GetMysqlTime().Time.Year()) case KindMysqlDuration: @@ -1183,7 +1185,7 @@ func (d *Datum) convertToMysqlYear(sc *stmtctx.StatementContext, target *FieldTy } y = ret.GetInt64() } - y, err = AdjustYear(y) + y, err = AdjustYear(y, fromStr) if err != nil { return invalidConv(d, target.Tp) } diff --git a/types/time.go b/types/time.go index 6c5a77f5bd479..b5a027657f198 100644 --- a/types/time.go +++ b/types/time.go @@ -777,7 +777,10 @@ func adjustYear(y int) int { } // AdjustYear is used for adjusting year and checking its validation. -func AdjustYear(y int64) (int64, error) { +func AdjustYear(y int64, fromStr bool) (int64, error) { + if y == 0 && !fromStr { + return y, nil + } y = int64(adjustYear(int(y))) if y < int64(MinYear) || y > int64(MaxYear) { return 0, errors.Trace(ErrInvalidYear) diff --git a/types/time_test.go b/types/time_test.go index 9969ffa1c60bf..d0e0cb25eac76 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -382,7 +382,7 @@ func (s *testTimeSuite) TestYear(c *C) { } for _, test := range valids { - _, err := types.AdjustYear(test.Year) + _, err := types.AdjustYear(test.Year, false) if test.Expect { c.Assert(err, IsNil) } else { @@ -390,6 +390,29 @@ func (s *testTimeSuite) TestYear(c *C) { } } + strYears := []struct { + Year int64 + Expect int64 + }{ + {0, 2000}, + } + for _, test := range strYears { + res, err := types.AdjustYear(test.Year, true) + c.Assert(err, IsNil) + c.Assert(res, Equals, test.Expect) + } + + numYears := []struct { + Year int64 + Expect int64 + }{ + {0, 0}, + } + for _, test := range numYears { + res, err := types.AdjustYear(test.Year, false) + c.Assert(err, IsNil) + c.Assert(res, Equals, test.Expect) + } } func (s *testTimeSuite) getLocation(c *C) *time.Location { From df466a8d37a43291f2f4e2fc0c68a0b7ad0f2725 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 22 Nov 2018 18:27:14 +0800 Subject: [PATCH 087/509] ast, plan: return error when the arg of VALUES() function is not a column (#7817) (#8403) --- ast/expressions.go | 4 +++- planner/core/logical_plan_builder.go | 4 ++++ planner/core/logical_plan_test.go | 1 + 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/ast/expressions.go b/ast/expressions.go index 440a89e06fdd6..e359b121c7444 100644 --- a/ast/expressions.go +++ b/ast/expressions.go @@ -927,7 +927,9 @@ func (n *ValuesExpr) Accept(v Visitor) (Node, bool) { if !ok { return n, false } - n.Column = node.(*ColumnNameExpr) + // `node` may be *ast.ValueExpr, to avoid panic, we write `ok` but do not use + // it. + n.Column, ok = node.(*ColumnNameExpr) return v.Leave(n) } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 90280d4851757..cc2d08a28d575 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1181,6 +1181,10 @@ func (g *gbyResolver) Leave(inNode ast.Node) (ast.Node, bool) { return inNode, false } return ret, true + case *ast.ValuesExpr: + if v.Column == nil { + g.err = ErrUnknownColumn.GenWithStackByArgs("", "VALUES() function") + } } return inNode, true } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 91d093b7fa79b..88a8aedb41057 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1995,6 +1995,7 @@ func (s *testPlanSuite) TestNameResolver(c *C) { {"select a from t group by t11.c1", "[planner:1054]Unknown column 't11.c1' in 'group statement'"}, {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, + {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, } From d39d3827d47f4d7f79aa90a7fc98eb0718d62c78 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 22 Nov 2018 19:17:48 +0800 Subject: [PATCH 088/509] executor: fix hash join sqllogic test block, data race. (#8392) --- executor/join.go | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/executor/join.go b/executor/join.go index 0ea7bd0a39271..60f9587edc4f9 100644 --- a/executor/join.go +++ b/executor/join.go @@ -262,6 +262,8 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context, chkCh chan<- *chunk.C select { case <-doneCh: return + case <-e.closeCh: + return default: if e.finished.Load().(bool) { return @@ -275,7 +277,12 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context, chkCh chan<- *chunk.C if chk.NumRows() == 0 { return } - chkCh <- chk + select { + case chkCh <- chk: + break + case <-e.closeCh: + return + } e.innerResult.Add(chk) } } @@ -527,19 +534,14 @@ func (e *HashJoinExec) fetchInnerAndBuildHashTable(ctx context.Context) { doneCh := make(chan struct{}) go util.WithRecovery(func() { e.fetchInnerRows(ctx, innerResultCh, doneCh) }, nil) - if e.finished.Load().(bool) { - return - } // TODO: Parallel build hash table. Currently not support because `mvmap` is not thread-safe. err := e.buildHashTableForList(innerResultCh) if err != nil { e.innerFinished <- errors.Trace(err) close(doneCh) - // fetchInnerRows may be blocked by this channel, so read from the channel to unblock it. - select { - case <-innerResultCh: - default: - } + } + // wait fetchInnerRows be finished. + for range innerResultCh { } } From fc4c93fcfba0be39f0c4370df23d205ee1f36ce6 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 22 Nov 2018 22:04:44 +0800 Subject: [PATCH 089/509] *: preparation for Go1.11 module cherry pick (#8388) --- Gopkg.lock | 243 +- Gopkg.toml | 8 +- Makefile | 45 +- ast/ast.go | 61 +- ast/base.go | 14 +- ast/ddl.go | 4 +- ast/ddl_test.go | 2 +- ast/dml.go | 6 +- ast/dml_test.go | 2 +- ast/expressions.go | 114 +- ast/expressions_test.go | 7 +- ast/flag.go | 6 +- ast/flag_test.go | 4 +- ast/format_test.go | 4 +- ast/functions.go | 8 +- ast/functions_test.go | 9 +- ast/misc.go | 19 +- ast/misc_test.go | 15 +- ast/stats.go | 2 +- cmd/benchdb/main.go | 2 +- cmd/benchfilesort/main.go | 2 +- cmd/benchkv/main.go | 2 +- cmd/benchraw/main.go | 2 +- cmd/explaintest/main.go | 2 +- cmd/importer/db.go | 2 +- cmd/importer/parser.go | 8 +- cmd/importer/stats.go | 2 +- ddl/callback.go | 2 +- ddl/callback_test.go | 2 +- ddl/column.go | 4 +- ddl/column_change_test.go | 4 +- ddl/column_test.go | 10 +- ddl/db_change_test.go | 11 +- ddl/db_integration_test.go | 2 +- ddl/db_test.go | 10 +- ddl/ddl.go | 8 +- ddl/ddl_api.go | 8 +- ddl/ddl_test.go | 6 +- ddl/ddl_worker.go | 4 +- ddl/ddl_worker_test.go | 6 +- ddl/delete_range.go | 4 +- ddl/fail_db_test.go | 4 +- ddl/fail_test.go | 4 +- ddl/foreign_key.go | 2 +- ddl/foreign_key_test.go | 4 +- ddl/generated_column.go | 2 +- ddl/index.go | 6 +- ddl/index_change_test.go | 4 +- ddl/mock.go | 4 +- ddl/partition.go | 8 +- ddl/reorg.go | 6 +- ddl/reorg_test.go | 2 +- ddl/schema.go | 2 +- ddl/schema_test.go | 4 +- ddl/session_pool.go | 2 +- ddl/stat_test.go | 2 +- ddl/table.go | 2 +- ddl/table_split_test.go | 2 +- ddl/table_test.go | 4 +- ddl/util/event.go | 2 +- ddl/util/util.go | 2 +- distsql/distsql_test.go | 4 +- distsql/request_builder.go | 2 +- distsql/select_result.go | 2 +- distsql/stream.go | 2 +- domain/domain.go | 6 +- domain/domain_test.go | 4 +- domain/global_vars_cache.go | 2 +- domain/info.go | 2 +- domain/schema_checker_test.go | 2 +- domain/topn_slow_query.go | 2 +- executor/adapter.go | 19 +- executor/admin.go | 6 +- executor/admin_test.go | 2 +- executor/aggfuncs/aggfunc_test.go | 2 +- executor/aggfuncs/builder.go | 4 +- executor/aggfuncs/func_avg.go | 2 +- executor/aggfuncs/func_avg_test.go | 4 +- executor/aggregate.go | 2 +- executor/aggregate_test.go | 2 +- executor/analyze.go | 4 +- executor/analyze_test.go | 4 +- executor/batch_checker.go | 2 +- executor/builder.go | 6 +- executor/checksum.go | 2 +- executor/compiler.go | 2 +- executor/ddl.go | 6 +- executor/ddl_test.go | 6 +- executor/delete.go | 2 +- executor/distsql.go | 6 +- executor/distsql_test.go | 2 +- executor/errors.go | 4 +- executor/executor.go | 8 +- executor/executor_pkg_test.go | 6 +- executor/executor_test.go | 8 +- executor/grant.go | 6 +- executor/grant_test.go | 2 +- executor/index_lookup_join.go | 4 +- executor/insert.go | 2 +- executor/insert_common.go | 6 +- executor/insert_test.go | 2 +- executor/join.go | 2 +- executor/load_data.go | 2 +- executor/metrics_test.go | 2 +- executor/pkg_test.go | 4 +- executor/point_get.go | 4 +- executor/prepared.go | 19 +- executor/prepared_test.go | 2 +- executor/revoke.go | 4 +- executor/revoke_test.go | 2 +- executor/set.go | 6 +- executor/set_test.go | 2 +- executor/show.go | 12 +- executor/show_stats.go | 2 +- executor/show_test.go | 6 +- executor/simple.go | 10 +- executor/simple_test.go | 8 +- executor/statement_context_test.go | 2 +- executor/table_reader.go | 2 +- executor/trace.go | 2 +- executor/union_scan.go | 2 +- executor/update.go | 2 +- executor/write.go | 4 +- executor/write_test.go | 2 +- expression/aggregation/agg_to_pb.go | 2 +- expression/aggregation/agg_to_pb_test.go | 6 +- expression/aggregation/aggregation.go | 8 +- expression/aggregation/aggregation_test.go | 4 +- expression/aggregation/avg.go | 4 +- expression/aggregation/bench_test.go | 4 +- expression/aggregation/descriptor.go | 8 +- expression/bench_test.go | 8 +- expression/builtin.go | 8 +- expression/builtin_arithmetic.go | 4 +- expression/builtin_arithmetic_test.go | 4 +- expression/builtin_cast.go | 10 +- expression/builtin_cast_test.go | 6 +- expression/builtin_compare.go | 8 +- expression/builtin_compare_test.go | 4 +- expression/builtin_control.go | 4 +- expression/builtin_control_test.go | 2 +- expression/builtin_encryption.go | 4 +- expression/builtin_encryption_test.go | 4 +- expression/builtin_info.go | 2 +- expression/builtin_info_test.go | 8 +- expression/builtin_json.go | 4 +- expression/builtin_json_test.go | 2 +- expression/builtin_math.go | 2 +- expression/builtin_math_test.go | 6 +- expression/builtin_miscellaneous.go | 4 +- expression/builtin_miscellaneous_test.go | 2 +- expression/builtin_op.go | 4 +- expression/builtin_op_test.go | 2 +- expression/builtin_other.go | 2 +- expression/builtin_other_test.go | 4 +- expression/builtin_string.go | 6 +- expression/builtin_string_test.go | 8 +- expression/builtin_test.go | 8 +- expression/builtin_time.go | 4 +- expression/builtin_time_test.go | 6 +- expression/chunk_executor.go | 4 +- expression/column.go | 4 +- expression/column_test.go | 4 +- expression/constant.go | 4 +- expression/constant_fold.go | 2 +- expression/constant_propagation.go | 6 +- expression/constant_test.go | 6 +- expression/distsql_builtin.go | 4 +- expression/distsql_builtin_test.go | 2 +- expression/errors.go | 4 +- expression/evaluator_test.go | 10 +- expression/expr_to_pb.go | 8 +- expression/expr_to_pb_test.go | 6 +- expression/expression.go | 8 +- expression/expression_test.go | 6 +- expression/function_traits.go | 2 +- expression/function_traits_test.go | 2 +- expression/helper.go | 9 +- expression/helper_test.go | 6 +- expression/integration_test.go | 8 +- expression/scalar_function.go | 8 +- expression/scalar_function_test.go | 6 +- expression/schema.go | 2 +- expression/schema_test.go | 2 +- expression/simple_rewriter.go | 17 +- expression/typeinfer_test.go | 6 +- expression/util.go | 8 +- expression/util_test.go | 4 +- infoschema/builder.go | 2 +- infoschema/infoschema.go | 6 +- infoschema/infoschema_test.go | 4 +- infoschema/tables.go | 6 +- infoschema/tables_test.go | 2 +- kv/error.go | 4 +- kv/memdb_buffer.go | 2 +- kv/txn.go | 2 +- kv/union_store_test.go | 2 +- meta/autoid/autoid.go | 2 +- meta/autoid/autoid_test.go | 2 +- meta/meta.go | 6 +- meta/meta_test.go | 2 +- metrics/server.go | 2 +- model/ddl.go | 2 +- model/model.go | 7 +- model/model_test.go | 2 +- mysql/const_test.go | 4 +- owner/fail_test.go | 2 +- owner/manager.go | 2 +- parser/bench_test.go | 50 - parser/goyacc/main.go | 819 - perfschema/init.go | 6 +- perfschema/perfschema.go | 2 +- perfschema/tables.go | 2 +- planner/core/cache.go | 2 +- planner/core/cache_test.go | 2 +- planner/core/cacheable_checker.go | 7 +- planner/core/cacheable_checker_test.go | 9 +- planner/core/common_plans.go | 11 +- planner/core/errors.go | 4 +- planner/core/exhaust_physical_plans.go | 8 +- planner/core/expression_rewriter.go | 17 +- planner/core/expression_test.go | 8 +- planner/core/find_best_task.go | 4 +- planner/core/logical_plan_builder.go | 21 +- planner/core/logical_plan_test.go | 10 +- planner/core/logical_plans.go | 6 +- planner/core/logical_plans_test.go | 6 +- planner/core/optimizer.go | 2 +- planner/core/physical_plan_test.go | 4 +- planner/core/physical_plans.go | 4 +- planner/core/plan.go | 2 +- planner/core/plan_to_pb.go | 2 +- planner/core/plan_to_pb_test.go | 4 +- planner/core/planbuilder.go | 21 +- planner/core/planbuilder_test.go | 4 +- planner/core/point_get_plan.go | 13 +- planner/core/preprocess.go | 27 +- planner/core/preprocess_test.go | 6 +- planner/core/rule_aggregation_push_down.go | 6 +- planner/core/rule_build_key_info.go | 4 +- planner/core/rule_column_pruning.go | 4 +- planner/core/rule_decorrelate.go | 4 +- planner/core/rule_join_reorder.go | 2 +- planner/core/rule_max_min_eliminate.go | 4 +- planner/core/rule_predicate_push_down.go | 6 +- planner/core/task.go | 6 +- planner/core/trace.go | 2 +- planner/core/util.go | 2 +- privilege/privilege.go | 4 +- privilege/privileges/cache.go | 6 +- privilege/privileges/cache_test.go | 2 +- privilege/privileges/privileges.go | 6 +- privilege/privileges/privileges_test.go | 4 +- server/column.go | 2 +- server/conn.go | 22 +- server/conn_stmt.go | 2 +- server/conn_stmt_test.go | 4 +- server/conn_test.go | 2 +- server/driver.go | 2 +- server/driver_tidb.go | 11 +- server/driver_tidb_test.go | 8 +- server/http_handler.go | 4 +- server/http_handler_test.go | 4 +- server/http_status.go | 4 +- server/packetio.go | 4 +- server/server.go | 6 +- server/server_test.go | 5 +- server/statistics_handler.go | 2 +- server/tidb_test.go | 2 +- server/util.go | 2 +- server/util_test.go | 2 +- session/bench_test.go | 4 +- session/bootstrap.go | 6 +- session/bootstrap_test.go | 12 +- session/session.go | 48 +- session/session_test.go | 10 +- session/tidb.go | 19 +- session/tidb_test.go | 8 +- sessionctx/binloginfo/binloginfo.go | 2 +- sessionctx/binloginfo/binloginfo_test.go | 2 +- sessionctx/stmtctx/stmtctx.go | 2 +- sessionctx/variable/session.go | 8 +- sessionctx/variable/sysvar.go | 6 +- sessionctx/variable/tidb_vars.go | 3 +- sessionctx/variable/varsutil.go | 2 +- sessionctx/variable/varsutil_test.go | 4 +- statistics/bootstrap.go | 6 +- statistics/ddl.go | 9 +- statistics/ddl_test.go | 2 +- statistics/dump.go | 4 +- statistics/dump_test.go | 2 +- statistics/feedback.go | 2 +- statistics/feedback_test.go | 2 +- statistics/handle.go | 2 +- statistics/handle_test.go | 2 +- statistics/histogram.go | 6 +- statistics/sample.go | 20 +- statistics/sample_test.go | 6 +- statistics/scalar.go | 2 +- statistics/scalar_test.go | 2 +- statistics/selectivity.go | 4 +- statistics/selectivity_test.go | 4 +- statistics/statistics_test.go | 25 +- statistics/table.go | 4 +- statistics/update.go | 2 +- statistics/update_test.go | 4 +- store/mockstore/mocktikv/analyze.go | 12 +- store/mockstore/mocktikv/cop_handler_dag.go | 6 +- store/mockstore/mocktikv/executor.go | 4 +- store/mockstore/mocktikv/mvcc_leveldb.go | 2 +- store/mockstore/mocktikv/rpc.go | 2 +- store/tikv/2pc.go | 2 +- store/tikv/2pc_fail_test.go | 2 +- store/tikv/backoff.go | 4 +- store/tikv/client.go | 2 +- store/tikv/error.go | 4 +- store/tikv/gcworker/gc_worker.go | 2 +- store/tikv/isolation_test.go | 2 +- store/tikv/safepoint_test.go | 2 +- store/tikv/sql_fail_test.go | 2 +- store/tikv/tikv_test.go | 2 +- structure/structure.go | 2 +- table/column.go | 8 +- table/column_test.go | 6 +- table/index.go | 2 +- table/table.go | 6 +- table/tables/gen_expr.go | 6 +- table/tables/gen_expr_test.go | 2 +- table/tables/index.go | 4 +- table/tables/index_test.go | 4 +- table/tables/partition.go | 2 +- table/tables/tables.go | 4 +- table/tables/tables_test.go | 2 +- tablecodec/tablecodec.go | 4 +- tablecodec/tablecodec_test.go | 2 +- terror/terror.go | 2 +- tidb-server/main.go | 4 +- types/compare_test.go | 2 +- types/convert.go | 4 +- types/convert_test.go | 8 +- types/datum.go | 6 +- types/datum_eval.go | 2 +- types/datum_test.go | 2 +- types/errors.go | 9 +- types/etc.go | 70 +- types/etc_test.go | 4 +- types/eval_type.go | 26 +- types/field_type.go | 226 +- types/field_type_test.go | 4 +- types/format_test.go | 2 +- types/json/binary.go | 2 +- types/json/constants.go | 4 +- types/mydecimal.go | 4 +- types/parser_driver/value_expr.go | 177 + types/time.go | 4 +- types/time_test.go | 2 +- util/admin/admin.go | 13 +- util/admin/admin_test.go | 2 +- util/auth/auth.go | 2 +- util/charset/charset.go | 2 +- util/chunk/chunk_test.go | 2 +- util/chunk/codec.go | 2 +- util/chunk/codec_test.go | 2 +- util/chunk/compare.go | 2 +- util/chunk/iterator_test.go | 2 +- util/chunk/list_test.go | 2 +- util/chunk/mutrow.go | 2 +- util/chunk/mutrow_test.go | 2 +- util/chunk/row.go | 2 +- util/codec/bench_test.go | 2 +- util/codec/codec.go | 4 +- util/codec/codec_test.go | 4 +- util/filesort/filesort.go | 2 +- util/kvencoder/kv_encoder.go | 2 +- util/memory/action.go | 4 +- util/mock/context.go | 3 +- util/printer/printer.go | 2 +- util/ranger/checker.go | 4 +- util/ranger/detacher.go | 4 +- util/ranger/points.go | 6 +- util/ranger/ranger.go | 6 +- util/ranger/ranger_test.go | 2 +- util/sqlexec/restricted_sql_executor.go | 42 +- util/testkit/testkit.go | 10 +- vendor/github.com/BurntSushi/toml/.gitignore | 5 + vendor/github.com/BurntSushi/toml/.travis.yml | 15 + vendor/github.com/BurntSushi/toml/COMPATIBLE | 3 + vendor/github.com/BurntSushi/toml/Makefile | 19 + vendor/github.com/BurntSushi/toml/README.md | 218 + .../toml/cmd/toml-test-decoder/COPYING | 14 - .../toml/cmd/toml-test-encoder/COPYING | 14 - .../BurntSushi/toml/cmd/tomlv/COPYING | 14 - .../github.com/BurntSushi/toml/decode_test.go | 1461 ++ .../github.com/BurntSushi/toml/encode_test.go | 615 + vendor/github.com/BurntSushi/toml/session.vim | 1 + vendor/github.com/apache/thrift/.clang-format | 56 + vendor/github.com/apache/thrift/.dockerignore | 1 + vendor/github.com/apache/thrift/.editorconfig | 112 + .../github.com/apache/thrift/.gitattributes | 1 + vendor/github.com/apache/thrift/.gitignore | 326 + vendor/github.com/apache/thrift/.travis.yml | 199 + vendor/github.com/apache/thrift/CHANGES | 2366 +++ .../github.com/apache/thrift/CMakeLists.txt | 117 + .../github.com/apache/thrift/CONTRIBUTING.md | 49 + vendor/github.com/apache/thrift/Dockerfile | 61 + vendor/github.com/apache/thrift/Makefile.am | 131 + vendor/github.com/apache/thrift/README.md | 166 + .../github.com/apache/thrift/Thrift.podspec | 18 + vendor/github.com/apache/thrift/appveyor.yml | 93 + vendor/github.com/apache/thrift/bootstrap.sh | 54 + vendor/github.com/apache/thrift/bower.json | 16 + vendor/github.com/apache/thrift/cleanup.sh | 89 + vendor/github.com/apache/thrift/composer.json | 30 + vendor/github.com/apache/thrift/configure.ac | 959 ++ .../apache/thrift/contrib/fb303/LICENSE | 16 - vendor/github.com/apache/thrift/doap.rdf | 132 + .../github.com/apache/thrift/lib/Makefile.am | 109 + .../apache/thrift/lib/dart/LICENSE_HEADER | 16 - .../apache/thrift/lib/go/Makefile.am | 42 + .../github.com/apache/thrift/lib/go/README.md | 81 + .../apache/thrift/lib/go/coding_standards.md | 1 + .../go/thrift/application_exception_test.go | 41 + .../lib/go/thrift/binary_protocol_test.go | 28 + .../lib/go/thrift/buffered_transport_test.go | 29 + .../lib/go/thrift/compact_protocol_test.go | 53 + .../thrift/lib/go/thrift/exception_test.go | 69 + .../lib/go/thrift/framed_transport_test.go | 29 + .../thrift/lib/go/thrift/http_client_test.go | 106 + .../lib/go/thrift/iostream_transport_test.go | 52 + .../lib/go/thrift/json_protocol_test.go | 649 + .../lib/go/thrift/lowlevel_benchmarks_test.go | 396 + .../lib/go/thrift/memory_buffer_test.go | 29 + .../thrift/lib/go/thrift/protocol_test.go | 479 + .../lib/go/thrift/rich_transport_test.go | 85 + .../thrift/lib/go/thrift/serializer_test.go | 169 + .../lib/go/thrift/serializer_types_test.go | 633 + .../lib/go/thrift/server_socket_test.go | 50 + .../thrift/lib/go/thrift/server_test.go | 28 + .../go/thrift/simple_json_protocol_test.go | 715 + .../lib/go/thrift/transport_exception_test.go | 60 + .../thrift/lib/go/thrift/transport_test.go | 176 + .../lib/go/thrift/zlib_transport_test.go | 33 + vendor/github.com/apache/thrift/package.json | 54 + .../apache/thrift/sonar-project.properties | 140 + vendor/github.com/beorn7/perks/.gitignore | 2 + vendor/github.com/beorn7/perks/README.md | 31 + .../beorn7/perks/quantile/bench_test.go | 63 + .../beorn7/perks/quantile/example_test.go | 121 + .../beorn7/perks/quantile/exampledata.txt | 2388 +++ .../beorn7/perks/quantile/stream_test.go | 188 + .../blacktear23/go-proxyprotocol/.travis.yml | 12 + .../blacktear23/go-proxyprotocol/README.md | 37 + .../go-proxyprotocol/proxy_protocol_test.go | 353 + .../codahale/hdrhistogram/.travis.yml | 5 + .../codahale/hdrhistogram/README.md | 15 + .../codahale/hdrhistogram/hdr_test.go | 388 + .../codahale/hdrhistogram/window_test.go | 64 + vendor/github.com/coreos/etcd/.dockerignore | 1 + vendor/github.com/coreos/etcd/.gitignore | 22 + vendor/github.com/coreos/etcd/.godir | 1 + vendor/github.com/coreos/etcd/.header | 13 + vendor/github.com/coreos/etcd/.semaphore.sh | 16 + vendor/github.com/coreos/etcd/.travis.yml | 89 + vendor/github.com/coreos/etcd/CONTRIBUTING.md | 62 + vendor/github.com/coreos/etcd/DCO | 36 + vendor/github.com/coreos/etcd/Dockerfile | 6 + .../github.com/coreos/etcd/Dockerfile-release | 17 + .../coreos/etcd/Dockerfile-release.arm64 | 11 + .../coreos/etcd/Dockerfile-release.ppc64le | 11 + vendor/github.com/coreos/etcd/Dockerfile-test | 57 + vendor/github.com/coreos/etcd/MAINTAINERS | 8 + vendor/github.com/coreos/etcd/NEWS | 106 + vendor/github.com/coreos/etcd/Procfile | 6 + vendor/github.com/coreos/etcd/README.md | 135 + vendor/github.com/coreos/etcd/ROADMAP.md | 23 + vendor/github.com/coreos/etcd/V2Procfile | 5 + .../coreos/etcd/auth/authpb/auth.proto | 37 + vendor/github.com/coreos/etcd/auth/doc.go | 16 + vendor/github.com/coreos/etcd/auth/jwt.go | 137 + .../coreos/etcd/auth/range_perm_cache.go | 133 + .../coreos/etcd/auth/range_perm_cache_test.go | 59 + .../coreos/etcd/auth/simple_token.go | 220 + vendor/github.com/coreos/etcd/auth/store.go | 1059 ++ .../github.com/coreos/etcd/auth/store_test.go | 656 + .../coreos/etcd/bill-of-materials.json | 388 + .../etcd/bill-of-materials.override.json | 26 + vendor/github.com/coreos/etcd/build | 64 + vendor/github.com/coreos/etcd/build.bat | 1 + vendor/github.com/coreos/etcd/build.ps1 | 81 + .../github.com/coreos/etcd/client/README.md | 117 + .../coreos/etcd/client/auth_role.go | 237 + .../coreos/etcd/client/auth_user.go | 320 + .../coreos/etcd/client/cancelreq.go | 18 + .../github.com/coreos/etcd/client/client.go | 704 + .../coreos/etcd/client/client_test.go | 1074 ++ .../coreos/etcd/client/cluster_error.go | 37 + vendor/github.com/coreos/etcd/client/curl.go | 70 + .../github.com/coreos/etcd/client/discover.go | 40 + vendor/github.com/coreos/etcd/client/doc.go | 73 + .../coreos/etcd/client/fake_transport_test.go | 40 + .../coreos/etcd/client/keys.generated.go | 1087 ++ vendor/github.com/coreos/etcd/client/keys.go | 682 + .../coreos/etcd/client/keys_bench_test.go | 87 + .../coreos/etcd/client/keys_test.go | 1429 ++ .../github.com/coreos/etcd/client/members.go | 304 + .../coreos/etcd/client/members_test.go | 599 + vendor/github.com/coreos/etcd/client/util.go | 53 + .../github.com/coreos/etcd/clientv3/README.md | 85 + .../coreos/etcd/clientv3/client_test.go | 156 + .../coreos/etcd/clientv3/compact_op_test.go | 30 + .../etcd/clientv3/example_cluster_test.go | 104 + .../coreos/etcd/clientv3/example_kv_test.go | 283 + .../etcd/clientv3/example_lease_test.go | 142 + .../etcd/clientv3/example_maintenence_test.go | 62 + .../etcd/clientv3/example_metrics_test.go | 85 + .../coreos/etcd/clientv3/example_test.go | 77 + .../etcd/clientv3/example_watch_test.go | 101 + .../coreos/etcd/clientv3/main_test.go | 73 + .../coreos/etcd/clientv3/op_test.go | 38 + .../coreos/etcd/clientv3/txn_test.go | 105 + .../coreos/etcd/clientv3/watch_test.go | 55 + vendor/github.com/coreos/etcd/cmd/etcd | 1 - vendor/github.com/coreos/etcd/cmd/etcdctl | 1 - vendor/github.com/coreos/etcd/cmd/tools | 1 - vendor/github.com/coreos/etcd/cover | 31 + .../coreos/etcd/etcd.conf.yml.sample | 141 + .../coreos/etcd/etcdserver/api/capability.go | 86 + .../coreos/etcd/etcdserver/api/cluster.go | 41 + .../coreos/etcd/etcdserver/api/doc.go | 16 + .../coreos/etcd/etcdserver/api/v3rpc/auth.go | 157 + .../coreos/etcd/etcdserver/api/v3rpc/codec.go | 34 + .../coreos/etcd/etcdserver/api/v3rpc/grpc.go | 71 + .../etcd/etcdserver/api/v3rpc/header.go | 46 + .../etcd/etcdserver/api/v3rpc/interceptor.go | 144 + .../coreos/etcd/etcdserver/api/v3rpc/key.go | 259 + .../coreos/etcd/etcdserver/api/v3rpc/lease.go | 133 + .../etcd/etcdserver/api/v3rpc/maintenance.go | 190 + .../etcd/etcdserver/api/v3rpc/member.go | 103 + .../etcd/etcdserver/api/v3rpc/metrics.go | 38 + .../coreos/etcd/etcdserver/api/v3rpc/quota.go | 89 + .../api/v3rpc/rpctypes/error_test.go | 42 + .../coreos/etcd/etcdserver/api/v3rpc/util.go | 141 + .../coreos/etcd/etcdserver/api/v3rpc/watch.go | 446 + .../coreos/etcd/etcdserver/apply.go | 878 ++ .../coreos/etcd/etcdserver/apply_auth.go | 196 + .../coreos/etcd/etcdserver/apply_v2.go | 140 + .../coreos/etcd/etcdserver/backend.go | 81 + .../coreos/etcd/etcdserver/cluster_util.go | 258 + .../etcd/etcdserver/cluster_util_test.go | 131 + .../coreos/etcd/etcdserver/config.go | 209 + .../coreos/etcd/etcdserver/config_test.go | 196 + .../etcd/etcdserver/consistent_index.go | 33 + .../etcd/etcdserver/consistent_index_test.go | 25 + .../github.com/coreos/etcd/etcdserver/doc.go | 16 + .../coreos/etcd/etcdserver/errors.go | 46 + .../etcdserver/etcdserverpb/etcdserver.proto | 34 + .../etcdserverpb/raft_internal.proto | 74 + .../etcd/etcdserver/etcdserverpb/rpc.proto | 984 ++ .../coreos/etcd/etcdserver/metrics.go | 102 + .../coreos/etcd/etcdserver/quota.go | 121 + .../github.com/coreos/etcd/etcdserver/raft.go | 603 + .../coreos/etcd/etcdserver/raft_test.go | 221 + .../coreos/etcd/etcdserver/server.go | 1704 +++ .../coreos/etcd/etcdserver/server_test.go | 1662 ++ .../coreos/etcd/etcdserver/snapshot_merge.go | 73 + .../coreos/etcd/etcdserver/storage.go | 98 + .../github.com/coreos/etcd/etcdserver/util.go | 97 + .../coreos/etcd/etcdserver/util_test.go | 90 + .../coreos/etcd/etcdserver/v2_server.go | 125 + .../coreos/etcd/etcdserver/v3_server.go | 686 + vendor/github.com/coreos/etcd/glide.lock | 176 + vendor/github.com/coreos/etcd/glide.yaml | 134 + vendor/github.com/coreos/etcd/main.go | 29 + vendor/github.com/coreos/etcd/main_test.go | 35 + vendor/github.com/coreos/etcd/mvcc/doc.go | 16 + vendor/github.com/coreos/etcd/mvcc/index.go | 219 + .../github.com/coreos/etcd/mvcc/index_test.go | 286 + .../github.com/coreos/etcd/mvcc/key_index.go | 332 + .../coreos/etcd/mvcc/key_index_test.go | 654 + vendor/github.com/coreos/etcd/mvcc/kv.go | 147 + vendor/github.com/coreos/etcd/mvcc/kv_test.go | 831 + vendor/github.com/coreos/etcd/mvcc/kv_view.go | 53 + vendor/github.com/coreos/etcd/mvcc/kvstore.go | 459 + .../coreos/etcd/mvcc/kvstore_bench_test.go | 124 + .../coreos/etcd/mvcc/kvstore_compaction.go | 66 + .../etcd/mvcc/kvstore_compaction_test.go | 135 + .../coreos/etcd/mvcc/kvstore_test.go | 717 + .../coreos/etcd/mvcc/kvstore_txn.go | 253 + vendor/github.com/coreos/etcd/mvcc/metrics.go | 174 + .../coreos/etcd/mvcc/metrics_txn.go | 67 + .../coreos/etcd/mvcc/mvccpb/kv.proto | 49 + .../github.com/coreos/etcd/mvcc/revision.go | 67 + .../coreos/etcd/mvcc/revision_test.go | 53 + vendor/github.com/coreos/etcd/mvcc/util.go | 56 + .../coreos/etcd/mvcc/watchable_store.go | 522 + .../etcd/mvcc/watchable_store_bench_test.go | 200 + .../coreos/etcd/mvcc/watchable_store_test.go | 468 + .../coreos/etcd/mvcc/watchable_store_txn.go | 53 + vendor/github.com/coreos/etcd/mvcc/watcher.go | 171 + .../coreos/etcd/mvcc/watcher_bench_test.go | 38 + .../coreos/etcd/mvcc/watcher_group.go | 283 + .../coreos/etcd/mvcc/watcher_test.go | 344 + vendor/github.com/coreos/etcd/pkg/README.md | 2 + .../coreos/etcd/pkg/types/id_test.go | 95 + .../coreos/etcd/pkg/types/set_test.go | 186 + .../coreos/etcd/pkg/types/slice_test.go | 30 + .../coreos/etcd/pkg/types/urls_test.go | 169 + .../coreos/etcd/pkg/types/urlsmap_test.go | 155 + vendor/github.com/coreos/etcd/test | 373 + vendor/github.com/cznic/golex/lex/api.go | 410 - vendor/github.com/cznic/golex/lex/doc.go | 40 - vendor/github.com/cznic/mathutil/AUTHORS | 12 + vendor/github.com/cznic/mathutil/CONTRIBUTORS | 10 + vendor/github.com/cznic/mathutil/Makefile | 31 + vendor/github.com/cznic/mathutil/README | 10 + vendor/github.com/cznic/mathutil/all_test.go | 3530 +++++ .../cznic/mathutil/mersenne/LICENSE | 27 - .../cznic/mathutil/nist-sts-2-1-1-report | 267 + vendor/github.com/cznic/parser/nquads/LICENSE | 27 - vendor/github.com/cznic/parser/yacc/LICENSE | 27 - vendor/github.com/cznic/parser/yacc/api.go | 92 - vendor/github.com/cznic/parser/yacc/ast.go | 514 - .../github.com/cznic/parser/yacc/generate.go | 128 - .../github.com/cznic/parser/yacc/goscanner.go | 436 - vendor/github.com/cznic/parser/yacc/lexer.go | 656 - vendor/github.com/cznic/parser/yacc/parser.go | 938 -- .../github.com/cznic/parser/yacc/scanner.go | 856 -- vendor/github.com/cznic/sortutil/AUTHORS | 11 + vendor/github.com/cznic/sortutil/CONTRIBUTORS | 10 + vendor/github.com/cznic/sortutil/Makefile | 35 + vendor/github.com/cznic/sortutil/README | 4 + vendor/github.com/cznic/sortutil/all_test.go | 407 + vendor/github.com/cznic/strutil/LICENSE | 27 - vendor/github.com/cznic/strutil/strutil.go | 645 - vendor/github.com/cznic/y/api.go | 966 -- vendor/github.com/cznic/y/y.go | 2199 --- vendor/github.com/etcd-io/gofail/.travis.yml | 17 + vendor/github.com/etcd-io/gofail/DCO | 36 + vendor/github.com/etcd-io/gofail/Makefile | 21 + vendor/github.com/etcd-io/gofail/README.md | 113 + .../etcd-io/gofail/code-of-conduct.md | 61 + vendor/github.com/etcd-io/gofail/gofail.go | 195 + .../etcd-io/gofail/runtime/terms_test.go | 76 + .../github.com/go-sql-driver/mysql/.gitignore | 9 + .../go-sql-driver/mysql/.travis.yml | 89 + vendor/github.com/go-sql-driver/mysql/AUTHORS | 73 + .../go-sql-driver/mysql/CHANGELOG.md | 119 + .../go-sql-driver/mysql/CONTRIBUTING.md | 23 + .../github.com/go-sql-driver/mysql/README.md | 478 + .../mysql/benchmark_go18_test.go | 93 + .../go-sql-driver/mysql/benchmark_test.go | 246 + .../go-sql-driver/mysql/connection_test.go | 67 + .../go-sql-driver/mysql/driver_go18_test.go | 522 + .../go-sql-driver/mysql/driver_test.go | 2001 +++ .../go-sql-driver/mysql/dsn_test.go | 249 + .../go-sql-driver/mysql/errors_test.go | 42 + .../go-sql-driver/mysql/packets_test.go | 283 + .../go-sql-driver/mysql/utils_go18_test.go | 54 + .../go-sql-driver/mysql/utils_test.go | 277 + vendor/github.com/gogo/protobuf/.gitignore | 6 + vendor/github.com/gogo/protobuf/.mailmap | 8 + vendor/github.com/gogo/protobuf/.travis.yml | 22 + vendor/github.com/gogo/protobuf/AUTHORS | 15 + vendor/github.com/gogo/protobuf/CONTRIBUTORS | 23 + .../gogo/protobuf/GOLANG_CONTRIBUTORS | 5 + vendor/github.com/gogo/protobuf/Makefile | 167 + vendor/github.com/gogo/protobuf/README | 299 + vendor/github.com/gogo/protobuf/Readme.md | 151 + vendor/github.com/gogo/protobuf/bench.md | 190 + .../github.com/gogo/protobuf/custom_types.md | 68 + vendor/github.com/gogo/protobuf/extensions.md | 164 + .../gogo/protobuf/install-protobuf.sh | 28 + .../github.com/gogo/protobuf/proto/Makefile | 43 + .../gogo/protobuf/proto/all_test.go | 2421 +++ .../gogo/protobuf/proto/any_test.go | 300 + .../gogo/protobuf/proto/clone_test.go | 397 + .../gogo/protobuf/proto/decode_test.go | 259 + .../gogo/protobuf/proto/discard_test.go | 169 + .../gogo/protobuf/proto/encode_test.go | 84 + .../gogo/protobuf/proto/equal_test.go | 244 + .../gogo/protobuf/proto/extensions_test.go | 691 + .../gogo/protobuf/proto/map_test.go | 70 + .../gogo/protobuf/proto/message_set_test.go | 77 + .../gogo/protobuf/proto/proto3_test.go | 151 + .../gogo/protobuf/proto/size2_test.go | 63 + .../gogo/protobuf/proto/size_test.go | 190 + .../gogo/protobuf/proto/text_parser_test.go | 706 + .../gogo/protobuf/proto/text_test.go | 518 + vendor/github.com/golang/protobuf/.gitignore | 17 + vendor/github.com/golang/protobuf/.travis.yml | 30 + vendor/github.com/golang/protobuf/AUTHORS | 3 + .../github.com/golang/protobuf/CONTRIBUTORS | 3 + vendor/github.com/golang/protobuf/Makefile | 48 + vendor/github.com/golang/protobuf/README.md | 283 + .../golang/protobuf/jsonpb/jsonpb_test.go | 1150 ++ .../golang/protobuf/proto/all_test.go | 2410 +++ .../golang/protobuf/proto/any_test.go | 300 + .../golang/protobuf/proto/clone_test.go | 390 + .../golang/protobuf/proto/decode_test.go | 255 + .../golang/protobuf/proto/discard_test.go | 170 + .../golang/protobuf/proto/encode_test.go | 85 + .../golang/protobuf/proto/equal_test.go | 244 + .../golang/protobuf/proto/extensions_test.go | 688 + .../golang/protobuf/proto/map_test.go | 70 + .../golang/protobuf/proto/message_set_test.go | 77 + .../golang/protobuf/proto/proto3_test.go | 151 + .../golang/protobuf/proto/size2_test.go | 63 + .../golang/protobuf/proto/size_test.go | 191 + .../golang/protobuf/proto/text_parser_test.go | 706 + .../golang/protobuf/proto/text_test.go | 518 + .../protoc-gen-go/descriptor/descriptor.proto | 872 ++ .../golang/protobuf/protoc-gen-go/doc.go | 51 + .../protobuf/protoc-gen-go/golden_test.go | 422 + .../protobuf/protoc-gen-go/link_grpc.go | 34 + .../golang/protobuf/protoc-gen-go/main.go | 98 + .../golang/protobuf/ptypes/any/any.proto | 149 + .../golang/protobuf/ptypes/any_test.go | 113 + .../protobuf/ptypes/duration/duration.proto | 117 + .../golang/protobuf/ptypes/duration_test.go | 121 + .../protobuf/ptypes/struct/struct.proto | 96 + .../protobuf/ptypes/timestamp/timestamp.proto | 133 + .../golang/protobuf/ptypes/timestamp_test.go | 153 + .../github.com/golang/protobuf/regenerate.sh | 53 + vendor/github.com/golang/snappy/AUTHORS | 14 + vendor/github.com/golang/snappy/CONTRIBUTORS | 36 + vendor/github.com/golang/snappy/README | 7 + .../github.com/golang/snappy/snappy_test.go | 377 + vendor/github.com/google/btree/.travis.yml | 1 + vendor/github.com/google/btree/README.md | 12 + vendor/github.com/google/btree/btree_test.go | 689 + vendor/github.com/gorilla/context/.travis.yml | 19 + vendor/github.com/gorilla/context/README.md | 7 + .../gorilla/context/context_test.go | 161 + vendor/github.com/gorilla/mux/.travis.yml | 22 + vendor/github.com/gorilla/mux/README.md | 340 + vendor/github.com/gorilla/mux/bench_test.go | 49 + .../gorilla/mux/context_gorilla_test.go | 40 + .../gorilla/mux/context_native_test.go | 32 + vendor/github.com/gorilla/mux/mux_test.go | 1655 ++ vendor/github.com/gorilla/mux/old_test.go | 710 + .../go-grpc-middleware/.gitignore | 199 + .../go-grpc-middleware/.travis.yml | 26 + .../grpc-ecosystem/go-grpc-middleware/DOC.md | 164 + .../go-grpc-middleware/README.md | 86 + .../go-grpc-middleware/chain_test.go | 172 + .../go-grpc-middleware/checkup.sh | 21 + .../go-grpc-middleware/fixup.sh | 36 + .../go-grpc-middleware/test_all.sh | 14 + .../tracing/opentracing/DOC.md | 109 + .../tracing/opentracing/README.md | 1 + .../tracing/opentracing/interceptors_test.go | 186 + .../go-grpc-middleware/util/metautils/DOC.md | 114 + .../util/metautils/README.md | 1 + .../util/metautils/nicemd_test.go | 86 + .../go-grpc-middleware/wrappers_test.go | 52 + .../go-grpc-prometheus/.gitignore | 197 + .../go-grpc-prometheus/.travis.yml | 13 + .../go-grpc-prometheus/README.md | 245 + .../go-grpc-prometheus/client_test.go | 212 + .../go-grpc-prometheus/server_test.go | 307 + .../golang_protobuf_extensions/.travis.yml | 8 + .../Makefile.TRAVIS | 15 + .../golang_protobuf_extensions/README.md | 20 + .../pbutil/.gitignore | 1 + .../pbutil/Makefile | 7 + .../pbutil/all_test.go | 178 + .../pbutil/decode_test.go | 99 + .../pbutil/encode_test.go | 67 + vendor/github.com/ngaut/pools/id_pool_test.go | 118 + .../github.com/ngaut/pools/numbered_test.go | 84 + .../ngaut/pools/resource_pool_test.go | 487 + .../github.com/ngaut/pools/roundrobin_test.go | 126 + .../y/LICENSE => ngaut/pools/vitess_license} | 9 +- vendor/github.com/ngaut/sync2/atomic_test.go | 32 + vendor/github.com/ngaut/sync2/cond_test.go | 276 + .../github.com/ngaut/sync2/semaphore_test.go | 41 + .../ngaut/sync2/service_manager_test.go | 176 + .../LICENSE => ngaut/sync2/vitess_license} | 9 +- .../opentracing/basictracer-go/.gitignore | 2 + .../opentracing/basictracer-go/.travis.yml | 13 + .../opentracing/basictracer-go/Makefile | 26 + .../opentracing/basictracer-go/README.md | 8 + .../opentracing/basictracer-go/bench_test.go | 184 + .../basictracer-go/concurrency_test.go | 119 + .../basictracer-go/propagation_test.go | 98 + .../basictracer-go/recorder_test.go | 29 + .../opentracing/basictracer-go/span_test.go | 224 + .../basictracer-go/testutil_test.go | 125 + .../basictracer-go/wire/carrier_test.go | 38 + .../basictracer-go/wire/wire.proto | 10 + .../opentracing/opentracing-go/.gitignore | 13 + .../opentracing/opentracing-go/.travis.yml | 14 + .../opentracing/opentracing-go/CHANGELOG.md | 14 + .../opentracing/opentracing-go/Makefile | 32 + .../opentracing/opentracing-go/README.md | 147 + .../opentracing-go/ext/tags_test.go | 148 + .../opentracing-go/gocontext_test.go | 81 + .../opentracing-go/log/field_test.go | 39 + .../opentracing-go/options_test.go | 31 + .../opentracing-go/propagation_test.go | 93 + .../opentracing-go/testtracer_test.go | 138 + vendor/github.com/petar/GoLLRB/.gitignore | 23 + vendor/github.com/petar/GoLLRB/AUTHORS | 4 + vendor/github.com/petar/GoLLRB/README.md | 66 + .../petar/GoLLRB/llrb/iterator_test.go | 76 + .../github.com/petar/GoLLRB/llrb/llrb_test.go | 239 + vendor/github.com/pingcap/check/.gitignore | 4 + vendor/github.com/pingcap/check/README.md | 13 + vendor/github.com/pingcap/check/TODO | 2 + .../pingcap/check/benchmark_test.go | 92 + .../pingcap/check/bootstrap_test.go | 83 + .../github.com/pingcap/check/check.v1_license | 25 + vendor/github.com/pingcap/check/check_test.go | 207 + .../pingcap/check/checkers2_test.go | 46 + .../github.com/pingcap/check/checkers_test.go | 273 + .../github.com/pingcap/check/export_test.go | 9 + .../github.com/pingcap/check/fixture_test.go | 484 + .../pingcap/check/foundation_test.go | 336 + .../pingcap/check/godropbox_license | 27 + .../github.com/pingcap/check/helpers_test.go | 520 + .../github.com/pingcap/check/printer_test.go | 94 + vendor/github.com/pingcap/check/run_test.go | 470 + vendor/github.com/pingcap/errors/.gitignore | 24 + vendor/github.com/pingcap/errors/.travis.yml | 14 + vendor/github.com/pingcap/errors/LICENSE | 23 + vendor/github.com/pingcap/errors/README.md | 52 + vendor/github.com/pingcap/errors/appveyor.yml | 32 + .../github.com/pingcap/errors/bench_test.go | 110 + vendor/github.com/pingcap/errors/errors.go | 324 + .../github.com/pingcap/errors/errors_test.go | 371 + .../github.com/pingcap/errors/example_test.go | 205 + .../github.com/pingcap/errors/format_test.go | 534 + vendor/github.com/pingcap/errors/group.go | 42 + .../github.com/pingcap/errors/juju_adaptor.go | 100 + vendor/github.com/pingcap/errors/stack.go | 226 + .../github.com/pingcap/errors/stack_test.go | 289 + .../github.com/pingcap/goleveldb/.travis.yml | 12 + vendor/github.com/pingcap/goleveldb/README.md | 107 + .../pingcap/goleveldb/leveldb/batch_test.go | 147 + .../pingcap/goleveldb/leveldb/bench_test.go | 507 + .../goleveldb/leveldb/cache/bench_test.go | 29 + .../goleveldb/leveldb/cache/cache_test.go | 563 + .../pingcap/goleveldb/leveldb/corrupt_test.go | 496 + .../pingcap/goleveldb/leveldb/db_test.go | 2925 ++++ .../goleveldb/leveldb/external_test.go | 117 + .../goleveldb/leveldb/filter/bloom_test.go | 142 + .../leveldb/iterator/array_iter_test.go | 30 + .../leveldb/iterator/indexed_iter_test.go | 83 + .../leveldb/iterator/iter_suite_test.go | 11 + .../leveldb/iterator/merged_iter_test.go | 60 + .../goleveldb/leveldb/journal/journal_test.go | 818 + .../pingcap/goleveldb/leveldb/key_test.go | 133 + .../goleveldb/leveldb/leveldb_suite_test.go | 11 + .../goleveldb/leveldb/memdb/bench_test.go | 98 + .../leveldb/memdb/memdb_suite_test.go | 57 + .../goleveldb/leveldb/memdb/memdb_test.go | 135 + .../goleveldb/leveldb/session_record_test.go | 62 + .../leveldb/storage/file_storage_test.go | 176 + .../leveldb/storage/mem_storage_test.go | 65 + .../goleveldb/leveldb/table/block_test.go | 139 + .../leveldb/table/table_suite_test.go | 11 + .../goleveldb/leveldb/table/table_test.go | 123 + .../goleveldb/leveldb/testutil_test.go | 91 + .../goleveldb/leveldb/util/buffer_test.go | 369 + .../goleveldb/leveldb/util/hash_test.go | 46 + .../pingcap/goleveldb/leveldb/version_test.go | 181 + .../github.com/pingcap/kvproto/.gitattributes | 3 + vendor/github.com/pingcap/kvproto/.gitignore | 12 + vendor/github.com/pingcap/kvproto/Cargo.toml | 13 + vendor/github.com/pingcap/kvproto/Makefile | 36 + vendor/github.com/pingcap/kvproto/README.md | 12 + vendor/github.com/pingcap/kvproto/common.sh | 36 + .../github.com/pingcap/kvproto/generate_go.sh | 83 + .../pingcap/kvproto/generate_rust.sh | 54 + vendor/github.com/pingcap/kvproto/glide.lock | 104 + vendor/github.com/pingcap/kvproto/glide.yaml | 81 + vendor/github.com/pingcap/parser/.gitignore | 6 + .../thrift/lib/hs => pingcap/parser}/LICENSE | 1 - vendor/github.com/pingcap/parser/Makefile | 37 + vendor/github.com/pingcap/parser/README.md | 56 + vendor/github.com/pingcap/parser/ast/ast.go | 156 + vendor/github.com/pingcap/parser/ast/base.go | 101 + vendor/github.com/pingcap/parser/ast/ddl.go | 894 ++ .../github.com/pingcap/parser/ast/ddl_test.go | 64 + vendor/github.com/pingcap/parser/ast/dml.go | 1211 ++ .../github.com/pingcap/parser/ast/dml_test.go | 74 + .../pingcap/parser/ast/expressions.go | 912 ++ .../pingcap/parser/ast/expressions_test.go | 103 + vendor/github.com/pingcap/parser/ast/flag.go | 156 + .../pingcap/parser/ast/flag_test.go | 153 + .../pingcap/parser/ast/format_test.go | 98 + .../pingcap/parser/ast/functions.go | 598 + .../pingcap/parser/ast/functions_test.go | 39 + vendor/github.com/pingcap/parser/ast/misc.go | 880 ++ .../pingcap/parser/ast/misc_test.go | 190 + .../pingcap/parser/ast/read_only_checker.go | 61 + .../parser/ast/read_only_checker_test.go | 44 + vendor/github.com/pingcap/parser/ast/stats.go | 91 + vendor/github.com/pingcap/parser/auth/auth.go | 103 + .../pingcap/parser/auth/auth_test.go | 50 + .../github.com/pingcap/parser/bench_test.go | 66 + .../pingcap/parser/charset/charset.go | 417 + .../pingcap/parser/charset/charset_test.go | 94 + .../pingcap/parser/charset/encoding_table.go | 260 + vendor/github.com/pingcap/parser/circle.yml | 18 + .../pingcap/parser}/consistent_test.go | 2 +- .../pingcap/parser/format/format.go | 195 + .../pingcap/parser/format/format_test.go | 60 + vendor/github.com/pingcap/parser/go.mod1 | 16 + vendor/github.com/pingcap/parser/go.sum1 | 257 + .../github.com/pingcap/parser}/lexer.go | 12 +- .../github.com/pingcap/parser}/lexer_test.go | 14 +- .../github.com/pingcap/parser}/misc.go | 42 +- vendor/github.com/pingcap/parser/model/ddl.go | 384 + .../github.com/pingcap/parser/model/flags.go | 43 + .../github.com/pingcap/parser/model/model.go | 582 + .../pingcap/parser/model/model_test.go | 307 + .../pingcap/parser/mysql/charset.go | 608 + .../github.com/pingcap/parser/mysql/const.go | 700 + .../pingcap/parser/mysql/errcode.go | 910 ++ .../pingcap/parser/mysql/errname.go | 907 ++ .../github.com/pingcap/parser/mysql/error.go | 71 + .../pingcap/parser/mysql/error_test.go | 37 + .../pingcap/parser/mysql/locale_format.go | 98 + .../github.com/pingcap/parser/mysql/state.go | 258 + .../github.com/pingcap/parser/mysql/type.go | 155 + .../pingcap/parser/mysql/type_test.go | 37 + .../github.com/pingcap/parser/mysql/util.go | 93 + .../pingcap/parser}/opcode/opcode.go | 0 .../pingcap/parser}/opcode/opcode_test.go | 0 vendor/github.com/pingcap/parser/parser.go | 12533 ++++++++++++++++ .../github.com/pingcap/parser}/parser.y | 636 +- .../pingcap/parser/parser_example_test.go | 39 + .../github.com/pingcap/parser}/parser_test.go | 193 +- .../pingcap/parser/terror/terror.go | 344 + .../pingcap/parser/terror/terror_test.go | 163 + vendor/github.com/pingcap/parser/test.sh | 11 + vendor/github.com/pingcap/parser/types/etc.go | 112 + .../pingcap/parser/types/eval_type.go | 42 + .../pingcap/parser/types/field_type.go | 258 + .../github.com/pingcap/parser}/yy_parser.go | 29 +- vendor/github.com/pingcap/pd/.gitignore | 9 + vendor/github.com/pingcap/pd/.travis.yml | 19 + vendor/github.com/pingcap/pd/CHANGELOG.md | 126 + vendor/github.com/pingcap/pd/CONTRIBUTING.md | 67 + vendor/github.com/pingcap/pd/Dockerfile | 14 + vendor/github.com/pingcap/pd/Jenkinsfile | 11 + vendor/github.com/pingcap/pd/Makefile | 124 + vendor/github.com/pingcap/pd/README.md | 113 + vendor/github.com/pingcap/pd/circle.yml | 12 + .../pingcap/pd/client/client_test.go | 315 + vendor/github.com/pingcap/pd/go.mod | 63 + vendor/github.com/pingcap/pd/go.sum | 118 + vendor/github.com/pingcap/pd/revive.toml | 52 + vendor/github.com/pingcap/pd/tools.json | 53 + .../github.com/pingcap/tidb-tools/.gitignore | 4 + .../github.com/pingcap/tidb-tools/Jenkinsfile | 7 + vendor/github.com/pingcap/tidb-tools/Makefile | 71 + .../github.com/pingcap/tidb-tools/README.md | 41 + .../github.com/pingcap/tidb-tools/glide.lock | 459 + .../github.com/pingcap/tidb-tools/glide.yaml | 139 + .../pingcap/tidb-tools/pkg/etcd/etcd_test.go | 184 + .../pingcap/tidb-tools/pkg/utils/urls_test.go | 73 + .../tidb-binlog/pump_client/client_test.go | 138 + vendor/github.com/pingcap/tipb/.gitignore | 2 + vendor/github.com/pingcap/tipb/Cargo.toml | 9 + vendor/github.com/pingcap/tipb/Makefile | 10 + vendor/github.com/pingcap/tipb/README.md | 36 + vendor/github.com/pingcap/tipb/_help.sh | 25 + .../pingcap/tipb/generate-binlog.sh | 16 + vendor/github.com/pingcap/tipb/generate-go.sh | 14 + .../github.com/pingcap/tipb/generate-rust.sh | 38 + .../tipb/sharedbytes/sharedbytes_test.go | 30 + vendor/github.com/pkg/errors/.gitignore | 24 + vendor/github.com/pkg/errors/.travis.yml | 14 + vendor/github.com/pkg/errors/README.md | 52 + vendor/github.com/pkg/errors/appveyor.yml | 32 + vendor/github.com/pkg/errors/bench_test.go | 110 + vendor/github.com/pkg/errors/errors_test.go | 371 + vendor/github.com/pkg/errors/example_test.go | 205 + vendor/github.com/pkg/errors/format_test.go | 534 + vendor/github.com/pkg/errors/stack_test.go | 289 + .../prometheus/client_golang/.gitignore | 26 + .../prometheus/client_golang/.travis.yml | 9 + .../prometheus/client_golang/AUTHORS.md | 18 + .../prometheus/client_golang/CHANGELOG.md | 109 + .../prometheus/client_golang/CONTRIBUTING.md | 18 + .../prometheus/client_golang/README.md | 45 + .../prometheus/client_golang/VERSION | 1 + .../client_golang/prometheus/.gitignore | 1 + .../client_golang/prometheus/README.md | 1 + .../prometheus/benchmark_test.go | 183 + .../client_golang/prometheus/counter_test.go | 58 + .../prometheus/example_clustermanager_test.go | 118 + .../client_golang/prometheus/examples_test.go | 751 + .../prometheus/expvar_collector_test.go | 97 + .../client_golang/prometheus/gauge_test.go | 182 + .../prometheus/go_collector_test.go | 123 + .../prometheus/histogram_test.go | 326 + .../client_golang/prometheus/http_test.go | 121 + .../client_golang/prometheus/metric_test.go | 35 + .../prometheus/process_collector_test.go | 58 + .../prometheus/push/examples_test.go | 56 + .../prometheus/push/push_test.go | 176 + .../client_golang/prometheus/registry_test.go | 545 + .../client_golang/prometheus/summary_test.go | 347 + .../client_golang/prometheus/vec_test.go | 312 + .../prometheus/client_model/.gitignore | 1 + .../prometheus/client_model/AUTHORS.md | 13 + .../prometheus/client_model/CONTRIBUTING.md | 18 + .../prometheus/client_model/Makefile | 61 + .../prometheus/client_model/README.md | 26 + .../prometheus/client_model/metrics.proto | 81 + .../prometheus/client_model/pom.xml | 130 + .../prometheus/client_model/setup.py | 23 + .../github.com/prometheus/common/.travis.yml | 7 + .../github.com/prometheus/common/AUTHORS.md | 11 + .../prometheus/common/CONTRIBUTING.md | 18 + vendor/github.com/prometheus/common/README.md | 12 + .../prometheus/common/expfmt/bench_test.go | 171 + .../prometheus/common/expfmt/decode_test.go | 367 + .../common/expfmt/text_create_test.go | 443 + .../common/expfmt/text_parse_test.go | 588 + .../bitbucket.org/ww/goautoneg/README.txt | 67 + .../ww/goautoneg/autoneg_test.go | 33 + .../prometheus/common/model/alert_test.go | 118 + .../prometheus/common/model/labels_test.go | 129 + .../prometheus/common/model/metric_test.go | 121 + .../prometheus/common/model/signature_test.go | 314 + .../prometheus/common/model/silence_test.go | 228 + .../prometheus/common/model/time_test.go | 129 + .../prometheus/common/model/value_test.go | 417 + .../github.com/prometheus/procfs/.travis.yml | 5 + .../github.com/prometheus/procfs/AUTHORS.md | 20 + .../prometheus/procfs/CONTRIBUTING.md | 18 + vendor/github.com/prometheus/procfs/Makefile | 6 + vendor/github.com/prometheus/procfs/README.md | 10 + .../github.com/prometheus/procfs/fs_test.go | 13 + .../github.com/prometheus/procfs/ipvs_test.go | 190 + .../prometheus/procfs/mdstat_test.go | 31 + .../prometheus/procfs/proc_io_test.go | 33 + .../prometheus/procfs/proc_limits_test.go | 31 + .../prometheus/procfs/proc_stat_test.go | 110 + .../github.com/prometheus/procfs/proc_test.go | 160 + .../github.com/prometheus/procfs/stat_test.go | 14 + vendor/github.com/sirupsen/logrus/.gitignore | 1 + vendor/github.com/sirupsen/logrus/.travis.yml | 8 + .../github.com/sirupsen/logrus/CHANGELOG.md | 94 + vendor/github.com/sirupsen/logrus/README.md | 479 + .../sirupsen/logrus/alt_exit_test.go | 74 + .../github.com/sirupsen/logrus/entry_test.go | 77 + .../sirupsen/logrus/formatter_bench_test.go | 101 + .../github.com/sirupsen/logrus/hook_test.go | 122 + .../sirupsen/logrus/json_formatter_test.go | 199 + .../sirupsen/logrus/logger_bench_test.go | 61 + .../github.com/sirupsen/logrus/logrus_test.go | 386 + .../sirupsen/logrus/text_formatter_test.go | 87 + .../github.com/spaolacci/murmur3/.gitignore | 22 + vendor/github.com/spaolacci/murmur3/README.md | 84 + .../spaolacci/murmur3/murmur_test.go | 251 + vendor/github.com/twinj/uuid/.gitignore | 1 + vendor/github.com/twinj/uuid/.travis.yml | 8 + vendor/github.com/twinj/uuid/README.md | 86 + vendor/github.com/twinj/uuid/array_test.go | 31 + vendor/github.com/twinj/uuid/rfc4122_test.go | 209 + vendor/github.com/twinj/uuid/saver_test.go | 100 + vendor/github.com/twinj/uuid/state_test.go | 92 + vendor/github.com/twinj/uuid/struct_test.go | 32 + .../github.com/twinj/uuid/timestamp_test.go | 20 + vendor/github.com/twinj/uuid/uuid_test.go | 119 + vendor/github.com/twinj/uuid/uuids_test.go | 449 + .../uber/jaeger-client-go/.gitignore | 14 + .../uber/jaeger-client-go/.gitmodules | 3 + .../uber/jaeger-client-go/.travis.yml | 40 + .../uber/jaeger-client-go/CHANGELOG.md | 118 + .../uber/jaeger-client-go/CONTRIBUTING.md | 75 + .../github.com/uber/jaeger-client-go/Makefile | 103 + .../uber/jaeger-client-go/README.md | 182 + .../jaeger-client-go/config/config_test.go | 182 + .../jaeger-client-go/config/example_test.go | 86 + .../jaeger-client-go/config/options_test.go | 67 + .../uber/jaeger-client-go/constants_test.go | 35 + .../uber/jaeger-client-go/context_test.go | 116 + .../uber/jaeger-client-go/glide.lock | 70 + .../uber/jaeger-client-go/glide.yaml | 33 + .../jaeger_thrift_span_test.go | 394 + .../uber/jaeger-client-go/log/logger_test.go | 32 + .../uber/jaeger-client-go/logger_test.go | 46 + .../uber/jaeger-client-go/metrics_test.go | 56 + .../uber/jaeger-client-go/observer_test.go | 115 + .../uber/jaeger-client-go/propagation_test.go | 270 + .../uber/jaeger-client-go/reporter_test.go | 278 + .../jaeger-client-go/rpcmetrics/README.md | 5 + .../rpcmetrics/endpoints_test.go | 49 + .../rpcmetrics/metrics_test.go | 67 + .../rpcmetrics/normalizer_test.go | 40 + .../rpcmetrics/observer_test.go | 183 + .../uber/jaeger-client-go/sampler_test.go | 709 + .../uber/jaeger-client-go/span_test.go | 96 + .../uber/jaeger-client-go/thrift/.nocover | 0 .../uber/jaeger-client-go/tracer_test.go | 358 + .../jaeger-client-go/transport_udp_test.go | 227 + .../jaeger-client-go/utils/http_json_test.go | 64 + .../utils/rate_limiter_test.go | 81 + .../uber/jaeger-client-go/utils/utils_test.go | 97 + .../uber/jaeger-client-go/zipkin_test.go | 74 + .../zipkin_thrift_span_test.go | 335 + vendor/github.com/uber/jaeger-lib/.gitignore | 11 + vendor/github.com/uber/jaeger-lib/.travis.yml | 20 + .../uber/jaeger-lib/CONTRIBUTING.md | 56 + vendor/github.com/uber/jaeger-lib/Makefile | 83 + vendor/github.com/uber/jaeger-lib/README.md | 16 + vendor/github.com/uber/jaeger-lib/glide.lock | 63 + vendor/github.com/uber/jaeger-lib/glide.yaml | 11 + .../uber/jaeger-lib/metrics/local_test.go | 116 + .../uber/jaeger-lib/metrics/metrics_test.go | 89 + vendor/golang.org/x/net/.gitattributes | 10 + vendor/golang.org/x/net/.gitignore | 2 + vendor/golang.org/x/net/AUTHORS | 3 + vendor/golang.org/x/net/CONTRIBUTING.md | 31 + vendor/golang.org/x/net/CONTRIBUTORS | 3 + vendor/golang.org/x/net/README | 3 + vendor/golang.org/x/net/codereview.cfg | 1 + .../golang.org/x/net/context/context_test.go | 583 + .../x/net/context/withtimeout_test.go | 26 + vendor/golang.org/x/net/http2/.gitignore | 2 + vendor/golang.org/x/net/http2/Dockerfile | 51 + vendor/golang.org/x/net/http2/Makefile | 3 + vendor/golang.org/x/net/http2/README | 20 + .../golang.org/x/net/http2/databuffer_test.go | 155 + vendor/golang.org/x/net/http2/errors_test.go | 24 + vendor/golang.org/x/net/http2/flow_test.go | 53 + vendor/golang.org/x/net/http2/frame_test.go | 1191 ++ vendor/golang.org/x/net/http2/go18_test.go | 79 + vendor/golang.org/x/net/http2/gotrack_test.go | 33 + .../x/net/http2/hpack/encode_test.go | 386 + .../x/net/http2/hpack/hpack_test.go | 718 + .../x/net/http2/hpack/tables_test.go | 214 + vendor/golang.org/x/net/http2/http2_test.go | 199 + vendor/golang.org/x/net/http2/pipe_test.go | 109 + .../x/net/http2/server_push_test.go | 521 + vendor/golang.org/x/net/http2/server_test.go | 3731 +++++ .../golang.org/x/net/http2/transport_test.go | 2916 ++++ .../x/net/http2/writesched_priority_test.go | 541 + .../x/net/http2/writesched_random_test.go | 44 + .../golang.org/x/net/http2/writesched_test.go | 125 + vendor/golang.org/x/net/http2/z_spec_test.go | 356 + vendor/golang.org/x/net/idna/example_test.go | 65 + vendor/golang.org/x/net/idna/idna_test.go | 43 + vendor/golang.org/x/net/idna/punycode_test.go | 198 + .../internal/timeseries/timeseries_test.go | 170 + .../x/net/lex/httplex/httplex_test.go | 119 + .../golang.org/x/net/trace/histogram_test.go | 325 + vendor/golang.org/x/net/trace/trace_test.go | 178 + vendor/golang.org/x/sys/.gitattributes | 10 + vendor/golang.org/x/sys/.gitignore | 2 + vendor/golang.org/x/sys/AUTHORS | 3 + vendor/golang.org/x/sys/CONTRIBUTING.md | 26 + vendor/golang.org/x/sys/CONTRIBUTORS | 3 + vendor/golang.org/x/sys/README.md | 18 + vendor/golang.org/x/sys/codereview.cfg | 1 + vendor/golang.org/x/sys/unix/.gitignore | 2 + vendor/golang.org/x/sys/unix/README.md | 173 + vendor/golang.org/x/sys/unix/creds_test.go | 149 + .../golang.org/x/sys/unix/dev_darwin_test.go | 51 + .../x/sys/unix/dev_dragonfly_test.go | 50 + .../golang.org/x/sys/unix/dev_linux_test.go | 53 + .../golang.org/x/sys/unix/dev_netbsd_test.go | 50 + .../golang.org/x/sys/unix/dev_openbsd_test.go | 54 + .../golang.org/x/sys/unix/dev_solaris_test.go | 51 + vendor/golang.org/x/sys/unix/example_test.go | 19 + vendor/golang.org/x/sys/unix/export_test.go | 9 + vendor/golang.org/x/sys/unix/mkall.sh | 188 + vendor/golang.org/x/sys/unix/mkerrors.sh | 596 + vendor/golang.org/x/sys/unix/mksyscall.pl | 341 + .../x/sys/unix/mksyscall_solaris.pl | 289 + .../golang.org/x/sys/unix/mksysctl_openbsd.pl | 264 + .../golang.org/x/sys/unix/mksysnum_darwin.pl | 39 + .../x/sys/unix/mksysnum_dragonfly.pl | 50 + .../golang.org/x/sys/unix/mksysnum_freebsd.pl | 50 + .../golang.org/x/sys/unix/mksysnum_netbsd.pl | 58 + .../golang.org/x/sys/unix/mksysnum_openbsd.pl | 50 + .../golang.org/x/sys/unix/mmap_unix_test.go | 35 + vendor/golang.org/x/sys/unix/openbsd_test.go | 113 + .../golang.org/x/sys/unix/syscall_bsd_test.go | 93 + .../x/sys/unix/syscall_freebsd_test.go | 297 + .../x/sys/unix/syscall_linux_test.go | 373 + .../x/sys/unix/syscall_solaris_test.go | 55 + vendor/golang.org/x/sys/unix/syscall_test.go | 60 + .../x/sys/unix/syscall_unix_test.go | 614 + .../golang.org/x/sys/unix/timestruct_test.go | 54 + vendor/golang.org/x/text/.gitattributes | 10 + vendor/golang.org/x/text/.gitignore | 6 + vendor/golang.org/x/text/AUTHORS | 3 + vendor/golang.org/x/text/CONTRIBUTING.md | 31 + vendor/golang.org/x/text/CONTRIBUTORS | 3 + vendor/golang.org/x/text/README | 23 + vendor/golang.org/x/text/codereview.cfg | 1 + vendor/golang.org/x/text/doc.go | 13 + .../x/text/encoding/charmap/charmap_test.go | 258 + .../x/text/encoding/encoding_test.go | 290 + .../x/text/encoding/example_test.go | 42 + .../x/text/encoding/japanese/all_test.go | 248 + .../x/text/encoding/korean/all_test.go | 94 + .../encoding/simplifiedchinese/all_test.go | 143 + .../encoding/traditionalchinese/all_test.go | 114 + .../x/text/encoding/unicode/unicode_test.go | 499 + vendor/golang.org/x/text/gen.go | 292 + vendor/golang.org/x/text/internal/gen.go | 52 + vendor/golang.org/x/text/internal/gen_test.go | 38 + vendor/golang.org/x/text/internal/internal.go | 51 + .../x/text/internal/internal_test.go | 38 + vendor/golang.org/x/text/internal/match.go | 67 + .../golang.org/x/text/internal/match_test.go | 56 + vendor/golang.org/x/text/internal/tables.go | 116 + .../x/text/internal/triegen/data_test.go | 875 ++ .../internal/triegen/example_compact_test.go | 71 + .../x/text/internal/triegen/example_test.go | 148 + .../x/text/internal/triegen/gen_test.go | 68 + .../x/text/internal/ucd/example_test.go | 81 + .../x/text/internal/ucd/ucd_test.go | 105 + vendor/golang.org/x/text/runes/cond_test.go | 282 + .../golang.org/x/text/runes/example_test.go | 60 + vendor/golang.org/x/text/runes/runes_test.go | 664 + .../x/text/secure/bidirule/bench_test.go | 54 + .../x/text/secure/bidirule/bidirule_test.go | 825 + vendor/golang.org/x/text/secure/doc.go | 6 + .../x/text/transform/examples_test.go | 37 + .../x/text/transform/transform_test.go | 1317 ++ .../x/text/unicode/bidi/core_test.go | 224 + .../x/text/unicode/bidi/ranges_test.go | 53 + .../x/text/unicode/bidi/tables_test.go | 82 + .../x/text/unicode/cldr/cldr_test.go | 27 + .../x/text/unicode/cldr/collate_test.go | 275 + .../x/text/unicode/cldr/data_test.go | 186 + .../x/text/unicode/cldr/examples_test.go | 21 + .../x/text/unicode/cldr/resolve_test.go | 368 + .../x/text/unicode/cldr/slice_test.go | 175 + vendor/golang.org/x/text/unicode/doc.go | 8 + .../x/text/unicode/norm/composition_test.go | 130 + .../x/text/unicode/norm/example_iter_test.go | 82 + .../x/text/unicode/norm/example_test.go | 27 + .../x/text/unicode/norm/forminfo_test.go | 54 + .../x/text/unicode/norm/iter_test.go | 98 + .../x/text/unicode/norm/norm_test.go | 14 + .../x/text/unicode/norm/normalize_test.go | 1287 ++ .../x/text/unicode/norm/readwriter_test.go | 56 + .../x/text/unicode/norm/transform_test.go | 101 + .../x/text/unicode/norm/ucd_test.go | 275 + .../x/text/unicode/rangetable/merge_test.go | 184 + .../unicode/rangetable/rangetable_test.go | 55 + vendor/google.golang.org/genproto/.travis.yml | 12 + .../genproto/CONTRIBUTING.md | 27 + vendor/google.golang.org/genproto/README.md | 28 + .../googleapis/api/authorization_config.pb.go | 80 + .../googleapis/api/experimental.pb.go | 56 + vendor/google.golang.org/genproto/regen.go | 123 + vendor/google.golang.org/genproto/regen.sh | 77 + vendor/google.golang.org/grpc/.travis.yml | 24 + vendor/google.golang.org/grpc/AUTHORS | 1 + vendor/google.golang.org/grpc/CONTRIBUTING.md | 36 + vendor/google.golang.org/grpc/Makefile | 48 + vendor/google.golang.org/grpc/README.md | 45 + vendor/google.golang.org/grpc/backoff_test.go | 29 + .../balancer/roundrobin/roundrobin_test.go | 477 + .../grpc/balancer_switching_test.go | 443 + .../google.golang.org/grpc/balancer_test.go | 804 + vendor/google.golang.org/grpc/call_test.go | 292 + .../google.golang.org/grpc/clientconn_test.go | 677 + vendor/google.golang.org/grpc/codec_test.go | 32 + vendor/google.golang.org/grpc/codegen.sh | 17 + .../grpc/codes/codes_test.go | 64 + .../grpc/credentials/credentials_test.go | 206 + .../encoding/proto/proto_benchmark_test.go | 100 + .../grpc/encoding/proto/proto_test.go | 129 + .../grpclb/grpc_lb_v1/messages/messages.proto | 155 + .../grpc/grpclb/grpclb_test.go | 981 ++ .../grpc/grpclb_util_test.go | 219 + .../grpc/grpclog/loggerv2_test.go | 62 + .../grpc/health/grpc_health_v1/health.proto | 44 + .../google.golang.org/grpc/health/health.go | 72 + .../grpc/metadata/metadata_test.go | 251 + .../grpc/naming/dns_resolver_test.go | 315 + .../grpc/naming/go17_test.go | 42 + .../grpc/naming/go18_test.go | 41 + .../grpc/picker_wrapper_test.go | 160 + .../google.golang.org/grpc/pickfirst_test.go | 360 + vendor/google.golang.org/grpc/proxy_test.go | 182 + .../grpc/resolver/dns/dns_resolver_test.go | 936 ++ .../grpc/resolver/dns/go17_test.go | 50 + .../grpc/resolver/dns/go18_test.go | 49 + .../grpc/resolver_conn_wrapper_test.go | 113 + .../google.golang.org/grpc/rpc_util_test.go | 286 + vendor/google.golang.org/grpc/server_test.go | 136 + .../grpc/service_config_test.go | 386 + .../grpc/stats/stats_test.go | 1257 ++ .../grpc/status/status_test.go | 329 + .../google.golang.org/grpc/stickiness_test.go | 288 + .../grpc/transport/handler_server_test.go | 482 + .../grpc/transport/http_util_test.go | 176 + .../grpc/transport/transport_test.go | 2151 +++ vendor/google.golang.org/grpc/vet.sh | 93 + .../natefinch/lumberjack.v2/.gitignore | 23 + .../natefinch/lumberjack.v2/README.md | 174 + .../natefinch/lumberjack.v2/example_test.go | 18 + .../natefinch/lumberjack.v2/linux_test.go | 104 + .../lumberjack.v2/lumberjack_test.go | 690 + .../natefinch/lumberjack.v2/rotate_test.go | 27 + .../natefinch/lumberjack.v2/testing_test.go | 91 + x-server/conn.go | 2 +- x-server/server.go | 4 +- 1311 files changed, 188606 insertions(+), 10672 deletions(-) delete mode 100644 parser/bench_test.go delete mode 100644 parser/goyacc/main.go create mode 100644 types/parser_driver/value_expr.go create mode 100644 vendor/github.com/BurntSushi/toml/.gitignore create mode 100644 vendor/github.com/BurntSushi/toml/.travis.yml create mode 100644 vendor/github.com/BurntSushi/toml/COMPATIBLE create mode 100644 vendor/github.com/BurntSushi/toml/Makefile create mode 100644 vendor/github.com/BurntSushi/toml/README.md delete mode 100644 vendor/github.com/BurntSushi/toml/cmd/toml-test-decoder/COPYING delete mode 100644 vendor/github.com/BurntSushi/toml/cmd/toml-test-encoder/COPYING delete mode 100644 vendor/github.com/BurntSushi/toml/cmd/tomlv/COPYING create mode 100644 vendor/github.com/BurntSushi/toml/decode_test.go create mode 100644 vendor/github.com/BurntSushi/toml/encode_test.go create mode 100644 vendor/github.com/BurntSushi/toml/session.vim create mode 100644 vendor/github.com/apache/thrift/.clang-format create mode 100644 vendor/github.com/apache/thrift/.dockerignore create mode 100644 vendor/github.com/apache/thrift/.editorconfig create mode 100644 vendor/github.com/apache/thrift/.gitattributes create mode 100644 vendor/github.com/apache/thrift/.gitignore create mode 100644 vendor/github.com/apache/thrift/.travis.yml create mode 100644 vendor/github.com/apache/thrift/CHANGES create mode 100644 vendor/github.com/apache/thrift/CMakeLists.txt create mode 100644 vendor/github.com/apache/thrift/CONTRIBUTING.md create mode 100644 vendor/github.com/apache/thrift/Dockerfile create mode 100644 vendor/github.com/apache/thrift/Makefile.am create mode 100644 vendor/github.com/apache/thrift/README.md create mode 100644 vendor/github.com/apache/thrift/Thrift.podspec create mode 100644 vendor/github.com/apache/thrift/appveyor.yml create mode 100644 vendor/github.com/apache/thrift/bootstrap.sh create mode 100644 vendor/github.com/apache/thrift/bower.json create mode 100644 vendor/github.com/apache/thrift/cleanup.sh create mode 100644 vendor/github.com/apache/thrift/composer.json create mode 100644 vendor/github.com/apache/thrift/configure.ac delete mode 100644 vendor/github.com/apache/thrift/contrib/fb303/LICENSE create mode 100644 vendor/github.com/apache/thrift/doap.rdf create mode 100644 vendor/github.com/apache/thrift/lib/Makefile.am delete mode 100644 vendor/github.com/apache/thrift/lib/dart/LICENSE_HEADER create mode 100644 vendor/github.com/apache/thrift/lib/go/Makefile.am create mode 100644 vendor/github.com/apache/thrift/lib/go/README.md create mode 100644 vendor/github.com/apache/thrift/lib/go/coding_standards.md create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/application_exception_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/binary_protocol_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/buffered_transport_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/compact_protocol_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/exception_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/framed_transport_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/http_client_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/iostream_transport_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/json_protocol_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/lowlevel_benchmarks_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/memory_buffer_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/protocol_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/rich_transport_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/serializer_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/serializer_types_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/server_socket_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/server_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/simple_json_protocol_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/transport_exception_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/transport_test.go create mode 100644 vendor/github.com/apache/thrift/lib/go/thrift/zlib_transport_test.go create mode 100644 vendor/github.com/apache/thrift/package.json create mode 100644 vendor/github.com/apache/thrift/sonar-project.properties create mode 100644 vendor/github.com/beorn7/perks/.gitignore create mode 100644 vendor/github.com/beorn7/perks/README.md create mode 100644 vendor/github.com/beorn7/perks/quantile/bench_test.go create mode 100644 vendor/github.com/beorn7/perks/quantile/example_test.go create mode 100644 vendor/github.com/beorn7/perks/quantile/exampledata.txt create mode 100644 vendor/github.com/beorn7/perks/quantile/stream_test.go create mode 100644 vendor/github.com/blacktear23/go-proxyprotocol/.travis.yml create mode 100644 vendor/github.com/blacktear23/go-proxyprotocol/README.md create mode 100644 vendor/github.com/blacktear23/go-proxyprotocol/proxy_protocol_test.go create mode 100644 vendor/github.com/codahale/hdrhistogram/.travis.yml create mode 100644 vendor/github.com/codahale/hdrhistogram/README.md create mode 100644 vendor/github.com/codahale/hdrhistogram/hdr_test.go create mode 100644 vendor/github.com/codahale/hdrhistogram/window_test.go create mode 100644 vendor/github.com/coreos/etcd/.dockerignore create mode 100644 vendor/github.com/coreos/etcd/.gitignore create mode 100644 vendor/github.com/coreos/etcd/.godir create mode 100644 vendor/github.com/coreos/etcd/.header create mode 100644 vendor/github.com/coreos/etcd/.semaphore.sh create mode 100644 vendor/github.com/coreos/etcd/.travis.yml create mode 100644 vendor/github.com/coreos/etcd/CONTRIBUTING.md create mode 100644 vendor/github.com/coreos/etcd/DCO create mode 100644 vendor/github.com/coreos/etcd/Dockerfile create mode 100644 vendor/github.com/coreos/etcd/Dockerfile-release create mode 100644 vendor/github.com/coreos/etcd/Dockerfile-release.arm64 create mode 100644 vendor/github.com/coreos/etcd/Dockerfile-release.ppc64le create mode 100644 vendor/github.com/coreos/etcd/Dockerfile-test create mode 100644 vendor/github.com/coreos/etcd/MAINTAINERS create mode 100644 vendor/github.com/coreos/etcd/NEWS create mode 100644 vendor/github.com/coreos/etcd/Procfile create mode 100644 vendor/github.com/coreos/etcd/README.md create mode 100644 vendor/github.com/coreos/etcd/ROADMAP.md create mode 100644 vendor/github.com/coreos/etcd/V2Procfile create mode 100644 vendor/github.com/coreos/etcd/auth/authpb/auth.proto create mode 100644 vendor/github.com/coreos/etcd/auth/doc.go create mode 100644 vendor/github.com/coreos/etcd/auth/jwt.go create mode 100644 vendor/github.com/coreos/etcd/auth/range_perm_cache.go create mode 100644 vendor/github.com/coreos/etcd/auth/range_perm_cache_test.go create mode 100644 vendor/github.com/coreos/etcd/auth/simple_token.go create mode 100644 vendor/github.com/coreos/etcd/auth/store.go create mode 100644 vendor/github.com/coreos/etcd/auth/store_test.go create mode 100644 vendor/github.com/coreos/etcd/bill-of-materials.json create mode 100644 vendor/github.com/coreos/etcd/bill-of-materials.override.json create mode 100644 vendor/github.com/coreos/etcd/build create mode 100644 vendor/github.com/coreos/etcd/build.bat create mode 100644 vendor/github.com/coreos/etcd/build.ps1 create mode 100644 vendor/github.com/coreos/etcd/client/README.md create mode 100644 vendor/github.com/coreos/etcd/client/auth_role.go create mode 100644 vendor/github.com/coreos/etcd/client/auth_user.go create mode 100644 vendor/github.com/coreos/etcd/client/cancelreq.go create mode 100644 vendor/github.com/coreos/etcd/client/client.go create mode 100644 vendor/github.com/coreos/etcd/client/client_test.go create mode 100644 vendor/github.com/coreos/etcd/client/cluster_error.go create mode 100644 vendor/github.com/coreos/etcd/client/curl.go create mode 100644 vendor/github.com/coreos/etcd/client/discover.go create mode 100644 vendor/github.com/coreos/etcd/client/doc.go create mode 100644 vendor/github.com/coreos/etcd/client/fake_transport_test.go create mode 100644 vendor/github.com/coreos/etcd/client/keys.generated.go create mode 100644 vendor/github.com/coreos/etcd/client/keys.go create mode 100644 vendor/github.com/coreos/etcd/client/keys_bench_test.go create mode 100644 vendor/github.com/coreos/etcd/client/keys_test.go create mode 100644 vendor/github.com/coreos/etcd/client/members.go create mode 100644 vendor/github.com/coreos/etcd/client/members_test.go create mode 100644 vendor/github.com/coreos/etcd/client/util.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/README.md create mode 100644 vendor/github.com/coreos/etcd/clientv3/client_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/compact_op_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_cluster_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_kv_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_lease_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_maintenence_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_metrics_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/example_watch_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/main_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/op_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/txn_test.go create mode 100644 vendor/github.com/coreos/etcd/clientv3/watch_test.go delete mode 120000 vendor/github.com/coreos/etcd/cmd/etcd delete mode 120000 vendor/github.com/coreos/etcd/cmd/etcdctl delete mode 120000 vendor/github.com/coreos/etcd/cmd/tools create mode 100644 vendor/github.com/coreos/etcd/cover create mode 100644 vendor/github.com/coreos/etcd/etcd.conf.yml.sample create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/capability.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/cluster.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/doc.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/auth.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/codec.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/grpc.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/header.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/interceptor.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/key.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/lease.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/maintenance.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/member.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/metrics.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/quota.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes/error_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/util.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/api/v3rpc/watch.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/apply.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/apply_auth.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/apply_v2.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/backend.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/cluster_util.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/cluster_util_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/config.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/config_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/consistent_index.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/consistent_index_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/doc.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/errors.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/etcdserver.proto create mode 100644 vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/raft_internal.proto create mode 100644 vendor/github.com/coreos/etcd/etcdserver/etcdserverpb/rpc.proto create mode 100644 vendor/github.com/coreos/etcd/etcdserver/metrics.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/quota.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/raft.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/raft_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/server.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/server_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/snapshot_merge.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/storage.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/util.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/util_test.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/v2_server.go create mode 100644 vendor/github.com/coreos/etcd/etcdserver/v3_server.go create mode 100644 vendor/github.com/coreos/etcd/glide.lock create mode 100644 vendor/github.com/coreos/etcd/glide.yaml create mode 100644 vendor/github.com/coreos/etcd/main.go create mode 100644 vendor/github.com/coreos/etcd/main_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/doc.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/index.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/index_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/key_index.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/key_index_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kv.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kv_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kv_view.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore_bench_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore_compaction.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore_compaction_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/kvstore_txn.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/metrics.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/metrics_txn.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/mvccpb/kv.proto create mode 100644 vendor/github.com/coreos/etcd/mvcc/revision.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/revision_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/util.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watchable_store.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watchable_store_bench_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watchable_store_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watchable_store_txn.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watcher.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watcher_bench_test.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watcher_group.go create mode 100644 vendor/github.com/coreos/etcd/mvcc/watcher_test.go create mode 100644 vendor/github.com/coreos/etcd/pkg/README.md create mode 100644 vendor/github.com/coreos/etcd/pkg/types/id_test.go create mode 100644 vendor/github.com/coreos/etcd/pkg/types/set_test.go create mode 100644 vendor/github.com/coreos/etcd/pkg/types/slice_test.go create mode 100644 vendor/github.com/coreos/etcd/pkg/types/urls_test.go create mode 100644 vendor/github.com/coreos/etcd/pkg/types/urlsmap_test.go create mode 100644 vendor/github.com/coreos/etcd/test delete mode 100644 vendor/github.com/cznic/golex/lex/api.go delete mode 100644 vendor/github.com/cznic/golex/lex/doc.go create mode 100644 vendor/github.com/cznic/mathutil/AUTHORS create mode 100644 vendor/github.com/cznic/mathutil/CONTRIBUTORS create mode 100644 vendor/github.com/cznic/mathutil/Makefile create mode 100644 vendor/github.com/cznic/mathutil/README create mode 100644 vendor/github.com/cznic/mathutil/all_test.go delete mode 100644 vendor/github.com/cznic/mathutil/mersenne/LICENSE create mode 100644 vendor/github.com/cznic/mathutil/nist-sts-2-1-1-report delete mode 100644 vendor/github.com/cznic/parser/nquads/LICENSE delete mode 100644 vendor/github.com/cznic/parser/yacc/LICENSE delete mode 100644 vendor/github.com/cznic/parser/yacc/api.go delete mode 100644 vendor/github.com/cznic/parser/yacc/ast.go delete mode 100644 vendor/github.com/cznic/parser/yacc/generate.go delete mode 100644 vendor/github.com/cznic/parser/yacc/goscanner.go delete mode 100644 vendor/github.com/cznic/parser/yacc/lexer.go delete mode 100644 vendor/github.com/cznic/parser/yacc/parser.go delete mode 100644 vendor/github.com/cznic/parser/yacc/scanner.go create mode 100644 vendor/github.com/cznic/sortutil/AUTHORS create mode 100644 vendor/github.com/cznic/sortutil/CONTRIBUTORS create mode 100644 vendor/github.com/cznic/sortutil/Makefile create mode 100644 vendor/github.com/cznic/sortutil/README create mode 100644 vendor/github.com/cznic/sortutil/all_test.go delete mode 100644 vendor/github.com/cznic/strutil/LICENSE delete mode 100644 vendor/github.com/cznic/strutil/strutil.go delete mode 100644 vendor/github.com/cznic/y/api.go delete mode 100644 vendor/github.com/cznic/y/y.go create mode 100644 vendor/github.com/etcd-io/gofail/.travis.yml create mode 100644 vendor/github.com/etcd-io/gofail/DCO create mode 100644 vendor/github.com/etcd-io/gofail/Makefile create mode 100644 vendor/github.com/etcd-io/gofail/README.md create mode 100644 vendor/github.com/etcd-io/gofail/code-of-conduct.md create mode 100644 vendor/github.com/etcd-io/gofail/gofail.go create mode 100644 vendor/github.com/etcd-io/gofail/runtime/terms_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/.gitignore create mode 100644 vendor/github.com/go-sql-driver/mysql/.travis.yml create mode 100644 vendor/github.com/go-sql-driver/mysql/AUTHORS create mode 100644 vendor/github.com/go-sql-driver/mysql/CHANGELOG.md create mode 100644 vendor/github.com/go-sql-driver/mysql/CONTRIBUTING.md create mode 100644 vendor/github.com/go-sql-driver/mysql/README.md create mode 100644 vendor/github.com/go-sql-driver/mysql/benchmark_go18_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/benchmark_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/connection_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/driver_go18_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/driver_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/dsn_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/errors_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/packets_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/utils_go18_test.go create mode 100644 vendor/github.com/go-sql-driver/mysql/utils_test.go create mode 100644 vendor/github.com/gogo/protobuf/.gitignore create mode 100644 vendor/github.com/gogo/protobuf/.mailmap create mode 100644 vendor/github.com/gogo/protobuf/.travis.yml create mode 100644 vendor/github.com/gogo/protobuf/AUTHORS create mode 100644 vendor/github.com/gogo/protobuf/CONTRIBUTORS create mode 100644 vendor/github.com/gogo/protobuf/GOLANG_CONTRIBUTORS create mode 100644 vendor/github.com/gogo/protobuf/Makefile create mode 100644 vendor/github.com/gogo/protobuf/README create mode 100644 vendor/github.com/gogo/protobuf/Readme.md create mode 100644 vendor/github.com/gogo/protobuf/bench.md create mode 100644 vendor/github.com/gogo/protobuf/custom_types.md create mode 100644 vendor/github.com/gogo/protobuf/extensions.md create mode 100644 vendor/github.com/gogo/protobuf/install-protobuf.sh create mode 100644 vendor/github.com/gogo/protobuf/proto/Makefile create mode 100644 vendor/github.com/gogo/protobuf/proto/all_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/any_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/clone_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/decode_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/discard_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/encode_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/equal_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/extensions_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/map_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/message_set_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/proto3_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/size2_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/size_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/text_parser_test.go create mode 100644 vendor/github.com/gogo/protobuf/proto/text_test.go create mode 100644 vendor/github.com/golang/protobuf/.gitignore create mode 100644 vendor/github.com/golang/protobuf/.travis.yml create mode 100644 vendor/github.com/golang/protobuf/AUTHORS create mode 100644 vendor/github.com/golang/protobuf/CONTRIBUTORS create mode 100644 vendor/github.com/golang/protobuf/Makefile create mode 100644 vendor/github.com/golang/protobuf/README.md create mode 100644 vendor/github.com/golang/protobuf/jsonpb/jsonpb_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/all_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/any_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/clone_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/decode_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/discard_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/encode_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/equal_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/extensions_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/map_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/message_set_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/proto3_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/size2_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/size_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/text_parser_test.go create mode 100644 vendor/github.com/golang/protobuf/proto/text_test.go create mode 100644 vendor/github.com/golang/protobuf/protoc-gen-go/descriptor/descriptor.proto create mode 100644 vendor/github.com/golang/protobuf/protoc-gen-go/doc.go create mode 100644 vendor/github.com/golang/protobuf/protoc-gen-go/golden_test.go create mode 100644 vendor/github.com/golang/protobuf/protoc-gen-go/link_grpc.go create mode 100644 vendor/github.com/golang/protobuf/protoc-gen-go/main.go create mode 100644 vendor/github.com/golang/protobuf/ptypes/any/any.proto create mode 100644 vendor/github.com/golang/protobuf/ptypes/any_test.go create mode 100644 vendor/github.com/golang/protobuf/ptypes/duration/duration.proto create mode 100644 vendor/github.com/golang/protobuf/ptypes/duration_test.go create mode 100644 vendor/github.com/golang/protobuf/ptypes/struct/struct.proto create mode 100644 vendor/github.com/golang/protobuf/ptypes/timestamp/timestamp.proto create mode 100644 vendor/github.com/golang/protobuf/ptypes/timestamp_test.go create mode 100644 vendor/github.com/golang/protobuf/regenerate.sh create mode 100644 vendor/github.com/golang/snappy/AUTHORS create mode 100644 vendor/github.com/golang/snappy/CONTRIBUTORS create mode 100644 vendor/github.com/golang/snappy/README create mode 100644 vendor/github.com/golang/snappy/snappy_test.go create mode 100644 vendor/github.com/google/btree/.travis.yml create mode 100644 vendor/github.com/google/btree/README.md create mode 100644 vendor/github.com/google/btree/btree_test.go create mode 100644 vendor/github.com/gorilla/context/.travis.yml create mode 100644 vendor/github.com/gorilla/context/README.md create mode 100644 vendor/github.com/gorilla/context/context_test.go create mode 100644 vendor/github.com/gorilla/mux/.travis.yml create mode 100644 vendor/github.com/gorilla/mux/README.md create mode 100644 vendor/github.com/gorilla/mux/bench_test.go create mode 100644 vendor/github.com/gorilla/mux/context_gorilla_test.go create mode 100644 vendor/github.com/gorilla/mux/context_native_test.go create mode 100644 vendor/github.com/gorilla/mux/mux_test.go create mode 100644 vendor/github.com/gorilla/mux/old_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/.gitignore create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/.travis.yml create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/DOC.md create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/README.md create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/checkup.sh create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/fixup.sh create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/test_all.sh create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/DOC.md create mode 120000 vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/README.md create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing/interceptors_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/DOC.md create mode 120000 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/README.md create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/util/metautils/nicemd_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.gitignore create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.travis.yml create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-prometheus/README.md create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_test.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_test.go create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/.travis.yml create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/Makefile.TRAVIS create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/README.md create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/.gitignore create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/Makefile create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/all_test.go create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/decode_test.go create mode 100644 vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/encode_test.go create mode 100644 vendor/github.com/ngaut/pools/id_pool_test.go create mode 100644 vendor/github.com/ngaut/pools/numbered_test.go create mode 100644 vendor/github.com/ngaut/pools/resource_pool_test.go create mode 100644 vendor/github.com/ngaut/pools/roundrobin_test.go rename vendor/github.com/{cznic/y/LICENSE => ngaut/pools/vitess_license} (83%) create mode 100644 vendor/github.com/ngaut/sync2/atomic_test.go create mode 100644 vendor/github.com/ngaut/sync2/cond_test.go create mode 100644 vendor/github.com/ngaut/sync2/semaphore_test.go create mode 100644 vendor/github.com/ngaut/sync2/service_manager_test.go rename vendor/github.com/{cznic/golex/LICENSE => ngaut/sync2/vitess_license} (83%) create mode 100644 vendor/github.com/opentracing/basictracer-go/.gitignore create mode 100644 vendor/github.com/opentracing/basictracer-go/.travis.yml create mode 100644 vendor/github.com/opentracing/basictracer-go/Makefile create mode 100644 vendor/github.com/opentracing/basictracer-go/README.md create mode 100644 vendor/github.com/opentracing/basictracer-go/bench_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/concurrency_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/propagation_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/recorder_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/span_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/testutil_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/wire/carrier_test.go create mode 100644 vendor/github.com/opentracing/basictracer-go/wire/wire.proto create mode 100644 vendor/github.com/opentracing/opentracing-go/.gitignore create mode 100644 vendor/github.com/opentracing/opentracing-go/.travis.yml create mode 100644 vendor/github.com/opentracing/opentracing-go/CHANGELOG.md create mode 100644 vendor/github.com/opentracing/opentracing-go/Makefile create mode 100644 vendor/github.com/opentracing/opentracing-go/README.md create mode 100644 vendor/github.com/opentracing/opentracing-go/ext/tags_test.go create mode 100644 vendor/github.com/opentracing/opentracing-go/gocontext_test.go create mode 100644 vendor/github.com/opentracing/opentracing-go/log/field_test.go create mode 100644 vendor/github.com/opentracing/opentracing-go/options_test.go create mode 100644 vendor/github.com/opentracing/opentracing-go/propagation_test.go create mode 100644 vendor/github.com/opentracing/opentracing-go/testtracer_test.go create mode 100644 vendor/github.com/petar/GoLLRB/.gitignore create mode 100644 vendor/github.com/petar/GoLLRB/AUTHORS create mode 100644 vendor/github.com/petar/GoLLRB/README.md create mode 100644 vendor/github.com/petar/GoLLRB/llrb/iterator_test.go create mode 100644 vendor/github.com/petar/GoLLRB/llrb/llrb_test.go create mode 100644 vendor/github.com/pingcap/check/.gitignore create mode 100644 vendor/github.com/pingcap/check/README.md create mode 100644 vendor/github.com/pingcap/check/TODO create mode 100644 vendor/github.com/pingcap/check/benchmark_test.go create mode 100644 vendor/github.com/pingcap/check/bootstrap_test.go create mode 100644 vendor/github.com/pingcap/check/check.v1_license create mode 100644 vendor/github.com/pingcap/check/check_test.go create mode 100644 vendor/github.com/pingcap/check/checkers2_test.go create mode 100644 vendor/github.com/pingcap/check/checkers_test.go create mode 100644 vendor/github.com/pingcap/check/export_test.go create mode 100644 vendor/github.com/pingcap/check/fixture_test.go create mode 100644 vendor/github.com/pingcap/check/foundation_test.go create mode 100644 vendor/github.com/pingcap/check/godropbox_license create mode 100644 vendor/github.com/pingcap/check/helpers_test.go create mode 100644 vendor/github.com/pingcap/check/printer_test.go create mode 100644 vendor/github.com/pingcap/check/run_test.go create mode 100644 vendor/github.com/pingcap/errors/.gitignore create mode 100644 vendor/github.com/pingcap/errors/.travis.yml create mode 100644 vendor/github.com/pingcap/errors/LICENSE create mode 100644 vendor/github.com/pingcap/errors/README.md create mode 100644 vendor/github.com/pingcap/errors/appveyor.yml create mode 100644 vendor/github.com/pingcap/errors/bench_test.go create mode 100644 vendor/github.com/pingcap/errors/errors.go create mode 100644 vendor/github.com/pingcap/errors/errors_test.go create mode 100644 vendor/github.com/pingcap/errors/example_test.go create mode 100644 vendor/github.com/pingcap/errors/format_test.go create mode 100644 vendor/github.com/pingcap/errors/group.go create mode 100644 vendor/github.com/pingcap/errors/juju_adaptor.go create mode 100644 vendor/github.com/pingcap/errors/stack.go create mode 100644 vendor/github.com/pingcap/errors/stack_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/.travis.yml create mode 100644 vendor/github.com/pingcap/goleveldb/README.md create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/batch_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/bench_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/cache/bench_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/cache/cache_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/corrupt_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/db_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/external_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/filter/bloom_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/iterator/array_iter_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/iterator/indexed_iter_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/iterator/iter_suite_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/iterator/merged_iter_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/journal/journal_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/key_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/leveldb_suite_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/memdb/bench_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/memdb/memdb_suite_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/memdb/memdb_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/session_record_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/storage/file_storage_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/storage/mem_storage_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/table/block_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/table/table_suite_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/table/table_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/testutil_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/util/buffer_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/util/hash_test.go create mode 100644 vendor/github.com/pingcap/goleveldb/leveldb/version_test.go create mode 100644 vendor/github.com/pingcap/kvproto/.gitattributes create mode 100644 vendor/github.com/pingcap/kvproto/.gitignore create mode 100644 vendor/github.com/pingcap/kvproto/Cargo.toml create mode 100644 vendor/github.com/pingcap/kvproto/Makefile create mode 100644 vendor/github.com/pingcap/kvproto/README.md create mode 100644 vendor/github.com/pingcap/kvproto/common.sh create mode 100644 vendor/github.com/pingcap/kvproto/generate_go.sh create mode 100644 vendor/github.com/pingcap/kvproto/generate_rust.sh create mode 100644 vendor/github.com/pingcap/kvproto/glide.lock create mode 100644 vendor/github.com/pingcap/kvproto/glide.yaml create mode 100644 vendor/github.com/pingcap/parser/.gitignore rename vendor/github.com/{apache/thrift/lib/hs => pingcap/parser}/LICENSE (99%) create mode 100644 vendor/github.com/pingcap/parser/Makefile create mode 100644 vendor/github.com/pingcap/parser/README.md create mode 100644 vendor/github.com/pingcap/parser/ast/ast.go create mode 100644 vendor/github.com/pingcap/parser/ast/base.go create mode 100644 vendor/github.com/pingcap/parser/ast/ddl.go create mode 100644 vendor/github.com/pingcap/parser/ast/ddl_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/dml.go create mode 100644 vendor/github.com/pingcap/parser/ast/dml_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/expressions.go create mode 100644 vendor/github.com/pingcap/parser/ast/expressions_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/flag.go create mode 100644 vendor/github.com/pingcap/parser/ast/flag_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/format_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/functions.go create mode 100644 vendor/github.com/pingcap/parser/ast/functions_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/misc.go create mode 100644 vendor/github.com/pingcap/parser/ast/misc_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/read_only_checker.go create mode 100644 vendor/github.com/pingcap/parser/ast/read_only_checker_test.go create mode 100644 vendor/github.com/pingcap/parser/ast/stats.go create mode 100644 vendor/github.com/pingcap/parser/auth/auth.go create mode 100644 vendor/github.com/pingcap/parser/auth/auth_test.go create mode 100644 vendor/github.com/pingcap/parser/bench_test.go create mode 100644 vendor/github.com/pingcap/parser/charset/charset.go create mode 100644 vendor/github.com/pingcap/parser/charset/charset_test.go create mode 100644 vendor/github.com/pingcap/parser/charset/encoding_table.go create mode 100644 vendor/github.com/pingcap/parser/circle.yml rename {parser => vendor/github.com/pingcap/parser}/consistent_test.go (97%) create mode 100644 vendor/github.com/pingcap/parser/format/format.go create mode 100644 vendor/github.com/pingcap/parser/format/format_test.go create mode 100644 vendor/github.com/pingcap/parser/go.mod1 create mode 100644 vendor/github.com/pingcap/parser/go.sum1 rename {parser => vendor/github.com/pingcap/parser}/lexer.go (97%) rename {parser => vendor/github.com/pingcap/parser}/lexer_test.go (95%) rename {parser => vendor/github.com/pingcap/parser}/misc.go (94%) create mode 100644 vendor/github.com/pingcap/parser/model/ddl.go create mode 100644 vendor/github.com/pingcap/parser/model/flags.go create mode 100644 vendor/github.com/pingcap/parser/model/model.go create mode 100644 vendor/github.com/pingcap/parser/model/model_test.go create mode 100644 vendor/github.com/pingcap/parser/mysql/charset.go create mode 100644 vendor/github.com/pingcap/parser/mysql/const.go create mode 100644 vendor/github.com/pingcap/parser/mysql/errcode.go create mode 100644 vendor/github.com/pingcap/parser/mysql/errname.go create mode 100644 vendor/github.com/pingcap/parser/mysql/error.go create mode 100644 vendor/github.com/pingcap/parser/mysql/error_test.go create mode 100644 vendor/github.com/pingcap/parser/mysql/locale_format.go create mode 100644 vendor/github.com/pingcap/parser/mysql/state.go create mode 100644 vendor/github.com/pingcap/parser/mysql/type.go create mode 100644 vendor/github.com/pingcap/parser/mysql/type_test.go create mode 100644 vendor/github.com/pingcap/parser/mysql/util.go rename {parser => vendor/github.com/pingcap/parser}/opcode/opcode.go (100%) rename {parser => vendor/github.com/pingcap/parser}/opcode/opcode_test.go (100%) create mode 100644 vendor/github.com/pingcap/parser/parser.go rename {parser => vendor/github.com/pingcap/parser}/parser.y (90%) create mode 100644 vendor/github.com/pingcap/parser/parser_example_test.go rename {parser => vendor/github.com/pingcap/parser}/parser_test.go (93%) create mode 100644 vendor/github.com/pingcap/parser/terror/terror.go create mode 100644 vendor/github.com/pingcap/parser/terror/terror_test.go create mode 100644 vendor/github.com/pingcap/parser/test.sh create mode 100644 vendor/github.com/pingcap/parser/types/etc.go create mode 100644 vendor/github.com/pingcap/parser/types/eval_type.go create mode 100644 vendor/github.com/pingcap/parser/types/field_type.go rename {parser => vendor/github.com/pingcap/parser}/yy_parser.go (91%) create mode 100644 vendor/github.com/pingcap/pd/.gitignore create mode 100644 vendor/github.com/pingcap/pd/.travis.yml create mode 100644 vendor/github.com/pingcap/pd/CHANGELOG.md create mode 100644 vendor/github.com/pingcap/pd/CONTRIBUTING.md create mode 100644 vendor/github.com/pingcap/pd/Dockerfile create mode 100644 vendor/github.com/pingcap/pd/Jenkinsfile create mode 100644 vendor/github.com/pingcap/pd/Makefile create mode 100644 vendor/github.com/pingcap/pd/README.md create mode 100644 vendor/github.com/pingcap/pd/circle.yml create mode 100644 vendor/github.com/pingcap/pd/client/client_test.go create mode 100644 vendor/github.com/pingcap/pd/go.mod create mode 100644 vendor/github.com/pingcap/pd/go.sum create mode 100644 vendor/github.com/pingcap/pd/revive.toml create mode 100644 vendor/github.com/pingcap/pd/tools.json create mode 100644 vendor/github.com/pingcap/tidb-tools/.gitignore create mode 100644 vendor/github.com/pingcap/tidb-tools/Jenkinsfile create mode 100644 vendor/github.com/pingcap/tidb-tools/Makefile create mode 100644 vendor/github.com/pingcap/tidb-tools/README.md create mode 100644 vendor/github.com/pingcap/tidb-tools/glide.lock create mode 100644 vendor/github.com/pingcap/tidb-tools/glide.yaml create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/etcd/etcd_test.go create mode 100644 vendor/github.com/pingcap/tidb-tools/pkg/utils/urls_test.go create mode 100644 vendor/github.com/pingcap/tidb-tools/tidb-binlog/pump_client/client_test.go create mode 100644 vendor/github.com/pingcap/tipb/.gitignore create mode 100644 vendor/github.com/pingcap/tipb/Cargo.toml create mode 100644 vendor/github.com/pingcap/tipb/Makefile create mode 100644 vendor/github.com/pingcap/tipb/README.md create mode 100644 vendor/github.com/pingcap/tipb/_help.sh create mode 100644 vendor/github.com/pingcap/tipb/generate-binlog.sh create mode 100644 vendor/github.com/pingcap/tipb/generate-go.sh create mode 100644 vendor/github.com/pingcap/tipb/generate-rust.sh create mode 100644 vendor/github.com/pingcap/tipb/sharedbytes/sharedbytes_test.go create mode 100644 vendor/github.com/pkg/errors/.gitignore create mode 100644 vendor/github.com/pkg/errors/.travis.yml create mode 100644 vendor/github.com/pkg/errors/README.md create mode 100644 vendor/github.com/pkg/errors/appveyor.yml create mode 100644 vendor/github.com/pkg/errors/bench_test.go create mode 100644 vendor/github.com/pkg/errors/errors_test.go create mode 100644 vendor/github.com/pkg/errors/example_test.go create mode 100644 vendor/github.com/pkg/errors/format_test.go create mode 100644 vendor/github.com/pkg/errors/stack_test.go create mode 100644 vendor/github.com/prometheus/client_golang/.gitignore create mode 100644 vendor/github.com/prometheus/client_golang/.travis.yml create mode 100644 vendor/github.com/prometheus/client_golang/AUTHORS.md create mode 100644 vendor/github.com/prometheus/client_golang/CHANGELOG.md create mode 100644 vendor/github.com/prometheus/client_golang/CONTRIBUTING.md create mode 100644 vendor/github.com/prometheus/client_golang/README.md create mode 100644 vendor/github.com/prometheus/client_golang/VERSION create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/.gitignore create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/README.md create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/benchmark_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/counter_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/example_clustermanager_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/examples_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/expvar_collector_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/gauge_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/go_collector_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/histogram_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/http_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/metric_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/process_collector_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/push/examples_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/push/push_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/registry_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/summary_test.go create mode 100644 vendor/github.com/prometheus/client_golang/prometheus/vec_test.go create mode 100644 vendor/github.com/prometheus/client_model/.gitignore create mode 100644 vendor/github.com/prometheus/client_model/AUTHORS.md create mode 100644 vendor/github.com/prometheus/client_model/CONTRIBUTING.md create mode 100644 vendor/github.com/prometheus/client_model/Makefile create mode 100644 vendor/github.com/prometheus/client_model/README.md create mode 100644 vendor/github.com/prometheus/client_model/metrics.proto create mode 100644 vendor/github.com/prometheus/client_model/pom.xml create mode 100644 vendor/github.com/prometheus/client_model/setup.py create mode 100644 vendor/github.com/prometheus/common/.travis.yml create mode 100644 vendor/github.com/prometheus/common/AUTHORS.md create mode 100644 vendor/github.com/prometheus/common/CONTRIBUTING.md create mode 100644 vendor/github.com/prometheus/common/README.md create mode 100644 vendor/github.com/prometheus/common/expfmt/bench_test.go create mode 100644 vendor/github.com/prometheus/common/expfmt/decode_test.go create mode 100644 vendor/github.com/prometheus/common/expfmt/text_create_test.go create mode 100644 vendor/github.com/prometheus/common/expfmt/text_parse_test.go create mode 100644 vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/README.txt create mode 100644 vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/autoneg_test.go create mode 100644 vendor/github.com/prometheus/common/model/alert_test.go create mode 100644 vendor/github.com/prometheus/common/model/labels_test.go create mode 100644 vendor/github.com/prometheus/common/model/metric_test.go create mode 100644 vendor/github.com/prometheus/common/model/signature_test.go create mode 100644 vendor/github.com/prometheus/common/model/silence_test.go create mode 100644 vendor/github.com/prometheus/common/model/time_test.go create mode 100644 vendor/github.com/prometheus/common/model/value_test.go create mode 100644 vendor/github.com/prometheus/procfs/.travis.yml create mode 100644 vendor/github.com/prometheus/procfs/AUTHORS.md create mode 100644 vendor/github.com/prometheus/procfs/CONTRIBUTING.md create mode 100644 vendor/github.com/prometheus/procfs/Makefile create mode 100644 vendor/github.com/prometheus/procfs/README.md create mode 100644 vendor/github.com/prometheus/procfs/fs_test.go create mode 100644 vendor/github.com/prometheus/procfs/ipvs_test.go create mode 100644 vendor/github.com/prometheus/procfs/mdstat_test.go create mode 100644 vendor/github.com/prometheus/procfs/proc_io_test.go create mode 100644 vendor/github.com/prometheus/procfs/proc_limits_test.go create mode 100644 vendor/github.com/prometheus/procfs/proc_stat_test.go create mode 100644 vendor/github.com/prometheus/procfs/proc_test.go create mode 100644 vendor/github.com/prometheus/procfs/stat_test.go create mode 100644 vendor/github.com/sirupsen/logrus/.gitignore create mode 100644 vendor/github.com/sirupsen/logrus/.travis.yml create mode 100644 vendor/github.com/sirupsen/logrus/CHANGELOG.md create mode 100644 vendor/github.com/sirupsen/logrus/README.md create mode 100644 vendor/github.com/sirupsen/logrus/alt_exit_test.go create mode 100644 vendor/github.com/sirupsen/logrus/entry_test.go create mode 100644 vendor/github.com/sirupsen/logrus/formatter_bench_test.go create mode 100644 vendor/github.com/sirupsen/logrus/hook_test.go create mode 100644 vendor/github.com/sirupsen/logrus/json_formatter_test.go create mode 100644 vendor/github.com/sirupsen/logrus/logger_bench_test.go create mode 100644 vendor/github.com/sirupsen/logrus/logrus_test.go create mode 100644 vendor/github.com/sirupsen/logrus/text_formatter_test.go create mode 100644 vendor/github.com/spaolacci/murmur3/.gitignore create mode 100644 vendor/github.com/spaolacci/murmur3/README.md create mode 100644 vendor/github.com/spaolacci/murmur3/murmur_test.go create mode 100644 vendor/github.com/twinj/uuid/.gitignore create mode 100644 vendor/github.com/twinj/uuid/.travis.yml create mode 100644 vendor/github.com/twinj/uuid/README.md create mode 100644 vendor/github.com/twinj/uuid/array_test.go create mode 100644 vendor/github.com/twinj/uuid/rfc4122_test.go create mode 100644 vendor/github.com/twinj/uuid/saver_test.go create mode 100644 vendor/github.com/twinj/uuid/state_test.go create mode 100644 vendor/github.com/twinj/uuid/struct_test.go create mode 100644 vendor/github.com/twinj/uuid/timestamp_test.go create mode 100644 vendor/github.com/twinj/uuid/uuid_test.go create mode 100644 vendor/github.com/twinj/uuid/uuids_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/.gitignore create mode 100644 vendor/github.com/uber/jaeger-client-go/.gitmodules create mode 100644 vendor/github.com/uber/jaeger-client-go/.travis.yml create mode 100644 vendor/github.com/uber/jaeger-client-go/CHANGELOG.md create mode 100644 vendor/github.com/uber/jaeger-client-go/CONTRIBUTING.md create mode 100644 vendor/github.com/uber/jaeger-client-go/Makefile create mode 100644 vendor/github.com/uber/jaeger-client-go/README.md create mode 100644 vendor/github.com/uber/jaeger-client-go/config/config_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/config/example_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/config/options_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/constants_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/context_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/glide.lock create mode 100644 vendor/github.com/uber/jaeger-client-go/glide.yaml create mode 100644 vendor/github.com/uber/jaeger-client-go/jaeger_thrift_span_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/log/logger_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/logger_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/metrics_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/observer_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/propagation_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/reporter_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/rpcmetrics/README.md create mode 100644 vendor/github.com/uber/jaeger-client-go/rpcmetrics/endpoints_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/rpcmetrics/metrics_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/rpcmetrics/normalizer_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/rpcmetrics/observer_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/sampler_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/span_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/thrift/.nocover create mode 100644 vendor/github.com/uber/jaeger-client-go/tracer_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/transport_udp_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/utils/http_json_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/utils/rate_limiter_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/utils/utils_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/zipkin_test.go create mode 100644 vendor/github.com/uber/jaeger-client-go/zipkin_thrift_span_test.go create mode 100644 vendor/github.com/uber/jaeger-lib/.gitignore create mode 100644 vendor/github.com/uber/jaeger-lib/.travis.yml create mode 100644 vendor/github.com/uber/jaeger-lib/CONTRIBUTING.md create mode 100644 vendor/github.com/uber/jaeger-lib/Makefile create mode 100644 vendor/github.com/uber/jaeger-lib/README.md create mode 100644 vendor/github.com/uber/jaeger-lib/glide.lock create mode 100644 vendor/github.com/uber/jaeger-lib/glide.yaml create mode 100644 vendor/github.com/uber/jaeger-lib/metrics/local_test.go create mode 100644 vendor/github.com/uber/jaeger-lib/metrics/metrics_test.go create mode 100644 vendor/golang.org/x/net/.gitattributes create mode 100644 vendor/golang.org/x/net/.gitignore create mode 100644 vendor/golang.org/x/net/AUTHORS create mode 100644 vendor/golang.org/x/net/CONTRIBUTING.md create mode 100644 vendor/golang.org/x/net/CONTRIBUTORS create mode 100644 vendor/golang.org/x/net/README create mode 100644 vendor/golang.org/x/net/codereview.cfg create mode 100644 vendor/golang.org/x/net/context/context_test.go create mode 100644 vendor/golang.org/x/net/context/withtimeout_test.go create mode 100644 vendor/golang.org/x/net/http2/.gitignore create mode 100644 vendor/golang.org/x/net/http2/Dockerfile create mode 100644 vendor/golang.org/x/net/http2/Makefile create mode 100644 vendor/golang.org/x/net/http2/README create mode 100644 vendor/golang.org/x/net/http2/databuffer_test.go create mode 100644 vendor/golang.org/x/net/http2/errors_test.go create mode 100644 vendor/golang.org/x/net/http2/flow_test.go create mode 100644 vendor/golang.org/x/net/http2/frame_test.go create mode 100644 vendor/golang.org/x/net/http2/go18_test.go create mode 100644 vendor/golang.org/x/net/http2/gotrack_test.go create mode 100644 vendor/golang.org/x/net/http2/hpack/encode_test.go create mode 100644 vendor/golang.org/x/net/http2/hpack/hpack_test.go create mode 100644 vendor/golang.org/x/net/http2/hpack/tables_test.go create mode 100644 vendor/golang.org/x/net/http2/http2_test.go create mode 100644 vendor/golang.org/x/net/http2/pipe_test.go create mode 100644 vendor/golang.org/x/net/http2/server_push_test.go create mode 100644 vendor/golang.org/x/net/http2/server_test.go create mode 100644 vendor/golang.org/x/net/http2/transport_test.go create mode 100644 vendor/golang.org/x/net/http2/writesched_priority_test.go create mode 100644 vendor/golang.org/x/net/http2/writesched_random_test.go create mode 100644 vendor/golang.org/x/net/http2/writesched_test.go create mode 100644 vendor/golang.org/x/net/http2/z_spec_test.go create mode 100644 vendor/golang.org/x/net/idna/example_test.go create mode 100644 vendor/golang.org/x/net/idna/idna_test.go create mode 100644 vendor/golang.org/x/net/idna/punycode_test.go create mode 100644 vendor/golang.org/x/net/internal/timeseries/timeseries_test.go create mode 100644 vendor/golang.org/x/net/lex/httplex/httplex_test.go create mode 100644 vendor/golang.org/x/net/trace/histogram_test.go create mode 100644 vendor/golang.org/x/net/trace/trace_test.go create mode 100644 vendor/golang.org/x/sys/.gitattributes create mode 100644 vendor/golang.org/x/sys/.gitignore create mode 100644 vendor/golang.org/x/sys/AUTHORS create mode 100644 vendor/golang.org/x/sys/CONTRIBUTING.md create mode 100644 vendor/golang.org/x/sys/CONTRIBUTORS create mode 100644 vendor/golang.org/x/sys/README.md create mode 100644 vendor/golang.org/x/sys/codereview.cfg create mode 100644 vendor/golang.org/x/sys/unix/.gitignore create mode 100644 vendor/golang.org/x/sys/unix/README.md create mode 100644 vendor/golang.org/x/sys/unix/creds_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_darwin_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_dragonfly_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_linux_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_netbsd_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_openbsd_test.go create mode 100644 vendor/golang.org/x/sys/unix/dev_solaris_test.go create mode 100644 vendor/golang.org/x/sys/unix/example_test.go create mode 100644 vendor/golang.org/x/sys/unix/export_test.go create mode 100644 vendor/golang.org/x/sys/unix/mkall.sh create mode 100644 vendor/golang.org/x/sys/unix/mkerrors.sh create mode 100644 vendor/golang.org/x/sys/unix/mksyscall.pl create mode 100644 vendor/golang.org/x/sys/unix/mksyscall_solaris.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysctl_openbsd.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysnum_darwin.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysnum_dragonfly.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysnum_freebsd.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysnum_netbsd.pl create mode 100644 vendor/golang.org/x/sys/unix/mksysnum_openbsd.pl create mode 100644 vendor/golang.org/x/sys/unix/mmap_unix_test.go create mode 100644 vendor/golang.org/x/sys/unix/openbsd_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_bsd_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_freebsd_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_linux_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_solaris_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_test.go create mode 100644 vendor/golang.org/x/sys/unix/syscall_unix_test.go create mode 100644 vendor/golang.org/x/sys/unix/timestruct_test.go create mode 100644 vendor/golang.org/x/text/.gitattributes create mode 100644 vendor/golang.org/x/text/.gitignore create mode 100644 vendor/golang.org/x/text/AUTHORS create mode 100644 vendor/golang.org/x/text/CONTRIBUTING.md create mode 100644 vendor/golang.org/x/text/CONTRIBUTORS create mode 100644 vendor/golang.org/x/text/README create mode 100644 vendor/golang.org/x/text/codereview.cfg create mode 100644 vendor/golang.org/x/text/doc.go create mode 100644 vendor/golang.org/x/text/encoding/charmap/charmap_test.go create mode 100644 vendor/golang.org/x/text/encoding/encoding_test.go create mode 100644 vendor/golang.org/x/text/encoding/example_test.go create mode 100644 vendor/golang.org/x/text/encoding/japanese/all_test.go create mode 100644 vendor/golang.org/x/text/encoding/korean/all_test.go create mode 100644 vendor/golang.org/x/text/encoding/simplifiedchinese/all_test.go create mode 100644 vendor/golang.org/x/text/encoding/traditionalchinese/all_test.go create mode 100644 vendor/golang.org/x/text/encoding/unicode/unicode_test.go create mode 100644 vendor/golang.org/x/text/gen.go create mode 100644 vendor/golang.org/x/text/internal/gen.go create mode 100644 vendor/golang.org/x/text/internal/gen_test.go create mode 100644 vendor/golang.org/x/text/internal/internal.go create mode 100644 vendor/golang.org/x/text/internal/internal_test.go create mode 100644 vendor/golang.org/x/text/internal/match.go create mode 100644 vendor/golang.org/x/text/internal/match_test.go create mode 100644 vendor/golang.org/x/text/internal/tables.go create mode 100644 vendor/golang.org/x/text/internal/triegen/data_test.go create mode 100644 vendor/golang.org/x/text/internal/triegen/example_compact_test.go create mode 100644 vendor/golang.org/x/text/internal/triegen/example_test.go create mode 100644 vendor/golang.org/x/text/internal/triegen/gen_test.go create mode 100644 vendor/golang.org/x/text/internal/ucd/example_test.go create mode 100644 vendor/golang.org/x/text/internal/ucd/ucd_test.go create mode 100644 vendor/golang.org/x/text/runes/cond_test.go create mode 100644 vendor/golang.org/x/text/runes/example_test.go create mode 100644 vendor/golang.org/x/text/runes/runes_test.go create mode 100644 vendor/golang.org/x/text/secure/bidirule/bench_test.go create mode 100644 vendor/golang.org/x/text/secure/bidirule/bidirule_test.go create mode 100644 vendor/golang.org/x/text/secure/doc.go create mode 100644 vendor/golang.org/x/text/transform/examples_test.go create mode 100644 vendor/golang.org/x/text/transform/transform_test.go create mode 100644 vendor/golang.org/x/text/unicode/bidi/core_test.go create mode 100644 vendor/golang.org/x/text/unicode/bidi/ranges_test.go create mode 100644 vendor/golang.org/x/text/unicode/bidi/tables_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/cldr_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/collate_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/data_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/examples_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/resolve_test.go create mode 100644 vendor/golang.org/x/text/unicode/cldr/slice_test.go create mode 100644 vendor/golang.org/x/text/unicode/doc.go create mode 100644 vendor/golang.org/x/text/unicode/norm/composition_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/example_iter_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/example_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/forminfo_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/iter_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/norm_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/normalize_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/readwriter_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/transform_test.go create mode 100644 vendor/golang.org/x/text/unicode/norm/ucd_test.go create mode 100644 vendor/golang.org/x/text/unicode/rangetable/merge_test.go create mode 100644 vendor/golang.org/x/text/unicode/rangetable/rangetable_test.go create mode 100644 vendor/google.golang.org/genproto/.travis.yml create mode 100644 vendor/google.golang.org/genproto/CONTRIBUTING.md create mode 100644 vendor/google.golang.org/genproto/README.md create mode 100644 vendor/google.golang.org/genproto/googleapis/api/authorization_config.pb.go create mode 100644 vendor/google.golang.org/genproto/googleapis/api/experimental.pb.go create mode 100644 vendor/google.golang.org/genproto/regen.go create mode 100644 vendor/google.golang.org/genproto/regen.sh create mode 100644 vendor/google.golang.org/grpc/.travis.yml create mode 100644 vendor/google.golang.org/grpc/AUTHORS create mode 100644 vendor/google.golang.org/grpc/CONTRIBUTING.md create mode 100644 vendor/google.golang.org/grpc/Makefile create mode 100644 vendor/google.golang.org/grpc/README.md create mode 100644 vendor/google.golang.org/grpc/backoff_test.go create mode 100644 vendor/google.golang.org/grpc/balancer/roundrobin/roundrobin_test.go create mode 100644 vendor/google.golang.org/grpc/balancer_switching_test.go create mode 100644 vendor/google.golang.org/grpc/balancer_test.go create mode 100644 vendor/google.golang.org/grpc/call_test.go create mode 100644 vendor/google.golang.org/grpc/clientconn_test.go create mode 100644 vendor/google.golang.org/grpc/codec_test.go create mode 100644 vendor/google.golang.org/grpc/codegen.sh create mode 100644 vendor/google.golang.org/grpc/codes/codes_test.go create mode 100644 vendor/google.golang.org/grpc/credentials/credentials_test.go create mode 100644 vendor/google.golang.org/grpc/encoding/proto/proto_benchmark_test.go create mode 100644 vendor/google.golang.org/grpc/encoding/proto/proto_test.go create mode 100644 vendor/google.golang.org/grpc/grpclb/grpc_lb_v1/messages/messages.proto create mode 100644 vendor/google.golang.org/grpc/grpclb/grpclb_test.go create mode 100644 vendor/google.golang.org/grpc/grpclb_util_test.go create mode 100644 vendor/google.golang.org/grpc/grpclog/loggerv2_test.go create mode 100644 vendor/google.golang.org/grpc/health/grpc_health_v1/health.proto create mode 100644 vendor/google.golang.org/grpc/health/health.go create mode 100644 vendor/google.golang.org/grpc/metadata/metadata_test.go create mode 100644 vendor/google.golang.org/grpc/naming/dns_resolver_test.go create mode 100644 vendor/google.golang.org/grpc/naming/go17_test.go create mode 100644 vendor/google.golang.org/grpc/naming/go18_test.go create mode 100644 vendor/google.golang.org/grpc/picker_wrapper_test.go create mode 100644 vendor/google.golang.org/grpc/pickfirst_test.go create mode 100644 vendor/google.golang.org/grpc/proxy_test.go create mode 100644 vendor/google.golang.org/grpc/resolver/dns/dns_resolver_test.go create mode 100644 vendor/google.golang.org/grpc/resolver/dns/go17_test.go create mode 100644 vendor/google.golang.org/grpc/resolver/dns/go18_test.go create mode 100644 vendor/google.golang.org/grpc/resolver_conn_wrapper_test.go create mode 100644 vendor/google.golang.org/grpc/rpc_util_test.go create mode 100644 vendor/google.golang.org/grpc/server_test.go create mode 100644 vendor/google.golang.org/grpc/service_config_test.go create mode 100644 vendor/google.golang.org/grpc/stats/stats_test.go create mode 100644 vendor/google.golang.org/grpc/status/status_test.go create mode 100644 vendor/google.golang.org/grpc/stickiness_test.go create mode 100644 vendor/google.golang.org/grpc/transport/handler_server_test.go create mode 100644 vendor/google.golang.org/grpc/transport/http_util_test.go create mode 100644 vendor/google.golang.org/grpc/transport/transport_test.go create mode 100644 vendor/google.golang.org/grpc/vet.sh create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/.gitignore create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/README.md create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/example_test.go create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/linux_test.go create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/lumberjack_test.go create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/rotate_test.go create mode 100644 vendor/gopkg.in/natefinch/lumberjack.v2/testing_test.go diff --git a/Gopkg.lock b/Gopkg.lock index 68b4329b332ee..d80362f1bee9c 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -3,44 +3,33 @@ [[projects]] branch = "master" - digest = "1:be3ccd9f881604e4dd6d15cccfa126aa309232f0ba075ae5f92d3ef729a62758" name = "github.com/BurntSushi/toml" packages = ["."] - pruneopts = "NUT" revision = "a368813c5e648fee92e5f6c30e3944ff9d5e8895" [[projects]] - digest = "1:9752dad5e89cd779096bf2477a4ded16bea7ac62de453c8d6b4bf841d51a8512" name = "github.com/apache/thrift" packages = ["lib/go/thrift"] - pruneopts = "NUT" revision = "b2a4d4ae21c789b689dd162deb819665567f481c" version = "0.10.0" [[projects]] - digest = "1:75d40fa0c338f4c56056a3985e91fa371e8fcd0293e45b80afa7debecaf56012" name = "github.com/beorn7/perks" packages = ["quantile"] - pruneopts = "NUT" revision = "3ac7bf7a47d159a033b107610db8a1b6575507a4" [[projects]] branch = "master" - digest = "1:7b12da6e82292eb06e24dfe544c628115a3f4316c152f9dcb87d4f60cbf7cd7d" name = "github.com/blacktear23/go-proxyprotocol" packages = ["."] - pruneopts = "NUT" revision = "62e368e1c4700c34b4b6f77afd49b215211574c2" [[projects]] - digest = "1:60142a898f3808b3e6aa604e5f3296bdef921e625ef3223b6019c1a345b8765c" name = "github.com/codahale/hdrhistogram" packages = ["."] - pruneopts = "NUT" revision = "f8ad88b59a584afeee9d334eff879b104439117b" [[projects]] - digest = "1:7a5c43af23a0c21f4f1762d54af95586f9c04836257e66631557d8f8200ef0e1" name = "github.com/coreos/etcd" packages = [ "auth/authpb", @@ -49,81 +38,40 @@ "etcdserver/api/v3rpc/rpctypes", "etcdserver/etcdserverpb", "mvcc/mvccpb", - "pkg/types", + "pkg/types" ] - pruneopts = "NUT" revision = "eddf599c689ec85f4752060edff5a72e81e9106a" version = "v3.2.18" [[projects]] - digest = "1:883ab1d72c3c5851413baad190ebe3ceaf31bb8dece09a943ac07f9e722f811c" - name = "github.com/cznic/golex" - packages = ["lex"] - pruneopts = "NUT" - revision = "da5a7153a51074477ecac5c45a7e5182a0c72448" - -[[projects]] - digest = "1:f80ed82cae006d02025cd63bd7cbe63a7e593de2714db785ea36d6323cc995eb" name = "github.com/cznic/mathutil" packages = ["."] - pruneopts = "NUT" revision = "78ad7f262603437f0ecfebc835d80094f89c8f54" [[projects]] branch = "master" - digest = "1:a7e4a0d213c4c29e79a0da8fe9d16bbcedce561763ff643bff252ea14260f61c" - name = "github.com/cznic/parser" - packages = ["yacc"] - pruneopts = "NUT" - revision = "31edd927e5b19d1c4a260c41a397e7f81d6694d9" - -[[projects]] - branch = "master" - digest = "1:809006f9378a46bcc70bc4330d14f43c1a7818ae9d93c09cab062e575d7e95a2" name = "github.com/cznic/sortutil" packages = ["."] - pruneopts = "NUT" revision = "4c7342852e65c2088c981288f2c5610d10b9f7f4" -[[projects]] - digest = "1:5799bcff29ed827fa851b40b2f650bc777df4b67fd9b8a3041d9d89b9eb2fd86" - name = "github.com/cznic/strutil" - packages = ["."] - pruneopts = "NUT" - revision = "1eb03e3cc9d345307a45ec82bd3016cde4bd4464" - -[[projects]] - digest = "1:71705901e4a356cde002ae6f0052fb661f59971c12262c12006b25cec40025b1" - name = "github.com/cznic/y" - packages = ["."] - pruneopts = "NUT" - revision = "9fdf92d4aac058959f814606bb729ed50f5e4240" - [[projects]] branch = "master" - digest = "1:f18dbc529543fe5fd5294f8385ea1f71681be964c43461f5f45335bb51ba83ae" name = "github.com/etcd-io/gofail" packages = ["runtime"] - pruneopts = "NUT" revision = "51ce9a71510a58bad5ae66ddd278ef28762a1550" [[projects]] - digest = "1:973dbcbbb1be662b61604319582383c315add70906a68d14894f31542ffc3a25" name = "github.com/go-sql-driver/mysql" packages = ["."] - pruneopts = "NUT" revision = "3955978caca48c1658a4bb7a9c6a0f084e326af3" [[projects]] - digest = "1:38e684375ef5b55e812332266d63f9fc5b6329ab303067c4cdda051db6d29ca4" name = "github.com/gogo/protobuf" packages = ["proto"] - pruneopts = "NUT" revision = "636bf0302bc95575d69441b25a2603156ffdddf1" version = "v1.1.1" [[projects]] - digest = "1:6aef947ba53156da1a66ee891d70d61835e0dcfc9f0d728ae1132db651e81c22" name = "github.com/golang/protobuf" packages = [ "jsonpb", @@ -133,127 +81,105 @@ "ptypes/any", "ptypes/duration", "ptypes/struct", - "ptypes/timestamp", + "ptypes/timestamp" ] - pruneopts = "NUT" revision = "b4deda0973fb4c70b50d226b1af49f3da59f5265" version = "v1.1.0" [[projects]] - digest = "1:c6dfb6c55c1989f1d89622b3c45b786127f76f47322c50e487585f823cb12543" name = "github.com/golang/snappy" packages = ["."] - pruneopts = "NUT" revision = "723cc1e459b8eea2dea4583200fd60757d40097a" [[projects]] branch = "master" - digest = "1:c0883bc20a7c1ff552ff53d414f3cdc28fe847d15fab48c4486aa772ab2fb131" name = "github.com/google/btree" packages = ["."] - pruneopts = "NUT" revision = "316fb6d3f031ae8f4d457c6c5186b9e3ded70435" [[projects]] - digest = "1:dbd86d229eacaa86a98b10f8fb3e3fc69a1913e0f4e010e7cc1f92bf12edca92" name = "github.com/gorilla/context" packages = ["."] - pruneopts = "NUT" revision = "1ea25387ff6f684839d82767c1733ff4d4d15d0a" version = "v1.1" [[projects]] - digest = "1:054b11c45900b575d23fa8e0fa3636a784e2c4d1d43d05e8e20fd592ebe0d5db" name = "github.com/gorilla/mux" packages = ["."] - pruneopts = "NUT" revision = "599cba5e7b6137d46ddf58fb1765f5d928e69604" [[projects]] - digest = "1:0aa5274053fdc232896f0835c712f4a39992d959f8e2363f189a7b0df36f136b" name = "github.com/grpc-ecosystem/go-grpc-middleware" packages = [ ".", "tags", "tracing/opentracing", - "util/metautils", + "util/metautils" ] - pruneopts = "NUT" revision = "82921fcf811d228d2fa202bc31238b356bf9f8d5" [[projects]] - digest = "1:96c558cff0532e2e9ffc0b6d7c8c7431c592d781b109343aa51e27f9fd9a6b82" name = "github.com/grpc-ecosystem/go-grpc-prometheus" packages = ["."] - pruneopts = "NUT" revision = "6b7015e65d366bf3f19b2b2a000a831940f0f7e0" version = "v1.1" [[projects]] branch = "master" - digest = "1:5985ef4caf91ece5d54817c11ea25f182697534f8ae6521eadcd628c142ac4b6" name = "github.com/matttproud/golang_protobuf_extensions" packages = ["pbutil"] - pruneopts = "NUT" revision = "c12348ce28de40eed0136aa2b644d0ee0650e56c" [[projects]] branch = "master" - digest = "1:b95cb972bfd3eb737d334b1703a51a82d1d1d3e92f7f355798d94796c7120c73" name = "github.com/ngaut/pools" packages = ["."] - pruneopts = "NUT" revision = "b7bc8c42aac787667ba45adea78233f53f548443" [[projects]] branch = "master" - digest = "1:7d5f99346aa63d23681f4d92708469f32384d0c26722a4de5725bd0f22caedac" name = "github.com/ngaut/sync2" packages = ["."] - pruneopts = "NUT" revision = "7a24ed77b2efb460c1468b7dc917821c66e80e55" [[projects]] - digest = "1:cc405544fecfb5a8e0c409127ef67ce3b91d11143a00121e5b822e4f8eabe7d2" name = "github.com/opentracing/basictracer-go" packages = [ ".", - "wire", + "wire" ] - pruneopts = "NUT" revision = "1b32af207119a14b1b231d451df3ed04a72efebf" version = "v1.0.0" [[projects]] - digest = "1:7da29c22bcc5c2ffb308324377dc00b5084650348c2799e573ed226d8cc9faf0" name = "github.com/opentracing/opentracing-go" packages = [ ".", "ext", - "log", + "log" ] - pruneopts = "NUT" revision = "1949ddbfd147afd4d964a9f00b24eb291e0e7c38" version = "v1.0.2" [[projects]] branch = "master" - digest = "1:3bf17a6e6eaa6ad24152148a631d18662f7212e21637c2699bff3369b7f00fa2" name = "github.com/petar/GoLLRB" packages = ["llrb"] - pruneopts = "NUT" revision = "53be0d36a84c2a886ca057d34b6aa4468df9ccb4" [[projects]] branch = "master" - digest = "1:dc85cc12f7f97c2b033157f9d40395b2f0458b7bd3083257f8f661ea6f48f89a" name = "github.com/pingcap/check" packages = ["."] - pruneopts = "NUT" revision = "1c287c953996ab3a0bf535dba9d53d809d3dc0b6" [[projects]] - digest = "1:8fd099a567b1e9b3a7e1f66d8547a0d2b1852427e86a0dae96fa59e9583e13e6" + name = "github.com/pingcap/errors" + packages = ["."] + revision = "1176802fff62540cc87d289bd40c52a2d6b2ea16" + version = "v0.11.0" + +[[projects]] name = "github.com/pingcap/goleveldb" packages = [ "leveldb", @@ -267,14 +193,12 @@ "leveldb/opt", "leveldb/storage", "leveldb/table", - "leveldb/util", + "leveldb/util" ] - pruneopts = "NUT" revision = "8d44bfdf1030639ae7130922c95df12d6d4da3b6" [[projects]] branch = "release-2.1" - digest = "1:85aa2923e4dac3cef4dda68972f580cbc10dc1450f019f98a8b7e476f5eb2b07" name = "github.com/pingcap/kvproto" packages = [ "pkg/coprocessor", @@ -284,112 +208,105 @@ "pkg/metapb", "pkg/pdpb", "pkg/raft_serverpb", - "pkg/tikvpb", + "pkg/tikvpb" ] - pruneopts = "NUT" revision = "8e3f33ac49297d7c93b61a955531191084a2f685" +[[projects]] + branch = "release-2.1" + name = "github.com/pingcap/parser" + packages = [ + ".", + "ast", + "auth", + "charset", + "format", + "model", + "mysql", + "opcode", + "terror", + "types" + ] + revision = "815d2f65fd25b8ee34100fbd8311998d2e7de2d6" + [[projects]] branch = "master" - digest = "1:8f576565a8479071ca1951bf678519b1c32480aad8dec2379a64858c31d9e6a9" name = "github.com/pingcap/pd" packages = ["client"] - pruneopts = "NUT" revision = "eb892dda1e33a0b76191d39894ad4a806f313f6e" [[projects]] - digest = "1:09802a4b5acb491d99be650e2c64115de817ff7b588ffd2cb016f3ca1f9c61aa" name = "github.com/pingcap/tidb-tools" packages = [ "pkg/etcd", "pkg/utils", "tidb-binlog/node", - "tidb-binlog/pump_client", + "tidb-binlog/pump_client" ] - pruneopts = "NUT" revision = "8dd71127bcf09130ae032ab6877dd22f32b53e14" [[projects]] branch = "master" - digest = "1:14d83225e335cfa449b1f74d03c03ed70db64bf6af83a6e04c43e6e0021d6b94" name = "github.com/pingcap/tipb" packages = [ "go-binlog", "go-tipb", - "sharedbytes", + "sharedbytes" ] - pruneopts = "NUT" revision = "371b48b15d93924a3a5375019e42b4420bc13d17" [[projects]] - digest = "1:4b96ce0f151b10c52b1d6876c179e9a6d2329f98939c4cf8148daaf27f981413" name = "github.com/pkg/errors" packages = ["."] - pruneopts = "NUT" revision = "1176802fff62540cc87d289bd40c52a2d6b2ea16" source = "https://github.com/pingcap/errors.git" version = "v0.11.0" [[projects]] - digest = "1:1ef3c4d6e78616bd3d1b5b7d8899febb9aa1b83d3373fbbdc2804408c7977b57" name = "github.com/prometheus/client_golang" packages = [ "prometheus", - "prometheus/push", + "prometheus/push" ] - pruneopts = "NUT" revision = "c5b7fccd204277076155f10851dad72b76a49317" version = "v0.8.0" [[projects]] - digest = "1:9fe8945a11a9f588a9d306b4741cad634da9015a704271b9506810e2cc77fa17" name = "github.com/prometheus/client_model" packages = ["go"] - pruneopts = "NUT" revision = "fa8ad6fec33561be4280a8f0514318c79d7f6cb6" [[projects]] - digest = "1:c90717fa0864d47e19eaa855af60b202b537795f485052c7f48333c679dd7310" name = "github.com/prometheus/common" packages = [ "expfmt", "internal/bitbucket.org/ww/goautoneg", - "model", + "model" ] - pruneopts = "NUT" revision = "4402f4e5ea79ec15f3c574773b6a5198fbea215f" [[projects]] - digest = "1:dcfff2d5e99e01dcb856dd8afb0b509c1d05443f0b523cc5333b33a819829ed9" name = "github.com/prometheus/procfs" packages = ["."] - pruneopts = "NUT" revision = "abf152e5f3e97f2fafac028d2cc06c1feb87ffa5" [[projects]] - digest = "1:4c173651d2deb815a0420aeb1b3f7ca3c4aef2d980ba164a501a53f6abf368ef" name = "github.com/sirupsen/logrus" packages = ["."] - pruneopts = "NUT" revision = "3bcb09397d6d88e7676a9bc8433ca11ba5304837" [[projects]] - digest = "1:0af5ed795eeb9df3e3d32e2c0229b012e2c107945f75a0556733d643c94e55be" name = "github.com/spaolacci/murmur3" packages = ["."] - pruneopts = "NUT" revision = "0d12bf811670bf6a1a63828dfbd003eded177fce" version = "v1.0" [[projects]] - digest = "1:194c26fad062f6b1530720ee1afd6cd6f40d79274b2434caef2693b1da5d2ab2" name = "github.com/twinj/uuid" packages = ["."] - pruneopts = "NUT" revision = "70cac2bcd273ef6a371bb96cde363d28b68734c3" [[projects]] - digest = "1:0e28a98b9579858cb5de885935499fefebc8bc44a652cde08a6d035ee7435603" name = "github.com/uber/jaeger-client-go" packages = [ ".", @@ -401,22 +318,18 @@ "thrift-gen/jaeger", "thrift-gen/sampling", "thrift-gen/zipkincore", - "utils", + "utils" ] - pruneopts = "NUT" revision = "d021e646f5187d77b55592c3efee1a2810e895d7" version = "v2.8.0" [[projects]] - digest = "1:0da2810678a062e0567c3215911869b0423da0e497c56683ff8e87e7a6952597" name = "github.com/uber/jaeger-lib" packages = ["metrics"] - pruneopts = "NUT" revision = "3b2a9ad2a045881ab7a0f81d465be54c8292ee4f" version = "v1.1.0" [[projects]] - digest = "1:686219a880e6ec42870431372756a66c19c1396e9fe203b659179422d3c6bf96" name = "golang.org/x/net" packages = [ "context", @@ -425,21 +338,17 @@ "idna", "internal/timeseries", "lex/httplex", - "trace", + "trace" ] - pruneopts = "NUT" revision = "d1e1b351919c6738fdeb9893d5c998b161464f0c" [[projects]] branch = "master" - digest = "1:2f375ec82e53522eb4a1670f2f24f064f407ef2b32e01bb217f5daa4a4d226d6" name = "golang.org/x/sys" packages = ["unix"] - pruneopts = "NUT" revision = "7dfd1290c7917b7ba22824b9d24954ab3002fe24" [[projects]] - digest = "1:a005696b163ffe1842de27eeb3ccfece9d3c2e70d02b83ea1d8c0eeab597c9e0" name = "golang.org/x/text" packages = [ "encoding", @@ -461,23 +370,19 @@ "unicode/bidi", "unicode/cldr", "unicode/norm", - "unicode/rangetable", + "unicode/rangetable" ] - pruneopts = "NUT" revision = "4ee4af566555f5fbe026368b75596286a312663a" [[projects]] - digest = "1:0efcfe82e59b828eb6f4115bba88ff45c0898c38e823fbe7f450bdffed9e739b" name = "google.golang.org/genproto" packages = [ "googleapis/api/annotations", - "googleapis/rpc/status", + "googleapis/rpc/status" ] - pruneopts = "NUT" revision = "6b7d9516179cd47f4714cfeb0103ad1dede756c4" [[projects]] - digest = "1:a893d24a604ae6f45f2d6e00ae9e817476e110e2dd5455152b35bb720ca2f626" name = "google.golang.org/grpc" packages = [ ".", @@ -504,89 +409,21 @@ "stats", "status", "tap", - "transport", + "transport" ] - pruneopts = "NUT" revision = "41344da2231b913fa3d983840a57a6b1b7b631a1" source = "https://github.com/grpc/grpc-go.git" version = "v1.12.0" [[projects]] - digest = "1:7d95d61ff5828a65cf072a46f3c68d67daffdd11e97d758b0af2176cde717fcd" name = "gopkg.in/natefinch/lumberjack.v2" packages = ["."] - pruneopts = "NUT" revision = "dd45e6a67c53f673bb49ca8a001fd3a63ceb640e" version = "v2.0" [solve-meta] analyzer-name = "dep" analyzer-version = 1 - input-imports = [ - "github.com/BurntSushi/toml", - "github.com/blacktear23/go-proxyprotocol", - "github.com/coreos/etcd/clientv3", - "github.com/coreos/etcd/clientv3/concurrency", - "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes", - "github.com/coreos/etcd/mvcc/mvccpb", - "github.com/cznic/mathutil", - "github.com/cznic/parser/yacc", - "github.com/cznic/sortutil", - "github.com/cznic/strutil", - "github.com/cznic/y", - "github.com/etcd-io/gofail/runtime", - "github.com/go-sql-driver/mysql", - "github.com/golang/protobuf/jsonpb", - "github.com/golang/protobuf/proto", - "github.com/google/btree", - "github.com/gorilla/mux", - "github.com/grpc-ecosystem/go-grpc-middleware", - "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing", - "github.com/grpc-ecosystem/go-grpc-prometheus", - "github.com/ngaut/pools", - "github.com/ngaut/sync2", - "github.com/opentracing/basictracer-go", - "github.com/opentracing/opentracing-go", - "github.com/pingcap/check", - "github.com/pingcap/goleveldb/leveldb", - "github.com/pingcap/goleveldb/leveldb/comparer", - "github.com/pingcap/goleveldb/leveldb/iterator", - "github.com/pingcap/goleveldb/leveldb/memdb", - "github.com/pingcap/goleveldb/leveldb/opt", - "github.com/pingcap/goleveldb/leveldb/storage", - "github.com/pingcap/goleveldb/leveldb/util", - "github.com/pingcap/kvproto/pkg/coprocessor", - "github.com/pingcap/kvproto/pkg/errorpb", - "github.com/pingcap/kvproto/pkg/kvrpcpb", - "github.com/pingcap/kvproto/pkg/metapb", - "github.com/pingcap/kvproto/pkg/tikvpb", - "github.com/pingcap/pd/client", - "github.com/pingcap/tidb-tools/tidb-binlog/node", - "github.com/pingcap/tidb-tools/tidb-binlog/pump_client", - "github.com/pingcap/tipb/go-binlog", - "github.com/pingcap/tipb/go-tipb", - "github.com/pkg/errors", - "github.com/prometheus/client_golang/prometheus", - "github.com/prometheus/client_golang/prometheus/push", - "github.com/sirupsen/logrus", - "github.com/spaolacci/murmur3", - "github.com/twinj/uuid", - "github.com/uber/jaeger-client-go/config", - "golang.org/x/net/context", - "golang.org/x/text/encoding", - "golang.org/x/text/encoding/charmap", - "golang.org/x/text/encoding/japanese", - "golang.org/x/text/encoding/korean", - "golang.org/x/text/encoding/simplifiedchinese", - "golang.org/x/text/encoding/traditionalchinese", - "golang.org/x/text/encoding/unicode", - "golang.org/x/text/transform", - "google.golang.org/grpc", - "google.golang.org/grpc/codes", - "google.golang.org/grpc/credentials", - "google.golang.org/grpc/keepalive", - "google.golang.org/grpc/metadata", - "gopkg.in/natefinch/lumberjack.v2", - ] + inputs-digest = "456ced89257d372e02ff41bc3219136a6cd941b5010707bf57dcac7acfc282c8" solver-name = "gps-cdcl" solver-version = 1 diff --git a/Gopkg.toml b/Gopkg.toml index b04b1587e98dd..e1eabdaaf68e7 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -28,10 +28,6 @@ required = ["github.com/golang/protobuf/jsonpb"] version = "1.1.0" name = "github.com/golang/protobuf" -[[constraint]] - branch = "master" - name = "github.com/cznic/parser" - [[constraint]] branch = "master" name = "github.com/cznic/sortutil" @@ -106,3 +102,7 @@ required = ["github.com/golang/protobuf/jsonpb"] [[constraint]] name = "github.com/pingcap/tidb-tools" revision = "8dd71127bcf09130ae032ab6877dd22f32b53e14" + +[[constraint]] + name = "github.com/pingcap/parser" + branch = "release-2.1" \ No newline at end of file diff --git a/Makefile b/Makefile index ad524f99eaa18..f79925cb0b821 100644 --- a/Makefile +++ b/Makefile @@ -28,7 +28,7 @@ FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go" | grep -vE "vendor" GOFAIL_ENABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|vendor)" | xargs gofail enable) GOFAIL_DISABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|vendor)" | xargs gofail disable) -LDFLAGS += -X "github.com/pingcap/tidb/mysql.TiDBReleaseVersion=$(shell git describe --tags --dirty)" +LDFLAGS += -X "github.com/pingcap/parser/mysql.TiDBReleaseVersion=$(shell git describe --tags --dirty)" LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBBuildTS=$(shell date -u '+%Y-%m-%d %I:%M:%S')" LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitHash=$(shell git rev-parse HEAD)" LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" @@ -40,7 +40,7 @@ CHECK_LDFLAGS += $(LDFLAGS) ${TEST_LDFLAGS} TARGET = "" -.PHONY: all build update parser clean todo test gotest interpreter server dev benchkv benchraw check parserlib checklist +.PHONY: all build update parser clean todo test gotest interpreter server dev benchkv benchraw check checklist parser default: server buildsucc @@ -51,35 +51,14 @@ buildsucc: all: dev server benchkv -dev: checklist parserlib test check +parser: + @echo "remove this command later" + +dev: checklist test check build: $(GOBUILD) -goyacc: - $(GOBUILD) -o bin/goyacc parser/goyacc/main.go - -parser: goyacc - bin/goyacc -o /dev/null parser/parser.y - bin/goyacc -o parser/parser.go parser/parser.y 2>&1 | egrep "(shift|reduce)/reduce" | awk '{print} END {if (NR > 0) {print "Find conflict in parser.y. Please check y.output for more information."; exit 1;}}' - rm -f y.output - - @if [ $(ARCH) = $(LINUX) ]; \ - then \ - sed -i -e 's|//line.*||' -e 's/yyEofCode/yyEOFCode/' parser/parser.go; \ - elif [ $(ARCH) = $(MAC) ]; \ - then \ - /usr/bin/sed -i "" 's|//line.*||' parser/parser.go; \ - /usr/bin/sed -i "" 's/yyEofCode/yyEOFCode/' parser/parser.go; \ - fi - - @awk 'BEGIN{print "// Code generated by goyacc"} {print $0}' parser/parser.go > tmp_parser.go && mv tmp_parser.go parser/parser.go; - -parserlib: parser/parser.go - -parser/parser.go: parser/parser.y - make parser - # The retool tools.json is setup from hack/retool-install.sh check-setup: @which retool >/dev/null 2>&1 || go get github.com/twitchtv/retool @@ -135,7 +114,7 @@ test: checklist gotest explaintest explaintest: server @cd cmd/explaintest && ./run-tests.sh -s ../../bin/tidb-server -gotest: parserlib +gotest: go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) ifeq ("$(TRAVIS_COVERAGE)", "1") @@ -152,21 +131,21 @@ else endif @$(GOFAIL_DISABLE) -race: parserlib +race: go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ $(GOTEST) -timeout 20m -race $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) -leak: parserlib +leak: go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ $(GOTEST) -tags leak $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) -tikv_integration_test: parserlib +tikv_integration_test: go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) $(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; } @@ -183,14 +162,14 @@ ifeq ("$(WITH_CHECK)", "1") CHECK_FLAG = $(TEST_LDFLAGS) endif -server: parserlib +server: ifeq ($(TARGET), "") $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go else $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go endif -server_check: parserlib +server_check: ifeq ($(TARGET), "") $(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o bin/tidb-server tidb-server/main.go else diff --git a/ast/ast.go b/ast/ast.go index 3cc2da1f20e1b..2f3be0b4b4f5a 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -18,10 +18,8 @@ package ast import ( "io" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" - "golang.org/x/net/context" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/types" ) // Node is the basic element of the AST. @@ -64,14 +62,6 @@ type ExprNode interface { SetType(tp *types.FieldType) // GetType gets the evaluation type of the expression. GetType() *types.FieldType - // SetValue sets value to the expression. - SetValue(val interface{}) - // GetValue gets value of the expression. - GetValue() interface{} - // SetDatum sets datum to the expression. - SetDatum(datum types.Datum) - // GetDatum gets datum of the expression. - GetDatum() *types.Datum // SetFlag sets flag to the expression. // Flag indicates whether the expression contains // parameter marker, reference, aggregate function... @@ -138,31 +128,6 @@ type ResultField struct { Referenced bool } -// RecordSet is an abstract result set interface to help get data from Plan. -type RecordSet interface { - // Fields gets result fields. - Fields() []*ResultField - - // Next reads records into chunk. - Next(ctx context.Context, chk *chunk.Chunk) error - - // NewChunk creates a new chunk with initial capacity. - NewChunk() *chunk.Chunk - - // Close closes the underlying iterator, call Next after Close will - // restart the iteration. - Close() error -} - -// RowToDatums converts row to datum slice. -func RowToDatums(row chunk.Row, fields []*ResultField) []types.Datum { - datums := make([]types.Datum, len(fields)) - for i, f := range fields { - datums[i] = row.GetDatum(i, &f.Column.FieldType) - } - return datums -} - // ResultSetNode interface has a ResultFields property, represents a Node that returns result set. // Implementations include SelectStmt, SubqueryExpr, TableSource, TableName and Join. type ResultSetNode interface { @@ -176,28 +141,6 @@ type SensitiveStmtNode interface { SecureText() string } -// Statement is an interface for SQL execution. -// NOTE: all Statement implementations must be safe for -// concurrent using by multiple goroutines. -// If the Exec method requires any Execution domain local data, -// they must be held out of the implementing instance. -type Statement interface { - // OriginText gets the origin SQL text. - OriginText() string - - // Exec executes SQL and gets a Recordset. - Exec(ctx context.Context) (RecordSet, error) - - // IsPrepared returns whether this statement is prepared statement. - IsPrepared() bool - - // IsReadOnly returns if the statement is read only. For example: SelectStmt without lock. - IsReadOnly() bool - - // RebuildPlan rebuilds the plan of the statement. - RebuildPlan() (schemaVersion int64, err error) -} - // Visitor visits a Node. type Visitor interface { // Enter is called before children nodes are visited. diff --git a/ast/base.go b/ast/base.go index 75ad175dce039..984d8e4d95802 100644 --- a/ast/base.go +++ b/ast/base.go @@ -13,7 +13,7 @@ package ast -import "github.com/pingcap/tidb/types" +import "github.com/pingcap/parser/types" // node is the struct implements node interface except for Accept method. // Node implementations should embed it in. @@ -62,20 +62,12 @@ func (dn *dmlNode) dmlStatement() {} // Expression implementations should embed it in. type exprNode struct { node - types.Datum Type types.FieldType flag uint64 } -// SetDatum implements ExprNode interface. -func (en *exprNode) SetDatum(datum types.Datum) { - en.Datum = datum -} - -// GetDatum implements ExprNode interface. -func (en *exprNode) GetDatum() *types.Datum { - return &en.Datum -} +// TexprNode is exported for parser driver. +type TexprNode = exprNode // SetType implements ExprNode interface. func (en *exprNode) SetType(tp *types.FieldType) { diff --git a/ast/ddl.go b/ast/ddl.go index 340027e881c6f..aefa2eed2dd13 100644 --- a/ast/ddl.go +++ b/ast/ddl.go @@ -14,8 +14,8 @@ package ast import ( - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/types" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/types" ) var ( diff --git a/ast/ddl_test.go b/ast/ddl_test.go index 1aa83a5092a19..a7fbd0f64264d 100644 --- a/ast/ddl_test.go +++ b/ast/ddl_test.go @@ -15,7 +15,7 @@ package ast_test import ( . "github.com/pingcap/check" - . "github.com/pingcap/tidb/ast" + . "github.com/pingcap/parser/ast" ) var _ = Suite(&testDDLSuite{}) diff --git a/ast/dml.go b/ast/dml.go index d3f6575dee0fe..2046f3ad24eb9 100644 --- a/ast/dml.go +++ b/ast/dml.go @@ -14,9 +14,9 @@ package ast import ( - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/util/auth" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" ) var ( diff --git a/ast/dml_test.go b/ast/dml_test.go index 9c6decc35c295..6553c3254b281 100644 --- a/ast/dml_test.go +++ b/ast/dml_test.go @@ -15,7 +15,7 @@ package ast_test import ( . "github.com/pingcap/check" - . "github.com/pingcap/tidb/ast" + . "github.com/pingcap/parser/ast" ) var _ = Suite(&testDMLSuite{}) diff --git a/ast/expressions.go b/ast/expressions.go index e359b121c7444..8706525234625 100644 --- a/ast/expressions.go +++ b/ast/expressions.go @@ -17,13 +17,10 @@ import ( "fmt" "io" "regexp" - "strconv" "strings" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" - "github.com/pingcap/tidb/types" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/opcode" ) var ( @@ -36,7 +33,6 @@ var ( _ ExprNode = &ExistsSubqueryExpr{} _ ExprNode = &IsNullExpr{} _ ExprNode = &IsTruthExpr{} - _ ExprNode = &ParamMarkerExpr{} _ ExprNode = &ParenthesesExpr{} _ ExprNode = &PatternInExpr{} _ ExprNode = &PatternLikeExpr{} @@ -45,7 +41,6 @@ var ( _ ExprNode = &RowExpr{} _ ExprNode = &SubqueryExpr{} _ ExprNode = &UnaryOperationExpr{} - _ ExprNode = &ValueExpr{} _ ExprNode = &ValuesExpr{} _ ExprNode = &VariableExpr{} @@ -53,81 +48,22 @@ var ( _ Node = &WhenClause{} ) -// ValueExpr is the simple value expression. -type ValueExpr struct { - exprNode - projectionOffset int -} - -// Format the ExprNode into a Writer. -func (n *ValueExpr) Format(w io.Writer) { - var s string - switch n.Kind() { - case types.KindNull: - s = "NULL" - case types.KindInt64: - if n.Type.Flag&mysql.IsBooleanFlag != 0 { - if n.GetInt64() > 0 { - s = "TRUE" - } else { - s = "FALSE" - } - } else { - s = strconv.FormatInt(n.GetInt64(), 10) - } - case types.KindUint64: - s = strconv.FormatUint(n.GetUint64(), 10) - case types.KindFloat32: - s = strconv.FormatFloat(n.GetFloat64(), 'e', -1, 32) - case types.KindFloat64: - s = strconv.FormatFloat(n.GetFloat64(), 'e', -1, 64) - case types.KindString, types.KindBytes: - s = strconv.Quote(n.GetString()) - case types.KindMysqlDecimal: - s = n.GetMysqlDecimal().String() - case types.KindBinaryLiteral: - if n.Type.Flag&mysql.UnsignedFlag != 0 { - s = fmt.Sprintf("x'%x'", n.GetBytes()) - } else { - s = n.GetBinaryLiteral().ToBitLiteralString(true) - } - default: - panic("Can't format to string") - } - fmt.Fprint(w, s) +// ValueExpr define a interface for ValueExpr. +type ValueExpr interface { + ExprNode + SetValue(val interface{}) + GetValue() interface{} + GetDatumString() string + GetString() string + GetProjectionOffset() int + SetProjectionOffset(offset int) } // NewValueExpr creates a ValueExpr with value, and sets default field type. -func NewValueExpr(value interface{}) *ValueExpr { - if ve, ok := value.(*ValueExpr); ok { - return ve - } - ve := &ValueExpr{} - ve.SetValue(value) - types.DefaultTypeForValue(value, &ve.Type) - ve.projectionOffset = -1 - return ve -} +var NewValueExpr func(interface{}) ValueExpr -// SetProjectionOffset sets ValueExpr.projectionOffset for logical plan builder. -func (n *ValueExpr) SetProjectionOffset(offset int) { - n.projectionOffset = offset -} - -// GetProjectionOffset returns ValueExpr.projectionOffset. -func (n *ValueExpr) GetProjectionOffset() int { - return n.projectionOffset -} - -// Accept implements Node interface. -func (n *ValueExpr) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ValueExpr) - return v.Leave(n) -} +// NewParamMarkerExpr creates a ParamMarkerExpr. +var NewParamMarkerExpr func(offset int) ParamMarkerExpr // BetweenExpr is for "between and" or "not between and" expression. type BetweenExpr struct { @@ -719,25 +655,9 @@ func (n *PatternLikeExpr) Accept(v Visitor) (Node, bool) { // ParamMarkerExpr expression holds a place for another expression. // Used in parsing prepare statement. -type ParamMarkerExpr struct { - exprNode - Offset int - Order int -} - -// Format the ExprNode into a Writer. -func (n *ParamMarkerExpr) Format(w io.Writer) { - panic("Not implemented") -} - -// Accept implements Node Accept interface. -func (n *ParamMarkerExpr) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ParamMarkerExpr) - return v.Leave(n) +type ParamMarkerExpr interface { + ValueExpr + SetOrder(int) } // ParenthesesExpr is the parentheses expression. diff --git a/ast/expressions_test.go b/ast/expressions_test.go index 9176e36b70f3a..80ce65d6e6be1 100644 --- a/ast/expressions_test.go +++ b/ast/expressions_test.go @@ -15,7 +15,8 @@ package ast_test import ( . "github.com/pingcap/check" - . "github.com/pingcap/tidb/ast" + . "github.com/pingcap/parser/ast" + _ "github.com/pingcap/tidb/types/parser_driver" ) var _ = Suite(&testExpressionsSuite{}) @@ -79,7 +80,7 @@ func (tc *testExpressionsSuite) TestExpresionsVisitorCover(c *C) { {&ExistsSubqueryExpr{Sel: ce}, 1, 1}, {&IsNullExpr{Expr: ce}, 1, 1}, {&IsTruthExpr{Expr: ce}, 1, 1}, - {&ParamMarkerExpr{}, 0, 0}, + {NewParamMarkerExpr(0), 0, 0}, {&ParenthesesExpr{Expr: ce}, 1, 1}, {&PatternInExpr{Expr: ce, List: []ExprNode{ce, ce, ce}, Sel: ce}, 5, 5}, {&PatternLikeExpr{Expr: ce, Pattern: ce}, 2, 2}, @@ -87,7 +88,7 @@ func (tc *testExpressionsSuite) TestExpresionsVisitorCover(c *C) { {&PositionExpr{}, 0, 0}, {&RowExpr{Values: []ExprNode{ce, ce}}, 2, 2}, {&UnaryOperationExpr{V: ce}, 1, 1}, - {&ValueExpr{}, 0, 0}, + {NewValueExpr(0), 0, 0}, {&ValuesExpr{Column: &ColumnNameExpr{Name: &ColumnName{}}}, 0, 0}, {&VariableExpr{Value: ce}, 1, 1}, } diff --git a/ast/flag.go b/ast/flag.go index 6883f82e138e6..773a2b44483e7 100644 --- a/ast/flag.go +++ b/ast/flag.go @@ -32,6 +32,9 @@ func (f *flagSetter) Enter(in Node) (Node, bool) { } func (f *flagSetter) Leave(in Node) (Node, bool) { + if x, ok := in.(ParamMarkerExpr); ok { + x.SetFlag(FlagHasParamMarker) + } switch x := in.(type) { case *AggregateFuncExpr: f.aggregateFunc(x) @@ -57,8 +60,6 @@ func (f *flagSetter) Leave(in Node) (Node, bool) { x.SetFlag(x.Expr.GetFlag()) case *IsTruthExpr: x.SetFlag(x.Expr.GetFlag()) - case *ParamMarkerExpr: - x.SetFlag(FlagHasParamMarker) case *ParenthesesExpr: x.SetFlag(x.Expr.GetFlag()) case *PatternInExpr: @@ -75,7 +76,6 @@ func (f *flagSetter) Leave(in Node) (Node, bool) { x.SetFlag(FlagHasSubquery) case *UnaryOperationExpr: x.SetFlag(x.V.GetFlag()) - case *ValueExpr: case *ValuesExpr: x.SetFlag(FlagHasReference) case *VariableExpr: diff --git a/ast/flag_test.go b/ast/flag_test.go index a3f7c5d3762c8..fc7f773d4726b 100644 --- a/ast/flag_test.go +++ b/ast/flag_test.go @@ -17,8 +17,8 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" ) func TestT(t *testing.T) { diff --git a/ast/format_test.go b/ast/format_test.go index a5a70030e69b2..45422cbc7ad00 100644 --- a/ast/format_test.go +++ b/ast/format_test.go @@ -5,8 +5,8 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" ) var _ = Suite(&testAstFormatSuite{}) diff --git a/ast/functions.go b/ast/functions.go index bdd81485b92d6..495c6e54f030b 100644 --- a/ast/functions.go +++ b/ast/functions.go @@ -17,8 +17,8 @@ import ( "fmt" "io" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/types" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/types" ) var ( @@ -346,10 +346,10 @@ func (n *FuncCallExpr) specialFormatArgs(w io.Writer) bool { n.Args[0].Format(w) fmt.Fprint(w, ", INTERVAL ") n.Args[1].Format(w) - fmt.Fprintf(w, " %s", n.Args[2].GetDatum().GetString()) + fmt.Fprintf(w, " %s", n.Args[2].(ValueExpr).GetDatumString()) return true case TimestampAdd, TimestampDiff: - fmt.Fprintf(w, "%s, ", n.Args[0].GetDatum().GetString()) + fmt.Fprintf(w, "%s, ", n.Args[0].(ValueExpr).GetDatumString()) n.Args[1].Format(w) fmt.Fprint(w, ", ") n.Args[2].Format(w) diff --git a/ast/functions_test.go b/ast/functions_test.go index f54120c717fd2..d07db759b337c 100644 --- a/ast/functions_test.go +++ b/ast/functions_test.go @@ -15,7 +15,7 @@ package ast_test import ( . "github.com/pingcap/check" - . "github.com/pingcap/tidb/ast" + . "github.com/pingcap/parser/ast" ) var _ = Suite(&testFunctionsSuite{}) @@ -24,10 +24,11 @@ type testFunctionsSuite struct { } func (ts *testFunctionsSuite) TestFunctionsVisitorCover(c *C) { + valueExpr := NewValueExpr(42) stmts := []Node{ - &AggregateFuncExpr{Args: []ExprNode{&ValueExpr{}}}, - &FuncCallExpr{Args: []ExprNode{&ValueExpr{}}}, - &FuncCastExpr{Expr: &ValueExpr{}}, + &AggregateFuncExpr{Args: []ExprNode{valueExpr}}, + &FuncCallExpr{Args: []ExprNode{valueExpr}}, + &FuncCastExpr{Expr: valueExpr}, } for _, stmt := range stmts { diff --git a/ast/misc.go b/ast/misc.go index e9e20b162e0f0..fdcc569d238be 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -18,9 +18,9 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/util/auth" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" ) var ( @@ -187,7 +187,7 @@ func (n *DeallocateStmt) Accept(v Visitor) (Node, bool) { // Prepared represents a prepared statement. type Prepared struct { Stmt StmtNode - Params []*ParamMarkerExpr + Params []ParamMarkerExpr SchemaVersion int64 UseCache bool } @@ -321,7 +321,7 @@ type VariableAssignment struct { // VariableAssignment should be able to store information for SetCharset/SetPWD Stmt. // For SetCharsetStmt, Value is charset, ExtendValue is collation. // TODO: Use SetStmt to implement set password statement. - ExtendValue *ValueExpr + ExtendValue ValueExpr } // Accept implements Node interface. @@ -869,3 +869,12 @@ func (n *TableOptimizerHint) Accept(v Visitor) (Node, bool) { n = newNode.(*TableOptimizerHint) return v.Leave(n) } + +// NewDecimal creates a types.Decimal value, it's provided by parser driver. +var NewDecimal func(string) (interface{}, error) + +// NewHexLiteral creates a types.HexLiteral value, it's provided by parser driver. +var NewHexLiteral func(string) (interface{}, error) + +// NewBitLiteral creates a types.BitLiteral value, it's provided by parser driver. +var NewBitLiteral func(string) (interface{}, error) diff --git a/ast/misc_test.go b/ast/misc_test.go index d9c14f753fe5f..7e1b80ec053f4 100644 --- a/ast/misc_test.go +++ b/ast/misc_test.go @@ -15,9 +15,9 @@ package ast_test import ( . "github.com/pingcap/check" - . "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/util/auth" + "github.com/pingcap/parser" + . "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" ) var _ = Suite(&testMiscSuite{}) @@ -44,6 +44,7 @@ func (visitor1) Enter(in Node) (Node, bool) { } func (ts *testMiscSuite) TestMiscVisitorCover(c *C) { + valueExpr := NewValueExpr(42) stmts := []Node{ &AdminStmt{}, &AlterUserStmt{}, @@ -53,15 +54,15 @@ func (ts *testMiscSuite) TestMiscVisitorCover(c *C) { &CreateUserStmt{}, &DeallocateStmt{}, &DoStmt{}, - &ExecuteStmt{UsingVars: []ExprNode{&ValueExpr{}}}, + &ExecuteStmt{UsingVars: []ExprNode{valueExpr}}, &ExplainStmt{Stmt: &ShowStmt{}}, &GrantStmt{}, - &PrepareStmt{SQLVar: &VariableExpr{Value: &ValueExpr{}}}, + &PrepareStmt{SQLVar: &VariableExpr{Value: valueExpr}}, &RollbackStmt{}, &SetPwdStmt{}, &SetStmt{Variables: []*VariableAssignment{ { - Value: &ValueExpr{}, + Value: valueExpr, }, }}, &UseStmt{}, @@ -72,7 +73,7 @@ func (ts *testMiscSuite) TestMiscVisitorCover(c *C) { }, &FlushStmt{}, &PrivElem{}, - &VariableAssignment{Value: &ValueExpr{}}, + &VariableAssignment{Value: valueExpr}, &KillStmt{}, &DropStatsStmt{Table: &TableName{}}, } diff --git a/ast/stats.go b/ast/stats.go index d31cb70a8d46a..54e464ee3d09c 100644 --- a/ast/stats.go +++ b/ast/stats.go @@ -13,7 +13,7 @@ package ast -import "github.com/pingcap/tidb/model" +import "github.com/pingcap/parser/model" var ( _ StmtNode = &AnalyzeTableStmt{} diff --git a/cmd/benchdb/main.go b/cmd/benchdb/main.go index e6a91936b9869..54dacd9bcfbc5 100644 --- a/cmd/benchdb/main.go +++ b/cmd/benchdb/main.go @@ -21,9 +21,9 @@ import ( "strings" "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/logutil" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/cmd/benchfilesort/main.go b/cmd/benchfilesort/main.go index 296afe0554bb6..3116638efb902 100644 --- a/cmd/benchfilesort/main.go +++ b/cmd/benchfilesort/main.go @@ -24,8 +24,8 @@ import ( "runtime/pprof" "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/filesort" diff --git a/cmd/benchkv/main.go b/cmd/benchkv/main.go index 06d35ec9a0139..9210170dba708 100644 --- a/cmd/benchkv/main.go +++ b/cmd/benchkv/main.go @@ -24,9 +24,9 @@ import ( _ "github.com/go-sql-driver/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" log "github.com/sirupsen/logrus" diff --git a/cmd/benchraw/main.go b/cmd/benchraw/main.go index fcbc3ec428101..f1c6f30b40279 100644 --- a/cmd/benchraw/main.go +++ b/cmd/benchraw/main.go @@ -22,9 +22,9 @@ import ( "sync" "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index 2da89d708e071..cc3e6dd9b6e61 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -26,7 +26,7 @@ import ( "flag" "github.com/go-sql-driver/mysql" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" diff --git a/cmd/importer/db.go b/cmd/importer/db.go index 5f5d174bc3f45..92bf182828ed0 100644 --- a/cmd/importer/db.go +++ b/cmd/importer/db.go @@ -21,7 +21,7 @@ import ( "strings" _ "github.com/go-sql-driver/mysql" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/cmd/importer/parser.go b/cmd/importer/parser.go index 13926e1928f58..b953403698d88 100644 --- a/cmd/importer/parser.go +++ b/cmd/importer/parser.go @@ -18,10 +18,10 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" _ "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" @@ -117,7 +117,7 @@ func (col *column) parseColumnOptions(ops []*ast.ColumnOption) { case ast.ColumnOptionPrimaryKey, ast.ColumnOptionUniqKey, ast.ColumnOptionAutoIncrement: col.table.uniqIndices[col.name] = col case ast.ColumnOptionComment: - col.comment = op.Expr.GetDatum().GetString() + col.comment = op.Expr.(ast.ValueExpr).GetDatumString() } } } diff --git a/cmd/importer/stats.go b/cmd/importer/stats.go index e9917cd1872c6..5ef2a543805e3 100644 --- a/cmd/importer/stats.go +++ b/cmd/importer/stats.go @@ -19,7 +19,7 @@ import ( "math/rand" "time" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" stats "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/ddl/callback.go b/ddl/callback.go index 0b290df42aead..340f0d2726165 100644 --- a/ddl/callback.go +++ b/ddl/callback.go @@ -14,8 +14,8 @@ package ddl import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "golang.org/x/net/context" ) diff --git a/ddl/callback_test.go b/ddl/callback_test.go index 57b23d96c9e5e..4b60b4d4f9afd 100644 --- a/ddl/callback_test.go +++ b/ddl/callback_test.go @@ -15,8 +15,8 @@ package ddl import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/ddl/column.go b/ddl/column.go index ace6ad60c2a83..aca665dfb96c4 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -14,11 +14,11 @@ package ddl import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 9ea837ae0a5b1..6655067a3f9c8 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -19,11 +19,11 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/ddl/column_test.go b/ddl/column_test.go index 4eea2c9b862d0..1859f1a696d73 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -18,16 +18,16 @@ import ( "sync" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 79837b9765ce9..926b57a105b56 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -21,19 +21,20 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" @@ -229,7 +230,7 @@ func (s *testStateChangeSuite) test(c *C, tableName, alterTableSQL string, testI type stateCase struct { session session.Session rawStmt ast.StmtNode - stmt ast.Statement + stmt sqlexec.Statement expectedExecErr error expectedCompileErr error } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 77f03ba8ad119..2698a82b5256c 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -17,6 +17,7 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -25,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" diff --git a/ddl/db_test.go b/ddl/db_test.go index 0044961c6836b..3c40da19ed597 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -24,16 +24,17 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + tmysql "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - tmysql "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" @@ -41,7 +42,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" diff --git a/ddl/ddl.go b/ddl/ddl.go index 75ebad6f7b3dd..e7154f96c6531 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -24,21 +24,21 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/ngaut/pools" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" tidbutil "github.com/pingcap/tidb/util" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 70109e9214981..3bb2807bf3afa 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -24,17 +24,17 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pkg/errors" ) diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 3968a7d04ffc2..d385df975f84d 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -20,14 +20,14 @@ import ( "github.com/coreos/etcd/clientv3" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 128737f065d74..e8dcbdc954627 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -20,13 +20,13 @@ import ( "time" "github.com/ngaut/pools" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 8aa62e5b684ba..dbba30142bea0 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -18,11 +18,11 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 4b12de2d2f419..bde86b2108853 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -19,12 +19,12 @@ import ( "math" "sync" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/ddl/fail_db_test.go b/ddl/fail_db_test.go index 724d59bdcf621..a55a64027ee6b 100644 --- a/ddl/fail_db_test.go +++ b/ddl/fail_db_test.go @@ -19,11 +19,11 @@ import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testkit" diff --git a/ddl/fail_test.go b/ddl/fail_test.go index f89c4a9aae2ff..f182dbe388c5f 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -16,8 +16,8 @@ package ddl import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/types" "golang.org/x/net/context" ) diff --git a/ddl/foreign_key.go b/ddl/foreign_key.go index e9aa4c0eb43ad..d6c3de7d75c1e 100644 --- a/ddl/foreign_key.go +++ b/ddl/foreign_key.go @@ -14,9 +14,9 @@ package ddl import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pkg/errors" ) diff --git a/ddl/foreign_key_test.go b/ddl/foreign_key_test.go index 75f5700a6b614..386b62e604e4e 100644 --- a/ddl/foreign_key_test.go +++ b/ddl/foreign_key_test.go @@ -18,9 +18,9 @@ import ( "sync" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/testleak" diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 899249a9fb658..3120b2d907258 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -14,7 +14,7 @@ package ddl import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/table" "github.com/pkg/errors" ) diff --git a/ddl/index.go b/ddl/index.go index 5eedd76f3dbc0..1f3e7993b0391 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -19,13 +19,13 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index ec7d1ca73b3fa..518b0847471c7 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -15,10 +15,10 @@ package ddl import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/ddl/mock.go b/ddl/mock.go index 3ab786fd0138c..6060eb9bba5bb 100644 --- a/ddl/mock.go +++ b/ddl/mock.go @@ -18,8 +18,8 @@ import ( "time" "github.com/coreos/etcd/clientv3" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pkg/errors" "golang.org/x/net/context" diff --git a/ddl/partition.go b/ddl/partition.go index 9a44f88e80dc3..fc28e0ee49de6 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -19,12 +19,12 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/ddl/reorg.go b/ddl/reorg.go index 035146c9976b1..73d58e16f8bbc 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -18,15 +18,15 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 84ba4b94bf60c..1bcfe9d21c402 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -17,9 +17,9 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" "golang.org/x/net/context" ) diff --git a/ddl/schema.go b/ddl/schema.go index 8e3272c033e1c..0c5cc0e82e21e 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -14,9 +14,9 @@ package ddl import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pkg/errors" ) diff --git a/ddl/schema_test.go b/ddl/schema_test.go index ffe058822e49b..e7fd0941eef77 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -17,12 +17,12 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/ddl/session_pool.go b/ddl/session_pool.go index 2fa4f39ce6966..7710355ce794b 100644 --- a/ddl/session_pool.go +++ b/ddl/session_pool.go @@ -17,7 +17,7 @@ import ( "sync" "github.com/ngaut/pools" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" "github.com/pkg/errors" diff --git a/ddl/stat_test.go b/ddl/stat_test.go index ac19d7984daf9..304be372c5176 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -17,7 +17,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" diff --git a/ddl/table.go b/ddl/table.go index cf7643f1a5ff4..295161a24b618 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -18,12 +18,12 @@ import ( "strconv" "strings" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pkg/errors" diff --git a/ddl/table_split_test.go b/ddl/table_split_test.go index 01a4bcd6def75..697f18c16aa64 100644 --- a/ddl/table_split_test.go +++ b/ddl/table_split_test.go @@ -18,8 +18,8 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" diff --git a/ddl/table_test.go b/ddl/table_test.go index 7fef7c9bd50b5..ad39b3abd9f41 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -17,11 +17,11 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/ddl/util/event.go b/ddl/util/event.go index d0575a76bccd8..a689c445c13c4 100644 --- a/ddl/util/event.go +++ b/ddl/util/event.go @@ -16,7 +16,7 @@ package util import ( "fmt" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" ) // Event is an event that a ddl operation happened. diff --git a/ddl/util/util.go b/ddl/util/util.go index f2b7c63a4ff14..c9a3c42a65633 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -17,9 +17,9 @@ import ( "encoding/hex" "fmt" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index b06dd9afa6c56..8d20dbd89f755 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -19,13 +19,13 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/execdetails" diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 5dddc206171d3..6dd12c6f20c22 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -16,8 +16,8 @@ package distsql import ( "math" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" diff --git a/distsql/select_result.go b/distsql/select_result.go index aca9ee0e9f561..9558f9e680ccc 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -16,11 +16,11 @@ package distsql import ( "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" diff --git a/distsql/stream.go b/distsql/stream.go index f5bd33583ccd6..0f651a53dc721 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -14,11 +14,11 @@ package distsql import ( + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" diff --git a/domain/domain.go b/domain/domain.go index 280112c45a76f..c59012a0b69e5 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -25,20 +25,20 @@ import ( "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/ngaut/pools" "github.com/ngaut/sync2" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/domain/domain_test.go b/domain/domain_test.go index 63f90deda925e..c516632a99487 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -19,8 +19,8 @@ import ( "github.com/ngaut/pools" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/domain/global_vars_cache.go b/domain/global_vars_cache.go index a246f5af20f6c..1b316b4195ad7 100644 --- a/domain/global_vars_cache.go +++ b/domain/global_vars_cache.go @@ -17,7 +17,7 @@ import ( "sync" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/util/chunk" ) diff --git a/domain/info.go b/domain/info.go index 7d11e63436957..226b395eed0c4 100644 --- a/domain/info.go +++ b/domain/info.go @@ -20,9 +20,9 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/coreos/etcd/clientv3/concurrency" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/printer" diff --git a/domain/schema_checker_test.go b/domain/schema_checker_test.go index b148525d200a2..bad9db98d0dc8 100644 --- a/domain/schema_checker_test.go +++ b/domain/schema_checker_test.go @@ -17,7 +17,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" ) func (s *testSuite) TestSchemaCheckerSimple(c *C) { diff --git a/domain/topn_slow_query.go b/domain/topn_slow_query.go index f8ae69c0be67e..0bf6721454d6c 100644 --- a/domain/topn_slow_query.go +++ b/domain/topn_slow_query.go @@ -19,7 +19,7 @@ import ( "sync" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/util/execdetails" ) diff --git a/executor/adapter.go b/executor/adapter.go index 23abc549d1ef2..a31ed75bcf3de 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -20,21 +20,22 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -44,7 +45,7 @@ type processinfoSetter interface { SetProcessInfo(string) } -// recordSet wraps an executor, implements ast.RecordSet interface +// recordSet wraps an executor, implements sqlexec.RecordSet interface type recordSet struct { fields []*ast.ResultField executor Executor @@ -125,7 +126,7 @@ func (a *recordSet) Close() error { return errors.Trace(err) } -// ExecStmt implements the ast.Statement interface, it builds a planner.Plan to an ast.Statement. +// ExecStmt implements the sqlexec.Statement interface, it builds a planner.Plan to an sqlexec.Statement. type ExecStmt struct { // InfoSchema stores a reference to the schema information. InfoSchema infoschema.InfoSchema @@ -186,8 +187,8 @@ func (a *ExecStmt) RebuildPlan() (int64, error) { // Exec builds an Executor from a plan. If the Executor doesn't return result, // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns -// result, execution is done after this function returns, in the returned ast.RecordSet Next method. -func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { +// result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. +func (a *ExecStmt) Exec(ctx context.Context) (sqlexec.RecordSet, error) { a.StartTime = time.Now() sctx := a.Ctx if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { @@ -252,7 +253,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { }, nil } -func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Context, e Executor, pi processinfoSetter) (ast.RecordSet, error) { +func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Context, e Executor, pi processinfoSetter) (sqlexec.RecordSet, error) { // Check if "tidb_snapshot" is set for the write executors. // In history read mode, we can not do write operations. switch e.(type) { diff --git a/executor/admin.go b/executor/admin.go index 1fe645ae15de3..d7dd92d6cc36c 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -16,17 +16,17 @@ package executor import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" diff --git a/executor/admin_test.go b/executor/admin_test.go index fa3ed027cb605..a82096707d03e 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -17,9 +17,9 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go index ac4c1df5f6b22..2f40e218431b4 100644 --- a/executor/aggfuncs/aggfunc_test.go +++ b/executor/aggfuncs/aggfunc_test.go @@ -18,7 +18,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index 7ba78f94585e6..a9a1547abd46f 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -17,10 +17,10 @@ import ( "fmt" "strconv" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" diff --git a/executor/aggfuncs/func_avg.go b/executor/aggfuncs/func_avg.go index f917c60e1d044..b31b0f96ed69a 100644 --- a/executor/aggfuncs/func_avg.go +++ b/executor/aggfuncs/func_avg.go @@ -15,7 +15,7 @@ package aggfuncs import ( "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/aggfuncs/func_avg_test.go b/executor/aggfuncs/func_avg_test.go index 4b5b5667910c2..9a0b4ddec5887 100644 --- a/executor/aggfuncs/func_avg_test.go +++ b/executor/aggfuncs/func_avg_test.go @@ -15,11 +15,11 @@ package aggfuncs_test import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) diff --git a/executor/aggregate.go b/executor/aggregate.go index 3a4c2f588e5ca..2debbc0a761c3 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -17,9 +17,9 @@ import ( "sync" "time" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f8a098097575c..da7ab15cea6b8 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -15,8 +15,8 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" ) diff --git a/executor/analyze.go b/executor/analyze.go index 12e49ff22649d..85fa0987cfe88 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -17,11 +17,11 @@ import ( "runtime" "strconv" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 2fc3fdf7dd924..ab1a20b483494 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -18,9 +18,9 @@ import ( "strings" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testkit" diff --git a/executor/batch_checker.go b/executor/batch_checker.go index 6f3ae9254ff2b..558e25884a55e 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -14,8 +14,8 @@ package executor import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" diff --git a/executor/builder.go b/executor/builder.go index 69aa6dae7185c..1953f8b3c8aff 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -24,7 +24,9 @@ import ( "github.com/cznic/mathutil" "github.com/cznic/sortutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor/aggfuncs" @@ -33,8 +35,6 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" diff --git a/executor/checksum.go b/executor/checksum.go index 04c5d18dc2dcd..e9d6832096339 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -16,9 +16,9 @@ package executor import ( "strconv" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/compiler.go b/executor/compiler.go index a56ed97462282..80d9d66b8d794 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -17,7 +17,7 @@ import ( "fmt" "github.com/opentracing/opentracing-go" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" diff --git a/executor/ddl.go b/executor/ddl.go index 34b4744385083..ba7e9c02390c0 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -17,12 +17,12 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 45884d1c01e64..9c9a6e42be2e6 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -20,13 +20,13 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testkit" diff --git a/executor/delete.go b/executor/delete.go index 026a1975666d8..c5b7c8283fe34 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -14,7 +14,7 @@ package executor import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" diff --git a/executor/distsql.go b/executor/distsql.go index 8f7beae3cc28e..89caee248ecf1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -22,17 +22,17 @@ import ( "time" "unsafe" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 070c10233e1be..c4b29c80071ca 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -22,9 +22,9 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/testkit" diff --git a/executor/errors.go b/executor/errors.go index e0aae2b46d2db..ad1c7d9b36344 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -14,8 +14,8 @@ package executor import ( - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" ) // Error codes that are not mapping to mysql error codes. diff --git a/executor/executor.go b/executor/executor.go index e3e412097d520..c2693e0304292 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -21,21 +21,21 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index ef1a158fc0801..51c8dc10370f9 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -17,10 +17,10 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" diff --git a/executor/executor_test.go b/executor/executor_test.go index 7953f1123f295..75da5b6b09298 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -29,14 +29,15 @@ import ( "github.com/golang/protobuf/proto" . "github.com/pingcap/check" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -50,7 +51,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" diff --git a/executor/grant.go b/executor/grant.go index 8bff64ddba034..2886e1de36791 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -17,11 +17,11 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/grant_test.go b/executor/grant_test.go index db198f94f454e..bb3d33262ebea 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -18,7 +18,7 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/testkit" ) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index b42684c451f0b..aa662bd000d62 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -21,11 +21,11 @@ import ( "time" "unsafe" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" diff --git a/executor/insert.go b/executor/insert.go index a62a2524c8003..43a4dbded119b 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -14,9 +14,9 @@ package executor import ( + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" diff --git a/executor/insert_common.go b/executor/insert_common.go index f3f2d413069d8..341ad23b3e9ad 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -16,11 +16,11 @@ package executor import ( "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/insert_test.go b/executor/insert_test.go index 0456096d9eb4b..6a43c0ba300d9 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -15,8 +15,8 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" ) diff --git a/executor/join.go b/executor/join.go index 60f9587edc4f9..e363ea54b892a 100644 --- a/executor/join.go +++ b/executor/join.go @@ -19,9 +19,9 @@ import ( "time" "unsafe" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/load_data.go b/executor/load_data.go index 83171cbbd8589..6ab6c7aa5050c 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -18,7 +18,7 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/executor/metrics_test.go b/executor/metrics_test.go index daf4c6fd9a5fc..989afe460fb90 100644 --- a/executor/metrics_test.go +++ b/executor/metrics_test.go @@ -17,8 +17,8 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/parser" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testkit" diff --git a/executor/pkg_test.go b/executor/pkg_test.go index e75e5fade8a55..0c9c0b624af35 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -4,9 +4,9 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/point_get.go b/executor/point_get.go index 3d41181675bfc..e74c29e8aafbb 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -14,10 +14,10 @@ package executor import ( + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" diff --git a/executor/prepared.go b/executor/prepared.go index 962d35db6ae3c..6e704b3a78bb2 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -17,14 +17,15 @@ import ( "math" "sort" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" @@ -38,7 +39,7 @@ var ( ) type paramMarkerSorter struct { - markers []*ast.ParamMarkerExpr + markers []ast.ParamMarkerExpr } func (p *paramMarkerSorter) Len() int { @@ -46,7 +47,7 @@ func (p *paramMarkerSorter) Len() int { } func (p *paramMarkerSorter) Less(i, j int) bool { - return p.markers[i].Offset < p.markers[j].Offset + return p.markers[i].(*driver.ParamMarkerExpr).Offset < p.markers[j].(*driver.ParamMarkerExpr).Offset } func (p *paramMarkerSorter) Swap(i, j int) { @@ -54,7 +55,7 @@ func (p *paramMarkerSorter) Swap(i, j int) { } type paramMarkerExtractor struct { - markers []*ast.ParamMarkerExpr + markers []ast.ParamMarkerExpr } func (e *paramMarkerExtractor) Enter(in ast.Node) (ast.Node, bool) { @@ -62,7 +63,7 @@ func (e *paramMarkerExtractor) Enter(in ast.Node) (ast.Node, bool) { } func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) { - if x, ok := in.(*ast.ParamMarkerExpr); ok { + if x, ok := in.(*driver.ParamMarkerExpr); ok { e.markers = append(e.markers, x) } return in, true @@ -144,7 +145,7 @@ func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error { sort.Sort(sorter) e.ParamCount = len(sorter.markers) for i := 0; i < e.ParamCount; i++ { - sorter.markers[i].Order = i + sorter.markers[i].SetOrder(i) } prepared := &ast.Prepared{ Stmt: stmt, @@ -155,7 +156,7 @@ func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error { // We try to build the real statement of preparedStmt. for i := range prepared.Params { - prepared.Params[i].SetDatum(types.NewIntDatum(0)) + prepared.Params[i].(*driver.ParamMarkerExpr).Datum = types.NewIntDatum(0) } var p plannercore.Plan p, err = plannercore.BuildLogicalPlan(e.ctx, stmt, e.is) @@ -238,7 +239,7 @@ func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error { } // CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement. -func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (ast.Statement, error) { +func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (sqlexec.Statement, error) { execStmt := &ast.ExecuteStmt{ExecID: ID} if err := ResetContextOfStmt(ctx, execStmt); err != nil { return nil, err diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 3e19ca79b9e95..2c5affa9b814a 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -18,9 +18,9 @@ import ( "strings" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/executor" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" "github.com/pkg/errors" "golang.org/x/net/context" diff --git a/executor/revoke.go b/executor/revoke.go index 3f2edb6c7249b..35992f885615f 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -16,10 +16,10 @@ package executor import ( "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/revoke_test.go b/executor/revoke_test.go index cc3025f7a55e5..de194356b0a95 100644 --- a/executor/revoke_test.go +++ b/executor/revoke_test.go @@ -18,7 +18,7 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/testkit" ) diff --git a/executor/set.go b/executor/set.go index 76562a6a4f8f9..f7f7853c8717c 100644 --- a/executor/set.go +++ b/executor/set.go @@ -18,14 +18,14 @@ import ( "strings" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" diff --git a/executor/set_test.go b/executor/set_test.go index a9ecf17a286f1..5574d32bb4b08 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -15,9 +15,9 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "golang.org/x/net/context" diff --git a/executor/show.go b/executor/show.go index 72d20117ccdca..e8b7e459e9484 100644 --- a/executor/show.go +++ b/executor/show.go @@ -22,19 +22,19 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/auth" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/format" "github.com/pkg/errors" diff --git a/executor/show_stats.go b/executor/show_stats.go index 6c8c4f0ddc258..a6e77cba4de7a 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -16,8 +16,8 @@ package executor import ( "time" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" diff --git a/executor/show_test.go b/executor/show_test.go index 21398c6c30ca6..ba272d2686fe1 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -19,16 +19,16 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "github.com/pkg/errors" diff --git a/executor/simple.go b/executor/simple.go index aed2e196973f3..ef4769d1092f2 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -17,16 +17,16 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" diff --git a/executor/simple_test.go b/executor/simple_test.go index a154304076d2c..e97d21b142902 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -15,14 +15,14 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/testkit" "golang.org/x/net/context" ) diff --git a/executor/statement_context_test.go b/executor/statement_context_test.go index 5808488fed575..df7d86d4115a8 100644 --- a/executor/statement_context_test.go +++ b/executor/statement_context_test.go @@ -18,8 +18,8 @@ import ( "unicode/utf8" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" ) diff --git a/executor/table_reader.go b/executor/table_reader.go index a941e3706bee8..3c81d402d141c 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -16,8 +16,8 @@ package executor import ( "time" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/distsql" - "github.com/pingcap/tidb/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" diff --git a/executor/trace.go b/executor/trace.go index 588df29065c1b..53747498e6667 100644 --- a/executor/trace.go +++ b/executor/trace.go @@ -18,7 +18,7 @@ import ( "github.com/opentracing/basictracer-go" opentracing "github.com/opentracing/opentracing-go" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/tracing" diff --git a/executor/union_scan.go b/executor/union_scan.go index 1ac6689c1b58d..aad9bb157fef4 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -17,8 +17,8 @@ import ( "sort" "time" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/executor/update.go b/executor/update.go index c4be098057a52..b257425e1d41b 100644 --- a/executor/update.go +++ b/executor/update.go @@ -14,9 +14,9 @@ package executor import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/executor/write.go b/executor/write.go index de34f67bce23b..7a4695bccd1d0 100644 --- a/executor/write.go +++ b/executor/write.go @@ -16,9 +16,9 @@ package executor import ( "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" diff --git a/executor/write_test.go b/executor/write_test.go index f4d9f0542b370..fb5e1b8b97388 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -19,9 +19,9 @@ import ( "sync/atomic" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" diff --git a/expression/aggregation/agg_to_pb.go b/expression/aggregation/agg_to_pb.go index 31a743d59621a..59d09db237701 100644 --- a/expression/aggregation/agg_to_pb.go +++ b/expression/aggregation/agg_to_pb.go @@ -14,7 +14,7 @@ package aggregation import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" diff --git a/expression/aggregation/agg_to_pb_test.go b/expression/aggregation/agg_to_pb_test.go index 2a17b120e5394..af8ca6d26f8a2 100644 --- a/expression/aggregation/agg_to_pb_test.go +++ b/expression/aggregation/agg_to_pb_test.go @@ -18,10 +18,10 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "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" diff --git a/expression/aggregation/aggregation.go b/expression/aggregation/aggregation.go index 9d2ac730767e3..62eba44691139 100644 --- a/expression/aggregation/aggregation.go +++ b/expression/aggregation/aggregation.go @@ -16,14 +16,14 @@ package aggregation import ( "bytes" "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" diff --git a/expression/aggregation/aggregation_test.go b/expression/aggregation/aggregation_test.go index 9040f06aeccc9..0ebbe8a330eef 100644 --- a/expression/aggregation/aggregation_test.go +++ b/expression/aggregation/aggregation_test.go @@ -17,9 +17,9 @@ import ( "math" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" diff --git a/expression/aggregation/avg.go b/expression/aggregation/avg.go index b934e6be2eb6a..04d70fa985d04 100644 --- a/expression/aggregation/avg.go +++ b/expression/aggregation/avg.go @@ -15,9 +15,9 @@ package aggregation import ( "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" diff --git a/expression/aggregation/bench_test.go b/expression/aggregation/bench_test.go index 5b7c2bcee2e4f..e49deebe00da3 100644 --- a/expression/aggregation/bench_test.go +++ b/expression/aggregation/bench_test.go @@ -16,9 +16,9 @@ package aggregation import ( "testing" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" ) diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 7c47ec85c0d64..e631c9cd6629d 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -21,14 +21,14 @@ import ( "strings" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" ) // AggFuncDesc describes an aggregation function signature, only used in planner. diff --git a/expression/bench_test.go b/expression/bench_test.go index 1704afa516d3c..bc4a58ed81b45 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -20,12 +20,12 @@ import ( "testing" "time" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" ) diff --git a/expression/builtin.go b/expression/builtin.go index ba165008278f9..51379440e7e28 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -18,13 +18,13 @@ package expression import ( - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" ) diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 85c620264d25d..1f7369699519a 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -18,9 +18,9 @@ import ( "math" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" diff --git a/expression/builtin_arithmetic_test.go b/expression/builtin_arithmetic_test.go index 68fe74333ef5a..61a47bbc5d334 100644 --- a/expression/builtin_arithmetic_test.go +++ b/expression/builtin_arithmetic_test.go @@ -17,8 +17,8 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index b3e8a539b9f7f..74b3e2cee22a5 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -26,14 +26,14 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index dd9d00daed014..1e4ef80c7a0e4 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -20,11 +20,11 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" ) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index c83f861b546b7..0744720694378 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -16,11 +16,11 @@ package expression import ( "math" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_compare_test.go b/expression/builtin_compare_test.go index 4ce6c3386e750..24cd01ea541dd 100644 --- a/expression/builtin_compare_test.go +++ b/expression/builtin_compare_test.go @@ -17,8 +17,8 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_control.go b/expression/builtin_control.go index 9900a22bed4a8..fc8013f6096cc 100644 --- a/expression/builtin_control.go +++ b/expression/builtin_control.go @@ -15,11 +15,11 @@ package expression import ( "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" diff --git a/expression/builtin_control_test.go b/expression/builtin_control_test.go index de589e1a0b4fc..37101ec326f88 100644 --- a/expression/builtin_control_test.go +++ b/expression/builtin_control_test.go @@ -18,7 +18,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index 94eaef8a84574..7772cd344a8a2 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -28,11 +28,11 @@ import ( "io" "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/encrypt" "github.com/pkg/errors" diff --git a/expression/builtin_encryption_test.go b/expression/builtin_encryption_test.go index 99f29ea972933..7a1b09254f6f2 100644 --- a/expression/builtin_encryption_test.go +++ b/expression/builtin_encryption_test.go @@ -18,9 +18,9 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" diff --git a/expression/builtin_info.go b/expression/builtin_info.go index b498e96f289e1..e26efd1e27c0a 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -18,7 +18,7 @@ package expression import ( - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_info_test.go b/expression/builtin_info_test.go index c71ce15c88bd5..b28b6b80defe0 100644 --- a/expression/builtin_info_test.go +++ b/expression/builtin_info_test.go @@ -17,11 +17,11 @@ import ( "math" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/printer" diff --git a/expression/builtin_json.go b/expression/builtin_json.go index 9a1fdb318546b..0ff0f53eb8d77 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -14,8 +14,8 @@ package expression import ( - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index ab4be5790ad9f..d6e6542fc119b 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -15,7 +15,7 @@ package expression import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 68ee63a974114..f0770420e0cf9 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -27,7 +27,7 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index db01a7864dfe5..4a557302bba65 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -20,10 +20,10 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index b249551247340..c046861911395 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -21,11 +21,11 @@ import ( "strings" "time" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" "github.com/twinj/uuid" diff --git a/expression/builtin_miscellaneous_test.go b/expression/builtin_miscellaneous_test.go index 213ed1e8d1cc7..79143dad631a4 100644 --- a/expression/builtin_miscellaneous_test.go +++ b/expression/builtin_miscellaneous_test.go @@ -17,7 +17,7 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/builtin_op.go b/expression/builtin_op.go index 1c4109da9c1b1..e98d83af53835 100644 --- a/expression/builtin_op.go +++ b/expression/builtin_op.go @@ -17,8 +17,8 @@ import ( "fmt" "math" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/builtin_op_test.go b/expression/builtin_op_test.go index 5e726be8745c7..f57f17aab557f 100644 --- a/expression/builtin_op_test.go +++ b/expression/builtin_op_test.go @@ -17,7 +17,7 @@ import ( "math" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/builtin_other.go b/expression/builtin_other.go index 7e07abeae88cc..44cef208b8ebc 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -16,7 +16,7 @@ package expression import ( "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index 893dad92110f3..713965ce81f03 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -18,8 +18,8 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" diff --git a/expression/builtin_string.go b/expression/builtin_string.go index fd429d893a354..ac1dc3f2a8b53 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -27,12 +27,12 @@ import ( "strings" "unicode/utf8" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index c90068ae365fe..944b763bb8121 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -20,12 +20,12 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/builtin_test.go b/expression/builtin_test.go index b12f49ada982a..2be7dccf751cf 100644 --- a/expression/builtin_test.go +++ b/expression/builtin_test.go @@ -17,12 +17,12 @@ import ( "reflect" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" diff --git a/expression/builtin_time.go b/expression/builtin_time.go index f9c6a9425b81b..0e7ff2fe98aa5 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -26,10 +26,10 @@ import ( "time" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index f2a46abb302eb..6fe40c5064895 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -20,13 +20,13 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/chunk_executor.go b/expression/chunk_executor.go index 857211d65a3e1..76efd21031af0 100644 --- a/expression/chunk_executor.go +++ b/expression/chunk_executor.go @@ -16,8 +16,8 @@ package expression import ( "strconv" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/column.go b/expression/column.go index 2c47c38b451c3..5306565373996 100644 --- a/expression/column.go +++ b/expression/column.go @@ -17,8 +17,8 @@ import ( "fmt" "strings" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" diff --git a/expression/column_test.go b/expression/column_test.go index 62a180aab604a..8f11509783055 100644 --- a/expression/column_test.go +++ b/expression/column_test.go @@ -15,8 +15,8 @@ package expression import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/constant.go b/expression/constant.go index c1479355d5c13..e2725ed553727 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -16,10 +16,10 @@ package expression import ( "fmt" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/constant_fold.go b/expression/constant_fold.go index 0f7e85fa18ba9..738def5d041bb 100644 --- a/expression/constant_fold.go +++ b/expression/constant_fold.go @@ -14,7 +14,7 @@ package expression import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/util/chunk" log "github.com/sirupsen/logrus" ) diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 8bf71a42c4abf..edfc634c7cfb1 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -14,10 +14,10 @@ package expression import ( - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/disjointset" diff --git a/expression/constant_test.go b/expression/constant_test.go index bda66eb8b3f46..e864c6e3a4fbb 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -19,9 +19,9 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index c2df220d869bf..7d90ff1db72af 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -17,8 +17,8 @@ import ( "fmt" "time" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 2df78ca40fe9b..25162e2edcdf2 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -17,7 +17,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/errors.go b/expression/errors.go index 517398bd0a9a3..f2560e96da774 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -14,9 +14,9 @@ package expression import ( - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" ) diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index 95999a46b6615..b938d5eb6b88f 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -18,13 +18,13 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 6dee4893ba66c..1e5384c60de6a 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -16,13 +16,13 @@ package expression import ( "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tipb/go-tipb" diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index a4d96e571411d..edd06d9b3dcd3 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -17,11 +17,11 @@ import ( "encoding/json" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" ) diff --git a/expression/expression.go b/expression/expression.go index 072bc5a62f1bf..5d6fa3a276723 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -17,12 +17,12 @@ import ( goJSON "encoding/json" "fmt" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/expression_test.go b/expression/expression_test.go index 0fe3b7da63bd5..d96135617bc7f 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -17,9 +17,9 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/function_traits.go b/expression/function_traits.go index ae3f4b05ea888..8d8913fd354d4 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -14,7 +14,7 @@ package expression import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" ) // UnCacheableFunctions stores functions which can not be cached to plan cache. diff --git a/expression/function_traits_test.go b/expression/function_traits_test.go index 13982d9e85907..11a21da4beb65 100644 --- a/expression/function_traits_test.go +++ b/expression/function_traits_test.go @@ -15,7 +15,7 @@ package expression import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/util/testleak" ) diff --git a/expression/helper.go b/expression/helper.go index fe193651712a1..7759d944b756b 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -18,12 +18,13 @@ import ( "strings" "time" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pkg/errors" ) @@ -74,7 +75,7 @@ func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int) (d ty return d, errors.Trace(err) } } - case *ast.ValueExpr: + case *driver.ValueExpr: switch x.Kind() { case types.KindString: value, err = types.ParseTime(sc, x.GetString(), tp, fsp) diff --git a/expression/helper_test.go b/expression/helper_test.go index f63109a5a28e6..e0acecfdf817b 100644 --- a/expression/helper_test.go +++ b/expression/helper_test.go @@ -18,9 +18,9 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" diff --git a/expression/integration_test.go b/expression/integration_test.go index f2ac973bbff5d..83567ebcc5159 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -21,19 +21,19 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 34f0d7673e7ec..7705c2ff48b08 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -17,12 +17,12 @@ import ( "bytes" "fmt" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/expression/scalar_function_test.go b/expression/scalar_function_test.go index f4088c2b010c1..ef466355c1f53 100644 --- a/expression/scalar_function_test.go +++ b/expression/scalar_function_test.go @@ -17,9 +17,9 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/schema.go b/expression/schema.go index ffa41b72498d9..50cdb9d6b4dbf 100644 --- a/expression/schema.go +++ b/expression/schema.go @@ -16,7 +16,7 @@ package expression import ( "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pkg/errors" ) diff --git a/expression/schema_test.go b/expression/schema_test.go index d28a41d7263f7..6ea5de276245b 100644 --- a/expression/schema_test.go +++ b/expression/schema_test.go @@ -17,7 +17,7 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" ) // generateKeys4Schema will generate keys for a given schema. Used only in this file. diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index 734228c8a45d4..6103cf2ddd4bb 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -14,13 +14,14 @@ package expression import ( - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pkg/errors" ) @@ -107,7 +108,7 @@ func (sr *simpleRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok boo return originInNode, false } sr.push(column) - case *ast.ValueExpr: + case *driver.ValueExpr: value := &Constant{Value: v.Datum, RetType: &v.Type} sr.push(value) case *ast.FuncCallExpr: @@ -137,10 +138,10 @@ func (sr *simpleRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok boo if v.Sel == nil { sr.inToExpression(len(v.List), v.Not, &v.Type) } - case *ast.ParamMarkerExpr: + case *driver.ParamMarkerExpr: tp := types.NewFieldType(mysql.TypeUnspecified) types.DefaultParamTypeForValue(v.GetValue(), tp) - value := &Constant{Value: v.Datum, RetType: tp} + value := &Constant{Value: v.ValueExpr.Datum, RetType: tp} sr.push(value) case *ast.RowExpr: sr.rowToScalarFunc(v) diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index bb86b140d5caf..1517cc674443f 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -17,14 +17,14 @@ import ( "math" . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" diff --git a/expression/util.go b/expression/util.go index 70cf52e9a7df7..1b3b3afaa4847 100644 --- a/expression/util.go +++ b/expression/util.go @@ -19,11 +19,11 @@ import ( "time" "unicode" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" diff --git a/expression/util_test.go b/expression/util_test.go index 2fdbc067dff9e..7e810180dc264 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/infoschema/builder.go b/infoschema/builder.go index 85caaee73e17f..2f6b1a6ebc3b8 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -17,9 +17,9 @@ import ( "fmt" "sort" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/perfschema" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 94469ce7f5c45..79da4b273b2b0 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -17,12 +17,12 @@ import ( "sort" "sync/atomic" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/terror" ) var ( diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 840730638092a..13b4217fcdbf1 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -18,11 +18,11 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/perfschema" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" diff --git a/infoschema/tables.go b/infoschema/tables.go index 80e80a54144b7..0d8082da5651a 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -19,16 +19,16 @@ import ( "sync" "time" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" ) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 0bdd327075f8a..7c9ded48def9e 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -15,11 +15,11 @@ package infoschema_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" ) diff --git a/kv/error.go b/kv/error.go index 68be6b86f04fb..b705165238783 100644 --- a/kv/error.go +++ b/kv/error.go @@ -16,8 +16,8 @@ package kv import ( "strings" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" ) // KV error codes. diff --git a/kv/memdb_buffer.go b/kv/memdb_buffer.go index 9a57e2cfe16fc..1de19e527af5b 100644 --- a/kv/memdb_buffer.go +++ b/kv/memdb_buffer.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/goleveldb/leveldb/iterator" "github.com/pingcap/goleveldb/leveldb/memdb" "github.com/pingcap/goleveldb/leveldb/util" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/kv/txn.go b/kv/txn.go index 6a8710dbf51dd..4bd5904c7976b 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -18,7 +18,7 @@ import ( "math/rand" "time" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/kv/union_store_test.go b/kv/union_store_test.go index 2b06b2f39d605..3297b14f969bc 100644 --- a/kv/union_store_test.go +++ b/kv/union_store_test.go @@ -15,7 +15,7 @@ package kv import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/testleak" ) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 095b7ba271db4..4c8ba2e189473 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -20,10 +20,10 @@ import ( "time" "github.com/cznic/mathutil" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 15caebe1cd36d..173466190c53e 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -20,10 +20,10 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/store/mockstore" "github.com/pkg/errors" ) diff --git a/meta/meta.go b/meta/meta.go index 877453d67c012..5234bc30f3ef0 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -24,12 +24,12 @@ import ( "sync" "time" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/structure" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/meta/meta_test.go b/meta/meta_test.go index 823f71b3e0167..81a987a3fb4ea 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -19,8 +19,8 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" diff --git a/metrics/server.go b/metrics/server.go index e8417274d2775..c18058f1461eb 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -16,7 +16,7 @@ package metrics import ( "strconv" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" ) diff --git a/model/ddl.go b/model/ddl.go index 23db9df3e2d90..843dbe1850726 100644 --- a/model/ddl.go +++ b/model/ddl.go @@ -20,7 +20,7 @@ import ( "sync" "time" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/model/model.go b/model/model.go index c488ac86a4791..71c2e77835d60 100644 --- a/model/model.go +++ b/model/model.go @@ -19,9 +19,8 @@ import ( "strings" "time" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/types" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" ) @@ -114,7 +113,7 @@ func (c *ColumnInfo) SetDefaultValue(value interface{}) error { // bit type default value will store in DefaultValueBit for fix bit default value decode/encode bug. func (c *ColumnInfo) GetDefaultValue() interface{} { if c.Tp == mysql.TypeBit && c.DefaultValueBit != nil { - return hack.String(c.DefaultValueBit) + return string(c.DefaultValueBit) } return c.DefaultValue } diff --git a/model/model_test.go b/model/model_test.go index a7cf298adb30d..77aa95588e61a 100644 --- a/model/model_test.go +++ b/model/model_test.go @@ -20,7 +20,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" ) diff --git a/mysql/const_test.go b/mysql/const_test.go index 4ad54fd5df097..11c61157b7cd9 100644 --- a/mysql/const_test.go +++ b/mysql/const_test.go @@ -18,10 +18,10 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" diff --git a/owner/fail_test.go b/owner/fail_test.go index acc94606441ec..5574c34846014 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -23,7 +23,7 @@ import ( "github.com/coreos/etcd/clientv3" gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" diff --git a/owner/manager.go b/owner/manager.go index 6ac8d8eac285f..640bb5d571cf6 100644 --- a/owner/manager.go +++ b/owner/manager.go @@ -26,8 +26,8 @@ import ( "github.com/coreos/etcd/clientv3/concurrency" "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/mvcc/mvccpb" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/parser/bench_test.go b/parser/bench_test.go deleted file mode 100644 index ddac12302c161..0000000000000 --- a/parser/bench_test.go +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2017 PingCAP, Inc. -// -// 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package parser - -import ( - "testing" -) - -func BenchmarkSysbenchSelect(b *testing.B) { - parser := New() - sql := "SELECT pad FROM sbtest1 WHERE id=1;" - b.ResetTimer() - for i := 0; i < b.N; i++ { - _, err := parser.Parse(sql, "", "") - if err != nil { - b.Fatal(err) - } - } - b.ReportAllocs() -} - -func BenchmarkParse(b *testing.B) { - var table = []string{ - "insert into t values (1), (2), (3)", - "insert into t values (4), (5), (6), (7)", - "select c from t where c > 2", - } - parser := New() - b.ResetTimer() - for i := 0; i < b.N; i++ { - for _, v := range table { - _, err := parser.Parse(v, "", "") - if err != nil { - b.Failed() - } - } - } - b.ReportAllocs() -} diff --git a/parser/goyacc/main.go b/parser/goyacc/main.go deleted file mode 100644 index 69da64b733ddb..0000000000000 --- a/parser/goyacc/main.go +++ /dev/null @@ -1,819 +0,0 @@ -// Copyright 2016 PingCAP, Inc. -// -// 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Copyright 2014 The goyacc Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -// This source code uses portions of code previously published in the Go tool -// yacc[0] program, the respective license can be found in the LICENSE-GO-YACC -// file. - -// Goyacc is a version of yacc generating Go parsers. -// -// Usage -// -// Note: If no non flag arguments are given, goyacc reads standard input. -// -// goyacc [options] [input] -// -// options and (defaults) -// -c Report state closures. (false) -// -cr Check all states are reducible. (false) -// -dlval Debug value when runtime yyDebug >= 3. ("lval") -// -dlvalf Debug format of -dlval. ("%+v") -// -ex Explain how were conflicts resolved. (false) -// -l Disable line directives, for compatibility only - ignored. (false) -// -la Report all lookahead sets. (false) -// -o outputFile Parser output. ("y.go") -// -p prefix Name prefix to use in generated code. ("yy") -// -v reportFile Create grammar report. ("y.output") -// -xe examplesFile Generate error messages by examples. ("") -// -xegen examplesFile Generate a file suitable for -xe automatically from the grammar. -// The file must not exist. ("") -// -// -// -// Changelog -// -// 2015-03-24: The search for a custom error message is now extended to include -// also the last state that was shifted into, if any. This change resolves a -// problem in which a lookahead symbol is valid for a reduce action in state A, -// but the same symbol is later never accepted by any shift action in some -// state B which is popped from the state stack after the reduction is -// performed. The computed from example state is A but when the error is -// actually detected, the state is now B and the custom error was thus not -// used. -// -// 2015-02-23: Added -xegen flag. It can be used to automagically generate a -// skeleton errors by example file which can be, for example, edited and/or -// submited later as an argument of the -xe option. -// -// 2014-12-18: Support %precedence for better bison compatibility[3]. The -// actual changes are in packages goyacc is dependent on. Goyacc users should -// rebuild the binary: -// -// $ go get -u github.com/cznic/goyacc -// -// 2014-12-02: Added support for the optional yyLexerEx interface. The Reduced -// method can be useful for debugging and/or automatically producing examples -// by parsing code fragments. If it returns true the parser exits immediately -// with return value -1. -// -// Overview -// -// The generated parser is reentrant and mostly backwards compatible with -// parsers generated by go tool yacc[0]. yyParse expects to be given an -// argument that conforms to the following interface: -// -// type yyLexer interface { -// Lex(lval *yySymType) int -// Errorf(format string, a ...interface{}) -// Errors() []error -// } -// -// Optionally the argument to yyParse may implement the following interface: -// -// type yyLexerEx interface { -// yyLexer -// // Hook for recording a reduction. -// Reduced(rule, state int, lval *yySymType) (stop bool) // Client should copy *lval. -// } -// -// Lex should return the token identifier, and place other token information in -// lval (which replaces the usual yylval). Error is equivalent to yyerror in -// the original yacc. -// -// Code inside the parser may refer to the variable yylex, which holds the -// yyLexer passed to Parse. -// -// Multiple grammars compiled into a single program should be placed in -// distinct packages. If that is impossible, the "-p prefix" flag to yacc sets -// the prefix, by default yy, that begins the names of symbols, including -// types, the parser, and the lexer, generated and referenced by yacc's -// generated code. Setting it to distinct values allows multiple grammars to be -// placed in a single package. -// -// Differences wrt go tool yacc -// -// - goyacc implements ideas from "Generating LR Syntax Error Messages from -// Examples"[1]. Use the -xe flag to pass a name of the example file. For more -// details about the example format please see [2]. -// -// - The grammar report includes example token sequences leading to the -// particular state. Can help understanding conflicts. -// -// - Minor changes in parser debug output. -// -// Links -// -// Referenced from elsewhere: -// -// [0]: http://golang.org/cmd/yacc/ -// [1]: http://people.via.ecp.fr/~stilgar/doc/compilo/parser/Generating%20LR%20Syntax%20Error%20Messages.pdf -// [2]: http://godoc.org/github.com/cznic/y#hdr-Error_Examples -// [3]: http://www.gnu.org/software/bison/manual/html_node/Precedence-Only.html#Precedence-Only -package main - -import ( - "bufio" - "bytes" - "flag" - "fmt" - "go/format" - "go/scanner" - "go/token" - "io" - "io/ioutil" - "log" - "os" - "runtime" - "sort" - "strings" - - "github.com/cznic/mathutil" - "github.com/cznic/parser/yacc" - "github.com/cznic/sortutil" - "github.com/cznic/strutil" - "github.com/cznic/y" -) - -var ( - //oNoDefault = flag.Bool("nodefault", false, "disable generating $default actions") - oClosures = flag.Bool("c", false, "report state closures") - oReducible = flag.Bool("cr", false, "check all states are reducible") - oDlval = flag.String("dlval", "lval", "debug value (runtime yyDebug >= 3)") - oDlvalf = flag.String("dlvalf", "%+v", "debug format of -dlval (runtime yyDebug >= 3)") - oLA = flag.Bool("la", false, "report all lookahead sets") - oNoLines = flag.Bool("l", false, "disable line directives (for compatibility ony - ignored)") - oOut = flag.String("o", "y.go", "parser output") - oPref = flag.String("p", "yy", "name prefix to use in generated code") - oReport = flag.String("v", "y.output", "create grammar report") - oResolved = flag.Bool("ex", false, "explain how were conflicts resolved") - oXErrors = flag.String("xe", "", "generate eXtra errors from examples source file") - oXErrorsGen = flag.String("xegen", "", "generate error from examples source file automatically from the grammar") -) - -func main() { - log.SetFlags(0) - - defer func() { - _, file, line, ok := runtime.Caller(2) - if e := recover(); e != nil { - switch { - case ok: - log.Fatalf("%s:%d: panic: %v", file, line, e) - default: - log.Fatalf("panic: %v", e) - } - } - }() - - flag.Parse() - var in string - switch flag.NArg() { - case 0: - in = os.Stdin.Name() - case 1: - in = flag.Arg(0) - default: - log.Fatal("expected at most one non flag argument") - } - - if err := main1(in); err != nil { - switch x := err.(type) { - case scanner.ErrorList: - for _, v := range x { - fmt.Fprintf(os.Stderr, "%v\n", v) - } - os.Exit(1) - default: - log.Fatal(err) - } - } -} - -type symUsed struct { - sym *y.Symbol - used int -} - -type symsUsed []symUsed - -func (s symsUsed) Len() int { return len(s) } -func (s symsUsed) Swap(i, j int) { s[i], s[j] = s[j], s[i] } - -func (s symsUsed) Less(i, j int) bool { - if s[i].used > s[j].used { - return true - } - - if s[i].used < s[j].used { - return false - } - - return strings.ToLower(s[i].sym.Name) < strings.ToLower(s[j].sym.Name) -} - -func main1(in string) (err error) { - var out io.Writer - if nm := *oOut; nm != "" { - var f *os.File - var e error - if f, err = os.Create(nm); err != nil { - return err - } - - defer func() { - if e1 := f.Close(); e1 != nil && err == nil { - err = e1 - } - }() - w := bufio.NewWriter(f) - defer func() { - if e1 := w.Flush(); e1 != nil && err == nil { - err = e1 - } - }() - buf := bytes.NewBuffer(nil) - out = buf - defer func() { - var dest []byte - if dest, e = format.Source(buf.Bytes()); e != nil { - dest = buf.Bytes() - } - - if _, e = w.Write(dest); e != nil && err == nil { - err = e - } - }() - } - - var rep io.Writer - if nm := *oReport; nm != "" { - f, err1 := os.Create(nm) - if err1 != nil { - return err1 - } - - defer func() { - if e := f.Close(); e != nil && err == nil { - err = e - } - }() - w := bufio.NewWriter(f) - defer func() { - if e := w.Flush(); e != nil && err == nil { - err = e - } - }() - rep = w - } - - var xerrors []byte - if nm := *oXErrors; nm != "" { - b, err1 := ioutil.ReadFile(nm) - if err1 != nil { - return err1 - } - - xerrors = b - } - - p, err := y.ProcessFile(token.NewFileSet(), in, &y.Options{ - //NoDefault: *oNoDefault, - AllowConflicts: true, - Closures: *oClosures, - LA: *oLA, - Reducible: *oReducible, - Report: rep, - Resolved: *oResolved, - XErrorsName: *oXErrors, - XErrorsSrc: xerrors, - }) - if err != nil { - return err - } - - if fn := *oXErrorsGen; fn != "" { - f, err := os.OpenFile(fn, os.O_RDWR|os.O_CREATE, 0666) - if err != nil { - return err - } - - b := bufio.NewWriter(f) - if err := p.SkeletonXErrors(b); err != nil { - return err - } - - if err := b.Flush(); err != nil { - return err - } - - if err := f.Close(); err != nil { - return err - } - } - - msu := make(map[*y.Symbol]int, len(p.Syms)) // sym -> usage - for nm, sym := range p.Syms { - if nm == "" || nm == "ε" || nm == "$accept" || nm == "#" { - continue - } - - msu[sym] = 0 - } - var minArg, maxArg int - for _, state := range p.Table { - for _, act := range state { - msu[act.Sym]++ - k, arg := act.Kind() - if k == 'a' { - continue - } - - if k == 'r' { - arg = -arg - } - minArg, maxArg = mathutil.Min(minArg, arg), mathutil.Max(maxArg, arg) - } - } - su := make(symsUsed, 0, len(msu)) - for sym, used := range msu { - su = append(su, symUsed{sym, used}) - } - sort.Sort(su) - - // ----------------------------------------------------------- Prologue - f := strutil.IndentFormatter(out, "\t") - mustFormat(f, "// CAUTION: Generated file - DO NOT EDIT.\n\n") - mustFormat(f, "%s", injectImport(p.Prologue)) - mustFormat(f, ` -type %[1]sSymType %i%s%u - -type %[1]sXError struct { - state, xsym int -} -`, *oPref, p.UnionSrc) - - // ---------------------------------------------------------- Constants - nsyms := map[string]*y.Symbol{} - a := make([]string, 0, len(msu)) - maxTokName := 0 - for sym := range msu { - nm := sym.Name - if nm == "$default" || nm == "$end" || sym.IsTerminal && nm[0] != '\'' && sym.Value > 0 { - maxTokName = mathutil.Max(maxTokName, len(nm)) - a = append(a, nm) - } - nsyms[nm] = sym - } - sort.Strings(a) - mustFormat(f, "\nconst (%i\n") - for _, v := range a { - nm := v - switch nm { - case "error": - nm = *oPref + "ErrCode" - case "$default": - nm = *oPref + "Default" - case "$end": - nm = *oPref + "EOFCode" - } - mustFormat(f, "%s%s = %d\n", nm, strings.Repeat(" ", maxTokName-len(nm)+1), nsyms[v].Value) - } - minArg-- // eg: [-13, 42], minArg -14 maps -13 to 1 so zero cell values -> empty. - mustFormat(f, "\n%sMaxDepth = 200\n", *oPref) - mustFormat(f, "%sTabOfs = %d\n", *oPref, minArg) - mustFormat(f, "%u)") - - // ---------------------------------------------------------- Variables - mustFormat(f, "\n\nvar (%i\n") - - // Lex translation table - mustFormat(f, "%sXLAT = map[int]int{%i\n", *oPref) - xlat := make(map[int]int, len(su)) - var errSym int - for i, v := range su { - if v.sym.Name == "error" { - errSym = i - } - xlat[v.sym.Value] = i - mustFormat(f, "%6d: %3d, // %s (%dx)\n", v.sym.Value, i, v.sym.Name, msu[v.sym]) - } - mustFormat(f, "%u}\n") - - // Symbol names - mustFormat(f, "\n%sSymNames = []string{%i\n", *oPref) - for _, v := range su { - mustFormat(f, "%q,\n", v.sym.Name) - } - mustFormat(f, "%u}\n") - - // Reduction table - mustFormat(f, "\n%sReductions = []struct{xsym, components int}{%i\n", *oPref) - for _, rule := range p.Rules { - mustFormat(f, "{%d, %d},\n", xlat[rule.Sym.Value], len(rule.Components)) - } - mustFormat(f, "%u}\n") - - // XError table - mustFormat(f, "\n%[1]sXErrors = map[%[1]sXError]string{%i\n", *oPref) - for _, xerr := range p.XErrors { - state := xerr.Stack[len(xerr.Stack)-1] - xsym := -1 - if xerr.Lookahead != nil { - xsym = xlat[xerr.Lookahead.Value] - } - mustFormat(f, "%[1]sXError{%d, %d}: \"%s\",\n", *oPref, state, xsym, xerr.Msg) - } - mustFormat(f, "%u}\n\n") - - // Parse table - tbits := 32 - switch n := mathutil.BitLen(maxArg - minArg + 1); { - case n < 8: - tbits = 8 - case n < 16: - tbits = 16 - } - mustFormat(f, "%sParseTab = [%d][]uint%d{%i\n", *oPref, len(p.Table), tbits) - nCells := 0 - var tabRow sortutil.Uint64Slice - for si, state := range p.Table { - tabRow = tabRow[:0] - max := 0 - for _, act := range state { - sym := act.Sym - xsym, ok := xlat[sym.Value] - if !ok { - panic("internal error 001") - } - - max = mathutil.Max(max, xsym) - kind, arg := act.Kind() - switch kind { - case 'a': - arg = 0 - case 'r': - arg *= -1 - } - tabRow = append(tabRow, uint64(xsym)<<32|uint64(arg-minArg)) - } - nCells += max - tabRow.Sort() - col := -1 - if si%5 == 0 { - mustFormat(f, "// %d\n", si) - } - mustFormat(f, "{") - for i, v := range tabRow { - xsym := int(uint32(v >> 32)) - arg := int(uint32(v)) - if col+1 != xsym { - mustFormat(f, "%d: ", xsym) - } - switch { - case i == len(tabRow)-1: - mustFormat(f, "%d", arg) - default: - mustFormat(f, "%d, ", arg) - } - col = xsym - } - mustFormat(f, "},\n") - } - mustFormat(f, "%u}\n") - fmt.Fprintf(os.Stderr, "Parse table entries: %d of %d, x %d bits == %d bytes\n", nCells, len(p.Table)*len(msu), tbits, nCells*tbits/8) - if n := p.ConflictsSR; n != 0 { - fmt.Fprintf(os.Stderr, "conflicts: %d shift/reduce\n", n) - } - if n := p.ConflictsRR; n != 0 { - fmt.Fprintf(os.Stderr, "conflicts: %d reduce/reduce\n", n) - } - - mustFormat(f, `%u) - -var %[1]sDebug = 0 - -type %[1]sLexer interface { - Lex(lval *%[1]sSymType) int - Errorf(format string, a ...interface{}) - Errors() []error -} - -type %[1]sLexerEx interface { - %[1]sLexer - Reduced(rule, state int, lval *%[1]sSymType) bool -} - -func %[1]sSymName(c int) (s string) { - x, ok := %[1]sXLAT[c] - if ok { - return %[1]sSymNames[x] - } - - return __yyfmt__.Sprintf("%%d", c) -} - -func %[1]slex1(yylex %[1]sLexer, lval *%[1]sSymType) (n int) { - n = yylex.Lex(lval) - if n <= 0 { - n = %[1]sEOFCode - } - if %[1]sDebug >= 3 { - __yyfmt__.Printf("\nlex %%s(%%#x %%d), %[4]s: %[3]s\n", %[1]sSymName(n), n, n, %[4]s) - } - return n -} - -func %[1]sParse(yylex %[1]sLexer, parser *Parser) int { - const yyError = %[2]d - - yyEx, _ := yylex.(%[1]sLexerEx) - var yyn int - parser.yylval = %[1]sSymType{} - parser.yyVAL = %[1]sSymType{} - yyS := parser.cache - - Nerrs := 0 /* number of errors */ - Errflag := 0 /* error recovery flag */ - yyerrok := func() { - if %[1]sDebug >= 2 { - __yyfmt__.Printf("yyerrok()\n") - } - Errflag = 0 - } - _ = yyerrok - yystate := 0 - yychar := -1 - var yyxchar int - var yyshift int - yyp := -1 - goto yystack - -ret0: - return 0 - -ret1: - return 1 - -yystack: - /* put a state and value onto the stack */ - yyp++ - if yyp >= len(yyS) { - nyys := make([]%[1]sSymType, len(yyS)*2) - copy(nyys, yyS) - yyS = nyys - parser.cache = yyS - } - yyS[yyp] = parser.yyVAL - yyS[yyp].yys = yystate - -yynewstate: - if yychar < 0 { - yychar = %[1]slex1(yylex, &parser.yylval) - var ok bool - if yyxchar, ok = %[1]sXLAT[yychar]; !ok { - yyxchar = len(%[1]sSymNames) // > tab width - } - } - if %[1]sDebug >= 4 { - var a []int - for _, v := range yyS[:yyp+1] { - a = append(a, v.yys) - } - __yyfmt__.Printf("state stack %%v\n", a) - } - row := %[1]sParseTab[yystate] - yyn = 0 - if yyxchar < len(row) { - if yyn = int(row[yyxchar]); yyn != 0 { - yyn += %[1]sTabOfs - } - } - switch { - case yyn > 0: // shift - yychar = -1 - parser.yyVAL = parser.yylval - yystate = yyn - yyshift = yyn - if %[1]sDebug >= 2 { - __yyfmt__.Printf("shift, and goto state %%d\n", yystate) - } - if Errflag > 0 { - Errflag-- - } - goto yystack - case yyn < 0: // reduce - case yystate == 1: // accept - if %[1]sDebug >= 2 { - __yyfmt__.Println("accept") - } - goto ret0 - } - - if yyn == 0 { - /* error ... attempt to resume parsing */ - switch Errflag { - case 0: /* brand new error */ - if %[1]sDebug >= 1 { - __yyfmt__.Printf("no action for %%s in state %%d\n", %[1]sSymName(yychar), yystate) - } - msg, ok := %[1]sXErrors[%[1]sXError{yystate, yyxchar}] - if !ok { - msg, ok = %[1]sXErrors[%[1]sXError{yystate, -1}] - } - if !ok && yyshift != 0 { - msg, ok = %[1]sXErrors[%[1]sXError{yyshift, yyxchar}] - } - if !ok { - msg, ok = %[1]sXErrors[%[1]sXError{yyshift, -1}] - } - if !ok || msg == "" { - msg = "syntax error" - } - // ignore goyacc error message - yylex.Errorf("") - Nerrs++ - fallthrough - - case 1, 2: /* incompletely recovered error ... try again */ - Errflag = 3 - - /* find a state where "error" is a legal shift action */ - for yyp >= 0 { - row := %[1]sParseTab[yyS[yyp].yys] - if yyError < len(row) { - yyn = int(row[yyError])+%[1]sTabOfs - if yyn > 0 { // hit - if %[1]sDebug >= 2 { - __yyfmt__.Printf("error recovery found error shift in state %%d\n", yyS[yyp].yys) - } - yystate = yyn /* simulate a shift of "error" */ - goto yystack - } - } - - /* the current p has no shift on "error", pop stack */ - if %[1]sDebug >= 2 { - __yyfmt__.Printf("error recovery pops state %%d\n", yyS[yyp].yys) - } - yyp-- - } - /* there is no state on the stack with an error shift ... abort */ - if %[1]sDebug >= 2 { - __yyfmt__.Printf("error recovery failed\n") - } - goto ret1 - - case 3: /* no shift yet; clobber input char */ - if %[1]sDebug >= 2 { - __yyfmt__.Printf("error recovery discards %%s\n", %[1]sSymName(yychar)) - } - if yychar == %[1]sEOFCode { - goto ret1 - } - - yychar = -1 - goto yynewstate /* try again in the same state */ - } - } - - r := -yyn - x0 := %[1]sReductions[r] - x, n := x0.xsym, x0.components - yypt := yyp - _ = yypt // guard against "declared and not used" - - yyp -= n - if yyp+1 >= len(yyS) { - nyys := make([]%[1]sSymType, len(yyS)*2) - copy(nyys, yyS) - yyS = nyys - parser.cache = yyS - } - parser.yyVAL = yyS[yyp+1] - - /* consult goto table to find next state */ - exState := yystate - yystate = int(%[1]sParseTab[yyS[yyp].yys][x])+%[1]sTabOfs - /* reduction by production r */ - if %[1]sDebug >= 2 { - __yyfmt__.Printf("reduce using rule %%v (%%s), and goto state %%d\n", r, %[1]sSymNames[x], yystate) - } - - switch r {%i -`, - *oPref, errSym, *oDlvalf, *oDlval) - for r, rule := range p.Rules { - if rule.Action == nil { - continue - } - - action := rule.Action.Values - if len(action) == 0 { - continue - } - - if len(action) == 1 { - part := action[0] - if part.Type == parser.ActionValueGo { - src := part.Src - src = src[1 : len(src)-1] // Remove lead '{' and trail '}' - if strings.TrimSpace(src) == "" { - continue - } - } - } - - components := rule.Components - typ := rule.Sym.Type - max := len(components) - if p1 := rule.Parent; p1 != nil { - max = rule.MaxParentDlr - components = p1.Components - } - mustFormat(f, "case %d: ", r) - for _, part := range action { - num := part.Num - switch part.Type { - case parser.ActionValueGo: - mustFormat(f, "%s", part.Src) - case parser.ActionValueDlrDlr: - mustFormat(f, "parser.yyVAL.%s", typ) - if typ == "" { - panic("internal error 002") - } - case parser.ActionValueDlrNum: - typ := p.Syms[components[num-1]].Type - if typ == "" { - panic("internal error 003") - } - mustFormat(f, "yyS[yypt-%d].%s", max-num, typ) - case parser.ActionValueDlrTagDlr: - mustFormat(f, "parser.yyVAL.%s", part.Tag) - case parser.ActionValueDlrTagNum: - mustFormat(f, "yyS[yypt-%d].%s", max-num, part.Tag) - } - } - mustFormat(f, "\n") - } - mustFormat(f, `%u - } - - if yyEx != nil && yyEx.Reduced(r, exState, &parser.yyVAL) { - return -1 - } - goto yystack /* stack new state and value */ -} - -%[2]s -`, *oPref, p.Tail) - _ = oNoLines //TODO Ignored for now - return nil -} - -func injectImport(src string) string { - const inj = ` - -import __yyfmt__ "fmt" -` - fset := token.NewFileSet() - file := fset.AddFile("", -1, len(src)) - var s scanner.Scanner - s.Init( - file, - []byte(src), - nil, - scanner.ScanComments, - ) - for { - switch _, tok, _ := s.Scan(); tok { - case token.EOF: - return inj + src - case token.PACKAGE: - s.Scan() // ident - pos, _, _ := s.Scan() - ofs := file.Offset(pos) - return src[:ofs] + inj + src[ofs:] - } - } -} - -func mustFormat(f strutil.Formatter, format string, args ...interface{}) { - _, err := f.Format(format, args...) - if err != nil { - log.Fatalf("format error %v", err) - } -} diff --git a/perfschema/init.go b/perfschema/init.go index 270afe27f94fd..477d60e5aab48 100644 --- a/perfschema/init.go +++ b/perfschema/init.go @@ -14,12 +14,12 @@ package perfschema import ( + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" ) type columnInfo struct { diff --git a/perfschema/perfschema.go b/perfschema/perfschema.go index 2e5c1a8106694..8d017b9e37630 100644 --- a/perfschema/perfschema.go +++ b/perfschema/perfschema.go @@ -14,7 +14,7 @@ package perfschema import ( - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/table" ) diff --git a/perfschema/tables.go b/perfschema/tables.go index 8f89614152cf4..2d925ae2b1b2a 100644 --- a/perfschema/tables.go +++ b/perfschema/tables.go @@ -14,9 +14,9 @@ package perfschema import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/planner/core/cache.go b/planner/core/cache.go index b86eae67aecd7..b16a99992a6cf 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -17,7 +17,7 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" diff --git a/planner/core/cache_test.go b/planner/core/cache_test.go index 66af8cdd4c1f9..2d3ab7ea76010 100644 --- a/planner/core/cache_test.go +++ b/planner/core/cache_test.go @@ -16,7 +16,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testleak" ) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 26b8f9d6ef4eb..2569bf89567a9 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -14,8 +14,9 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/types/parser_driver" ) // Cacheable checks whether the input ast is cacheable. @@ -52,13 +53,13 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren } case *ast.Limit: if node.Count != nil { - if _, isParamMarker := node.Count.(*ast.ParamMarkerExpr); isParamMarker { + if _, isParamMarker := node.Count.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false return in, true } } if node.Offset != nil { - if _, isParamMarker := node.Offset.(*ast.ParamMarkerExpr); isParamMarker { + if _, isParamMarker := node.Offset.(*driver.ParamMarkerExpr); isParamMarker { checker.cacheable = false return in, true } diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index ebb4a7d7e44a9..9b6c1367e3042 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -15,9 +15,10 @@ package core import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/types/parser_driver" ) var _ = Suite(&testCacheableSuite{}) @@ -66,7 +67,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { c.Assert(Cacheable(stmt), IsFalse) limitStmt := &ast.Limit{ - Count: &ast.ParamMarkerExpr{}, + Count: &driver.ParamMarkerExpr{}, } stmt = &ast.SelectStmt{ Limit: limitStmt, @@ -74,7 +75,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { c.Assert(Cacheable(stmt), IsFalse) limitStmt = &ast.Limit{ - Offset: &ast.ParamMarkerExpr{}, + Offset: &driver.ParamMarkerExpr{}, } stmt = &ast.SelectStmt{ Limit: limitStmt, diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 7f750c67cb260..4a2386a502793 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -19,15 +19,16 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/auth" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/ranger" @@ -162,7 +163,7 @@ func (e *Execute) optimizePreparedPlan(ctx sessionctx.Context, is infoschema.Inf if err != nil { return errors.Trace(err) } - prepared.Params[i].SetDatum(val) + prepared.Params[i].(*driver.ParamMarkerExpr).Datum = val vars.PreparedParams = append(vars.PreparedParams, val) } if prepared.SchemaVersion != is.SchemaMetaVersion() { diff --git a/planner/core/errors.go b/planner/core/errors.go index 2b4ce819c6cfc..55f01919ddf1f 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -14,8 +14,8 @@ package core import ( - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" ) const ( diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 6a111bc5d63da..4ed1f03cefab7 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -16,13 +16,13 @@ package core import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/planner/property" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/pkg/errors" diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 4d5ceed793b33..104c9939396e7 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -17,16 +17,17 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" ) @@ -36,7 +37,7 @@ var EvalSubquery func(p PhysicalPlan, is infoschema.InfoSchema, ctx sessionctx.C // evalAstExpr evaluates ast expression directly. func evalAstExpr(ctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error) { - if val, ok := expr.(*ast.ValueExpr); ok { + if val, ok := expr.(*driver.ValueExpr); ok { return val.Datum, nil } b := &planBuilder{ @@ -748,10 +749,10 @@ func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok switch v := inNode.(type) { case *ast.AggregateFuncExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.WhenClause, *ast.SubqueryExpr, *ast.ExistsSubqueryExpr, *ast.CompareSubqueryExpr, *ast.ValuesExpr: - case *ast.ValueExpr: + case *driver.ValueExpr: value := &expression.Constant{Value: v.Datum, RetType: &v.Type} er.ctxStack = append(er.ctxStack, value) - case *ast.ParamMarkerExpr: + case *driver.ParamMarkerExpr: tp := types.NewFieldType(mysql.TypeUnspecified) types.DefaultParamTypeForValue(v.GetValue(), tp) value := &expression.Constant{Value: v.Datum, RetType: tp} @@ -815,7 +816,7 @@ func datumToConstant(d types.Datum, tp byte) *expression.Constant { return &expression.Constant{Value: d, RetType: types.NewFieldType(tp)} } -func (er *expressionRewriter) getParamExpression(v *ast.ParamMarkerExpr) expression.Expression { +func (er *expressionRewriter) getParamExpression(v *driver.ParamMarkerExpr) expression.Expression { f, err := expression.NewFunction(er.ctx, ast.GetParam, &v.Type, diff --git a/planner/core/expression_test.go b/planner/core/expression_test.go index 71584c1a65cf0..21bd952f0ed9d 100644 --- a/planner/core/expression_test.go +++ b/planner/core/expression_test.go @@ -17,12 +17,12 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 50c0ae748e4bd..7cc9dc118d71e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -16,10 +16,10 @@ package core import ( "math" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index cc2d08a28d575..2b2fe33ae864a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -22,21 +22,22 @@ import ( "unicode" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" ) @@ -518,7 +519,7 @@ func (b *planBuilder) buildProjectionFieldNameFromExpressions(field *ast.SelectF } innerExpr := getInnerFromParentheses(field.Expr) - valueExpr, isValueExpr := innerExpr.(*ast.ValueExpr) + valueExpr, isValueExpr := innerExpr.(*driver.ValueExpr) // Non-literal: Output as inputed, except that comments need to be removed. if !isValueExpr { @@ -835,13 +836,13 @@ func getUintForLimitOffset(sc *stmtctx.StatementContext, val interface{}) (uint6 func extractLimitCountOffset(sc *stmtctx.StatementContext, limit *ast.Limit) (count uint64, offset uint64, err error) { if limit.Count != nil { - count, err = getUintForLimitOffset(sc, limit.Count.GetValue()) + count, err = getUintForLimitOffset(sc, limit.Count.(ast.ValueExpr).GetValue()) if err != nil { return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT") } } if limit.Offset != nil { - offset, err = getUintForLimitOffset(sc, limit.Offset.GetValue()) + offset, err = getUintForLimitOffset(sc, limit.Offset.(ast.ValueExpr).GetValue()) if err != nil { return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT") } @@ -954,7 +955,7 @@ func (a *havingAndOrderbyExprResolver) Enter(n ast.Node) (node ast.Node, skipChi switch n.(type) { case *ast.AggregateFuncExpr: a.inAggFunc = true - case *ast.ParamMarkerExpr, *ast.ColumnNameExpr, *ast.ColumnName: + case *driver.ParamMarkerExpr, *ast.ColumnNameExpr, *ast.ColumnName: case *ast.SubqueryExpr, *ast.ExistsSubqueryExpr: // Enter a new context, skip it. // For example: select sum(c) + c + exists(select c from t) from t; @@ -1136,7 +1137,7 @@ func (g *gbyResolver) Enter(inNode ast.Node) (ast.Node, bool) { switch inNode.(type) { case *ast.SubqueryExpr, *ast.CompareSubqueryExpr, *ast.ExistsSubqueryExpr: return inNode, true - case *ast.ValueExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.ColumnName: + case *driver.ValueExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.ColumnName: default: g.inExpr = true } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 88a8aedb41057..5c2dc2e13004c 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -19,15 +19,15 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index a87ddede8aff0..201ce7bf124bf 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -16,11 +16,11 @@ package core import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index 03e2524da74c4..acda1c36ffbf1 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -17,10 +17,10 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 7b2292b2a021f..727c34a329750 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -16,7 +16,7 @@ package core import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 58e47867914c9..858a42f8346a9 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -15,10 +15,10 @@ package core_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 48ceea5f48001..4c66330fabdfe 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -14,10 +14,10 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" diff --git a/planner/core/plan.go b/planner/core/plan.go index 4ac105b0354b4..80831f63bdd81 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -17,7 +17,7 @@ import ( "fmt" "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 1494052fcd591..258fd5b30bc46 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -14,9 +14,9 @@ package core import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" diff --git a/planner/core/plan_to_pb_test.go b/planner/core/plan_to_pb_test.go index e507be3295f14..fd1ff48291a70 100644 --- a/planner/core/plan_to_pb_test.go +++ b/planner/core/plan_to_pb_test.go @@ -15,9 +15,9 @@ package core import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tipb/go-tipb" diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3601e4b712532..f49bd69471d40 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -18,16 +18,18 @@ import ( "strings" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/ranger" "github.com/pkg/errors" ) @@ -237,8 +239,8 @@ func (b *planBuilder) buildSet(v *ast.SetStmt) (Plan, error) { } if vars.ExtendValue != nil { assign.ExtendValue = &expression.Constant{ - Value: vars.ExtendValue.Datum, - RetType: &vars.ExtendValue.Type, + Value: vars.ExtendValue.(*driver.ValueExpr).Datum, + RetType: &vars.ExtendValue.(*driver.ValueExpr).Type, } } p.VarAssigns = append(p.VarAssigns, assign) @@ -375,7 +377,8 @@ func (b *planBuilder) buildPrepare(x *ast.PrepareStmt) Plan { Name: x.Name, } if x.SQLVar != nil { - p.SQLText, _ = x.SQLVar.GetValue().(string) + // TODO: Prepared statement from variable expression do not work as expected. + // p.SQLText, _ = x.SQLVar.GetValue().(string) } else { p.SQLText = x.SQLText } @@ -552,7 +555,7 @@ func (b *planBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, erro return nil, errors.Trace(err) } expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) - genColumnName := model.GetTableColumnID(tableInfo, column.ColumnInfo) + genColumnName := admin.GetTableColumnID(tableInfo, column.ColumnInfo) p.GenExprs[genColumnName] = expr } } @@ -1234,7 +1237,7 @@ func (b *planBuilder) buildValuesListOfInsert(insert *ast.InsertStmt, insertPlan } else { expr, err = b.getDefaultValue(affectedValuesCols[j]) } - case *ast.ValueExpr: + case *driver.ValueExpr: expr = &expression.Constant{ Value: x.Datum, RetType: &x.Type, diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 6d181b4558b57..037a6173f95f7 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -15,8 +15,8 @@ package core import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" ) var _ = Suite(&testPlanBuilderSuite{}) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 86f5968099ac1..74e976c9ea705 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -17,15 +17,16 @@ import ( "bytes" "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" ) @@ -335,9 +336,9 @@ func getNameValuePairs(nvPairs []nameValuePair, expr ast.ExprNode) []nameValuePa } var d types.Datum switch x := binOp.R.(type) { - case *ast.ValueExpr: + case *driver.ValueExpr: d = x.Datum - case *ast.ParamMarkerExpr: + case *driver.ParamMarkerExpr: d = x.Datum } if d.IsNull() { diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index d0a17036e1e7f..b4ca94946604f 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -17,15 +17,16 @@ import ( "math" "strings" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pkg/errors" ) @@ -95,7 +96,7 @@ func (p *preprocessor) Leave(in ast.Node) (out ast.Node, ok bool) { p.checkContainDotColumn(x) case *ast.DropTableStmt, *ast.AlterTableStmt, *ast.RenameTableStmt: p.inCreateOrDropTable = false - case *ast.ParamMarkerExpr: + case *driver.ParamMarkerExpr: if !p.inPrepare { p.err = parser.ErrSyntax.GenWithStack("syntax error, unexpected '?'") return @@ -134,14 +135,20 @@ func checkAutoIncrementOp(colDef *ast.ColumnDef, num int) (bool, error) { return hasAutoIncrement, nil } for _, op := range colDef.Options[num+1:] { - if op.Tp == ast.ColumnOptionDefaultValue && !op.Expr.GetDatum().IsNull() { - return hasAutoIncrement, errors.Errorf("Invalid default value for '%s'", colDef.Name.Name.O) + if op.Tp == ast.ColumnOptionDefaultValue { + if tmp, ok := op.Expr.(*driver.ValueExpr); ok { + if !tmp.Datum.IsNull() { + return hasAutoIncrement, errors.Errorf("Invalid default value for '%s'", colDef.Name.Name.O) + } + } } } } if colDef.Options[num].Tp == ast.ColumnOptionDefaultValue && len(colDef.Options) != num+1 { - if colDef.Options[num].Expr.GetDatum().IsNull() { - return hasAutoIncrement, nil + if tmp, ok := colDef.Options[num].Expr.(*driver.ValueExpr); ok { + if tmp.Datum.IsNull() { + return hasAutoIncrement, nil + } } for _, op := range colDef.Options[num+1:] { if op.Tp == ast.ColumnOptionAutoIncrement { diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 17deb8a09d48b..fb6b1ae659990 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -15,14 +15,14 @@ package core_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 818f0a28d1685..5c3fe26b5a99f 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -15,11 +15,11 @@ package core import ( "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" ) diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 161ed782d4aa8..92c2e67a99bf0 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -14,9 +14,9 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" ) type buildKeySolver struct{} diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index afa552019588e..d74132806d4e7 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -14,10 +14,10 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" log "github.com/sirupsen/logrus" ) diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 4cb9661e68116..13f6400e42e7b 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -16,10 +16,10 @@ package core import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" "github.com/pkg/errors" ) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 70e39cbaa0494..c6e71192a3940 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -16,7 +16,7 @@ package core import ( "sort" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" log "github.com/sirupsen/logrus" diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index 46bc666b371f8..c53cbf091475a 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -13,9 +13,9 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" ) diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index a22bc51417837..8335f33e99d0a 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -13,10 +13,10 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" ) diff --git a/planner/core/task.go b/planner/core/task.go index a7c0e322e4381..17f2568c465cb 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -17,13 +17,13 @@ import ( "fmt" "math" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" ) // task is a new version of `PhysicalPlanInfo`. It stores cost information for a task. diff --git a/planner/core/trace.go b/planner/core/trace.go index 765ab9d7f0a02..ae879cf885e86 100644 --- a/planner/core/trace.go +++ b/planner/core/trace.go @@ -1,7 +1,7 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" ) // Trace represents a trace plan. diff --git a/planner/core/util.go b/planner/core/util.go index 2fe957687fd78..308344beff531 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -14,7 +14,7 @@ package core import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" ) diff --git a/privilege/privilege.go b/privilege/privilege.go index 34c210086c671..b837940ceb86b 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -14,10 +14,10 @@ package privilege import ( - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" ) type keyType int diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 08974a2374ec9..278df39dfb60b 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -20,10 +20,10 @@ import ( "sync/atomic" "time" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" diff --git a/privilege/privileges/cache_test.go b/privilege/privileges/cache_test.go index 723373e80ffa3..a66664c566d29 100644 --- a/privilege/privileges/cache_test.go +++ b/privilege/privileges/cache_test.go @@ -15,9 +15,9 @@ package privileges_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 2230a44d7be52..b2e29e851c8ad 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -16,12 +16,12 @@ package privileges import ( "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" log "github.com/sirupsen/logrus" ) diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 359114f48570a..01334da8ba5ca 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -19,14 +19,14 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" "golang.org/x/net/context" diff --git a/server/column.go b/server/column.go index 27ca3c23d734e..850b02eb3097c 100644 --- a/server/column.go +++ b/server/column.go @@ -14,7 +14,7 @@ package server import ( - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" ) // ColumnInfo contains information of a column diff --git a/server/conn.go b/server/conn.go index d2b49011ae6b7..fd38018a354d6 100644 --- a/server/conn.go +++ b/server/conn.go @@ -49,15 +49,14 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/arena" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/memory" @@ -392,17 +391,16 @@ func (cc *clientConn) openSessionAndDoAuth(authData []byte) error { if err != nil { return errors.Trace(err) } - host := variable.DefHostname - if !cc.server.isUnixSocket() { - addr := cc.bufReadConn.RemoteAddr().String() + if !cc.server.skipAuth() { // Do Auth. - host, _, err = net.SplitHostPort(addr) - if err != nil { + addr := cc.bufReadConn.RemoteAddr().String() + host, _, err1 := net.SplitHostPort(addr) + if err1 != nil { return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, addr, "YES")) } - } - if !cc.ctx.Auth(&auth.UserIdentity{Username: cc.user, Hostname: host}, authData, cc.salt) { - return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, host, "YES")) + if !cc.ctx.Auth(&auth.UserIdentity{Username: cc.user, Hostname: host}, authData, cc.salt) { + return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, host, "YES")) + } } if cc.dbname != "" { err = cc.useDB(context.Background(), cc.dbname) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 2e62b7803548b..51b30bf7f6154 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -40,7 +40,7 @@ import ( "math" "strconv" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index d131e39d6f22c..8aae1b6e0b08f 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -15,8 +15,8 @@ package server import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/types" ) diff --git a/server/conn_test.go b/server/conn_test.go index 037399dd7dd4a..8e56bad74c9e3 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -19,7 +19,7 @@ import ( "encoding/binary" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" ) type ConnTestSuite struct{} diff --git a/server/driver.go b/server/driver.go index ebc2d7278ccc0..df8bb2b00e36e 100644 --- a/server/driver.go +++ b/server/driver.go @@ -17,9 +17,9 @@ import ( "crypto/tls" "fmt" + "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "golang.org/x/net/context" ) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 103556477765c..19d041833219e 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -17,16 +17,17 @@ import ( "crypto/tls" "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" ) @@ -331,7 +332,7 @@ func (tc *TiDBContext) GetSessionVars() *variable.SessionVars { } type tidbResultSet struct { - recordSet ast.RecordSet + recordSet sqlexec.RecordSet columns []*ColumnInfo rows []chunk.Row closed bool diff --git a/server/driver_tidb_test.go b/server/driver_tidb_test.go index 97e7b87b05040..b2dcab0a60b87 100644 --- a/server/driver_tidb_test.go +++ b/server/driver_tidb_test.go @@ -15,11 +15,11 @@ package server import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" ) type tidbResultSetTestSuite struct{} diff --git a/server/http_handler.go b/server/http_handler.go index 5fdebb28d1062..9802b9e84c839 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -30,12 +30,13 @@ import ( "github.com/gorilla/mux" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -45,7 +46,6 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 0dc9b279a7100..a43a3b5ddad7a 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -30,12 +30,12 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" diff --git a/server/http_status.go b/server/http_status.go index 1d9789b6dc423..7771e3e0cebcb 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -20,9 +20,9 @@ import ( "net/http/pprof" "github.com/gorilla/mux" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/printer" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" diff --git a/server/packetio.go b/server/packetio.go index ae16f5d143063..1e95b6a0e2e35 100644 --- a/server/packetio.go +++ b/server/packetio.go @@ -38,8 +38,8 @@ import ( "bufio" "io" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/server/server.go b/server/server.go index add925488b65b..51cc6930a6c9d 100644 --- a/server/server.go +++ b/server/server.go @@ -43,11 +43,11 @@ import ( "time" "github.com/blacktear23/go-proxyprotocol" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util" "github.com/pkg/errors" log "github.com/sirupsen/logrus" @@ -129,7 +129,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn { return cc } -func (s *Server) isUnixSocket() bool { +func (s *Server) skipAuth() bool { return s.cfg.Socket != "" } diff --git a/server/server_test.go b/server/server_test.go index ace52e763d7cb..7ff59ae67008e 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -27,8 +27,8 @@ import ( "github.com/go-sql-driver/mysql" . "github.com/pingcap/check" + tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - tmysql "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" log "github.com/sirupsen/logrus" @@ -161,9 +161,6 @@ func (dbt *DBTest) mustQueryRows(query string, args ...interface{}) { func runTestRegression(c *C, overrider configOverrider, dbName string) { runTestsOnNewDB(c, overrider, dbName, func(dbt *DBTest) { - // Show the user - dbt.mustExec("select user()") - // Create Table dbt.mustExec("CREATE TABLE test (val TINYINT)") diff --git a/server/statistics_handler.go b/server/statistics_handler.go index 2dd1db833ed7b..40761114073f1 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -17,8 +17,8 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" ) diff --git a/server/tidb_test.go b/server/tidb_test.go index af00149189542..a46e9eaf4ed50 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -28,11 +28,11 @@ import ( "github.com/go-sql-driver/mysql" . "github.com/pingcap/check" + tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - tmysql "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pkg/errors" diff --git a/server/util.go b/server/util.go index 463967b0aac19..f401b4b005504 100644 --- a/server/util.go +++ b/server/util.go @@ -42,7 +42,7 @@ import ( "strconv" "time" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" diff --git a/server/util_test.go b/server/util_test.go index 46116cab842b1..fe70a73bfe98c 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -15,9 +15,9 @@ package server import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" diff --git a/session/bench_test.go b/session/bench_test.go index f19dd711e257f..b68a1bdf76823 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -19,10 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/sqlexec" log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -83,7 +83,7 @@ func prepareJoinBenchData(se Session, colType string, valueFormat string, valueC mustExecute(se, "commit") } -func readResult(ctx context.Context, rs ast.RecordSet, count int) { +func readResult(ctx context.Context, rs sqlexec.RecordSet, count int) { chk := rs.NewChunk() for count > 0 { err := rs.Next(ctx, chk) diff --git a/session/bootstrap.go b/session/bootstrap.go index 62986e734e58f..424779630306a 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -25,12 +25,12 @@ import ( "strings" "time" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 49787fa5df0ab..451292d768f37 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -17,14 +17,14 @@ import ( "fmt" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/auth" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) @@ -55,7 +55,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { err := r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) - datums := ast.RowToDatums(chk.GetRow(0), r.Fields()) + datums := statistics.RowToDatums(chk.GetRow(0), r.Fields()) match(c, datums, []byte(`%`), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "anyhost"}, []byte(""), []byte("")), IsTrue) @@ -91,7 +91,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { chk = r.NewChunk() err = r.Next(ctx, chk) c.Assert(err, IsNil) - datums = ast.RowToDatums(chk.GetRow(0), r.Fields()) + datums = statistics.RowToDatums(chk.GetRow(0), r.Fields()) match(c, datums, 3) mustExecSQL(c, se, "drop table if exists t") se.Close() @@ -159,7 +159,7 @@ func (s *testBootstrapSuite) TestBootstrapWithError(c *C) { c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row := chk.GetRow(0) - datums := ast.RowToDatums(row, r.Fields()) + datums := statistics.RowToDatums(row, r.Fields()) match(c, datums, []byte(`%`), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") c.Assert(r.Close(), IsNil) diff --git a/session/session.go b/session/session.go index 79e5c4a67d949..01c0e9d9a2151 100644 --- a/session/session.go +++ b/session/session.go @@ -28,16 +28,19 @@ import ( "time" "github.com/ngaut/pools" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/owner" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" @@ -46,13 +49,11 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/auth" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" @@ -63,17 +64,17 @@ import ( // Session context type Session interface { sessionctx.Context - Status() uint16 // Flag of current status, such as autocommit. - LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. - AffectedRows() uint64 // Affected rows by latest executed stmt. - Execute(context.Context, string) ([]ast.RecordSet, error) // Execute a sql statement. - String() string // String is used to debug. + Status() uint16 // Flag of current status, such as autocommit. + LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. + AffectedRows() uint64 // Affected rows by latest executed stmt. + Execute(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a sql statement. + String() string // String is used to debug. CommitTxn(context.Context) error RollbackTxn(context.Context) error // PrepareStmt executes prepare statement in binary protocol. PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) // ExecutePreparedStmt executes a prepared statement. - ExecutePreparedStmt(ctx context.Context, stmtID uint32, param ...interface{}) (ast.RecordSet, error) + ExecutePreparedStmt(ctx context.Context, stmtID uint32, param ...interface{}) (sqlexec.RecordSet, error) DropPreparedStmt(stmtID uint32) error SetClientCapability(uint32) // Set client capability flags. SetConnectionID(uint64) @@ -95,7 +96,7 @@ var ( type stmtRecord struct { stmtID uint32 - st ast.Statement + st sqlexec.Statement stmtCtx *stmtctx.StatementContext params []interface{} } @@ -106,7 +107,7 @@ type StmtHistory struct { } // Add appends a stmt to history list. -func (h *StmtHistory) Add(stmtID uint32, st ast.Statement, stmtCtx *stmtctx.StatementContext, params ...interface{}) { +func (h *StmtHistory) Add(stmtID uint32, st sqlexec.Statement, stmtCtx *stmtctx.StatementContext, params ...interface{}) { s := &stmtRecord{ stmtID: stmtID, st: st, @@ -622,7 +623,7 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R } } -func drainRecordSet(ctx context.Context, se *session, rs ast.RecordSet) ([]chunk.Row, error) { +func drainRecordSet(ctx context.Context, se *session, rs sqlexec.RecordSet) ([]chunk.Row, error) { var rows []chunk.Row chk := rs.NewChunk() for { @@ -740,7 +741,7 @@ func (s *session) SetProcessInfo(sql string) { s.processInfo.Store(pi) } -func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode ast.StmtNode, stmt ast.Statement, recordSets []ast.RecordSet) ([]ast.RecordSet, error) { +func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode ast.StmtNode, stmt sqlexec.Statement, recordSets []sqlexec.RecordSet) ([]sqlexec.RecordSet, error) { s.SetValue(sessionctx.QueryString, stmt.OriginText()) if _, ok := stmtNode.(ast.DDLNode); ok { s.SetValue(sessionctx.LastExecuteDDL, true) @@ -765,7 +766,7 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode return recordSets, nil } -func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.RecordSet, err error) { +func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec.RecordSet, err error) { if recordSets, err = s.execute(ctx, sql); err != nil { err = errors.Trace(err) s.sessionVars.StmtCtx.AppendError(err) @@ -773,7 +774,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.Rec return } -func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.RecordSet, err error) { +func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec.RecordSet, err error) { s.PrepareTxnCtx(ctx) connID := s.sessionVars.ConnectionID err = s.loadCommonGlobalVariablesIfNeeded() @@ -913,7 +914,7 @@ func checkArgs(args ...interface{}) error { } // ExecutePreparedStmt executes a prepared statement. -func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args ...interface{}) (ast.RecordSet, error) { +func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args ...interface{}) (sqlexec.RecordSet, error) { err := checkArgs(args...) if err != nil { return nil, errors.Trace(err) @@ -1023,13 +1024,10 @@ func (s *session) GetSessionVars() *variable.SessionVars { func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []byte) bool { pm := privilege.GetPrivilegeManager(s) - // Check IP or localhost. + // Check IP. if pm.ConnectionVerification(user.Username, user.Hostname, authentication, salt) { s.sessionVars.User = user return true - } else if user.Hostname == variable.DefHostname { - log.Errorf("User connection verification failed %s", user) - return false } // Check Hostname. @@ -1049,7 +1047,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication []byte, salt []by func getHostByIP(ip string) []string { if ip == "127.0.0.1" { - return []string{variable.DefHostname} + return []string{"localhost"} } addrs, err := net.LookupAddr(ip) terror.Log(errors.Trace(err)) diff --git a/session/session_test.go b/session/session_test.go index 7be247bde8771..860c327dca0a8 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -20,13 +20,15 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" @@ -36,9 +38,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" diff --git a/session/tidb.go b/session/tidb.go index 9a8472cf83c9a..c32e4701afedb 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -23,17 +23,18 @@ import ( "sync" "time" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -132,16 +133,16 @@ func Parse(ctx sessionctx.Context, src string) ([]ast.StmtNode, error) { } // Compile is safe for concurrent use by multiple goroutines. -func Compile(ctx context.Context, sctx sessionctx.Context, stmtNode ast.StmtNode) (ast.Statement, error) { +func Compile(ctx context.Context, sctx sessionctx.Context, stmtNode ast.StmtNode) (sqlexec.Statement, error) { compiler := executor.Compiler{Ctx: sctx} stmt, err := compiler.Compile(ctx, stmtNode) return stmt, errors.Trace(err) } -// runStmt executes the ast.Statement and commit or rollback the current transaction. -func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast.RecordSet, error) { +// runStmt executes the sqlexec.Statement and commit or rollback the current transaction. +func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (sqlexec.RecordSet, error) { var err error - var rs ast.RecordSet + var rs sqlexec.RecordSet se := sctx.(*session) rs, err = s.Exec(ctx) // All the history should be added here. @@ -203,7 +204,7 @@ func GetHistory(ctx sessionctx.Context) *StmtHistory { } // GetRows4Test gets all the rows from a RecordSet, only used for test. -func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs ast.RecordSet) ([]chunk.Row, error) { +func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs sqlexec.RecordSet) ([]chunk.Row, error) { if rs == nil { return nil, nil } diff --git a/session/tidb_test.go b/session/tidb_test.go index 99eaa32723ef1..38b3f693f1e80 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -22,13 +22,13 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" "golang.org/x/net/context" @@ -178,7 +178,7 @@ func removeStore(c *C, dbPath string) { os.RemoveAll(dbPath) } -func exec(se Session, sql string, args ...interface{}) (ast.RecordSet, error) { +func exec(se Session, sql string, args ...interface{}) (sqlexec.RecordSet, error) { ctx := context.Background() if len(args) == 0 { rs, err := se.Execute(ctx, sql) @@ -198,7 +198,7 @@ func exec(se Session, sql string, args ...interface{}) (ast.RecordSet, error) { return rs, nil } -func mustExecSQL(c *C, se Session, sql string, args ...interface{}) ast.RecordSet { +func mustExecSQL(c *C, se Session, sql string, args ...interface{}) sqlexec.RecordSet { rs, err := exec(se, sql, args...) c.Assert(err, IsNil) return rs diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 235614cede25d..1a5c100695796 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -21,12 +21,12 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb-tools/tidb-binlog/node" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" binlog "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index a6f1e9cc1f9cf..5a8b93a724028 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -22,6 +22,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" @@ -30,7 +31,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index e59d2da7194cb..66db63081c5b8 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -18,7 +18,7 @@ import ( "sync" "time" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" ) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 385c7f1643418..198d8921b9739 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -21,16 +21,16 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/timeutil" diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cc7a383e06d0c..4e46cd2a84777 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -17,11 +17,11 @@ import ( "strconv" "strings" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/logutil" ) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 85d9b7f44bc6f..50a2cae5f5bfc 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -16,7 +16,7 @@ package variable import ( "os" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" ) /* @@ -204,7 +204,6 @@ const ( // Default TiDB system variable values. const ( - DefHostname = "localhost" DefIndexLookupConcurrency = 4 DefIndexLookupJoinConcurrency = 4 DefIndexSerialScanConcurrency = 1 diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 215e82a5b142d..8c4a5eaa08176 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -22,8 +22,8 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/timeutil" "github.com/pkg/errors" diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 0b9e71d661a3d..013b46c44e4b1 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -19,9 +19,9 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" ) diff --git a/statistics/bootstrap.go b/statistics/bootstrap.go index c4322a03acc56..66104e88fbe41 100644 --- a/statistics/bootstrap.go +++ b/statistics/bootstrap.go @@ -16,11 +16,11 @@ package statistics import ( "fmt" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" diff --git a/statistics/ddl.go b/statistics/ddl.go index acabf0fd71907..3abfc0b48896e 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -16,11 +16,10 @@ package statistics import ( "fmt" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl/util" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" @@ -97,7 +96,7 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) (er // If we didn't update anything by last SQL, it means the stats of this table does not exist. if h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 { // By this step we can get the count of this table, then we can sure the count and repeats of bucket. - var rs []ast.RecordSet + var rs []sqlexec.RecordSet rs, err = exec.Execute(ctx, fmt.Sprintf("select count from mysql.stats_meta where table_id = %d", tableID)) if len(rs) > 0 { defer terror.Call(rs[0].Close) diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index b5906e7224c18..8cda510bd55d1 100644 --- a/statistics/ddl_test.go +++ b/statistics/ddl_test.go @@ -15,7 +15,7 @@ package statistics_test import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" diff --git a/statistics/dump.go b/statistics/dump.go index 7e1fedd83b93d..505635dc2d2b6 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -16,9 +16,9 @@ package statistics import ( "time" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tipb/go-tipb" diff --git a/statistics/dump_test.go b/statistics/dump_test.go index 5564c688bdd82..c3824d38b88f3 100644 --- a/statistics/dump_test.go +++ b/statistics/dump_test.go @@ -17,9 +17,9 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" diff --git a/statistics/feedback.go b/statistics/feedback.go index 81eccc6787e88..29023d56b4d12 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -23,9 +23,9 @@ import ( "time" "github.com/cznic/mathutil" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index f5b83b44ee663..08058c386bef2 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -15,7 +15,7 @@ package statistics import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" ) diff --git a/statistics/handle.go b/statistics/handle.go index 3b0160343d569..d019cafa52083 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -19,9 +19,9 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" diff --git a/statistics/handle_test.go b/statistics/handle_test.go index eb46e0758f023..31b890bcfcc13 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -18,9 +18,9 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" diff --git a/statistics/histogram.go b/statistics/histogram.go index f0c304d4da365..3c00b0531c057 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -20,12 +20,12 @@ import ( "strings" "time" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" diff --git a/statistics/sample.go b/statistics/sample.go index 6b58fb5916c38..70eed697a812a 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -17,12 +17,13 @@ import ( "fmt" "math/rand" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" "golang.org/x/net/context" @@ -132,7 +133,7 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e // Also, if primary key is handle, it will directly build histogram for it. type SampleBuilder struct { Sc *stmtctx.StatementContext - RecordSet ast.RecordSet + RecordSet sqlexec.RecordSet ColLen int // ColLen is the number of columns need to be sampled. PkBuilder *SortedBuilder MaxBucketSize int64 @@ -175,7 +176,7 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, panic(fmt.Sprintf("%T", s.RecordSet)) } for row := it.Begin(); row != it.End(); row = it.Next() { - datums := ast.RowToDatums(row, s.RecordSet.Fields()) + datums := RowToDatums(row, s.RecordSet.Fields()) if s.PkBuilder != nil { err = s.PkBuilder.Iterate(datums[0]) if err != nil { @@ -192,3 +193,12 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, } } } + +// RowToDatums converts row to datum slice. +func RowToDatums(row chunk.Row, fields []*ast.ResultField) []types.Datum { + datums := make([]types.Datum, len(fields)) + for i, f := range fields { + datums[i] = row.GetDatum(i, &f.Column.FieldType) + } + return datums +} diff --git a/statistics/sample_test.go b/statistics/sample_test.go index dc3cc3e222303..dfc7b59df597b 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -17,18 +17,18 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/sqlexec" ) var _ = Suite(&testSampleSuite{}) type testSampleSuite struct { count int - rs ast.RecordSet + rs sqlexec.RecordSet } func (s *testSampleSuite) SetUpSuite(c *C) { diff --git a/statistics/scalar.go b/statistics/scalar.go index 1ea756a6e6d0c..446a78a7383f6 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -17,7 +17,7 @@ import ( "encoding/binary" "math" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" ) diff --git a/statistics/scalar_test.go b/statistics/scalar_test.go index 31da575d12c12..47015fab2825e 100644 --- a/statistics/scalar_test.go +++ b/statistics/scalar_test.go @@ -17,7 +17,7 @@ import ( "math" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" ) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index f7e87a4913308..e8764bba9c7d4 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -16,9 +16,9 @@ package statistics import ( "math" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/ranger" "github.com/pkg/errors" diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 128f87ec55c92..ed98c6520f8c2 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -22,10 +22,10 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index d719edad22df9..fb79e934c18be 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -19,9 +19,9 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" ) @@ -43,8 +44,8 @@ var _ = Suite(&testStatisticsSuite{}) type testStatisticsSuite struct { count int samples []types.Datum - rc ast.RecordSet - pk ast.RecordSet + rc sqlexec.RecordSet + pk sqlexec.RecordSet } type recordSet struct { @@ -169,7 +170,7 @@ func encodeKey(key types.Datum) types.Datum { return types.NewBytesDatum(buf) } -func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSet) (int64, *Histogram, error) { +func buildPK(sctx sessionctx.Context, numBuckets, id int64, records sqlexec.RecordSet) (int64, *Histogram, error) { b := NewSortedBuilder(sctx.GetSessionVars().StmtCtx, numBuckets, id, types.NewFieldType(mysql.TypeLonglong)) ctx := context.Background() for { @@ -183,7 +184,7 @@ func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSe } it := chunk.NewIterator4Chunk(chk) for row := it.Begin(); row != it.End(); row = it.Next() { - datums := ast.RowToDatums(row, records.Fields()) + datums := RowToDatums(row, records.Fields()) err = b.Iterate(datums[0]) if err != nil { return 0, nil, errors.Trace(err) @@ -193,7 +194,7 @@ func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSe return b.Count, b.hist, nil } -func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSet) (int64, *Histogram, *CMSketch, error) { +func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records sqlexec.RecordSet) (int64, *Histogram, *CMSketch, error) { b := NewSortedBuilder(sctx.GetSessionVars().StmtCtx, numBuckets, id, types.NewFieldType(mysql.TypeBlob)) cms := NewCMSketch(8, 2048) ctx := context.Background() @@ -208,7 +209,7 @@ func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records ast.Recor break } for row := it.Begin(); row != it.End(); row = it.Next() { - datums := ast.RowToDatums(row, records.Fields()) + datums := RowToDatums(row, records.Fields()) buf, err := codec.EncodeKey(sctx.GetSessionVars().StmtCtx, nil, datums...) if err != nil { return 0, nil, nil, errors.Trace(err) @@ -282,7 +283,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { checkRepeats(c, col) c.Assert(col.Len(), Equals, 250) - tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, ast.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, sqlexec.RecordSet(s.rc)) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -299,7 +300,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(int(count), Equals, 0) s.pk.(*recordSet).cursor = 0 - tblCount, col, err = buildPK(ctx, bucketCount, 4, ast.RecordSet(s.pk)) + tblCount, col, err = buildPK(ctx, bucketCount, 4, sqlexec.RecordSet(s.pk)) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -338,7 +339,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { func (s *testStatisticsSuite) TestHistogramProtoConversion(c *C) { ctx := mock.NewContext() s.rc.Close() - tblCount, col, _, err := buildIndex(ctx, 256, 1, ast.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, 256, 1, sqlexec.RecordSet(s.rc)) c.Check(err, IsNil) c.Check(int(tblCount), Equals, 100000) diff --git a/statistics/table.go b/statistics/table.go index 65c84a06499c6..275ad7f37f374 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -19,9 +19,9 @@ import ( "strings" "sync" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" diff --git a/statistics/update.go b/statistics/update.go index dc0bebd5a48d3..6e30bc3e2eb9f 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -21,9 +21,9 @@ import ( "sync" "time" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/chunk" diff --git a/statistics/update_test.go b/statistics/update_test.go index 710be1cb946c4..8a176f581fec2 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -19,10 +19,10 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index bbda6c3020b00..2659a4cde994e 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -16,13 +16,13 @@ package mocktikv import ( "github.com/golang/protobuf/proto" "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tipb/go-tipb" @@ -188,7 +188,7 @@ func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe return &coprocessor.Response{Data: data}, nil } -// Fields implements the ast.RecordSet Fields interface. +// Fields implements the sqlexec.RecordSet Fields interface. func (e *analyzeColumnsExec) Fields() []*ast.ResultField { return e.fields } @@ -232,7 +232,7 @@ func (e *analyzeColumnsExec) NewChunk() *chunk.Chunk { return chunk.NewChunkWithCapacity(fields, 1) } -// Close implements the ast.RecordSet Close interface. +// Close implements the sqlexec.RecordSet Close interface. func (e *analyzeColumnsExec) Close() error { return nil } diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 0d874793e121b..913f839c6d9e0 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -23,14 +23,14 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/tikvpb" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" mockpkg "github.com/pingcap/tidb/util/mock" diff --git a/store/mockstore/mocktikv/executor.go b/store/mockstore/mocktikv/executor.go index b3d78396fdf3e..29f478b8e277f 100644 --- a/store/mockstore/mocktikv/executor.go +++ b/store/mockstore/mocktikv/executor.go @@ -19,10 +19,10 @@ import ( "sort" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/mockstore/mocktikv/mvcc_leveldb.go index 34abd116a3449..53cdb92c5a5d9 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/mockstore/mocktikv/mvcc_leveldb.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/goleveldb/leveldb/storage" "github.com/pingcap/goleveldb/leveldb/util" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 8a51bce389ccb..380974cbed0fe 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" "golang.org/x/net/context" ) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 345ae5de2996c..2c1655e2add88 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -21,12 +21,12 @@ import ( "time" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" binlog "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/store/tikv/2pc_fail_test.go b/store/tikv/2pc_fail_test.go index 5fe4b2b0d8b4e..edb015a7d86f6 100644 --- a/store/tikv/2pc_fail_test.go +++ b/store/tikv/2pc_fail_test.go @@ -16,8 +16,8 @@ package tikv import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" "golang.org/x/net/context" ) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 3fe5ae9b954c6..b52aeb552a044 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -20,10 +20,10 @@ import ( "strings" "time" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/store/tikv/client.go b/store/tikv/client.go index b1e95abc3b1e6..1633a01cde8ee 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -26,10 +26,10 @@ import ( "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/tikvpb" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" diff --git a/store/tikv/error.go b/store/tikv/error.go index 9c78022c5fcae..9329de2eaf221 100644 --- a/store/tikv/error.go +++ b/store/tikv/error.go @@ -14,8 +14,8 @@ package tikv import ( - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index bf15962e867e3..c0bd0557175da 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/parser/terror" "github.com/pingcap/pd/client" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" @@ -35,7 +36,6 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/terror" tidbutil "github.com/pingcap/tidb/util" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/store/tikv/isolation_test.go b/store/tikv/isolation_test.go index eeb5aa80e185d..e95028269a135 100644 --- a/store/tikv/isolation_test.go +++ b/store/tikv/isolation_test.go @@ -22,8 +22,8 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/terror" "golang.org/x/net/context" ) diff --git a/store/tikv/safepoint_test.go b/store/tikv/safepoint_test.go index 76e3449d9a95f..975f099bfaece 100644 --- a/store/tikv/safepoint_test.go +++ b/store/tikv/safepoint_test.go @@ -18,8 +18,8 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" "golang.org/x/net/context" ) diff --git a/store/tikv/sql_fail_test.go b/store/tikv/sql_fail_test.go index 4fb976c557198..30bbc95c25ff7 100644 --- a/store/tikv/sql_fail_test.go +++ b/store/tikv/sql_fail_test.go @@ -20,10 +20,10 @@ import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" . "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "golang.org/x/net/context" diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go index 2f759076cb6ab..a19a7d941075f 100644 --- a/store/tikv/tikv_test.go +++ b/store/tikv/tikv_test.go @@ -21,7 +21,7 @@ import ( "strings" . "github.com/pingcap/check" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" ) // OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. diff --git a/structure/structure.go b/structure/structure.go index 81853deeeafed..4f7eb0e50ec7d 100644 --- a/structure/structure.go +++ b/structure/structure.go @@ -14,8 +14,8 @@ package structure import ( + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/terror" ) // structure error codes. diff --git a/table/column.go b/table/column.go index 8e6bb6925a13d..d2b57b66e1727 100644 --- a/table/column.go +++ b/table/column.go @@ -21,15 +21,15 @@ import ( "strings" "unicode/utf8" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/table/column_test.go b/table/column_test.go index 4bf3bc104c0c3..3d34a1436892f 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -17,11 +17,11 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" ) diff --git a/table/index.go b/table/index.go index 8957025549dcd..1b7d247c6ed13 100644 --- a/table/index.go +++ b/table/index.go @@ -14,8 +14,8 @@ package table import ( + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" diff --git a/table/table.go b/table/table.go index ac99df34f397e..021bb6740ad4d 100644 --- a/table/table.go +++ b/table/table.go @@ -18,12 +18,12 @@ package table import ( + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" ) diff --git a/table/tables/gen_expr.go b/table/tables/gen_expr.go index fa94ca009c74d..f0f88e2924cc7 100644 --- a/table/tables/gen_expr.go +++ b/table/tables/gen_expr.go @@ -16,9 +16,9 @@ package tables import ( "fmt" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser" + "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pkg/errors" ) diff --git a/table/tables/gen_expr_test.go b/table/tables/gen_expr_test.go index 89de193f5cd42..9723e32e63841 100644 --- a/table/tables/gen_expr_test.go +++ b/table/tables/gen_expr_test.go @@ -15,7 +15,7 @@ package tables import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" ) var _ = Suite(&testGenExprSuite{}) diff --git a/table/tables/index.go b/table/tables/index.go index f4580d0e93978..f34330a83de5c 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -19,14 +19,14 @@ import ( "io" "unicode/utf8" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" ) diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 5e8350e8dd46c..1abe161696f6f 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -18,14 +18,14 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" diff --git a/table/tables/partition.go b/table/tables/partition.go index 762c4e7976e7f..efcaf596d7b62 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -19,9 +19,9 @@ import ( "sort" "strings" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" diff --git a/table/tables/tables.go b/table/tables/tables.go index ff411bc98e14f..483a9701cdea5 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -24,10 +24,10 @@ import ( "strings" "time" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 0446a2d5c00d4..54e9919f6409a 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -17,9 +17,9 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index ec07f38b0f05e..3f31f5613a9c3 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -19,10 +19,10 @@ import ( "math" "time" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 0c048e26e69c7..96605ee6abd01 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -21,8 +21,8 @@ import ( gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" "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/codec" diff --git a/terror/terror.go b/terror/terror.go index 3cc1c2d17105c..c38e0e18a1b24 100644 --- a/terror/terror.go +++ b/terror/terror.go @@ -18,7 +18,7 @@ import ( "fmt" "strconv" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/tidb-server/main.go b/tidb-server/main.go index f65ae2e7fa085..41058eedd8381 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -23,6 +23,8 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/pd/client" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" @@ -30,7 +32,6 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/server" @@ -41,7 +42,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/gcworker" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/signal" diff --git a/types/compare_test.go b/types/compare_test.go index 8bdd621b61ff3..cf51757fc637c 100644 --- a/types/compare_test.go +++ b/types/compare_test.go @@ -17,7 +17,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/testleak" ) diff --git a/types/convert.go b/types/convert.go index 08ec314b927bf..1ee25fe72f23d 100644 --- a/types/convert.go +++ b/types/convert.go @@ -22,9 +22,9 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" diff --git a/types/convert_test.go b/types/convert_test.go index c9103e92c17cb..96e48387db6fd 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -20,11 +20,11 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" ) @@ -464,7 +464,7 @@ func (s *testTypeConvertSuite) TestStrToNum(c *C) { func (s *testTypeConvertSuite) TestFieldTypeToStr(c *C) { defer testleak.AfterTest(c)() v := TypeToStr(mysql.TypeUnspecified, "not binary") - c.Assert(v, Equals, type2Str[mysql.TypeUnspecified]) + c.Assert(v, Equals, TypeStr(mysql.TypeUnspecified)) v = TypeToStr(mysql.TypeBlob, charset.CharsetBin) c.Assert(v, Equals, "blob") v = TypeToStr(mysql.TypeString, charset.CharsetBin) diff --git a/types/datum.go b/types/datum.go index bf0f03878deff..3665537800a4b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -22,11 +22,11 @@ import ( "time" "unicode/utf8" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" log "github.com/sirupsen/logrus" diff --git a/types/datum_eval.go b/types/datum_eval.go index af8a7f993402b..9cbdaa4fa84fa 100644 --- a/types/datum_eval.go +++ b/types/datum_eval.go @@ -15,7 +15,7 @@ package types import ( "github.com/cznic/mathutil" - "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/parser/opcode" "github.com/pkg/errors" ) diff --git a/types/datum_test.go b/types/datum_test.go index fc925ee7ecc5b..1d2eb562f5b73 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -19,7 +19,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types/json" ) diff --git a/types/errors.go b/types/errors.go index 52558ea5f18d6..9e1919b45d97e 100644 --- a/types/errors.go +++ b/types/errors.go @@ -14,8 +14,9 @@ package types import ( - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + parser_types "github.com/pingcap/parser/types" ) var ( @@ -45,12 +46,12 @@ var ( ErrWrongFieldSpec = terror.ClassTypes.New(codeWrongFieldSpec, "Wrong Field Spec") // ErrBadNumber is return when parsing an invalid binary decimal number. ErrBadNumber = terror.ClassTypes.New(codeBadNumber, "Bad Number") + // ErrInvalidDefault is returned when meet a invalid default value. + ErrInvalidDefault = parser_types.ErrInvalidDefault // ErrCastAsSignedOverflow is returned when positive out-of-range integer, and convert to it's negative complement. ErrCastAsSignedOverflow = terror.ClassTypes.New(codeUnknown, msgCastAsSignedOverflow) // ErrCastNegIntAsUnsigned is returned when a negative integer be casted to an unsigned int. ErrCastNegIntAsUnsigned = terror.ClassTypes.New(codeUnknown, msgCastNegIntAsUnsigned) - // ErrInvalidDefault is returned when meet a invalid default value. - ErrInvalidDefault = terror.ClassTypes.New(codeInvalidDefault, "Invalid default value for '%s'") // ErrMBiggerThanD is returned when precision less than the scale. ErrMBiggerThanD = terror.ClassTypes.New(codeMBiggerThanD, mysql.MySQLErrName[mysql.ErrMBiggerThanD]) // ErrWarnDataOutOfRange is returned when the value in a numeric column that is outside the permissible range of the column data type. diff --git a/types/etc.go b/types/etc.go index 4f1bf1115d74e..626139a7d328a 100644 --- a/types/etc.go +++ b/types/etc.go @@ -19,30 +19,21 @@ package types import ( "io" - "strings" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" - "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/charset" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/opcode" + "github.com/pingcap/parser/terror" + ast "github.com/pingcap/parser/types" "github.com/pkg/errors" ) // IsTypeBlob returns a boolean indicating whether the tp is a blob type. -func IsTypeBlob(tp byte) bool { - switch tp { - case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob: - return true - default: - return false - } -} +var IsTypeBlob = ast.IsTypeBlob // IsTypeChar returns a boolean indicating // whether the tp is the char type like a string type or a varchar type. -func IsTypeChar(tp byte) bool { - return tp == mysql.TypeString || tp == mysql.TypeVarchar -} +var IsTypeChar = ast.IsTypeChar // IsTypeVarchar returns a boolean indicating // whether the tp is the varchar type like a varstring type or a varchar type. @@ -113,36 +104,6 @@ func IsString(tp byte) bool { return IsTypeChar(tp) || IsTypeBlob(tp) || IsTypeVarchar(tp) || IsTypeUnspecified(tp) } -var type2Str = map[byte]string{ - mysql.TypeBit: "bit", - mysql.TypeBlob: "text", - mysql.TypeDate: "date", - mysql.TypeDatetime: "datetime", - mysql.TypeDecimal: "unspecified", - mysql.TypeNewDecimal: "decimal", - mysql.TypeDouble: "double", - mysql.TypeEnum: "enum", - mysql.TypeFloat: "float", - mysql.TypeGeometry: "geometry", - mysql.TypeInt24: "mediumint", - mysql.TypeJSON: "json", - mysql.TypeLong: "int", - mysql.TypeLonglong: "bigint", - mysql.TypeLongBlob: "longtext", - mysql.TypeMediumBlob: "mediumtext", - mysql.TypeNull: "null", - mysql.TypeSet: "set", - mysql.TypeShort: "smallint", - mysql.TypeString: "char", - mysql.TypeDuration: "time", - mysql.TypeTimestamp: "timestamp", - mysql.TypeTiny: "tinyint", - mysql.TypeTinyBlob: "tinytext", - mysql.TypeVarchar: "varchar", - mysql.TypeVarString: "var_string", - mysql.TypeYear: "year", -} - var kind2Str = map[byte]string{ KindNull: "null", KindInt64: "bigint", @@ -166,9 +127,7 @@ var kind2Str = map[byte]string{ } // TypeStr converts tp to a string. -func TypeStr(tp byte) (r string) { - return type2Str[tp] -} +var TypeStr = ast.TypeStr // KindStr converts kind to a string. func KindStr(kind byte) (r string) { @@ -181,18 +140,7 @@ func KindStr(kind byte) (r string) { // Args: // tp: type enum // cs: charset -func TypeToStr(tp byte, cs string) (r string) { - ts := type2Str[tp] - if cs != charset.CharsetBin { - return ts - } - if IsTypeBlob(tp) { - ts = strings.Replace(ts, "text", "blob", 1) - } else if IsTypeChar(tp) { - ts = strings.Replace(ts, "char", "binary", 1) - } - return ts -} +var TypeToStr = ast.TypeToStr // EOFAsNil filtrates errors, // If err is equal to io.EOF returns nil. diff --git a/types/etc_test.go b/types/etc_test.go index 60d272bff2866..32de5aa88ee20 100644 --- a/types/etc_test.go +++ b/types/etc_test.go @@ -18,8 +18,8 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/testleak" ) diff --git a/types/eval_type.go b/types/eval_type.go index 47775953d97c5..3eb17cae856f9 100644 --- a/types/eval_type.go +++ b/types/eval_type.go @@ -13,30 +13,26 @@ package types +import ast "github.com/pingcap/parser/types" + // EvalType indicates the specified types that arguments and result of a built-in function should be. -type EvalType byte +type EvalType = ast.EvalType const ( // ETInt represents type INT in evaluation. - ETInt EvalType = iota + ETInt = ast.ETInt // ETReal represents type REAL in evaluation. - ETReal + ETReal = ast.ETReal // ETDecimal represents type DECIMAL in evaluation. - ETDecimal + ETDecimal = ast.ETDecimal // ETString represents type STRING in evaluation. - ETString + ETString = ast.ETString // ETDatetime represents type DATETIME in evaluation. - ETDatetime + ETDatetime = ast.ETDatetime // ETTimestamp represents type TIMESTAMP in evaluation. - ETTimestamp + ETTimestamp = ast.ETTimestamp // ETDuration represents type DURATION in evaluation. - ETDuration + ETDuration = ast.ETDuration // ETJson represents type JSON in evaluation. - ETJson + ETJson = ast.ETJson ) - -// IsStringKind returns true for ETString, ETDatetime, ETTimestamp, ETDuration, ETJson EvalTypes. -func (et EvalType) IsStringKind() bool { - return et == ETString || et == ETDatetime || - et == ETTimestamp || et == ETDuration || et == ETJson -} diff --git a/types/field_type.go b/types/field_type.go index bb247ba8e7e20..9da5c8044a0fd 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -14,15 +14,12 @@ package types import ( - "fmt" - "io" "strconv" - "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" + ast "github.com/pingcap/parser/types" "github.com/pingcap/tidb/types/json" - "github.com/pingcap/tidb/util/charset" - "github.com/pingcap/tidb/util/format" ) // UnspecifiedLength is unspecified length. @@ -31,16 +28,7 @@ const ( ) // FieldType records field type information. -type FieldType struct { - Tp byte - Flag uint - Flen int - Decimal int - Charset string - Collate string - // Elems is the element list for enum and set type. - Elems []string -} +type FieldType = ast.FieldType // NewFieldType returns a FieldType, // with a type and other information about field type. @@ -52,27 +40,6 @@ func NewFieldType(tp byte) *FieldType { } } -// Equal checks whether two FieldType objects are equal. -func (ft *FieldType) Equal(other *FieldType) bool { - // We do not need to compare whole `ft.Flag == other.Flag` when wrapping cast upon an Expression. - // but need compare unsigned_flag of ft.Flag. - partialEqual := ft.Tp == other.Tp && - ft.Flen == other.Flen && - ft.Decimal == other.Decimal && - ft.Charset == other.Charset && - ft.Collate == other.Collate && - mysql.HasUnsignedFlag(ft.Flag) == mysql.HasUnsignedFlag(other.Flag) - if !partialEqual || len(ft.Elems) != len(other.Elems) { - return false - } - for i := range ft.Elems { - if ft.Elems[i] != other.Elems[i] { - return false - } - } - return true -} - // AggFieldType aggregates field types for a multi-argument function like `IF`, `IFNULL`, `COALESCE` // whose return type is determined by the arguments' FieldTypes. // Aggregation is performed by MergeFieldType function. @@ -152,172 +119,6 @@ func setTypeFlag(flag *uint, flagItem uint, on bool) { } } -// EvalType gets the type in evaluation. -func (ft *FieldType) EvalType() EvalType { - switch ft.Tp { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, - mysql.TypeBit, mysql.TypeYear: - return ETInt - case mysql.TypeFloat, mysql.TypeDouble: - return ETReal - case mysql.TypeNewDecimal: - return ETDecimal - case mysql.TypeDate, mysql.TypeDatetime: - return ETDatetime - case mysql.TypeTimestamp: - return ETTimestamp - case mysql.TypeDuration: - return ETDuration - case mysql.TypeJSON: - return ETJson - } - return ETString -} - -// Hybrid checks whether a type is a hybrid type, which can represent different types of value in specific context. -func (ft *FieldType) Hybrid() bool { - return ft.Tp == mysql.TypeEnum || ft.Tp == mysql.TypeBit || ft.Tp == mysql.TypeSet -} - -// Init initializes the FieldType data. -func (ft *FieldType) Init(tp byte) { - ft.Tp = tp - ft.Flen = UnspecifiedLength - ft.Decimal = UnspecifiedLength -} - -// CompactStr only considers Tp/CharsetBin/Flen/Deimal. -// This is used for showing column type in infoschema. -func (ft *FieldType) CompactStr() string { - ts := TypeToStr(ft.Tp, ft.Charset) - suffix := "" - - defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.Tp) - isDecimalNotDefault := ft.Decimal != defaultDecimal && ft.Decimal != 0 && ft.Decimal != UnspecifiedLength - - // displayFlen and displayDecimal are flen and decimal values with `-1` substituted with default value. - displayFlen, displayDecimal := ft.Flen, ft.Decimal - if displayFlen == 0 || displayFlen == UnspecifiedLength { - displayFlen = defaultFlen - } - if displayDecimal == 0 || displayDecimal == UnspecifiedLength { - displayDecimal = defaultDecimal - } - - switch ft.Tp { - case mysql.TypeEnum, mysql.TypeSet: - // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') - es := make([]string, 0, len(ft.Elems)) - for _, e := range ft.Elems { - e = format.OutputFormat(e) - es = append(es, e) - } - suffix = fmt.Sprintf("('%s')", strings.Join(es, "','")) - case mysql.TypeTimestamp, mysql.TypeDatetime, mysql.TypeDuration: - if isDecimalNotDefault { - suffix = fmt.Sprintf("(%d)", displayDecimal) - } - case mysql.TypeDouble, mysql.TypeFloat: - // 1. Flen Not Default, Decimal Not Default -> Valid - // 2. Flen Not Default, Decimal Default (-1) -> Invalid - // 3. Flen Default, Decimal Not Default -> Valid - // 4. Flen Default, Decimal Default -> Valid (hide) - if isDecimalNotDefault { - suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) - } - case mysql.TypeNewDecimal: - suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) - case mysql.TypeBit, mysql.TypeShort, mysql.TypeTiny, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: - // Flen is always shown. - suffix = fmt.Sprintf("(%d)", displayFlen) - } - return ts + suffix -} - -// InfoSchemaStr joins the CompactStr with unsigned flag and -// returns a string. -func (ft *FieldType) InfoSchemaStr() string { - suffix := "" - if mysql.HasUnsignedFlag(ft.Flag) { - suffix = " unsigned" - } - return ft.CompactStr() + suffix -} - -// String joins the information of FieldType and returns a string. -// Note: when flen or decimal is unspecified, this function will use the default value instead of -1. -func (ft *FieldType) String() string { - strs := []string{ft.CompactStr()} - if mysql.HasUnsignedFlag(ft.Flag) { - strs = append(strs, "UNSIGNED") - } - if mysql.HasZerofillFlag(ft.Flag) { - strs = append(strs, "ZEROFILL") - } - if mysql.HasBinaryFlag(ft.Flag) && ft.Tp != mysql.TypeString { - strs = append(strs, "BINARY") - } - - if IsTypeChar(ft.Tp) || IsTypeBlob(ft.Tp) { - if ft.Charset != "" && ft.Charset != charset.CharsetBin { - strs = append(strs, fmt.Sprintf("CHARACTER SET %s", ft.Charset)) - } - if ft.Collate != "" && ft.Collate != charset.CharsetBin { - strs = append(strs, fmt.Sprintf("COLLATE %s", ft.Collate)) - } - } - - return strings.Join(strs, " ") -} - -// FormatAsCastType is used for write AST back to string. -func (ft *FieldType) FormatAsCastType(w io.Writer) { - switch ft.Tp { - case mysql.TypeVarString: - if ft.Charset == charset.CharsetBin && ft.Collate == charset.CollationBin { - fmt.Fprint(w, "BINARY") - } else { - fmt.Fprint(w, "CHAR") - } - if ft.Flen != UnspecifiedLength { - fmt.Fprintf(w, "(%d)", ft.Flen) - } - if ft.Flag&mysql.BinaryFlag != 0 { - fmt.Fprint(w, " BINARY") - } - if ft.Charset != charset.CharsetBin && ft.Charset != charset.CharsetUTF8 { - fmt.Fprintf(w, " %s", ft.Charset) - } - case mysql.TypeDate: - fmt.Fprint(w, "DATE") - case mysql.TypeDatetime: - fmt.Fprint(w, "DATETIME") - if ft.Decimal > 0 { - fmt.Fprintf(w, "(%d)", ft.Decimal) - } - case mysql.TypeNewDecimal: - fmt.Fprint(w, "DECIMAL") - if ft.Flen > 0 && ft.Decimal > 0 { - fmt.Fprintf(w, "(%d, %d)", ft.Flen, ft.Decimal) - } else if ft.Flen > 0 { - fmt.Fprintf(w, "(%d)", ft.Flen) - } - case mysql.TypeDuration: - fmt.Fprint(w, "TIME") - if ft.Decimal > 0 { - fmt.Fprintf(w, "(%d)", ft.Decimal) - } - case mysql.TypeLonglong: - if ft.Flag&mysql.UnsignedFlag != 0 { - fmt.Fprint(w, "UNSIGNED") - } else { - fmt.Fprint(w, "SIGNED") - } - case mysql.TypeJSON: - fmt.Fprint(w, "JSON") - } -} - // DefaultParamTypeForValue returns the default FieldType for the parameterized value. func DefaultParamTypeForValue(value interface{}, tp *FieldType) { switch value.(type) { @@ -1416,21 +1217,4 @@ func SetBinChsClnFlag(ft *FieldType) { } // VarStorageLen indicates this column is a variable length column. -const VarStorageLen = -1 - -// StorageLength is the length of stored value for the type. -func (ft *FieldType) StorageLength() int { - switch ft.Tp { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, - mysql.TypeLonglong, mysql.TypeDouble, mysql.TypeFloat, mysql.TypeYear, mysql.TypeDuration, - mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeEnum, mysql.TypeSet, - mysql.TypeBit: - // This may not be the accurate length, because we may encode them as varint. - return 8 - case mysql.TypeNewDecimal: - precision, frac := ft.Flen-ft.Decimal, ft.Decimal - return precision/digitsPerWord*wordSize + dig2bytes[precision%digitsPerWord] + frac/digitsPerWord*wordSize + dig2bytes[frac%digitsPerWord] - default: - return VarStorageLen - } -} +const VarStorageLen = ast.VarStorageLen diff --git a/types/field_type_test.go b/types/field_type_test.go index 71a6ea2983f55..f1d0292270223 100644 --- a/types/field_type_test.go +++ b/types/field_type_test.go @@ -15,8 +15,8 @@ package types import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/util/charset" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/util/testleak" ) diff --git a/types/format_test.go b/types/format_test.go index fb9b199c1f61b..16fbf6a963971 100644 --- a/types/format_test.go +++ b/types/format_test.go @@ -15,7 +15,7 @@ package types_test import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" ) diff --git a/types/json/binary.go b/types/json/binary.go index 0d0cd022d0dbb..a4b61511d1c2e 100644 --- a/types/json/binary.go +++ b/types/json/binary.go @@ -25,7 +25,7 @@ import ( "strings" "unicode/utf8" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/hack" "github.com/pkg/errors" ) diff --git a/types/json/constants.go b/types/json/constants.go index f5bc92daf336c..03c9a5aa7a5a2 100644 --- a/types/json/constants.go +++ b/types/json/constants.go @@ -17,8 +17,8 @@ import ( "encoding/binary" "unicode/utf8" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" ) // TypeCode indicates JSON type. diff --git a/types/mydecimal.go b/types/mydecimal.go index 97d037b80455b..dec19cfee333d 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -17,8 +17,8 @@ import ( "math" "strconv" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/types/parser_driver/value_expr.go b/types/parser_driver/value_expr.go new file mode 100644 index 0000000000000..09a8a8a1fb430 --- /dev/null +++ b/types/parser_driver/value_expr.go @@ -0,0 +1,177 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package driver + +import ( + "fmt" + "io" + "strconv" + + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/hack" +) + +// The purpose of driver package is to decompose the dependency of the parser and +// types package. +// It provides the NewValueExpr function for the ast package, so the ast package +// do not depends on the concrete definition of `types.Datum`, thus get rid of +// the dependency of the types package. +// The parser package depends on the ast package, but not the types package. +// The whole relationship: +// ast imports [] +// tidb/types imports [parser/types] +// parser imports [ast, parser/types] +// driver imports [ast, tidb/types] +// tidb imports [parser, driver] + +func init() { + ast.NewValueExpr = newValueExpr + ast.NewParamMarkerExpr = newParamMarkerExpr + ast.NewDecimal = func(str string) (interface{}, error) { + dec := new(types.MyDecimal) + err := dec.FromString(hack.Slice(str)) + return dec, err + } + ast.NewHexLiteral = func(str string) (interface{}, error) { + h, err := types.NewHexLiteral(str) + return h, err + } + ast.NewBitLiteral = func(str string) (interface{}, error) { + b, err := types.NewBitLiteral(str) + return b, err + } +} + +var ( + _ ast.ParamMarkerExpr = &ParamMarkerExpr{} + _ ast.ValueExpr = &ValueExpr{} +) + +// ValueExpr is the simple value expression. +type ValueExpr struct { + ast.TexprNode + types.Datum + projectionOffset int +} + +// GetDatumString implements the ast.ValueExpr interface. +func (n *ValueExpr) GetDatumString() string { + return n.GetString() +} + +// Format the ExprNode into a Writer. +func (n *ValueExpr) Format(w io.Writer) { + var s string + switch n.Kind() { + case types.KindNull: + s = "NULL" + case types.KindInt64: + if n.Type.Flag&mysql.IsBooleanFlag != 0 { + if n.GetInt64() > 0 { + s = "TRUE" + } else { + s = "FALSE" + } + } else { + s = strconv.FormatInt(n.GetInt64(), 10) + } + case types.KindUint64: + s = strconv.FormatUint(n.GetUint64(), 10) + case types.KindFloat32: + s = strconv.FormatFloat(n.GetFloat64(), 'e', -1, 32) + case types.KindFloat64: + s = strconv.FormatFloat(n.GetFloat64(), 'e', -1, 64) + case types.KindString, types.KindBytes: + s = strconv.Quote(n.GetString()) + case types.KindMysqlDecimal: + s = n.GetMysqlDecimal().String() + case types.KindBinaryLiteral: + if n.Type.Flag&mysql.UnsignedFlag != 0 { + s = fmt.Sprintf("x'%x'", n.GetBytes()) + } else { + s = n.GetBinaryLiteral().ToBitLiteralString(true) + } + default: + panic("Can't format to string") + } + fmt.Fprint(w, s) +} + +// newValueExpr creates a ValueExpr with value, and sets default field type. +func newValueExpr(value interface{}) ast.ValueExpr { + if ve, ok := value.(*ValueExpr); ok { + return ve + } + ve := &ValueExpr{} + ve.SetValue(value) + types.DefaultTypeForValue(value, &ve.Type) + ve.projectionOffset = -1 + return ve +} + +// SetProjectionOffset sets ValueExpr.projectionOffset for logical plan builder. +func (n *ValueExpr) SetProjectionOffset(offset int) { + n.projectionOffset = offset +} + +// GetProjectionOffset returns ValueExpr.projectionOffset. +func (n *ValueExpr) GetProjectionOffset() int { + return n.projectionOffset +} + +// Accept implements Node interface. +func (n *ValueExpr) Accept(v ast.Visitor) (ast.Node, bool) { + newNode, skipChildren := v.Enter(n) + if skipChildren { + return v.Leave(newNode) + } + n = newNode.(*ValueExpr) + return v.Leave(n) +} + +// ParamMarkerExpr expression holds a place for another expression. +// Used in parsing prepare statement. +type ParamMarkerExpr struct { + ValueExpr + Offset int + Order int +} + +func newParamMarkerExpr(offset int) ast.ParamMarkerExpr { + return &ParamMarkerExpr{ + Offset: offset, + } +} + +// Format the ExprNode into a Writer. +func (n *ParamMarkerExpr) Format(w io.Writer) { + panic("Not implemented") +} + +// Accept implements Node Accept interface. +func (n *ParamMarkerExpr) Accept(v ast.Visitor) (ast.Node, bool) { + newNode, skipChildren := v.Enter(n) + if skipChildren { + return v.Leave(newNode) + } + n = newNode.(*ParamMarkerExpr) + return v.Leave(n) +} + +// SetOrder implements the ast.ParamMarkerExpr interface. +func (n *ParamMarkerExpr) SetOrder(order int) { + n.Order = order +} diff --git a/types/time.go b/types/time.go index b5a027657f198..5133835985411 100644 --- a/types/time.go +++ b/types/time.go @@ -23,9 +23,9 @@ import ( gotime "time" "unicode" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pkg/errors" log "github.com/sirupsen/logrus" ) diff --git a/types/time_test.go b/types/time_test.go index d0e0cb25eac76..6a5736bd27f13 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -18,7 +18,7 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" diff --git a/util/admin/admin.go b/util/admin/admin.go index a137c41ab410f..4d7a110de7471 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -18,17 +18,17 @@ import ( "io" "sort" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -730,7 +730,7 @@ func fillGenColData(sessCtx sessionctx.Context, rowMap map[int64]types.Datum, t if !col.IsGenerated() || col.GeneratedStored == true { continue } - genColumnName := model.GetTableColumnID(tableInfo, col.ColumnInfo) + genColumnName := GetTableColumnID(tableInfo, col.ColumnInfo) if expr, ok := genExprs[genColumnName]; ok { var val types.Datum val, err = expr.Eval(chunk.MutRowFromDatums(row).ToRow()) @@ -747,6 +747,11 @@ func fillGenColData(sessCtx sessionctx.Context, rowMap map[int64]types.Datum, t return nil } +// GetTableColumnID gets a ID of a column with table ID // TableColumnID is composed by table ID and column ID. +func GetTableColumnID(tableInfo *model.TableInfo, col *model.ColumnInfo) string { + return fmt.Sprintf("%d_%d", tableInfo.ID, col.ID) +} + // admin error codes. const ( codeDataNotEqual terror.ErrCode = 1 diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 1e88fe08a67bc..d678f7d4543fc 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -19,10 +19,10 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" diff --git a/util/auth/auth.go b/util/auth/auth.go index 0c60b7d67d892..d3873768cc00f 100644 --- a/util/auth/auth.go +++ b/util/auth/auth.go @@ -19,7 +19,7 @@ import ( "encoding/hex" "fmt" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/terror" "github.com/pkg/errors" ) diff --git a/util/charset/charset.go b/util/charset/charset.go index 6967b9537a65c..c065ada4aed72 100644 --- a/util/charset/charset.go +++ b/util/charset/charset.go @@ -16,7 +16,7 @@ package charset import ( "strings" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pkg/errors" ) diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 799a6e703ca64..af61a97ae5bd3 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -23,7 +23,7 @@ import ( "unsafe" "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" diff --git a/util/chunk/codec.go b/util/chunk/codec.go index 20401f95f762b..43a5f5b17327d 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -19,7 +19,7 @@ import ( "unsafe" "github.com/cznic/mathutil" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" ) diff --git a/util/chunk/codec_test.go b/util/chunk/codec_test.go index 99aa4027647d1..abf5351784cc8 100644 --- a/util/chunk/codec_test.go +++ b/util/chunk/codec_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" diff --git a/util/chunk/compare.go b/util/chunk/compare.go index 60dc70ced817b..a8adbfc4174a9 100644 --- a/util/chunk/compare.go +++ b/util/chunk/compare.go @@ -16,7 +16,7 @@ package chunk import ( "sort" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" ) diff --git a/util/chunk/iterator_test.go b/util/chunk/iterator_test.go index eb4f9d7c04cf7..5438f062a1383 100644 --- a/util/chunk/iterator_test.go +++ b/util/chunk/iterator_test.go @@ -15,7 +15,7 @@ package chunk import ( "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" ) diff --git a/util/chunk/list_test.go b/util/chunk/list_test.go index 646812331ceb8..18982727ba40a 100644 --- a/util/chunk/list_test.go +++ b/util/chunk/list_test.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" ) diff --git a/util/chunk/mutrow.go b/util/chunk/mutrow.go index 1eba29a2f9e5c..17a3c4f4dd31c 100644 --- a/util/chunk/mutrow.go +++ b/util/chunk/mutrow.go @@ -18,7 +18,7 @@ import ( "math" "unsafe" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" diff --git a/util/chunk/mutrow_test.go b/util/chunk/mutrow_test.go index bf2e925c7fb41..1405773f027f4 100644 --- a/util/chunk/mutrow_test.go +++ b/util/chunk/mutrow_test.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" diff --git a/util/chunk/row.go b/util/chunk/row.go index dcf573eea66b5..df2bd96df91f6 100644 --- a/util/chunk/row.go +++ b/util/chunk/row.go @@ -17,7 +17,7 @@ import ( "time" "unsafe" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" diff --git a/util/codec/bench_test.go b/util/codec/bench_test.go index 4929458dcf994..0dd8d2d05b5d3 100644 --- a/util/codec/bench_test.go +++ b/util/codec/bench_test.go @@ -16,7 +16,7 @@ package codec import ( "testing" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) diff --git a/util/codec/codec.go b/util/codec/codec.go index 01d330ae8f68e..80839b5ca5340 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -17,9 +17,9 @@ import ( "encoding/binary" "time" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index 07b38e79c5cbe..56a1489addd04 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -20,9 +20,9 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" diff --git a/util/filesort/filesort.go b/util/filesort/filesort.go index 6701ca825324e..5b61cc904b7c2 100644 --- a/util/filesort/filesort.go +++ b/util/filesort/filesort.go @@ -25,8 +25,8 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" diff --git a/util/kvencoder/kv_encoder.go b/util/kvencoder/kv_encoder.go index 9fb4167172619..9184e0d439aec 100644 --- a/util/kvencoder/kv_encoder.go +++ b/util/kvencoder/kv_encoder.go @@ -20,11 +20,11 @@ import ( "sync" "sync/atomic" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/tablecodec" diff --git a/util/memory/action.go b/util/memory/action.go index 1873d3c2cca24..7e8ddaf49ded1 100644 --- a/util/memory/action.go +++ b/util/memory/action.go @@ -16,8 +16,8 @@ package memory import ( "sync" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/terror" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" log "github.com/sirupsen/logrus" ) diff --git a/util/mock/context.go b/util/mock/context.go index 0ddd2d02f4493..1461b4141cc66 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -19,7 +19,6 @@ import ( "sync" "time" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" @@ -58,7 +57,7 @@ func (txn *wrapTxn) Valid() bool { } // Execute implements sqlexec.SQLExecutor Execute interface. -func (c *Context) Execute(ctx context.Context, sql string) ([]ast.RecordSet, error) { +func (c *Context) Execute(ctx context.Context, sql string) ([]sqlexec.RecordSet, error) { return nil, errors.Errorf("Not Support.") } diff --git a/util/printer/printer.go b/util/printer/printer.go index 510c87c2592da..fdae8b19f32bc 100644 --- a/util/printer/printer.go +++ b/util/printer/printer.go @@ -18,8 +18,8 @@ import ( "encoding/json" "fmt" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/util/israce" log "github.com/sirupsen/logrus" ) diff --git a/util/ranger/checker.go b/util/ranger/checker.go index 6973d6bec8304..74ea582ca94de 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -14,9 +14,9 @@ package ranger import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" ) diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 31be5566d0843..529f4cb92c9a8 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -14,9 +14,9 @@ package ranger import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pkg/errors" diff --git a/util/ranger/points.go b/util/ranger/points.go index 4f31943d88f53..0b830545e578f 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -18,11 +18,11 @@ import ( "math" "sort" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pkg/errors" diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 2a7c1ca87d7b4..5631c1ff7c995 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -19,14 +19,14 @@ import ( "sort" "unicode/utf8" - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/codec" "github.com/pkg/errors" ) diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 56ec9261417d3..8947aec481cc5 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -18,10 +18,10 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index e95574ea8c354..672845de616bc 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -14,7 +14,7 @@ package sqlexec import ( - "github.com/pingcap/tidb/ast" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" "golang.org/x/net/context" @@ -42,7 +42,7 @@ type RestrictedSQLExecutor interface { // For example, privilege/privileges package need execute SQL, if it use // session.Session.Execute, then privilege/privileges and tidb would become a circle. type SQLExecutor interface { - Execute(ctx context.Context, sql string) ([]ast.RecordSet, error) + Execute(ctx context.Context, sql string) ([]RecordSet, error) } // SQLParser is an interface provides parsing sql statement. @@ -52,3 +52,41 @@ type SQLExecutor interface { type SQLParser interface { ParseSQL(sql, charset, collation string) ([]ast.StmtNode, error) } + +// Statement is an interface for SQL execution. +// NOTE: all Statement implementations must be safe for +// concurrent using by multiple goroutines. +// If the Exec method requires any Execution domain local data, +// they must be held out of the implementing instance. +type Statement interface { + // OriginText gets the origin SQL text. + OriginText() string + + // Exec executes SQL and gets a Recordset. + Exec(ctx context.Context) (RecordSet, error) + + // IsPrepared returns whether this statement is prepared statement. + IsPrepared() bool + + // IsReadOnly returns if the statement is read only. For example: SelectStmt without lock. + IsReadOnly() bool + + // RebuildPlan rebuilds the plan of the statement. + RebuildPlan() (schemaVersion int64, err error) +} + +// RecordSet is an abstract result set interface to help get data from Plan. +type RecordSet interface { + // Fields gets result fields. + Fields() []*ast.ResultField + + // Next reads records into chunk. + Next(ctx context.Context, chk *chunk.Chunk) error + + // NewChunk creates a new chunk with initial capacity. + NewChunk() *chunk.Chunk + + // Close closes the underlying iterator, call Next after Close will + // restart the iteration. + Close() error +} diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 08cc9a2d87355..cea25aab3ef75 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -20,9 +20,9 @@ import ( "sync/atomic" "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testutil" "github.com/pkg/errors" "golang.org/x/net/context" @@ -123,7 +123,7 @@ func NewTestKitWithInit(c *check.C, store kv.Storage) *TestKit { var connectionID uint64 // Exec executes a sql statement. -func (tk *TestKit) Exec(sql string, args ...interface{}) (ast.RecordSet, error) { +func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, error) { var err error if tk.Se == nil { tk.Se, err = session.CreateSession4Test(tk.store) @@ -133,7 +133,7 @@ func (tk *TestKit) Exec(sql string, args ...interface{}) (ast.RecordSet, error) } ctx := context.Background() if len(args) == 0 { - var rss []ast.RecordSet + var rss []sqlexec.RecordSet rss, err = tk.Se.Execute(ctx, sql) if err == nil && len(rss) > 0 { return rss[0], nil @@ -180,9 +180,9 @@ func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { return tk.ResultSetToResult(rs, comment) } -// ResultSetToResult converts ast.RecordSet to testkit.Result. +// ResultSetToResult converts sqlexec.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. -func (tk *TestKit) ResultSetToResult(rs ast.RecordSet, comment check.CommentInterface) *Result { +func (tk *TestKit) ResultSetToResult(rs sqlexec.RecordSet, comment check.CommentInterface) *Result { rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) err = rs.Close() diff --git a/vendor/github.com/BurntSushi/toml/.gitignore b/vendor/github.com/BurntSushi/toml/.gitignore new file mode 100644 index 0000000000000..0cd3800377d4d --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/.gitignore @@ -0,0 +1,5 @@ +TAGS +tags +.*.swp +tomlcheck/tomlcheck +toml.test diff --git a/vendor/github.com/BurntSushi/toml/.travis.yml b/vendor/github.com/BurntSushi/toml/.travis.yml new file mode 100644 index 0000000000000..8b8afc4f0e00d --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/.travis.yml @@ -0,0 +1,15 @@ +language: go +go: + - 1.1 + - 1.2 + - 1.3 + - 1.4 + - 1.5 + - 1.6 + - tip +install: + - go install ./... + - go get github.com/BurntSushi/toml-test +script: + - export PATH="$PATH:$HOME/gopath/bin" + - make test diff --git a/vendor/github.com/BurntSushi/toml/COMPATIBLE b/vendor/github.com/BurntSushi/toml/COMPATIBLE new file mode 100644 index 0000000000000..6efcfd0ce55ef --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/COMPATIBLE @@ -0,0 +1,3 @@ +Compatible with TOML version +[v0.4.0](https://github.com/toml-lang/toml/blob/v0.4.0/versions/en/toml-v0.4.0.md) + diff --git a/vendor/github.com/BurntSushi/toml/Makefile b/vendor/github.com/BurntSushi/toml/Makefile new file mode 100644 index 0000000000000..3600848d331ab --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/Makefile @@ -0,0 +1,19 @@ +install: + go install ./... + +test: install + go test -v + toml-test toml-test-decoder + toml-test -encoder toml-test-encoder + +fmt: + gofmt -w *.go */*.go + colcheck *.go */*.go + +tags: + find ./ -name '*.go' -print0 | xargs -0 gotags > TAGS + +push: + git push origin master + git push github master + diff --git a/vendor/github.com/BurntSushi/toml/README.md b/vendor/github.com/BurntSushi/toml/README.md new file mode 100644 index 0000000000000..7c1b37ecc7a02 --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/README.md @@ -0,0 +1,218 @@ +## TOML parser and encoder for Go with reflection + +TOML stands for Tom's Obvious, Minimal Language. This Go package provides a +reflection interface similar to Go's standard library `json` and `xml` +packages. This package also supports the `encoding.TextUnmarshaler` and +`encoding.TextMarshaler` interfaces so that you can define custom data +representations. (There is an example of this below.) + +Spec: https://github.com/toml-lang/toml + +Compatible with TOML version +[v0.4.0](https://github.com/toml-lang/toml/blob/master/versions/en/toml-v0.4.0.md) + +Documentation: https://godoc.org/github.com/BurntSushi/toml + +Installation: + +```bash +go get github.com/BurntSushi/toml +``` + +Try the toml validator: + +```bash +go get github.com/BurntSushi/toml/cmd/tomlv +tomlv some-toml-file.toml +``` + +[![Build Status](https://travis-ci.org/BurntSushi/toml.svg?branch=master)](https://travis-ci.org/BurntSushi/toml) [![GoDoc](https://godoc.org/github.com/BurntSushi/toml?status.svg)](https://godoc.org/github.com/BurntSushi/toml) + +### Testing + +This package passes all tests in +[toml-test](https://github.com/BurntSushi/toml-test) for both the decoder +and the encoder. + +### Examples + +This package works similarly to how the Go standard library handles `XML` +and `JSON`. Namely, data is loaded into Go values via reflection. + +For the simplest example, consider some TOML file as just a list of keys +and values: + +```toml +Age = 25 +Cats = [ "Cauchy", "Plato" ] +Pi = 3.14 +Perfection = [ 6, 28, 496, 8128 ] +DOB = 1987-07-05T05:45:00Z +``` + +Which could be defined in Go as: + +```go +type Config struct { + Age int + Cats []string + Pi float64 + Perfection []int + DOB time.Time // requires `import time` +} +``` + +And then decoded with: + +```go +var conf Config +if _, err := toml.Decode(tomlData, &conf); err != nil { + // handle error +} +``` + +You can also use struct tags if your struct field name doesn't map to a TOML +key value directly: + +```toml +some_key_NAME = "wat" +``` + +```go +type TOML struct { + ObscureKey string `toml:"some_key_NAME"` +} +``` + +### Using the `encoding.TextUnmarshaler` interface + +Here's an example that automatically parses duration strings into +`time.Duration` values: + +```toml +[[song]] +name = "Thunder Road" +duration = "4m49s" + +[[song]] +name = "Stairway to Heaven" +duration = "8m03s" +``` + +Which can be decoded with: + +```go +type song struct { + Name string + Duration duration +} +type songs struct { + Song []song +} +var favorites songs +if _, err := toml.Decode(blob, &favorites); err != nil { + log.Fatal(err) +} + +for _, s := range favorites.Song { + fmt.Printf("%s (%s)\n", s.Name, s.Duration) +} +``` + +And you'll also need a `duration` type that satisfies the +`encoding.TextUnmarshaler` interface: + +```go +type duration struct { + time.Duration +} + +func (d *duration) UnmarshalText(text []byte) error { + var err error + d.Duration, err = time.ParseDuration(string(text)) + return err +} +``` + +### More complex usage + +Here's an example of how to load the example from the official spec page: + +```toml +# This is a TOML document. Boom. + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T07:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] +``` + +And the corresponding Go types are: + +```go +type tomlConfig struct { + Title string + Owner ownerInfo + DB database `toml:"database"` + Servers map[string]server + Clients clients +} + +type ownerInfo struct { + Name string + Org string `toml:"organization"` + Bio string + DOB time.Time +} + +type database struct { + Server string + Ports []int + ConnMax int `toml:"connection_max"` + Enabled bool +} + +type server struct { + IP string + DC string +} + +type clients struct { + Data [][]interface{} + Hosts []string +} +``` + +Note that a case insensitive match will be tried if an exact match can't be +found. + +A working example of the above can be found in `_examples/example.{go,toml}`. diff --git a/vendor/github.com/BurntSushi/toml/cmd/toml-test-decoder/COPYING b/vendor/github.com/BurntSushi/toml/cmd/toml-test-decoder/COPYING deleted file mode 100644 index 5a8e332545f66..0000000000000 --- a/vendor/github.com/BurntSushi/toml/cmd/toml-test-decoder/COPYING +++ /dev/null @@ -1,14 +0,0 @@ - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - Version 2, December 2004 - - Copyright (C) 2004 Sam Hocevar - - Everyone is permitted to copy and distribute verbatim or modified - copies of this license document, and changing it is allowed as long - as the name is changed. - - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. You just DO WHAT THE FUCK YOU WANT TO. - diff --git a/vendor/github.com/BurntSushi/toml/cmd/toml-test-encoder/COPYING b/vendor/github.com/BurntSushi/toml/cmd/toml-test-encoder/COPYING deleted file mode 100644 index 5a8e332545f66..0000000000000 --- a/vendor/github.com/BurntSushi/toml/cmd/toml-test-encoder/COPYING +++ /dev/null @@ -1,14 +0,0 @@ - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - Version 2, December 2004 - - Copyright (C) 2004 Sam Hocevar - - Everyone is permitted to copy and distribute verbatim or modified - copies of this license document, and changing it is allowed as long - as the name is changed. - - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. You just DO WHAT THE FUCK YOU WANT TO. - diff --git a/vendor/github.com/BurntSushi/toml/cmd/tomlv/COPYING b/vendor/github.com/BurntSushi/toml/cmd/tomlv/COPYING deleted file mode 100644 index 5a8e332545f66..0000000000000 --- a/vendor/github.com/BurntSushi/toml/cmd/tomlv/COPYING +++ /dev/null @@ -1,14 +0,0 @@ - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - Version 2, December 2004 - - Copyright (C) 2004 Sam Hocevar - - Everyone is permitted to copy and distribute verbatim or modified - copies of this license document, and changing it is allowed as long - as the name is changed. - - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. You just DO WHAT THE FUCK YOU WANT TO. - diff --git a/vendor/github.com/BurntSushi/toml/decode_test.go b/vendor/github.com/BurntSushi/toml/decode_test.go new file mode 100644 index 0000000000000..95bc987f0c74d --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/decode_test.go @@ -0,0 +1,1461 @@ +package toml + +import ( + "fmt" + "log" + "math" + "reflect" + "strings" + "testing" + "time" +) + +func TestDecodeSimple(t *testing.T) { + var testSimple = ` +age = 250 +andrew = "gallant" +kait = "brady" +now = 1987-07-05T05:45:00Z +nowEast = 2017-06-22T16:15:21+08:00 +nowWest = 2017-06-22T02:14:36-06:00 +yesOrNo = true +pi = 3.14 +colors = [ + ["red", "green", "blue"], + ["cyan", "magenta", "yellow", "black"], +] + +[My.Cats] +plato = "cat 1" +cauchy = "cat 2" +` + + type cats struct { + Plato string + Cauchy string + } + type simple struct { + Age int + Colors [][]string + Pi float64 + YesOrNo bool + Now time.Time + NowEast time.Time + NowWest time.Time + Andrew string + Kait string + My map[string]cats + } + + var val simple + _, err := Decode(testSimple, &val) + if err != nil { + t.Fatal(err) + } + + now, err := time.Parse("2006-01-02T15:04:05", "1987-07-05T05:45:00") + if err != nil { + panic(err) + } + nowEast, err := time.Parse("2006-01-02T15:04:05-07:00", "2017-06-22T16:15:21+08:00") + if err != nil { + panic(err) + } + nowWest, err := time.Parse("2006-01-02T15:04:05-07:00", "2017-06-22T02:14:36-06:00") + if err != nil { + panic(err) + } + var answer = simple{ + Age: 250, + Andrew: "gallant", + Kait: "brady", + Now: now, + NowEast: nowEast, + NowWest: nowWest, + YesOrNo: true, + Pi: 3.14, + Colors: [][]string{ + {"red", "green", "blue"}, + {"cyan", "magenta", "yellow", "black"}, + }, + My: map[string]cats{ + "Cats": {Plato: "cat 1", Cauchy: "cat 2"}, + }, + } + if !reflect.DeepEqual(val, answer) { + t.Fatalf("Expected\n-----\n%#v\n-----\nbut got\n-----\n%#v\n", + answer, val) + } +} + +func TestDecodeEmbedded(t *testing.T) { + type Dog struct{ Name string } + type Age int + type cat struct{ Name string } + + for _, test := range []struct { + label string + input string + decodeInto interface{} + wantDecoded interface{} + }{ + { + label: "embedded struct", + input: `Name = "milton"`, + decodeInto: &struct{ Dog }{}, + wantDecoded: &struct{ Dog }{Dog{"milton"}}, + }, + { + label: "embedded non-nil pointer to struct", + input: `Name = "milton"`, + decodeInto: &struct{ *Dog }{}, + wantDecoded: &struct{ *Dog }{&Dog{"milton"}}, + }, + { + label: "embedded nil pointer to struct", + input: ``, + decodeInto: &struct{ *Dog }{}, + wantDecoded: &struct{ *Dog }{nil}, + }, + { + label: "unexported embedded struct", + input: `Name = "socks"`, + decodeInto: &struct{ cat }{}, + wantDecoded: &struct{ cat }{cat{"socks"}}, + }, + { + label: "embedded int", + input: `Age = -5`, + decodeInto: &struct{ Age }{}, + wantDecoded: &struct{ Age }{-5}, + }, + } { + _, err := Decode(test.input, test.decodeInto) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(test.wantDecoded, test.decodeInto) { + t.Errorf("%s: want decoded == %+v, got %+v", + test.label, test.wantDecoded, test.decodeInto) + } + } +} + +func TestDecodeIgnoredFields(t *testing.T) { + type simple struct { + Number int `toml:"-"` + } + const input = ` +Number = 123 +- = 234 +` + var s simple + if _, err := Decode(input, &s); err != nil { + t.Fatal(err) + } + if s.Number != 0 { + t.Errorf("got: %d; want 0", s.Number) + } +} + +func TestTableArrays(t *testing.T) { + var tomlTableArrays = ` +[[albums]] +name = "Born to Run" + + [[albums.songs]] + name = "Jungleland" + + [[albums.songs]] + name = "Meeting Across the River" + +[[albums]] +name = "Born in the USA" + + [[albums.songs]] + name = "Glory Days" + + [[albums.songs]] + name = "Dancing in the Dark" +` + + type Song struct { + Name string + } + + type Album struct { + Name string + Songs []Song + } + + type Music struct { + Albums []Album + } + + expected := Music{[]Album{ + {"Born to Run", []Song{{"Jungleland"}, {"Meeting Across the River"}}}, + {"Born in the USA", []Song{{"Glory Days"}, {"Dancing in the Dark"}}}, + }} + var got Music + if _, err := Decode(tomlTableArrays, &got); err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +func TestTableNesting(t *testing.T) { + for _, tt := range []struct { + t string + want []string + }{ + {"[a.b.c]", []string{"a", "b", "c"}}, + {`[a."b.c"]`, []string{"a", "b.c"}}, + {`[a.'b.c']`, []string{"a", "b.c"}}, + {`[a.' b ']`, []string{"a", " b "}}, + {"[ d.e.f ]", []string{"d", "e", "f"}}, + {"[ g . h . i ]", []string{"g", "h", "i"}}, + {`[ j . "ʞ" . 'l' ]`, []string{"j", "ʞ", "l"}}, + } { + var m map[string]interface{} + if _, err := Decode(tt.t, &m); err != nil { + t.Errorf("Decode(%q): got error: %s", tt.t, err) + continue + } + if keys := extractNestedKeys(m); !reflect.DeepEqual(keys, tt.want) { + t.Errorf("Decode(%q): got nested keys %#v; want %#v", + tt.t, keys, tt.want) + } + } +} + +func extractNestedKeys(v map[string]interface{}) []string { + var result []string + for { + if len(v) != 1 { + return result + } + for k, m := range v { + result = append(result, k) + var ok bool + v, ok = m.(map[string]interface{}) + if !ok { + return result + } + } + + } +} + +// Case insensitive matching tests. +// A bit more comprehensive than needed given the current implementation, +// but implementations change. +// Probably still missing demonstrations of some ugly corner cases regarding +// case insensitive matching and multiple fields. +func TestCase(t *testing.T) { + var caseToml = ` +tOpString = "string" +tOpInt = 1 +tOpFloat = 1.1 +tOpBool = true +tOpdate = 2006-01-02T15:04:05Z +tOparray = [ "array" ] +Match = "i should be in Match only" +MatcH = "i should be in MatcH only" +once = "just once" +[nEst.eD] +nEstedString = "another string" +` + + type InsensitiveEd struct { + NestedString string + } + + type InsensitiveNest struct { + Ed InsensitiveEd + } + + type Insensitive struct { + TopString string + TopInt int + TopFloat float64 + TopBool bool + TopDate time.Time + TopArray []string + Match string + MatcH string + Once string + OncE string + Nest InsensitiveNest + } + + tme, err := time.Parse(time.RFC3339, time.RFC3339[:len(time.RFC3339)-5]) + if err != nil { + panic(err) + } + expected := Insensitive{ + TopString: "string", + TopInt: 1, + TopFloat: 1.1, + TopBool: true, + TopDate: tme, + TopArray: []string{"array"}, + MatcH: "i should be in MatcH only", + Match: "i should be in Match only", + Once: "just once", + OncE: "", + Nest: InsensitiveNest{ + Ed: InsensitiveEd{NestedString: "another string"}, + }, + } + var got Insensitive + if _, err := Decode(caseToml, &got); err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +func TestPointers(t *testing.T) { + type Object struct { + Type string + Description string + } + + type Dict struct { + NamedObject map[string]*Object + BaseObject *Object + Strptr *string + Strptrs []*string + } + s1, s2, s3 := "blah", "abc", "def" + expected := &Dict{ + Strptr: &s1, + Strptrs: []*string{&s2, &s3}, + NamedObject: map[string]*Object{ + "foo": {"FOO", "fooooo!!!"}, + "bar": {"BAR", "ba-ba-ba-ba-barrrr!!!"}, + }, + BaseObject: &Object{"BASE", "da base"}, + } + + ex1 := ` +Strptr = "blah" +Strptrs = ["abc", "def"] + +[NamedObject.foo] +Type = "FOO" +Description = "fooooo!!!" + +[NamedObject.bar] +Type = "BAR" +Description = "ba-ba-ba-ba-barrrr!!!" + +[BaseObject] +Type = "BASE" +Description = "da base" +` + dict := new(Dict) + _, err := Decode(ex1, dict) + if err != nil { + t.Errorf("Decode error: %v", err) + } + if !reflect.DeepEqual(expected, dict) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, dict) + } +} + +func TestDecodeDatetime(t *testing.T) { + const noTimestamp = "2006-01-02T15:04:05" + for _, tt := range []struct { + s string + t string + format string + }{ + {"1979-05-27T07:32:00Z", "1979-05-27T07:32:00Z", time.RFC3339}, + {"1979-05-27T00:32:00-07:00", "1979-05-27T00:32:00-07:00", time.RFC3339}, + { + "1979-05-27T00:32:00.999999-07:00", + "1979-05-27T00:32:00.999999-07:00", + time.RFC3339, + }, + {"1979-05-27T07:32:00", "1979-05-27T07:32:00", noTimestamp}, + { + "1979-05-27T00:32:00.999999", + "1979-05-27T00:32:00.999999", + noTimestamp, + }, + {"1979-05-27", "1979-05-27T00:00:00", noTimestamp}, + } { + var x struct{ D time.Time } + input := "d = " + tt.s + if _, err := Decode(input, &x); err != nil { + t.Errorf("Decode(%q): got error: %s", input, err) + continue + } + want, err := time.ParseInLocation(tt.format, tt.t, time.Local) + if err != nil { + panic(err) + } + if !x.D.Equal(want) { + t.Errorf("Decode(%q): got %s; want %s", input, x.D, want) + } + } +} + +func TestDecodeBadDatetime(t *testing.T) { + var x struct{ T time.Time } + for _, s := range []string{ + "123", + "2006-01-50T00:00:00Z", + "2006-01-30T00:00", + "2006-01-30T", + } { + input := "T = " + s + if _, err := Decode(input, &x); err == nil { + t.Errorf("Expected invalid DateTime error for %q", s) + } + } +} + +func TestDecodeMultilineStrings(t *testing.T) { + var x struct { + S string + } + const s0 = `s = """ +a b \n c +d e f +"""` + if _, err := Decode(s0, &x); err != nil { + t.Fatal(err) + } + if want := "a b \n c\nd e f\n"; x.S != want { + t.Errorf("got: %q; want: %q", x.S, want) + } + const s1 = `s = """a b c\ +"""` + if _, err := Decode(s1, &x); err != nil { + t.Fatal(err) + } + if want := "a b c"; x.S != want { + t.Errorf("got: %q; want: %q", x.S, want) + } +} + +type sphere struct { + Center [3]float64 + Radius float64 +} + +func TestDecodeSimpleArray(t *testing.T) { + var s1 sphere + if _, err := Decode(`center = [0.0, 1.5, 0.0]`, &s1); err != nil { + t.Fatal(err) + } +} + +func TestDecodeArrayWrongSize(t *testing.T) { + var s1 sphere + if _, err := Decode(`center = [0.1, 2.3]`, &s1); err == nil { + t.Fatal("Expected array type mismatch error") + } +} + +func TestDecodeLargeIntoSmallInt(t *testing.T) { + type table struct { + Value int8 + } + var tab table + if _, err := Decode(`value = 500`, &tab); err == nil { + t.Fatal("Expected integer out-of-bounds error.") + } +} + +func TestDecodeSizedInts(t *testing.T) { + type table struct { + U8 uint8 + U16 uint16 + U32 uint32 + U64 uint64 + U uint + I8 int8 + I16 int16 + I32 int32 + I64 int64 + I int + } + answer := table{1, 1, 1, 1, 1, -1, -1, -1, -1, -1} + toml := ` + u8 = 1 + u16 = 1 + u32 = 1 + u64 = 1 + u = 1 + i8 = -1 + i16 = -1 + i32 = -1 + i64 = -1 + i = -1 + ` + var tab table + if _, err := Decode(toml, &tab); err != nil { + t.Fatal(err.Error()) + } + if answer != tab { + t.Fatalf("Expected %#v but got %#v", answer, tab) + } +} + +func TestDecodeInts(t *testing.T) { + for _, tt := range []struct { + s string + want int64 + }{ + {"0", 0}, + {"+99", 99}, + {"-10", -10}, + {"1_234_567", 1234567}, + {"1_2_3_4", 1234}, + {"-9_223_372_036_854_775_808", math.MinInt64}, + {"9_223_372_036_854_775_807", math.MaxInt64}, + } { + var x struct{ N int64 } + input := "n = " + tt.s + if _, err := Decode(input, &x); err != nil { + t.Errorf("Decode(%q): got error: %s", input, err) + continue + } + if x.N != tt.want { + t.Errorf("Decode(%q): got %d; want %d", input, x.N, tt.want) + } + } +} + +func TestDecodeFloats(t *testing.T) { + for _, tt := range []struct { + s string + want float64 + }{ + {"+1.0", 1}, + {"3.1415", 3.1415}, + {"-0.01", -0.01}, + {"5e+22", 5e22}, + {"1e6", 1e6}, + {"-2E-2", -2e-2}, + {"6.626e-34", 6.626e-34}, + {"9_224_617.445_991_228_313", 9224617.445991228313}, + {"9_876.54_32e1_0", 9876.5432e10}, + } { + var x struct{ N float64 } + input := "n = " + tt.s + if _, err := Decode(input, &x); err != nil { + t.Errorf("Decode(%q): got error: %s", input, err) + continue + } + if x.N != tt.want { + t.Errorf("Decode(%q): got %f; want %f", input, x.N, tt.want) + } + } +} + +func TestDecodeMalformedNumbers(t *testing.T) { + for _, tt := range []struct { + s string + want string + }{ + {"++99", "expected a digit"}, + {"0..1", "must be followed by one or more digits"}, + {"0.1.2", "Invalid float value"}, + {"1e2.3", "Invalid float value"}, + {"1e2e3", "Invalid float value"}, + {"_123", "expected value"}, + {"123_", "surrounded by digits"}, + {"1._23", "surrounded by digits"}, + {"1e__23", "surrounded by digits"}, + {"123.", "must be followed by one or more digits"}, + {"1.e2", "must be followed by one or more digits"}, + } { + var x struct{ N interface{} } + input := "n = " + tt.s + _, err := Decode(input, &x) + if err == nil { + t.Errorf("Decode(%q): got nil, want error containing %q", + input, tt.want) + continue + } + if !strings.Contains(err.Error(), tt.want) { + t.Errorf("Decode(%q): got %q, want error containing %q", + input, err, tt.want) + } + } +} + +func TestDecodeBadValues(t *testing.T) { + for _, tt := range []struct { + v interface{} + want string + }{ + {3, "non-pointer int"}, + {(*int)(nil), "nil"}, + } { + _, err := Decode(`x = 3`, tt.v) + if err == nil { + t.Errorf("Decode(%v): got nil; want error containing %q", + tt.v, tt.want) + continue + } + if !strings.Contains(err.Error(), tt.want) { + t.Errorf("Decode(%v): got %q; want error containing %q", + tt.v, err, tt.want) + } + } +} + +func TestUnmarshaler(t *testing.T) { + + var tomlBlob = ` +[dishes.hamboogie] +name = "Hamboogie with fries" +price = 10.99 + +[[dishes.hamboogie.ingredients]] +name = "Bread Bun" + +[[dishes.hamboogie.ingredients]] +name = "Lettuce" + +[[dishes.hamboogie.ingredients]] +name = "Real Beef Patty" + +[[dishes.hamboogie.ingredients]] +name = "Tomato" + +[dishes.eggsalad] +name = "Egg Salad with rice" +price = 3.99 + +[[dishes.eggsalad.ingredients]] +name = "Egg" + +[[dishes.eggsalad.ingredients]] +name = "Mayo" + +[[dishes.eggsalad.ingredients]] +name = "Rice" +` + m := &menu{} + if _, err := Decode(tomlBlob, m); err != nil { + t.Fatal(err) + } + + if len(m.Dishes) != 2 { + t.Log("two dishes should be loaded with UnmarshalTOML()") + t.Errorf("expected %d but got %d", 2, len(m.Dishes)) + } + + eggSalad := m.Dishes["eggsalad"] + if _, ok := interface{}(eggSalad).(dish); !ok { + t.Errorf("expected a dish") + } + + if eggSalad.Name != "Egg Salad with rice" { + t.Errorf("expected the dish to be named 'Egg Salad with rice'") + } + + if len(eggSalad.Ingredients) != 3 { + t.Log("dish should be loaded with UnmarshalTOML()") + t.Errorf("expected %d but got %d", 3, len(eggSalad.Ingredients)) + } + + found := false + for _, i := range eggSalad.Ingredients { + if i.Name == "Rice" { + found = true + break + } + } + if !found { + t.Error("Rice was not loaded in UnmarshalTOML()") + } + + // test on a value - must be passed as * + o := menu{} + if _, err := Decode(tomlBlob, &o); err != nil { + t.Fatal(err) + } + +} + +func TestDecodeInlineTable(t *testing.T) { + input := ` +[CookieJar] +Types = {Chocolate = "yummy", Oatmeal = "best ever"} + +[Seasons] +Locations = {NY = {Temp = "not cold", Rating = 4}, MI = {Temp = "freezing", Rating = 9}} +` + type cookieJar struct { + Types map[string]string + } + type properties struct { + Temp string + Rating int + } + type seasons struct { + Locations map[string]properties + } + type wrapper struct { + CookieJar cookieJar + Seasons seasons + } + var got wrapper + + meta, err := Decode(input, &got) + if err != nil { + t.Fatal(err) + } + want := wrapper{ + CookieJar: cookieJar{ + Types: map[string]string{ + "Chocolate": "yummy", + "Oatmeal": "best ever", + }, + }, + Seasons: seasons{ + Locations: map[string]properties{ + "NY": { + Temp: "not cold", + Rating: 4, + }, + "MI": { + Temp: "freezing", + Rating: 9, + }, + }, + }, + } + if !reflect.DeepEqual(got, want) { + t.Fatalf("after decode, got:\n\n%#v\n\nwant:\n\n%#v", got, want) + } + if len(meta.keys) != 12 { + t.Errorf("after decode, got %d meta keys; want 12", len(meta.keys)) + } + if len(meta.types) != 12 { + t.Errorf("after decode, got %d meta types; want 12", len(meta.types)) + } +} + +func TestDecodeInlineTableArray(t *testing.T) { + type point struct { + X, Y, Z int + } + var got struct { + Points []point + } + // Example inline table array from the spec. + const in = ` +points = [ { x = 1, y = 2, z = 3 }, + { x = 7, y = 8, z = 9 }, + { x = 2, y = 4, z = 8 } ] + +` + if _, err := Decode(in, &got); err != nil { + t.Fatal(err) + } + want := []point{ + {X: 1, Y: 2, Z: 3}, + {X: 7, Y: 8, Z: 9}, + {X: 2, Y: 4, Z: 8}, + } + if !reflect.DeepEqual(got.Points, want) { + t.Errorf("got %#v; want %#v", got.Points, want) + } +} + +func TestDecodeMalformedInlineTable(t *testing.T) { + for _, tt := range []struct { + s string + want string + }{ + {"{,}", "unexpected comma"}, + {"{x = 3 y = 4}", "expected a comma or an inline table terminator"}, + {"{x=3,,y=4}", "unexpected comma"}, + {"{x=3,\ny=4}", "newlines not allowed"}, + {"{x=3\n,y=4}", "newlines not allowed"}, + } { + var x struct{ A map[string]int } + input := "a = " + tt.s + _, err := Decode(input, &x) + if err == nil { + t.Errorf("Decode(%q): got nil, want error containing %q", + input, tt.want) + continue + } + if !strings.Contains(err.Error(), tt.want) { + t.Errorf("Decode(%q): got %q, want error containing %q", + input, err, tt.want) + } + } +} + +type menu struct { + Dishes map[string]dish +} + +func (m *menu) UnmarshalTOML(p interface{}) error { + m.Dishes = make(map[string]dish) + data, _ := p.(map[string]interface{}) + dishes := data["dishes"].(map[string]interface{}) + for n, v := range dishes { + if d, ok := v.(map[string]interface{}); ok { + nd := dish{} + nd.UnmarshalTOML(d) + m.Dishes[n] = nd + } else { + return fmt.Errorf("not a dish") + } + } + return nil +} + +type dish struct { + Name string + Price float32 + Ingredients []ingredient +} + +func (d *dish) UnmarshalTOML(p interface{}) error { + data, _ := p.(map[string]interface{}) + d.Name, _ = data["name"].(string) + d.Price, _ = data["price"].(float32) + ingredients, _ := data["ingredients"].([]map[string]interface{}) + for _, e := range ingredients { + n, _ := interface{}(e).(map[string]interface{}) + name, _ := n["name"].(string) + i := ingredient{name} + d.Ingredients = append(d.Ingredients, i) + } + return nil +} + +type ingredient struct { + Name string +} + +func TestDecodeSlices(t *testing.T) { + type T struct { + S []string + } + for i, tt := range []struct { + v T + input string + want T + }{ + {T{}, "", T{}}, + {T{[]string{}}, "", T{[]string{}}}, + {T{[]string{"a", "b"}}, "", T{[]string{"a", "b"}}}, + {T{}, "S = []", T{[]string{}}}, + {T{[]string{}}, "S = []", T{[]string{}}}, + {T{[]string{"a", "b"}}, "S = []", T{[]string{}}}, + {T{}, `S = ["x"]`, T{[]string{"x"}}}, + {T{[]string{}}, `S = ["x"]`, T{[]string{"x"}}}, + {T{[]string{"a", "b"}}, `S = ["x"]`, T{[]string{"x"}}}, + } { + if _, err := Decode(tt.input, &tt.v); err != nil { + t.Errorf("[%d] %s", i, err) + continue + } + if !reflect.DeepEqual(tt.v, tt.want) { + t.Errorf("[%d] got %#v; want %#v", i, tt.v, tt.want) + } + } +} + +func TestDecodePrimitive(t *testing.T) { + type S struct { + P Primitive + } + type T struct { + S []int + } + slicep := func(s []int) *[]int { return &s } + arrayp := func(a [2]int) *[2]int { return &a } + mapp := func(m map[string]int) *map[string]int { return &m } + for i, tt := range []struct { + v interface{} + input string + want interface{} + }{ + // slices + {slicep(nil), "", slicep(nil)}, + {slicep([]int{}), "", slicep([]int{})}, + {slicep([]int{1, 2, 3}), "", slicep([]int{1, 2, 3})}, + {slicep(nil), "P = [1,2]", slicep([]int{1, 2})}, + {slicep([]int{}), "P = [1,2]", slicep([]int{1, 2})}, + {slicep([]int{1, 2, 3}), "P = [1,2]", slicep([]int{1, 2})}, + + // arrays + {arrayp([2]int{2, 3}), "", arrayp([2]int{2, 3})}, + {arrayp([2]int{2, 3}), "P = [3,4]", arrayp([2]int{3, 4})}, + + // maps + {mapp(nil), "", mapp(nil)}, + {mapp(map[string]int{}), "", mapp(map[string]int{})}, + {mapp(map[string]int{"a": 1}), "", mapp(map[string]int{"a": 1})}, + {mapp(nil), "[P]\na = 2", mapp(map[string]int{"a": 2})}, + {mapp(map[string]int{}), "[P]\na = 2", mapp(map[string]int{"a": 2})}, + {mapp(map[string]int{"a": 1, "b": 3}), "[P]\na = 2", mapp(map[string]int{"a": 2, "b": 3})}, + + // structs + {&T{nil}, "[P]", &T{nil}}, + {&T{[]int{}}, "[P]", &T{[]int{}}}, + {&T{[]int{1, 2, 3}}, "[P]", &T{[]int{1, 2, 3}}}, + {&T{nil}, "[P]\nS = [1,2]", &T{[]int{1, 2}}}, + {&T{[]int{}}, "[P]\nS = [1,2]", &T{[]int{1, 2}}}, + {&T{[]int{1, 2, 3}}, "[P]\nS = [1,2]", &T{[]int{1, 2}}}, + } { + var s S + md, err := Decode(tt.input, &s) + if err != nil { + t.Errorf("[%d] Decode error: %s", i, err) + continue + } + if err := md.PrimitiveDecode(s.P, tt.v); err != nil { + t.Errorf("[%d] PrimitiveDecode error: %s", i, err) + continue + } + if !reflect.DeepEqual(tt.v, tt.want) { + t.Errorf("[%d] got %#v; want %#v", i, tt.v, tt.want) + } + } +} + +func TestDecodeErrors(t *testing.T) { + for _, s := range []string{ + `x="`, + `x='`, + `x='''`, + + // Cases found by fuzzing in + // https://github.com/BurntSushi/toml/issues/155. + `""�`, // used to panic with index out of range + `e="""`, // used to hang + } { + var x struct{} + _, err := Decode(s, &x) + if err == nil { + t.Errorf("Decode(%q): got nil error", s) + } + } +} + +// Test for https://github.com/BurntSushi/toml/pull/166. +func TestDecodeBoolArray(t *testing.T) { + for _, tt := range []struct { + s string + got interface{} + want interface{} + }{ + { + "a = [true, false]", + &struct{ A []bool }{}, + &struct{ A []bool }{[]bool{true, false}}, + }, + { + "a = {a = true, b = false}", + &struct{ A map[string]bool }{}, + &struct{ A map[string]bool }{map[string]bool{"a": true, "b": false}}, + }, + } { + if _, err := Decode(tt.s, tt.got); err != nil { + t.Errorf("Decode(%q): %s", tt.s, err) + continue + } + if !reflect.DeepEqual(tt.got, tt.want) { + t.Errorf("Decode(%q): got %#v; want %#v", tt.s, tt.got, tt.want) + } + } +} + +func ExampleMetaData_PrimitiveDecode() { + var md MetaData + var err error + + var tomlBlob = ` +ranking = ["Springsteen", "J Geils"] + +[bands.Springsteen] +started = 1973 +albums = ["Greetings", "WIESS", "Born to Run", "Darkness"] + +[bands."J Geils"] +started = 1970 +albums = ["The J. Geils Band", "Full House", "Blow Your Face Out"] +` + + type band struct { + Started int + Albums []string + } + type classics struct { + Ranking []string + Bands map[string]Primitive + } + + // Do the initial decode. Reflection is delayed on Primitive values. + var music classics + if md, err = Decode(tomlBlob, &music); err != nil { + log.Fatal(err) + } + + // MetaData still includes information on Primitive values. + fmt.Printf("Is `bands.Springsteen` defined? %v\n", + md.IsDefined("bands", "Springsteen")) + + // Decode primitive data into Go values. + for _, artist := range music.Ranking { + // A band is a primitive value, so we need to decode it to get a + // real `band` value. + primValue := music.Bands[artist] + + var aBand band + if err = md.PrimitiveDecode(primValue, &aBand); err != nil { + log.Fatal(err) + } + fmt.Printf("%s started in %d.\n", artist, aBand.Started) + } + // Check to see if there were any fields left undecoded. + // Note that this won't be empty before decoding the Primitive value! + fmt.Printf("Undecoded: %q\n", md.Undecoded()) + + // Output: + // Is `bands.Springsteen` defined? true + // Springsteen started in 1973. + // J Geils started in 1970. + // Undecoded: [] +} + +func ExampleDecode() { + var tomlBlob = ` +# Some comments. +[alpha] +ip = "10.0.0.1" + + [alpha.config] + Ports = [ 8001, 8002 ] + Location = "Toronto" + Created = 1987-07-05T05:45:00Z + +[beta] +ip = "10.0.0.2" + + [beta.config] + Ports = [ 9001, 9002 ] + Location = "New Jersey" + Created = 1887-01-05T05:55:00Z +` + + type serverConfig struct { + Ports []int + Location string + Created time.Time + } + + type server struct { + IP string `toml:"ip,omitempty"` + Config serverConfig `toml:"config"` + } + + type servers map[string]server + + var config servers + if _, err := Decode(tomlBlob, &config); err != nil { + log.Fatal(err) + } + + for _, name := range []string{"alpha", "beta"} { + s := config[name] + fmt.Printf("Server: %s (ip: %s) in %s created on %s\n", + name, s.IP, s.Config.Location, + s.Config.Created.Format("2006-01-02")) + fmt.Printf("Ports: %v\n", s.Config.Ports) + } + + // Output: + // Server: alpha (ip: 10.0.0.1) in Toronto created on 1987-07-05 + // Ports: [8001 8002] + // Server: beta (ip: 10.0.0.2) in New Jersey created on 1887-01-05 + // Ports: [9001 9002] +} + +type duration struct { + time.Duration +} + +func (d *duration) UnmarshalText(text []byte) error { + var err error + d.Duration, err = time.ParseDuration(string(text)) + return err +} + +// Example Unmarshaler shows how to decode TOML strings into your own +// custom data type. +func Example_unmarshaler() { + blob := ` +[[song]] +name = "Thunder Road" +duration = "4m49s" + +[[song]] +name = "Stairway to Heaven" +duration = "8m03s" +` + type song struct { + Name string + Duration duration + } + type songs struct { + Song []song + } + var favorites songs + if _, err := Decode(blob, &favorites); err != nil { + log.Fatal(err) + } + + // Code to implement the TextUnmarshaler interface for `duration`: + // + // type duration struct { + // time.Duration + // } + // + // func (d *duration) UnmarshalText(text []byte) error { + // var err error + // d.Duration, err = time.ParseDuration(string(text)) + // return err + // } + + for _, s := range favorites.Song { + fmt.Printf("%s (%s)\n", s.Name, s.Duration) + } + // Output: + // Thunder Road (4m49s) + // Stairway to Heaven (8m3s) +} + +// Example StrictDecoding shows how to detect whether there are keys in the +// TOML document that weren't decoded into the value given. This is useful +// for returning an error to the user if they've included extraneous fields +// in their configuration. +func Example_strictDecoding() { + var blob = ` +key1 = "value1" +key2 = "value2" +key3 = "value3" +` + type config struct { + Key1 string + Key3 string + } + + var conf config + md, err := Decode(blob, &conf) + if err != nil { + log.Fatal(err) + } + fmt.Printf("Undecoded keys: %q\n", md.Undecoded()) + // Output: + // Undecoded keys: ["key2"] +} + +// Example UnmarshalTOML shows how to implement a struct type that knows how to +// unmarshal itself. The struct must take full responsibility for mapping the +// values passed into the struct. The method may be used with interfaces in a +// struct in cases where the actual type is not known until the data is +// examined. +func Example_unmarshalTOML() { + + var blob = ` +[[parts]] +type = "valve" +id = "valve-1" +size = 1.2 +rating = 4 + +[[parts]] +type = "valve" +id = "valve-2" +size = 2.1 +rating = 5 + +[[parts]] +type = "pipe" +id = "pipe-1" +length = 2.1 +diameter = 12 + +[[parts]] +type = "cable" +id = "cable-1" +length = 12 +rating = 3.1 +` + o := &order{} + err := Unmarshal([]byte(blob), o) + if err != nil { + log.Fatal(err) + } + + fmt.Println(len(o.parts)) + + for _, part := range o.parts { + fmt.Println(part.Name()) + } + + // Code to implement UmarshalJSON. + + // type order struct { + // // NOTE `order.parts` is a private slice of type `part` which is an + // // interface and may only be loaded from toml using the + // // UnmarshalTOML() method of the Umarshaler interface. + // parts parts + // } + + // func (o *order) UnmarshalTOML(data interface{}) error { + + // // NOTE the example below contains detailed type casting to show how + // // the 'data' is retrieved. In operational use, a type cast wrapper + // // may be preferred e.g. + // // + // // func AsMap(v interface{}) (map[string]interface{}, error) { + // // return v.(map[string]interface{}) + // // } + // // + // // resulting in: + // // d, _ := AsMap(data) + // // + + // d, _ := data.(map[string]interface{}) + // parts, _ := d["parts"].([]map[string]interface{}) + + // for _, p := range parts { + + // typ, _ := p["type"].(string) + // id, _ := p["id"].(string) + + // // detect the type of part and handle each case + // switch p["type"] { + // case "valve": + + // size := float32(p["size"].(float64)) + // rating := int(p["rating"].(int64)) + + // valve := &valve{ + // Type: typ, + // ID: id, + // Size: size, + // Rating: rating, + // } + + // o.parts = append(o.parts, valve) + + // case "pipe": + + // length := float32(p["length"].(float64)) + // diameter := int(p["diameter"].(int64)) + + // pipe := &pipe{ + // Type: typ, + // ID: id, + // Length: length, + // Diameter: diameter, + // } + + // o.parts = append(o.parts, pipe) + + // case "cable": + + // length := int(p["length"].(int64)) + // rating := float32(p["rating"].(float64)) + + // cable := &cable{ + // Type: typ, + // ID: id, + // Length: length, + // Rating: rating, + // } + + // o.parts = append(o.parts, cable) + + // } + // } + + // return nil + // } + + // type parts []part + + // type part interface { + // Name() string + // } + + // type valve struct { + // Type string + // ID string + // Size float32 + // Rating int + // } + + // func (v *valve) Name() string { + // return fmt.Sprintf("VALVE: %s", v.ID) + // } + + // type pipe struct { + // Type string + // ID string + // Length float32 + // Diameter int + // } + + // func (p *pipe) Name() string { + // return fmt.Sprintf("PIPE: %s", p.ID) + // } + + // type cable struct { + // Type string + // ID string + // Length int + // Rating float32 + // } + + // func (c *cable) Name() string { + // return fmt.Sprintf("CABLE: %s", c.ID) + // } + + // Output: + // 4 + // VALVE: valve-1 + // VALVE: valve-2 + // PIPE: pipe-1 + // CABLE: cable-1 + +} + +type order struct { + // NOTE `order.parts` is a private slice of type `part` which is an + // interface and may only be loaded from toml using the UnmarshalTOML() + // method of the Umarshaler interface. + parts parts +} + +func (o *order) UnmarshalTOML(data interface{}) error { + + // NOTE the example below contains detailed type casting to show how + // the 'data' is retrieved. In operational use, a type cast wrapper + // may be preferred e.g. + // + // func AsMap(v interface{}) (map[string]interface{}, error) { + // return v.(map[string]interface{}) + // } + // + // resulting in: + // d, _ := AsMap(data) + // + + d, _ := data.(map[string]interface{}) + parts, _ := d["parts"].([]map[string]interface{}) + + for _, p := range parts { + + typ, _ := p["type"].(string) + id, _ := p["id"].(string) + + // detect the type of part and handle each case + switch p["type"] { + case "valve": + + size := float32(p["size"].(float64)) + rating := int(p["rating"].(int64)) + + valve := &valve{ + Type: typ, + ID: id, + Size: size, + Rating: rating, + } + + o.parts = append(o.parts, valve) + + case "pipe": + + length := float32(p["length"].(float64)) + diameter := int(p["diameter"].(int64)) + + pipe := &pipe{ + Type: typ, + ID: id, + Length: length, + Diameter: diameter, + } + + o.parts = append(o.parts, pipe) + + case "cable": + + length := int(p["length"].(int64)) + rating := float32(p["rating"].(float64)) + + cable := &cable{ + Type: typ, + ID: id, + Length: length, + Rating: rating, + } + + o.parts = append(o.parts, cable) + + } + } + + return nil +} + +type parts []part + +type part interface { + Name() string +} + +type valve struct { + Type string + ID string + Size float32 + Rating int +} + +func (v *valve) Name() string { + return fmt.Sprintf("VALVE: %s", v.ID) +} + +type pipe struct { + Type string + ID string + Length float32 + Diameter int +} + +func (p *pipe) Name() string { + return fmt.Sprintf("PIPE: %s", p.ID) +} + +type cable struct { + Type string + ID string + Length int + Rating float32 +} + +func (c *cable) Name() string { + return fmt.Sprintf("CABLE: %s", c.ID) +} diff --git a/vendor/github.com/BurntSushi/toml/encode_test.go b/vendor/github.com/BurntSushi/toml/encode_test.go new file mode 100644 index 0000000000000..673b7b0093ca7 --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/encode_test.go @@ -0,0 +1,615 @@ +package toml + +import ( + "bytes" + "fmt" + "log" + "net" + "testing" + "time" +) + +func TestEncodeRoundTrip(t *testing.T) { + type Config struct { + Age int + Cats []string + Pi float64 + Perfection []int + DOB time.Time + Ipaddress net.IP + } + + var inputs = Config{ + 13, + []string{"one", "two", "three"}, + 3.145, + []int{11, 2, 3, 4}, + time.Now(), + net.ParseIP("192.168.59.254"), + } + + var firstBuffer bytes.Buffer + e := NewEncoder(&firstBuffer) + err := e.Encode(inputs) + if err != nil { + t.Fatal(err) + } + var outputs Config + if _, err := Decode(firstBuffer.String(), &outputs); err != nil { + t.Logf("Could not decode:\n-----\n%s\n-----\n", + firstBuffer.String()) + t.Fatal(err) + } + + // could test each value individually, but I'm lazy + var secondBuffer bytes.Buffer + e2 := NewEncoder(&secondBuffer) + err = e2.Encode(outputs) + if err != nil { + t.Fatal(err) + } + if firstBuffer.String() != secondBuffer.String() { + t.Error( + firstBuffer.String(), + "\n\n is not identical to\n\n", + secondBuffer.String()) + } +} + +// XXX(burntsushi) +// I think these tests probably should be removed. They are good, but they +// ought to be obsolete by toml-test. +func TestEncode(t *testing.T) { + type Embedded struct { + Int int `toml:"_int"` + } + type NonStruct int + + date := time.Date(2014, 5, 11, 20, 30, 40, 0, time.FixedZone("IST", 3600)) + dateStr := "2014-05-11T19:30:40Z" + + tests := map[string]struct { + input interface{} + wantOutput string + wantError error + }{ + "bool field": { + input: struct { + BoolTrue bool + BoolFalse bool + }{true, false}, + wantOutput: "BoolTrue = true\nBoolFalse = false\n", + }, + "int fields": { + input: struct { + Int int + Int8 int8 + Int16 int16 + Int32 int32 + Int64 int64 + }{1, 2, 3, 4, 5}, + wantOutput: "Int = 1\nInt8 = 2\nInt16 = 3\nInt32 = 4\nInt64 = 5\n", + }, + "uint fields": { + input: struct { + Uint uint + Uint8 uint8 + Uint16 uint16 + Uint32 uint32 + Uint64 uint64 + }{1, 2, 3, 4, 5}, + wantOutput: "Uint = 1\nUint8 = 2\nUint16 = 3\nUint32 = 4" + + "\nUint64 = 5\n", + }, + "float fields": { + input: struct { + Float32 float32 + Float64 float64 + }{1.5, 2.5}, + wantOutput: "Float32 = 1.5\nFloat64 = 2.5\n", + }, + "string field": { + input: struct{ String string }{"foo"}, + wantOutput: "String = \"foo\"\n", + }, + "string field and unexported field": { + input: struct { + String string + unexported int + }{"foo", 0}, + wantOutput: "String = \"foo\"\n", + }, + "datetime field in UTC": { + input: struct{ Date time.Time }{date}, + wantOutput: fmt.Sprintf("Date = %s\n", dateStr), + }, + "datetime field as primitive": { + // Using a map here to fail if isStructOrMap() returns true for + // time.Time. + input: map[string]interface{}{ + "Date": date, + "Int": 1, + }, + wantOutput: fmt.Sprintf("Date = %s\nInt = 1\n", dateStr), + }, + "array fields": { + input: struct { + IntArray0 [0]int + IntArray3 [3]int + }{[0]int{}, [3]int{1, 2, 3}}, + wantOutput: "IntArray0 = []\nIntArray3 = [1, 2, 3]\n", + }, + "slice fields": { + input: struct{ IntSliceNil, IntSlice0, IntSlice3 []int }{ + nil, []int{}, []int{1, 2, 3}, + }, + wantOutput: "IntSlice0 = []\nIntSlice3 = [1, 2, 3]\n", + }, + "datetime slices": { + input: struct{ DatetimeSlice []time.Time }{ + []time.Time{date, date}, + }, + wantOutput: fmt.Sprintf("DatetimeSlice = [%s, %s]\n", + dateStr, dateStr), + }, + "nested arrays and slices": { + input: struct { + SliceOfArrays [][2]int + ArrayOfSlices [2][]int + SliceOfArraysOfSlices [][2][]int + ArrayOfSlicesOfArrays [2][][2]int + SliceOfMixedArrays [][2]interface{} + ArrayOfMixedSlices [2][]interface{} + }{ + [][2]int{{1, 2}, {3, 4}}, + [2][]int{{1, 2}, {3, 4}}, + [][2][]int{ + { + {1, 2}, {3, 4}, + }, + { + {5, 6}, {7, 8}, + }, + }, + [2][][2]int{ + { + {1, 2}, {3, 4}, + }, + { + {5, 6}, {7, 8}, + }, + }, + [][2]interface{}{ + {1, 2}, {"a", "b"}, + }, + [2][]interface{}{ + {1, 2}, {"a", "b"}, + }, + }, + wantOutput: `SliceOfArrays = [[1, 2], [3, 4]] +ArrayOfSlices = [[1, 2], [3, 4]] +SliceOfArraysOfSlices = [[[1, 2], [3, 4]], [[5, 6], [7, 8]]] +ArrayOfSlicesOfArrays = [[[1, 2], [3, 4]], [[5, 6], [7, 8]]] +SliceOfMixedArrays = [[1, 2], ["a", "b"]] +ArrayOfMixedSlices = [[1, 2], ["a", "b"]] +`, + }, + "empty slice": { + input: struct{ Empty []interface{} }{[]interface{}{}}, + wantOutput: "Empty = []\n", + }, + "(error) slice with element type mismatch (string and integer)": { + input: struct{ Mixed []interface{} }{[]interface{}{1, "a"}}, + wantError: errArrayMixedElementTypes, + }, + "(error) slice with element type mismatch (integer and float)": { + input: struct{ Mixed []interface{} }{[]interface{}{1, 2.5}}, + wantError: errArrayMixedElementTypes, + }, + "slice with elems of differing Go types, same TOML types": { + input: struct { + MixedInts []interface{} + MixedFloats []interface{} + }{ + []interface{}{ + int(1), int8(2), int16(3), int32(4), int64(5), + uint(1), uint8(2), uint16(3), uint32(4), uint64(5), + }, + []interface{}{float32(1.5), float64(2.5)}, + }, + wantOutput: "MixedInts = [1, 2, 3, 4, 5, 1, 2, 3, 4, 5]\n" + + "MixedFloats = [1.5, 2.5]\n", + }, + "(error) slice w/ element type mismatch (one is nested array)": { + input: struct{ Mixed []interface{} }{ + []interface{}{1, []interface{}{2}}, + }, + wantError: errArrayMixedElementTypes, + }, + "(error) slice with 1 nil element": { + input: struct{ NilElement1 []interface{} }{[]interface{}{nil}}, + wantError: errArrayNilElement, + }, + "(error) slice with 1 nil element (and other non-nil elements)": { + input: struct{ NilElement []interface{} }{ + []interface{}{1, nil}, + }, + wantError: errArrayNilElement, + }, + "simple map": { + input: map[string]int{"a": 1, "b": 2}, + wantOutput: "a = 1\nb = 2\n", + }, + "map with interface{} value type": { + input: map[string]interface{}{"a": 1, "b": "c"}, + wantOutput: "a = 1\nb = \"c\"\n", + }, + "map with interface{} value type, some of which are structs": { + input: map[string]interface{}{ + "a": struct{ Int int }{2}, + "b": 1, + }, + wantOutput: "b = 1\n\n[a]\n Int = 2\n", + }, + "nested map": { + input: map[string]map[string]int{ + "a": {"b": 1}, + "c": {"d": 2}, + }, + wantOutput: "[a]\n b = 1\n\n[c]\n d = 2\n", + }, + "nested struct": { + input: struct{ Struct struct{ Int int } }{ + struct{ Int int }{1}, + }, + wantOutput: "[Struct]\n Int = 1\n", + }, + "nested struct and non-struct field": { + input: struct { + Struct struct{ Int int } + Bool bool + }{struct{ Int int }{1}, true}, + wantOutput: "Bool = true\n\n[Struct]\n Int = 1\n", + }, + "2 nested structs": { + input: struct{ Struct1, Struct2 struct{ Int int } }{ + struct{ Int int }{1}, struct{ Int int }{2}, + }, + wantOutput: "[Struct1]\n Int = 1\n\n[Struct2]\n Int = 2\n", + }, + "deeply nested structs": { + input: struct { + Struct1, Struct2 struct{ Struct3 *struct{ Int int } } + }{ + struct{ Struct3 *struct{ Int int } }{&struct{ Int int }{1}}, + struct{ Struct3 *struct{ Int int } }{nil}, + }, + wantOutput: "[Struct1]\n [Struct1.Struct3]\n Int = 1" + + "\n\n[Struct2]\n", + }, + "nested struct with nil struct elem": { + input: struct { + Struct struct{ Inner *struct{ Int int } } + }{ + struct{ Inner *struct{ Int int } }{nil}, + }, + wantOutput: "[Struct]\n", + }, + "nested struct with no fields": { + input: struct { + Struct struct{ Inner struct{} } + }{ + struct{ Inner struct{} }{struct{}{}}, + }, + wantOutput: "[Struct]\n [Struct.Inner]\n", + }, + "struct with tags": { + input: struct { + Struct struct { + Int int `toml:"_int"` + } `toml:"_struct"` + Bool bool `toml:"_bool"` + }{ + struct { + Int int `toml:"_int"` + }{1}, true, + }, + wantOutput: "_bool = true\n\n[_struct]\n _int = 1\n", + }, + "embedded struct": { + input: struct{ Embedded }{Embedded{1}}, + wantOutput: "_int = 1\n", + }, + "embedded *struct": { + input: struct{ *Embedded }{&Embedded{1}}, + wantOutput: "_int = 1\n", + }, + "nested embedded struct": { + input: struct { + Struct struct{ Embedded } `toml:"_struct"` + }{struct{ Embedded }{Embedded{1}}}, + wantOutput: "[_struct]\n _int = 1\n", + }, + "nested embedded *struct": { + input: struct { + Struct struct{ *Embedded } `toml:"_struct"` + }{struct{ *Embedded }{&Embedded{1}}}, + wantOutput: "[_struct]\n _int = 1\n", + }, + "embedded non-struct": { + input: struct{ NonStruct }{5}, + wantOutput: "NonStruct = 5\n", + }, + "array of tables": { + input: struct { + Structs []*struct{ Int int } `toml:"struct"` + }{ + []*struct{ Int int }{{1}, {3}}, + }, + wantOutput: "[[struct]]\n Int = 1\n\n[[struct]]\n Int = 3\n", + }, + "array of tables order": { + input: map[string]interface{}{ + "map": map[string]interface{}{ + "zero": 5, + "arr": []map[string]int{ + { + "friend": 5, + }, + }, + }, + }, + wantOutput: "[map]\n zero = 5\n\n [[map.arr]]\n friend = 5\n", + }, + "(error) top-level slice": { + input: []struct{ Int int }{{1}, {2}, {3}}, + wantError: errNoKey, + }, + "(error) slice of slice": { + input: struct { + Slices [][]struct{ Int int } + }{ + [][]struct{ Int int }{{{1}}, {{2}}, {{3}}}, + }, + wantError: errArrayNoTable, + }, + "(error) map no string key": { + input: map[int]string{1: ""}, + wantError: errNonString, + }, + "(error) empty key name": { + input: map[string]int{"": 1}, + wantError: errAnything, + }, + "(error) empty map name": { + input: map[string]interface{}{ + "": map[string]int{"v": 1}, + }, + wantError: errAnything, + }, + } + for label, test := range tests { + encodeExpected(t, label, test.input, test.wantOutput, test.wantError) + } +} + +func TestEncodeNestedTableArrays(t *testing.T) { + type song struct { + Name string `toml:"name"` + } + type album struct { + Name string `toml:"name"` + Songs []song `toml:"songs"` + } + type springsteen struct { + Albums []album `toml:"albums"` + } + value := springsteen{ + []album{ + {"Born to Run", + []song{{"Jungleland"}, {"Meeting Across the River"}}}, + {"Born in the USA", + []song{{"Glory Days"}, {"Dancing in the Dark"}}}, + }, + } + expected := `[[albums]] + name = "Born to Run" + + [[albums.songs]] + name = "Jungleland" + + [[albums.songs]] + name = "Meeting Across the River" + +[[albums]] + name = "Born in the USA" + + [[albums.songs]] + name = "Glory Days" + + [[albums.songs]] + name = "Dancing in the Dark" +` + encodeExpected(t, "nested table arrays", value, expected, nil) +} + +func TestEncodeArrayHashWithNormalHashOrder(t *testing.T) { + type Alpha struct { + V int + } + type Beta struct { + V int + } + type Conf struct { + V int + A Alpha + B []Beta + } + + val := Conf{ + V: 1, + A: Alpha{2}, + B: []Beta{{3}}, + } + expected := "V = 1\n\n[A]\n V = 2\n\n[[B]]\n V = 3\n" + encodeExpected(t, "array hash with normal hash order", val, expected, nil) +} + +func TestEncodeWithOmitEmpty(t *testing.T) { + type simple struct { + Bool bool `toml:"bool,omitempty"` + String string `toml:"string,omitempty"` + Array [0]byte `toml:"array,omitempty"` + Slice []int `toml:"slice,omitempty"` + Map map[string]string `toml:"map,omitempty"` + } + + var v simple + encodeExpected(t, "fields with omitempty are omitted when empty", v, "", nil) + v = simple{ + Bool: true, + String: " ", + Slice: []int{2, 3, 4}, + Map: map[string]string{"foo": "bar"}, + } + expected := `bool = true +string = " " +slice = [2, 3, 4] + +[map] + foo = "bar" +` + encodeExpected(t, "fields with omitempty are not omitted when non-empty", + v, expected, nil) +} + +func TestEncodeWithOmitZero(t *testing.T) { + type simple struct { + Number int `toml:"number,omitzero"` + Real float64 `toml:"real,omitzero"` + Unsigned uint `toml:"unsigned,omitzero"` + } + + value := simple{0, 0.0, uint(0)} + expected := "" + + encodeExpected(t, "simple with omitzero, all zero", value, expected, nil) + + value.Number = 10 + value.Real = 20 + value.Unsigned = 5 + expected = `number = 10 +real = 20.0 +unsigned = 5 +` + encodeExpected(t, "simple with omitzero, non-zero", value, expected, nil) +} + +func TestEncodeOmitemptyWithEmptyName(t *testing.T) { + type simple struct { + S []int `toml:",omitempty"` + } + v := simple{[]int{1, 2, 3}} + expected := "S = [1, 2, 3]\n" + encodeExpected(t, "simple with omitempty, no name, non-empty field", + v, expected, nil) +} + +func TestEncodeAnonymousStruct(t *testing.T) { + type Inner struct{ N int } + type Outer0 struct{ Inner } + type Outer1 struct { + Inner `toml:"inner"` + } + + v0 := Outer0{Inner{3}} + expected := "N = 3\n" + encodeExpected(t, "embedded anonymous untagged struct", v0, expected, nil) + + v1 := Outer1{Inner{3}} + expected = "[inner]\n N = 3\n" + encodeExpected(t, "embedded anonymous tagged struct", v1, expected, nil) +} + +func TestEncodeAnonymousStructPointerField(t *testing.T) { + type Inner struct{ N int } + type Outer0 struct{ *Inner } + type Outer1 struct { + *Inner `toml:"inner"` + } + + v0 := Outer0{} + expected := "" + encodeExpected(t, "nil anonymous untagged struct pointer field", v0, expected, nil) + + v0 = Outer0{&Inner{3}} + expected = "N = 3\n" + encodeExpected(t, "non-nil anonymous untagged struct pointer field", v0, expected, nil) + + v1 := Outer1{} + expected = "" + encodeExpected(t, "nil anonymous tagged struct pointer field", v1, expected, nil) + + v1 = Outer1{&Inner{3}} + expected = "[inner]\n N = 3\n" + encodeExpected(t, "non-nil anonymous tagged struct pointer field", v1, expected, nil) +} + +func TestEncodeIgnoredFields(t *testing.T) { + type simple struct { + Number int `toml:"-"` + } + value := simple{} + expected := "" + encodeExpected(t, "ignored field", value, expected, nil) +} + +func encodeExpected( + t *testing.T, label string, val interface{}, wantStr string, wantErr error, +) { + var buf bytes.Buffer + enc := NewEncoder(&buf) + err := enc.Encode(val) + if err != wantErr { + if wantErr != nil { + if wantErr == errAnything && err != nil { + return + } + t.Errorf("%s: want Encode error %v, got %v", label, wantErr, err) + } else { + t.Errorf("%s: Encode failed: %s", label, err) + } + } + if err != nil { + return + } + if got := buf.String(); wantStr != got { + t.Errorf("%s: want\n-----\n%q\n-----\nbut got\n-----\n%q\n-----\n", + label, wantStr, got) + } +} + +func ExampleEncoder_Encode() { + date, _ := time.Parse(time.RFC822, "14 Mar 10 18:00 UTC") + var config = map[string]interface{}{ + "date": date, + "counts": []int{1, 1, 2, 3, 5, 8}, + "hash": map[string]string{ + "key1": "val1", + "key2": "val2", + }, + } + buf := new(bytes.Buffer) + if err := NewEncoder(buf).Encode(config); err != nil { + log.Fatal(err) + } + fmt.Println(buf.String()) + + // Output: + // counts = [1, 1, 2, 3, 5, 8] + // date = 2010-03-14T18:00:00Z + // + // [hash] + // key1 = "val1" + // key2 = "val2" +} diff --git a/vendor/github.com/BurntSushi/toml/session.vim b/vendor/github.com/BurntSushi/toml/session.vim new file mode 100644 index 0000000000000..562164be06030 --- /dev/null +++ b/vendor/github.com/BurntSushi/toml/session.vim @@ -0,0 +1 @@ +au BufWritePost *.go silent!make tags > /dev/null 2>&1 diff --git a/vendor/github.com/apache/thrift/.clang-format b/vendor/github.com/apache/thrift/.clang-format new file mode 100644 index 0000000000000..a62eef8576c14 --- /dev/null +++ b/vendor/github.com/apache/thrift/.clang-format @@ -0,0 +1,56 @@ +--- +Language: Cpp +# BasedOnStyle: LLVM +AccessModifierOffset: -2 +ConstructorInitializerIndentWidth: 2 +AlignEscapedNewlinesLeft: false +AlignTrailingComments: true +AllowAllParametersOfDeclarationOnNextLine: false +AllowShortBlocksOnASingleLine: false +AllowShortIfStatementsOnASingleLine: false +AllowShortLoopsOnASingleLine: false +AllowShortFunctionsOnASingleLine: Inline +AlwaysBreakTemplateDeclarations: true +AlwaysBreakBeforeMultilineStrings: true +BreakBeforeBinaryOperators: true +BreakBeforeTernaryOperators: true +BreakConstructorInitializersBeforeComma: false +BinPackParameters: false +ColumnLimit: 100 +ConstructorInitializerAllOnOneLineOrOnePerLine: true +DerivePointerAlignment: false +IndentCaseLabels: false +IndentWrappedFunctionNames: false +IndentFunctionDeclarationAfterType: false +MaxEmptyLinesToKeep: 1 +KeepEmptyLinesAtTheStartOfBlocks: true +NamespaceIndentation: None +ObjCSpaceAfterProperty: false +ObjCSpaceBeforeProtocolList: true +PenaltyBreakBeforeFirstCallParameter: 190 +PenaltyBreakComment: 300 +PenaltyBreakString: 10000 +PenaltyBreakFirstLessLess: 120 +PenaltyExcessCharacter: 1000000 +PenaltyReturnTypeOnItsOwnLine: 1200 +PointerAlignment: Left +SpacesBeforeTrailingComments: 1 +Cpp11BracedListStyle: true +Standard: Auto +IndentWidth: 2 +TabWidth: 4 +UseTab: Never +BreakBeforeBraces: Attach +SpacesInParentheses: false +SpacesInAngles: false +SpaceInEmptyParentheses: false +SpacesInCStyleCastParentheses: false +SpacesInContainerLiterals: true +SpaceBeforeAssignmentOperators: true +ContinuationIndentWidth: 4 +CommentPragmas: '^ IWYU pragma:' +ForEachMacros: [ foreach, Q_FOREACH, BOOST_FOREACH ] +SpaceBeforeParens: ControlStatements +DisableFormat: false +... + diff --git a/vendor/github.com/apache/thrift/.dockerignore b/vendor/github.com/apache/thrift/.dockerignore new file mode 100644 index 0000000000000..2d2ecd68da83a --- /dev/null +++ b/vendor/github.com/apache/thrift/.dockerignore @@ -0,0 +1 @@ +.git/ diff --git a/vendor/github.com/apache/thrift/.editorconfig b/vendor/github.com/apache/thrift/.editorconfig new file mode 100644 index 0000000000000..3611762c8a3d9 --- /dev/null +++ b/vendor/github.com/apache/thrift/.editorconfig @@ -0,0 +1,112 @@ +# +## Licensed to the Apache Software Foundation (ASF) under one +## or more contributor license agreements. See the NOTICE file +## distributed with this work for additional information +## regarding copyright ownership. The ASF licenses this file +## to you 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. +## +# + +# EditorConfig: http://editorconfig.org +# see doc/coding_standards.md + +root = true + +[*] +end_of_line = lf +charset = utf-8 +trim_trailing_whitespace = true +insert_final_newline = true + +# ActionScript +# [*.as] + +# C +# [*.c] + +# C++ +[*.cpp] +indent_style = space +indent_size = 2 + +# C-Sharp +# [*.cs] + +# D +# [*.d] + +# Erlang +# [*.erl] + +# Go-lang +[*.go] +indent_style = tab +indent_size = 8 + +# C header files +# [*.h] + +# Haskell +# [*.hs] + +# Haxe +# [*.hx] + +# Java +# [*.java] + +# Javascript +[*.js] +indent_style = space +indent_size = 2 + +# JSON +[*.json] +indent_style = space +indent_size = 2 + +# Lua +# [*.lua] + +[*.markdown] +indent_style = space +trim_trailing_whitespace = false + +[*.md] +indent_style = space +trim_trailing_whitespace = false + +# OCaml +# [*.ml] + +# Delphi Pascal +# [*.pas] + +# PHP +# [*.php] + +# Perl +# [*.pm] + +# Python +# [*.py] + +# Ruby +# [*.rb] + +# Typescript +# [*.ts] + +# XML +# [*.xml] diff --git a/vendor/github.com/apache/thrift/.gitattributes b/vendor/github.com/apache/thrift/.gitattributes new file mode 100644 index 0000000000000..176a458f94e0e --- /dev/null +++ b/vendor/github.com/apache/thrift/.gitattributes @@ -0,0 +1 @@ +* text=auto diff --git a/vendor/github.com/apache/thrift/.gitignore b/vendor/github.com/apache/thrift/.gitignore new file mode 100644 index 0000000000000..140c93b003fa7 --- /dev/null +++ b/vendor/github.com/apache/thrift/.gitignore @@ -0,0 +1,326 @@ +# generic ignores +*.la +*.lo +*.o +*.deps +*.dirstamp +*.libs +*.log +*.trs +*.suo +*.pyc +*.cache +*.user +*.ipch +*.sdf +*.jar +*.exe +*.dll +*_ReSharper* +*.opensdf +*.swp +*.hi +*~ + +.*project +junit*.properties +.idea +gen-* +Makefile +Makefile.in +aclocal.m4 +acinclude.m4 +autom4te.cache +cmake-* +node_modules +compile +test-driver +erl_crash.dump + +.sonar +.DS_Store +.svn +.vagrant + +/contrib/.vagrant/ +/aclocal/libtool.m4 +/aclocal/lt*.m4 +/autoscan.log +/autoscan-*.log +/cmake_* +/compiler/cpp/compiler.VC.db +/compiler/cpp/compiler.VC.VC.opendb +/compiler/cpp/test/plugin/t_cpp_generator.cc +/compiler/cpp/src/thrift/plugin/plugin_constants.cpp +/compiler/cpp/src/thrift/plugin/plugin_constants.h +/compiler/cpp/src/thrift/plugin/plugin_types.cpp +/compiler/cpp/src/thrift/plugin/plugin_types.h +/compiler/cpp/test/*test +/compiler/cpp/test/thrift-gen-* +/compiler/cpp/src/thrift/thrift-bootstrap +/compiler/cpp/src/thrift/plugin/gen.stamp +/compiler/cpp/Debug +/compiler/cpp/Release +/compiler/cpp/src/thrift/libparse.a +/compiler/cpp/src/thrift/thriftl.cc +/compiler/cpp/src/thrift/thrifty.cc +/compiler/cpp/src/thrift/thrifty.hh +/compiler/cpp/src/thrift/windows/version.h +/compiler/cpp/thrift +/compiler/cpp/thriftl.cc +/compiler/cpp/thrifty.cc +/compiler/cpp/lex.yythriftl.cc +/compiler/cpp/thrifty.h +/compiler/cpp/thrifty.hh +/compiler/cpp/src/thrift/version.h +/config.* +/configure +/configure.lineno +/configure.scan +/contrib/fb303/config.cache +/contrib/fb303/config.log +/contrib/fb303/config.status +/contrib/fb303/configure +/contrib/fb303/cpp/libfb303.a +/contrib/fb303/java/build/ +/contrib/fb303/py/build/ +/contrib/fb303/py/fb303/FacebookService-remote +/contrib/fb303/py/fb303/FacebookService.py +/contrib/fb303/py/fb303/__init__.py +/contrib/fb303/py/fb303/constants.py +/contrib/fb303/py/fb303/ttypes.py +/depcomp +/install-sh +/lib/cpp/Debug/ +/lib/cpp/Debug-mt/ +/lib/cpp/Release/ +/lib/cpp/Release-mt/ +/lib/cpp/src/thrift/qt/moc_TQTcpServer.cpp +/lib/cpp/src/thrift/qt/moc__TQTcpServer.cpp +/lib/cpp/src/thrift/config.h +/lib/cpp/src/thrift/stamp-h2 +/lib/cpp/test/Benchmark +/lib/cpp/test/AllProtocolsTest +/lib/cpp/test/DebugProtoTest +/lib/cpp/test/DenseProtoTest +/lib/cpp/test/EnumTest +/lib/cpp/test/JSONProtoTest +/lib/cpp/test/OptionalRequiredTest +/lib/cpp/test/SecurityTest +/lib/cpp/test/SpecializationTest +/lib/cpp/test/ReflectionTest +/lib/cpp/test/RecursiveTest +/lib/cpp/test/TFDTransportTest +/lib/cpp/test/TFileTransportTest +/lib/cpp/test/TInterruptTest +/lib/cpp/test/TNonblockingServerTest +/lib/cpp/test/TPipedTransportTest +/lib/cpp/test/TServerIntegrationTest +/lib/cpp/test/TSocketInterruptTest +/lib/cpp/test/TransportTest +/lib/cpp/test/UnitTests +/lib/cpp/test/ZlibTest +/lib/cpp/test/OpenSSLManualInitTest +/lib/cpp/test/concurrency_test +/lib/cpp/test/link_test +/lib/cpp/test/processor_test +/lib/cpp/test/tests.xml +/lib/cpp/concurrency_test +/lib/cpp/*.pc +/lib/cpp/x64/Debug/ +/lib/cpp/x64/Debug-mt/ +/lib/cpp/x64/Release +/lib/cpp/x64/Release-mt +/lib/c_glib/*.gcda +/lib/c_glib/*.gcno +/lib/c_glib/*.loT +/lib/c_glib/src/thrift/config.h +/lib/c_glib/src/thrift/stamp-h3 +/lib/c_glib/test/*.gcno +/lib/c_glib/test/testwrapper.sh +/lib/c_glib/test/testwrapper-test* +/lib/c_glib/test/testapplicationexception +/lib/c_glib/test/testbinaryprotocol +/lib/c_glib/test/testcompactprotocol +/lib/c_glib/test/testbufferedtransport +/lib/c_glib/test/testcontainertest +/lib/c_glib/test/testdebugproto +/lib/c_glib/test/testfdtransport +/lib/c_glib/test/testframedtransport +/lib/c_glib/test/testmemorybuffer +/lib/c_glib/test/testoptionalrequired +/lib/c_glib/test/testsimpleserver +/lib/c_glib/test/teststruct +/lib/c_glib/test/testthrifttest +/lib/c_glib/test/testthrifttestclient +/lib/c_glib/test/testtransportsocket +/lib/c_glib/test/testserialization +/lib/c_glib/thriftc.pc +/lib/c_glib/thrift_c_glib.pc +/lib/csharp/**/bin/ +/lib/csharp/**/obj/ +/lib/csharp/src/packages +/lib/d/test/*.pem +/lib/d/libthriftd*.a +/lib/d/test/async_test +/lib/d/test/client_pool_test +/lib/d/test/serialization_benchmark +/lib/d/test/stress_test_server +/lib/d/test/thrift_test_client +/lib/d/test/thrift_test_server +/lib/d/test/transport_test +/lib/d/unittest/ +/lib/dart/coverage +/lib/dart/**/.packages +/lib/dart/**/packages +/lib/dart/**/.pub/ +/lib/dart/**/pubspec.lock +/lib/delphi/src/*.dcu +/lib/delphi/test/*.identcache +/lib/delphi/test/*.local +/lib/delphi/test/*.dcu +/lib/delphi/test/*.2007 +/lib/delphi/test/*.dproj +/lib/delphi/test/*.dproj +/lib/delphi/test/codegen/*.bat +/lib/delphi/test/skip/*.local +/lib/delphi/test/skip/*.identcache +/lib/delphi/test/skip/*.identcache +/lib/delphi/test/skip/*.dproj +/lib/delphi/test/skip/*.dproj +/lib/delphi/test/skip/*.2007 +/lib/delphi/test/serializer/*.identcache +/lib/delphi/test/serializer/*.dproj +/lib/delphi/test/serializer/*.local +/lib/delphi/test/serializer/*.2007 +/lib/delphi/test/serializer/*.dcu +/lib/delphi/test/multiplexed/*.dproj +/lib/delphi/test/multiplexed/*.2007 +/lib/delphi/test/multiplexed/*.local +/lib/delphi/test/multiplexed/*.identcache +/lib/delphi/test/multiplexed/*.dcu +/lib/delphi/test/typeregistry/*.2007 +/lib/delphi/test/typeregistry/*.dproj +/lib/delphi/test/typeregistry/*.identcache +/lib/delphi/test/typeregistry/*.local +/lib/delphi/test/typeregistry/*.dcu +/lib/erl/.generated +/lib/erl/.eunit +/lib/erl/ebin +/lib/erl/deps/ +/lib/erl/src/thrift.app.src +/lib/erl/test/*.hrl +/lib/erl/test/*.beam +/lib/haxe/test/bin +/lib/hs/dist +/lib/java/build +/lib/js/test/build +/lib/nodejs/coverage +/lib/nodejs/node_modules/ +/lib/perl/MANIFEST +/lib/perl/MYMETA.json +/lib/perl/MYMETA.yml +/lib/perl/Makefile-perl.mk +/lib/perl/blib +/lib/perl/pm_to_blib +/lib/py/build +/lib/py/thrift.egg-info/ +/lib/rb/Gemfile.lock +/lib/rb/debug_proto_test +/lib/rb/.config +/lib/rb/ext/conftest.dSYM/ +/lib/rb/ext/mkmf.log +/lib/rb/ext/thrift_native.bundle +/lib/rb/ext/thrift_native.so +/lib/rb/test/ +/lib/rb/thrift-*.gem +/lib/php/src/ext/thrift_protocol/Makefile.* +/lib/php/src/ext/thrift_protocol/build/ +/lib/php/src/ext/thrift_protocol/config.* +/lib/php/src/ext/thrift_protocol/configure +/lib/php/src/ext/thrift_protocol/configure.in +/lib/php/src/ext/thrift_protocol/install-sh +/lib/php/src/ext/thrift_protocol/libtool +/lib/php/src/ext/thrift_protocol/ltmain.sh +/lib/php/src/ext/thrift_protocol/missing +/lib/php/src/ext/thrift_protocol/mkinstalldirs +/lib/php/src/ext/thrift_protocol/modules/ +/lib/php/src/ext/thrift_protocol/php_thrift_protocol.lo +/lib/php/src/ext/thrift_protocol/run-tests.php +/lib/php/src/ext/thrift_protocol/thrift_protocol.la +/lib/php/src/ext/thrift_protocol/tmp-php.ini +/lib/php/src/packages/ +/lib/php/test/TEST-*.xml +/lib/php/test/packages/ +/lib/py/dist/ +/lib/erl/logs/ +/lib/go/test/gopath/ +/lib/go/test/ThriftTest.thrift +/libtool +/ltmain.sh +/missing +/node_modules/ +/stamp-h1 +/test/features/results.json +/test/results.json +/test/c_glib/test_client +/test/c_glib/test_server +/test/cpp/StressTest +/test/cpp/StressTestNonBlocking +/test/cpp/TestClient +/test/cpp/TestServer +/test/dart/**/.packages +/test/dart/**/packages +/test/dart/**/.pub/ +/test/dart/**/pubspec.lock +/test/log/ +/test/test.log +/test/erl/.generated +/test/erl/ebin +/test/go/bin/ +/test/go/ThriftTest.thrift +/test/go/gopath +/test/go/pkg/ +/test/go/src/code.google.com/ +/test/go/src/github.com/golang/ +/test/go/src/gen/ +/test/go/src/thrift +/test/haxe/bin +/test/hs/TestClient +/test/hs/TestServer +/test/py.twisted/_trial_temp/ +/test/rb/Gemfile.lock +/tutorial/cpp/TutorialClient +/tutorial/cpp/TutorialServer +/tutorial/c_glib/tutorial_client +/tutorial/c_glib/tutorial_server +/tutorial/csharp/CsharpServer/obj +/tutorial/csharp/CsharpServer/bin +/tutorial/csharp/CsharpClient/obj +/tutorial/csharp/CsharpClient/bin +/tutorial/d/async_client +/tutorial/d/client +/tutorial/d/server +/tutorial/dart/**/.packages +/tutorial/dart/**/packages +/tutorial/dart/**/.pub/ +/tutorial/dart/**/pubspec.lock +/tutorial/delphi/*.dsk +/tutorial/delphi/*.local +/tutorial/delphi/*.tvsconfig +/tutorial/delphi/DelphiClient/dcu +/tutorial/delphi/DelphiServer/dcu +/tutorial/delphi/DelphiClient/*.local +/tutorial/delphi/DelphiClient/*.identcache +/tutorial/delphi/DelphiServer/*.identcache +/tutorial/delphi/DelphiServer/*.local +/tutorial/go/go-tutorial +/tutorial/go/calculator-remote +/tutorial/go/src/shared +/tutorial/go/src/tutorial +/tutorial/go/src/git.apache.org +/tutorial/haxe/bin +/tutorial/hs/dist/ +/tutorial/java/build/ +/tutorial/js/build/ +/ylwrap diff --git a/vendor/github.com/apache/thrift/.travis.yml b/vendor/github.com/apache/thrift/.travis.yml new file mode 100644 index 0000000000000..81a88c3ce0da7 --- /dev/null +++ b/vendor/github.com/apache/thrift/.travis.yml @@ -0,0 +1,199 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +# build Apache Thrift on Travis CI - https://travis-ci.org/ + +sudo: required +dist: trusty + +services: + - docker + +install: + - (travis_wait ./build/docker/check_unmodified.sh $DISTRO && touch .unmodified) || true + - if [ ! -f .unmodified ]; then travis_retry travis_wait docker build -q -t thrift-build:$DISTRO build/docker/$DISTRO; fi + +script: + - docker run --net=host -e BUILD_LIBS="$BUILD_LIBS" $BUILD_ENV -v $(pwd):/thrift/src -it thrift-build:$DISTRO build/docker/scripts/$SCRIPT $BUILD_ARG + +env: + global: + - TEST_NAME="" + - SCRIPT="cmake.sh" + - BUILD_ARG="" + - BUILD_ENV="-e CC=clang -e CXX=clang++" + - DISTRO=ubuntu + - BUILD_LIBS="CPP C_GLIB HASKELL JAVA PYTHON TESTING TUTORIALS" # only meaningful for CMake builds + + matrix: + - TEST_NAME="Cross Language Tests (Binary and Header Protocols)" + SCRIPT="cross-test.sh" + BUILD_ARG="-'(binary|header)'" + BUILD_ENV="-e CC=clang -e CXX=clang++ -e THRIFT_CROSSTEST_CONCURRENCY=4" + + - TEST_NAME="Cross Language Tests (Debian) (Binary and Header Protocols)" + SCRIPT="cross-test.sh" + BUILD_ARG="-'(binary|header)'" + BUILD_ENV="-e CC=clang -e CXX=clang++ -e THRIFT_CROSSTEST_CONCURRENCY=4" + DISTRO=debian + + - TEST_NAME="Cross Language Tests (Compact and JSON Protocols)" + SCRIPT="cross-test.sh" + BUILD_ARG="-'(compact|json)'" + BUILD_ENV="-e CC=clang -e CXX=clang++ -e THRIFT_CROSSTEST_CONCURRENCY=4" + + - TEST_NAME="Cross Language Tests (Debian) (Compact and JSON Protocols)" + SCRIPT="cross-test.sh" + BUILD_ARG="-'(compact|json)'" + BUILD_ENV="-e CC=clang -e CXX=clang++ -e THRIFT_CROSSTEST_CONCURRENCY=4" + DISTRO=debian + + # TODO: Remove them once migrated to CMake + # Autotools builds + - TEST_NAME="C C++ C# D Erlang Haxe Go (automake)" + SCRIPT="autotools.sh" + BUILD_ARG="--without-dart --without-haskell --without-java --without-lua --without-nodejs --without-perl --without-php --without-php_extension --without-python --without-ruby" + + - TEST_NAME="C C++ - GCC (automake)" + SCRIPT="autotools.sh" + BUILD_ARG="--without-csharp --without-java --without-erlang --without-nodejs --without-lua --without-python --without-perl --without-php --without-php_extension --without-dart --without-ruby --without-haskell --without-go --without-haxe --without-d" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="Java Lua PHP Ruby Dart (automake)" + SCRIPT="autotools.sh" + BUILD_ARG="--without-cpp --without-haskell --without-c_glib --without-csharp --without-d --without-erlang --without-go --without-haxe --without-nodejs --without-python --without-perl" + + # These are flaky (due to cabal and npm network/server failures) and also have lengthy output + - TEST_NAME="Haskell Node.js Python Perl (automake)" + SCRIPT="autotools.sh" + BUILD_ARG="--without-cpp --without-c_glib --without-csharp --without-d --without-dart --without-erlang --without-go --without-haxe --without-java --without-lua --without-php --without-php_extension --without-ruby" + + # CMake build + - TEST_NAME="All" + + - TEST_NAME="All (Debian)" + DISTRO=debian + + - TEST_NAME="C C++ - GCC" + BUILD_LIBS="CPP C_GLIB TESTING TUTORIALS" + BUILD_ARG="-DWITH_PYTHON=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="C++ (Boost Thread)" + BUILD_LIBS="CPP TESTING TUTORIALS" + BUILD_ARG="-DWITH_BOOSTTHREADS=ON -DWITH_PYTHON=OFF -DWITH_C_GLIB=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + + - TEST_NAME="C++ (Boost Thread - GCC)" + BUILD_LIBS="CPP TESTING TUTORIALS" + BUILD_ARG="-DWITH_BOOSTTHREADS=ON -DWITH_PYTHON=OFF -DWITH_C_GLIB=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="C++ (Std Thread)" + BUILD_LIBS="CPP TESTING TUTORIALS" + BUILD_ARG="-DWITH_STDTHREADS=ON -DCMAKE_CXX_FLAGS='-std=c++11' -DWITH_PYTHON=OFF -DWITH_C_GLIB=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + + - TEST_NAME="C++ (Std Thread - GCC)" + BUILD_LIBS="CPP TESTING TUTORIALS" + BUILD_ARG="-DWITH_STDTHREADS=ON -DCMAKE_CXX_FLAGS='-std=c++11' -DWITH_PYTHON=OFF -DWITH_C_GLIB=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="Compiler (mingw)" + BUILD_LIBS="" + BUILD_ARG="-DCMAKE_TOOLCHAIN_FILE=../build/cmake/mingw32-toolchain.cmake -DBUILD_COMPILER=ON -DBUILD_LIBRARIES=OFF -DBUILD_TESTING=OFF -DBUILD_EXAMPLES=OFF" + BUILD_ENV="" + + - TEST_NAME="All - GCC (CentOS)" + BUILD_ENV="-e CC=gcc -e CXX=g++" + DISTRO=centos + + - TEST_NAME="C C++ - Clang (CentOS)" + BUILD_LIBS="CPP C_GLIB TESTING TUTORIALS" + BUILD_ARG="-DWITH_PYTHON=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + DISTRO=centos + + - TEST_NAME="Python 2.6 (CentOS 6)" + BUILD_LIBS="PYTHON TESTING TUTORIALS" + BUILD_ARG="-DWITH_PYTHON=ON -DWITH_CPP=OFF -DWITH_JAVA=OFF -DWITH_HASKELL=OFF" + BUILD_ENV="-e CC=gcc -e CXX=g++" + DISTRO=centos6 + + # Distribution + - TEST_NAME="make dist" + SCRIPT="make-dist.sh" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="Debian Packages" + SCRIPT="dpkg.sh" + BUILD_ENV="-e CC=gcc -e CXX=g++" + + - TEST_NAME="make dist (Debian)" + SCRIPT="make-dist.sh" + BUILD_ENV="-e CC=gcc -e CXX=g++" + DISTRO=debian + + - TEST_NAME="Debian Packages (Debian)" + SCRIPT="dpkg.sh" + BUILD_ENV="-e CC=gcc -e CXX=g++" + DISTRO=debian + +matrix: + include: + # QA jobs for code analytics and metrics + # + # C/C++ static code analysis with cppcheck + # add --error-exitcode=1 to --enable=all as soon as everything is fixed + # + # Python code style check with flake8 + # + # search for TODO etc within source tree + # some statistics about the code base + # some info about the build machine + - env: TEST_NAME="cppcheck, flake8, TODO FIXME HACK, LoC and system info" + install: + - travis_retry sudo apt-get update + - travis_retry sudo apt-get install -ym cppcheck sloccount python-flake8 + script: + # Compiler cppcheck (All) + - cppcheck --force --quiet --inline-suppr --enable=all -j2 compiler/cpp/src + # C++ cppcheck (All) + - cppcheck --force --quiet --inline-suppr --enable=all -j2 lib/cpp/src lib/cpp/test test/cpp tutorial/cpp + # C Glib cppcheck (All) + - cppcheck --force --quiet --inline-suppr --enable=all -j2 lib/c_glib/src lib/c_glib/test test/c_glib/src tutorial/c_glib + # Silent error checks + - cppcheck --force --quiet --inline-suppr --error-exitcode=1 -j2 compiler/cpp/src + - cppcheck --force --quiet --inline-suppr --error-exitcode=1 -j2 lib/cpp/src lib/cpp/test test/cpp tutorial/cpp + - cppcheck --force --quiet --inline-suppr --error-exitcode=1 -j2 lib/c_glib/src lib/c_glib/test test/c_glib/src tutorial/c_glib + # Python code style + - flake8 --ignore=E501 lib/py + - flake8 tutorial/py + - flake8 --ignore=E501 test/py + - flake8 test/py.twisted + - flake8 test/py.tornado + - flake8 --ignore=E501 test/test.py + - flake8 --ignore=E501 test/crossrunner + - flake8 test/features + # TODO etc + - grep -r TODO * + - grep -r FIXME * + - grep -r HACK * + # LoC + - sloccount . + # System Info + - dpkg -l + - uname -a diff --git a/vendor/github.com/apache/thrift/CHANGES b/vendor/github.com/apache/thrift/CHANGES new file mode 100644 index 0000000000000..7b674d6f65862 --- /dev/null +++ b/vendor/github.com/apache/thrift/CHANGES @@ -0,0 +1,2366 @@ +Apache Thrift Changelog + +Thrift 0.10.0 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1840] - Thrift Generated Code Causes Global Variable Leaks + * [THRIFT-1828] - moc_TQTcpServer.cpp was removed from source tree but is in thrift-0.9.0.tar.gz + * [THRIFT-1790] - cocoa: Duplicate interface definition error + * [THRIFT-1776] - TPipeServer should implement "listen", so that TServerEventHandler preServe will work right + * [THRIFT-1351] - Compiler does not care about binary strings + * [THRIFT-1229] - Python fastbinary.c can not handle unicode as generated python code + * [THRIFT-749] - C++ TBufferedTransports do not flush their buffers on delete + * [THRIFT-747] - C++ TSocket->close calls shutdown breaking forked parent process + * [THRIFT-732] - server exits abnormally when client calls send_xxx function without calling recv_xxx function + * [THRIFT-3942] - TSSLSocket does not honor send and receive timeouts + * [THRIFT-3941] - WinXP version of thrift_poll() relies on undefined behavior by passing a destructed variable to select() + * [THRIFT-3940] - Visual Studio project file for compiler is broken + * [THRIFT-3943] - Coverity Scan identified some high severity defects + * [THRIFT-3929] - PHP "nsglobal" Option Results in Syntax Error in Generated Code (Trailing Backslash) + * [THRIFT-3936] - Cannot compile 0.10.0 development tip with VS2013 and earlier (snprintf, uint32_t) + * [THRIFT-3935] - Incorrect skipping of map and set + * [THRIFT-3920] - Ruby: Ensuring that HTTP failures will clear the http transport outbuf var + * [THRIFT-3919] - C# TTLSServerSocket does not use clientTimeout + * [THRIFT-3917] - Check backports.ssl_match_hostname module version + * [THRIFT-3909] - Fix c_glib static lib CMake build + * [THRIFT-3904] - Typo in node tutorial leads to wrong transport being used + * [THRIFT-3848] - As an implementer of a perl socket server, I do not want to have to remember to ignore SIGCHLD for it to work properly + * [THRIFT-3844] - thrift_protocol cannot compile in 7.0.7 + * [THRIFT-3843] - integer issues with Haxe PHP targets cause ZigZag encoding to fail + * [THRIFT-3842] - Dart generates incorrect code for a const struct + * [THRIFT-3841] - dart compact protocol incorrectly serializes/deserialized doubles + * [THRIFT-3708] - NameError: global name 'TProtocol' is not defined + * [THRIFT-3704] - "TConnectedClient died: Could not refill buffer" message shown when using HTTP Server + * [THRIFT-3678] - Fix javadoc errors on JDK 8 + * [THRIFT-3014] - AppVeyor support + * [THRIFT-2994] - Node.js TJSONProtocol cannot be used for object serialization. + * [THRIFT-2974] - writeToParcel throws NPE for optional enum fields + * [THRIFT-2948] - Python TJSONProtocol doesn't handle structs with binary fields containing invalid unicode. + * [THRIFT-2845] - ChildService.Plo: No such file or directory + * [THRIFT-3276] - Binary data does not decode correctly using the TJSONProtocol when the base64 encoded data is padded. + * [THRIFT-3253] - Using latest version of D gives deprecation notices + * [THRIFT-2883] - TTwisted.py, during ConnectionLost processing: exceptions.RuntimeError: dictionary changed size during iteration + * [THRIFT-2019] - Writing on a disconnected socket on Mac causes SIG PIPE + * [THRIFT-2020] - Thrift library has some empty files that haven't really been deleted + * [THRIFT-2049] - Go compiler doesn't build on native Windows + * [THRIFT-2024] - TServer.cpp warns on 64-bit platforms about truncating an rlim_t into an int + * [THRIFT-2023] - gettimeofday implementation on Windows errors when no time zone is passed in. + * [THRIFT-2022] - CoB and dense code generation still uses TR1 bind, even though that doesn't work with clang + * [THRIFT-2027] - Minor 64-bit and NOMINMAX issues in C++ library + * [THRIFT-2156] - TServerSocket::listen() is throwing exceptions with misleading information + * [THRIFT-2154] - Missing #deepCopy should return T + * [THRIFT-3157] - TBase signature should be TBase, F extends TFieldIdEnum> + * [THRIFT-3156] - Node TLS: server executes processing logic two full times + * [THRIFT-3154] - tutorial/py.tornado throw EOF exception + * [THRIFT-3063] - C++ build -Wunused-parameter warnings on processor_test, TransportTest + * [THRIFT-3056] - Add string/collection length limits for Python protocol readers + * [THRIFT-3237] - Fix TNamedPipeServer::createNamedPipe memory leak + * [THRIFT-3233] - Fix C++ ThreadManager::Impl::removeWorker worker join + * [THRIFT-3232] - Cannot deserialize json messages created with fieldNamesAsString + * [THRIFT-3206] - Fix Visual Studio build failure due 'pthread_self': identifier not found + * [THRIFT-3200] - JS and nodejs do not encode JSON protocol binary fields as base64 + * [THRIFT-3199] - Exception field has basic metadata + * [THRIFT-3182] - TFramedTransport is in an invalid state after frame size exception + * [THRIFT-2536] - new TSocket, uninitialised value reported by valgrind + * [THRIFT-2527] - Apache Thrift IDL Compiler code generated for Node.js should be jshint clean + * [THRIFT-2519] - "processor" class is not being generated + * [THRIFT-2431] - TFileTransportTest fails with "check delta < XXX failed" + * [THRIFT-2708] - Erlang library does not support "oneway" message type + * [THRIFT-3377] - Deep copy is actually shallow when using typedef members + * [THRIFT-3376] - C# and Python JSON protocol double values lose precision + * [THRIFT-3373] - Various fixes for cross test servers and clients + * [THRIFT-3370] - errno extern variable redefined. Not compiling for Android + * [THRIFT-3379] - Potential out of range panic in Go JSON protocols + * [THRIFT-3371] - Abstract namespace Unix domain sockets broken in C++ + * [THRIFT-3380] - nodejs: 0.9.2 -> 0.9.3 upgrade breaks Protocol and Transport requires + * [THRIFT-3367] - Fix bad links to coding_standards.md #634 + * [THRIFT-3401] - Nested collections emit Objective-C code that cannot compile + * [THRIFT-3403] - JSON String reader doesn't recognize UTF-16 surrogate pairs + * [THRIFT-3362] - make check fails for C++ at the SecurityTest + * [THRIFT-3395] - Cocoa compiler produces corrupt code when boxing enums inside map. + * [THRIFT-3394] - compiler generates uncompilable code + * [THRIFT-3388] - hash doesn't work on set/list + * [THRIFT-3391] - Wrong bool formatting in test server + * [THRIFT-3390] - TTornado server doesn't handle closed connections properly + * [THRIFT-3382] - TBase class for C++ Library + * [THRIFT-3392] - Java TZlibTransport does not close its wrapper streams upon close() + * [THRIFT-3383] - i64 related warnings + * [THRIFT-3386] - misc. warnings with make check + * [THRIFT-3385] - warning: format ‘%lu’ expects ‘long unsigned int’, but has type ‘std::basic_string::size_type {aka unsigned int} + * [THRIFT-3355] - npm WARN package.json thrift@1.0.0-dev No license field. + * [THRIFT-3360] - Improve cross test servers and clients further + * [THRIFT-3359] - Binary field incompatibilities + * [THRIFT-3354] - Fix word-extraction substr bug in initialism code + * [THRIFT-3350] - Python JSON protocol does not encode binary as Base64 + * [THRIFT-3577] - assertion failed at line 512 of testcontainertest.c + * [THRIFT-3576] - Boost test --log_format arg does not accept lowercase + * [THRIFT-3575] - Go compiler tries to use unexported library methods when using read_write_private + * [THRIFT-3574] - Cocoa generator makes uncompilable imports + * [THRIFT-3570] - Remove duplicate instances that are added by upstream + * [THRIFT-3571] - Make feature test result browsable + * [THRIFT-3569] - c_glib protocols do not check number of bytes read by transport + * [THRIFT-3568] - THeader server crashes on readSlow + * [THRIFT-3567] - GLib-GObject-CRITICAL **: g_object_unref: assertion 'G_IS_OBJECT (object)' failed + * [THRIFT-3566] - C++/Qt: TQTcpServerTest::test_communicate() is never executed + * [THRIFT-3564] - C++/Qt: potential core dump in TQTcpServer in case an exception occurs in TAsyncProcessor::process() + * [THRIFT-3558] - typos in c_glib tests + * [THRIFT-3559] - Fix awkward extra semi-colons with Cocoa container literals + * [THRIFT-3555] - 'configure' script does not honor --with-openssl= for libcrypto for BN_init + * [THRIFT-3554] - Constant decls may lead to "Error: internal error: prepare_member_name_mapping() already active for different struct" + * [THRIFT-3552] - glib_c Memory Leak + * [THRIFT-3551] - Thrift perl library missing package declaration + * [THRIFT-3549] - Exceptions are not properly stringified in Perl library + * [THRIFT-3546] - NodeJS code should not be namespaced (and is currently not strict-mode compliant) + * [THRIFT-3545] - Container type literals do not compile + * [THRIFT-3538] - Remove UnboundMethodType in TProtocolDecorator + * [THRIFT-3536] - Error 'char' does not contain a definition for 'IsLowSurrogate' for WP7 target + * [THRIFT-3534] - Link error when building with Qt5 + * [THRIFT-3533] - Can not send nil pointer as service method argument + * [THRIFT-3507] - THttpClient does not use proxy from http_proxy, https_proxy environment variables + * [THRIFT-3502] - C++ TServerSocket passes small buffer to getsockname + * [THRIFT-3501] - Forward slash in comment causes compiler error + * [THRIFT-3498] - C++ library assumes optional function pthread_attr_setschedpolicy is available + * [THRIFT-3497] - Build fails with "invalid use of incomplete type" + * [THRIFT-3496] - C++: Cob style client fails when sending a consecutive request + * [THRIFT-3493] - libthrift does not compile on windows using visual studio + * [THRIFT-3488] - warning: unused variable 'program' + * [THRIFT-3489] - warning: deprecated conversion from string constant to 'char*' [-Wwrite-strings] + * [THRIFT-3487] - Full support for newer Delphi versions + * [THRIFT-3528] - Fix warnings in thrift.ll + * [THRIFT-3527] - -gen py:dynamic,utf8strings ignores utf8strings option + * [THRIFT-3526] - Code generated by py:utf8strings does not work for Python3 + * [THRIFT-3524] - dcc32 warning "W1000 Symbol 'IsLowSurrogate' is deprecated: 'Use TCharHelper'" in Thrift.Protocol.JSON.pas + * [THRIFT-3525] - py:dynamic fails to handle binary list/set/map element + * [THRIFT-3521] - TSimpleJSONProtocolTest is not deterministic (fails when run on JDK 8) + * [THRIFT-3520] - Dart TSocket onError stream should be typed as Object + * [THRIFT-3519] - fastbinary does not work with -gen py:utf8strings + * [THRIFT-3518] - TConcurrentClientSyncInfo files were missing for Visual Studio + * [THRIFT-3512] - c_glib: Build fails due to missing features.h + * [THRIFT-3483] - Incorrect empty binary handling introduced by THRIFT-3359 + * [THRIFT-3479] - Oneway calls should not return exceptions to clients + * [THRIFT-3478] - Restore dropped method to THsHaServer.java + * [THRIFT-3477] - Parser fails on enum item that starts with 'E' letter and continues with number + * [THRIFT-3476] - Missing include in ./src/thrift/protocol/TJSONProtocol.cpp + * [THRIFT-3474] - Docker: thrift-compiler + * [THRIFT-3473] - When "optional' is used with a struct member, C++ server seems to not return it correctly + * [THRIFT-3468] - Dart TSocketTransport onError handler is too restrictive + * [THRIFT-3451] - thrift_protocol PHP extension missing config.m4 file + * [THRIFT-3456] - rounding issue in static assert + * [THRIFT-3455] - struct write method's return value is incorrect + * [THRIFT-3454] - Python Tornado tutorial is broken + * [THRIFT-3463] - Java can't be disabled in CMake build + * [THRIFT-3450] - NPE when using SSL + * [THRIFT-3449] - TBaseAsyncProcessor fb.responseReady() never called for oneway functions + * [THRIFT-3471] - Dart generator does not handle uppercase argument names + * [THRIFT-3470] - Sporadic timeouts with pipes + * [THRIFT-3465] - Go Code With Complex Const Initializer Compilation Depends On Struct Order + * [THRIFT-3464] - Fix several defects in c_glib code generator + * [THRIFT-3462] - Cocoa generates Incorrect #import header names + * [THRIFT-3453] - remove rat_exclude + * [THRIFT-3418] - Use of ciphers in ssl.wrap_socket() breaks python 2.6 compatibility + * [THRIFT-3417] - "namespace xsd" is not really working + * [THRIFT-3413] - Thrift code generation bug in Go when extending service + * [THRIFT-3420] - C++: TSSLSockets are not interruptable + * [THRIFT-3415] - include unistd.h conditionally + * [THRIFT-3414] - #include in THeaderTransport.h breaks windows build + * [THRIFT-3411] - Go generates remotes with wrong package qualifiers when including + * [THRIFT-3430] - Go THttpClient does not read HTTP response body to completion when closing + * [THRIFT-3423] - First call to thrift_transport:read_exact fails to dispatch correct function + * [THRIFT-3422] - Go TServerSocket doesn't close on Interrupt + * [THRIFT-3421] - rebar as dependency instead of bundling (was: rebar fails if PWD contains Unicode) + * [THRIFT-3428] - Go test fails when running make check + * [THRIFT-3445] - Throwable messages are hidden from JVM stack trace output + * [THRIFT-3443] - Thrift include can generate uncompilable code + * [THRIFT-3444] - Large 64 bit Integer does not preserve value through Node.js JSONProtocol + * [THRIFT-3436] - misc. cross test issues with UTF-8 path names + * [THRIFT-3435] - Put generated Java code for fullcamel tests in a separate package/namespace + * [THRIFT-3433] - Doubles aren't interpreted correctly + * [THRIFT-3437] - Mingw-w64 build fail + * [THRIFT-3434] - Dart generator produces empty name in pubspec.yaml for includes without namespaces + * [THRIFT-3408] - JSON generator emits incorrect types + * [THRIFT-3406] - Cocoa client should not schedule streams on main runloop + * [THRIFT-3404] - JSON String reader doesn't recognize UTF-16 surrogate pair + * [THRIFT-3636] - Double precision is not fully preserved in C++ TJSONProtocol + * [THRIFT-3632] - c_glib testserialization fails with glib assertion + * [THRIFT-3619] - Using Thrift 0.9.3 with googletest on Linux gcc 4.9 / C++11 + * [THRIFT-3617] - CMake does not build gv/xml generators + * [THRIFT-3615] - Fix Python SSL client resource leak on connection failure + * [THRIFT-3616] - lib/py/test/test_sslsocket.py is flaky + * [THRIFT-3643] - Perl SSL server crushes if a client disconnect without handshake + * [THRIFT-3639] - C# Thrift library forces TLS 1.0, thwarting TLS 1.2 usage + * [THRIFT-3633] - Travis "C C++ - GCC" build was using clang + * [THRIFT-3634] - Fix Python TSocket resource leak on connection failure + * [THRIFT-3630] - Debian/Ubuntu install docs need an update + * [THRIFT-3629] - Parser sets exitcode on errors, but generator does not + * [THRIFT-3608] - lib/cpp/test/SecurityTest is flaky in jenkins Thrift-precommit build. + * [THRIFT-3601] - Better conformance to PEP8 for generated code + * [THRIFT-3599] - Validate client IP address against cert's SubjectAltName + * [THRIFT-3598] - TBufferedTransport doesn't instantiate client connection + * [THRIFT-3597] - `make check` hangs in go tests + * [THRIFT-3589] - Dart generator uses wrong name in constructor for uppercase arguments with defaults + * [THRIFT-3588] - Using TypeScript with --noImplicitAny fails + * [THRIFT-3584] - boolean false value cannot be transferred + * [THRIFT-3578] - Make THeaderTransport detect TCompact framed and unframed + * [THRIFT-3323] - Python library does not handle escaped forward slash ("/") in JSON + * [THRIFT-3322] - CMake generated "make check" failes on python_test + * [THRIFT-3321] - Thrift can't be added as a subdirectory of another CMake-based project + * [THRIFT-3314] - Dots in file names of includes causes dots in javascript variable names + * [THRIFT-3307] - Segfault in Ruby serializer + * [THRIFT-3309] - Missing TConstant.php in /lib/php/Makefile.am + * [THRIFT-3810] - unresolved external symbol public: virtual void __cdecl apache::thrift::server::TServerFramework::serve(void) + * [THRIFT-3736] - C++ library build fails if OpenSSL does not surrpot SSLv3 + * [THRIFT-3878] - Compile error in TSSLSocket.cpp with new OpenSSL [CRYPTO_num_locks] + * [THRIFT-3949] - missing make dist entry for compiler/cpp/test + * [THRIFT-449] - The wire format of the JSON Protocol may not always be valid JSON if it contains non-UTF8 encoded strings + * [THRIFT-162] - Thrift structures are unhashable, preventing them from being used as set elements + * [THRIFT-3961] - TConnectedClient does not terminate the connection to the client if an exception while processing the received message occures. + * [THRIFT-3881] - Travis CI builds are failing due to docker failures (three retries, and gives up) + * [THRIFT-3937] - Cannot compile 0.10.0 development tip with gcc-4.6.x + * [THRIFT-3964] - Unsupported mechanism type ????? due to dependency on default OS-dependent charset + * [THRIFT-3038] - Use of volatile in cpp library + * [THRIFT-3301] - Java generated code uses imports that can lead to class name collisions with IDL defined types + * [THRIFT-3348] - PHP TCompactProtocol bool&int64 readvalue bug + * [THRIFT-3955] - TThreadedServer Memory Leak + * [THRIFT-3829] - Thrift does not install Python Libraries if Twisted is not installed + * [THRIFT-3932] - C++ ThreadManager has a rare termination race + * [THRIFT-3828] - cmake fails when Boost_INCLUDE_DIRS (and other variables passed to include_directories()) is empty + * [THRIFT-3958] - CMake WITH_MT option for windows static runtime linking does not support the cmake build type RelWithDebInfo + * [THRIFT-3957] - TConnectedClient does not disconnect from clients when their timeout is reached. + * [THRIFT-3953] - TSSLSocket::close should handle exceptions from waitForEvent because it is called by the destructor. + * [THRIFT-3977] - PHP extension creates undefined values when deserializing sets + * [THRIFT-3947] - sockaddr type isn't always large enough for the return of getsockname + * [THRIFT-2755] - ThreadSanitizer reports data race in ThreadManager::Impl::addWorker + * [THRIFT-3948] - errno is not the correct method of getting the error in windows + * [THRIFT-4008] - broken ci due to upstream dependency versioning break + * [THRIFT-3999] - Fix Debian & Ubuntu package dependencies + * [THRIFT-3886] - PHP cross test client returns 0 even when failing + * [THRIFT-3997] - building thrift libs does not support new openssl + +## Documentation + * [THRIFT-3867] - Specify BinaryProtocol and CompactProtocol + +## Epic + * [THRIFT-3049] - As an iOS developer, I want a generator and library that produces Swift code + * [THRIFT-2336] - UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + +## Improvement + * [THRIFT-1867] - Python client/server should support client-side certificates. + * [THRIFT-1313] - c_glib compact support + * [THRIFT-1385] - make install doesn't install java library in the setted folder + * [THRIFT-1437] - Update RPM spec + * [THRIFT-847] - Test Framework harmonization across all languages + * [THRIFT-819] - add Enumeration for protocol, transport and server types + * [THRIFT-3927] - Emit an error instead of throw an error in the async callback + * [THRIFT-3931] - TSimpleServer: If process request encounter UNKNOWN_METHOD, don't close transport. + * [THRIFT-3934] - Automatically resolve OpenSSL binary version on Windows CI + * [THRIFT-3918] - Run subset of make cross + * [THRIFT-3908] - Remove redundant dependencies from Dockerfile + * [THRIFT-3907] - Skip Docker image build on CI when unchanged + * [THRIFT-3868] - Java struct equals should do identity check before field comparison + * [THRIFT-3849] - Port Go serializer and deserializer to dart + * [THRIFT-2989] - Complete CMake build for Apache Thrift + * [THRIFT-2980] - ThriftMemoryBuffer doesn't have a constructor option to take an existing buffer + * [THRIFT-2856] - refactor erlang basic transports and unify interfaces + * [THRIFT-2877] - Optimize generated hashCode + * [THRIFT-2869] - JSON: run schema validation from tests + * [THRIFT-3112] - [Java] AsyncMethodCallback should be typed in generated AsyncIface + * [THRIFT-3263] - PHP jsonSerialize() should cast scalar types + * [THRIFT-2905] - Cocoa compiler should have option to produce "modern" Objective-C + * [THRIFT-2821] - Enable the use of custom HTTP-Header in the Transport + * [THRIFT-2093] - added the ability to set compression level in C++ zlib transport + * [THRIFT-2089] - Compiler ignores duplicate typenames + * [THRIFT-2056] - Moved all #include config.h statements to #include + * [THRIFT-2031] - Make SO_KEEPALIVE configurable for C++ lib + * [THRIFT-2021] - Improve large binary protocol string performance + * [THRIFT-2028] - Cleanup threading headers / libraries + * [THRIFT-2014] - Change C++ lib includes to use style throughout + * [THRIFT-2312] - travis.yml: build everything + * [THRIFT-1915] - Multiplexing Services + * [THRIFT-1736] - Visual Studio top level project files within msvc + * [THRIFT-1735] - integrate tutorial into regular build + * [THRIFT-1533] - Make TTransport should be Closeable + * [THRIFT-35] - Move language tests into their appropriate library directory + * [THRIFT-1079] - Support i64 in AS3 + * [THRIFT-1108] - SSL support for the Ruby library + * [THRIFT-3856] - update debian package deependencies + * [THRIFT-3833] - haxe http server implementation (by embeding into php web server) + * [THRIFT-3839] - Performance issue with big message deserialization using php extension + * [THRIFT-3820] - Erlang: Detect OTP >= 18 to use new time correction + * [THRIFT-3816] - Reduce docker build duration on Travis-CI + * [THRIFT-3815] - Put appveyor dependency versions to one place + * [THRIFT-3788] - Compatibility improvements and Win64 support + * [THRIFT-3792] - Timeouts for anonymous pipes should be configurable + * [THRIFT-3794] - Split Delphi application, protocol and transport exception subtypes into separate exceptions + * [THRIFT-3774] - The generated code should have exception_names meta info + * [THRIFT-3762] - Fix build warnings for deprecated Thrift "byte" fields + * [THRIFT-3756] - Improve requiredness documentation + * [THRIFT-3761] - Add debian package for Python3 + * [THRIFT-3742] - haxe php cli support + * [THRIFT-3733] - Socket timeout improvements + * [THRIFT-3728] - http transport for thrift-lua + * [THRIFT-3905] - Dart compiler does not initialize bool, int, and double properties + * [THRIFT-3911] - Loosen Ruby dev dependency version requirements + * [THRIFT-3906] - Run C# tests with make check + * [THRIFT-3900] - Add Python SSL flags + * [THRIFT-3897] - Provide meaningful exception type based on WebExceptionStatus in case of timeout + * [THRIFT-3808] - Missing `DOUBLE` in thrift type enumeration + * [THRIFT-3803] - Remove "file" attribute from XML generator + * [THRIFT-3660] - Add V4 mapped address to test client cert's altname + * [THRIFT-3661] - Use https to download meck in erlang test build + * [THRIFT-3659] - Check configure result of CMake on CI + * [THRIFT-3667] - Add TLS SNI support to clients + * [THRIFT-3651] - Make backports.match_hostname and ipaddress optional + * [THRIFT-3666] - Build D tutorial as part of Autotools build + * [THRIFT-3665] - Add D libevent and OpenSSL to docker images + * [THRIFT-3664] - Remove md5.c + * [THRIFT-3662] - Add Haskell to debian docker image + * [THRIFT-3711] - Add D to cross language test + * [THRIFT-3691] - Run flake8 Python style check on Travis-CI + * [THRIFT-3692] - (Re)enable Appveyor C++ and Python build + * [THRIFT-3677] - Improve CMake Java build + * [THRIFT-3679] - Add stdout log to testBinary in Java test server + * [THRIFT-3718] - Reduce size of docker image for build environment + * [THRIFT-3698] - [Travis-CI] Introduce retry to apt commands + * [THRIFT-3127] - switch -recurse to --recurse and reserve -r + * [THRIFT-3087] - Pass on errors like "connection closed" + * [THRIFT-3240] - Thrift Python client should support subjectAltName and wildcard certs in TSSLSocket + * [THRIFT-3213] - make cross should indicate when it skips a known failing test + * [THRIFT-3208] - Fix Visual Studio solution build failure due to missing source + * [THRIFT-3186] - Add TServerHTTP to Go library + * [THRIFT-2342] - Add __FILE__ and __LINE__ to Thrift C++ excpetions + * [THRIFT-3372] - Add dart generator to Visual Studio project + * [THRIFT-3366] - ThriftTest to implement standard return values + * [THRIFT-3402] - Provide a perl Unix Socket implementation + * [THRIFT-3361] - Improve C# library + * [THRIFT-3393] - Introduce i8 to provide consistent set of Thrift IDL integer types + * [THRIFT-3339] - Support for database/sql + * [THRIFT-3565] - C++: T[Async]Processor::getEventHandler() should be declared as const member functions + * [THRIFT-3563] - C++/Qt: removed usage of macro QT_PREPEND_NAMESPACE as it isn't consequently used for all references to Qt types. + * [THRIFT-3562] - Removed unused TAsyncProcessor::getAsyncServer() + * [THRIFT-3561] - C++/Qt: make use of Q_DISABLE_COPY() to get rid of copy ctor and assignment operator + * [THRIFT-3556] - c_glib file descriptor transport + * [THRIFT-3544] - Make cross test fail when server process died unexpectedly + * [THRIFT-3540] - Make python tutorial more in line with PEP8 + * [THRIFT-3535] - Dart generator argument to produce a file structure usable in parent library + * [THRIFT-3505] - Enhance Python TSSLSocket + * [THRIFT-3506] - Eliminate old style classes from library code + * [THRIFT-3503] - Enable py:utf8string by default + * [THRIFT-3499] - Add package_prefix to python generator + * [THRIFT-3495] - Minor enhancements and fixes for cross test + * [THRIFT-3486] - Java generated `getFieldValue` is incompatible with `setFieldValue` for binary values. + * [THRIFT-3484] - Consolidate temporary buffers in Java's TCompactProtocol + * [THRIFT-3516] - Add feature test for THeader TBinaryProtocol interop + * [THRIFT-3515] - Python 2.6 compatibility and test on CI + * [THRIFT-3514] - PHP 7 compatible version of binary protocol + * [THRIFT-3469] - Docker: Debian support + * [THRIFT-3416] - Retire old "xxx_namespace" declarations from the IDL + * [THRIFT-3426] - Align autogen comment in XSD + * [THRIFT-3424] - Add CMake android build option + * [THRIFT-3439] - Run make cross using Python3 when available + * [THRIFT-3440] - Python make check takes too much time + * [THRIFT-3441] - Stabilize Travis-CI builds + * [THRIFT-3431] - Avoid "schemes" HashMap lookups during struct reads/writes + * [THRIFT-3432] - Add a TByteBuffer transport to the Java library + * [THRIFT-3438] - Enable py:new_style by default + * [THRIFT-3405] - Go THttpClient misuses http.Client objects + * [THRIFT-3614] - Improve logging of test_sslsocket.py + * [THRIFT-3647] - Fix php extension build warnings + * [THRIFT-3642] - Speed up cross test runner + * [THRIFT-3637] - Implement compact protocol for dart + * [THRIFT-3613] - Port Python C extension to Python 3 + * [THRIFT-3612] - Add Python C extension for compact protocol + * [THRIFT-3611] - Add --regex filter to cross test runner + * [THRIFT-3631] - JSON protocol implementation for Lua + * [THRIFT-3609] - Remove or replace TestPortFixture.h + * [THRIFT-3605] - Have the compiler complain about invalid arguments and options + * [THRIFT-3596] - Better conformance to PEP8 + * [THRIFT-3585] - Compact protocol implementation for Lua + * [THRIFT-3582] - Erlang libraries should have service metadata + * [THRIFT-3579] - Introduce retry to make cross + * [THRIFT-3306] - Java: TBinaryProtocol: Use 1 temp buffer instead of allocating 8 + * [THRIFT-3910] - Do not invoke pip as part of build process + * [THRIFT-1857] - Python 3.X Support + * [THRIFT-1944] - Binding to zero port + * [THRIFT-3954] - Enable the usage of structs called "Object" in Java + * [THRIFT-3981] - Enable analyzer strong mode in Dart library + * [THRIFT-3998] - Document ability to add custom tags to thrift structs + * [THRIFT-4006] - Add a removeEventListener method on TSocket + +## New Feature + * [THRIFT-640] - Support deprecation + * [THRIFT-948] - SSL socket support for PHP + * [THRIFT-764] - add Support for Vala language + * [THRIFT-3046] - Allow PSR4 class loading for generated classes (PHP) + * [THRIFT-2113] - Erlang SSL Socket Support + * [THRIFT-1482] - Unix domain socket support under PHP + * [THRIFT-519] - Support collections of types without having to explicitly define it + * [THRIFT-468] - Rack Middleware Application for Rails + * [THRIFT-1708] - Add event handlers for processor events + * [THRIFT-3834] - Erlang namespacing and exception metadata + * [THRIFT-2510] - Implement TNonblockingServer's ability to listen on unix domain sockets + * [THRIFT-3397] - Implement TProcessorFactory in C# to enable per-client processors + * [THRIFT-3523] - XML Generator + * [THRIFT-3510] - Add HttpTaskAsyncHandler implementation + * [THRIFT-3318] - PHP: SimpleJSONProtocol Implementation + * [THRIFT-3299] - Dart language bindings in Thrift + * [THRIFT-2835] - Add possibility to distribute generators separately from thrift core, and load them dynamically + * [THRIFT-184] - Add OSGi Manifest headers to the libthrift java library to be able to use Thrift in the OSGi runtime + * [THRIFT-141] - If a required field is not present on serialization, throw an exception + * [THRIFT-1891] - Add Windows ALPC transport which is right counterpart of Unix domain sockets + +## Question + * [THRIFT-1808] - The Thrift struct should be considered self-contained? + * [THRIFT-2895] - Tutorial cpp + * [THRIFT-3860] - Elephant-bird application Test fails for Thrift + * [THRIFT-3811] - HTTPS Support for C++ applications + * [THRIFT-3509] - "make check" error + +## Story + * [THRIFT-3452] - .travis.yml: Migrating from legacy to container-based infrastructure + +## Sub-task + * [THRIFT-1811] - ruby tutorial as part of the regular build + * [THRIFT-2779] - PHP TJSONProtocol encode unicode into UCS-4LE which can't be parsed by other language bindings + * [THRIFT-2110] - Erlang: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-3852] - A Travis-CI job fails with "write error" + * [THRIFT-3740] - Fix haxelib.json classpath + * [THRIFT-3653] - incorrect union serialization + * [THRIFT-3652] - incorrect serialization of optionals + * [THRIFT-3655] - incorrect union serialization + * [THRIFT-3654] - incorrect serialization of optionals + * [THRIFT-3656] - incorrect serialization of optionals + * [THRIFT-3699] - Fix integer limit symbol includes in Python C extension + * [THRIFT-3693] - Fix include issue in C++ TSSLSocketInterruptTest on Windows + * [THRIFT-3694] - [Windows] Disable tests of a few servers that are not supported + * [THRIFT-3696] - Install pip to CentOS Docker images to fix Python builds + * [THRIFT-3638] - Fix haxelib.json + * [THRIFT-3251] - Add http transport for server to Go lib + * [THRIFT-2424] - Recursive Types + * [THRIFT-2423] - THeader + * [THRIFT-2413] - Python: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2409] - Java: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2412] - D: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2411] - C++: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2410] - JavaMe: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2668] - TestSuite: detailed result on passed tests by feature + * [THRIFT-2659] - python Test Server fails when throwing TException + * [THRIFT-3398] - Add CMake build for Haskell library and tests + * [THRIFT-3396] - DART: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-3364] - Fix ruby binary field encoding in TJSONProtocol + * [THRIFT-3381] - Fix for misc. codegen issues with THRIFT-2905 + * [THRIFT-3573] - No rule to make target `../../../test/c_glib/src/.deps/testthrifttest-thrift_test_handler.Po'. + * [THRIFT-3572] - "Unable to determine the behavior of a signed right shift" + * [THRIFT-3542] - Add length limit support to Java test server + * [THRIFT-3537] - Remove the (now obsolete) csharp:asyncctp flag + * [THRIFT-3532] - Add configurable string and container read size limit to Python protocols + * [THRIFT-3531] - Create cross lang feature test for string and container read length limit + * [THRIFT-3482] - Haskell JSON protocol does not encode binary field as Base64 + * [THRIFT-3425] - Minor fixes + simplification for CentOS Dockerfile + * [THRIFT-3442] - Run CMake tests on Appveyor + * [THRIFT-3409] - NodeJS binary field issues + * [THRIFT-3621] - Fix lib/cpp/test/SecurityTest.cpp to use ephemeral ports + * [THRIFT-3628] - Fix lib/cpp/test/TServerIntegrationTest.cpp to use ephemeral ports + * [THRIFT-3625] - Kill unused #include "TestPortFixture.h" in lib/cpp/test/TServerTransportTest.cpp. + * [THRIFT-3646] - Fix Python extension build warnings + * [THRIFT-3626] - Fix lib/cpp/test/TSocketInterruptTest.cpp to use ephemeral ports. + * [THRIFT-3624] - Fix lib/cpp/test/TServerSocketTest.cpp to use ephemeral ports + * [THRIFT-3623] - Fix Fix cpp/lib/test/TSSLSocketInterruptTest.cpp to use ephemeral ports + * [THRIFT-3592] - Add basic test client + * [THRIFT-3980] - add TExtendedBinaryProtocol.java + +## Task + * [THRIFT-1801] - Sync up TApplicationException codes across languages and thrift implementations + * [THRIFT-1259] - Automate versioning + +## Test + * [THRIFT-3400] - Add Erlang to cross test + * [THRIFT-3504] - Fix FastbinaryTest.py + +## Wish + * [THRIFT-3923] - Maybe remove Aereo from the "Powered by" list + * [THRIFT-2149] - Add an option to disable the generation of default operators + + + +Thrift 0.9.3 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-2441] - Cannot shutdown TThreadedServer when clients are still connected + * [THRIFT-2465] - TBinaryProtocolT breaks if copied/moved + * [THRIFT-2474] - thrift.h causes a compile failure + * [THRIFT-2540] - Running configure from outside the source directory fails + * [THRIFT-2598] - Add check for minimum Go version to configure.ac + * [THRIFT-2647] - compiler-hs: don't decapitalize field names, do decapitalize argument bindings + * [THRIFT-2773] - Generated Java code for 'oneway' methods is incorrect. + * [THRIFT-2789] - TNonblockingServer leaks socket FD's under load + * [THRIFT-2682] - TThreadedServer leaks per-thread memory + * [THRIFT-2674] - JavaScript: declare Accept: and Content-Type: in request + * [THRIFT-3078] - TNonblockingServerSocket's logger is not named after TNonblockingServerSocket + * [THRIFT-3077] - C++ TFileTransport ignores return code from ftruncate + * [THRIFT-3067] - C++ cppcheck performance related warnings + * [THRIFT-3066] - C++ TDenseProtocol assert modifies instead of checks + * [THRIFT-3071] - bootstrap.sh on Ubuntu 12.04 (Precise) automake error + * [THRIFT-3069] - C++ TServerSocket leaks socket on fcntl get or set flags error + * [THRIFT-3079] - TNonblockingServerSocket's logger is not named after TNonblockingServerSocket + * [THRIFT-3080] - C++ TNonblockingServer connection leak while accept huge number connections. + * [THRIFT-3086] - C++ Valgrind Error Cleanup + * [THRIFT-3085] - thrift_reconnecting_client never try to reconnect + * [THRIFT-3123] - Missing include in compiler/cpp/src/main.h breaks build in some environments + * [THRIFT-3125] - Fix the list of exported headers in automake input + * [THRIFT-3126] - PHP JSON serializer converts empty or int-indexed maps to lists + * [THRIFT-3132] - Properly format date in Java @Generated annotations + * [THRIFT-3137] - Travis build hangs after failure + * [THRIFT-3138] - "make check" parallel execution is underministic + * [THRIFT-3139] - JS library test is flaky + * [THRIFT-3140] - ConcurrentModificationException is thrown by JavaScript test server + * [THRIFT-3124] - Some signed/unsigned warnings while building compiler + * [THRIFT-3128] - Go generated code produces name collisions between services + * [THRIFT-3146] - Graphviz generates function name collisions between services + * [THRIFT-3147] - Segfault while receiving data + * [THRIFT-3148] - Markdown links to coding_standards are dead + * [THRIFT-3090] - cmake build is broken on MacOSX + * [THRIFT-3097] - cmake targets unconditionally depend on optional libraries + * [THRIFT-3094] - master as of 2015-APR-13 fails -DBOOST_THREADS cmake build + * [THRIFT-3099] - cmake build is broken on FreeBSD + * [THRIFT-3089] - Assigning default ENUM values results in non-compilable java code if java namespace is not defined + * [THRIFT-3093] - mingw compile fixes for c++ library 0.9.2 + * [THRIFT-3098] - Thrift does not pretty print binary typedefs the way it does binary fields + * [THRIFT-3091] - c_glib service method should return result from handler method + * [THRIFT-3088] - TThreadPoolServer with Sasl auth may leak CLOSE_WAIT socket + * [THRIFT-3109] - Cross test log file cannot be browsed when served in HTTP server + * [THRIFT-3113] - m4 C++11 macro issue + * [THRIFT-3105] - C++ libthriftnb library on Windows build failure + * [THRIFT-3115] - Uncompileable code due to name collision with predefined used types + * [THRIFT-3117] - Java TSSLTransportFactory can't load certificates within JAR archive + * [THRIFT-3102] - could not make check for Go Library + * [THRIFT-3120] - Minor spelling errors and an outdated URL + * [THRIFT-3121] - Librt does not exist on OS X + * [THRIFT-3152] - Compiler error on Mac OSX (missing #include ) + * [THRIFT-3162] - make fails for dmd 2.067 + * [THRIFT-3164] - Thrift C++ library SSL socket by default allows for unsecure SSLv3 negotiation + * [THRIFT-3168] - Fix Maven POM + * [THRIFT-3170] - Initialism code in the Go compiler causes chaos + * [THRIFT-3169] - Do not export thrift.TestStruct and thrift.TestEnum in thrift Go library + * [THRIFT-3191] - Perl compiler does not add support for unexpected exception handling + * [THRIFT-3178] - glib C does not compile + * [THRIFT-3189] - Perl ServerSocket should allow a specific interface to be listened to + * [THRIFT-3252] - Missing TConcurrentClientSyncInfo.h in cpp Makefile, so doesn't install + * [THRIFT-3255] - Thrift generator doesn't exclude 'package' keyword for thrift property names breaking java builds + * [THRIFT-3260] - multiple warnings in c_glib tutorial + * [THRIFT-3256] - Some D test timings are too aggressive for slow machines + * [THRIFT-3257] - warning: extra tokens at end of #endif directive + * [THRIFT-3184] - Thrift Go leaves file descriptors open + * [THRIFT-3203] - DOAP - please fix "Ocaml" => "OCaml" + * [THRIFT-3210] - (uncompileable) code generated for server events while are events not enabled + * [THRIFT-3215] - TJSONProtocol '(c++) uses "throw new" to throw exceptions instead of "throw" + * [THRIFT-3202] - Allow HSHAServer to configure min and max worker threads separately. + * [THRIFT-3205] - TCompactProtocol return a wrong error when the io.EOF happens + * [THRIFT-3209] - LGPL mentioned in license file + * [THRIFT-3197] - keepAliveTime is hard coded as 60 sec in TThreadPoolServer + * [THRIFT-3196] - Misspelling in lua TBinaryProtocol (stirctWrite => strictWrite) + * [THRIFT-3198] - Allow construction of TTransportFactory with a specified maxLength + * [THRIFT-3192] - Go import paths changed in 1.4, and expired June 1 + * [THRIFT-3271] - Could not find or load main class configtest_ax_javac_and_java on some non-english systems + * [THRIFT-3273] - c_glib: Generated code tries to convert between function and void pointers + * [THRIFT-3264] - Fix Erlang 16 namespaced types + * [THRIFT-3270] - reusing TNonblockingServer::TConnection cause dirty TSocket + * [THRIFT-3267] - c_glib: "Critical" failure during unit tests + * [THRIFT-3277] - THttpClient leaks connections if it's used for multiple requests + * [THRIFT-3278] - NodeJS: Fix exception stack traces and names + * [THRIFT-3279] - Fix a bug in retry_max_delay (NodeJS) + * [THRIFT-3280] - Initialize retry variables on construction + * [THRIFT-3283] - c_glib: Tutorial server always exits with warning + * [THRIFT-3284] - c_glib: Empty service produces unused-variable warning + * [THRIFT-1925] - c_glib generated code does not compile + * [THRIFT-1849] - after transport->open() opens isOpen returns true and next open() goes thru when it shall not + * [THRIFT-1866] - java compiler generates non-compiling code with const's defined in a thrift when name includes non-identifier chars + * [THRIFT-1938] - FunctionRunner.h -- uses wrong path for Thread.h when installed + * [THRIFT-1844] - Password string not cleared + * [THRIFT-2004] - Thrift::Union violates :== method contract and crashes + * [THRIFT-2073] - Thrift C++ THttpClient error: cannot refill buffer + * [THRIFT-2127] - Autoconf scripting does not properly account for cross-compile + * [THRIFT-2180] - Integer types issues in Cocoa lib on ARM64 + * [THRIFT-2189] - Go needs "isset" to fully support "union" type (and optionals) + * [THRIFT-2192] - autotools on Redhat based systems + * [THRIFT-2546] - cross language tests fails at 'TestMultiException' when using nodejs server + * [THRIFT-2547] - nodejs servers and clients fails to connect with cpp using compact protocol + * [THRIFT-2548] - Nodejs servers and clients does not work properly with -ssl + * [THRIFT-1471] - toString() does not print ByteBuffer values when nested in a List + * [THRIFT-1201] - getaddrinfo resource leak + * [THRIFT-615] - TThreadPoolServer doesn't call task_done after pulling tasks from it's clients queue + * [THRIFT-162] - Thrift structures are unhashable, preventing them from being used as set elements + * [THRIFT-810] - Crashed client on TSocket::close under loads + * [THRIFT-557] - charset problem with file Autogenerated by Thrift + * [THRIFT-233] - IDL doesn't support negative hex literals + * [THRIFT-1649] - contrib/zeromq does not build in 0.8.0 + * [THRIFT-1642] - Miscalculation lead to throw unexpected "TTransportException::TIMED_OUT"(or called "EAGAIN (timed out)") exception + * [THRIFT-1587] - TSocket::setRecvTimeout error + * [THRIFT-1248] - pointer subtraction in TMemoryBuffer relies on undefined behavior + * [THRIFT-1774] - Sasl Transport client would hang when trying to connect non-sasl transport server + * [THRIFT-1754] - RangeError in buffer handling + * [THRIFT-1618] - static structMap in FieldMetaData is not thread safe and can lead to deadlocks + * [THRIFT-2335] - thrift incompatibility with py:tornado as server, java as client + * [THRIFT-2803] - TCP_DEFER_ACCEPT not supported with domain sockets + * [THRIFT-2799] - Build Problem(s): ld: library not found for -l:libboost_unit_test_framework.a + * [THRIFT-2801] - C++ test suite compilation warnings + * [THRIFT-2802] - C++ tutorial compilation warnings + * [THRIFT-2795] - thrift_binary_protocol.c: 'dereferencing type-punned pointer will break strict-aliasing rules' + * [THRIFT-2817] - TSimpleJSONProtocol reads beyond end of message + * [THRIFT-2826] - html:standalone sometimes ignored + * [THRIFT-2829] - Support haxelib installation via github + * [THRIFT-2828] - slightly wrong help screen indent + * [THRIFT-2831] - Removes dead code in web_server.js introduced in THRIFT-2819 + * [THRIFT-2823] - All JS-tests are failing when run with grunt test + * [THRIFT-2827] - Thrift 0.9.2 fails to compile on Yosemite due to tr1/functional include in ProcessorTest.cpp + * [THRIFT-2843] - Automake configure.ac has possible typo related to Java + * [THRIFT-2813] - multiple haxe library fixes/improvements + * [THRIFT-2825] - Supplying unicode to python Thrift client can cause next request arguments to get overwritten + * [THRIFT-2840] - Cabal file points to LICENSE file outside the path of the Haskell project. + * [THRIFT-2818] - Trailing commas in array + * [THRIFT-2830] - Clean up ant warnings in tutorial dir + * [THRIFT-2842] - Erlang thrift client has infinite timeout + * [THRIFT-2810] - Do not leave the underlying ServerSocket open if construction of TServerSocket fails + * [THRIFT-2812] - Go server adding redundant buffering layer + * [THRIFT-2839] - TFramedTransport read bug + * [THRIFT-2844] - Nodejs support broken when running under Browserify + * [THRIFT-2814] - args/result classes not found when no namespace is set + * [THRIFT-2847] - function IfValue() is a duplicate of System.StrUtils.IfThen + * [THRIFT-2848] - certain Delphi tests do not build if TypeRegistry is used + * [THRIFT-2854] - Go Struct writer and reader looses important error information + * [THRIFT-2858] - Enable header field case insensitive match in THttpServer + * [THRIFT-2857] - C# generator creates uncompilable code for struct constants + * [THRIFT-2860] - Delphi server closes connection on unexpected exceptions + * [THRIFT-2868] - Enhance error handling in the Go client + * [THRIFT-2879] - TMemoryBuffer: using lua string in wrong way + * [THRIFT-2851] - Remove strange public Peek() from Go transports + * [THRIFT-2852] - Better Open/IsOpen/Close behavior for StreamTransport. + * [THRIFT-2871] - Missing semicolon in thrift.js + * [THRIFT-2872] - ThreadManager deadlock for task expiration + * [THRIFT-2881] - Handle errors from Accept() correctly + * [THRIFT-2849] - Spell errors reported by codespell tool + * [THRIFT-2870] - C++ TJSONProtocol using locale dependent formatting + * [THRIFT-2882] - Lua Generator: using string.len funtion to get struct(map,list,set) size + * [THRIFT-2864] - JSON generator missing from Visual Studio build project + * [THRIFT-2878] - Go validation support of required fields + * [THRIFT-2873] - TPipe and TPipeServer don't compile on Windows with UNICODE enabled + * [THRIFT-2888] - import of is missing in JSON generator + * [THRIFT-2900] - Python THttpClient does not reset socket timeout on exception + * [THRIFT-2907] - 'ntohll' macro redefined + * [THRIFT-2884] - Map does not serialize correctly for JSON protocol in Go library + * [THRIFT-2887] - --with-openssl configure flag is ignored + * [THRIFT-2894] - PHP json serializer skips maps with int/bool keys + * [THRIFT-2904] - json_protocol_test.go fails + * [THRIFT-2906] - library not found for -l:libboost_unit_test_framework.a + * [THRIFT-2890] - binary data may lose bytes with JSON transport under specific circumstances + * [THRIFT-2891] - binary data may cause a failure with JSON transport under specific circumstances + * [THRIFT-2901] - Fix for generated TypeScript functions + indentation of JavaScript maps + * [THRIFT-2916] - make check fails for D language + * [THRIFT-2918] - Race condition in Python TProcessPoolServer test + * [THRIFT-2920] - Erlang Thrift test uses wrong IDL file + * [THRIFT-2922] - $TRIAL is used with Python tests but not tested accordingly + * [THRIFT-2912] - Autotool build for C++ Qt library is invalid + * [THRIFT-2914] - explicit dependency to Lua5.2 fails on some systems + * [THRIFT-2910] - libevent is not really optional + * [THRIFT-2911] - fix c++ version zeromq transport, the old version cannot work + * [THRIFT-2915] - Lua generator missing from Visual Studio build project + * [THRIFT-2917] - "make clean" breaks test/c_glib + * [THRIFT-2919] - Haxe test server timeout too large + * [THRIFT-2923] - JavaScript client assumes a message being written + * [THRIFT-2924] - TNonblockingServer crashes when user-provided event_base is used + * [THRIFT-2925] - CMake build does not work with OpenSSL nor anything installed in non-system location + * [THRIFT-2931] - Access to undeclared static property: Thrift\Protocol\TProtocol::$TBINARYPROTOCOLACCELERATED + * [THRIFT-2893] - CMake build fails with boost thread or std thread + * [THRIFT-2902] - Generated c_glib code does not compile with clang + * [THRIFT-2903] - Qt4 library built with CMake does not work + * [THRIFT-2942] - CSharp generate invalid code for property named read or write + * [THRIFT-2932] - Node.js Thrift connection libraries throw Exceptions into event emitter + * [THRIFT-2933] - v0.9.2: doubles encoded in node with compact protocol cannot be decoded by python + * [THRIFT-2934] - createServer signature mismatch + * [THRIFT-2981] - IDL with no namespace produces unparsable PHP + * [THRIFT-2999] - Addition of .gitattributes text auto in THRIFT-2724 causes modified files on checkout + * [THRIFT-2949] - typo in compiler/cpp/README.md + * [THRIFT-2957] - warning: source file %s is in a subdirectory, but option 'subdir-objects' is disabled + * [THRIFT-2953] - TNamedPipeServerTransport is not Stop()able + * [THRIFT-2962] - Docker Thrift env for development and testing + * [THRIFT-2971] - C++ test and tutorial parallel build is unstable + * [THRIFT-2972] - Missing backslash in lib/cpp/test/Makefile.am + * [THRIFT-2951] - Fix Erlang name conflict test + * [THRIFT-2955] - Using list of typedefs does not compile on Go + * [THRIFT-2960] - namespace regression for Ruby + * [THRIFT-2959] - nodejs: fix binary unit tests + * [THRIFT-2966] - nodejs: Fix bad references to TProtocolException and TProtocolExceptionType + * [THRIFT-2970] - grunt-jsdoc fails due to dependency issues + * [THRIFT-3001] - C# Equals fails for binary fields (byte[]) + * [THRIFT-3003] - Missing LICENSE file prevents package from being installed + * [THRIFT-3008] - Node.js server does not fully support exception + * [THRIFT-3007] - Travis build is broken because of directory conflict + * [THRIFT-3009] - TSSLSocket does not use the correct hostname (breaks certificate checks) + * [THRIFT-3011] - C# test server testException() not implemented according to specs + * [THRIFT-3012] - Timing problems in NamedPipe implementation due to unnecessary open/close + * [THRIFT-3019] - Golang generator missing docstring for structs + * [THRIFT-3021] - Service remote tool does not import stub package with package prefix + * [THRIFT-3026] - TMultiplexedProcessor does not have a constructor + * [THRIFT-3028] - Regression caused by THRIFT-2180 + * [THRIFT-3017] - order of map key/value types incorrect for one CTOR + * [THRIFT-3020] - Cannot compile thrift as C++03 + * [THRIFT-3024] - User-Agent "BattleNet" used in some Thrift library files + * [THRIFT-3047] - Uneven calls to indent_up and indent_down in Cocoa generator + * [THRIFT-3048] - NodeJS decoding of I64 is inconsistent across protocols + * [THRIFT-3043] - go compiler generator uses non C++98 code + * [THRIFT-3044] - Docker README.md paths to Dockerfiles are incorrect + * [THRIFT-3040] - bower.json wrong "main" path + * [THRIFT-3051] - Go Thrift generator creates bad go code + * [THRIFT-3057] - Java compiler build is broken + * [THRIFT-3061] - C++ TSSLSocket shutdown delay/vulnerability + * [THRIFT-3062] - C++ TServerSocket invalid port number (over 999999) causes stack corruption + * [THRIFT-3065] - Update libthrift dependencies (slf4j, httpcore, httpclient) + * [THRIFT-3244] - TypeScript: fix namespace of included types + * [THRIFT-3246] - Reduce the number of trivial warnings in Windows C++ CMake builds + * [THRIFT-3224] - Fix TNamedPipeServer unpredictable behavior on accept + * [THRIFT-3230] - Python compiler generates wrong code if there is function throwing a typedef of exception with another namespace + * [THRIFT-3236] - MaxSkipDepth never checked + * [THRIFT-3239] - Limit recursion depth + * [THRIFT-3241] - fatal error: runtime: cannot map pages in arena address space + * [THRIFT-3242] - OSGi Import-Package directive is missing the Apache HTTP packages + * [THRIFT-3234] - Limit recursion depth + * [THRIFT-3222] - TypeScript: Generated Enums are quoted + * [THRIFT-3229] - unexpected Timeout exception when desired bytes are only partially available + * [THRIFT-3231] - CPP: Limit recursion depth to 64 + * [THRIFT-3235] - Limit recursion depth + * [THRIFT-3175] - fastbinary.c python deserialize can cause huge allocations from garbage + * [THRIFT-3176] - Union incorrectly implements == + * [THRIFT-3177] - Fails to run rake test + * [THRIFT-3180] - lua plugin: framed transport do not work + * [THRIFT-3179] - lua plugin cant connect to remote server because function l_socket_create_and_connect always bind socket to localhost + * [THRIFT-3248] - TypeScript: additional comma in method signature without parameters + * [THRIFT-3302] - Go JSON protocol should encode Thrift byte type as signed integer string + * [THRIFT-3297] - c_glib: an abstract base class is not generated + * [THRIFT-3294] - TZlibTransport for Java does not write data correctly + * [THRIFT-3296] - Go cross test does not conform to spec + * [THRIFT-3295] - C# library does not build on Mono 4.0.2.5 or later + * [THRIFT-3293] - JavaScript: null values turn into empty structs in constructor + * [THRIFT-3310] - lib/erl/README.md has incorrect formatting + * [THRIFT-3319] - CSharp tutorial will not build using the *.sln + * [THRIFT-3335] - Ruby server does not handle processor exception + * [THRIFT-3338] - Stray underscore in generated go when service name starts with "New" + * [THRIFT-3324] - Update Go Docs for pulling all packages + * [THRIFT-3345] - Clients blocked indefinitely when a java.lang.Error is thrown + * [THRIFT-3332] - make dist fails on clean build + * [THRIFT-3326] - Tests do not compile under *BSD + * [THRIFT-3334] - Markdown notation of protocol spec is malformed + * [THRIFT-3331] - warning: ‘etype’ may be used uninitialized in this function + * [THRIFT-3349] - Python server does not handle processor exception + * [THRIFT-3343] - Fix haskell README + * [THRIFT-3340] - Python: enable json tests again + * [THRIFT-3311] - Top level README.md has incorrect formmating + * [THRIFT-2936] - Minor memory leak in SSL + * [THRIFT-3290] - Using from in variable names causes the generated Python code to have errors + * [THRIFT-3225] - Fix TPipeServer unpredictable behavior on interrupt() + * [THRIFT-3354] - Fix word-extraction substr bug in initialism code + * [THRIFT-2006] - TBinaryProtocol message header call name length is not validated and can be used to core the server + * [THRIFT-3329] - C++ library unit tests don't compile against the new boost-1.59 unit test framework + * [THRIFT-2630] - windows7 64bit pc. ipv4 and ipv6 pc.can't use + * [THRIFT-3336] - Thrift generated streaming operators added in 0.9.2 cannot be overridden + * [THRIFT-2681] - Core of unwind_cleanup + * [THRIFT-3317] - cpp namespace org.apache issue appears in 0.9 + +## Documentation + * [THRIFT-3286] - Apache Ant is a necessary dependency + +## Improvement + * [THRIFT-227] - Byte[] in collections aren't pretty printed like regular binary fields + * [THRIFT-2744] - Vagrantfile for Centos 6.5 + * [THRIFT-2644] - Haxe support + * [THRIFT-2756] - register Media Type @ IANA + * [THRIFT-3076] - Compatibility with Haxe 3.2.0 + * [THRIFT-3081] - C++ Consolidate client processing loops in TServers + * [THRIFT-3083] - C++ Consolidate server processing loops in TSimpleServer, TThreadedServer, TThreadPoolServer + * [THRIFT-3084] - C++ add concurrent client limit to threaded servers + * [THRIFT-3074] - Add compiler/cpp/lex.yythriftl.cc to gitignore. + * [THRIFT-3134] - Remove use of deprecated "phantom.args" + * [THRIFT-3133] - Allow "make cross" and "make precross" to run without building all languages + * [THRIFT-3142] - Make JavaScript use downloaded libraries + * [THRIFT-3141] - Improve logging of JavaScript test + * [THRIFT-3144] - Proposal: make String representation of enums in generated go code less verbose + * [THRIFT-3130] - Remove the last vestiges of THRIFT_OVERLOAD_IF from THRIFT-1316 + * [THRIFT-3131] - Consolidate suggested import path for go thrift library to git.apache.org in docs and code + * [THRIFT-3092] - Generated Haskell types should derive Generic + * [THRIFT-3110] - Print error log after cross test failures on Travis + * [THRIFT-3114] - Using local temp variables to not pollute the global table + * [THRIFT-3106] - CMake summary should give more information why a library is set to off + * [THRIFT-3119] - Java's TThreadedSelectorServer has indistinguishable log messages in run() + * [THRIFT-3122] - Javascript struct constructor should properly initialize struct and container members from plain js arguments + * [THRIFT-3151] - Fix links to git-wip* - should be git.apache.org + * [THRIFT-3167] - Windows build from source instructions need to be revised + * [THRIFT-3155] - move contrib/mingw32-toolchain.cmake to build/cmake/ + * [THRIFT-3160] - Make generated go enums implement TextMarshaller and TextUnmarshaller interfaces + * [THRIFT-3150] - Add an option to thrift go generator to make Read and Write methods private + * [THRIFT-3149] - Make ReadFieldN methods in generated Go code private + * [THRIFT-3172] - Add tutorial to Thrift web site + * [THRIFT-3214] - Add Erlang option for using maps instead of dicts + * [THRIFT-3201] - Capture github test artifacts for failed builds + * [THRIFT-3266] - c_glib: Multiple compiler warnings building unit tests + * [THRIFT-3285] - c_glib: Build library with all warnings enabled, no warnings generated + * [THRIFT-1954] - Allow for a separate connection timeout value + * [THRIFT-2098] - Add support for Qt5+ + * [THRIFT-2199] - Remove Dense protocol (was: move to Contrib) + * [THRIFT-406] - C++ Test suite cleanup + * [THRIFT-902] - socket and connect timeout in TSocket should be distinguished + * [THRIFT-388] - Use a separate wire format for async calls + * [THRIFT-727] - support native C++ language specific exception message + * [THRIFT-1784] - pep-3110 compliance for exception handling + * [THRIFT-1025] - C++ ServerSocket should inherit from Socket with the necessary Ctor to listen on connections from a specific host + * [THRIFT-2269] - Can deploy libthrift-source.jar to maven center repository + * [THRIFT-2804] - Pull an interface out of TBaseAsyncProcessor + * [THRIFT-2806] - more whitespace fixups + * [THRIFT-2811] - Make remote socket address accessible + * [THRIFT-2809] - .gitignore update for compiler's visual project + * [THRIFT-2846] - Expose ciphers parameter from ssl.wrap_socket() + * [THRIFT-2859] - JSON generator: output complete descriptors + * [THRIFT-2861] - add buffered transport + * [THRIFT-2865] - Test case for Go: SeqId out of sequence + * [THRIFT-2866] - Go generator source code is hard to read and maintain + * [THRIFT-2880] - Read the network address from the listener if available. + * [THRIFT-2875] - Typo in TDenseProtocol.h comment + * [THRIFT-2874] - TBinaryProtocol member variable "string_buf_" is never used. + * [THRIFT-2855] - Move contributing.md to the root of the repository + * [THRIFT-2862] - Enable RTTI and/or build macros for generated code + * [THRIFT-2876] - Add test for THRIFT-2526 Assignment operators and copy constructors in c++ don't copy the __isset struct + * [THRIFT-2897] - Generate -isEqual: and -hash methods + * [THRIFT-2909] - Improve travis build + * [THRIFT-2921] - Make Erlang impl ready for OTP 18 release (dict/0 and set/0 are deprecated) + * [THRIFT-2928] - Rename the erlang test_server module + * [THRIFT-2940] - Allow installing Thrift from git as NPM module by providing package.json in top level directory + * [THRIFT-2937] - Allow setting a maximum frame size in TFramedTransport + * [THRIFT-2976] - nodejs: xhr and websocket support for browserify + * [THRIFT-2996] - Test for Haxe 3.1.3 or better + * [THRIFT-2969] - nodejs: DRY up library tests + * [THRIFT-2973] - Update Haxe lib readme regarding Haxe 3.1.3 + * [THRIFT-2952] - Improve handling of Server.Stop() + * [THRIFT-2964] - nodejs: move protocols and transports into separate files + * [THRIFT-2963] - nodejs - add test coverage + * [THRIFT-3006] - Attach 'omitempty' json tag for optional fields in Go + * [THRIFT-3027] - Go compiler does not ensure common initialisms have consistent case + * [THRIFT-3030] - TThreadedServer: Property for number of clientThreads + * [THRIFT-3023] - Go compiler is a little overly conservative with names of attributes + * [THRIFT-3018] - Compact protocol for Delphi + * [THRIFT-3025] - Change pure Int constants into @enums (where possible) + * [THRIFT-3031] - migrate "shouldStop" flag to TServer + * [THRIFT-3022] - Compact protocol for Haxe + * [THRIFT-3041] - Generate asynchronous clients for Cocoa + * [THRIFT-3053] - Perl SSL Socket Support (Encryption) + * [THRIFT-3247] - Generate a C++ thread-safe client + * [THRIFT-3217] - Provide a little endian variant of the binary protocol in C++ + * [THRIFT-3223] - TypeScript: Add initial support for Enum Maps + * [THRIFT-3220] - Option to suppress @Generated Annotation entirely + * [THRIFT-3300] - Reimplement TZlibTransport in Java using streams + * [THRIFT-3288] - c_glib: Build unit tests with all warnings enabled, no warnings generated + * [THRIFT-3347] - Improve cross test servers and clients + * [THRIFT-3342] - Improve ruby cross test client and server compatibility + * [THRIFT-2296] - Add C++ Base class for service + * [THRIFT-3337] - Add testBool method to cross tests + * [THRIFT-3303] - Disable concurrent cabal jobs on Travis to avoid GHC crash + * [THRIFT-2623] - Docker container for Thrift + * [THRIFT-3298] - thrift endian converters may conflict with other libraries + * [THRIFT-1559] - Provide memory pool for TBinaryProtocol to eliminate memory fragmentation + * [THRIFT-424] - Steal ProtocolBuffers' VarInt implementation for C++ + +## New Feature + * [THRIFT-3070] - Add ability to set the LocalCertificateSelectionCallback + * [THRIFT-1909] - Java: Add compiler flag to use the "option pattern" for optional fields + * [THRIFT-2099] - Stop TThreadPoolServer with alive connections. + * [THRIFT-123] - implement TZlibTransport in Java + * [THRIFT-2368] - New option: reuse-objects for Java generator + * [THRIFT-2836] - Optionally generate C++11 MoveConstructible types + * [THRIFT-2824] - Flag to disable html escaping doctext + * [THRIFT-2819] - Add WebsSocket client to node.js + * [THRIFT-3050] - Client certificate authentication for non-http TLS in C# + * [THRIFT-3292] - Implement TZlibTransport in Go + +## Question + * [THRIFT-2583] - Thrift on xPC target (SpeedGoat) + * [THRIFT-2592] - thrift server using c_glib + * [THRIFT-2832] - c_glib: Handle string lists correctly + * [THRIFT-3136] - thrift installation problem on mac + * [THRIFT-3346] - c_glib: Tutorials example crashes saying Calculator.ping implementation returned FALSE but did not set an error + +## Sub-task + * [THRIFT-2578] - Moving 'make cross' from test.sh to test.py + * [THRIFT-2734] - Go coding standards + * [THRIFT-2748] - Add Vagrantfile for Centos 6.5 + * [THRIFT-2753] - Misc. Haxe improvements + * [THRIFT-2640] - Compact Protocol in Cocoa + * [THRIFT-3262] - warning: overflow in implicit constant conversion in DenseProtoTest.cpp + * [THRIFT-3194] - Can't build with go enabled. gomock SCC path incorrect. + * [THRIFT-3275] - c_glib tutorial warnings in generated code + * [THRIFT-1125] - Multiplexing support for the Ruby Library + * [THRIFT-2807] - PHP Code Style + * [THRIFT-2841] - Add comprehensive integration tests for the whole Go stack + * [THRIFT-2815] - Haxe: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-2886] - Integrate binary type in standard Thrift cross test + * [THRIFT-2946] - Enhance usability of cross test framework + * [THRIFT-2967] - Add .editorconfig to root + * [THRIFT-3033] - Perl: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-3174] - Initialism code in the Go compiler doesn't check first word + * [THRIFT-3193] - Option to supress date value in @Generated annotation + * [THRIFT-3305] - Missing dist files for 0.9.3 release candidate + * [THRIFT-3341] - Add testBool methods + * [THRIFT-3308] - Fix broken test cases for 0.9.3 release candidate + +## Task + * [THRIFT-2834] - Remove semi-colons from python code generator + * [THRIFT-2853] - Adjust comments not applying anymore after THRIFT-2852 + +## Test + * [THRIFT-3211] - Add make cross support for php TCompactProtocol + +## Wish + * [THRIFT-2838] - TNonblockingServer can bind to port 0 (i.e., get an OS-assigned port) but there is no way to get the port number + + + +Thrift 0.9.2 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-2793] - Go compiler produces uncompilable code + * [THRIFT-1481] - Unix domain sockets in C++ do not support the abstract namespace + * [THRIFT-1455] - TBinaryProtocolT::writeString casts from size_t to uint32_t, which is not safe on 64-bit platforms + * [THRIFT-1579] - PHP Extention - function thrift_protocol_read_binary not working from TBinarySerializer::deserialize + * [THRIFT-1584] - Error: could not SetMinThreads in ThreadPool on single-core machines + * [THRIFT-1614] - Thrift build from svn repo sources fails with automake-1.12 + * [THRIFT-1047] - rb_thrift_memory_buffer_write treats arg as string without check, segfaults if you pass non-string + * [THRIFT-1639] - Java/Python: Serialization/Deserialization of double type using CompactProtocol + * [THRIFT-1647] - NodeJS BufferedTransport does not work beyond the hello-world example + * [THRIFT-2130] - Thrift's D library/test: parts of "make check" code do not compile with recent dmd-2.062 through dmd-2.064alpha + * [THRIFT-2140] - Error compiling cpp tutorials + * [THRIFT-2139] - MSVC 2012 Error - Cannot compile due to BoostThreadFactory + * [THRIFT-2138] - pkgconfig file created with wrong include path + * [THRIFT-2160] - Warning in thrift.h when compiling with -Wunused and NDEBUG + * [THRIFT-2158] - Compact, JSON, and SimpleJSON protocols are not working correctly + * [THRIFT-2167] - nodejs lib throws error if options argument isn't passed + * [THRIFT-2288] - Go impl of Thrift JSON protocol wrongly writes/expects true/false for bools + * [THRIFT-2147] - Thrift IDL grammar allows for dotted identifier names + * [THRIFT-2145] - Rack and Thin are not just development dependencies + * [THRIFT-2267] - Should be able to choose socket family in Python TSocket + * [THRIFT-2276] - java path in spec file needs updating + * [THRIFT-2281] - Generated send/recv code ignores errors returned by the underlying protocol + * [THRIFT-2280] - TJSONProtocol.Flush() does not really flush the transport + * [THRIFT-2274] - TNonblockingServer and TThreadedSelectorServer do not close their channel selectors on exit and leak file descriptors + * [THRIFT-2265] - php library doesn't build + * [THRIFT-2232] - IsSet* broken in Go + * [THRIFT-2246] - Unset enum value is printed by ToString() + * [THRIFT-2240] - thrift.vim (contrib) does not correctly handle 'union' + * [THRIFT-2243] - TNonblockingServer in thrift crashes when TFramedTransport opens + * [THRIFT-2230] - Cannot Build on RHEL/Centos/Amazon Linux 6.x + * [THRIFT-2247] - Go generator doesn't deal well with map keys of type binary + * [THRIFT-2253] - Python Tornado TTornadoServer base class change + * [THRIFT-2261] - java: error: unmappable character for encoding ASCII + * [THRIFT-2259] - C#: unexpected null logDelegate() pointer causes AV in TServer.serve() + * [THRIFT-2225] - SSLContext destroy before cleanupOpenSSL + * [THRIFT-2224] - TSSLSocket.h and TSSLServerSocket.h should use the platfromsocket too + * [THRIFT-2229] - thrift failed to build on OSX 10.9 GM + * [THRIFT-2227] - Thrift compiler generates spurious warnings with Xlint + * [THRIFT-2219] - Thrift gem fails to build on OS X Mavericks with 1.9.3 rubies + * [THRIFT-2226] - TServerSocket - keepAlive wrong initialization order + * [THRIFT-2285] - TJsonProtocol implementation for Java doesn't allow a slash (/) to be escaped (\/) + * [THRIFT-2216] - Extraneous semicolon in TProtocolUtil.h makes clang mad + * [THRIFT-2215] - Generated HTML/Graphviz lists referenced enum identifiers as UNKNOWN. + * [THRIFT-2211] - Exception constructor does not contain namespace prefix. + * [THRIFT-2210] - lib/java TSimpleJSONProtocol can emit invalid JSON + * [THRIFT-2209] - Ruby generator -- please namespace classes + * [THRIFT-2202] - Delphi TServerImpl.DefaultLogDelegate may stop the server with I/O-Error 105 + * [THRIFT-2201] - Ternary operator returns different types (build error for some compilers) + * [THRIFT-2200] - nested structs cause generate_fingerprint() to slow down at excessive CPU load + * [THRIFT-2197] - fix jar output directory in rpm spec file + * [THRIFT-2196] - Fix invalid dependency in Makefile.am + * [THRIFT-2194] - Node: Not actually prepending residual data in TFramedTransport.receiver + * [THRIFT-2193] - Java code generator emits spurious semicolon when deep copying binary data + * [THRIFT-2191] - Fix charp JSONProtocol.ReadJSONDouble (specify InvariantCulture) + * [THRIFT-2214] - System header sys/param.h is included inside the Thrift namespace + * [THRIFT-2178] - Thrift generator returns error exit code on --version + * [THRIFT-2171] - NodeJS implementation has extremely low test coverage + * [THRIFT-2183] - gem install fails on zsh + * [THRIFT-2182] - segfault in regression tests (GC bug in rb_thrift_memory_buffer_write) + * [THRIFT-2181] - oneway calls don't work in NodeJS + * [THRIFT-2169] - JavaME Thrift Library causes "java.io.IOException: No Response Entries Available" after using the Thrift client for some time + * [THRIFT-2168] - Node.js appears broken (at least, examples don't work as intended) + * [THRIFT-2293] - TSSLTransportFactory.createSSLContext() leaves files open + * [THRIFT-2279] - TSerializer only returns the first 1024 bytes serialized + * [THRIFT-2278] - Buffered transport doesn't support writes > buffer size + * [THRIFT-2275] - Fix memory leak in golang compact_protocol. + * [THRIFT-2282] - Incorect code generated for some typedefs + * [THRIFT-2009] - Go redeclaration error + * [THRIFT-1964] - 'Isset' causes problems with C#/.NET serializers + * [THRIFT-2026] - Fix TCompactProtocol 64 bit builds + * [THRIFT-2108] - Fix TAsyncClientManager timeout race + * [THRIFT-2068] - Multiple calls from same connection are not processed in node + * [THRIFT-1750] - Make compiler build cleanly under visual studio 10 + * [THRIFT-1755] - Comment parsing bug + * [THRIFT-1771] - "make check" fails on x64 for libboost_unit_test_framework.a + * [THRIFT-1841] - NodeJS Thrift incorrectly parses non-UTF8-string types + * [THRIFT-1908] - Using php thrift_protocol accelerated transfer causes core dump + * [THRIFT-1892] - Socket timeouts are declared in milli-seconds, but are actually set in micro-seconds + * [THRIFT-2303] - TBufferredTransport not properly closing underlying transport + * [THRIFT-2313] - nodejs server crash after processing the first request when using MultiplexedProcessor/FramedBuffer/BinaryProtocol + * [THRIFT-2311] - Go: invalid code generated when exception name is a go keyword + * [THRIFT-2308] - node: TJSONProtocol parse error when reading from buffered message + * [THRIFT-2316] - ccp: TFileTransportTest + * [THRIFT-2352] - msvc failed to compile thrift tests + * [THRIFT-2337] - Golang does not report TIMED_OUT exceptions + * [THRIFT-2340] - Generated server implementation does not send response type EXCEPTION on the Thrift.TApplicationExceptionType.UNKNOWN_METHOD exception + * [THRIFT-2354] - Connection errors can lead to case_clause exceptions + * [THRIFT-2339] - Uncaught exception in thrift c# driver + * [THRIFT-2356] - c++ thrift client not working with ssl (SSL_connect hangs) + * [THRIFT-2331] - Missing call to ReadStructBegin() in TApplicationException.Read() + * [THRIFT-2323] - Uncompileable Delphi code generated for typedef'd structs + * [THRIFT-2322] - Correctly show the number of times ExecutorService (java) has rejected the client. + * [THRIFT-2389] - namespaces handled wrongly in acrionscript 3.0 implementation + * [THRIFT-2388] - GoLang - Fix data races in simple_server and server_socket + * [THRIFT-2386] - Thrift refuses to link yylex + * [THRIFT-2375] - Excessive
's in generated HTML + * [THRIFT-2373] - warning CS0414 in THttpClient.cs: private field 'Thrift.Transport.THttpClient.connection' assigned but never used + * [THRIFT-2372] - thrift/json_protocol.go:160: function ends without a return statement + * [THRIFT-2371] - ruby bundler version fails on ~1.3.1, remove and take latest avail + * [THRIFT-2370] - Compiler SEGFAULTs generating HTML documentation for complex strucre + * [THRIFT-2384] - Binary map keys produce uncompilable code in go + * [THRIFT-2380] - unreachable code (CID 1174546, CID 1174679) + * [THRIFT-2378] - service method arguments of binary type lead to uncompileable Go code + * [THRIFT-2363] - Issue with character encoding of Success returned from Login using Thrift Proxy and NodeJS + * [THRIFT-2359] - TBufferedTransport doesn't clear it's buffer on a failed flush call + * [THRIFT-2428] - Python 3 setup.py support + * [THRIFT-2367] - Build failure: stdlib and boost both define uint64_t + * [THRIFT-2365] - C# decodes too many binary bytes from JSON + * [THRIFT-2402] - byte count of FrameBuffer in AWAITING_CLOSE state is not subtracted from readBufferBytesAllocated + * [THRIFT-2396] - Build Error on MacOSX + * [THRIFT-2395] - thrift Ruby gem requires development dependency 'thin' regardless of environment + * [THRIFT-2414] - c_glib fix several bug. + * [THRIFT-2420] - Go argument parser for methods without arguments does not skip fields + * [THRIFT-2439] - Bug in TProtocolDecorator Class causes parsing errors + * [THRIFT-2419] - golang - Fix fmt.Errorf in generated code + * [THRIFT-2418] - Go handler function panics on internal error + * [THRIFT-2405] - Node.js Multiplexer tests fail (silently) + * [THRIFT-2581] - TFDTransport destructor should not throw + * [THRIFT-2575] - Thrift includes siginfo_t within apache::thrift::protocol namespace + * [THRIFT-2577] - TFileTransport missuse of closesocket on windows platform + * [THRIFT-2576] - Implement Thrift.Protocol.prototype.skip method in JavaScript library + * [THRIFT-2588] - Thrift compiler is not buildable in Visual Studio 2010 + * [THRIFT-2594] - JS Compiler: Single quotes are not being escaped in constants. + * [THRIFT-2591] - TFramedTransport does not handle payloads split across packets correctly + * [THRIFT-2599] - Uncompileable Delphi code due to naming conflicts with IDL + * [THRIFT-2590] - C++ Visual Studio solution doesn't include Multiplexing support + * [THRIFT-2595] - Node.js: Fix global leaks and copy-paste errors + * [THRIFT-2565] - autoconf fails to find mingw-g++ cross compiler on travis CI + * [THRIFT-2555] - excessive "unused field" comments + * [THRIFT-2554] - double initialization in generated Read() method + * [THRIFT-2551] - OutOfMemoryError "unable to create new native thread" kills serve thread + * [THRIFT-2543] - Generated enum type in haskell should be qualified + * [THRIFT-2560] - Thrift compiler generator tries to concat ints with strings using + + * [THRIFT-2559] - Centos 6.5 unable to "make" with Thrift 0.9.1 + * [THRIFT-2526] - Assignment operators and copy constructors in c++ don't copy the __isset struct + * [THRIFT-2454] - c_glib: There is no gethostbyname_r() in some OS. + * [THRIFT-2451] - Do not use pointers for optional fields with defaults. Do not write such fields if its value set to default. Also, do not use pointers for any optional fields mapped to go map or slice. generate Get accessors + * [THRIFT-2450] - include HowToContribute in the src repo + * [THRIFT-2448] - thrift/test/test.sh has incorrect Node.js test path + * [THRIFT-2460] - unopened socket fd must be less than zero. + * [THRIFT-2459] - --version should not exit 1 + * [THRIFT-2468] - Timestamp handling + * [THRIFT-2467] - Unable to build contrib/fb303 on OSX 10.9.2 + * [THRIFT-2466] - Improper error handling for SSL/TLS connections that don't complete a handshake + * [THRIFT-2463] - test/py/RunClientServer.py fails sometimes + * [THRIFT-2458] - Generated golang server code for "oneway" methods is incorrect + * [THRIFT-2456] - THttpClient fails when using async support outside Silverlight + * [THRIFT-2524] - Visual Studio project is missing TThreadedServer files + * [THRIFT-2523] - Visual Studio project is missing OverlappedSubmissionThread files + * [THRIFT-2520] - cpp:cob_style generates incorrect .tcc file + * [THRIFT-2508] - Uncompileable C# code due to language keywords in IDL + * [THRIFT-2506] - Update TProtocolException error codes to be used consistently throughout the library + * [THRIFT-2505] - go: struct should always be a pointer to avoid copying of potentially size-unbounded structs + * [THRIFT-2515] - TLS Method error during make + * [THRIFT-2503] - C++: Fix name collision when a struct has a member named "val" + * [THRIFT-2477] - thrift --help text with misplaced comma + * [THRIFT-2492] - test/cpp does not compile on mac + * [THRIFT-2500] - sending random data crashes thrift(golang) service + * [THRIFT-2475] - c_glib: buffered_transport_write function return always TRUE. + * [THRIFT-2495] - JavaScript/Node string constants lack proper escaping + * [THRIFT-2491] - unable to import generated ThriftTest service + * [THRIFT-2490] - c_glib: if fail to read a exception from server, client may be occurred double free + * [THRIFT-2470] - THttpHandler swallows exceptions from processor + * [THRIFT-2533] - Boost version in requirements should be updated + * [THRIFT-2532] - Java version in installation requirements should be updated + * [THRIFT-2529] - TBufferedTransport split Tcp data bug in nodeJs + * [THRIFT-2537] - Path for "go get" does not work (pull request 115) + * [THRIFT-2443] - Node fails cross lang tests + * [THRIFT-2437] - Author fields in Python setup.py must be strings not lists. + * [THRIFT-2435] - Java compiler doesn't like struct member names that are identical to an existing enum or struct type + * [THRIFT-2434] - Missing namespace import for php TMultiplexedProcessor implementation + * [THRIFT-2432] - Flaky parallel build + * [THRIFT-2430] - Crash during TThreadPoolServer shutdown + * [THRIFT-667] - Period should not be allowed in identifier names + * [THRIFT-1212] - Members capital case conflict + * [THRIFT-2584] - Error handler not listened on javascript client + * [THRIFT-2294] - Incorrect Makefile generation + * [THRIFT-2601] - Fix vagrant to work again for builds again + * [THRIFT-2092] - TNonblocking server should release handler as soon as connection closes + * [THRIFT-2557] - CS0542 member names cannot be the same as their enclosing type + * [THRIFT-2605] - TSocket warning on gcc 4.8.3 + * [THRIFT-2607] - ThreadManager.cpp warning on clang++ 3.4 + * [THRIFT-1998] - TCompactProtocol.tcc - one more warning on Visual 2010 + * [THRIFT-2610] - MSVC warning in TSocket.cpp + * [THRIFT-2614] - TNonblockingServer.cpp warnings on MSVC + * [THRIFT-2608] - TNonblockingServer.cpp warnings on clang 3.4 + * [THRIFT-2606] - ThreadManager.h warning in clang++ 3.4 + * [THRIFT-2609] - TFileTransport.h unused field warning (clang 3.4) + * [THRIFT-2416] - Cannot use TCompactProtocol with MSVC + * [THRIFT-1803] - Ruby Thrift 0.9.0 tries to encode UUID to UTF8 and crashes + * [THRIFT-2385] - Problem with gethostbyname2 during make check + * [THRIFT-2262] - thrift server 'MutateRow' operation gives no indication of success / failure + * [THRIFT-2048] - Prefer boolean context to nullptr_t conversion + * [THRIFT-2528] - Thrift Erlang Library: Multiple thrift applications in one bundle + * [THRIFT-1999] - warning on gcc 4.7 while compiling BoostMutex.cpp + * [THRIFT-2104] - Structs lose binary data when transferred from server to client in Java + * [THRIFT-2184] - undefined method rspec_verify for Thrift::MemoryBufferTransport + * [THRIFT-2351] - PHP TCompactProtocol has fails to decode messages + * [THRIFT-2016] - Resource Leak in thrift struct under compiler/cpp/src/parse/t_function.h + * [THRIFT-2273] - Please delete old releases from mirroring system + * [THRIFT-2270] - Faulty library version numbering at build or documentation + * [THRIFT-2203] - Tests keeping failing on Jenkins and Travis CI + * [THRIFT-2399] - thrift.el: recognize "//"-style comments in emacs thrift-mode + * [THRIFT-2582] - "FileTransport error" exception is raised when trying to use Java's TFileTransport + * [THRIFT-1682] - Multiple thread calling a Service function unsafely causes message corruption and terminates with Broken Pipe + * [THRIFT-2357] - recurse option has no effect when generating php + * [THRIFT-2248] - Go generator doesn't deal well with map keys of type binary + * [THRIFT-2426] - clarify IP rights and contributions from fbthrift + * [THRIFT-2041] - TNonblocking server compilation on windows (ARITHMETIC_RIGHT_SHIFT) + * [THRIFT-2400] - thrift.el: recognize "//"-style comments in emacs thrift-mode + * [THRIFT-1717] - Fix deb build in jenkins + * [THRIFT-2266] - ThreadManager.h:24:10: fatal error: 'tr1/functional' file not found on Mac 10.9 (Mavericks) + * [THRIFT-1300] - Test failures with parallel builds (make -j) + * [THRIFT-2487] - Tutorial requires two IDL files but only one is linked from the Thrift web site + * [THRIFT-2329] - missing release tags within git + * [THRIFT-2306] - concurent client calls with nodejs + * [THRIFT-2222] - ruby gem cannot be compiled on OS X mavericks + * [THRIFT-2381] - code which generated by thrift2/hbase.thrift compile error + * [THRIFT-2390] - no close event when connection lost + * [THRIFT-2146] - Unable to pass multiple "--gen" options to the thrift compiler + * [THRIFT-2438] - Unexpected readFieldEnd call causes JSON Parsing errors + * [THRIFT-2498] - Error message "Invalid method name" while trying to call HBase Thrift API + * [THRIFT-841] - Build cruft + * [THRIFT-2570] - Wrong URL given in http://thrift.apache.org/developers + * [THRIFT-2604] - Fix debian packaging + * [THRIFT-2618] - Unignore /aclocal files required for build + * [THRIFT-2562] - ./configure create MakeFile in lib/d with errors + * [THRIFT-2593] - Unable to build thrift on ubuntu-12.04 (Precise) + * [THRIFT-2461] - Can't install thrift-0.8.0 on OS X 10.9.2 + * [THRIFT-2602] - Fix missing dist files + * [THRIFT-2620] - Fix python packaging + * [THRIFT-2545] - Test CPP fails to build (possibly typo) + +## Documentation + * [THRIFT-2155] - Adding one liner guide to rename the version.h.in and rename thrifty.cc.h + * [THRIFT-1991] - Add exceptions to examples + * [THRIFT-2334] - add a tutorial for node JS + * [THRIFT-2392] - Actionscript tutorial + * [THRIFT-2383] - contrib: sample for connecting Thrift with Rebus + * [THRIFT-2382] - contrib: sample for connecting Thrift with STOMP + +## Improvement + * [THRIFT-1457] - Capacity of TframedTransport write buffer is never reset + * [THRIFT-1135] - Node.js tutorial + * [THRIFT-1371] - Socket timeouts (SO_RCVTIMEO and SO_SNDTIMEO) not supported on Solaris + * [THRIFT-2142] - Minor tweaks to thrift.el for better emacs package compatibility + * [THRIFT-2268] - Modify TSaslTransport to ignore TCP health checks from loadbalancers + * [THRIFT-2264] - GitHub page incorrectly states that Thrift is still incubating + * [THRIFT-2263] - Always generate good hashCode for Java + * [THRIFT-2233] - Java compiler should defensively copy its binary inputs + * [THRIFT-2239] - Address FindBugs errors + * [THRIFT-2249] - Add SMP Build option to thrift.spec (and three config defines) + * [THRIFT-2254] - Exceptions generated by Go compiler should implement error interface + * [THRIFT-2260] - Thrift imposes unneeded dependency on commons-lang3 + * [THRIFT-2258] - Add TLS v1.1/1.2 support to TSSLSocket.cpp + * [THRIFT-2205] - Node.js Test Server to support test.js JavaScript Browser test and sundry fixes + * [THRIFT-2204] - SSL client for the cocoa client + * [THRIFT-2172] - Java compiler allocates optionals array for every struct with an optional field + * [THRIFT-2185] - use cabal instead of runhaskell in haskell library + * [THRIFT-1926] - PHP Constant Generation Refactoring + * [THRIFT-2029] - Port C++ tests to Windows + * [THRIFT-2054] - TSimpleFileTransport - Java Lib has no straight forward TTransport based file transport + * [THRIFT-2040] - "uninitialized variable" warnings on MSVC/windows + * [THRIFT-2034] - Give developers' C++ code direct access to socket FDs on server side + * [THRIFT-2095] - Use print function for Python 3 compatiblity + * [THRIFT-1868] - Make the TPC backlog configurable in the Java servers + * [THRIFT-1813] - Add @Generated annotation to generated classes + * [THRIFT-1815] - Code generators line buffer output + * [THRIFT-2305] - TFramedTransport empty constructor should probably be private + * [THRIFT-2304] - Move client assignments from construtor in method + * [THRIFT-2309] - Ruby (gem) & PHP RPM subpackages + * [THRIFT-2318] - perl: dependency Class::Accessor not checked + * [THRIFT-2317] - exclude tutorial from build + * [THRIFT-2320] - Program level doctext does not get attached by parser + * [THRIFT-2349] - Golang - improve tutorial + * [THRIFT-2348] - PHP Generator: add array typehint to functions + * [THRIFT-2344] - configure.ac: compiler-only option + * [THRIFT-2343] - Golang - Return a single error for all exceptions instead of multiple return values + * [THRIFT-2341] - Enable generation of Delphi XMLDoc comments (a.k.a. "Help Insight") + * [THRIFT-2355] - Add SSL and Web Socket Support to Node and JavaScript + * [THRIFT-2350] - Add async calls to normal JavaScript + * [THRIFT-2330] - Generate PHPDoc comments + * [THRIFT-2332] - RPMBUILD: run bootstrap (if needed) + * [THRIFT-2391] - simple socket transport for actionscript 3.0 + * [THRIFT-2376] - nodejs: allow Promise style calls for client and server + * [THRIFT-2369] - Add ssl support for nodejs implementation + * [THRIFT-2401] - Haskell tutorial compiles + * [THRIFT-2417] - C# Union classes are not partial + * [THRIFT-2415] - Named pipes server performance & message mode + * [THRIFT-2404] - emit warning on (typically inefficient) list + * [THRIFT-2398] - Improve Node Server Library + * [THRIFT-2397] - Add CORS and CSP support for JavaScript and Node.js libraries + * [THRIFT-2407] - use markdown (rename README => README.md) + * [THRIFT-2300] - D configure info output should follow same format as other languages + * [THRIFT-2579] - Windows CE support + * [THRIFT-2574] - Compiler option to generate namespace directories for Ruby + * [THRIFT-2571] - Simplify cross compilation using CMake + * [THRIFT-2569] - Introduce file to specify third party library locations on Windows + * [THRIFT-2568] - Implement own certificate handler + * [THRIFT-2552] - eliminate warning from configure.ac + * [THRIFT-2549] - Generate json tag for struct members. use go.tag annotation to override the default generated tag. + * [THRIFT-2544] - Add support for socket transport for c# library when using Windows Phone projects + * [THRIFT-2453] - haskell tutorial: fix up division by 0 example + * [THRIFT-2449] - Enhance typedef structure to distinguish between forwards and real typedefs + * [THRIFT-2446] - There is no way to handle server stream errors + * [THRIFT-2455] - Allow client certificates to be used with THttpClient + * [THRIFT-2511] - Node.js needs the compact protocol + * [THRIFT-2493] - Node.js lib needs HTTP client + * [THRIFT-2502] - Optimize go implementations of binary and compact protocols for speed + * [THRIFT-2494] - Add enum toString helper function in c_glib + * [THRIFT-2471] - Make cpp.ref annotation language agnostic + * [THRIFT-2497] - server and client for test/go, also several fixes and improvements + * [THRIFT-2535] - TJSONProtocol when serialized yields TField ids rather than names + * [THRIFT-2220] - Add a new struct structv? + * [THRIFT-1352] - Thrift server + * [THRIFT-989] - Push boost m4 macros upstream + * [THRIFT-1349] - Remove unnecessary print outs + * [THRIFT-2496] - server and client for test/go, also several fixes and improvements + * [THRIFT-1114] - Maven publish shouldn't require passwords hardcoded in settings.xml + * [THRIFT-2043] - visual 2010 warnings - unreachable code + * [THRIFT-1683] - Implement alternatives to Javascript Client side Transport protocol, just as NPAPI and WebSocket. + * [THRIFT-1746] - provide a SPDX file + * [THRIFT-1772] - Serialization does not check types of embedded structures. + * [THRIFT-2387] - nodejs: external imports should be centralized in index.js + * [THRIFT-2037] - More general macro THRIFT_UNUSED_VARIABLE + +## New Feature + * [THRIFT-1012] - Transport for DataInput DataOutput interface + * [THRIFT-2256] - Using c++11/c++0x std library replace boost library + * [THRIFT-2250] - JSON and MemoryBuffer for JavaME + * [THRIFT-2114] - Python Service Remote SSL Option + * [THRIFT-1719] - SASL client support for Python + * [THRIFT-1894] - Thrift multi-threaded async Java Server using Java 7 AsynchronousChannelGroup + * [THRIFT-1893] - HTTP/JSON server/client for node js + * [THRIFT-2347] - C# TLS Transport based on THRIFT-181 + * [THRIFT-2377] - Allow addition of custom HTTP Headers to an HTTP Transport + * [THRIFT-2408] - Named Pipe Transport Option for C# + * [THRIFT-2572] - Add string/collection length limit checks (from C++) to java protocol readers + * [THRIFT-2469] - "java:fullcamel" option to automatically camel-case underscored attribute names + * [THRIFT-795] - Importing service functions (simulation multiple inheritance) + * [THRIFT-2164] - Add a Get/Post Http Server to Node along with examples + * [THRIFT-2255] - add Parent Class for generated Struct class + +## Question + * [THRIFT-2539] - Tsocket.cpp addrinfo ai_flags = AI_ADDRCONFIG + * [THRIFT-2440] - how to connect as3 to java by thrift , + * [THRIFT-2379] - Memmory leaking while using multithreading in C++ server. + * [THRIFT-2277] - Thrift: installing fb303 error + * [THRIFT-2567] - Csharp slow ? + * [THRIFT-2573] - thrift 0.9.2 release + +## Sub-task + * [THRIFT-981] - cocoa: add version Info to the library + * [THRIFT-2132] - Go: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-2299] - TJsonProtocol implementation for Ruby does not allow for both possible slash (solidus) encodings + * [THRIFT-2298] - TJsonProtocol implementation for C# does not allow for both possible slash (solidus) encodings + * [THRIFT-2297] - TJsonProtocol implementation for Delphi does not allow for both possible slash (solidus) encodings + * [THRIFT-2271] - JavaScript: Support for Multiplexing Services + * [THRIFT-2251] - go test for compact protocol is not running + * [THRIFT-2195] - Delphi: Add event handlers for server and processing events + * [THRIFT-2176] - TSimpleJSONProtocol.ReadFieldBegin() does not return field type and ID + * [THRIFT-2175] - Wrong field type set for binary + * [THRIFT-2174] - Deserializing JSON fails in specific cases + * [THRIFT-2053] - NodeJS: Support for Multiplexing Services + * [THRIFT-1914] - Python: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-1810] - add ruby to test/test.sh + * [THRIFT-2310] - PHP: Client-side support for Multiplexing Services + * [THRIFT-2346] - C#: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2345] - Delphi: UTF-8 sent by PHP as JSON is not understood by TJsonProtocol + * [THRIFT-2338] - First doctext wrongly interpreted as program doctext in some cases + * [THRIFT-2325] - SSL test certificates + * [THRIFT-2358] - C++: add compact protocol to cross language test suite + * [THRIFT-2425] - PHP: Server-side support for Multiplexing Services + * [THRIFT-2421] - Tree/Recursive struct support in thrift + * [THRIFT-2290] - Update Go tutorial to align with THRIFT-2232 + * [THRIFT-2558] - CSharp compiler generator tries to concat ints with strings using + + * [THRIFT-2507] - Additional LUA TProtocolException error code needed? + * [THRIFT-2499] - Compiler: allow annotations without "= value" + * [THRIFT-2534] - Cross language test results should recorded to a status.md or status.html file automatically + * [THRIFT-66] - Java: Allow multiplexing multiple services over a single TCP connection + * [THRIFT-1681] - Add Lua Support + * [THRIFT-1727] - Ruby-1.9: data loss: "binary" fields are re-encoded + * [THRIFT-1726] - Ruby-1.9: "binary" fields are represented by string whose encoding is "UTF-8" + * [THRIFT-988] - perl: add version Info to the library via configure + * [THRIFT-334] - Compact Protocol for PHP + * [THRIFT-2444] - pull request 88: thrift: clean up enum value assignment + +## Task + * [THRIFT-2223] - Spam links on wiki + * [THRIFT-2566] - Please create a DOAP file for your TLP + * [THRIFT-2237] - Update archive to contain all versions + * [THRIFT-962] - Tutorial page on our website is really unhelpful + +## Test + * [THRIFT-2327] - nodejs: nodejs test suite should be bundled with the library + * [THRIFT-2445] - THRIFT-2384 (code generation for go maps with binary keys) should be tested + * [THRIFT-2501] - C# The test parameters from the TestServer and TestClient are different from the http://thrift.apache.org/test/ + +## Wish + * [THRIFT-2190] - Add the JavaScript thrift.js lib to the Bower registry + * [THRIFT-2076] - boost::optional instead of __isset + + + +Thrift 0.9.1 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1440] - debian packaging: minor-ish policy problems + * [THRIFT-1402] - Generated Y_types.js does not require() X_types.js when an include in the IDL file was used + * [THRIFT-1551] - 2 thrift file define only struct (no service), one include another, the gen nodejs file didn't have "requires" at the top + * [THRIFT-1264] - TSocketClient is queried by run loop after deallocation in Cocoa + * [THRIFT-1600] - Thrift Go Compiler and Library out of date with Go 1 Release. + * [THRIFT-1603] - Thrift IDL allows for multiple exceptions, args or struct member names to be the same + * [THRIFT-1062] - Problems with python tutorials + * [THRIFT-864] - default value fails if identifier is a struct + * [THRIFT-930] - Ruby and Haskell bindings don't properly support DESTDIR (makes packaging painful) + * [THRIFT-820] - The readLength attribute of TBinaryProtocol is used as an instance variable and is decremented on each call of checkReadLength + * [THRIFT-1640] - None of the tutorials linked on the website contain content + * [THRIFT-1637] - NPM registry does not include version 0.8 + * [THRIFT-1648] - NodeJS clients always receive 0 for 'double' values. + * [THRIFT-1660] - Python Thrift library can be installed with pip but not easy_install + * [THRIFT-1657] - Chrome browser sending OPTIONS method before POST in xmlHttpRequest + * [THRIFT-2118] - Certificate error handling still incorrect + * [THRIFT-2137] - Ruby test lib fails jenkins build #864 + * [THRIFT-2136] - Vagrant build not compiling java, ruby, php, go libs due to missing dependencies + * [THRIFT-2135] - GO lib leaves behind test files that are auto generated + * [THRIFT-2134] - mingw-cross-compile script failing with strip errors + * [THRIFT-2133] - java TestTBinaryProtocol.java test failing + * [THRIFT-2126] - lib/cpp/src/thrift/concurrency/STD* files missing from DIST + * [THRIFT-2125] - debian missing from DIST + * [THRIFT-2124] - .o, .so, .la, .deps, .libs, gen-* files left tutorials, test and lib/cpp when making DIST + * [THRIFT-2123] - GO lib missing files in DIST build + * [THRIFT-2121] - Compilation bug for Node.js + * [THRIFT-2129] - php ext missing from dist + * [THRIFT-2128] - lib GO tests fail with funct ends without a return statement + * [THRIFT-2286] - Failed to compile Thrift0.9.1 with boost1.55 by VS2010 if select Debug-mt&x64 mode. + * [THRIFT-1973] - TCompactProtocol in C# lib does not serialize and deserialize negative int32 and int64 number correctly + * [THRIFT-1992] - casts in TCompactProtocol.tcc causing "dereferencing type-punned pointer will break strict-aliasing rules" warnings from gcc + * [THRIFT-1930] - C# generates unsigned byte for Thrift "byte" type + * [THRIFT-1929] - Update website to use Mirrors for downloads + * [THRIFT-1928] - Race may still exist in TFileTransport::flush() + * [THRIFT-1934] - Tabs in Example section on main page are not working + * [THRIFT-1933] - Delphi generator crashes when a typedef references another typedef from an included file + * [THRIFT-1942] - Binary accelerated cpp extension does not use Thrift namespaces for Exceptions + * [THRIFT-1959] - C#: Add Union TMemoryBuffer support + * [THRIFT-1958] - C#: Use static Object.Equals instead of .Equals() calls in equals + * [THRIFT-1957] - NodeJS TFramedTransport and TBufferedTransport read bytes as unsigned + * [THRIFT-1955] - Union Type writer generated in C# does not WriteStructBegin + * [THRIFT-1952] - Travis CI + * [THRIFT-1949] - WP7 build broken + * [THRIFT-1943] - docstrings for enum values are ignored + * [THRIFT-2070] - Improper `HexChar' and 'HexVal' implementation in TJSONProtocol.cs + * [THRIFT-2017] - Resource Leak in thrift struct under compiler/cpp/src/parse/t_program.h + * [THRIFT-2032] - C# client leaks sockets/handles + * [THRIFT-1996] - JavaME Constants generation is broken / inconsistent with regular Java generation + * [THRIFT-2002] - Haskell: Test use Data.Maybe instead of Maybe + * [THRIFT-2051] - Vagrant fails to build erlang + * [THRIFT-2050] - Vagrant C# lib compile fails with TException missing + * [THRIFT-1978] - Ruby: Thrift should allow for the SSL verify mode to be set + * [THRIFT-1984] - namespace collision in python bindings + * [THRIFT-1988] - When trying to build a debian package it fails as the file NEWS doesn't exist + * [THRIFT-1975] - TBinaryProtocol CheckLength can't be used for a client + * [THRIFT-1995] - '.' allowed at end of identifier generates non-compilable code + * [THRIFT-2112] - Error in Go generator when using typedefs in map keys + * [THRIFT-2088] - Typos in Thrift compiler help text + * [THRIFT-2080] - C# multiplex processor does not catch IOException + * [THRIFT-2082] - Executing "gmake clean" is broken + * [THRIFT-2102] - constants are not referencing to correct type when included from another thrift file + * [THRIFT-2100] - typedefs are not correctly referenced when including from other thrift files + * [THRIFT-2066] - 'make install' does not install two headers required for C++ bindings + * [THRIFT-2065] - Not valid constants filename in Java + * [THRIFT-2047] - Thrift.Protocol.TCompactProtocol, intToZigZag data lost (TCompactProtocol.cs) + * [THRIFT-2036] - Thrift gem warns about class variable access from top level + * [THRIFT-2057] - Vagrant fails on php tests + * [THRIFT-2105] - Generated code for default values of collections ignores t_field::T_REQUIRED + * [THRIFT-2091] - Unnecessary 'friend' declaration causes warning in TWinsockSingleton + * [THRIFT-2090] - Go generator, fix including of other thrift files + * [THRIFT-2106] - Fix support for namespaces in GO generator + * [THRIFT-1783] - C# doesn't handle required fields correctly + * [THRIFT-1782] - async only defined in silverlight + * [THRIFT-1779] - Missing process_XXXX method in generated TProcessor implementation for all 'oneway' service functions + * [THRIFT-1692] - SO_REUSEADDR allows for socket hijacking on Windows + * [THRIFT-1720] - JRuby times out on successful connection + * [THRIFT-1713] - Named and Anonymous Pipe transport (Delphi) + * [THRIFT-1699] - Native Union#read has extra read_field_end call + * [THRIFT-1749] - Python TSSLSocket error handling obscures actual error + * [THRIFT-1748] - Guard and RWGuard macros defined in global namespace + * [THRIFT-1734] - Front webpage is still advertising v0.8 as current release + * [THRIFT-1729] - C glib refactor left empty folders in svn + * [THRIFT-1767] - unions can't have required fields (Delphi) + * [THRIFT-1765] - Incorrect error message printed for null or negative keys + * [THRIFT-1778] - Configure requires manual intervention due to tar failure + * [THRIFT-1777] - TPipeServer is UNSTOPPABLE + * [THRIFT-1753] - Multiple C++ Windows, OSX, and iOS portability issues + * [THRIFT-1756] - 'make -j 8' fails with "unterminated #ifdef" error + * [THRIFT-1773] - Python library should run on python 2.4 + * [THRIFT-1769] - unions can't have required fields (C++) + * [THRIFT-1768] - unions can't have required fields (Compiler) + * [THRIFT-1666] - htonll usage in TBinaryProtocol.tcc generates warning with MSVC2010 + * [THRIFT-1919] - libthrift depends on httpcore-4.1.3 (directly) and httpcore-4.1.4 (transitively) + * [THRIFT-1864] - implement event handler for non-blocking server + * [THRIFT-1859] - Generated error c++ code with -out and include_prefix param + * [THRIFT-1869] - TThreadPoolServer (java) dies when threadpool is consumed + * [THRIFT-1842] - Memory leak with Pipes + * [THRIFT-1838] - Can't build compiler on OS X because of missing thrifty.h + * [THRIFT-1846] - Restore socket.h header to support builds with Android NDK + * [THRIFT-1850] - make check hangs on TSocket tests in TransportTest.cpp + * [THRIFT-1873] - Binary protocol factory ignores struct read/write flags + * [THRIFT-1872] - issues with TBufferedTransport buffer + * [THRIFT-1904] - Incorrect code is generated for typedefs which use included types + * [THRIFT-1903] - PHP namespaces cause binary protocols to not be used + * [THRIFT-1895] - Delphi: reserved variable name "result" not detected properly + * [THRIFT-1881] - TNonblockingServer does not release open connections or threads on shutdown + * [THRIFT-1888] - Java Thrift client can't connect to Python Thrift server on same host + * [THRIFT-1831] - Bug in list deserializer + * [THRIFT-1824] - many compile warning, becase Thread.h includes config.h + * [THRIFT-1823] - Missing parenthesis breaks "IS_..." macro in generated code + * [THRIFT-1806] - Python generation always truncates __init__.py files + * [THRIFT-1795] - Race condition in TThreadedServerPool java implementation + * [THRIFT-1794] - C# asyncctp broken + * [THRIFT-1804] - Binary+compact protocol single byte error in Ruby library (ARM architecture): caused by different char signedness + * [THRIFT-1800] - Documentation text not always escaped correctly when rendered to HTML + * [THRIFT-1788] - C#: Constants static constructor does not compile + * [THRIFT-1816] - Need "require" included thrift files in "xxx_types.js" + * [THRIFT-1907] - Compiling namespace and sub-namespace directives for unrecognized generators should only be a warning + * [THRIFT-1913] - skipping unknown fields in java unions + * [THRIFT-2553] - C++ linker error - transport/TSocket + * [THRIFT-274] - Towards a working release/versioning process + +## Documentation + * [THRIFT-1971] - [Graphviz] Adds tutorial/general description documentation + * [THRIFT-2001] - http://thrift.apache.org/ Example "C++ Server" tab is broken + +## Improvement + * [THRIFT-1574] - Apache project branding requirements: DOAP file [PATCH] + * [THRIFT-1347] - Unify the exceptions returned in generated Go code + * [THRIFT-1353] - Switch to performance branch, get rid of BinaryParser + * [THRIFT-1629] - Ruby 1.9 Compatibility during Thrift configure, make, install + * [THRIFT-991] - Refactor Haskell code and generator + * [THRIFT-990] - Sanify gettimeofday usage codebase-wide + * [THRIFT-791] - Let C++ TSimpleServer be driven by an external main loop + * [THRIFT-2117] - Cocoa TBinaryProtocol strictWrite should be set to true by default + * [THRIFT-2014] - Change C++ lib includes to use style throughout + * [THRIFT-1972] - Add support for async processors + * [THRIFT-1970] - [Graphviz] Adds option to render exceptions relationships + * [THRIFT-1966] - Support different files for SSL certificates and keys + * [THRIFT-1965] - Adds Graphviz (graph description language) generator + * [THRIFT-1956] - Switch to Apache Commons Lang 3 + * [THRIFT-1962] - Multiplex processor should send any TApplicationException back to client + * [THRIFT-1960] - main() declares 22 unused gen bools + * [THRIFT-1951] - libthrift.jar has source files in it + * [THRIFT-1997] - Add accept backlog configuration method to TServerSocket + * [THRIFT-2003] - Deprecate senum + * [THRIFT-2052] - Vagrant machine image defaults to only 384MB of RAM + * [THRIFT-1980] - Modernize Go tooling, fix go client libary. + * [THRIFT-1977] - C# compiler should generate constant files prefixed with thrift file name + * [THRIFT-1985] - add a Vagrantfile to build and test Apache Thrift fully reproducable + * [THRIFT-1994] - Deprecate slist + * [THRIFT-1993] - Factory to create instances from known (generated) interface types with Delphi + * [THRIFT-2081] - Specified timeout should be used in TSocket.Open() + * [THRIFT-2084] - Delphi: Ability to create entity Thrift-generated instances based on TypeInfo + * [THRIFT-2083] - Improve the go lib: buffered Transport, save memory allocation, handle concurrent request + * [THRIFT-2109] - Secure connections should be supported in Go + * [THRIFT-2107] - minor Go generator fixes + * [THRIFT-1695] - allow warning-free compilation in VS 2012 and GNU 4.6 + * [THRIFT-1735] - integrate tutorial into regular build + * [THRIFT-1716] - max allowed connections should be PIPE_UNLIMITED_INSTANCES + * [THRIFT-1715] - Allow excluding python parts when building contrib/fb303 + * [THRIFT-1733] - Fix RPM build issues on RHEL6/OL6 systems + * [THRIFT-1728] - Upgradation of httpcomponents + * [THRIFT-1876] - Use enum names instead of casted integers in assignments + * [THRIFT-1874] - timeout for the server-side end of a named pipe + * [THRIFT-1897] - Support validation of required fields + * [THRIFT-1896] - Add TBase protocol for Cocoa + * [THRIFT-1880] - Make named pipes server work asynchronously (overlapped) to allow for clean server stops + * [THRIFT-1878] - Add the possibility to send custom headers + * [THRIFT-1882] - Use single include + * [THRIFT-1793] - C#: Use static read instead of instance read + * [THRIFT-1799] - Option to generate HTML in "standalone mode" + * [THRIFT-1815] - Code generators line buffer output + * [THRIFT-1890] - C++: Make named pipes server work asynchronously + * [THRIFT-474] - Generating Ruby on Rails friendly code + +## New Feature + * [THRIFT-801] - Provide an interactive shell (irb) when generating ruby bindings + * [THRIFT-2292] - Android Library Project + * [THRIFT-2012] - Modernizing Go + * [THRIFT-1969] - C#: Tests not properly linked from the solution + * [THRIFT-1785] - C#: Add TMemoryBuffer serializer/deserializer + * [THRIFT-1780] - Add option to generate nullable values + * [THRIFT-1786] - C# Union Typing + * [THRIFT-591] - Make the C++ runtime library be compatible with Windows and Visual Studio + * [THRIFT-514] - Add option to configure compiler output directory + +## Question + * [THRIFT-1764] - how to get the context of client when on a rpc call in server side? + * [THRIFT-1791] - thrift's namespace directive when generating haskell code + +## Sub-task + * [THRIFT-1594] - Java test clients should have a return codes that reflect whether it succeeds or not. + * [THRIFT-1595] - Java test server should follow the documented behavior as of THRIFT-1590 + * [THRIFT-986] - st: add version Info to the library + * [THRIFT-985] - php: add version Info to the library + * [THRIFT-984] - ocaml: add version Info to the library + * [THRIFT-1924] - Delphi: Inconsistency in serialization of optional fields + * [THRIFT-1922] - C#: Inconsistency in serialization of optional fields + * [THRIFT-1961] - C# tests should be in lib/csharp/test/... + * [THRIFT-1822] - PHP unit test does not work + * [THRIFT-1902] - C++: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-1901] - C#: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-1899] - Delphi: Support for Multiplexing Services on any Transport, Protocol and Server + * [THRIFT-563] - Support for Multiplexing Services on any Transport, Protocol and Server + + + +Thrift 0.9 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1438] - lib/cpp/src/windows/config.h should read version from configure.ac rather than a #define + * [THRIFT-1446] - Compile error with Delphi 2009 in constant initializer + * [THRIFT-1450] - Problems building thrift 0.8.0 for Python and Ruby + * [THRIFT-1449] - Ruby client does not work on solaris (?) + * [THRIFT-1447] - NullpointerException in ProcessFunction.class :in "oneway" method + * [THRIFT-1433] - TServerSocket fix for MSVC + * [THRIFT-1429] - The nonblocking servers is supposed to use TransportFactory to read the data + * [THRIFT-1427] - PHP library uses non-multibyte safe functions with mbstring function overloading + * [THRIFT-1421] - Debian Packages can not be built + * [THRIFT-1394] - Treatment of optional fields is not consistent between C++ and Java + * [THRIFT-1511] - Server with oneway support ( JAVA ) + * [THRIFT-1496] - PHP compiler not namespacing enums + * [THRIFT-1495] - PHP TestClient fatals on missing class + * [THRIFT-1508] - TServerSocket does not allow for the user to specify the IP address to bind to + * [THRIFT-1504] - Cocoa Generator should use local file imports for base Thrift headers + * [THRIFT-1512] - Thrift socket support for Windows XP + * [THRIFT-1502] - TSimpleServer::serve(): Do not print out error message if server was stopped. + * [THRIFT-1501] - PHP old namespaces not generated for enums + * [THRIFT-1483] - java compiler does not generate type parameters for services in extended clauses + * [THRIFT-1479] - Compiled PHP process functions missing writeMessageEnd() + * [THRIFT-1492] - enabling c_glib render thrift unusable (even for C++ code) + * [THRIFT-1491] - Uninitialize processorFactory_ member in TServer.h + * [THRIFT-1475] - Incomplete records generation for Erlang + * [THRIFT-1486] - Javascript manual testserver not returning content types + * [THRIFT-1488] - src/concurrency/Thread.h:91:58: error: invalid conversion from 'pthread_t {aka _opaque_pthread_t*}' to 'apache::thrift::concurrency::Thread::id_t {aka long long unsigned int}' [-fpermissive] + * [THRIFT-1490] - Windows-specific header files - fixes & tweaks + * [THRIFT-1526] - Union TupleSchemeFactory returns StandardSchemes + * [THRIFT-1527] - Generated implementation of tupleReadStruct in unions return null when the setfield is unrecognized + * [THRIFT-1524] - TNonBlockingServer does not compile in Visual Studio 2010 + * [THRIFT-1529] - TupleProtocol can unintentionally include an extra byte in bit vectors when number of optional fields is an integral of 8 + * [THRIFT-1473] - JSON context stack may be left in an incorrect state when an exception is thrown during read or write operations + * [THRIFT-1456] - System.Net.HttpWebRequest' does not contain a definition for 'Proxy' + * [THRIFT-1468] - Memory leak in TSaslServerTransport + * [THRIFT-1461] - Recent TNonblockingServer changes broke --enable-boostthreads=yes, Windows + * [THRIFT-1460] - why not add unicode strings support to python directly? + * [THRIFT-1464] - AbstractNonblockingServer.FrameBuffer TNonblockingTransport accessor changed from public to private + * [THRIFT-1467] - Possible AV with empty strings when using JSON protocol + * [THRIFT-1523] - clientTimeout not worked as expected in TServerSocket created by TSSLTransportFactory + * [THRIFT-1537] - TFramedTransport issues + * [THRIFT-1519] - Thirft Build Failure referencing rb_intern2 symbol + * [THRIFT-1518] - Generated C++ code only sends the first optional field in the write() function for a struct. + * [THRIFT-1515] - NameError: global name 'TApplicationException' is not defined + * [THRIFT-1554] - Inherited service methods are not resolved in derived service implementations + * [THRIFT-1553] - thrift nodejs service side can't read map structure, key as enum, value as Object + * [THRIFT-1575] - Typo in server/TThreadPoolServer.h + * [THRIFT-1327] - Fix Spec Suite under Ruby-1.8.7 (works for MRI Ruby-1.9.2) + * [THRIFT-1326] - on some platforms, #include is necessary to be included in Thrift.h + * [THRIFT-1159] - THttpClient->Flush() issue (connection thru proxy) + * [THRIFT-1277] - Node.js serializes false booleans as null + * [THRIFT-1224] - Cannot insert UTF-8 text + * [THRIFT-1267] - Node.js can't throw exceptions. + * [THRIFT-1338] - Do not use an unpatched autoconf 2.65 to generate release tarball + * [THRIFT-1128] - MAC OS X: thrift.h incompatibility with Thrift.h + * [THRIFT-1631] - Fix C++ server constructor typos + * [THRIFT-1602] - PHP C Extension is not Compatible with PHP 5.4 + * [THRIFT-1610] - IWebProxy not available on WP7 platform + * [THRIFT-1606] - Race condition in BoostThreadFactory.cpp + * [THRIFT-1604] - Python exception handeling for changes from PEP 3110 + * [THRIFT-1607] - Incorrect file modes for several source files + * [THRIFT-1583] - c_glib leaks memory + * [THRIFT-1582] - Bad includes of nested thrift files in c_glib + * [THRIFT-1578] - C_GLib generated code does not compile + * [THRIFT-1597] - TJSONProtocol.php is missing from Makefile.am + * [THRIFT-1591] - Enable TCP_NODELAY for ruby gem + * [THRIFT-1624] - Isset Generated differently on different platforms + * [THRIFT-1622] - Incorrect size returned on read + * [THRIFT-1621] - Memory leaks + * [THRIFT-1612] - Base64 encoding is broken + * [THRIFT-1627] - compiler built using compilers.vcxproj cannot be used to build some test .thrift files + * [THRIFT-1571] - Update Ruby HTTP transport for recent Ruby versions + * [THRIFT-1023] - Thrift encoding (UTF-8) issue with Ruby 1.9.2 + * [THRIFT-1090] - Document the generation of a file called "Constants.java" + * [THRIFT-1082] - Thrift::FramedTransport sometimes calls close() on an undefined value + * [THRIFT-956] - Python module's version meta-data should be updated + * [THRIFT-973] - Cocoa library won't compile using clang + * [THRIFT-1632] - ruby: data corruption in thrift_native implementation of MemoryBufferTransport + * [THRIFT-1665] - TBinaryProtocol: exceeded message length raises generic TException + * [THRIFT-1664] - Reference to non-existing variable in build script + * [THRIFT-1663] - Java Thrift server is not throwing exceptions + * [THRIFT-1662] - "removeObject:" should be "removeObserver:" in [-TSocketServer dealloc]? + * [THRIFT-1643] - Denial of Service attack in TBinaryProtocol.readString + * [THRIFT-1674] - Update Thrift D library to be compatible with 2.060 + * [THRIFT-1673] - Ruby compile flags for extension for multi arch builds (os x) + * [THRIFT-1655] - Configure still trying to use thrift_generators in output + * [THRIFT-1654] - c_glib thrift_socket_read() returns corrupted data + * [THRIFT-1653] - TThreadedSelectorServer leaks CLOSE_WAIT sockets + * [THRIFT-1658] - Java thrift server is not throwing TApplicationException + * [THRIFT-1656] - Setting proper headers in THttpServer.cpp so that "Cross-Origin Resource Sharing" on js client can work. + * [THRIFT-1652] - TSaslTransport does not log the error when kerberos auth fails + * [THRIFT-2272] - CLONE - Denial of Service attack in TBinaryProtocol.readString + * [THRIFT-2086] - Invalid generated code for Node.JS when using namespaces + * [THRIFT-1686] - t_php_generator.cc uses "and" instead of "&&", and causes compiler errors with Visual Studio + * [THRIFT-1693] - libthrift has dependency on two different versions of httpcore + * [THRIFT-1689] - don't exit(-1) in TNonblockingServer + * [THRIFT-1679] - NodeJS: protocol readString() should treat string as utf8, not binary + * [THRIFT-1721] - Dist broken due to 0.8.0 to 0.9.0 changes + * [THRIFT-1710] - Minor issues in test case code + * [THRIFT-1709] - Warning "Bitwise-or operator used on a sign-extended operand; consider casting to a smaller unsigned type first" in TBinaryProtocol.cs at ReadInt64() + * [THRIFT-1707] - [ruby] Adjust server_spec.rb for RSpec 2.11.x and Ruby 1.9.3 + * [THRIFT-1671] - Cocoa code generator does not put keywords into generated method calls + * [THRIFT-1670] - Incompatibilities between different versions of a Thrift interface + * [THRIFT-1669] - NameError: global name 'TApplicationException' is not defined + * [THRIFT-1668] - Compile error in contrib/fb303, thrift/TDispatchProcessor.h: No such file or directory + * [THRIFT-1845] - Fix compiler warning caused by implicit string conversion with Xcode 4.6 + * [THRIFT-304] - Building the Python library requires development headers + * [THRIFT-369] - sets and maps break equality + * [THRIFT-556] - Ruby compiler does not correctly referred to top-level modules when a submodule masks the top-level name + * [THRIFT-481] - indentation of ruby classes is off by a few + +## Improvement + * [THRIFT-1498] - Allow TThreadedPoolServer.Args to pass a ExecutorService + * [THRIFT-1444] - FunctionRunner - add syntactic sugar to create shared_ptrs + * [THRIFT-1443] - define a TProcessor helper class to implement process() + * [THRIFT-1441] - Generate constructor with parameters for exception class to let it update message property automatically. + * [THRIFT-1520] - Embed version number in erlang .app file + * [THRIFT-1480] - python: remove tabs, adjust whitespace and address PEP8 warnings + * [THRIFT-1485] - Performance: pass large and/or refcounted arguments as "const" + * [THRIFT-1484] - Introduce phpunit test suite + * [THRIFT-1532] - The type specifications in the generated Erlang code should include "undefined" where it's used as a default value + * [THRIFT-1534] - Required fields in the Delphi code generator. + * [THRIFT-1469] - Java isset space optimization + * [THRIFT-1465] - Visibility of methods in generated java code + * [THRIFT-1453] - Don't change types of arguments when serializing with thrift php extension + * [THRIFT-1452] - generate a swap() method for all generated structs + * [THRIFT-1451] - FramedTransport: Prevent infinite loop when writing + * [THRIFT-1521] - Two patches for more Performance + * [THRIFT-1555] - Delphi version of the tutorial code + * [THRIFT-1535] - Why thrift don't use wrapped class for optional fields ? + * [THRIFT-1204] - Ruby autogenerated files should require 'thrift' gem + * [THRIFT-1344] - Using the httpc module directly rather than the deprecated http layer + * [THRIFT-1343] - no_auto_import min/2 to avoid compile warning + * [THRIFT-1340] - Add support of ARC to Objective-C + * [THRIFT-1611] - Improved code generation for typedefs + * [THRIFT-1593] - Pass on errors like "connection closed" to the handler module + * [THRIFT-1615] - PHP Namespace + * [THRIFT-1567] - Thrift/cpp: Allow alternate classes to be used for + * [THRIFT-1072] - Missing - (id) initWithSharedProcessor in TSharedProcessorFactory.h + * [THRIFT-1650] - [ruby] Update clean items and svn:ignore entries for OS X artifacts + * [THRIFT-1661] - [PATCH] Add --with-qt4 configure option + * [THRIFT-1675] - Do we have any plan to support scala? + * [THRIFT-1645] - Replace Object#tee with more conventional Object#tap in specs + * [THRIFT-1644] - Upgrade RSpec to 2.10.x and refactor specs as needed + * [THRIFT-1672] - MonoTouch (and Mono for Android) compatibility + * [THRIFT-1702] - a thrift manual + * [THRIFT-1694] - Re-Enable serialization for WP7 Silverlight + * [THRIFT-1691] - Serializer/deserializer support for Delphi + * [THRIFT-1688] - Update IDL page markup + * [THRIFT-1725] - Tutorial web pages for Delphi and C# + * [THRIFT-1714] - [ruby] Explicitly add CWD to Ruby test_suites.rb + * [THRIFT-317] - Issues with Java struct validation + * [THRIFT-164] - Build web tutorial on Incubator web site + * [THRIFT-541] - Cocoa code generator doesn't put keywords before all arguments. + * [THRIFT-681] - The HTML generator does not handle JavaDoc style comments very well + +## New Feature + * [THRIFT-1500] - D programming language support + * [THRIFT-1510] - There should be an implementation of the JsonProtocol for ruby + * [THRIFT-1115] - python TBase class for dynamic (de)serialization, and __slots__ option for memory savings + * [THRIFT-1953] - support for asp.net mvc 3 + +## Question + * [THRIFT-1235] - How could I use THttpServerTransportFactory withTNonBlockingServer + * [THRIFT-1368] - TNonblockingServer usage + * [THRIFT-1061] - Read an invalid frame size of 0. Are you using TFramedTransport on the client side? + * [THRIFT-491] - Ripping raw pthreads out of TFileTransport and associated test issues + +## Sub-task + * [THRIFT-1596] - Delphi: Test clients should have a return codes that reflect whether they succeeded or not + * [THRIFT-982] - javame: add version Info to the library + * [THRIFT-1722] - C# WP7 Assembly addition beaks mono build + * [THRIFT-336] - Compact Protocol in C# + +## Test + * [THRIFT-1613] - Add code back into empty source file ToStringTest.java + * [THRIFT-1718] - Incorrect check in TFileTransportTest + +## Wish + * [THRIFT-1463] - Decouple Thrift IDL from generators + * [THRIFT-1466] - Proper Documentation for Thrift C Glib + * [THRIFT-1539] - Build and distribute the fb303 python libraries along with thrift + * [THRIFT-1685] - Please add "aereo.com" to "Powered by Apache Thrift" list in about page + * [THRIFT-330] - TProcessor - additional method to called when connection is broken + + + +Thrift 0.8 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1436] - pip install thrift fails on Windows with "Unable to find vcvarsall.bat" + * [THRIFT-1432] - Javascript struct constants declared in the same file as their struct definition will cause an error + * [THRIFT-1428] - shared.thrft does not include namespace for php, so thrift compiler generate incorrect name + * [THRIFT-1426] - Dist package missing files for release 0.8 + * [THRIFT-1425] - The Node package is incompatible with latest node (0.6) & npm (1.0.27) + * [THRIFT-1416] - Python Unit test is broken on ci + * [THRIFT-1419] - AbstractNonBlockingServer does not catch errors when invoking the processor + * [THRIFT-1424] - Ruby specs fail when run with rake + * [THRIFT-1420] - Nonblocking and HsHa server should make sure to close all their socket connections when the selector exits + * [THRIFT-1413] - Generated code does not read MapEnd / ListEnd / SetEnd + * [THRIFT-1409] - Name conflict check does not work properly for exception object(Delphi). + * [THRIFT-1408] - Delphi Test Server: Exception test case fails due to naming conflict with e.message + * [THRIFT-1407] - Typo in Python socket server causes Thrift to fail when we enable a global socket timout + * [THRIFT-1397] - CI server fails during build due to unused parameters in delphi generator + * [THRIFT-1404] - Delphi compiler generates struct reader code with problem. + * [THRIFT-1400] - Ruby native extension aborts with __stack_chk_fail in OSX + * [THRIFT-1399] - One of the TServerImpl.Create CTORs lacks implementation + * [THRIFT-1390] - Debian packages build fix for Squeeze (build from the official 0.7.0 tarball) + * [THRIFT-1393] - TTransportException's thrown from THttpClient contain superfluous slashes in the Exception message + * [THRIFT-1392] - Enabling both namespaces and autoloading in generated PHP code won't work. + * [THRIFT-1406] - Build error after applying THRIFT-1395 + * [THRIFT-1405] - Delphi compiler does not generates container serializer properly. + * [THRIFT-1411] - java generator does not provide type parameter for TBaseProcessor + * [THRIFT-1473] - JSON context stack may be left in an incorrect state when an exception is thrown during read or write operations + * [THRIFT-1331] - Ruby library deserializes an empty map to nil + * [THRIFT-1330] - PHP Namespaces no longer generated + * [THRIFT-1328] - TBaseHelper.toString(...) appends ByteBuffer data outside of valid buffer range + * [THRIFT-1322] - OCaml lib fail to compile: Thrift.ml line 305, int vs int32 mismatch + * [THRIFT-1143] - Build doesn't detect correct architecture type on 64bit osx + * [THRIFT-1205] - port server unduly fragile with arbitrary input + * [THRIFT-1279] - type set is handled incorrectly when writing object + * [THRIFT-1298] - Standard scheme doesn't read or write metadata along with field values + * [THRIFT-1265] - C++ container deserialize + * [THRIFT-1263] - publish ruby client to rubygems + * [THRIFT-1384] - Java help menu missing newline near javame flag + * [THRIFT-1382] - Bundle install doesnot work because thrift crashes + * [THRIFT-1381] - Thrift C++ libs have incorrectly versioned names + * [THRIFT-1350] - Go library code does not build as of r60 (most recent release) + * [THRIFT-1365] - TupleProtocol#writeBitSet unintentionally writes a variable length byte array + * [THRIFT-1359] - --gen-cob cpp:cob_style does not compile anymore + * [THRIFT-1319] - Mismatch between how a union reads and writes a container + * [THRIFT-1309] - libfb303-0.7.0.jar missing in maven repository + * [THRIFT-1238] - Thrift JS client cannot read map of structures + * [THRIFT-1254] - Code can't be compiled against a regular JRE: Object.clone() override has a different return type + * [THRIFT-1367] - Mac OSX build fails with "no such file to load -- spec/rake/spectask" + * [THRIFT-1355] - Running make in lib/rb doesn't build the native extensions + * [THRIFT-1370] - Debian packaging should Build-Depend on libglib2.0-dev + * [THRIFT-1342] - Compilation problem on Windows of fastbinary.c + * [THRIFT-1341] - TProtocol.h endian detection wrong with boost + * [THRIFT-1583] - c_glib leaks memory + * [THRIFT-1582] - Bad includes of nested thrift files in c_glib + * [THRIFT-1578] - C_GLib generated code does not compile + * [THRIFT-1027] - 'make -j 16' fails with "unterminated #ifdef" error + * [THRIFT-1121] - Java server performance regression in 0.6 + * [THRIFT-857] - tests run by "make install" fail if generators are disabled + * [THRIFT-380] - Use setuptools for python build + +## Dependency upgrade + * [THRIFT-1257] - thrift's dependency scope on javax.servlet:servlet-api should be 'provided' + +## Improvement + * [THRIFT-1445] - minor C++ generator variable cleanup + * [THRIFT-1435] - make TException.Message property conformant to the usual expectations + * [THRIFT-1431] - Rename 'sys' module to 'util' + * [THRIFT-1396] - Dephi generator has dependacy on boost 1.42 later. + * [THRIFT-1395] - Patch to prevent warnings for integer types in some cases + * [THRIFT-1275] - thrift: always prefix namespaces with " ::" + * [THRIFT-1274] - thrift: fail compilation if an unexpected token is + * [THRIFT-1271] - thrift: fix missing namespace in generated local + * [THRIFT-1270] - thrift: add --allow-neg-keys argument to allow + * [THRIFT-1345] - Allow building without tests + * [THRIFT-1286] - Modernize the Thrift Ruby Library Dev Environment + * [THRIFT-1284] - thrift: fix processor inheritance + * [THRIFT-1283] - thrift: wrap t_cpp_generator::generate_process_function() to 80 + * [THRIFT-1282] - Upgrade httpclient to 4.1.2 (from 4.0.1) + * [THRIFT-1281] - add @generated to the docblock + * [THRIFT-1280] - Thrift: Improve Monitor exception-free interfaces + * [THRIFT-1278] - javadoc warnings - compilation + * [THRIFT-1227] - Erlang implementation of thrift JSON protocol + * [THRIFT-1295] - Duplicate include in TSocket.cpp + * [THRIFT-1294] - thrift: fix log message typos in TSimpleServer + * [THRIFT-1293] - thrift: improve handling of exceptions thrown by + * [THRIFT-1292] - thrift: silence log spew from TThreadedServer + * [THRIFT-1288] - Allow typedefed exceptions in throws clauses + * [THRIFT-1290] - thrift: TNonblockingServer: clean up state in the + * [THRIFT-1287] - thrift: start refactoring some of the C++ processor + * [THRIFT-1289] - thrift: implement TNonblockingServer::stop() + * [THRIFT-1305] - thrift: make TConnection a private inner class of + * [THRIFT-1304] - TNonblockingServer: pass in the connection context to + * [THRIFT-1302] - thrift: raise an exception if send() times out in + * [THRIFT-1301] - thrift: consolidate common code in TNonblockingServer + * [THRIFT-1377] - abort PHP deserialization on unknown field type + * [THRIFT-1379] - fix uninitialized enum values in thrift C++ objects + * [THRIFT-1376] - Make port specification option in thrift remote + * [THRIFT-1375] - fixed a hex char conversion bug in TJSONProtocol + * [THRIFT-1373] - Fix user-defined exception generation in thrift (python) + * [THRIFT-1361] - Optional replacement of pthread by boost::thread + * [THRIFT-1320] - Consistency of configure generated config.h + * [THRIFT-1317] - Remove copy constructibility from + * [THRIFT-1316] - thrift: update server classes to accept + * [THRIFT-1315] - thrift: generate server interface factory classes + * [THRIFT-1314] - thrift: add TProcessorFactory + * [THRIFT-1335] - Add accept timeout to TServerSocket + * [THRIFT-1334] - Add more info to IllegalStateException + * [THRIFT-1333] - Make RWGuard not copyable + * [THRIFT-1332] - TSSLTransportParameters class uses hard coded value keyManagerType: SunX509 + * [THRIFT-1251] - Generated java code should indicate which fields are required and which are optional + * [THRIFT-1387] - Build MSVC libraries with Boost Threads instead of Pthreads + * [THRIFT-1339] - Extend Tuple Protocol to TUnions + * [THRIFT-1031] - Patch to compile Thrift for vc++ 9.0 and 10.0 + * [THRIFT-1130] - Add the ability to specify symbolic default value for optional boolean + * [THRIFT-1123] - Patch to compile Thrift server and client for vc++ 9.0 and 10.0 + * [THRIFT-386] - Make it possible to build the Python library without the extension + +## New Feature + * [THRIFT-1401] - JSON-protocol for Delphi XE Libraries + * [THRIFT-1167] - Java nonblocking server with more than one thread for select and handling IO + * [THRIFT-1366] - Delphi generator, lirbrary and unit test. + * [THRIFT-1354] - Add rake task to build just the gem file + * [THRIFT-769] - Pluggable Serializers + +## Sub-task + * [THRIFT-1415] - delphi: add version Info to the library + * [THRIFT-1391] - Improved Delphi XE test cases + + + +Thrift 0.7 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1140] - Framed Transport Client using C (Glib) Library hangs when connecting to Ruby Server + * [THRIFT-1154] - HttpClient does not specify the connection close parameter + * [THRIFT-1153] - HttpClient does not specify the connection close parameter + * [THRIFT-1149] - Nonblocking server fails when client connection is reset + * [THRIFT-1146] - Android Incompatibility : in Android < 2.3 java.io.IOException doesn't support for Throwable parameter in constructor + * [THRIFT-1133] - Java and JavaScript tutorial is broken since we have Java maven deployment + * [THRIFT-1132] - Deserialization error in TApplicationException C# + * [THRIFT-1131] - C# JSON Protocol is unable to decode escaped characters in string + * [THRIFT-1208] - python TCompactProtocol.py writeBool and readBool not follow the compact-proto-spec-2.txt spec for CONTAINER_WRITE, CONTAINER_READ + * [THRIFT-1200] - JS compiler generates code that clobbers existing namespaces + * [THRIFT-1183] - Pure-ruby CompactProtocol raises ArgumentError when deserializing under Ruby 1.9 + * [THRIFT-1182] - Native deserializer segfaults on incorrect list element type + * [THRIFT-1181] - AS3 compiler generates incorrect code for setting default values in constructor + * [THRIFT-1234] - thrift --help is missing doc on py:utf8strings + * [THRIFT-1180] - AS3 compiler generates uncompilable code for binary types. + * [THRIFT-1194] - Java lib does not install artifacts to local dir correctly + * [THRIFT-1193] - Potential infinite loop in nonblocking_server + * [THRIFT-1192] - Typo: TProtocol.h tests for HAVE_SYS_PARAM_H_ + * [THRIFT-1190] - readBufferBytesAllocated in TNonblockingServer.java should be AtomicLong to fix FD leakage and general server malfunction + * [THRIFT-1187] - nonblocking_server shutdown race under Ruby 1.9 + * [THRIFT-1178] - Java: TBase signature should be T extends TBase + * [THRIFT-1164] - Segmentation fault on NULL pointer in t_js_generator::generate_const + * [THRIFT-1171] - Perl write/readDouble assumes little-endian platform + * [THRIFT-1222] - Unhandled exception for TEvhttpServer request + * [THRIFT-1220] - TProcessor::process never returns false + * [THRIFT-1285] - Stable 0.7.0 Windows compiler exe available on the webside is not the good one + * [THRIFT-1218] - c_glib uses wrong name in pkg-config + * [THRIFT-1215] - Undefined property Thirft in lib/js/thrift.js + * [THRIFT-1211] - When using THttpClient, non 200 responses leave the connection open + * [THRIFT-1228] - The php accelerator module calls flush incorrectly + * [THRIFT-1308] - libfb303-0.7.0.jar missing in maven repository + * [THRIFT-1255] - Mismatch of method name between JavaME's lib and generated code (compareTo/compareObjects) + * [THRIFT-1253] - Code generated for maps is not compiling + * [THRIFT-1252] - Segfault in Ruby deserializer + * [THRIFT-1094] - bug in TCompactProto python readMessageEnd method and updated test cases + * [THRIFT-1093] - several bugs in python TCompactProtocol + * [THRIFT-1092] - generated validate() method has wrong indentation + * [THRIFT-1011] - Error generating package imports when using classes from other packages + * [THRIFT-1050] - Declaring an argument named "manager" to a service method produces code that fails compile due to name conflicts with protected ivars in TAsyncClient + * [THRIFT-1074] - .keystore and .truststore are missing from the 0.6.0 distribution + * [THRIFT-1067] - Tons of bugs in php implementation + * [THRIFT-1065] - Unexpected exceptions not proper handled on JS + * [THRIFT-1076] - Erlang Thrift socket server has a bug that causes java thrift client of framed binary client to throw "out of sequence" exception + * [THRIFT-1057] - casts in TBinaryProtocol.tcc causing "dereferencing type-punned pointer will break strict-aliasing rules" warnings from gcc + * [THRIFT-1055] - csharp TServerSocket and TSocket do not disable Nagle via Socket.NoDelay = true like cpp and java do + * [THRIFT-1054] - explicit call to PKG_PROG_PKG_CONFIG is missing and first use of PKG_CHECK_MODULES may not happen, causes mono detection to fail + * [THRIFT-1117] - JavaScript Unit Test does not work anymore because libthrift*.jar where moved by Maven Deployment + * [THRIFT-1111] - The HTML generator does not distinguish between string and binary types + * [THRIFT-1032] - "make dist" fails due to c_glib problem + * [THRIFT-1036] - Auto-generated C++ code fails to compile with "-Werror -Wextra -Wall" g++ compiler flags + * [THRIFT-1041] - TDeserializer holds onto a reference of the array it reads after it is done deserializing + * [THRIFT-1106] - C++ code TAsyncProtocolProcessor.h & TAsyncBufferProcessor.h dont have virtual functions but no virtual destructor. Causes warnings on -Wall + * [THRIFT-1105] - OCaml generator does not prefix methods of included structs with their type + * [THRIFT-1104] - INSTALLDIRS should be included in configure script + * [THRIFT-1102] - typo in configure.ac: "==" operator in 'test' (instead of"'=") + * [THRIFT-1101] - bytebuffer length calculation in TBinaryProtocol writeBinary + * [THRIFT-1098] - Undefined properties in TBinaryProtocolFactory + * [THRIFT-1081] - PHP tests broken and somewhat incomplete + * [THRIFT-1080] - erlang test's 'make' fails on Mac OSX + * [THRIFT-1078] - ThriftTest.thrift generates invalid PHP library + * [THRIFT-1120] - proto.WriteListEnd being called in the wrong place + * [THRIFT-1119] - TJSONProtocol fails to UTF8 decode strings + * [THRIFT-867] - PHP accelerator module's output transport is incompatible with TFramedTransport + * [THRIFT-826] - PHP TSocket Write Timeout + * [THRIFT-835] - Bad AS3 syntax in constructors that set default values + * [THRIFT-788] - thrift_protocol.so: multiget/multiget_slice does not handle more than 17 keys correctly + * [THRIFT-125] - OCaml libraries don't compile with 32-bit ocaml + * [THRIFT-342] - PHP: can't have sets of complex types + * [THRIFT-731] - configure doesn't check for ant >= 1.7 + * [THRIFT-690] - Update TApplicationException codes + * [THRIFT-638] - BufferedTransport + C extensions block until recv timeout is reached on last fread call + +## Dependency upgrade + * [THRIFT-1177] - Update thrift to reflect changes in Go's networking libraries + +## Improvement + * [THRIFT-1155] - Remove log4j dependency from java client + * [THRIFT-1151] - Produce more informative runtime error in case of schema and data mismatch during serialization + * [THRIFT-1207] - Support DESTDIR on "make install" of ruby libs + * [THRIFT-1199] - Union structs should have generated methods to test whether a specific field is currently set + * [THRIFT-1233] - Remove unused include in generated C++ code + * [THRIFT-1189] - Ruby deserializer speed improvements + * [THRIFT-1170] - Thrift Generated Code and Java 5 + * [THRIFT-1174] - Publish as3 client implementation via Maven for use by flex-mojos users + * [THRIFT-1225] - TCompactProtocol for PHP + * [THRIFT-1221] - Remove SimpleCallback.h + * [THRIFT-1217] - Use evutil_socketpair instead of pipe (Windows port) + * [THRIFT-1216] - build Java Library behind a proxy + * [THRIFT-1231] - Remove bogus include + * [THRIFT-1213] - Membuffer should provide a way to get back the buffer + * [THRIFT-1237] - Java fb303 missing some methods + * [THRIFT-1063] - Fix Erlang Tutorial Files + * [THRIFT-1053] - Make remote client's IP address available for all socket related transports + * [THRIFT-1109] - Deploy fb303 along side libthrift to maven repo + * [THRIFT-1107] - improvement for compiler-generated python for 'None' object comparisons + * [THRIFT-1069] - Add command line option to prevent thrift from inserting gen-* directories + * [THRIFT-1049] - Allow for TServerSocket python library to bind to a specific host + * [THRIFT-1126] - Extending struct_info for erlang bindings + * [THRIFT-1100] - python TSSLSocket improvements, including certificate validation + * [THRIFT-994] - Don't try to invoke phpize if we don't have it + * [THRIFT-993] - Some improvements in C++ stubs for oneway operations + * [THRIFT-997] - Using valueOf for base types in getFieldValue + * [THRIFT-418] - Don't do runtime sorting of struct fields + * [THRIFT-151] - TSSLServerSocket and TSSLSocket implementation + * [THRIFT-27] - Generated erlang types don't contain default values for records + * [THRIFT-113] - to-string methods should omit optional null fields from output + * [THRIFT-363] - Maven Deploy + * [THRIFT-447] - Make an abstract base Client class so we can generate less code + * [THRIFT-627] - should c++ have setters for optional fields? + +## New Feature + * [THRIFT-1236] - Erlang Reconnecting Thrift Client + * [THRIFT-1021] - Framed transport support for OCaml + * [THRIFT-1068] - Python SSL Socket Support + * [THRIFT-1103] - TZlibTransport for python, a zlib compressed transport + * [THRIFT-1083] - Preforking python process pool server + * [THRIFT-999] - Add TForkingServer + +## Sub-task + * [THRIFT-1152] - Attributes from private to protected + * [THRIFT-1038] - Generated Java code for structures containing binary fields (or collections thereof) are not serializable (in the Java sense) even though they implement java.io.Serializable + +## Task + * [THRIFT-892] - Refactor erlang build system with rebar + +## Wish + * [THRIFT-625] - Add support for 'Go' + + + +Thrift 0.6.1 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1133] - Java and JavaScript tutorial is broken since we have Java maven deployment + * [THRIFT-1131] - C# JSON Protocol is unable to decode escaped characters in string + * [THRIFT-1074] - .keystore and .truststore are missing from the 0.6.0 distribution + +## Improvement + * [THRIFT-1109] - Deploy fb303 along side libthrift to maven repo + * [THRIFT-363] - Maven Deploy + +## Question + * [THRIFT-1206] - did the THRIFT 0.6.1 merge THRIFT-563 ? + +## Sub-task + * [THRIFT-1163] - How can i use multi service in one program? + +## Task + * [THRIFT-1112] - Apply THRIFT-363 to 0.6 branch + * [THRIFT-1113] - Apply THRIFT-1074 to 0.6 branch + + + +Thrift 0.6 +-------------------------------------------------------------------------------- +## Bug + * [THRIFT-1020] - OCaml compiler generates invalid OCaml + * [THRIFT-1015] - TUnion does not handle ByteBuffer in toString + * [THRIFT-1013] - generated java code may have name clashes with thrift library + * [THRIFT-1009] - TUnion does not correctly deep copy a ByteBuffer + * [THRIFT-1032] - "make dist" fails due to c_glib problem + * [THRIFT-868] - Referencing constant values doesn't work with with typedef types + * [THRIFT-971] - java module can't be compiled without ivy and network connection + * [THRIFT-970] - Under heavy load, THttpClient may fail with "too many open files" + * [THRIFT-969] - Java Tutorial broken, move CalculatorHandler to a separate file + * [THRIFT-807] - JavaScript: Initialization of Base Types with 0 instead of null + * [THRIFT-955] - Thrift compiler for Windows uses lowercase names and directories which is inconsistent with compiling on other platforms + * [THRIFT-992] - Naming convention in C# constructor is not consistent with other fields causes compile errors + * [THRIFT-1008] - byte[] accessors throw NPE on unset field + * [THRIFT-1006] - Impossible to correctly qualify an enum constant in an external thrift file + * [THRIFT-950] - Haskell bindings treat 'byte' as unsigned 8-bit int (Data.Word.Word8), java/cpp as signed (byte/int8_t). + * [THRIFT-975] - lib/c_glib/README is missing => breaks make dist + * [THRIFT-944] - Support all version-4s of base + * [THRIFT-939] - optional binary fields throw NPE on default byte[] getters + * [THRIFT-935] - PHP Extension aborts the build if php-config is not installed + * [THRIFT-933] - Haskell's Thrift.cabal has warnings + * [THRIFT-932] - Haskell tests need to be run through 'make check' (and probably 'cabal check') too + * [THRIFT-904] - C# TSocket should disable nagle and linger + * [THRIFT-941] - Make PHP C Extension use the defined Protocol writeMessageBegin function + * [THRIFT-940] - 'make check' fails if boost is not in the std include and link paths + * [THRIFT-924] - Fix generated php structure constants + * [THRIFT-979] - ruby bindings used to work on jruby + * [THRIFT-977] - Hex Conversion Bug in C++ TJSONProtocol + * [THRIFT-347] - PHP TSocket Timeout Issues + * [THRIFT-517] - TExceptions thrown by server result in cryptic error message on client - Tried to read 4 bytes, but only got 0 bytes + +## Improvement + * [THRIFT-1024] - Add Python Twisted example to the Tutorial + * [THRIFT-958] - Change accessmodifer on trans_ field in the FrameBuffer class to public. + * [THRIFT-957] - THsHaServer: Change access modifier of the invoker field. + * [THRIFT-1002] - CodeStyle: t_c_glib_generator.cc + * [THRIFT-1005] - Give unions byte[] signature methods to go along with their ByteBuffer counterparts + * [THRIFT-951] - Add a new isServing() method to TServer + * [THRIFT-943] - Silly readme typo fix. + * [THRIFT-961] - JavaScript TestSuite using ant/ivy and Java's ServerTestBase Handler + * [THRIFT-960] - add TestServer, TestNonblockingServer and TestClient again + * [THRIFT-949] - Modify the TEnum interface so it defines a method similar to findByValue + * [THRIFT-946] - Augment FieldValueMetaData so it differentiates 'string' and 'binary' fields. + * [THRIFT-903] - custom ThreadFactory in THsHaServer + * [THRIFT-913] - Test Case for Url encoded strings + simple enhancement to lib/js/test/RunTestServer.sh + * [THRIFT-926] - Miscellaneous C++ improvements + * [THRIFT-929] - Improvements to the C++ test suite + * [THRIFT-893] - add JavaScript to the tutorial examples + * [THRIFT-1003] - Polishing c_glib code + * [THRIFT-71] - Debian packaging for thrift + +## New Feature + * [THRIFT-1033] - Node.js language target + * [THRIFT-947] - Provide a helper method to determine the TProtocol used to serialize some data. + * [THRIFT-928] - Make more statistics available in C++ servers + * [THRIFT-922] - Templatized [de]serialization code for C++ + * [THRIFT-923] - Event-driven client and server support for C++ + * [THRIFT-925] - Provide name<->value map for enums in C++ + * [THRIFT-927] - Add option to modify the PHP include path + * [THRIFT-377] - TFileTransport port in Java + * [THRIFT-106] - TSSLServerSocket + * [THRIFT-582] - C implementation of Thrift + * [THRIFT-745] - Make it easier to instantiate servers + +## Sub-task + * [THRIFT-1038] - Generated Java code for structures containing binary fields (or collections thereof) are not serializable (in the Java sense) even though they implement java.io.Serializable + +## Task + * [THRIFT-862] - Async client issues / improvements + +## Test + * [THRIFT-581] - Add a testsuite for txThrift (Twisted) + + + +Thrift 0.5.0 - Incubating +-------------------------------------------------------------------------------- +THRIFT-505 Build Make configure give a summary of the enabled components (David Reiss) +THRIFT-506 Build Allow Thrift to be built without the C++ library (David Reiss) +THRIFT-844 Build Build Requirements state autoconf 2.59+ is required, but 2.60+ is needed (Harlan Lieberman-Berg) +THRIFT-850 Build Perl runtime requires Bit::Vector which may not be installed by default, but configure does not fail (Michael Lum) +THRIFT-854 Build Provide configure option and make rules to build/install php extension (Anthony Molinaro) +THRIFT-858 Build Have bootstrap.sh check for a suitable autoconf version before running (David Reiss) +THRIFT-871 Build Thrift compiler for WIndows (binary distribution) (David Reiss) +THRIFT-323 C# TJSONProtocol (Roger Meier) +THRIFT-634 C# C# Compiler Generates Incorrect Code For Fields which begin with an uppercase letter (Jon S Akhtar) +THRIFT-881 C# add csharp to the tutorial (Roger Meier) +THRIFT-856 C++ Building cpp library fails on OS X with malloc and free not being declared in scope (James Clarke) +THRIFT-865 C++ C++ compiler build depends on libfl even when flex/lex not detected (David Reiss) +THRIFT-900 C++ Unix domain socket (Roger Meier) +THRIFT-920 C++ C++ Test and Tutorial does not compile anymore due to the change within Enum handling (Roger Meier) +THRIFT-567 C++ Can't immediately stop a TSimpleServer thread that is idle (Rush Manbert) +THRIFT-756 C++ Exposing TSocket(int) constructor to public (Rajat Goel) +THRIFT-798 C++ TNonblockingServer leaks resources when destroyed (David Reiss) +THRIFT-812 C++, Python Demo of Thrift over ZeroMQ (David Reiss) +THRIFT-629 Cocoa Unused Field In TSocketServer Appears To Break iPhone Build (Jon S Akhtar) +THRIFT-838 Cocoa Generated Cocoa classes have useless @dynamic declarations (Kevin Ballard) +THRIFT-805 Cocoa Don't generate process_XXXX methods for oneway methods (Brad Taylor) +THRIFT-507 Compiler Remove the compiler's dependency on Boost (David Reiss) +THRIFT-895 Compiler (General) Thrift compiler does not allow two different enumerations to have the same key name for one of the enum values (David Reiss) +THRIFT-852 Compiler (General) Missing newline causes many compiler warnings (Anthony Molinaro) +THRIFT-877 Compiler (General) smalltalk namespace doesn't work (Bruce Lowekamp) +THRIFT-897 Compiler (General) Don't allow unqualified constant access to enum values (Bryan Duxbury) +THRIFT-9 Compiler (General) Add a default namespace declaration for all languages (David Reiss) +THRIFT-599 Erlang Don't use unnecessary processes in the Erlang transports and clients (David Reiss) +THRIFT-646 Erlang Erlang library is missing install target (David Reiss) +THRIFT-698 Erlang Generated module list should contain atoms, not strings (Anthony Molinaro) +THRIFT-866 Erlang term() in spec definitions seems to not work in erlang R12 (Anthony Molinaro) +THRIFT-886 Erlang Dialyzer warning (Anthony Molinaro) +THRIFT-785 Erlang Framed transport server problems (Anthony Molinaro) +THRIFT-884 HTML HTML Generator: add Key attribute to the Data Types Tables (Roger Meier) +THRIFT-652 Haskell Generated field name for strut is not capitalized correctly (Christian Lavoie) +THRIFT-743 Haskell compile error with GHC 6.12.1 (Christian Lavoie) +THRIFT-901 Haskell Allow the bindings to compile without -fglasgow-exts and with -Wall -Werror (Christian Lavoie) +THRIFT-905 Haskell Make haskell thrift bindings use automake to compile and install (Christian Lavoie) +THRIFT-906 Haskell Improve type mappings (Christian Lavoie) +THRIFT-914 Haskell Make haskell bindings 'easily' compilable (Christian Lavoie) +THRIFT-918 Haskell Make haskell tests run again (Christian Lavoie) +THRIFT-919 Haskell Update Haskell bindings README (Christian Lavoie) +THRIFT-787 Haskell Enums are not read correctly (Christian Lavoie) +THRIFT-250 Java ExecutorService as a constructor parameter for TServer (Ed Ceaser) +THRIFT-693 Java Thrift compiler generated java code that throws compiler warnings about deprecated methods. (Bryan Duxbury) +THRIFT-843 Java TNonblockingSocket connects without a timeout (Bryan Duxbury) +THRIFT-845 Java async client does not respect timeout (Ning Liang) +THRIFT-870 Java Java constants don't get Javadoc comments (Bryan Duxbury) +THRIFT-873 Java Java tests fail due to Too many open files (Todd Lipcon) +THRIFT-876 Java Add SASL support (Aaron T. Myers) +THRIFT-879 Java Remove @Override from TUnion.clear (Dave Engberg) +THRIFT-882 Java deep copy of binary fields does not copy ByteBuffer characteristics (arrayOffset, position) (Bryan Duxbury) +THRIFT-888 Java async client should also have nonblocking connect (Eric Jensen) +THRIFT-890 Java Java tutorial doesn't work (Todd Lipcon) +THRIFT-894 Java Make default accessors for binary fields return byte[]; provide new accessors to get ByteBuffer version (Bryan Duxbury) +THRIFT-896 Java TNonblockingSocket.isOpen() returns true even after close() (Eric Jensen) +THRIFT-907 Java libfb303 doesn't compile in 0.4.0 (Todd Lipcon) +THRIFT-912 Java Improvements and bug fixes to SASL implementation (Todd Lipcon) +THRIFT-917 Java THsHaServer should not accept an ExecutorService without catching RejectedExecutionException (Ed Ceaser) +THRIFT-931 Java Use log4j for Java tests (Todd Lipcon) +THRIFT-880 JavaME JavaME code generator and runtime library (Dave Engberg) +THRIFT-846 JavaScript JavaScript Test Framwork: extended Testcases (Roger Meier) +THRIFT-885 JavaScript Url encoded strings never get decoded? How do we fix this? (T Jake Luciani) +THRIFT-911 JavaScript (JavaScript compiler) Const structs, maps, sets, and lists generate a trailing comma (T Jake Luciani) +THRIFT-860 OCaml copy method and reset method (Lev Walkin) +THRIFT-682 PHP PHP extension doesn't compile on Mac OS X (Bryan Duxbury) +THRIFT-851 PHP php extension fails to compile on centos 5.x (Todd Lipcon) +THRIFT-840 Perl Perl protocol handler could be more robust against unrecognised types (Conrad Hughes) +THRIFT-758 Perl incorrect deference in exception handling (Yann Kerherve) +THRIFT-257 Python Support validation of required fields (Esteve Fernandez) +THRIFT-335 Python Compact Protocol for Python (David Reiss) +THRIFT-596 Python Make Python's TBufferedTransport use a configurable input buffer (David Reiss) +THRIFT-597 Python Python THttpServer performance improvements (David Reiss) +THRIFT-598 Python Allow Python's threading servers to use daemon threads (David Reiss) +THRIFT-666 Python Allow the handler to override HTTP responses in THttpServer (David Reiss) +THRIFT-673 Python Generated Python code has whitespace issues (Ian Eure) +THRIFT-721 Python THttpClient ignores url parameters (Thomas Kho) +THRIFT-824 Python TApplicationException.__str__() refers to class constants as globals (Peter Schuller) +THRIFT-855 Python Include optimized compiled python objects in install (Anthony Molinaro) +THRIFT-859 Python Allow py:twisted to be generated in different namespace than py (Bruce Lowekamp) +THRIFT-869 Python TSocket.py on Mac (and FreeBSD) doesn't handle ECONNRESET from recv() (Steven Knight) +THRIFT-875 Python Include python setup.cfg in dist (Anthony Molinaro) +THRIFT-610 Ruby binary_protocol.rb segfaults [line 86] (Unassigned) +THRIFT-899 Ruby Ruby read timeouts can sometimes be 2x what they should be (Ryan King) +THRIFT-909 Ruby allow block argument to struct constructor (Michael Stockton) +THRIFT-456 Test Suite Bad IP address string in test/cpp/src/main.cpp (Rush Manbert) + + +Thrift 0.4.0 - Incubating +-------------------------------------------------------------------------------- +THRIFT-650 Build Make Check fails on Centos/OSX with 0.2.0 tarball (Anthony Molinaro) +THRIFT-770 Build Get 'make dist' to work without first compiling source code (Anthony Molinaro) +THRIFT-160 C# Created THttpTransport for the C# library based on WebHttpRequest (Michael Greene) +THRIFT-834 C# THttpClient resends contents of message after transport errors (Anatoly Fayngelerin) +THRIFT-247 C++ THttpServer Transport (Unassigned) +THRIFT-676 C++ Change C++ code generator so that generated classes can be wrapped with SWIG (Unassigned) +THRIFT-570 Compiler Thrift compiler does not error when duplicate method names are present (Bruce Simpson) +THRIFT-808 Compiler Segfault when constant declaration references a struct field that doesn't exist (Bryan Duxbury) +THRIFT-646 Erlang Erlang library is missing install target (Anthony Molinaro) +THRIFT-544 General multiple enums with the same key generate invalid code (Ben Taitelbaum) +THRIFT-434 General ruby compiler should warn when a reserved word is used (Michael Stockton) +THRIFT-799 General Files missing proper Apache license header (Bryan Duxbury) +THRIFT-832 HTML HTML generator shows unspecified struct fields as 'required' (Bryan Duxbury) +THRIFT-226 Java Collections with binary keys or values break equals() (Bryan Duxbury) +THRIFT-484 Java Ability to use a slice of a buffer instead of a direct byte[] for binary fields (Bryan Duxbury) +THRIFT-714 Java maxWorkerThreads parameter to THsHaServer has no effect (Bryan Duxbury) +THRIFT-751 Java Add clear() method to TBase (Bryan Duxbury) +THRIFT-765 Java Improved string encoding and decoding performance (Bryan Duxbury) +THRIFT-768 Java Async client for Java (Bryan Duxbury) +THRIFT-774 Java TDeserializer should provide a partialDeserialize method for primitive types (Piotr Kozikowski) +THRIFT-783 Java .equals java method is broken on structs containing binary-type fields (Unassigned) +THRIFT-804 Java CompareTo is broken for unions set to map, set, or list (Bryan Duxbury) +THRIFT-814 Java Include a TServlet in the standard Thrift distribution (Mathias Herberts) +THRIFT-818 Java Async client doesn't send method args (Bryan Duxbury) +THRIFT-830 Java Switch binary field implementation from byte[] to ByteBuffer (Bryan Duxbury) +THRIFT-831 Java FramedTransport implementation that reuses its buffers (Bryan Duxbury) +THRIFT-833 Java build.xml in lib/java is missing a classpathref attribute for the javadoc task (Bryan Duxbury) +THRIFT-836 Java Race condition causes CancelledKeyException in TAsyncClientManager (Bryan Duxbury) +THRIFT-842 Java Upgrade to current version of commons-lang (2.5 instead of 2.4) and/or change dependency in ivy.xml to not be exact (Bryan Duxbury) +THRIFT-815 JavaScript Deserialization of lists is critically broken. (T Jake Luciani) +THRIFT-827 OCaml OCaml generator to take default values into account (Lev Walkin) +THRIFT-647 PHP PHP library is missing install target (Anthony Molinaro) +THRIFT-682 PHP PHP extension doesn't compile on Mac OS X (Bryan Duxbury) +THRIFT-718 PHP Thrift PHP library includes closing tags and extraneous whitespace (Nicholas Telford) +THRIFT-778 PHP PHP socket listening server (Nick Jones) +THRIFT-780 PHP PHP extension sometimes causes an abort with two exceptions at the same time (David Reiss) +THRIFT-837 PHP PHP accelerator bug for writes > 8k (Thomas Kho) +THRIFT-782 Perl Perl code for writing containers doesn't count length of write*Begin or write*End (Conrad Hughes) +THRIFT-395 Python Python library + compiler does not support unicode strings (Unassigned) +THRIFT-133 Ruby 'namespace ruby' should error out, or be an alias to 'namespace rb' (Bryan Duxbury) +THRIFT-664 Ruby Ruby extension fails to build with Ruby 1.9.1 (Rajesh Malepati) +THRIFT-699 Ruby Excise unused "native protocol method table" stuff from thrift_native (Bryan Duxbury) +THRIFT-767 Ruby ruby compiler does not keep comments for enum values (Bryan Duxbury) +THRIFT-811 Ruby http_client_transport.rb: allow custom http headers (Tony Kamenick) +THRIFT-459 Ruby Ruby installation always tries to write to /Library/Ruby/site (Matthieu Imbert) + + +Thrift 0.1.0 - Incubating (not released) +-------------------------------------------------------------------------------- +Compatibility Breaking Changes: + C++: + * It's quite possible that regenerating code and rebuilding will be + required. Make sure your headers match your libs! + + Java: + + Python: + + Ruby: + * Generated files now have underscored names [THRIFT-421] + * The library has been rearranged to be more Ruby-like [THRIFT-276] + + Erlang: + * Generated code will have to be regenerated, and the new code will + have to be deployed atomically with the new library code [THRIFT-136] + +New Features and Bug Fixes: + C++: + * Support for TCompactProtocol [THRIFT-333] + + Java: + * Support for TCompactProtocol [THRIFT-110] + + Python: + * Support for Twisted [THRIFT-148] + + Ruby: + * Support for TCompactProtocol [THRIFT-332] + diff --git a/vendor/github.com/apache/thrift/CMakeLists.txt b/vendor/github.com/apache/thrift/CMakeLists.txt new file mode 100644 index 0000000000000..93ed8d2ac58e9 --- /dev/null +++ b/vendor/github.com/apache/thrift/CMakeLists.txt @@ -0,0 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +cmake_minimum_required(VERSION 2.8.12) + +project("Apache Thrift") + +set(CMAKE_MODULE_PATH "${CMAKE_MODULE_PATH}" "${CMAKE_CURRENT_SOURCE_DIR}/build/cmake") + +# TODO: add `git rev-parse --short HEAD` +# Read the version information from the Autoconf file +file (STRINGS "${CMAKE_CURRENT_SOURCE_DIR}/configure.ac" CONFIGURE_AC REGEX "AC_INIT\\(.*\\)" ) + +# The following variable is used in the version.h.in file +string(REGEX REPLACE "AC_INIT\\(\\[.*\\], \\[([0-9]+\\.[0-9]+\\.[0-9]+(-dev)?)\\]\\)" "\\1" PACKAGE_VERSION ${CONFIGURE_AC}) +message(STATUS "Parsed Thrift package version: ${PACKAGE_VERSION}") + +# These are internal to CMake +string(REGEX REPLACE "([0-9]+\\.[0-9]+\\.[0-9]+)(-dev)?" "\\1" thrift_VERSION ${PACKAGE_VERSION}) +string(REGEX REPLACE "([0-9]+)\\.[0-9]+\\.[0-9]+" "\\1" thrift_VERSION_MAJOR ${thrift_VERSION}) +string(REGEX REPLACE "[0-9]+\\.([0-9])+\\.[0-9]+" "\\1" thrift_VERSION_MINOR ${thrift_VERSION}) +string(REGEX REPLACE "[0-9]+\\.[0-9]+\\.([0-9]+)" "\\1" thrift_VERSION_PATCH ${thrift_VERSION}) +message(STATUS "Parsed Thrift version: ${thrift_VERSION} (${thrift_VERSION_MAJOR}.${thrift_VERSION_MINOR}.${thrift_VERSION_PATCH})") + +# Some default settings +include(DefineCMakeDefaults) + +# Build time options are defined here +include(DefineOptions) +include(DefineInstallationPaths) + +# Based on the options set some platform specifics +include(DefinePlatformSpecifc) + +# Generate the config.h file +include(ConfigureChecks) + +# Package it +include(CPackConfig) + + +find_package(Threads) + +include(CTest) +if(BUILD_TESTING) + message(STATUS "Building with unittests") + + enable_testing() + # Define "make check" as alias for "make test" + add_custom_target(check COMMAND ctest) +else () + message(STATUS "Building without tests") +endif () + +if(BUILD_COMPILER) + if(NOT EXISTS ${THRIFT_COMPILER}) + set(THRIFT_COMPILER $) + endif() + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/compiler/cpp) +elseif(EXISTS ${THRIFT_COMPILER}) + add_executable(thrift-compiler IMPORTED) + set_property(TARGET thrift-compiler PROPERTY IMPORTED_LOCATION ${THRIFT_COMPILER}) +endif() + +if(BUILD_CPP) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/lib/cpp) + if(BUILD_TUTORIALS) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/tutorial/cpp) + endif() + if(BUILD_TESTING) + if(WITH_LIBEVENT AND WITH_ZLIB AND WITH_OPENSSL) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/test/cpp) + else() + message(WARNING "libevent and/or ZLIB and/or OpenSSL not found or disabled; will not build some tests") + endif() + endif() +endif() + +if(BUILD_C_GLIB) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/lib/c_glib) +endif() + +if(BUILD_JAVA) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/lib/java) +endif() + +if(BUILD_PYTHON) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/lib/py) + if(BUILD_TESTING) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/test/py) + endif() +endif() + +if(BUILD_HASKELL) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/lib/hs) + if(BUILD_TESTING) + add_subdirectory(${CMAKE_CURRENT_SOURCE_DIR}/test/hs) + endif() +endif() + +PRINT_CONFIG_SUMMARY() diff --git a/vendor/github.com/apache/thrift/CONTRIBUTING.md b/vendor/github.com/apache/thrift/CONTRIBUTING.md new file mode 100644 index 0000000000000..316da9a00de83 --- /dev/null +++ b/vendor/github.com/apache/thrift/CONTRIBUTING.md @@ -0,0 +1,49 @@ +## How to contribute + 1. Help to review and verify existing patches + 1. Make sure your issue is not all ready in the [Jira issue tracker](http://issues.apache.org/jira/browse/THRIFT) + 1. If not, create a ticket describing the change you're proposing in the [Jira issue tracker](http://issues.apache.org/jira/browse/THRIFT) + 1. Contribute your patch using one of the two methods below + +### Contributing via a patch + +1. Check out the latest version of the source code + + * git clone https://git-wip-us.apache.org/repos/asf/thrift.git thrift + +1. Modify the source to include the improvement/bugfix + + * Remember to provide *tests* for all submited changes + * When bugfixing: add test that will isolate bug *before* applying change that fixes it + * Verify that you follow [Thrift Coding Standards](/docs/coding_standards) (you can run 'make style', which ensures proper format for some languages) + +1. Create a patch from project root directory (e.g. you@dev:~/thrift $ ): + + * git diff > ../thrift-XXX-my-new-feature.patch + +1. Attach the newly generated patch to the issue +1. Wait for other contributors or committers to review your new addition +1. Wait for a committer to commit your patch + +### Contributing via GitHub pull requests + +1. Create a fork for http://github.com/apache/thrift +1. Create a branch for your changes(best practice is issue as branch name, e.g. THRIFT-9999) +1. Modify the source to include the improvement/bugfix + + * Remember to provide *tests* for all submited changes + * When bugfixing: add test that will isolate bug *before* applying change that fixes it + * Verify that you follow [Thrift Coding Standards](/docs/coding_standards) (you can run 'make style', which ensures proper format for some languages) + * Verify that your change works on other platforms by adding a GitHub service hook to [Travis CI](http://docs.travis-ci.com/user/getting-started/#Step-one%3A-Sign-in) and [AppVeyor](http://www.appveyor.com/docs) + +1. Commit and push changes to your branch (please use issue name and description as commit title, e.g. THRIFT-9999 make it perfect) +1. Issue a pull request with the jira ticket number you are working on in it's name +1. Wait for other contributors or committers to review your new addition +1. Wait for a committer to commit your patch + +### More info + + Plenty of information on why and how to contribute is available on the Apache Software Foundation (ASF) web site. In particular, we recommend the following: + + * [Contributors Tech Guide](http://www.apache.org/dev/contributors) + * [Get involved!](http://www.apache.org/foundation/getinvolved.html) + * [Legal aspects on Submission of Contributions (Patches)](http://www.apache.org/licenses/LICENSE-2.0.html#contributions) diff --git a/vendor/github.com/apache/thrift/Dockerfile b/vendor/github.com/apache/thrift/Dockerfile new file mode 100644 index 0000000000000..0d7ad2175e353 --- /dev/null +++ b/vendor/github.com/apache/thrift/Dockerfile @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +# Goal: provide a thrift-compiler Docker image +# +# Usage: +# docker run -v "${PWD}:/data" thrift/thrift-compiler -gen cpp -o /data/ /data/test/ThriftTest.thrift +# +# further details on docker for thrift is here build/docker/ +# +# TODO: push to apache/thrift-compiler instead of thrift/thrift-compiler + +FROM debian:jessie +MAINTAINER Apache Thrift + +ENV DEBIAN_FRONTEND noninteractive + +ADD . /thrift + +RUN buildDeps=" \ + flex \ + bison \ + g++ \ + make \ + cmake \ + curl \ + "; \ + apt-get update && apt-get install -y --no-install-recommends $buildDeps \ + && mkdir /tmp/cmake-build && cd /tmp/cmake-build \ + && cmake \ + -DBUILD_COMPILER=ON \ + -DBUILD_LIBRARIES=OFF \ + -DBUILD_TESTING=OFF \ + -DBUILD_EXAMPLES=OFF \ + /thrift \ + && cmake --build . --config Release \ + && make install \ + && curl -k -sSL "https://storage.googleapis.com/golang/go1.5.2.linux-amd64.tar.gz" -o /tmp/go.tar.gz \ + && tar xzf /tmp/go.tar.gz -C /tmp \ + && cp /tmp/go/bin/gofmt /usr/bin/gofmt \ + && apt-get purge -y --auto-remove $buildDeps \ + && apt-get clean \ + && rm -rf /tmp/* \ + && rm -rf /var/lib/apt/lists/* + +ENTRYPOINT ["thrift"] diff --git a/vendor/github.com/apache/thrift/Makefile.am b/vendor/github.com/apache/thrift/Makefile.am new file mode 100644 index 0000000000000..ed58265acefc4 --- /dev/null +++ b/vendor/github.com/apache/thrift/Makefile.am @@ -0,0 +1,131 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +ACLOCAL_AMFLAGS = -I ./aclocal + +if WITH_PLUGIN +# To enable bootstrap, build order is lib/cpp -> compiler -> others +SUBDIRS = lib/cpp compiler/cpp lib +if WITH_TESTS +SUBDIRS += lib/cpp/test +endif +else +SUBDIRS = compiler/cpp lib +endif + +if WITH_TESTS +SUBDIRS += test +endif + +if WITH_TUTORIAL +SUBDIRS += tutorial +endif + +dist-hook: + find $(distdir) -type f \( -iname ".DS_Store" -or -iname "._*" -or -iname ".gitignore" \) | xargs rm -rf + find $(distdir) -type d \( -iname ".deps" -or -iname ".libs" \) | xargs rm -rf + find $(distdir) -type d \( -iname ".svn" -or -iname ".git" \) | xargs rm -rf + +print-version: + @echo $(VERSION) + +.PHONY: precross cross +precross-%: all + $(MAKE) -C $* precross +precross: all precross-test precross-lib + +empty := +space := $(empty) $(empty) +comma := , + +CROSS_LANGS = @MAYBE_CPP@ @MAYBE_C_GLIB@ @MAYBE_D@ @MAYBE_JAVA@ @MAYBE_CSHARP@ @MAYBE_PYTHON@ @MAYBE_PY3@ @MAYBE_RUBY@ @MAYBE_HASKELL@ @MAYBE_PERL@ @MAYBE_PHP@ @MAYBE_GO@ @MAYBE_NODEJS@ @MAYBE_DART@ @MAYBE_ERLANG@ @MAYBE_LUA@ +CROSS_LANGS_COMMA_SEPARATED = $(subst $(space),$(comma),$(CROSS_LANGS)) + +if WITH_PY3 +CROSS_PY=$(PYTHON3) +else +CROSS_PY=$(PYTHON) +endif + +if WITH_PYTHON +crossfeature: precross + $(CROSS_PY) test/test.py --retry-count 3 --features .* --skip-known-failures --server $(CROSS_LANGS_COMMA_SEPARATED) +else +# feature test needs python build +crossfeature: +endif + +cross-%: precross crossfeature + $(CROSS_PY) test/test.py --retry-count 3 --skip-known-failures --server $(CROSS_LANGS_COMMA_SEPARATED) --client $(CROSS_LANGS_COMMA_SEPARATED) --regex "$*" + +cross: cross-.* + +TIMES = 1 2 3 +fail: precross + $(CROSS_PY) test/test.py || true + $(CROSS_PY) test/test.py --update-expected-failures=overwrite + $(foreach var,$(TIMES),test/test.py -s || true;test/test.py --update-expected-failures=merge;) + +codespell_skip_files = \ + *.jar \ + *.class \ + *.so \ + *.a \ + *.la \ + *.o \ + *.p12 \ + *OCamlMakefile \ + .keystore \ + .truststore \ + CHANGES \ + config.sub \ + configure \ + depcomp \ + libtool.m4 \ + output.* \ + rebar \ + thrift + +skipped_files = $(subst $(space),$(comma),$(codespell_skip_files)) + +style-local: + codespell --write-changes --skip=$(skipped_files) --disable-colors + +EXTRA_DIST = \ + .clang-format \ + .editorconfig \ + .travis.yml \ + appveyor.yml \ + bower.json \ + build \ + CMakeLists.txt \ + composer.json \ + contrib \ + CONTRIBUTING.md \ + debian \ + doc \ + doap.rdf \ + package.json \ + sonar-project.properties \ + Dockerfile \ + LICENSE \ + CHANGES \ + NOTICE \ + README.md \ + Thrift.podspec diff --git a/vendor/github.com/apache/thrift/README.md b/vendor/github.com/apache/thrift/README.md new file mode 100644 index 0000000000000..07cd32f09824b --- /dev/null +++ b/vendor/github.com/apache/thrift/README.md @@ -0,0 +1,166 @@ +Apache Thrift +============= + ++[![Build Status](https://travis-ci.org/apache/thrift.svg?branch=master)](https://travis-ci.org/apache/thrift) +- +[![AppVeyor Build status](https://ci.appveyor.com/api/projects/status/e2qks7enyp9gw7ma?svg=true)](https://ci.appveyor.com/project/apache/thrift) + + +Introduction +============ + +Thrift is a lightweight, language-independent software stack with an +associated code generation mechanism for RPC. Thrift provides clean +abstractions for data transport, data serialization, and application +level processing. The code generation system takes a simple definition +language as its input and generates code across programming languages that +uses the abstracted stack to build interoperable RPC clients and servers. + +Thrift is specifically designed to support non-atomic version changes +across client and server code. + +For more details on Thrift's design and implementation, take a gander at +the Thrift whitepaper included in this distribution or at the README.md files +in your particular subdirectory of interest. + +Hierarchy +========= + +thrift/ + + compiler/ + + Contains the Thrift compiler, implemented in C++. + + lib/ + + Contains the Thrift software library implementation, subdivided by + language of implementation. + + cpp/ + go/ + java/ + php/ + py/ + rb/ + + test/ + + Contains sample Thrift files and test code across the target programming + languages. + + tutorial/ + + Contains a basic tutorial that will teach you how to develop software + using Thrift. + +Requirements +============ + +See http://thrift.apache.org/docs/install for an up-to-date list of build requirements. + +Resources +========= + +More information about Thrift can be obtained on the Thrift webpage at: + + http://thrift.apache.org + +Acknowledgments +=============== + +Thrift was inspired by pillar, a lightweight RPC tool written by Adam D'Angelo, +and also by Google's protocol buffers. + +Installation +============ + +If you are building from the first time out of the source repository, you will +need to generate the configure scripts. (This is not necessary if you +downloaded a tarball.) From the top directory, do: + + ./bootstrap.sh + +Once the configure scripts are generated, thrift can be configured. +From the top directory, do: + + ./configure + +You may need to specify the location of the boost files explicitly. +If you installed boost in /usr/local, you would run configure as follows: + + ./configure --with-boost=/usr/local + +Note that by default the thrift C++ library is typically built with debugging +symbols included. If you want to customize these options you should use the +CXXFLAGS option in configure, as such: + + ./configure CXXFLAGS='-g -O2' + ./configure CFLAGS='-g -O2' + ./configure CPPFLAGS='-DDEBUG_MY_FEATURE' + +To enable gcov required options -fprofile-arcs -ftest-coverage enable them: + + ./configure --enable-coverage + +Run ./configure --help to see other configuration options + +Please be aware that the Python library will ignore the --prefix option +and just install wherever Python's distutils puts it (usually along +the lines of /usr/lib/pythonX.Y/site-packages/). If you need to control +where the Python modules are installed, set the PY_PREFIX variable. +(DESTDIR is respected for Python and C++.) + +Make thrift: + + make + +From the top directory, become superuser and do: + + make install + +Note that some language packages must be installed manually using build tools +better suited to those languages (at the time of this writing, this applies +to Java, Ruby, PHP). + +Look for the README.md file in the lib// folder for more details on the +installation of each language library package. + +Testing +======= + +There are a large number of client library tests that can all be run +from the top-level directory. + + make -k check + +This will make all of the libraries (as necessary), and run through +the unit tests defined in each of the client libraries. If a single +language fails, the make check will continue on and provide a synopsis +at the end. + +To run the cross-language test suite, please run: + + make cross + +This will run a set of tests that use different language clients and +servers. + +License +======= + +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. diff --git a/vendor/github.com/apache/thrift/Thrift.podspec b/vendor/github.com/apache/thrift/Thrift.podspec new file mode 100644 index 0000000000000..2ead0e93df379 --- /dev/null +++ b/vendor/github.com/apache/thrift/Thrift.podspec @@ -0,0 +1,18 @@ +Pod::Spec.new do |s| + s.name = "Thrift" + s.version = "0.10.0" + s.summary = "Apache Thrift is a lightweight, language-independent software stack with an associated code generation mechanism for RPC." + s.description = <<-DESC +The Apache Thrift software framework, for scalable cross-language services development, combines a software stack with a code generation engine to build services that work efficiently and seamlessly between C++, Java, Python, PHP, Ruby, Erlang, Perl, Haskell, C#, Cocoa, JavaScript, Node.js, Smalltalk, OCaml and Delphi and other languages. + DESC + s.homepage = "http://thrift.apache.org" + s.license = { :type => 'Apache License, Version 2.0', :url => 'https://raw.github.com/apache/thrift/thrift-0.9.0/LICENSE' } + s.author = { "The Apache Software Foundation" => "apache@apache.org" } + s.requires_arc = true + s.ios.deployment_target = '7.0' + s.osx.deployment_target = '10.8' + s.ios.framework = 'CFNetwork' + s.osx.framework = 'CoreServices' + s.source = { :git => "https://github.com/apache/thrift.git", :tag => "thrift-0.10.0" } + s.source_files = 'lib/cocoa/src/**/*.{h,m,swift}' +end \ No newline at end of file diff --git a/vendor/github.com/apache/thrift/appveyor.yml b/vendor/github.com/apache/thrift/appveyor.yml new file mode 100644 index 0000000000000..03ee29543584e --- /dev/null +++ b/vendor/github.com/apache/thrift/appveyor.yml @@ -0,0 +1,93 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +# build Apache Thrift on AppVeyor - https://ci.appveyor.com + +shallow_clone: true +clone_depth: 10 + +version: '{build}' +os: +# - Windows Server 2012 R2 +- Visual Studio 2015 + +environment: + BOOST_ROOT: C:\Libraries\boost_1_59_0 + BOOST_LIBRARYDIR: C:\Libraries\boost_1_59_0\lib64-msvc-14.0 + # Unfurtunately, this version needs manual update because old versions are quickly deleted. + ANT_VERSION: 1.9.7 + +install: +- '"C:\Program Files (x86)\Microsoft Visual Studio 14.0\VC\vcvarsall.bat" x64' +- cd \ + # Zlib +- appveyor DownloadFile https://github.com/madler/zlib/archive/v1.2.8.tar.gz +- 7z x v1.2.8.tar.gz -so | 7z x -si -ttar > nul +- cd zlib-1.2.8 +- cmake -G "Visual Studio 14 2015 Win64" . +- cmake --build . --config release +- cd .. + # OpenSSL +- C:\Python35-x64\python %APPVEYOR_BUILD_FOLDER%\build\appveyor\download_openssl.py +- ps: Start-Process "Win64OpenSSL.exe" -ArgumentList "/silent /verysilent /sp- /suppressmsgboxes" -Wait + # Libevent +- appveyor DownloadFile https://github.com/libevent/libevent/releases/download/release-2.0.22-stable/libevent-2.0.22-stable.tar.gz +- 7z x libevent-2.0.22-stable.tar.gz -so | 7z x -si -ttar > nul +- cd libevent-2.0.22-stable +- nmake -f Makefile.nmake +- mkdir lib +- move *.lib lib\ +- move WIN32-Code\event2\* include\event2\ +- move *.h include\ +- cd .. +- appveyor-retry cinst -y winflexbison +- appveyor DownloadFile http://www.us.apache.org/dist/ant/binaries/apache-ant-%ANT_VERSION%-bin.zip +- 7z x apache-ant-%ANT_VERSION%-bin.zip > nul +- cd %APPVEYOR_BUILD_FOLDER% +# TODO: Enable Haskell build +# - cinst HaskellPlatform -version 2014.2.0.0 + + +build_script: +- set PATH=C:\ProgramData\chocolatey\bin;C:\apache-ant-%ANT_VERSION%\bin;%PATH% +- set JAVA_HOME=C:\Program Files\Java\jdk1.7.0 +- set PATH=%JAVA_HOME%\bin;%PATH% +# - set PATH=%PATH%;C:\Program Files (x86)\Haskell Platform\2014.2.0.0\bin +# - set PATH=%PATH%;C:\Program Files (x86)\Haskell Platform\2014.2.0.0\lib\extralibs\bin +- set PATH=C:\Python27-x64\scripts;C:\Python27-x64;%PATH% +- pip install ipaddress backports.ssl_match_hostname tornado twisted +- mkdir cmake-build +- cd cmake-build +- cmake -G "Visual Studio 14 2015 Win64" -DWITH_SHARED_LIB=OFF -DLIBEVENT_ROOT=C:\libevent-2.0.22-stable -DZLIB_INCLUDE_DIR=C:\zlib-1.2.8 -DZLIB_LIBRARY=C:\zlib-1.2.8\release\zlibstatic.lib -DBOOST_ROOT="%BOOST_ROOT%" -DBOOST_LIBRARYDIR="%BOOST_LIBRARYDIR%" .. +- findstr /b /e BUILD_COMPILER:BOOL=ON CMakeCache.txt +- findstr /b /e BUILD_CPP:BOOL=ON CMakeCache.txt +- findstr /b /e BUILD_JAVA:BOOL=ON CMakeCache.txt +- findstr /b /e BUILD_PYTHON:BOOL=ON CMakeCache.txt +# - findstr /b /e BUILD_C_GLIB:BOOL=ON CMakeCache.txt +# - findstr /b /e BUILD_HASKELL:BOOL=ON CMakeCache.txt +- findstr /b /e BUILD_TESTING:BOOL=ON CMakeCache.txt +# - cmake --build . +- cmake --build . --config Release +# TODO: Fix cpack +# - cpack +# TODO: Run more tests +# CTest fails to invoke ant seemingly due to "ant.bat" v.s. "ant" (shell script) conflict. +# Currently, everything that involves OpenSSL seems to hang forever on our Appveyor setup. +# Also a few C++ tests hang (on Appveyor or on Windows in general). +- ctest -C Release --timeout 600 -VV -E "(StressTestNonBlocking|PythonTestSSLSocket|python_test$|^Java)" +# TODO make it perfect ;-r diff --git a/vendor/github.com/apache/thrift/bootstrap.sh b/vendor/github.com/apache/thrift/bootstrap.sh new file mode 100644 index 0000000000000..52ecda47b3ada --- /dev/null +++ b/vendor/github.com/apache/thrift/bootstrap.sh @@ -0,0 +1,54 @@ +#!/bin/sh + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +./cleanup.sh +if test -d lib/php/src/ext/thrift_protocol ; then + if phpize -v >/dev/null 2>/dev/null ; then + (cd lib/php/src/ext/thrift_protocol && phpize) + fi +fi + +set -e + +# libtoolize is called "glibtoolize" on OSX. +if libtoolize --version 1 >/dev/null 2>/dev/null; then + LIBTOOLIZE=libtoolize +elif glibtoolize --version 1 >/dev/null 2>/dev/null; then + LIBTOOLIZE=glibtoolize +else + echo >&2 "Couldn't find libtoolize!" + exit 1 +fi + +# we require automake 1.13 or later +# check must happen externally due to use of newer macro +AUTOMAKE_VERSION=`automake --version | grep automake | egrep -o '([0-9]{1,}\.)+[0-9]{1,}'` +if [ "$AUTOMAKE_VERSION" \< "1.13" ]; then + echo >&2 "automake version $AUTOMAKE_VERSION is too old (need 1.13 or later)" + exit 1 +fi + +autoscan +$LIBTOOLIZE --copy --automake +aclocal -I ./aclocal +autoheader +autoconf +automake --copy --add-missing --foreign diff --git a/vendor/github.com/apache/thrift/bower.json b/vendor/github.com/apache/thrift/bower.json new file mode 100644 index 0000000000000..9ec59fcb5dc62 --- /dev/null +++ b/vendor/github.com/apache/thrift/bower.json @@ -0,0 +1,16 @@ +{ + "name": "thrift", + "version": "0.10.0", + "homepage": "https://git-wip-us.apache.org/repos/asf/thrift.git", + "authors": [ + "Apache Thrift " + ], + "description": "Apache Thrift", + "main": "lib/js/src/thrift.js", + "keywords": [ + "thrift" + ], + "license": "Apache v2", + "ignore": [ + ] +} diff --git a/vendor/github.com/apache/thrift/cleanup.sh b/vendor/github.com/apache/thrift/cleanup.sh new file mode 100644 index 0000000000000..f110721ac9f59 --- /dev/null +++ b/vendor/github.com/apache/thrift/cleanup.sh @@ -0,0 +1,89 @@ +#!/bin/sh + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +topsrcdir="`dirname $0`" +cd "$topsrcdir" + +make -k clean >/dev/null 2>&1 +make -k distclean >/dev/null 2>&1 +find . -name Makefile.in -exec rm -f {} \; +rm -rf \ +AUTHORS \ +ChangeLog \ +INSTALL \ +Makefile \ +Makefile.in \ +Makefile.orig \ +aclocal/libtool.m4 \ +aclocal/ltoptions.m4 \ +aclocal/ltsugar.m4 \ +aclocal/ltversion.m4 \ +aclocal/lt~obsolete.m4 \ +aclocal.m4 \ +autom4te.cache \ +autoscan.log \ +config.guess \ +config.h \ +config.hin \ +config.hin~ \ +config.log \ +config.status \ +config.status.lineno \ +config.sub \ +configure \ +configure.lineno \ +configure.scan \ +depcomp \ +.deps \ +install-sh \ +.libs \ +libtool \ +ltmain.sh \ +missing \ +ylwrap \ +if/gen-* \ +test/gen-* \ +lib/php/src/ext/thrift_protocol/.deps \ +lib/php/src/ext/thrift_protocol/Makefile \ +lib/php/src/ext/thrift_protocol/Makefile.fragments \ +lib/php/src/ext/thrift_protocol/Makefile.global \ +lib/php/src/ext/thrift_protocol/Makefile.objects \ +lib/php/src/ext/thrift_protocol/acinclude.m4 \ +lib/php/src/ext/thrift_protocol/aclocal.m4 \ +lib/php/src/ext/thrift_protocol/autom4te.cache \ +lib/php/src/ext/thrift_protocol/build \ +lib/php/src/ext/thrift_protocol/config.guess \ +lib/php/src/ext/thrift_protocol/config.h \ +lib/php/src/ext/thrift_protocol/config.h.in \ +lib/php/src/ext/thrift_protocol/config.log \ +lib/php/src/ext/thrift_protocol/config.nice \ +lib/php/src/ext/thrift_protocol/config.status \ +lib/php/src/ext/thrift_protocol/config.sub \ +lib/php/src/ext/thrift_protocol/configure \ +lib/php/src/ext/thrift_protocol/configure.in \ +lib/php/src/ext/thrift_protocol/include \ +lib/php/src/ext/thrift_protocol/install-sh \ +lib/php/src/ext/thrift_protocol/libtool \ +lib/php/src/ext/thrift_protocol/ltmain.sh \ +lib/php/src/ext/thrift_protocol/missing \ +lib/php/src/ext/thrift_protocol/mkinstalldirs \ +lib/php/src/ext/thrift_protocol/modules \ +lib/php/src/ext/thrift_protocol/run-tests.php diff --git a/vendor/github.com/apache/thrift/composer.json b/vendor/github.com/apache/thrift/composer.json new file mode 100644 index 0000000000000..d937bc7ac2155 --- /dev/null +++ b/vendor/github.com/apache/thrift/composer.json @@ -0,0 +1,30 @@ +{ + "name": "apache/thrift", + "description": "Apache Thrift RPC system", + "homepage": "http://thrift.apache.org/", + "type": "library", + "license": "Apache-2.0", + "authors": [ + { + "name": "Apache Thrift Developers", + "email": "dev@thrift.apache.org", + "homepage": "http://thrift.apache.org" + } + ], + "support": { + "email": "dev@thrift.apache.org", + "issues": "https://issues.apache.org/jira/browse/THRIFT" + }, + "require": { + "php": ">=5.3.0" + }, + "autoload": { + "psr-0": {"Thrift": "lib/php/lib/"} + }, + "minimum-stability": "dev", + "extra": { + "branch-alias": { + "dev-master": "0.10.0" + } + } +} diff --git a/vendor/github.com/apache/thrift/configure.ac b/vendor/github.com/apache/thrift/configure.ac new file mode 100644 index 0000000000000..0972abfc9d984 --- /dev/null +++ b/vendor/github.com/apache/thrift/configure.ac @@ -0,0 +1,959 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +AC_PREREQ(2.65) +AC_CONFIG_MACRO_DIR([./aclocal]) + +AC_INIT([thrift], [0.10.0]) + +AC_CONFIG_AUX_DIR([.]) + +AM_INIT_AUTOMAKE([1.13 subdir-objects tar-ustar]) +PKG_PROG_PKG_CONFIG + +AC_ARG_VAR([PY_PREFIX], [Prefix for installing Python modules. + (Normal --prefix is ignored for Python because + Python has different conventions.) + Default = "/usr"]) +AS_IF([test "x$PY_PREFIX" = x], [PY_PREFIX="/usr"]) + +AC_ARG_VAR([JAVA_PREFIX], [Prefix for installing the Java lib jar. + Default = "/usr/local/lib"]) +AS_IF([test "x$JAVA_PREFIX" != x], [JAVA_PREFIX="$JAVA_PREFIX/usr/local/lib"], + [test "x$PREFIX" != x], [JAVA_PREFIX="$PREFIX/usr/local/lib"], + [JAVA_PREFIX="/usr/local/lib"]) + +AC_ARG_VAR([RUBY_PREFIX], [Prefix for installing Ruby modules. + (Normal --prefix is ignored for Ruby because + Ruby has different conventions.) + Default = none, let ruby setup decide]) + +AC_ARG_VAR([PHP_PREFIX], [Prefix for installing PHP modules. + (Normal --prefix is ignored for PHP because + PHP has different conventions.) + Default = "/usr/lib/php"]) +AS_IF([test "x$PHP_PREFIX" = x], [PHP_PREFIX="/usr/lib/php"]) + +AC_ARG_VAR([PHP_CONFIG_PREFIX], + [Prefix for installing PHP extension module .ini file. + (Normal --prefix is ignored for PHP because PHP has + different conventions.) + Default = "/etc/php.d"]) +AS_IF([test "x$PHP_CONFIG_PREFIX" = x], [PHP_CONFIG_PREFIX="/etc/php.d"]) + +AC_ARG_VAR([INSTALLDIRS], [When installing Perl modules, specifies which + of the sets of installation directories + to choose: perl, site or vendor. + Default = "vendor"]) +AS_IF([test "x$INSTALLDIRS" = x], [INSTALLDIRS="vendor"]) + +AC_ARG_VAR([PERL_PREFIX], [Prefix for installing Perl modules. + (Normal --prefix is ignored for Perl because + Perl has different conventions.) + Ignored, when INSTALLDIRS set to site or vendor. + Default = "/usr/local/lib"]) +AS_IF([test "x$PERL_PREFIX" = x], [PERL_PREFIX="/usr/local"]) + +AC_ARG_VAR([CABAL_CONFIGURE_FLAGS], + [Extra flags to pass to cabal: "cabal Setup.lhs configure $CABAL_CONFIGURE_FLAGS". + (Typically used to set --user or force --global.)]) + +AC_SUBST(CABAL_CONFIGURE_FLAGS) + +AC_ARG_VAR([D_IMPORT_PREFIX], [Prefix for installing D modules. + [INCLUDEDIR/d2]]) +AS_IF([test "x$D_IMPORT_PREFIX" = x], [D_IMPORT_PREFIX="${includedir}/d2"]) + +AC_ARG_VAR([DMD_LIBEVENT_FLAGS], [DMD flags for linking libevent (auto-detected if not set).]) +AC_ARG_VAR([DMD_OPENSSL_FLAGS], [DMD flags for linking OpenSSL (auto-detected if not set).]) + +AC_PROG_CC +AC_PROG_CPP +AC_PROG_CXX +AC_PROG_INSTALL +AC_PROG_LIBTOOL +AC_PROG_MAKE_SET +AC_PROG_BISON(2.5) +AC_PROG_YACC +AC_PROG_LEX +AM_PROG_LEX +AC_PROG_LN_S +AC_PROG_MKDIR_P +AC_PROG_AWK +AC_PROG_RANLIB + +AC_LANG([C++]) +AX_CXX_COMPILE_STDCXX_11([noext], [optional]) + +AM_EXTRA_RECURSIVE_TARGETS([style]) +AC_SUBST(CPPSTYLE_CMD, 'find . -type f \( -iname "*.h" -or -iname "*.cpp" -or -iname "*.cc" -or -iname "*.tcc" \) -printf "Reformatting: %h/%f\n" -exec clang-format -i {} \;') + +AC_ARG_ENABLE([libs], + AS_HELP_STRING([--enable-libs], [build the Apache Thrift libraries [default=yes]]), + [], enable_libs=yes +) +have_libs=yes +if test "$enable_libs" = "no"; then + have_libs="no" + with_cpp="no" + with_c_glib="no" + with_java="no" + with_csharp="no" + with_python="no" + with_ruby="no" + with_haskell="no" + with_haxe="no" + with_perl="no" + with_php="no" + with_php_extension="no" + with_dart="no" + with_erlang="no" + with_go="no" + with_d="no" + with_nodejs="no" + with_lua="no" +fi + + +AX_THRIFT_LIB(cpp, [C++], yes) +have_cpp=no +if test "$with_cpp" = "yes"; then + AX_BOOST_BASE([1.53.0]) + if test "x$succeeded" = "xyes" ; then + AC_SUBST([BOOST_LIB_DIR], [$(echo "$BOOST_LDFLAGS" | sed -e 's/^\-L//')]) + AC_SUBST([BOOST_CHRONO_LDADD], [$(echo "$BOOST_LIB_DIR/libboost_chrono.a")]) + AC_SUBST([BOOST_FILESYSTEM_LDADD], [$(echo "$BOOST_LIB_DIR/libboost_filesystem.a")]) + AC_SUBST([BOOST_SYSTEM_LDADD], [$(echo "$BOOST_LIB_DIR/libboost_system.a")]) + AC_SUBST([BOOST_TEST_LDADD], [$(echo "$BOOST_LIB_DIR/libboost_unit_test_framework.a")]) + AC_SUBST([BOOST_THREAD_LDADD], [$(echo "$BOOST_LIB_DIR/libboost_thread.a")]) + have_cpp="yes" + fi + + AX_CHECK_OPENSSL() + + AX_LIB_EVENT([1.0]) + have_libevent=$success + + AX_LIB_ZLIB([1.2.3]) + have_zlib=$success + + AX_THRIFT_LIB(qt4, [Qt], yes) + have_qt=no + if test "$with_qt4" = "yes"; then + PKG_CHECK_MODULES([QT], [QtCore >= 4.3, QtNetwork >= 4.3], have_qt=yes, have_qt=no) + fi + if test "$have_qt" = "yes"; then + AC_PATH_PROGS([QT_MOC], [moc-qt4 moc], "fail") + if test "$QT_MOC" = "fail"; then + have_qt=no + fi + fi + + AX_THRIFT_LIB(qt5, [Qt5], yes) + have_qt5=no + qt_reduce_reloc="" + if test "$with_qt5" = "yes"; then + PKG_CHECK_MODULES([QT5], [Qt5Core >= 5.0, Qt5Network >= 5.0], + [have_qt5=yes;qt_reduce_reloc=`$PKG_CONFIG --variable=qt_config Qt5Core | grep "reduce_relocations"`], + [have_qt5=no]) + fi + if test "$have_qt5" = "yes"; then + AC_PATH_PROGS([QT5_MOC], [moc-qt5 moc], "fail") + if test "$QT5_MOC" = "fail"; then + have_qt5=no + fi + fi +fi +AM_CONDITIONAL([WITH_CPP], [test "$have_cpp" = "yes"]) +AM_CONDITIONAL([AMX_HAVE_LIBEVENT], [test "$have_libevent" = "yes"]) +AM_CONDITIONAL([AMX_HAVE_ZLIB], [test "$have_zlib" = "yes"]) +AM_CONDITIONAL([AMX_HAVE_QT], [test "$have_qt" = "yes"]) +AM_CONDITIONAL([AMX_HAVE_QT5], [test "$have_qt5" = "yes"]) +AM_CONDITIONAL([QT5_REDUCE_RELOCATIONS], [test "x$qt_reduce_reloc" != "x"]) + +AX_THRIFT_LIB(c_glib, [C (GLib)], yes) +if test "$with_c_glib" = "yes"; then + PKG_CHECK_MODULES([GLIB], [glib-2.0 >= 2.0], have_glib2=yes, have_glib2=no) + PKG_CHECK_MODULES([GOBJECT], [gobject-2.0 >= 2.0], have_gobject2=yes, have_gobject2=no) + if test "$have_glib2" = "yes" -a "$have_gobject2" = "yes" ; then + have_c_glib="yes" + fi +fi +AM_CONDITIONAL(WITH_C_GLIB, [test "$have_glib2" = "yes" -a "$have_gobject2" = "yes"]) + +AX_THRIFT_LIB(csharp, [C#], yes) +if test "$with_csharp" = "yes"; then + PKG_CHECK_MODULES(MONO, mono >= 2.11.0, mono_2_11=yes, mono_2_11=no) + if test "$mono_2_11" == "yes"; then + AC_PATH_PROG([MCS], [mcs]) + if test "x$MCS" != "x"; then + mono_mcs="yes" + fi + fi + PKG_CHECK_MODULES(MONO, mono >= 2.0.0, net_3_5=yes, net_3_5=no) + PKG_CHECK_MODULES(MONO, mono >= 1.2.4, have_mono=yes, have_mono=no) + if test "$have_mono" = "yes" ; then + have_csharp="yes" + fi +fi +AM_CONDITIONAL(WITH_MONO, [test "$have_csharp" = "yes"]) +AM_CONDITIONAL(NET_2_0, [test "$net_3_5" = "no"]) +AM_CONDITIONAL(MONO_MCS, [test "$mono_mcs" = "yes"]) + +AX_THRIFT_LIB(java, [Java], yes) +if test "$with_java" = "yes"; then + AX_JAVAC_AND_JAVA + AC_PATH_PROG([ANT], [ant]) + AX_CHECK_ANT_VERSION($ANT, 1.7) + AC_SUBST(CLASSPATH) + AC_SUBST(ANT_FLAGS) + if test "x$JAVA" != "x" && test "x$JAVAC" != "x" && test "x$ANT" != "x" ; then + have_java="yes" + fi +fi +AM_CONDITIONAL([WITH_JAVA], [test "$have_java" = "yes"]) + +AX_THRIFT_LIB(erlang, [Erlang], yes) +if test "$with_erlang" = "yes"; then + AC_ERLANG_PATH_ERL + AC_ERLANG_PATH_ERLC + AC_PATH_PROG([REBAR], [rebar]) + if test -n "$ERLC" ; then + AC_ERLANG_SUBST_LIB_DIR + # Install into the detected Erlang directory instead of $libdir/erlang/lib + ERLANG_INSTALL_LIB_DIR="$ERLANG_LIB_DIR" + AC_ERLANG_SUBST_INSTALL_LIB_SUBDIR(AC_PACKAGE_NAME, AC_PACKAGE_VERSION) + fi + if test -n "$ERL" -a -n "$ERLC" && test "x$REBAR" != "x" ; then + have_erlang="yes" + + # otp_release is simply a number (like "17") for OTP17+ while "R16..." for OTP16 or less. + # OTP version is currently only used for running tests. + if $ERL -eval 'erlang:display(erlang:system_info(otp_release)),halt().' -noshell | grep "^\"R" >/dev/null; then + erlang_otp16_or_less="yes" + fi + fi +fi +AM_CONDITIONAL(WITH_ERLANG, [test "$have_erlang" = "yes"]) +AM_CONDITIONAL(ERLANG_OTP16, [test "$erlang_otp16_or_less" = "yes"]) + +AX_THRIFT_LIB(nodejs, [Nodejs], yes) +have_nodejs=no +if test "$with_nodejs" = "yes"; then + AC_PATH_PROGS([NODEJS], [nodejs node]) + AC_PATH_PROG([NPM], [npm]) + if test "x$NODEJS" != "x" -a "x$NPM" != "x"; then + have_nodejs="yes" + fi +fi +AM_CONDITIONAL(WITH_NODEJS, [test "$have_nodejs" = "yes"]) +AM_CONDITIONAL(HAVE_NPM, [test "x$NPM" != "x"]) + +AX_THRIFT_LIB(lua, [Lua], yes) +have_lua=no +if test "$with_lua" = "yes"; then + AX_PROG_LUA(5.2,, have_lua="yes", have_lua="no") + if test "$have_lua" = "yes"; then + AX_LUA_HEADERS(, have_lua="no") + AX_LUA_LIBS(, have_lua="no") + fi +fi +AM_CONDITIONAL(WITH_LUA, [test "$have_lua" = "yes"]) + +# Find python regardless of with_python value, because it's needed by make cross +AM_PATH_PYTHON(2.6,, :) +AX_THRIFT_LIB(python, [Python], yes) +if test "$with_python" = "yes"; then + if test -n "$PYTHON"; then + have_python="yes" + fi + AC_PATH_PROG([TRIAL], [trial]) + if test -n "$TRIAL"; then + have_trial="yes" + fi +fi +AM_CONDITIONAL(WITH_PYTHON, [test "$have_python" = "yes"]) +AM_CONDITIONAL(WITH_TWISTED_TEST, [test "$have_trial" = "yes"]) + +# Find "python3" executable. +# It's distro specific and far from ideal but needed to cross test py2-3 at once. +# TODO: find "python2" if it's 3.x +if python --version 2>&1 | grep -q "Python 2"; then + AC_PATH_PROGS([PYTHON3], [python3 python3.5 python35 python3.4 python34]) + if test -n "$PYTHON3"; then + have_py3="yes" + fi +fi +AM_CONDITIONAL(WITH_PY3, [test "$have_py3" = "yes"]) + +AX_THRIFT_LIB(perl, [Perl], yes) +if test "$with_perl" = "yes"; then + AC_PATH_PROG([PERL], [perl]) + if test -n "$PERL" ; then + AC_PROG_PERL_MODULES([Bit::Vector], success="yes", success="no") + have_perl_bit_vector="$success" + AC_PROG_PERL_MODULES([Class::Accessor], success="yes", success="no") + have_perl_class_accessor="$success" + fi + if test -n "$PERL" -a "$have_perl_bit_vector" = "yes" ; then + if test -n "$PERL" -a "$have_perl_class_accessor" = "yes" ; then + have_perl="yes" + fi + fi +fi +AM_CONDITIONAL(WITH_PERL, [test "$have_perl" = "yes"]) + +AX_THRIFT_LIB(php, [PHP], yes) +if test "$with_php" = "yes"; then + AC_PATH_PROG([PHP], [php]) + if test -n "$PHP" ; then + have_php="yes" + fi +fi +AM_CONDITIONAL(WITH_PHP, [test "$have_php" = "yes"]) + +AX_THRIFT_LIB(php_extension, [PHP_EXTENSION], yes) +if test "$with_php_extension" = "yes"; then + if test -f "lib/php/src/ext/thrift_protocol/configure"; then + AC_PATH_PROG([PHP_CONFIG], [php-config]) + if test -n "$PHP_CONFIG" ; then + AC_CONFIG_SUBDIRS([lib/php/src/ext/thrift_protocol]) + have_php_extension="yes" + fi + fi +fi +AM_CONDITIONAL(WITH_PHP_EXTENSION, [test "$have_php_extension" = "yes"]) + +AC_PATH_PROG([PHPUNIT], [phpunit]) +AM_CONDITIONAL(HAVE_PHPUNIT, [test "x$PHPUNIT" != "x"]) + +AX_THRIFT_LIB(dart, [DART], yes) +if test "$with_dart" = "yes"; then + AC_PATH_PROG([DART], [dart]) + AC_PATH_PROG([DARTPUB], [pub]) + if test "x$DART" != "x" -a "x$DARTPUB" != "x"; then + have_dart="yes" + fi +fi +AM_CONDITIONAL(WITH_DART, [test "$have_dart" = "yes"]) + +AX_THRIFT_LIB(ruby, [Ruby], yes) +have_ruby=no +if test "$with_ruby" = "yes"; then + AC_PATH_PROG([RUBY], [ruby]) + AC_PATH_PROG([BUNDLER], [bundle]) + if test "x$RUBY" != "x" -a "x$BUNDLER" != "x"; then + have_ruby="yes" + fi +fi +AM_CONDITIONAL(WITH_RUBY, [test "$have_ruby" = "yes"]) +AM_CONDITIONAL(HAVE_BUNDLER, [test "x$BUNDLER" != "x"]) + +AX_THRIFT_LIB(haskell, [Haskell], yes) +have_haskell=no +RUNHASKELL=true +CABAL=true +if test "$with_haskell" = "yes"; then + AC_PATH_PROG([CABAL], [cabal]) + AC_PATH_PROG([RUNHASKELL], [runhaskell]) + if test "x$CABAL" != "x" -a "x$RUNHASKELL" != "x"; then + have_haskell="yes" + else + RUNHASKELL=true + CABAL=true + fi +fi +AC_SUBST(CABAL) +AC_SUBST(RUNHASKELL) +AM_CONDITIONAL(WITH_HASKELL, [test "$have_haskell" = "yes"]) + +AX_THRIFT_LIB(go, [Go], yes) +if test "$with_go" = "yes"; then + AC_PATH_PROG([GO], [go]) + if [[ -x "$GO" ]] ; then + AS_IF([test -n "$GO"],[ + ax_go_version="1.4" + + AC_MSG_CHECKING([for Go version]) + golang_version=`$GO version 2>&1 | $SED -e 's/\(go \)\(version \)\(go\)\(@<:@0-9@:>@.@<:@0-9@:>@.@<:@0-9@:>@\)\(@<:@\*@:>@*\).*/\4/'` + AC_MSG_RESULT($golang_version) + AC_SUBST([golang_version],[$golang_version]) + AX_COMPARE_VERSION([$ax_go_version],[le],[$golang_version],[ + : + have_go="yes" + ],[ + : + have_go="no" + ]) + ],[ + AC_MSG_WARN([could not find Go ]) + have_go="no" + ]) + fi +fi +AM_CONDITIONAL(WITH_GO, [test "$have_go" = "yes"]) + + +AX_THRIFT_LIB(haxe, [Haxe], yes) +if test "$with_haxe" = "yes"; then + AC_PATH_PROG([HAXE], [haxe]) + if [[ -x "$HAXE" ]] ; then + AX_PROG_HAXE_VERSION( [3.1.3], have_haxe="yes", have_haxe="no") + fi +fi +AM_CONDITIONAL(WITH_HAXE, [test "$have_haxe" = "yes"]) + + +AX_THRIFT_LIB(d, [D], yes) +if test "$with_d" = "yes"; then + AX_DMD + AC_SUBST(DMD) + if test "x$DMD" != "x"; then + have_d="yes" + fi +fi + +# Determine actual name of the generated D library for use in the command line +# when compiling tests. This is needed because the -l syntax doesn't work +# with OPTLINK (Windows). +lib_prefix=lib +lib_suffix=a +case "$host_os" in + cygwin* | mingw* | pw32* | cegcc*) + lib_prefix="" + lib_suffix=lib + ;; +esac +D_LIB_NAME="${lib_prefix}thriftd.${lib_suffix}" +AC_SUBST(D_LIB_NAME) +D_EVENT_LIB_NAME="${lib_prefix}thriftd-event.${lib_suffix}" +AC_SUBST(D_EVENT_LIB_NAME) +D_SSL_LIB_NAME="${lib_prefix}thriftd-ssl.${lib_suffix}" +AC_SUBST(D_SSL_LIB_NAME) + +if test "$have_d" = "yes"; then + AX_CHECK_D_MODULE(deimos.event2.event) + have_deimos_event2=$success + + with_d_event_tests="no" + if test "$have_deimos_event2" = "yes"; then + if test "x$DMD_LIBEVENT_FLAGS" = "x"; then + if test "$dmd_optlink" = "yes"; then + AC_MSG_WARN([D libevent interface found, but cannot auto-detect \ +linker flags for OPTLINK. Please set DMD_LIBEVENT_FLAGS manually.]) + else + AX_LIB_EVENT([2.0]) + if test "$success" = "yes"; then + DMD_LIBEVENT_FLAGS=$(echo "$LIBEVENT_LDFLAGS $LIBEVENT_LIBS" | \ + sed -e 's/^ *//g;s/ *$//g;s/^\(.\)/-L\1/g;s/ */ -L/g') + with_d_event_tests="yes" + else + AC_MSG_WARN([D libevent interface present, but libevent library not found.]) + fi + fi + else + with_d_event_tests="yes" + fi + fi + + AX_CHECK_D_MODULE(deimos.openssl.ssl) + have_deimos_openssl=$success + + with_d_ssl_tests="no" + if test "$have_deimos_openssl" = "yes"; then + if test "x$DMD_OPENSSL_FLAGS" = "x"; then + if test "$dmd_optlink" = "yes"; then + AC_MSG_WARN([D OpenSSL interface found, but cannot auto-detect \ +linker flags for OPTLINK. Please set DMD_OPENSSL_FLAGS manually.]) + else + AX_CHECK_OPENSSL([with_d_ssl_tests="yes"]) + if test "$with_d_ssl_tests" = "yes"; then + DMD_OPENSSL_FLAGS=$(echo "$OPENSSL_LDFLAGS $OPENSSL_LIBS" | \ + sed -e 's/^ *//g;s/ *$//g;s/^\(.\)/-L\1/g;s/ */ -L/g') + else + AC_MSG_WARN([D OpenSSL interface present, but OpenSSL library not found.]) + fi + fi + else + with_d_ssl_tests="yes" + fi + fi +fi + +AM_CONDITIONAL(WITH_D, [test "$have_d" = "yes"]) +AM_CONDITIONAL(DMD_OPTLINK, [test "$dmd_optlink" = "yes"]) +AC_SUBST(DMD_OF_DIRSEP, "$dmd_of_dirsep") +AM_CONDITIONAL(HAVE_DEIMOS_EVENT2, [test "$have_deimos_event2" = "yes"]) +AM_CONDITIONAL(WITH_D_EVENT_TESTS, [test "$with_d_event_tests" = "yes"]) +AC_SUBST(DMD_LIBEVENT_FLAGS) +AM_CONDITIONAL(HAVE_DEIMOS_OPENSSL, [test "$have_deimos_openssl" = "yes"]) +AM_CONDITIONAL(WITH_D_SSL_TESTS, [test "$with_d_ssl_tests" = "yes"]) +AC_SUBST(DMD_OPENSSL_FLAGS) + +AC_ARG_ENABLE([tests], + AS_HELP_STRING([--enable-tests], [build tests [default=yes]]), + [], enable_tests=yes +) +have_tests=yes +if test "$enable_tests" = "no"; then + have_tests="no" +fi +AM_CONDITIONAL(WITH_TESTS, [test "$have_tests" = "yes"]) + +AC_ARG_ENABLE([plugin], + AS_HELP_STRING([--enable-plugin], [build compiler plugin support [default=yes]]), + [], enable_plugin=yes +) +have_plugin=yes +if test "$have_cpp" = "no" ; then + have_plugin="no" +fi +if test "$enable_plugin" = "no"; then + have_plugin="no" +fi +if test "$have_plugin" = "yes" ; then + AC_CONFIG_LINKS([compiler/cpp/test/plugin/t_cpp_generator.cc:compiler/cpp/src/thrift/generate/t_cpp_generator.cc]) +fi +AM_CONDITIONAL(WITH_PLUGIN, [test "$have_plugin" = "yes"]) + +AC_ARG_ENABLE([tutorial], + AS_HELP_STRING([--enable-tutorial], [build tutorial [default=yes]]), + [], enable_tutorial=yes +) +have_tutorial=yes +if test "$enable_tutorial" = "no"; then + have_tutorial="no" +fi +AM_CONDITIONAL(WITH_TUTORIAL, [test "$have_tutorial" = "yes"]) + +AM_CONDITIONAL(MINGW, false) +case "${host_os}" in +*mingw*) + mingw32_support="yes" + AC_CHECK_HEADER(windows.h) + AM_CONDITIONAL(MINGW, true) + ;; +*) + AC_ISC_POSIX + ;; +esac + +AC_C_CONST +AC_C_INLINE +AC_C_VOLATILE + +AC_HEADER_STDBOOL +AC_HEADER_STDC +AC_HEADER_TIME +AC_HEADER_SYS_WAIT +AC_TYPE_SIGNAL +AC_CHECK_HEADERS([arpa/inet.h]) +AC_CHECK_HEADERS([sys/param.h]) +AC_CHECK_HEADERS([fcntl.h]) +AC_CHECK_HEADERS([inttypes.h]) +AC_CHECK_HEADERS([limits.h]) +AC_CHECK_HEADERS([netdb.h]) +AC_CHECK_HEADERS([netinet/in.h]) +AC_CHECK_HEADERS([pthread.h]) +AC_CHECK_HEADERS([stddef.h]) +AC_CHECK_HEADERS([stdlib.h]) +AC_CHECK_HEADERS([sys/socket.h]) +AC_CHECK_HEADERS([sys/time.h]) +AC_CHECK_HEADERS([sys/un.h]) +AC_CHECK_HEADERS([sys/poll.h]) +AC_CHECK_HEADERS([sys/resource.h]) +AC_CHECK_HEADERS([unistd.h]) +AC_CHECK_HEADERS([libintl.h]) +AC_CHECK_HEADERS([malloc.h]) +AC_CHECK_HEADERS([openssl/ssl.h]) +AC_CHECK_HEADERS([openssl/rand.h]) +AC_CHECK_HEADERS([openssl/x509v3.h]) +AC_CHECK_HEADERS([sched.h]) +AC_CHECK_HEADERS([wchar.h]) + +AC_CHECK_LIB(pthread, pthread_create) +dnl NOTE(dreiss): I haven't been able to find any really solid docs +dnl on what librt is and how it fits into various Unix systems. +dnl My best guess is that it is where glibc stashes its implementation +dnl of the POSIX Real-Time Extensions. This seems necessary on Linux, +dnl and we haven't yet found a system where this is a problem. +AC_CHECK_LIB(rt, clock_gettime) +AC_CHECK_LIB(socket, setsockopt) + +AC_TYPE_INT16_T +AC_TYPE_INT32_T +AC_TYPE_INT64_T +AC_TYPE_INT8_T +AC_TYPE_MODE_T +AC_TYPE_OFF_T +AC_TYPE_SIZE_T +AC_TYPE_SSIZE_T +AC_TYPE_UINT16_T +AC_TYPE_UINT32_T +AC_TYPE_UINT64_T +AC_TYPE_UINT8_T +AC_CHECK_TYPES([ptrdiff_t], [], [exit 1]) + +AC_STRUCT_TM + +dnl NOTE(dreiss): AI_ADDRCONFIG is not defined on OpenBSD. +AC_CHECK_DECL([AI_ADDRCONFIG], [], + [AC_DEFINE([AI_ADDRCONFIG], 0, + [Define if the AI_ADDRCONFIG symbol is unavailable])], + [ + #include + #include + #include +]) + +AC_FUNC_ALLOCA +AC_FUNC_FORK +AC_FUNC_MALLOC +AC_FUNC_MEMCMP +AC_FUNC_REALLOC +AC_FUNC_SELECT_ARGTYPES +AC_FUNC_STAT +AC_FUNC_STRERROR_R +AC_FUNC_STRFTIME +AC_FUNC_VPRINTF +AC_CHECK_FUNCS([strtoul]) +AC_CHECK_FUNCS([bzero]) +AC_CHECK_FUNCS([ftruncate]) +AC_CHECK_FUNCS([gethostbyname]) +AC_CHECK_FUNCS([gethostbyname_r]) +AC_CHECK_FUNCS([gettimeofday]) +AC_CHECK_FUNCS([memmove]) +AC_CHECK_FUNCS([memset]) +AC_CHECK_FUNCS([mkdir]) +AC_CHECK_FUNCS([realpath]) +AC_CHECK_FUNCS([select]) +AC_CHECK_FUNCS([setlocale]) +AC_CHECK_FUNCS([socket]) +AC_CHECK_FUNCS([strchr]) +AC_CHECK_FUNCS([strdup]) +AC_CHECK_FUNCS([strerror]) +AC_CHECK_FUNCS([strstr]) +AC_CHECK_FUNCS([strtol]) +AC_CHECK_FUNCS([sqrt]) +dnl The following functions are optional. +AC_CHECK_FUNCS([alarm]) +AC_CHECK_FUNCS([clock_gettime]) +AC_CHECK_FUNCS([sched_get_priority_min]) +AC_CHECK_FUNCS([sched_get_priority_max]) +AC_CHECK_FUNCS([inet_ntoa]) +AC_CHECK_FUNCS([pow]) + +if test "$cross_compiling" = "no" ; then + AX_SIGNED_RIGHT_SHIFT +fi + +dnl autoscan thinks we need this macro because we have a member function +dnl called "error". Invoke the macro but don't run the check so autoscan +dnl thinks we are in the clear. It's highly unlikely that we will ever +dnl actually use the function that this checks for. +if false ; then + AC_FUNC_ERROR_AT_LINE +fi + +# --- Coverage hooks --- + +AC_ARG_ENABLE(coverage, + [ --enable-coverage turn on -fprofile-arcs -ftest-coverage], + [case "${enableval}" in + yes) ENABLE_COVERAGE=1 ;; + no) ENABLE_COVERAGE=0 ;; + *) AC_MSG_ERROR(bad value ${enableval} for --enable-cov) ;; + esac], + [ENABLE_COVERAGE=2]) + +if test "x[$]ENABLE_COVERAGE" = "x1"; then + AC_MSG_WARN(enable coverage) + GCOV_CFLAGS="`echo \"[$]CFLAGS\" | perl -pe 's/-O\d+//g;'` -fprofile-arcs -ftest-coverage" + GCOV_CXXFLAGS="`echo \"[$]CXXFLAGS\" | perl -pe 's/-O\d+//g;'` -fprofile-arcs -ftest-coverage" + GCOV_LDFLAGS="-XCClinker -fprofile-arcs -XCClinker -ftest-coverage" +fi + +AC_SUBST(ENABLE_COVERAGE) +AC_SUBST(GCOV_CFLAGS) +AC_SUBST(GCOV_CXXFLAGS) +AC_SUBST(GCOV_LDFLAGS) + +AC_ARG_ENABLE(boostthreads, + [ --enable-boostthreads use boost threads, instead of POSIX pthread (experimental) ], + [case "${enableval}" in + yes) ENABLE_BOOSTTHREADS=1 ;; + no) ENABLE_BOOSTTHREADS=0 ;; + *) AC_MSG_ERROR(bad value ${enableval} for --enable-cov) ;; + esac], + [ENABLE_BOOSTTHREADS=2]) + + +if test "x[$]ENABLE_BOOSTTHREADS" = "x1"; then + AC_MSG_WARN(enable boostthreads) + AC_DEFINE([USE_BOOST_THREAD], [1], [experimental --enable-boostthreads that replaces POSIX pthread by boost::thread]) + LIBS="-lboost_thread $LIBS" +fi + +AM_CONDITIONAL([WITH_BOOSTTHREADS], [test "x[$]ENABLE_BOOSTTHREADS" = "x1"]) + +AC_CONFIG_HEADERS(config.h:config.hin) +AC_CONFIG_HEADERS(lib/cpp/src/thrift/config.h:config.hin) +AC_CONFIG_HEADERS(lib/c_glib/src/thrift/config.h:config.hin) +# gruard against pre defined config.h +AH_TOP([ +#ifndef CONFIG_H +#define CONFIG_H +]) +AH_BOTTOM([ +#endif +]) + + +AC_CONFIG_FILES([ + Makefile + compiler/cpp/Makefile + compiler/cpp/src/Makefile + compiler/cpp/src/thrift/plugin/Makefile + compiler/cpp/test/Makefile + compiler/cpp/src/thrift/version.h + lib/Makefile + lib/cpp/Makefile + lib/cpp/test/Makefile + lib/cpp/thrift-nb.pc + lib/cpp/thrift-z.pc + lib/cpp/thrift-qt.pc + lib/cpp/thrift-qt5.pc + lib/cpp/thrift.pc + lib/c_glib/Makefile + lib/c_glib/thrift_c_glib.pc + lib/c_glib/test/Makefile + lib/csharp/Makefile + lib/csharp/test/Multiplex/Makefile + lib/d/Makefile + lib/d/test/Makefile + lib/erl/Makefile + lib/go/Makefile + lib/go/test/Makefile + lib/haxe/test/Makefile + lib/hs/Makefile + lib/java/Makefile + lib/js/Makefile + lib/js/test/Makefile + lib/json/Makefile + lib/json/test/Makefile + lib/nodejs/Makefile + lib/perl/Makefile + lib/perl/test/Makefile + lib/php/Makefile + lib/php/test/Makefile + lib/dart/Makefile + lib/py/Makefile + lib/rb/Makefile + lib/lua/Makefile + lib/xml/Makefile + lib/xml/test/Makefile + test/Makefile + test/features/Makefile + test/c_glib/Makefile + test/cpp/Makefile + test/csharp/Makefile + test/erl/Makefile + test/go/Makefile + test/haxe/Makefile + test/hs/Makefile + test/lua/Makefile + test/php/Makefile + test/dart/Makefile + test/perl/Makefile + test/py/Makefile + test/py.twisted/Makefile + test/py.tornado/Makefile + test/rb/Makefile + tutorial/Makefile + tutorial/c_glib/Makefile + tutorial/cpp/Makefile + tutorial/d/Makefile + tutorial/go/Makefile + tutorial/haxe/Makefile + tutorial/hs/Makefile + tutorial/java/Makefile + tutorial/js/Makefile + tutorial/nodejs/Makefile + tutorial/dart/Makefile + tutorial/py/Makefile + tutorial/py.twisted/Makefile + tutorial/py.tornado/Makefile + tutorial/rb/Makefile +]) + +if test "$have_cpp" = "yes" ; then MAYBE_CPP="cpp" ; else MAYBE_CPP="" ; fi +AC_SUBST([MAYBE_CPP]) +if test "$have_c_glib" = "yes" ; then MAYBE_C_GLIB="c_glib" ; else MAYBE_C_GLIB="" ; fi +AC_SUBST([MAYBE_C_GLIB]) +if test "$have_d" = "yes" -a "$have_deimos_event2" = "yes" -a "$have_deimos_openssl" = "yes"; then MAYBE_D="d" ; else MAYBE_D="" ; fi +AC_SUBST([MAYBE_D]) +if test "$have_java" = "yes" ; then MAYBE_JAVA="java" ; else MAYBE_JAVA="" ; fi +AC_SUBST([MAYBE_JAVA]) +if test "$have_csharp" = "yes" ; then MAYBE_CSHARP="csharp" ; else MAYBE_CSHARP="" ; fi +AC_SUBST([MAYBE_CSHARP]) +if test "$have_python" = "yes" ; then MAYBE_PYTHON="py" ; else MAYBE_PYTHON="" ; fi +AC_SUBST([MAYBE_PYTHON]) +if test "$have_py3" = "yes" ; then MAYBE_PY3="py3" ; else MAYBE_PY3="" ; fi +AC_SUBST([MAYBE_PY3]) +if test "$have_ruby" = "yes" ; then MAYBE_RUBY="rb" ; else MAYBE_RUBY="" ; fi +AC_SUBST([MAYBE_RUBY]) +if test "$have_haskell" = "yes" ; then MAYBE_HASKELL="hs" ; else MAYBE_HASKELL="" ; fi +AC_SUBST([MAYBE_HASKELL]) +if test "$have_perl" = "yes" ; then MAYBE_PERL="perl" ; else MAYBE_PERL="" ; fi +AC_SUBST([MAYBE_PERL]) +if test "$have_php" = "yes" ; then MAYBE_PHP="php" ; else MAYBE_PHP="" ; fi +AC_SUBST([MAYBE_PHP]) +if test "$have_dart" = "yes" ; then MAYBE_DART="dart" ; else MAYBE_DART="" ; fi +AC_SUBST([MAYBE_DART]) +if test "$have_go" = "yes" ; then MAYBE_GO="go" ; else MAYBE_GO="" ; fi +AC_SUBST([MAYBE_GO]) +if test "$have_nodejs" = "yes" ; then MAYBE_NODEJS="nodejs" ; else MAYBE_NODEJS="" ; fi +AC_SUBST([MAYBE_NODEJS]) +if test "$have_erlang" = "yes" ; then MAYBE_ERLANG="erl" ; else MAYBE_ERLANG="" ; fi +AC_SUBST([MAYBE_ERLANG]) +if test "$have_lua" = "yes" ; then MAYBE_LUA="lua" ; else MAYBE_LUA="" ; fi +AC_SUBST([MAYBE_LUA]) + +AC_OUTPUT + + +echo +echo "$PACKAGE $VERSION" +echo +echo "Building Plugin Support ...... : $have_plugin" +echo "Building C++ Library ......... : $have_cpp" +echo "Building C (GLib) Library .... : $have_c_glib" +echo "Building Java Library ........ : $have_java" +echo "Building C# Library .......... : $have_csharp" +echo "Building Python Library ...... : $have_python" +echo "Building Ruby Library ........ : $have_ruby" +echo "Building Haxe Library ........ : $have_haxe" +echo "Building Haskell Library ..... : $have_haskell" +echo "Building Perl Library ........ : $have_perl" +echo "Building PHP Library ......... : $have_php" +echo "Building Dart Library ........ : $have_dart" +echo "Building Erlang Library ...... : $have_erlang" +echo "Building Go Library .......... : $have_go" +echo "Building D Library ........... : $have_d" +echo "Building NodeJS Library ...... : $have_nodejs" +echo "Building Lua Library ......... : $have_lua" + +if test "$have_cpp" = "yes" ; then + echo + echo "C++ Library:" + echo " Build TZlibTransport ...... : $have_zlib" + echo " Build TNonblockingServer .. : $have_libevent" + echo " Build TQTcpServer (Qt4) .... : $have_qt" + echo " Build TQTcpServer (Qt5) .... : $have_qt5" +fi +if test "$have_java" = "yes" ; then + echo + echo "Java Library:" + echo " Using javac ............... : $JAVAC" + echo " Using java ................ : $JAVA" + echo " Using ant ................. : $ANT" +fi +if test "$have_csharp" = "yes" ; then + echo + echo "C# Library:" + echo " Using .NET 3.5 ............ : $net_3_5" +fi +if test "$have_python" = "yes" ; then + echo + echo "Python Library:" + echo " Using Python .............. : $PYTHON" + if test "$have_py3" = "yes" ; then + echo " Using Python3 ............. : $PYTHON3" + fi + if test "$have_trial" = "yes"; then + echo " Using trial ............... : $TRIAL" + fi +fi +if test "$have_php" = "yes" ; then + echo + echo "PHP Library:" + echo " Using php-config .......... : $PHP_CONFIG" +fi +if test "$have_dart" = "yes" ; then + echo + echo "Dart Library:" + echo " Using Dart ................ : $DART" + echo " Using Pub ................. : $DARTPUB" +fi +if test "$have_ruby" = "yes" ; then + echo + echo "Ruby Library:" + echo " Using Ruby ................ : $RUBY" +fi +if test "$have_haskell" = "yes" ; then + echo + echo "Haskell Library:" + echo " Using Haskell ............. : $RUNHASKELL" + echo " Using Cabal ............... : $CABAL" +fi +if test "$have_haxe" = "yes" ; then + echo + echo "Haxe Library:" + echo " Using Haxe ................ : $HAXE" + echo " Using Haxe version ........ : $HAXE_VERSION" +fi +if test "$have_perl" = "yes" ; then + echo + echo "Perl Library:" + echo " Using Perl ................ : $PERL" +fi +if test "$have_erlang" = "yes" ; then + echo + echo "Erlang Library:" + echo " Using erlc ................ : $ERLC" + echo " Using rebar ............... : $REBAR" +fi +if test "$have_go" = "yes" ; then + echo + echo "Go Library:" + echo " Using Go................... : $GO" + echo " Using Go version........... : $($GO version)" +fi +if test "$have_d" = "yes" ; then + echo + echo "D Library:" + echo " Using D Compiler .......... : $DMD" + echo " Building D libevent tests . : $with_d_event_tests" + echo " Building D SSL tests ...... : $with_d_ssl_tests" +fi +if test "$have_nodejs" = "yes" ; then + echo + echo "NodeJS Library:" + echo " Using NodeJS .............. : $NODEJS" + echo " Using NodeJS version....... : $($NODEJS --version)" +fi +if test "$have_lua" = "yes" ; then + echo + echo "Lua Library:" + echo " Using Lua .............. : $LUA" +fi +echo +echo "If something is missing that you think should be present," +echo "please skim the output of configure to find the missing" +echo "component. Details are present in config.log." diff --git a/vendor/github.com/apache/thrift/contrib/fb303/LICENSE b/vendor/github.com/apache/thrift/contrib/fb303/LICENSE deleted file mode 100644 index 4eacb64317973..0000000000000 --- a/vendor/github.com/apache/thrift/contrib/fb303/LICENSE +++ /dev/null @@ -1,16 +0,0 @@ -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you 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. diff --git a/vendor/github.com/apache/thrift/doap.rdf b/vendor/github.com/apache/thrift/doap.rdf new file mode 100644 index 0000000000000..14a8d6638d562 --- /dev/null +++ b/vendor/github.com/apache/thrift/doap.rdf @@ -0,0 +1,132 @@ + + + + + + 2012-04-14 + + Apache Thrift + + + Apache Thrift software provides a framework for scalable cross-language services development, combines a software stack with a code generation engine to build services that work efficiently and seamlessly between C++, Java, Python, PHP, Ruby, Erlang, Perl, Haskell, C#, Cocoa, JavaScript, Node.js, Smalltalk, OCaml and Delphi and other languages. + Apache Thrift allows you to define data types and service interfaces in a simple definition file. Taking that file as input, the compiler generates code to be used to easily build RPC clients and servers that communicate seamlessly across programming languages. Instead of writing a load of boilerplate code to serialize and transport your objects and invoke remote methods, you can get right down to business. + + + + ActionScript + C + C# + C++ + Cocoa + D + Delphi + Erlang + Go + Haskell + Java + JavaScript + node.js + OCaml + Perl + PHP + Python + SmallTalk + + + + + + + Apache Thrift + 2015-09-25 + 0.9.3 + + + Apache Thrift + 2014-11-05 + 0.9.2 + + + Apache Thrift + 2013-08-22 + 0.9.1 + + + Apache Thrift + 2012-10-15 + 0.9.0 + + + Apache Thrift + 2011-11-29 + 0.8.0 + + + Apache Thrift + 2011-08-13 + 0.7.0 + + + Apache Thrift + 2011-04-25 + 0.6.1 + + + Apache Thrift + 2011-02-08 + 0.6.0 + + + Apache Thrift (incubating) + 2010-10-07 + 0.5.0 + + + Apache Thrift (incubating) + 2010-08-23 + 0.4.0 + + + Apache Thrift (incubating) + 2010-08-04 + 0.3.0 + + + Apache Thrift (incubating) + 2009-12-11 + 0.2.0 + + + + + + + + + + + Apache Thrift PMC + + + + + diff --git a/vendor/github.com/apache/thrift/lib/Makefile.am b/vendor/github.com/apache/thrift/lib/Makefile.am new file mode 100644 index 0000000000000..5f17fca882d42 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/Makefile.am @@ -0,0 +1,109 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +SUBDIRS = json xml +PRECROSS_TARGET = + +if WITH_CPP +# cpp dir is picked directly by plugin build +if !WITH_PLUGIN +SUBDIRS += cpp +endif +endif + +if WITH_C_GLIB +SUBDIRS += c_glib +endif + +if WITH_MONO +SUBDIRS += csharp +endif + +if WITH_JAVA +SUBDIRS += java +PRECROSS_TARGET += precross-java +# JavaScript unit test depends on java +# so test only if java, ant & co is available +SUBDIRS += js +endif + +if WITH_PYTHON +SUBDIRS += py +endif + +if WITH_ERLANG +SUBDIRS += erl +endif + +if WITH_RUBY +SUBDIRS += rb +endif + +if WITH_HASKELL +SUBDIRS += hs +endif + +if WITH_PERL +SUBDIRS += perl +endif + +if WITH_PHP +SUBDIRS += php +endif + +if WITH_DART +SUBDIRS += dart +endif + +if WITH_GO +SUBDIRS += go +endif + +if WITH_D +SUBDIRS += d +PRECROSS_TARGET += precross-d +endif + +if WITH_NODEJS +SUBDIRS += nodejs +PRECROSS_TARGET += precross-nodejs +endif + +if WITH_LUA +SUBDIRS += lua +endif + +# All of the libs that don't use Automake need to go in here +# so they will end up in our release tarballs. +EXTRA_DIST = \ + as3 \ + cocoa \ + d \ + dart \ + delphi \ + haxe \ + javame \ + js \ + ocaml \ + st \ + ts + +precross-%: + $(MAKE) -C $* precross +precross: $(PRECROSS_TARGET) diff --git a/vendor/github.com/apache/thrift/lib/dart/LICENSE_HEADER b/vendor/github.com/apache/thrift/lib/dart/LICENSE_HEADER deleted file mode 100644 index 4eacb64317973..0000000000000 --- a/vendor/github.com/apache/thrift/lib/dart/LICENSE_HEADER +++ /dev/null @@ -1,16 +0,0 @@ -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you 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. diff --git a/vendor/github.com/apache/thrift/lib/go/Makefile.am b/vendor/github.com/apache/thrift/lib/go/Makefile.am new file mode 100644 index 0000000000000..ff946ea8d0cfd --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/Makefile.am @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +SUBDIRS = . + +if WITH_TESTS +SUBDIRS += test +endif + +install: + @echo '##############################################################' + @echo '##############################################################' + @echo 'The Go client library should be installed via "go get", please see /lib/go/README.md' + @echo '##############################################################' + @echo '##############################################################' + +check-local: + $(GO) test ./thrift + +all-local: + $(GO) build ./thrift + +EXTRA_DIST = \ + thrift \ + coding_standards.md \ + README.md diff --git a/vendor/github.com/apache/thrift/lib/go/README.md b/vendor/github.com/apache/thrift/lib/go/README.md new file mode 100644 index 0000000000000..7440474c325c1 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/README.md @@ -0,0 +1,81 @@ +Thrift Go Software Library + +License +======= + +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. + + +Using Thrift with Go +==================== + +In following Go conventions, we recommend you use the 'go' tool to install +Thrift for go. + + $ go get git.apache.org/thrift.git/lib/go/thrift/... + +Will retrieve and install the most recent version of the package. + + +A note about optional fields +============================ + +The thrift-to-Go compiler tries to represent thrift IDL structs as Go structs. +We must be able to distinguish between optional fields that are set to their +default value and optional values which are actually unset, so the generated +code represents optional fields via pointers. + +This is generally intuitive and works well much of the time, but Go does not +have a syntax for creating a pointer to a constant in a single expression. That +is, given a struct like + + struct SomeIDLType { + OptionalField *int32 + } + +, the following will not compile: + + x := &SomeIDLType{ + OptionalField: &(3), + } + +(Nor is there any other syntax that's built in to the language) + +As such, we provide some helpers that do just this under lib/go/thrift/. E.g., + + x := &SomeIDLType{ + OptionalField: thrift.Int32Ptr(3), + } + +And so on. The code generator also creates analogous helpers for user-defined +typedefs and enums. + +Adding custom tags to generated Thrift structs +============================================== + +You can add tags to the auto-generated thrift structs using the following format: + + struct foo { + 1: required string Bar (go.tag = "some_tag:\"some_tag_value\"") + } + +which will generate: + + type Foo struct { + Bar string `thrift:"bar,1,required" some_tag:"some_tag_value"` + } diff --git a/vendor/github.com/apache/thrift/lib/go/coding_standards.md b/vendor/github.com/apache/thrift/lib/go/coding_standards.md new file mode 100644 index 0000000000000..fa0390bb57747 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/coding_standards.md @@ -0,0 +1 @@ +Please follow [General Coding Standards](/doc/coding_standards.md) diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/application_exception_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/application_exception_test.go new file mode 100644 index 0000000000000..7010f868f1892 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/application_exception_test.go @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestTApplicationException(t *testing.T) { + exc := NewTApplicationException(UNKNOWN_APPLICATION_EXCEPTION, "") + if exc.Error() != "" { + t.Fatalf("Expected empty string for exception but found '%s'", exc.Error()) + } + if exc.TypeId() != UNKNOWN_APPLICATION_EXCEPTION { + t.Fatalf("Expected type UNKNOWN for exception but found '%s'", exc.TypeId()) + } + exc = NewTApplicationException(WRONG_METHOD_NAME, "junk_method") + if exc.Error() != "junk_method" { + t.Fatalf("Expected 'junk_method' for exception but found '%s'", exc.Error()) + } + if exc.TypeId() != WRONG_METHOD_NAME { + t.Fatalf("Expected type WRONG_METHOD_NAME for exception but found '%s'", exc.TypeId()) + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/binary_protocol_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/binary_protocol_test.go new file mode 100644 index 0000000000000..0462cc79deead --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/binary_protocol_test.go @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestReadWriteBinaryProtocol(t *testing.T) { + ReadWriteProtocolTest(t, NewTBinaryProtocolFactoryDefault()) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/buffered_transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/buffered_transport_test.go new file mode 100644 index 0000000000000..95ec0cbd2c012 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/buffered_transport_test.go @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestBufferedTransport(t *testing.T) { + trans := NewTBufferedTransport(NewTMemoryBuffer(), 10240) + TransportTest(t, trans, trans) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/compact_protocol_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/compact_protocol_test.go new file mode 100644 index 0000000000000..72812f9cb0d28 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/compact_protocol_test.go @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "bytes" + "testing" +) + +func TestReadWriteCompactProtocol(t *testing.T) { + ReadWriteProtocolTest(t, NewTCompactProtocolFactory()) + transports := []TTransport{ + NewTMemoryBuffer(), + NewStreamTransportRW(bytes.NewBuffer(make([]byte, 0, 16384))), + NewTFramedTransport(NewTMemoryBuffer()), + } + for _, trans := range transports { + p := NewTCompactProtocol(trans); + ReadWriteBool(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteByte(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteI16(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteI32(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteI64(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteDouble(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteString(t, p, trans); + p = NewTCompactProtocol(trans); + ReadWriteBinary(t, p, trans); + trans.Close(); + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/exception_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/exception_test.go new file mode 100644 index 0000000000000..71f5e2c7e79e6 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/exception_test.go @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "errors" + "testing" +) + +func TestPrependError(t *testing.T) { + err := NewTApplicationException(INTERNAL_ERROR, "original error") + err2, ok := PrependError("Prepend: ", err).(TApplicationException) + if !ok { + t.Fatal("Couldn't cast error TApplicationException") + } + if err2.Error() != "Prepend: original error" { + t.Fatal("Unexpected error string") + } + if err2.TypeId() != INTERNAL_ERROR { + t.Fatal("Unexpected type error") + } + + err3 := NewTProtocolExceptionWithType(INVALID_DATA, errors.New("original error")) + err4, ok := PrependError("Prepend: ", err3).(TProtocolException) + if !ok { + t.Fatal("Couldn't cast error TProtocolException") + } + if err4.Error() != "Prepend: original error" { + t.Fatal("Unexpected error string") + } + if err4.TypeId() != INVALID_DATA { + t.Fatal("Unexpected type error") + } + + err5 := NewTTransportException(TIMED_OUT, "original error") + err6, ok := PrependError("Prepend: ", err5).(TTransportException) + if !ok { + t.Fatal("Couldn't cast error TTransportException") + } + if err6.Error() != "Prepend: original error" { + t.Fatal("Unexpected error string") + } + if err6.TypeId() != TIMED_OUT { + t.Fatal("Unexpected type error") + } + + err7 := errors.New("original error") + err8 := PrependError("Prepend: ", err7) + if err8.Error() != "Prepend: original error" { + t.Fatal("Unexpected error string") + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/framed_transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/framed_transport_test.go new file mode 100644 index 0000000000000..8f683ef306251 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/framed_transport_test.go @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestFramedTransport(t *testing.T) { + trans := NewTFramedTransport(NewTMemoryBuffer()) + TransportTest(t, trans, trans) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/http_client_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/http_client_test.go new file mode 100644 index 0000000000000..453680ace87b3 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/http_client_test.go @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "net/http" + "testing" +) + +func TestHttpClient(t *testing.T) { + l, addr := HttpClientSetupForTest(t) + if l != nil { + defer l.Close() + } + trans, err := NewTHttpPostClient("http://" + addr.String()) + if err != nil { + l.Close() + t.Fatalf("Unable to connect to %s: %s", addr.String(), err) + } + TransportTest(t, trans, trans) +} + +func TestHttpClientHeaders(t *testing.T) { + l, addr := HttpClientSetupForTest(t) + if l != nil { + defer l.Close() + } + trans, err := NewTHttpPostClient("http://" + addr.String()) + if err != nil { + l.Close() + t.Fatalf("Unable to connect to %s: %s", addr.String(), err) + } + TransportHeaderTest(t, trans, trans) +} + +func TestHttpCustomClient(t *testing.T) { + l, addr := HttpClientSetupForTest(t) + if l != nil { + defer l.Close() + } + + httpTransport := &customHttpTransport{} + + trans, err := NewTHttpPostClientWithOptions("http://"+addr.String(), THttpClientOptions{ + Client: &http.Client{ + Transport: httpTransport, + }, + }) + if err != nil { + l.Close() + t.Fatalf("Unable to connect to %s: %s", addr.String(), err) + } + TransportHeaderTest(t, trans, trans) + + if !httpTransport.hit { + t.Fatalf("Custom client was not used") + } +} + +func TestHttpCustomClientPackageScope(t *testing.T) { + l, addr := HttpClientSetupForTest(t) + if l != nil { + defer l.Close() + } + httpTransport := &customHttpTransport{} + DefaultHttpClient = &http.Client{ + Transport: httpTransport, + } + + trans, err := NewTHttpPostClient("http://" + addr.String()) + if err != nil { + l.Close() + t.Fatalf("Unable to connect to %s: %s", addr.String(), err) + } + TransportHeaderTest(t, trans, trans) + + if !httpTransport.hit { + t.Fatalf("Custom client was not used") + } +} + +type customHttpTransport struct { + hit bool +} + +func (c *customHttpTransport) RoundTrip(req *http.Request) (*http.Response, error) { + c.hit = true + return http.DefaultTransport.RoundTrip(req) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/iostream_transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/iostream_transport_test.go new file mode 100644 index 0000000000000..15a6116427d05 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/iostream_transport_test.go @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "bytes" + "testing" +) + +func TestStreamTransport(t *testing.T) { + trans := NewStreamTransportRW(bytes.NewBuffer(make([]byte, 0, 1024))) + TransportTest(t, trans, trans) +} + +func TestStreamTransportOpenClose(t *testing.T) { + trans := NewStreamTransportRW(bytes.NewBuffer(make([]byte, 0, 1024))) + if !trans.IsOpen() { + t.Fatal("StreamTransport should be already open") + } + if trans.Open() == nil { + t.Fatal("StreamTransport should return error when open twice") + } + if trans.Close() != nil { + t.Fatal("StreamTransport should not return error when closing open transport") + } + if trans.IsOpen() { + t.Fatal("StreamTransport should not be open after close") + } + if trans.Close() == nil { + t.Fatal("StreamTransport should return error when closing a non open transport") + } + if trans.Open() == nil { + t.Fatal("StreamTransport should not be able to reopen") + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/json_protocol_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/json_protocol_test.go new file mode 100644 index 0000000000000..7104ce3a02e78 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/json_protocol_test.go @@ -0,0 +1,649 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "encoding/base64" + "encoding/json" + "fmt" + "math" + "strconv" + "testing" +) + +func TestWriteJSONProtocolBool(t *testing.T) { + thetype := "boolean" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range BOOL_VALUES { + if e := p.WriteBool(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + expected := "" + if value { + expected = "1" + } else { + expected = "0" + } + if s != expected { + t.Fatalf("Bad value for %s %v: %s expected", thetype, value, s) + } + v := -1 + if err := json.Unmarshal([]byte(s), &v); err != nil || (v != 0) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolBool(t *testing.T) { + thetype := "boolean" + for _, value := range BOOL_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + if value { + trans.Write([]byte{'1'}) // not JSON_TRUE + } else { + trans.Write([]byte{'0'}) // not JSON_FALSE + } + trans.Flush() + s := trans.String() + v, e := p.ReadBool() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + vv := -1 + if err := json.Unmarshal([]byte(s), &vv); err != nil || (vv != 0) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, vv) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolByte(t *testing.T) { + thetype := "byte" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range BYTE_VALUES { + if e := p.WriteByte(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int8(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolByte(t *testing.T) { + thetype := "byte" + for _, value := range BYTE_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadByte() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolI16(t *testing.T) { + thetype := "int16" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range INT16_VALUES { + if e := p.WriteI16(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int16(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolI16(t *testing.T) { + thetype := "int16" + for _, value := range INT16_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadI16() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolI32(t *testing.T) { + thetype := "int32" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range INT32_VALUES { + if e := p.WriteI32(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int32(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolI32(t *testing.T) { + thetype := "int32" + for _, value := range INT32_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadI32() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolI64(t *testing.T) { + thetype := "int64" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range INT64_VALUES { + if e := p.WriteI64(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolI64(t *testing.T) { + thetype := "int64" + for _, value := range INT64_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(strconv.FormatInt(value, 10)) + trans.Flush() + s := trans.String() + v, e := p.ReadI64() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolDouble(t *testing.T) { + thetype := "double" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if math.IsInf(value, 1) { + if s != jsonQuote(JSON_INFINITY) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_INFINITY)) + } + } else if math.IsInf(value, -1) { + if s != jsonQuote(JSON_NEGATIVE_INFINITY) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_NEGATIVE_INFINITY)) + } + } else if math.IsNaN(value) { + if s != jsonQuote(JSON_NAN) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_NAN)) + } + } else { + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := float64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolDouble(t *testing.T) { + thetype := "double" + for _, value := range DOUBLE_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + n := NewNumericFromDouble(value) + trans.WriteString(n.String()) + trans.Flush() + s := trans.String() + v, e := p.ReadDouble() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if math.IsInf(value, 1) { + if !math.IsInf(v, 1) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else if math.IsInf(value, -1) { + if !math.IsInf(v, -1) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else if math.IsNaN(value) { + if !math.IsNaN(v) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else { + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolString(t *testing.T) { + thetype := "string" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + for _, value := range STRING_VALUES { + if e := p.WriteString(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s[0] != '"' || s[len(s)-1] != '"' { + t.Fatalf("Bad value for %s '%v', wrote '%v', expected: %v", thetype, value, s, fmt.Sprint("\"", value, "\"")) + } + v := new(string) + if err := json.Unmarshal([]byte(s), v); err != nil || *v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadJSONProtocolString(t *testing.T) { + thetype := "string" + for _, value := range STRING_VALUES { + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(jsonQuote(value)) + trans.Flush() + s := trans.String() + v, e := p.ReadString() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + v1 := new(string) + if err := json.Unmarshal([]byte(s), v1); err != nil || *v1 != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v1) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteJSONProtocolBinary(t *testing.T) { + thetype := "binary" + value := protocol_bdata + b64value := make([]byte, base64.StdEncoding.EncodedLen(len(protocol_bdata))) + base64.StdEncoding.Encode(b64value, value) + b64String := string(b64value) + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + if e := p.WriteBinary(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + expectedString := fmt.Sprint("\"", b64String, "\"") + if s != expectedString { + t.Fatalf("Bad value for %s %v\n wrote: \"%v\"\nexpected: \"%v\"", thetype, value, s, expectedString) + } + v1, err := p.ReadBinary() + if err != nil { + t.Fatalf("Unable to read binary: %s", err.Error()) + } + if len(v1) != len(value) { + t.Fatalf("Invalid value for binary\nexpected: \"%v\"\n read: \"%v\"", value, v1) + } + for k, v := range value { + if v1[k] != v { + t.Fatalf("Invalid value for binary at %v\nexpected: \"%v\"\n read: \"%v\"", k, v, v1[k]) + } + } + trans.Close() +} + +func TestReadJSONProtocolBinary(t *testing.T) { + thetype := "binary" + value := protocol_bdata + b64value := make([]byte, base64.StdEncoding.EncodedLen(len(protocol_bdata))) + base64.StdEncoding.Encode(b64value, value) + b64String := string(b64value) + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + trans.WriteString(jsonQuote(b64String)) + trans.Flush() + s := trans.String() + v, e := p.ReadBinary() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if len(v) != len(value) { + t.Fatalf("Bad value for %s value length %v, wrote: %v, received length: %v", thetype, len(value), s, len(v)) + } + for i := 0; i < len(v); i++ { + if v[i] != value[i] { + t.Fatalf("Bad value for %s at index %d value %v, wrote: %v, received: %v", thetype, i, value[i], s, v[i]) + } + } + v1 := new(string) + if err := json.Unmarshal([]byte(s), v1); err != nil || *v1 != b64String { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v1) + } + trans.Reset() + trans.Close() +} + +func TestWriteJSONProtocolList(t *testing.T) { + thetype := "list" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + p.WriteListBegin(TType(DOUBLE), len(DOUBLE_VALUES)) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteListEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + str1 := new([]interface{}) + err := json.Unmarshal([]byte(str), str1) + if err != nil { + t.Fatalf("Unable to decode %s, wrote: %s", thetype, str) + } + l := *str1 + if len(l) < 2 { + t.Fatalf("List must be at least of length two to include metadata") + } + if l[0] != "dbl" { + t.Fatal("Invalid type for list, expected: ", STRING, ", but was: ", l[0]) + } + if int(l[1].(float64)) != len(DOUBLE_VALUES) { + t.Fatal("Invalid length for list, expected: ", len(DOUBLE_VALUES), ", but was: ", l[1]) + } + for k, value := range DOUBLE_VALUES { + s := l[k+2] + if math.IsInf(value, 1) { + if s.(string) != JSON_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_INFINITY), str) + } + } else if math.IsInf(value, 0) { + if s.(string) != JSON_NEGATIVE_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY), str) + } + } else if math.IsNaN(value) { + if s.(string) != JSON_NAN { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NAN), str) + } + } else { + if s.(float64) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s'", thetype, value, s) + } + } + trans.Reset() + } + trans.Close() +} + +func TestWriteJSONProtocolSet(t *testing.T) { + thetype := "set" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + p.WriteSetBegin(TType(DOUBLE), len(DOUBLE_VALUES)) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteSetEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + str1 := new([]interface{}) + err := json.Unmarshal([]byte(str), str1) + if err != nil { + t.Fatalf("Unable to decode %s, wrote: %s", thetype, str) + } + l := *str1 + if len(l) < 2 { + t.Fatalf("Set must be at least of length two to include metadata") + } + if l[0] != "dbl" { + t.Fatal("Invalid type for set, expected: ", DOUBLE, ", but was: ", l[0]) + } + if int(l[1].(float64)) != len(DOUBLE_VALUES) { + t.Fatal("Invalid length for set, expected: ", len(DOUBLE_VALUES), ", but was: ", l[1]) + } + for k, value := range DOUBLE_VALUES { + s := l[k+2] + if math.IsInf(value, 1) { + if s.(string) != JSON_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_INFINITY), str) + } + } else if math.IsInf(value, 0) { + if s.(string) != JSON_NEGATIVE_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY), str) + } + } else if math.IsNaN(value) { + if s.(string) != JSON_NAN { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NAN), str) + } + } else { + if s.(float64) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s'", thetype, value, s) + } + } + trans.Reset() + } + trans.Close() +} + +func TestWriteJSONProtocolMap(t *testing.T) { + thetype := "map" + trans := NewTMemoryBuffer() + p := NewTJSONProtocol(trans) + p.WriteMapBegin(TType(I32), TType(DOUBLE), len(DOUBLE_VALUES)) + for k, value := range DOUBLE_VALUES { + if e := p.WriteI32(int32(k)); e != nil { + t.Fatalf("Unable to write %s key int32 value %v due to error: %s", thetype, k, e.Error()) + } + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value float64 value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteMapEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + if str[0] != '[' || str[len(str)-1] != ']' { + t.Fatalf("Bad value for %s, wrote: %q, in go: %q", thetype, str, DOUBLE_VALUES) + } + expectedKeyType, expectedValueType, expectedSize, err := p.ReadMapBegin() + if err != nil { + t.Fatalf("Error while reading map begin: %s", err.Error()) + } + if expectedKeyType != I32 { + t.Fatal("Expected map key type ", I32, ", but was ", expectedKeyType) + } + if expectedValueType != DOUBLE { + t.Fatal("Expected map value type ", DOUBLE, ", but was ", expectedValueType) + } + if expectedSize != len(DOUBLE_VALUES) { + t.Fatal("Expected map size of ", len(DOUBLE_VALUES), ", but was ", expectedSize) + } + for k, value := range DOUBLE_VALUES { + ik, err := p.ReadI32() + if err != nil { + t.Fatalf("Bad key for %s index %v, wrote: %v, expected: %v, error: %s", thetype, k, ik, string(k), err.Error()) + } + if int(ik) != k { + t.Fatalf("Bad key for %s index %v, wrote: %v, expected: %v", thetype, k, ik, k) + } + dv, err := p.ReadDouble() + if err != nil { + t.Fatalf("Bad value for %s index %v, wrote: %v, expected: %v, error: %s", thetype, k, dv, value, err.Error()) + } + s := strconv.FormatFloat(dv, 'g', 10, 64) + if math.IsInf(value, 1) { + if !math.IsInf(dv, 1) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_INFINITY)) + } + } else if math.IsInf(value, 0) { + if !math.IsInf(dv, 0) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY)) + } + } else if math.IsNaN(value) { + if !math.IsNaN(dv) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_NAN)) + } + } else { + expected := strconv.FormatFloat(value, 'g', 10, 64) + if s != expected { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected %v", thetype, k, value, s, expected) + } + v := float64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + } + err = p.ReadMapEnd() + if err != nil { + t.Fatalf("Error while reading map end: %s", err.Error()) + } + trans.Close() +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/lowlevel_benchmarks_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/lowlevel_benchmarks_test.go new file mode 100644 index 0000000000000..a5094ae97ce2c --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/lowlevel_benchmarks_test.go @@ -0,0 +1,396 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "bytes" + "testing" +) + +var binaryProtoF = NewTBinaryProtocolFactoryDefault() +var compactProtoF = NewTCompactProtocolFactory() + +var buf = bytes.NewBuffer(make([]byte, 0, 1024)) + +var tfv = []TTransportFactory{ + NewTMemoryBufferTransportFactory(1024), + NewStreamTransportFactory(buf, buf, true), + NewTFramedTransportFactory(NewTMemoryBufferTransportFactory(1024)), +} + +func BenchmarkBinaryBool_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkBinaryByte_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkBinaryI16_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkBinaryI32_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkBinaryI64_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkBinaryDouble_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkBinaryString_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkBinaryBinary_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} + +func BenchmarkBinaryBool_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkBinaryByte_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkBinaryI16_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkBinaryI32_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkBinaryI64_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkBinaryDouble_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkBinaryString_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkBinaryBinary_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} + +func BenchmarkBinaryBool_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkBinaryByte_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkBinaryI16_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkBinaryI32_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkBinaryI64_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkBinaryDouble_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkBinaryString_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkBinaryBinary_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := binaryProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} + +func BenchmarkCompactBool_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkCompactByte_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkCompactI16_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkCompactI32_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkCompactI64_0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkCompactDouble0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkCompactString0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkCompactBinary0(b *testing.B) { + trans := tfv[0].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} + +func BenchmarkCompactBool_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkCompactByte_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkCompactI16_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkCompactI32_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkCompactI64_1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkCompactDouble1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkCompactString1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkCompactBinary1(b *testing.B) { + trans := tfv[1].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} + +func BenchmarkCompactBool_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBool(b, p, trans) + } +} + +func BenchmarkCompactByte_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteByte(b, p, trans) + } +} + +func BenchmarkCompactI16_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI16(b, p, trans) + } +} + +func BenchmarkCompactI32_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI32(b, p, trans) + } +} +func BenchmarkCompactI64_2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteI64(b, p, trans) + } +} +func BenchmarkCompactDouble2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteDouble(b, p, trans) + } +} +func BenchmarkCompactString2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteString(b, p, trans) + } +} +func BenchmarkCompactBinary2(b *testing.B) { + trans := tfv[2].GetTransport(nil) + p := compactProtoF.GetProtocol(trans) + for i := 0; i < b.N; i++ { + ReadWriteBinary(b, p, trans) + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/memory_buffer_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/memory_buffer_test.go new file mode 100644 index 0000000000000..af2e8bfe52224 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/memory_buffer_test.go @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestMemoryBuffer(t *testing.T) { + trans := NewTMemoryBufferLen(1024) + TransportTest(t, trans, trans) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/protocol_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/protocol_test.go new file mode 100644 index 0000000000000..613eae6bc8846 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/protocol_test.go @@ -0,0 +1,479 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "bytes" + "io/ioutil" + "math" + "net" + "net/http" + "testing" +) + +const PROTOCOL_BINARY_DATA_SIZE = 155 + +var ( + data string // test data for writing + protocol_bdata []byte // test data for writing; same as data + BOOL_VALUES []bool + BYTE_VALUES []int8 + INT16_VALUES []int16 + INT32_VALUES []int32 + INT64_VALUES []int64 + DOUBLE_VALUES []float64 + STRING_VALUES []string +) + +func init() { + protocol_bdata = make([]byte, PROTOCOL_BINARY_DATA_SIZE) + for i := 0; i < PROTOCOL_BINARY_DATA_SIZE; i++ { + protocol_bdata[i] = byte((i + 'a') % 255) + } + data = string(protocol_bdata) + BOOL_VALUES = []bool{false, true, false, false, true} + BYTE_VALUES = []int8{117, 0, 1, 32, 127, -128, -1} + INT16_VALUES = []int16{459, 0, 1, -1, -128, 127, 32767, -32768} + INT32_VALUES = []int32{459, 0, 1, -1, -128, 127, 32767, 2147483647, -2147483535} + INT64_VALUES = []int64{459, 0, 1, -1, -128, 127, 32767, 2147483647, -2147483535, 34359738481, -35184372088719, -9223372036854775808, 9223372036854775807} + DOUBLE_VALUES = []float64{459.3, 0.0, -1.0, 1.0, 0.5, 0.3333, 3.14159, 1.537e-38, 1.673e25, 6.02214179e23, -6.02214179e23, INFINITY.Float64(), NEGATIVE_INFINITY.Float64(), NAN.Float64()} + STRING_VALUES = []string{"", "a", "st[uf]f", "st,u:ff with spaces", "stuff\twith\nescape\\characters'...\"lots{of}fun"} +} + +type HTTPEchoServer struct{} +type HTTPHeaderEchoServer struct{} + +func (p *HTTPEchoServer) ServeHTTP(w http.ResponseWriter, req *http.Request) { + buf, err := ioutil.ReadAll(req.Body) + if err != nil { + w.WriteHeader(http.StatusBadRequest) + w.Write(buf) + } else { + w.WriteHeader(http.StatusOK) + w.Write(buf) + } +} + +func (p *HTTPHeaderEchoServer) ServeHTTP(w http.ResponseWriter, req *http.Request) { + buf, err := ioutil.ReadAll(req.Body) + if err != nil { + w.WriteHeader(http.StatusBadRequest) + w.Write(buf) + } else { + w.WriteHeader(http.StatusOK) + w.Write(buf) + } +} + +func HttpClientSetupForTest(t *testing.T) (net.Listener, net.Addr) { + addr, err := FindAvailableTCPServerPort(40000) + if err != nil { + t.Fatalf("Unable to find available tcp port addr: %s", err) + return nil, addr + } + l, err := net.Listen(addr.Network(), addr.String()) + if err != nil { + t.Fatalf("Unable to setup tcp listener on %s: %s", addr.String(), err) + return l, addr + } + go http.Serve(l, &HTTPEchoServer{}) + return l, addr +} + +func HttpClientSetupForHeaderTest(t *testing.T) (net.Listener, net.Addr) { + addr, err := FindAvailableTCPServerPort(40000) + if err != nil { + t.Fatalf("Unable to find available tcp port addr: %s", err) + return nil, addr + } + l, err := net.Listen(addr.Network(), addr.String()) + if err != nil { + t.Fatalf("Unable to setup tcp listener on %s: %s", addr.String(), err) + return l, addr + } + go http.Serve(l, &HTTPHeaderEchoServer{}) + return l, addr +} + +func ReadWriteProtocolTest(t *testing.T, protocolFactory TProtocolFactory) { + buf := bytes.NewBuffer(make([]byte, 0, 1024)) + l, addr := HttpClientSetupForTest(t) + defer l.Close() + transports := []TTransportFactory{ + NewTMemoryBufferTransportFactory(1024), + NewStreamTransportFactory(buf, buf, true), + NewTFramedTransportFactory(NewTMemoryBufferTransportFactory(1024)), + NewTHttpPostClientTransportFactory("http://" + addr.String()), + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteBool(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteByte(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteI16(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteI32(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteI64(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteDouble(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteString(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteBinary(t, p, trans) + trans.Close() + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + p := protocolFactory.GetProtocol(trans) + ReadWriteI64(t, p, trans) + ReadWriteDouble(t, p, trans) + ReadWriteBinary(t, p, trans) + ReadWriteByte(t, p, trans) + trans.Close() + } +} + +func ReadWriteBool(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(BOOL) + thelen := len(BOOL_VALUES) + err := p.WriteListBegin(thetype, thelen) + if err != nil { + t.Errorf("%s: %T %T %q Error writing list begin: %q", "ReadWriteBool", p, trans, err, thetype) + } + for k, v := range BOOL_VALUES { + err = p.WriteBool(v) + if err != nil { + t.Errorf("%s: %T %T %q Error writing bool in list at index %d: %q", "ReadWriteBool", p, trans, err, k, v) + } + } + p.WriteListEnd() + if err != nil { + t.Errorf("%s: %T %T %q Error writing list end: %q", "ReadWriteBool", p, trans, err, BOOL_VALUES) + } + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteBool", p, trans, err, BOOL_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteBool", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteBool", p, trans, thelen, thelen2) + } + } + for k, v := range BOOL_VALUES { + value, err := p.ReadBool() + if err != nil { + t.Errorf("%s: %T %T %q Error reading bool at index %d: %q", "ReadWriteBool", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: index %d %q %q %q != %q", "ReadWriteBool", k, p, trans, v, value) + } + } + err = p.ReadListEnd() + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteBool", p, trans, err) + } +} + +func ReadWriteByte(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(BYTE) + thelen := len(BYTE_VALUES) + err := p.WriteListBegin(thetype, thelen) + if err != nil { + t.Errorf("%s: %T %T %q Error writing list begin: %q", "ReadWriteByte", p, trans, err, thetype) + } + for k, v := range BYTE_VALUES { + err = p.WriteByte(v) + if err != nil { + t.Errorf("%s: %T %T %q Error writing byte in list at index %d: %q", "ReadWriteByte", p, trans, err, k, v) + } + } + err = p.WriteListEnd() + if err != nil { + t.Errorf("%s: %T %T %q Error writing list end: %q", "ReadWriteByte", p, trans, err, BYTE_VALUES) + } + err = p.Flush() + if err != nil { + t.Errorf("%s: %T %T %q Error flushing list of bytes: %q", "ReadWriteByte", p, trans, err, BYTE_VALUES) + } + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteByte", p, trans, err, BYTE_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteByte", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteByte", p, trans, thelen, thelen2) + } + } + for k, v := range BYTE_VALUES { + value, err := p.ReadByte() + if err != nil { + t.Errorf("%s: %T %T %q Error reading byte at index %d: %q", "ReadWriteByte", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: %T %T %d != %d", "ReadWriteByte", p, trans, v, value) + } + } + err = p.ReadListEnd() + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteByte", p, trans, err) + } +} + +func ReadWriteI16(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(I16) + thelen := len(INT16_VALUES) + p.WriteListBegin(thetype, thelen) + for _, v := range INT16_VALUES { + p.WriteI16(v) + } + p.WriteListEnd() + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteI16", p, trans, err, INT16_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteI16", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteI16", p, trans, thelen, thelen2) + } + } + for k, v := range INT16_VALUES { + value, err := p.ReadI16() + if err != nil { + t.Errorf("%s: %T %T %q Error reading int16 at index %d: %q", "ReadWriteI16", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: %T %T %d != %d", "ReadWriteI16", p, trans, v, value) + } + } + err = p.ReadListEnd() + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteI16", p, trans, err) + } +} + +func ReadWriteI32(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(I32) + thelen := len(INT32_VALUES) + p.WriteListBegin(thetype, thelen) + for _, v := range INT32_VALUES { + p.WriteI32(v) + } + p.WriteListEnd() + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteI32", p, trans, err, INT32_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteI32", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteI32", p, trans, thelen, thelen2) + } + } + for k, v := range INT32_VALUES { + value, err := p.ReadI32() + if err != nil { + t.Errorf("%s: %T %T %q Error reading int32 at index %d: %q", "ReadWriteI32", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: %T %T %d != %d", "ReadWriteI32", p, trans, v, value) + } + } + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteI32", p, trans, err) + } +} + +func ReadWriteI64(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(I64) + thelen := len(INT64_VALUES) + p.WriteListBegin(thetype, thelen) + for _, v := range INT64_VALUES { + p.WriteI64(v) + } + p.WriteListEnd() + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteI64", p, trans, err, INT64_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteI64", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteI64", p, trans, thelen, thelen2) + } + } + for k, v := range INT64_VALUES { + value, err := p.ReadI64() + if err != nil { + t.Errorf("%s: %T %T %q Error reading int64 at index %d: %q", "ReadWriteI64", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: %T %T %q != %q", "ReadWriteI64", p, trans, v, value) + } + } + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteI64", p, trans, err) + } +} + +func ReadWriteDouble(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(DOUBLE) + thelen := len(DOUBLE_VALUES) + p.WriteListBegin(thetype, thelen) + for _, v := range DOUBLE_VALUES { + p.WriteDouble(v) + } + p.WriteListEnd() + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteDouble", p, trans, err, DOUBLE_VALUES) + } + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteDouble", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteDouble", p, trans, thelen, thelen2) + } + for k, v := range DOUBLE_VALUES { + value, err := p.ReadDouble() + if err != nil { + t.Errorf("%s: %T %T %q Error reading double at index %d: %q", "ReadWriteDouble", p, trans, err, k, v) + } + if math.IsNaN(v) { + if !math.IsNaN(value) { + t.Errorf("%s: %T %T math.IsNaN(%q) != math.IsNaN(%q)", "ReadWriteDouble", p, trans, v, value) + } + } else if v != value { + t.Errorf("%s: %T %T %v != %q", "ReadWriteDouble", p, trans, v, value) + } + } + err = p.ReadListEnd() + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteDouble", p, trans, err) + } +} + +func ReadWriteString(t testing.TB, p TProtocol, trans TTransport) { + thetype := TType(STRING) + thelen := len(STRING_VALUES) + p.WriteListBegin(thetype, thelen) + for _, v := range STRING_VALUES { + p.WriteString(v) + } + p.WriteListEnd() + p.Flush() + thetype2, thelen2, err := p.ReadListBegin() + if err != nil { + t.Errorf("%s: %T %T %q Error reading list: %q", "ReadWriteString", p, trans, err, STRING_VALUES) + } + _, ok := p.(*TSimpleJSONProtocol) + if !ok { + if thetype != thetype2 { + t.Errorf("%s: %T %T type %s != type %s", "ReadWriteString", p, trans, thetype, thetype2) + } + if thelen != thelen2 { + t.Errorf("%s: %T %T len %s != len %s", "ReadWriteString", p, trans, thelen, thelen2) + } + } + for k, v := range STRING_VALUES { + value, err := p.ReadString() + if err != nil { + t.Errorf("%s: %T %T %q Error reading string at index %d: %q", "ReadWriteString", p, trans, err, k, v) + } + if v != value { + t.Errorf("%s: %T %T %d != %d", "ReadWriteString", p, trans, v, value) + } + } + if err != nil { + t.Errorf("%s: %T %T Unable to read list end: %q", "ReadWriteString", p, trans, err) + } +} + +func ReadWriteBinary(t testing.TB, p TProtocol, trans TTransport) { + v := protocol_bdata + p.WriteBinary(v) + p.Flush() + value, err := p.ReadBinary() + if err != nil { + t.Errorf("%s: %T %T Unable to read binary: %s", "ReadWriteBinary", p, trans, err.Error()) + } + if len(v) != len(value) { + t.Errorf("%s: %T %T len(v) != len(value)... %d != %d", "ReadWriteBinary", p, trans, len(v), len(value)) + } else { + for i := 0; i < len(v); i++ { + if v[i] != value[i] { + t.Errorf("%s: %T %T %s != %s", "ReadWriteBinary", p, trans, v, value) + } + } + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/rich_transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/rich_transport_test.go new file mode 100644 index 0000000000000..41513f812b7f6 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/rich_transport_test.go @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "bytes" + "errors" + "io" + "reflect" + "testing" +) + +func TestEnsureTransportsAreRich(t *testing.T) { + buf := bytes.NewBuffer(make([]byte, 0, 1024)) + + transports := []TTransportFactory{ + NewTMemoryBufferTransportFactory(1024), + NewStreamTransportFactory(buf, buf, true), + NewTFramedTransportFactory(NewTMemoryBufferTransportFactory(1024)), + NewTHttpPostClientTransportFactory("http://127.0.0.1"), + } + for _, tf := range transports { + trans := tf.GetTransport(nil) + _, ok := trans.(TRichTransport) + if !ok { + t.Errorf("Transport %s does not implement TRichTransport interface", reflect.ValueOf(trans)) + } + } +} + +// TestReadByte tests whether readByte handles error cases correctly. +func TestReadByte(t *testing.T) { + for i, test := range readByteTests { + v, err := readByte(test.r) + if v != test.v { + t.Fatalf("TestReadByte %d: value differs. Expected %d, got %d", i, test.v, test.r.v) + } + if err != test.err { + t.Fatalf("TestReadByte %d: error differs. Expected %s, got %s", i, test.err, test.r.err) + } + } +} + +var someError = errors.New("Some error") +var readByteTests = []struct { + r *mockReader + v byte + err error +}{ + {&mockReader{0, 55, io.EOF}, 0, io.EOF}, // reader sends EOF w/o data + {&mockReader{0, 55, someError}, 0, someError}, // reader sends some other error + {&mockReader{1, 55, nil}, 55, nil}, // reader sends data w/o error + {&mockReader{1, 55, io.EOF}, 55, nil}, // reader sends data with EOF + {&mockReader{1, 55, someError}, 55, someError}, // reader sends data withsome error +} + +type mockReader struct { + n int + v byte + err error +} + +func (r *mockReader) Read(p []byte) (n int, err error) { + if r.n > 0 { + p[0] = r.v + } + return r.n, r.err +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/serializer_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/serializer_test.go new file mode 100644 index 0000000000000..06d27a16b7bb6 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/serializer_test.go @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "errors" + "fmt" + "testing" +) + +type ProtocolFactory interface { + GetProtocol(t TTransport) TProtocol +} + +func compareStructs(m, m1 MyTestStruct) (bool, error) { + switch { + case m.On != m1.On: + return false, errors.New("Boolean not equal") + case m.B != m1.B: + return false, errors.New("Byte not equal") + case m.Int16 != m1.Int16: + return false, errors.New("Int16 not equal") + case m.Int32 != m1.Int32: + return false, errors.New("Int32 not equal") + case m.Int64 != m1.Int64: + return false, errors.New("Int64 not equal") + case m.D != m1.D: + return false, errors.New("Double not equal") + case m.St != m1.St: + return false, errors.New("String not equal") + + case len(m.Bin) != len(m1.Bin): + return false, errors.New("Binary size not equal") + case len(m.Bin) == len(m1.Bin): + for i := range m.Bin { + if m.Bin[i] != m1.Bin[i] { + return false, errors.New("Binary not equal") + } + } + case len(m.StringMap) != len(m1.StringMap): + return false, errors.New("StringMap size not equal") + case len(m.StringList) != len(m1.StringList): + return false, errors.New("StringList size not equal") + case len(m.StringSet) != len(m1.StringSet): + return false, errors.New("StringSet size not equal") + + case m.E != m1.E: + return false, errors.New("MyTestEnum not equal") + + default: + return true, nil + + } + return true, nil +} + +func ProtocolTest1(test *testing.T, pf ProtocolFactory) (bool, error) { + t := NewTSerializer() + t.Protocol = pf.GetProtocol(t.Transport) + var m = MyTestStruct{} + m.On = true + m.B = int8(0) + m.Int16 = 1 + m.Int32 = 2 + m.Int64 = 3 + m.D = 4.1 + m.St = "Test" + m.Bin = make([]byte, 10) + m.StringMap = make(map[string]string, 5) + m.StringList = make([]string, 5) + m.StringSet = make(map[string]struct{}, 5) + m.E = 2 + + s, err := t.WriteString(&m) + if err != nil { + return false, errors.New(fmt.Sprintf("Unable to Serialize struct\n\t %s", err)) + } + + t1 := NewTDeserializer() + t1.Protocol = pf.GetProtocol(t1.Transport) + var m1 = MyTestStruct{} + if err = t1.ReadString(&m1, s); err != nil { + return false, errors.New(fmt.Sprintf("Unable to Deserialize struct\n\t %s", err)) + + } + + return compareStructs(m, m1) + +} + +func ProtocolTest2(test *testing.T, pf ProtocolFactory) (bool, error) { + t := NewTSerializer() + t.Protocol = pf.GetProtocol(t.Transport) + var m = MyTestStruct{} + m.On = false + m.B = int8(0) + m.Int16 = 1 + m.Int32 = 2 + m.Int64 = 3 + m.D = 4.1 + m.St = "Test" + m.Bin = make([]byte, 10) + m.StringMap = make(map[string]string, 5) + m.StringList = make([]string, 5) + m.StringSet = make(map[string]struct{}, 5) + m.E = 2 + + s, err := t.WriteString(&m) + if err != nil { + return false, errors.New(fmt.Sprintf("Unable to Serialize struct\n\t %s", err)) + + } + + t1 := NewTDeserializer() + t1.Protocol = pf.GetProtocol(t1.Transport) + var m1 = MyTestStruct{} + if err = t1.ReadString(&m1, s); err != nil { + return false, errors.New(fmt.Sprintf("Unable to Deserialize struct\n\t %s", err)) + + } + + return compareStructs(m, m1) + +} + +func TestSerializer(t *testing.T) { + + var protocol_factories map[string]ProtocolFactory + protocol_factories = make(map[string]ProtocolFactory) + protocol_factories["Binary"] = NewTBinaryProtocolFactoryDefault() + protocol_factories["Compact"] = NewTCompactProtocolFactory() + //protocol_factories["SimpleJSON"] = NewTSimpleJSONProtocolFactory() - write only, can't be read back by design + protocol_factories["JSON"] = NewTJSONProtocolFactory() + + var tests map[string]func(*testing.T, ProtocolFactory) (bool, error) + tests = make(map[string]func(*testing.T, ProtocolFactory) (bool, error)) + tests["Test 1"] = ProtocolTest1 + tests["Test 2"] = ProtocolTest2 + //tests["Test 3"] = ProtocolTest3 // Example of how to add additional tests + + for name, pf := range protocol_factories { + + for test, f := range tests { + + if s, err := f(t, pf); !s || err != nil { + t.Errorf("%s Failed for %s protocol\n\t %s", test, name, err) + } + + } + } + +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/serializer_types_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/serializer_types_test.go new file mode 100644 index 0000000000000..38ab8d6d60052 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/serializer_types_test.go @@ -0,0 +1,633 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +// Autogenerated by Thrift Compiler (1.0.0-dev) +// DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + +/* THE FOLLOWING THRIFT FILE WAS USED TO CREATE THIS + +enum MyTestEnum { + FIRST = 1, + SECOND = 2, + THIRD = 3, + FOURTH = 4, +} + +struct MyTestStruct { + 1: bool on, + 2: byte b, + 3: i16 int16, + 4: i32 int32, + 5: i64 int64, + 6: double d, + 7: string st, + 8: binary bin, + 9: map stringMap, + 10: list stringList, + 11: set stringSet, + 12: MyTestEnum e, +} +*/ + +import ( + "fmt" +) + +// (needed to ensure safety because of naive import list construction.) +var _ = ZERO +var _ = fmt.Printf + +var GoUnusedProtection__ int + +type MyTestEnum int64 + +const ( + MyTestEnum_FIRST MyTestEnum = 1 + MyTestEnum_SECOND MyTestEnum = 2 + MyTestEnum_THIRD MyTestEnum = 3 + MyTestEnum_FOURTH MyTestEnum = 4 +) + +func (p MyTestEnum) String() string { + switch p { + case MyTestEnum_FIRST: + return "FIRST" + case MyTestEnum_SECOND: + return "SECOND" + case MyTestEnum_THIRD: + return "THIRD" + case MyTestEnum_FOURTH: + return "FOURTH" + } + return "" +} + +func MyTestEnumFromString(s string) (MyTestEnum, error) { + switch s { + case "FIRST": + return MyTestEnum_FIRST, nil + case "SECOND": + return MyTestEnum_SECOND, nil + case "THIRD": + return MyTestEnum_THIRD, nil + case "FOURTH": + return MyTestEnum_FOURTH, nil + } + return MyTestEnum(0), fmt.Errorf("not a valid MyTestEnum string") +} + +func MyTestEnumPtr(v MyTestEnum) *MyTestEnum { return &v } + +type MyTestStruct struct { + On bool `thrift:"on,1" json:"on"` + B int8 `thrift:"b,2" json:"b"` + Int16 int16 `thrift:"int16,3" json:"int16"` + Int32 int32 `thrift:"int32,4" json:"int32"` + Int64 int64 `thrift:"int64,5" json:"int64"` + D float64 `thrift:"d,6" json:"d"` + St string `thrift:"st,7" json:"st"` + Bin []byte `thrift:"bin,8" json:"bin"` + StringMap map[string]string `thrift:"stringMap,9" json:"stringMap"` + StringList []string `thrift:"stringList,10" json:"stringList"` + StringSet map[string]struct{} `thrift:"stringSet,11" json:"stringSet"` + E MyTestEnum `thrift:"e,12" json:"e"` +} + +func NewMyTestStruct() *MyTestStruct { + return &MyTestStruct{} +} + +func (p *MyTestStruct) GetOn() bool { + return p.On +} + +func (p *MyTestStruct) GetB() int8 { + return p.B +} + +func (p *MyTestStruct) GetInt16() int16 { + return p.Int16 +} + +func (p *MyTestStruct) GetInt32() int32 { + return p.Int32 +} + +func (p *MyTestStruct) GetInt64() int64 { + return p.Int64 +} + +func (p *MyTestStruct) GetD() float64 { + return p.D +} + +func (p *MyTestStruct) GetSt() string { + return p.St +} + +func (p *MyTestStruct) GetBin() []byte { + return p.Bin +} + +func (p *MyTestStruct) GetStringMap() map[string]string { + return p.StringMap +} + +func (p *MyTestStruct) GetStringList() []string { + return p.StringList +} + +func (p *MyTestStruct) GetStringSet() map[string]struct{} { + return p.StringSet +} + +func (p *MyTestStruct) GetE() MyTestEnum { + return p.E +} +func (p *MyTestStruct) Read(iprot TProtocol) error { + if _, err := iprot.ReadStructBegin(); err != nil { + return PrependError(fmt.Sprintf("%T read error: ", p), err) + } + for { + _, fieldTypeId, fieldId, err := iprot.ReadFieldBegin() + if err != nil { + return PrependError(fmt.Sprintf("%T field %d read error: ", p, fieldId), err) + } + if fieldTypeId == STOP { + break + } + switch fieldId { + case 1: + if err := p.readField1(iprot); err != nil { + return err + } + case 2: + if err := p.readField2(iprot); err != nil { + return err + } + case 3: + if err := p.readField3(iprot); err != nil { + return err + } + case 4: + if err := p.readField4(iprot); err != nil { + return err + } + case 5: + if err := p.readField5(iprot); err != nil { + return err + } + case 6: + if err := p.readField6(iprot); err != nil { + return err + } + case 7: + if err := p.readField7(iprot); err != nil { + return err + } + case 8: + if err := p.readField8(iprot); err != nil { + return err + } + case 9: + if err := p.readField9(iprot); err != nil { + return err + } + case 10: + if err := p.readField10(iprot); err != nil { + return err + } + case 11: + if err := p.readField11(iprot); err != nil { + return err + } + case 12: + if err := p.readField12(iprot); err != nil { + return err + } + default: + if err := iprot.Skip(fieldTypeId); err != nil { + return err + } + } + if err := iprot.ReadFieldEnd(); err != nil { + return err + } + } + if err := iprot.ReadStructEnd(); err != nil { + return PrependError(fmt.Sprintf("%T read struct end error: ", p), err) + } + return nil +} + +func (p *MyTestStruct) readField1(iprot TProtocol) error { + if v, err := iprot.ReadBool(); err != nil { + return PrependError("error reading field 1: ", err) + } else { + p.On = v + } + return nil +} + +func (p *MyTestStruct) readField2(iprot TProtocol) error { + if v, err := iprot.ReadByte(); err != nil { + return PrependError("error reading field 2: ", err) + } else { + temp := int8(v) + p.B = temp + } + return nil +} + +func (p *MyTestStruct) readField3(iprot TProtocol) error { + if v, err := iprot.ReadI16(); err != nil { + return PrependError("error reading field 3: ", err) + } else { + p.Int16 = v + } + return nil +} + +func (p *MyTestStruct) readField4(iprot TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { + return PrependError("error reading field 4: ", err) + } else { + p.Int32 = v + } + return nil +} + +func (p *MyTestStruct) readField5(iprot TProtocol) error { + if v, err := iprot.ReadI64(); err != nil { + return PrependError("error reading field 5: ", err) + } else { + p.Int64 = v + } + return nil +} + +func (p *MyTestStruct) readField6(iprot TProtocol) error { + if v, err := iprot.ReadDouble(); err != nil { + return PrependError("error reading field 6: ", err) + } else { + p.D = v + } + return nil +} + +func (p *MyTestStruct) readField7(iprot TProtocol) error { + if v, err := iprot.ReadString(); err != nil { + return PrependError("error reading field 7: ", err) + } else { + p.St = v + } + return nil +} + +func (p *MyTestStruct) readField8(iprot TProtocol) error { + if v, err := iprot.ReadBinary(); err != nil { + return PrependError("error reading field 8: ", err) + } else { + p.Bin = v + } + return nil +} + +func (p *MyTestStruct) readField9(iprot TProtocol) error { + _, _, size, err := iprot.ReadMapBegin() + if err != nil { + return PrependError("error reading map begin: ", err) + } + tMap := make(map[string]string, size) + p.StringMap = tMap + for i := 0; i < size; i++ { + var _key0 string + if v, err := iprot.ReadString(); err != nil { + return PrependError("error reading field 0: ", err) + } else { + _key0 = v + } + var _val1 string + if v, err := iprot.ReadString(); err != nil { + return PrependError("error reading field 0: ", err) + } else { + _val1 = v + } + p.StringMap[_key0] = _val1 + } + if err := iprot.ReadMapEnd(); err != nil { + return PrependError("error reading map end: ", err) + } + return nil +} + +func (p *MyTestStruct) readField10(iprot TProtocol) error { + _, size, err := iprot.ReadListBegin() + if err != nil { + return PrependError("error reading list begin: ", err) + } + tSlice := make([]string, 0, size) + p.StringList = tSlice + for i := 0; i < size; i++ { + var _elem2 string + if v, err := iprot.ReadString(); err != nil { + return PrependError("error reading field 0: ", err) + } else { + _elem2 = v + } + p.StringList = append(p.StringList, _elem2) + } + if err := iprot.ReadListEnd(); err != nil { + return PrependError("error reading list end: ", err) + } + return nil +} + +func (p *MyTestStruct) readField11(iprot TProtocol) error { + _, size, err := iprot.ReadSetBegin() + if err != nil { + return PrependError("error reading set begin: ", err) + } + tSet := make(map[string]struct{}, size) + p.StringSet = tSet + for i := 0; i < size; i++ { + var _elem3 string + if v, err := iprot.ReadString(); err != nil { + return PrependError("error reading field 0: ", err) + } else { + _elem3 = v + } + p.StringSet[_elem3] = struct{}{} + } + if err := iprot.ReadSetEnd(); err != nil { + return PrependError("error reading set end: ", err) + } + return nil +} + +func (p *MyTestStruct) readField12(iprot TProtocol) error { + if v, err := iprot.ReadI32(); err != nil { + return PrependError("error reading field 12: ", err) + } else { + temp := MyTestEnum(v) + p.E = temp + } + return nil +} + +func (p *MyTestStruct) Write(oprot TProtocol) error { + if err := oprot.WriteStructBegin("MyTestStruct"); err != nil { + return PrependError(fmt.Sprintf("%T write struct begin error: ", p), err) + } + if err := p.writeField1(oprot); err != nil { + return err + } + if err := p.writeField2(oprot); err != nil { + return err + } + if err := p.writeField3(oprot); err != nil { + return err + } + if err := p.writeField4(oprot); err != nil { + return err + } + if err := p.writeField5(oprot); err != nil { + return err + } + if err := p.writeField6(oprot); err != nil { + return err + } + if err := p.writeField7(oprot); err != nil { + return err + } + if err := p.writeField8(oprot); err != nil { + return err + } + if err := p.writeField9(oprot); err != nil { + return err + } + if err := p.writeField10(oprot); err != nil { + return err + } + if err := p.writeField11(oprot); err != nil { + return err + } + if err := p.writeField12(oprot); err != nil { + return err + } + if err := oprot.WriteFieldStop(); err != nil { + return PrependError("write field stop error: ", err) + } + if err := oprot.WriteStructEnd(); err != nil { + return PrependError("write struct stop error: ", err) + } + return nil +} + +func (p *MyTestStruct) writeField1(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("on", BOOL, 1); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 1:on: ", p), err) + } + if err := oprot.WriteBool(bool(p.On)); err != nil { + return PrependError(fmt.Sprintf("%T.on (1) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 1:on: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField2(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("b", BYTE, 2); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 2:b: ", p), err) + } + if err := oprot.WriteByte(int8(p.B)); err != nil { + return PrependError(fmt.Sprintf("%T.b (2) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 2:b: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField3(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("int16", I16, 3); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 3:int16: ", p), err) + } + if err := oprot.WriteI16(int16(p.Int16)); err != nil { + return PrependError(fmt.Sprintf("%T.int16 (3) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 3:int16: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField4(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("int32", I32, 4); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 4:int32: ", p), err) + } + if err := oprot.WriteI32(int32(p.Int32)); err != nil { + return PrependError(fmt.Sprintf("%T.int32 (4) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 4:int32: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField5(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("int64", I64, 5); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 5:int64: ", p), err) + } + if err := oprot.WriteI64(int64(p.Int64)); err != nil { + return PrependError(fmt.Sprintf("%T.int64 (5) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 5:int64: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField6(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("d", DOUBLE, 6); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 6:d: ", p), err) + } + if err := oprot.WriteDouble(float64(p.D)); err != nil { + return PrependError(fmt.Sprintf("%T.d (6) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 6:d: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField7(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("st", STRING, 7); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 7:st: ", p), err) + } + if err := oprot.WriteString(string(p.St)); err != nil { + return PrependError(fmt.Sprintf("%T.st (7) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 7:st: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField8(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("bin", STRING, 8); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 8:bin: ", p), err) + } + if err := oprot.WriteBinary(p.Bin); err != nil { + return PrependError(fmt.Sprintf("%T.bin (8) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 8:bin: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField9(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("stringMap", MAP, 9); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 9:stringMap: ", p), err) + } + if err := oprot.WriteMapBegin(STRING, STRING, len(p.StringMap)); err != nil { + return PrependError("error writing map begin: ", err) + } + for k, v := range p.StringMap { + if err := oprot.WriteString(string(k)); err != nil { + return PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + if err := oprot.WriteString(string(v)); err != nil { + return PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteMapEnd(); err != nil { + return PrependError("error writing map end: ", err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 9:stringMap: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField10(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("stringList", LIST, 10); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 10:stringList: ", p), err) + } + if err := oprot.WriteListBegin(STRING, len(p.StringList)); err != nil { + return PrependError("error writing list begin: ", err) + } + for _, v := range p.StringList { + if err := oprot.WriteString(string(v)); err != nil { + return PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteListEnd(); err != nil { + return PrependError("error writing list end: ", err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 10:stringList: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField11(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("stringSet", SET, 11); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 11:stringSet: ", p), err) + } + if err := oprot.WriteSetBegin(STRING, len(p.StringSet)); err != nil { + return PrependError("error writing set begin: ", err) + } + for v, _ := range p.StringSet { + if err := oprot.WriteString(string(v)); err != nil { + return PrependError(fmt.Sprintf("%T. (0) field write error: ", p), err) + } + } + if err := oprot.WriteSetEnd(); err != nil { + return PrependError("error writing set end: ", err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 11:stringSet: ", p), err) + } + return err +} + +func (p *MyTestStruct) writeField12(oprot TProtocol) (err error) { + if err := oprot.WriteFieldBegin("e", I32, 12); err != nil { + return PrependError(fmt.Sprintf("%T write field begin error 12:e: ", p), err) + } + if err := oprot.WriteI32(int32(p.E)); err != nil { + return PrependError(fmt.Sprintf("%T.e (12) field write error: ", p), err) + } + if err := oprot.WriteFieldEnd(); err != nil { + return PrependError(fmt.Sprintf("%T write field end error 12:e: ", p), err) + } + return err +} + +func (p *MyTestStruct) String() string { + if p == nil { + return "" + } + return fmt.Sprintf("MyTestStruct(%+v)", *p) +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/server_socket_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/server_socket_test.go new file mode 100644 index 0000000000000..f08e8e900de76 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/server_socket_test.go @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "fmt" + "testing" +) + +func TestSocketIsntListeningAfterInterrupt(t *testing.T) { + host := "127.0.0.1" + port := 9090 + addr := fmt.Sprintf("%s:%d", host, port) + + socket := CreateServerSocket(t, addr) + socket.Listen() + socket.Interrupt() + + newSocket := CreateServerSocket(t, addr) + err := newSocket.Listen() + defer newSocket.Interrupt() + if err != nil { + t.Fatalf("Failed to rebinds: %s", err) + } +} + +func CreateServerSocket(t *testing.T, addr string) *TServerSocket { + socket, err := NewTServerSocket(addr) + if err != nil { + t.Fatalf("Failed to create server socket: %s", err) + } + return socket +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/server_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/server_test.go new file mode 100644 index 0000000000000..ffaf457027d31 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/server_test.go @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "testing" +) + +func TestNothing(t *testing.T) { + +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/simple_json_protocol_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/simple_json_protocol_test.go new file mode 100644 index 0000000000000..8f0dcc9dfe26d --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/simple_json_protocol_test.go @@ -0,0 +1,715 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "encoding/base64" + "encoding/json" + "fmt" + "math" + "strconv" + "strings" + "testing" +) + +func TestWriteSimpleJSONProtocolBool(t *testing.T) { + thetype := "boolean" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range BOOL_VALUES { + if e := p.WriteBool(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := false + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolBool(t *testing.T) { + thetype := "boolean" + for _, value := range BOOL_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + if value { + trans.Write(JSON_TRUE) + } else { + trans.Write(JSON_FALSE) + } + trans.Flush() + s := trans.String() + v, e := p.ReadBool() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteSimpleJSONProtocolByte(t *testing.T) { + thetype := "byte" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range BYTE_VALUES { + if e := p.WriteByte(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int8(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolByte(t *testing.T) { + thetype := "byte" + for _, value := range BYTE_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadByte() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteSimpleJSONProtocolI16(t *testing.T) { + thetype := "int16" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range INT16_VALUES { + if e := p.WriteI16(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int16(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolI16(t *testing.T) { + thetype := "int16" + for _, value := range INT16_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadI16() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestWriteSimpleJSONProtocolI32(t *testing.T) { + thetype := "int32" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range INT32_VALUES { + if e := p.WriteI32(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int32(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolI32(t *testing.T) { + thetype := "int32" + for _, value := range INT32_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(strconv.Itoa(int(value))) + trans.Flush() + s := trans.String() + v, e := p.ReadI32() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestReadSimpleJSONProtocolI32Null(t *testing.T) { + thetype := "int32" + value := "null" + + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(value) + trans.Flush() + s := trans.String() + v, e := p.ReadI32() + + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != 0 { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + trans.Reset() + trans.Close() +} + +func TestWriteSimpleJSONProtocolI64(t *testing.T) { + thetype := "int64" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range INT64_VALUES { + if e := p.WriteI64(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := int64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolI64(t *testing.T) { + thetype := "int64" + for _, value := range INT64_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(strconv.FormatInt(value, 10)) + trans.Flush() + s := trans.String() + v, e := p.ReadI64() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + trans.Reset() + trans.Close() + } +} + +func TestReadSimpleJSONProtocolI64Null(t *testing.T) { + thetype := "int32" + value := "null" + + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(value) + trans.Flush() + s := trans.String() + v, e := p.ReadI64() + + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != 0 { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + trans.Reset() + trans.Close() +} + +func TestWriteSimpleJSONProtocolDouble(t *testing.T) { + thetype := "double" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if math.IsInf(value, 1) { + if s != jsonQuote(JSON_INFINITY) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_INFINITY)) + } + } else if math.IsInf(value, -1) { + if s != jsonQuote(JSON_NEGATIVE_INFINITY) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_NEGATIVE_INFINITY)) + } + } else if math.IsNaN(value) { + if s != jsonQuote(JSON_NAN) { + t.Fatalf("Bad value for %s %v, wrote: %v, expected: %v", thetype, value, s, jsonQuote(JSON_NAN)) + } + } else { + if s != fmt.Sprint(value) { + t.Fatalf("Bad value for %s %v: %s", thetype, value, s) + } + v := float64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolDouble(t *testing.T) { + thetype := "double" + for _, value := range DOUBLE_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + n := NewNumericFromDouble(value) + trans.WriteString(n.String()) + trans.Flush() + s := trans.String() + v, e := p.ReadDouble() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if math.IsInf(value, 1) { + if !math.IsInf(v, 1) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else if math.IsInf(value, -1) { + if !math.IsInf(v, -1) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else if math.IsNaN(value) { + if !math.IsNaN(v) { + t.Fatalf("Bad value for %s %v, wrote: %v, received: %v", thetype, value, s, v) + } + } else { + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + trans.Reset() + trans.Close() + } +} + +func TestWriteSimpleJSONProtocolString(t *testing.T) { + thetype := "string" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + for _, value := range STRING_VALUES { + if e := p.WriteString(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s[0] != '"' || s[len(s)-1] != '"' { + t.Fatalf("Bad value for %s '%v', wrote '%v', expected: %v", thetype, value, s, fmt.Sprint("\"", value, "\"")) + } + v := new(string) + if err := json.Unmarshal([]byte(s), v); err != nil || *v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v) + } + trans.Reset() + } + trans.Close() +} + +func TestReadSimpleJSONProtocolString(t *testing.T) { + thetype := "string" + for _, value := range STRING_VALUES { + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(jsonQuote(value)) + trans.Flush() + s := trans.String() + v, e := p.ReadString() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != value { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + v1 := new(string) + if err := json.Unmarshal([]byte(s), v1); err != nil || *v1 != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v1) + } + trans.Reset() + trans.Close() + } +} +func TestReadSimpleJSONProtocolStringNull(t *testing.T) { + thetype := "string" + value := "null" + + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(value) + trans.Flush() + s := trans.String() + v, e := p.ReadString() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != "" { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + trans.Reset() + trans.Close() +} + +func TestWriteSimpleJSONProtocolBinary(t *testing.T) { + thetype := "binary" + value := protocol_bdata + b64value := make([]byte, base64.StdEncoding.EncodedLen(len(protocol_bdata))) + base64.StdEncoding.Encode(b64value, value) + b64String := string(b64value) + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + if e := p.WriteBinary(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s value %v due to error flushing: %s", thetype, value, e.Error()) + } + s := trans.String() + if s != fmt.Sprint("\"", b64String, "\"") { + t.Fatalf("Bad value for %s %v\n wrote: %v\nexpected: %v", thetype, value, s, "\""+b64String+"\"") + } + v1 := new(string) + if err := json.Unmarshal([]byte(s), v1); err != nil || *v1 != b64String { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v1) + } + trans.Close() +} + +func TestReadSimpleJSONProtocolBinary(t *testing.T) { + thetype := "binary" + value := protocol_bdata + b64value := make([]byte, base64.StdEncoding.EncodedLen(len(protocol_bdata))) + base64.StdEncoding.Encode(b64value, value) + b64String := string(b64value) + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(jsonQuote(b64String)) + trans.Flush() + s := trans.String() + v, e := p.ReadBinary() + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if len(v) != len(value) { + t.Fatalf("Bad value for %s value length %v, wrote: %v, received length: %v", thetype, len(value), s, len(v)) + } + for i := 0; i < len(v); i++ { + if v[i] != value[i] { + t.Fatalf("Bad value for %s at index %d value %v, wrote: %v, received: %v", thetype, i, value[i], s, v[i]) + } + } + v1 := new(string) + if err := json.Unmarshal([]byte(s), v1); err != nil || *v1 != b64String { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, *v1) + } + trans.Reset() + trans.Close() +} + +func TestReadSimpleJSONProtocolBinaryNull(t *testing.T) { + thetype := "binary" + value := "null" + + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + trans.WriteString(value) + trans.Flush() + s := trans.String() + b, e := p.ReadBinary() + v := string(b) + + if e != nil { + t.Fatalf("Unable to read %s value %v due to error: %s", thetype, value, e.Error()) + } + if v != "" { + t.Fatalf("Bad value for %s value %v, wrote: %v, received: %v", thetype, value, s, v) + } + trans.Reset() + trans.Close() +} + +func TestWriteSimpleJSONProtocolList(t *testing.T) { + thetype := "list" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + p.WriteListBegin(TType(DOUBLE), len(DOUBLE_VALUES)) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteListEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + str1 := new([]interface{}) + err := json.Unmarshal([]byte(str), str1) + if err != nil { + t.Fatalf("Unable to decode %s, wrote: %s", thetype, str) + } + l := *str1 + if len(l) < 2 { + t.Fatalf("List must be at least of length two to include metadata") + } + if int(l[0].(float64)) != DOUBLE { + t.Fatal("Invalid type for list, expected: ", DOUBLE, ", but was: ", l[0]) + } + if int(l[1].(float64)) != len(DOUBLE_VALUES) { + t.Fatal("Invalid length for list, expected: ", len(DOUBLE_VALUES), ", but was: ", l[1]) + } + for k, value := range DOUBLE_VALUES { + s := l[k+2] + if math.IsInf(value, 1) { + if s.(string) != JSON_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_INFINITY), str) + } + } else if math.IsInf(value, 0) { + if s.(string) != JSON_NEGATIVE_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY), str) + } + } else if math.IsNaN(value) { + if s.(string) != JSON_NAN { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NAN), str) + } + } else { + if s.(float64) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s'", thetype, value, s) + } + } + trans.Reset() + } + trans.Close() +} + +func TestWriteSimpleJSONProtocolSet(t *testing.T) { + thetype := "set" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + p.WriteSetBegin(TType(DOUBLE), len(DOUBLE_VALUES)) + for _, value := range DOUBLE_VALUES { + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteSetEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + str1 := new([]interface{}) + err := json.Unmarshal([]byte(str), str1) + if err != nil { + t.Fatalf("Unable to decode %s, wrote: %s", thetype, str) + } + l := *str1 + if len(l) < 2 { + t.Fatalf("Set must be at least of length two to include metadata") + } + if int(l[0].(float64)) != DOUBLE { + t.Fatal("Invalid type for set, expected: ", DOUBLE, ", but was: ", l[0]) + } + if int(l[1].(float64)) != len(DOUBLE_VALUES) { + t.Fatal("Invalid length for set, expected: ", len(DOUBLE_VALUES), ", but was: ", l[1]) + } + for k, value := range DOUBLE_VALUES { + s := l[k+2] + if math.IsInf(value, 1) { + if s.(string) != JSON_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_INFINITY), str) + } + } else if math.IsInf(value, 0) { + if s.(string) != JSON_NEGATIVE_INFINITY { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY), str) + } + } else if math.IsNaN(value) { + if s.(string) != JSON_NAN { + t.Fatalf("Bad value for %s at index %v %v, wrote: %q, expected: %q, originally wrote: %q", thetype, k, value, s, jsonQuote(JSON_NAN), str) + } + } else { + if s.(float64) != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s'", thetype, value, s) + } + } + trans.Reset() + } + trans.Close() +} + +func TestWriteSimpleJSONProtocolMap(t *testing.T) { + thetype := "map" + trans := NewTMemoryBuffer() + p := NewTSimpleJSONProtocol(trans) + p.WriteMapBegin(TType(I32), TType(DOUBLE), len(DOUBLE_VALUES)) + for k, value := range DOUBLE_VALUES { + if e := p.WriteI32(int32(k)); e != nil { + t.Fatalf("Unable to write %s key int32 value %v due to error: %s", thetype, k, e.Error()) + } + if e := p.WriteDouble(value); e != nil { + t.Fatalf("Unable to write %s value float64 value %v due to error: %s", thetype, value, e.Error()) + } + } + p.WriteMapEnd() + if e := p.Flush(); e != nil { + t.Fatalf("Unable to write %s due to error flushing: %s", thetype, e.Error()) + } + str := trans.String() + if str[0] != '[' || str[len(str)-1] != ']' { + t.Fatalf("Bad value for %s, wrote: %q, in go: %q", thetype, str, DOUBLE_VALUES) + } + l := strings.Split(str[1:len(str)-1], ",") + if len(l) < 3 { + t.Fatal("Expected list of at least length 3 for map for metadata, but was of length ", len(l)) + } + expectedKeyType, _ := strconv.Atoi(l[0]) + expectedValueType, _ := strconv.Atoi(l[1]) + expectedSize, _ := strconv.Atoi(l[2]) + if expectedKeyType != I32 { + t.Fatal("Expected map key type ", I32, ", but was ", l[0]) + } + if expectedValueType != DOUBLE { + t.Fatal("Expected map value type ", DOUBLE, ", but was ", l[1]) + } + if expectedSize != len(DOUBLE_VALUES) { + t.Fatal("Expected map size of ", len(DOUBLE_VALUES), ", but was ", l[2]) + } + for k, value := range DOUBLE_VALUES { + strk := l[k*2+3] + strv := l[k*2+4] + ik, err := strconv.Atoi(strk) + if err != nil { + t.Fatalf("Bad value for %s index %v, wrote: %v, expected: %v, error: %s", thetype, k, strk, string(k), err.Error()) + } + if ik != k { + t.Fatalf("Bad value for %s index %v, wrote: %v, expected: %v", thetype, k, strk, k) + } + s := strv + if math.IsInf(value, 1) { + if s != jsonQuote(JSON_INFINITY) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_INFINITY)) + } + } else if math.IsInf(value, 0) { + if s != jsonQuote(JSON_NEGATIVE_INFINITY) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_NEGATIVE_INFINITY)) + } + } else if math.IsNaN(value) { + if s != jsonQuote(JSON_NAN) { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected: %v", thetype, k, value, s, jsonQuote(JSON_NAN)) + } + } else { + expected := strconv.FormatFloat(value, 'g', 10, 64) + if s != expected { + t.Fatalf("Bad value for %s at index %v %v, wrote: %v, expected %v", thetype, k, value, s, expected) + } + v := float64(0) + if err := json.Unmarshal([]byte(s), &v); err != nil || v != value { + t.Fatalf("Bad json-decoded value for %s %v, wrote: '%s', expected: '%v'", thetype, value, s, v) + } + } + trans.Reset() + } + trans.Close() +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/transport_exception_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/transport_exception_test.go new file mode 100644 index 0000000000000..b44314f490d10 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/transport_exception_test.go @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "fmt" + "io" + + "testing" +) + +type timeout struct{ timedout bool } + +func (t *timeout) Timeout() bool { + return t.timedout +} + +func (t *timeout) Error() string { + return fmt.Sprintf("Timeout: %v", t.timedout) +} + +func TestTExceptionTimeout(t *testing.T) { + timeout := &timeout{true} + exception := NewTTransportExceptionFromError(timeout) + if timeout.Error() != exception.Error() { + t.Fatalf("Error did not match: expected %q, got %q", timeout.Error(), exception.Error()) + } + + if exception.TypeId() != TIMED_OUT { + t.Fatalf("TypeId was not TIMED_OUT: expected %v, got %v", TIMED_OUT, exception.TypeId()) + } +} + +func TestTExceptionEOF(t *testing.T) { + exception := NewTTransportExceptionFromError(io.EOF) + if io.EOF.Error() != exception.Error() { + t.Fatalf("Error did not match: expected %q, got %q", io.EOF.Error(), exception.Error()) + } + + if exception.TypeId() != END_OF_FILE { + t.Fatalf("TypeId was not END_OF_FILE: expected %v, got %v", END_OF_FILE, exception.TypeId()) + } +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/transport_test.go new file mode 100644 index 0000000000000..864958a9da016 --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/transport_test.go @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "io" + "net" + "strconv" + "testing" +) + +const TRANSPORT_BINARY_DATA_SIZE = 4096 + +var ( + transport_bdata []byte // test data for writing; same as data + transport_header map[string]string +) + +func init() { + transport_bdata = make([]byte, TRANSPORT_BINARY_DATA_SIZE) + for i := 0; i < TRANSPORT_BINARY_DATA_SIZE; i++ { + transport_bdata[i] = byte((i + 'a') % 255) + } + transport_header = map[string]string{"key": "User-Agent", + "value": "Mozilla/5.0 (Windows NT 6.2; Win64; x64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1667.0 Safari/537.36"} +} + +func TransportTest(t *testing.T, writeTrans TTransport, readTrans TTransport) { + buf := make([]byte, TRANSPORT_BINARY_DATA_SIZE) + if !writeTrans.IsOpen() { + t.Fatalf("Transport %T not open: %s", writeTrans, writeTrans) + } + if !readTrans.IsOpen() { + t.Fatalf("Transport %T not open: %s", readTrans, readTrans) + } + _, err := writeTrans.Write(transport_bdata) + if err != nil { + t.Fatalf("Transport %T cannot write binary data of length %d: %s", writeTrans, len(transport_bdata), err) + } + err = writeTrans.Flush() + if err != nil { + t.Fatalf("Transport %T cannot flush write of binary data: %s", writeTrans, err) + } + n, err := io.ReadFull(readTrans, buf) + if err != nil { + t.Errorf("Transport %T cannot read binary data of length %d: %s", readTrans, TRANSPORT_BINARY_DATA_SIZE, err) + } + if n != TRANSPORT_BINARY_DATA_SIZE { + t.Errorf("Transport %T read only %d instead of %d bytes of binary data", readTrans, n, TRANSPORT_BINARY_DATA_SIZE) + } + for k, v := range buf { + if v != transport_bdata[k] { + t.Fatalf("Transport %T read %d instead of %d for index %d of binary data 2", readTrans, v, transport_bdata[k], k) + } + } + _, err = writeTrans.Write(transport_bdata) + if err != nil { + t.Fatalf("Transport %T cannot write binary data 2 of length %d: %s", writeTrans, len(transport_bdata), err) + } + err = writeTrans.Flush() + if err != nil { + t.Fatalf("Transport %T cannot flush write binary data 2: %s", writeTrans, err) + } + buf = make([]byte, TRANSPORT_BINARY_DATA_SIZE) + read := 1 + for n = 0; n < TRANSPORT_BINARY_DATA_SIZE && read != 0; { + read, err = readTrans.Read(buf[n:]) + if err != nil { + t.Errorf("Transport %T cannot read binary data 2 of total length %d from offset %d: %s", readTrans, TRANSPORT_BINARY_DATA_SIZE, n, err) + } + n += read + } + if n != TRANSPORT_BINARY_DATA_SIZE { + t.Errorf("Transport %T read only %d instead of %d bytes of binary data 2", readTrans, n, TRANSPORT_BINARY_DATA_SIZE) + } + for k, v := range buf { + if v != transport_bdata[k] { + t.Fatalf("Transport %T read %d instead of %d for index %d of binary data 2", readTrans, v, transport_bdata[k], k) + } + } +} + +func TransportHeaderTest(t *testing.T, writeTrans TTransport, readTrans TTransport) { + buf := make([]byte, TRANSPORT_BINARY_DATA_SIZE) + if !writeTrans.IsOpen() { + t.Fatalf("Transport %T not open: %s", writeTrans, writeTrans) + } + if !readTrans.IsOpen() { + t.Fatalf("Transport %T not open: %s", readTrans, readTrans) + } + // Need to assert type of TTransport to THttpClient to expose the Setter + httpWPostTrans := writeTrans.(*THttpClient) + httpWPostTrans.SetHeader(transport_header["key"], transport_header["value"]) + + _, err := writeTrans.Write(transport_bdata) + if err != nil { + t.Fatalf("Transport %T cannot write binary data of length %d: %s", writeTrans, len(transport_bdata), err) + } + err = writeTrans.Flush() + if err != nil { + t.Fatalf("Transport %T cannot flush write of binary data: %s", writeTrans, err) + } + // Need to assert type of TTransport to THttpClient to expose the Getter + httpRPostTrans := readTrans.(*THttpClient) + readHeader := httpRPostTrans.GetHeader(transport_header["key"]) + if err != nil { + t.Errorf("Transport %T cannot read HTTP Header Value", httpRPostTrans) + } + + if transport_header["value"] != readHeader { + t.Errorf("Expected HTTP Header Value %s, got %s", transport_header["value"], readHeader) + } + n, err := io.ReadFull(readTrans, buf) + if err != nil { + t.Errorf("Transport %T cannot read binary data of length %d: %s", readTrans, TRANSPORT_BINARY_DATA_SIZE, err) + } + if n != TRANSPORT_BINARY_DATA_SIZE { + t.Errorf("Transport %T read only %d instead of %d bytes of binary data", readTrans, n, TRANSPORT_BINARY_DATA_SIZE) + } + for k, v := range buf { + if v != transport_bdata[k] { + t.Fatalf("Transport %T read %d instead of %d for index %d of binary data 2", readTrans, v, transport_bdata[k], k) + } + } +} + +func CloseTransports(t *testing.T, readTrans TTransport, writeTrans TTransport) { + err := readTrans.Close() + if err != nil { + t.Errorf("Transport %T cannot close read transport: %s", readTrans, err) + } + if writeTrans != readTrans { + err = writeTrans.Close() + if err != nil { + t.Errorf("Transport %T cannot close write transport: %s", writeTrans, err) + } + } +} + +func FindAvailableTCPServerPort(startPort int) (net.Addr, error) { + for i := startPort; i < 65535; i++ { + s := "127.0.0.1:" + strconv.Itoa(i) + l, err := net.Listen("tcp", s) + if err == nil { + l.Close() + return net.ResolveTCPAddr("tcp", s) + } + } + return nil, NewTTransportException(UNKNOWN_TRANSPORT_EXCEPTION, "Could not find available server port") +} + +func valueInSlice(value string, slice []string) bool { + for _, v := range slice { + if value == v { + return true + } + } + return false +} diff --git a/vendor/github.com/apache/thrift/lib/go/thrift/zlib_transport_test.go b/vendor/github.com/apache/thrift/lib/go/thrift/zlib_transport_test.go new file mode 100644 index 0000000000000..f57610cbd2a0d --- /dev/null +++ b/vendor/github.com/apache/thrift/lib/go/thrift/zlib_transport_test.go @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package thrift + +import ( + "compress/zlib" + "testing" +) + +func TestZlibTransport(t *testing.T) { + trans, err := NewTZlibTransport(NewTMemoryBuffer(), zlib.BestCompression) + if err != nil { + t.Fatal(err) + } + TransportTest(t, trans, trans) +} diff --git a/vendor/github.com/apache/thrift/package.json b/vendor/github.com/apache/thrift/package.json new file mode 100644 index 0000000000000..edfc3553ed5ce --- /dev/null +++ b/vendor/github.com/apache/thrift/package.json @@ -0,0 +1,54 @@ +{ + "name": "thrift", + "description": "node.js bindings for the Apache Thrift RPC system", + "homepage": "http://thrift.apache.org/", + "repository": { + "type": "git", + "url": "https://git-wip-us.apache.org/repos/asf/thrift.git" + }, + "version": "0.10.0", + "author": { + "name": "Apache Thrift Developers", + "email": "dev@thrift.apache.org", + "url": "http://thrift.apache.org" + }, + "license": "Apache-2.0", + "licenses": [ + { + "type": "Apache-2.0", + "url": "http://www.apache.org/licenses/LICENSE-2.0" + } + ], + "bugs": { + "mail": "dev@thrift.apache.org", + "url": "https://issues.apache.org/jira/browse/THRIFT" + }, + "files": [ + "lib/nodejs/lib/thrift", + "lib/nodejs/README.md" + ], + "directories": { + "lib": "./lib/nodejs/lib/thrift" + }, + "main": "./lib/nodejs/lib/thrift", + "engines": { + "node": ">= 0.2.4" + }, + "dependencies": { + "node-int64": "~0.3.0", + "q": "1.0.x", + "ws": "~0.4.32" + }, + "devDependencies": { + "buffer-equals": "^1.0.3", + "commander": "2.1.x", + "connect": "2.7.x", + "istanbul": "^0.3.5", + "run-browser": "^2.0.1", + "tape": "~3.5.0" + }, + "scripts": { + "cover": "lib/nodejs/test/testAll.sh COVER", + "test": "lib/nodejs/test/testAll.sh" + } +} diff --git a/vendor/github.com/apache/thrift/sonar-project.properties b/vendor/github.com/apache/thrift/sonar-project.properties new file mode 100644 index 0000000000000..6e6c5db90d1c4 --- /dev/null +++ b/vendor/github.com/apache/thrift/sonar-project.properties @@ -0,0 +1,140 @@ +# Apache Thrift © The Apache Software Foundation +# http://www.apache.org/licenses/LICENSE-2.0 +# SPDX-License-Identifier: Apache-2.0 + +# File: sonar-project.properties +# Apache Thrift configuration file for Sonar https://analysis.apache.org/ +# Sonar is an open platform to manage code quality http://www.sonarsource.org/ + + +# required metadata +sonar.projectKey=org.apache.thrift +sonar.projectName=Apache Thrift +sonar.projectDescription= +The Apache Thrift software framework, for scalable cross-language services +development, combines a software stack with a code generation engine to build +services that work efficiently and seamlessly between all major languages. + +# Apache Thrift Version +sonar.projectVersion=0.10.0 +# use this to set another version string +# $ sonar-runner -D sonar.projectVersion=`git rev-parse HEAD` +# set projectDate in combination with projectVersion for imports of old releases +#sonar.projectDate=yyyy-MM-dd + +# TODO add website (sonar.projectUrl does not work) +#sonar.XXXX=http//thrift.apache.org + +# Some properties that will be inherited by the modules +sonar.sources=src +sonar.language=java,js,c++,py,c +sonar.sourceEncoding=UTF-8 + +# scm +sonar.scm.url=scm:git:https://git-wip-us.apache.org/repos/asf/thrift + +# cppcheck -q --error-exitcode=0 --xml . 2> cppcheck-result.xml +sonar.cxx.cppcheck.reportPath=cppcheck-result.xml + +# List of the module identifiers +sonar.modules=module1,module3,module4,module5,module6,module7,module8,module9,module10,module11,module12,module14 + + + +# we need sonar-runner 2.1 for this, see http://jira.codehaus.org/browse/SONARPLUGINS-2421 +#sonar.modules=module2 + +# delph plugin is broken +#sonar.modules=module13 + +# phpunit plugin is broken +#sonar.modules=module15 + +module1.sonar.projectName=Apache Thrift - Java Library +module1.sonar.projectBaseDir=lib/java +module1.sonar.sources=src +module1.sonar.tests=test +module1.sonar.binaries=build/libthrift-0.10.0.jar +module1.sonar.libraries=build/lib/*.jar +module1.sonar.language=java + +module2.sonar.projectName=Apache Thrift - Java Tutorial +module2.sonar.projectBaseDir=. +module2.sonar.sources=tutorial/java/src, tutorial/java/gen-java +module2.sonar.binaries=tutorial/java/tutorial.jar +module2.sonar.libraries=lib/java/build/lib/*.jar,lib/java/build/libthrift-1.0.0.jar +module2.sonar.language=java + +module3.sonar.projectName=Apache Thrift - JavaScript Library +module3.sonar.projectBaseDir=lib/js +module3.sonar.sources=. +module3.sonar.exclusions=test/**/* +module3.sonar.language=js + +module4.sonar.projectName=Apache Thrift - JavaScript Tutorial +module4.sonar.projectBaseDir=tutorial/js +module4.sonar.sources=. +module4.sonar.language=web + +module5.sonar.projectName=Apache Thrift - C++ Library +module5.sonar.projectBaseDir=lib/cpp +module5.sonar.sources=src +module5.sonar.tests=test +module5.sonar.language=c++ + +module6.sonar.projectName=Apache Thrift - C++ Tutorial +module6.sonar.projectBaseDir=tutorial/cpp +module6.sonar.sources=. +module6.sonar.exclusions=gen-cpp/**/* +module6.sonar.language=c++ + +module7.sonar.projectName=Apache Thrift - C++ Cross Language Test +module7.sonar.projectBaseDir=test/cpp +module7.sonar.sources=src +module7.sonar.language=c++ + +module8.sonar.projectName=Apache Thrift - Compiler +module8.sonar.projectBaseDir=compiler/cpp +module8.sonar.sources=src +module8.sonar.language=c++ + +module9.sonar.projectName=Apache Thrift - Python Library +module9.sonar.projectBaseDir=lib/py +module9.sonar.sources=src +module9.sonar.language=py + +module10.sonar.projectName=Apache Thrift - Python Tutorial +module10.sonar.projectBaseDir=tutorial/py +module10.sonar.sources=. +module10.sonar.exclusions=gen-py/**/* +module10.sonar.language=py + +module11.sonar.projectName=Apache Thrift - Python Cross Language Test +module11.sonar.projectBaseDir=test/py +module11.sonar.sources=. +module11.sonar.exclusions=gen-*/**/* +module11.sonar.language=py + +module12.sonar.projectName=Apache Thrift - c_glib Library +module12.sonar.projectBaseDir=lib/c_glib +module12.sonar.sources=src +module12.sonar.language=c + +module13.sonar.projectName=Apache Thrift - Delphi Library +module13.sonar.projectBaseDir=lib/delphi +module13.sonar.sources=src +module13.sonar.tests=test +module13.sonar.language=delph + +module14.sonar.projectName=Apache Thrift - Flex (as3) Library +module14.sonar.projectBaseDir=lib/as3 +module14.sonar.sources=src +module14.sonar.language=flex + +module15.sonar.projectName=Apache Thrift - PHP Library +module15.sonar.projectBaseDir=lib/php +module15.sonar.sources=src +module15.sonar.language=php + +# TODO add some more languages here + diff --git a/vendor/github.com/beorn7/perks/.gitignore b/vendor/github.com/beorn7/perks/.gitignore new file mode 100644 index 0000000000000..1bd9209aa195a --- /dev/null +++ b/vendor/github.com/beorn7/perks/.gitignore @@ -0,0 +1,2 @@ +*.test +*.prof diff --git a/vendor/github.com/beorn7/perks/README.md b/vendor/github.com/beorn7/perks/README.md new file mode 100644 index 0000000000000..fc05777701ab7 --- /dev/null +++ b/vendor/github.com/beorn7/perks/README.md @@ -0,0 +1,31 @@ +# Perks for Go (golang.org) + +Perks contains the Go package quantile that computes approximate quantiles over +an unbounded data stream within low memory and CPU bounds. + +For more information and examples, see: +http://godoc.org/github.com/bmizerany/perks + +A very special thank you and shout out to Graham Cormode (Rutgers University), +Flip Korn (AT&T Labs–Research), S. Muthukrishnan (Rutgers University), and +Divesh Srivastava (AT&T Labs–Research) for their research and publication of +[Effective Computation of Biased Quantiles over Data Streams](http://www.cs.rutgers.edu/~muthu/bquant.pdf) + +Thank you, also: +* Armon Dadgar (@armon) +* Andrew Gerrand (@nf) +* Brad Fitzpatrick (@bradfitz) +* Keith Rarick (@kr) + +FAQ: + +Q: Why not move the quantile package into the project root? +A: I want to add more packages to perks later. + +Copyright (C) 2013 Blake Mizerany + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/vendor/github.com/beorn7/perks/quantile/bench_test.go b/vendor/github.com/beorn7/perks/quantile/bench_test.go new file mode 100644 index 0000000000000..0bd0e4e775217 --- /dev/null +++ b/vendor/github.com/beorn7/perks/quantile/bench_test.go @@ -0,0 +1,63 @@ +package quantile + +import ( + "testing" +) + +func BenchmarkInsertTargeted(b *testing.B) { + b.ReportAllocs() + + s := NewTargeted(Targets) + b.ResetTimer() + for i := float64(0); i < float64(b.N); i++ { + s.Insert(i) + } +} + +func BenchmarkInsertTargetedSmallEpsilon(b *testing.B) { + s := NewTargeted(TargetsSmallEpsilon) + b.ResetTimer() + for i := float64(0); i < float64(b.N); i++ { + s.Insert(i) + } +} + +func BenchmarkInsertBiased(b *testing.B) { + s := NewLowBiased(0.01) + b.ResetTimer() + for i := float64(0); i < float64(b.N); i++ { + s.Insert(i) + } +} + +func BenchmarkInsertBiasedSmallEpsilon(b *testing.B) { + s := NewLowBiased(0.0001) + b.ResetTimer() + for i := float64(0); i < float64(b.N); i++ { + s.Insert(i) + } +} + +func BenchmarkQuery(b *testing.B) { + s := NewTargeted(Targets) + for i := float64(0); i < 1e6; i++ { + s.Insert(i) + } + b.ResetTimer() + n := float64(b.N) + for i := float64(0); i < n; i++ { + s.Query(i / n) + } +} + +func BenchmarkQuerySmallEpsilon(b *testing.B) { + s := NewTargeted(TargetsSmallEpsilon) + for i := float64(0); i < 1e6; i++ { + s.Insert(i) + } + b.ResetTimer() + n := float64(b.N) + for i := float64(0); i < n; i++ { + s.Query(i / n) + } +} diff --git a/vendor/github.com/beorn7/perks/quantile/example_test.go b/vendor/github.com/beorn7/perks/quantile/example_test.go new file mode 100644 index 0000000000000..ab3293aaf2a09 --- /dev/null +++ b/vendor/github.com/beorn7/perks/quantile/example_test.go @@ -0,0 +1,121 @@ +// +build go1.1 + +package quantile_test + +import ( + "bufio" + "fmt" + "log" + "os" + "strconv" + "time" + + "github.com/beorn7/perks/quantile" +) + +func Example_simple() { + ch := make(chan float64) + go sendFloats(ch) + + // Compute the 50th, 90th, and 99th percentile. + q := quantile.NewTargeted(map[float64]float64{ + 0.50: 0.005, + 0.90: 0.001, + 0.99: 0.0001, + }) + for v := range ch { + q.Insert(v) + } + + fmt.Println("perc50:", q.Query(0.50)) + fmt.Println("perc90:", q.Query(0.90)) + fmt.Println("perc99:", q.Query(0.99)) + fmt.Println("count:", q.Count()) + // Output: + // perc50: 5 + // perc90: 16 + // perc99: 223 + // count: 2388 +} + +func Example_mergeMultipleStreams() { + // Scenario: + // We have multiple database shards. On each shard, there is a process + // collecting query response times from the database logs and inserting + // them into a Stream (created via NewTargeted(0.90)), much like the + // Simple example. These processes expose a network interface for us to + // ask them to serialize and send us the results of their + // Stream.Samples so we may Merge and Query them. + // + // NOTES: + // * These sample sets are small, allowing us to get them + // across the network much faster than sending the entire list of data + // points. + // + // * For this to work correctly, we must supply the same quantiles + // a priori the process collecting the samples supplied to NewTargeted, + // even if we do not plan to query them all here. + ch := make(chan quantile.Samples) + getDBQuerySamples(ch) + q := quantile.NewTargeted(map[float64]float64{0.90: 0.001}) + for samples := range ch { + q.Merge(samples) + } + fmt.Println("perc90:", q.Query(0.90)) +} + +func Example_window() { + // Scenario: We want the 90th, 95th, and 99th percentiles for each + // minute. + + ch := make(chan float64) + go sendStreamValues(ch) + + tick := time.NewTicker(1 * time.Minute) + q := quantile.NewTargeted(map[float64]float64{ + 0.90: 0.001, + 0.95: 0.0005, + 0.99: 0.0001, + }) + for { + select { + case t := <-tick.C: + flushToDB(t, q.Samples()) + q.Reset() + case v := <-ch: + q.Insert(v) + } + } +} + +func sendStreamValues(ch chan float64) { + // Use your imagination +} + +func flushToDB(t time.Time, samples quantile.Samples) { + // Use your imagination +} + +// This is a stub for the above example. In reality this would hit the remote +// servers via http or something like it. +func getDBQuerySamples(ch chan quantile.Samples) {} + +func sendFloats(ch chan<- float64) { + f, err := os.Open("exampledata.txt") + if err != nil { + log.Fatal(err) + } + sc := bufio.NewScanner(f) + for sc.Scan() { + b := sc.Bytes() + v, err := strconv.ParseFloat(string(b), 64) + if err != nil { + log.Fatal(err) + } + ch <- v + } + if sc.Err() != nil { + log.Fatal(sc.Err()) + } + close(ch) +} diff --git a/vendor/github.com/beorn7/perks/quantile/exampledata.txt b/vendor/github.com/beorn7/perks/quantile/exampledata.txt new file mode 100644 index 0000000000000..1602287d7ce52 --- /dev/null +++ b/vendor/github.com/beorn7/perks/quantile/exampledata.txt @@ -0,0 +1,2388 @@ +8 +5 +26 +12 +5 +235 +13 +6 +28 +30 +3 +3 +3 +3 +5 +2 +33 +7 +2 +4 +7 +12 +14 +5 +8 +3 +10 +4 +5 +3 +6 +6 +209 +20 +3 +10 +14 +3 +4 +6 +8 +5 +11 +7 +3 +2 +3 +3 +212 +5 +222 +4 +10 +10 +5 +6 +3 +8 +3 +10 +254 +220 +2 +3 +5 +24 +5 +4 +222 +7 +3 +3 +223 +8 +15 +12 +14 +14 +3 +2 +2 +3 +13 +3 +11 +4 +4 +6 +5 +7 +13 +5 +3 +5 +2 +5 +3 +5 +2 +7 +15 +17 +14 +3 +6 +6 +3 +17 +5 +4 +7 +6 +4 +4 +8 +6 +8 +3 +9 +3 +6 +3 +4 +5 +3 +3 +660 +4 +6 +10 +3 +6 +3 +2 +5 +13 +2 +4 +4 +10 +4 +8 +4 +3 +7 +9 +9 +3 +10 +37 +3 +13 +4 +12 +3 +6 +10 +8 +5 +21 +2 +3 +8 +3 +2 +3 +3 +4 +12 +2 +4 +8 +8 +4 +3 +2 +20 +1 +6 +32 +2 +11 +6 +18 +3 +8 +11 +3 +212 +3 +4 +2 +6 +7 +12 +11 +3 +2 +16 +10 +6 +4 +6 +3 +2 +7 +3 +2 +2 +2 +2 +5 +6 +4 +3 +10 +3 +4 +6 +5 +3 +4 +4 +5 +6 +4 +3 +4 +4 +5 +7 +5 +5 +3 +2 +7 +2 +4 +12 +4 +5 +6 +2 +4 +4 +8 +4 +15 +13 +7 +16 +5 +3 +23 +5 +5 +7 +3 +2 +9 +8 +7 +5 +8 +11 +4 +10 +76 +4 +47 +4 +3 +2 +7 +4 +2 +3 +37 +10 +4 +2 +20 +5 +4 +4 +10 +10 +4 +3 +7 +23 +240 +7 +13 +5 +5 +3 +3 +2 +5 +4 +2 +8 +7 +19 +2 +23 +8 +7 +2 +5 +3 +8 +3 +8 +13 +5 +5 +5 +2 +3 +23 +4 +9 +8 +4 +3 +3 +5 +220 +2 +3 +4 +6 +14 +3 +53 +6 +2 +5 +18 +6 +3 +219 +6 +5 +2 +5 +3 +6 +5 +15 +4 +3 +17 +3 +2 +4 +7 +2 +3 +3 +4 +4 +3 +2 +664 +6 +3 +23 +5 +5 +16 +5 +8 +2 +4 +2 +24 +12 +3 +2 +3 +5 +8 +3 +5 +4 +3 +14 +3 +5 +8 +2 +3 +7 +9 +4 +2 +3 +6 +8 +4 +3 +4 +6 +5 +3 +3 +6 +3 +19 +4 +4 +6 +3 +6 +3 +5 +22 +5 +4 +4 +3 +8 +11 +4 +9 +7 +6 +13 +4 +4 +4 +6 +17 +9 +3 +3 +3 +4 +3 +221 +5 +11 +3 +4 +2 +12 +6 +3 +5 +7 +5 +7 +4 +9 +7 +14 +37 +19 +217 +16 +3 +5 +2 +2 +7 +19 +7 +6 +7 +4 +24 +5 +11 +4 +7 +7 +9 +13 +3 +4 +3 +6 +28 +4 +4 +5 +5 +2 +5 +6 +4 +4 +6 +10 +5 +4 +3 +2 +3 +3 +6 +5 +5 +4 +3 +2 +3 +7 +4 +6 +18 +16 +8 +16 +4 +5 +8 +6 +9 +13 +1545 +6 +215 +6 +5 +6 +3 +45 +31 +5 +2 +2 +4 +3 +3 +2 +5 +4 +3 +5 +7 +7 +4 +5 +8 +5 +4 +749 +2 +31 +9 +11 +2 +11 +5 +4 +4 +7 +9 +11 +4 +5 +4 +7 +3 +4 +6 +2 +15 +3 +4 +3 +4 +3 +5 +2 +13 +5 +5 +3 +3 +23 +4 +4 +5 +7 +4 +13 +2 +4 +3 +4 +2 +6 +2 +7 +3 +5 +5 +3 +29 +5 +4 +4 +3 +10 +2 +3 +79 +16 +6 +6 +7 +7 +3 +5 +5 +7 +4 +3 +7 +9 +5 +6 +5 +9 +6 +3 +6 +4 +17 +2 +10 +9 +3 +6 +2 +3 +21 +22 +5 +11 +4 +2 +17 +2 +224 +2 +14 +3 +4 +4 +2 +4 +4 +4 +4 +5 +3 +4 +4 +10 +2 +6 +3 +3 +5 +7 +2 +7 +5 +6 +3 +218 +2 +2 +5 +2 +6 +3 +5 +222 +14 +6 +33 +3 +2 +5 +3 +3 +3 +9 +5 +3 +3 +2 +7 +4 +3 +4 +3 +5 +6 +5 +26 +4 +13 +9 +7 +3 +221 +3 +3 +4 +4 +4 +4 +2 +18 +5 +3 +7 +9 +6 +8 +3 +10 +3 +11 +9 +5 +4 +17 +5 +5 +6 +6 +3 +2 +4 +12 +17 +6 +7 +218 +4 +2 +4 +10 +3 +5 +15 +3 +9 +4 +3 +3 +6 +29 +3 +3 +4 +5 +5 +3 +8 +5 +6 +6 +7 +5 +3 +5 +3 +29 +2 +31 +5 +15 +24 +16 +5 +207 +4 +3 +3 +2 +15 +4 +4 +13 +5 +5 +4 +6 +10 +2 +7 +8 +4 +6 +20 +5 +3 +4 +3 +12 +12 +5 +17 +7 +3 +3 +3 +6 +10 +3 +5 +25 +80 +4 +9 +3 +2 +11 +3 +3 +2 +3 +8 +7 +5 +5 +19 +5 +3 +3 +12 +11 +2 +6 +5 +5 +5 +3 +3 +3 +4 +209 +14 +3 +2 +5 +19 +4 +4 +3 +4 +14 +5 +6 +4 +13 +9 +7 +4 +7 +10 +2 +9 +5 +7 +2 +8 +4 +6 +5 +5 +222 +8 +7 +12 +5 +216 +3 +4 +4 +6 +3 +14 +8 +7 +13 +4 +3 +3 +3 +3 +17 +5 +4 +3 +33 +6 +6 +33 +7 +5 +3 +8 +7 +5 +2 +9 +4 +2 +233 +24 +7 +4 +8 +10 +3 +4 +15 +2 +16 +3 +3 +13 +12 +7 +5 +4 +207 +4 +2 +4 +27 +15 +2 +5 +2 +25 +6 +5 +5 +6 +13 +6 +18 +6 +4 +12 +225 +10 +7 +5 +2 +2 +11 +4 +14 +21 +8 +10 +3 +5 +4 +232 +2 +5 +5 +3 +7 +17 +11 +6 +6 +23 +4 +6 +3 +5 +4 +2 +17 +3 +6 +5 +8 +3 +2 +2 +14 +9 +4 +4 +2 +5 +5 +3 +7 +6 +12 +6 +10 +3 +6 +2 +2 +19 +5 +4 +4 +9 +2 +4 +13 +3 +5 +6 +3 +6 +5 +4 +9 +6 +3 +5 +7 +3 +6 +6 +4 +3 +10 +6 +3 +221 +3 +5 +3 +6 +4 +8 +5 +3 +6 +4 +4 +2 +54 +5 +6 +11 +3 +3 +4 +4 +4 +3 +7 +3 +11 +11 +7 +10 +6 +13 +223 +213 +15 +231 +7 +3 +7 +228 +2 +3 +4 +4 +5 +6 +7 +4 +13 +3 +4 +5 +3 +6 +4 +6 +7 +2 +4 +3 +4 +3 +3 +6 +3 +7 +3 +5 +18 +5 +6 +8 +10 +3 +3 +3 +2 +4 +2 +4 +4 +5 +6 +6 +4 +10 +13 +3 +12 +5 +12 +16 +8 +4 +19 +11 +2 +4 +5 +6 +8 +5 +6 +4 +18 +10 +4 +2 +216 +6 +6 +6 +2 +4 +12 +8 +3 +11 +5 +6 +14 +5 +3 +13 +4 +5 +4 +5 +3 +28 +6 +3 +7 +219 +3 +9 +7 +3 +10 +6 +3 +4 +19 +5 +7 +11 +6 +15 +19 +4 +13 +11 +3 +7 +5 +10 +2 +8 +11 +2 +6 +4 +6 +24 +6 +3 +3 +3 +3 +6 +18 +4 +11 +4 +2 +5 +10 +8 +3 +9 +5 +3 +4 +5 +6 +2 +5 +7 +4 +4 +14 +6 +4 +4 +5 +5 +7 +2 +4 +3 +7 +3 +3 +6 +4 +5 +4 +4 +4 +3 +3 +3 +3 +8 +14 +2 +3 +5 +3 +2 +4 +5 +3 +7 +3 +3 +18 +3 +4 +4 +5 +7 +3 +3 +3 +13 +5 +4 +8 +211 +5 +5 +3 +5 +2 +5 +4 +2 +655 +6 +3 +5 +11 +2 +5 +3 +12 +9 +15 +11 +5 +12 +217 +2 +6 +17 +3 +3 +207 +5 +5 +4 +5 +9 +3 +2 +8 +5 +4 +3 +2 +5 +12 +4 +14 +5 +4 +2 +13 +5 +8 +4 +225 +4 +3 +4 +5 +4 +3 +3 +6 +23 +9 +2 +6 +7 +233 +4 +4 +6 +18 +3 +4 +6 +3 +4 +4 +2 +3 +7 +4 +13 +227 +4 +3 +5 +4 +2 +12 +9 +17 +3 +7 +14 +6 +4 +5 +21 +4 +8 +9 +2 +9 +25 +16 +3 +6 +4 +7 +8 +5 +2 +3 +5 +4 +3 +3 +5 +3 +3 +3 +2 +3 +19 +2 +4 +3 +4 +2 +3 +4 +4 +2 +4 +3 +3 +3 +2 +6 +3 +17 +5 +6 +4 +3 +13 +5 +3 +3 +3 +4 +9 +4 +2 +14 +12 +4 +5 +24 +4 +3 +37 +12 +11 +21 +3 +4 +3 +13 +4 +2 +3 +15 +4 +11 +4 +4 +3 +8 +3 +4 +4 +12 +8 +5 +3 +3 +4 +2 +220 +3 +5 +223 +3 +3 +3 +10 +3 +15 +4 +241 +9 +7 +3 +6 +6 +23 +4 +13 +7 +3 +4 +7 +4 +9 +3 +3 +4 +10 +5 +5 +1 +5 +24 +2 +4 +5 +5 +6 +14 +3 +8 +2 +3 +5 +13 +13 +3 +5 +2 +3 +15 +3 +4 +2 +10 +4 +4 +4 +5 +5 +3 +5 +3 +4 +7 +4 +27 +3 +6 +4 +15 +3 +5 +6 +6 +5 +4 +8 +3 +9 +2 +6 +3 +4 +3 +7 +4 +18 +3 +11 +3 +3 +8 +9 +7 +24 +3 +219 +7 +10 +4 +5 +9 +12 +2 +5 +4 +4 +4 +3 +3 +19 +5 +8 +16 +8 +6 +22 +3 +23 +3 +242 +9 +4 +3 +3 +5 +7 +3 +3 +5 +8 +3 +7 +5 +14 +8 +10 +3 +4 +3 +7 +4 +6 +7 +4 +10 +4 +3 +11 +3 +7 +10 +3 +13 +6 +8 +12 +10 +5 +7 +9 +3 +4 +7 +7 +10 +8 +30 +9 +19 +4 +3 +19 +15 +4 +13 +3 +215 +223 +4 +7 +4 +8 +17 +16 +3 +7 +6 +5 +5 +4 +12 +3 +7 +4 +4 +13 +4 +5 +2 +5 +6 +5 +6 +6 +7 +10 +18 +23 +9 +3 +3 +6 +5 +2 +4 +2 +7 +3 +3 +2 +5 +5 +14 +10 +224 +6 +3 +4 +3 +7 +5 +9 +3 +6 +4 +2 +5 +11 +4 +3 +3 +2 +8 +4 +7 +4 +10 +7 +3 +3 +18 +18 +17 +3 +3 +3 +4 +5 +3 +3 +4 +12 +7 +3 +11 +13 +5 +4 +7 +13 +5 +4 +11 +3 +12 +3 +6 +4 +4 +21 +4 +6 +9 +5 +3 +10 +8 +4 +6 +4 +4 +6 +5 +4 +8 +6 +4 +6 +4 +4 +5 +9 +6 +3 +4 +2 +9 +3 +18 +2 +4 +3 +13 +3 +6 +6 +8 +7 +9 +3 +2 +16 +3 +4 +6 +3 +2 +33 +22 +14 +4 +9 +12 +4 +5 +6 +3 +23 +9 +4 +3 +5 +5 +3 +4 +5 +3 +5 +3 +10 +4 +5 +5 +8 +4 +4 +6 +8 +5 +4 +3 +4 +6 +3 +3 +3 +5 +9 +12 +6 +5 +9 +3 +5 +3 +2 +2 +2 +18 +3 +2 +21 +2 +5 +4 +6 +4 +5 +10 +3 +9 +3 +2 +10 +7 +3 +6 +6 +4 +4 +8 +12 +7 +3 +7 +3 +3 +9 +3 +4 +5 +4 +4 +5 +5 +10 +15 +4 +4 +14 +6 +227 +3 +14 +5 +216 +22 +5 +4 +2 +2 +6 +3 +4 +2 +9 +9 +4 +3 +28 +13 +11 +4 +5 +3 +3 +2 +3 +3 +5 +3 +4 +3 +5 +23 +26 +3 +4 +5 +6 +4 +6 +3 +5 +5 +3 +4 +3 +2 +2 +2 +7 +14 +3 +6 +7 +17 +2 +2 +15 +14 +16 +4 +6 +7 +13 +6 +4 +5 +6 +16 +3 +3 +28 +3 +6 +15 +3 +9 +2 +4 +6 +3 +3 +22 +4 +12 +6 +7 +2 +5 +4 +10 +3 +16 +6 +9 +2 +5 +12 +7 +5 +5 +5 +5 +2 +11 +9 +17 +4 +3 +11 +7 +3 +5 +15 +4 +3 +4 +211 +8 +7 +5 +4 +7 +6 +7 +6 +3 +6 +5 +6 +5 +3 +4 +4 +26 +4 +6 +10 +4 +4 +3 +2 +3 +3 +4 +5 +9 +3 +9 +4 +4 +5 +5 +8 +2 +4 +2 +3 +8 +4 +11 +19 +5 +8 +6 +3 +5 +6 +12 +3 +2 +4 +16 +12 +3 +4 +4 +8 +6 +5 +6 +6 +219 +8 +222 +6 +16 +3 +13 +19 +5 +4 +3 +11 +6 +10 +4 +7 +7 +12 +5 +3 +3 +5 +6 +10 +3 +8 +2 +5 +4 +7 +2 +4 +4 +2 +12 +9 +6 +4 +2 +40 +2 +4 +10 +4 +223 +4 +2 +20 +6 +7 +24 +5 +4 +5 +2 +20 +16 +6 +5 +13 +2 +3 +3 +19 +3 +2 +4 +5 +6 +7 +11 +12 +5 +6 +7 +7 +3 +5 +3 +5 +3 +14 +3 +4 +4 +2 +11 +1 +7 +3 +9 +6 +11 +12 +5 +8 +6 +221 +4 +2 +12 +4 +3 +15 +4 +5 +226 +7 +218 +7 +5 +4 +5 +18 +4 +5 +9 +4 +4 +2 +9 +18 +18 +9 +5 +6 +6 +3 +3 +7 +3 +5 +4 +4 +4 +12 +3 +6 +31 +5 +4 +7 +3 +6 +5 +6 +5 +11 +2 +2 +11 +11 +6 +7 +5 +8 +7 +10 +5 +23 +7 +4 +3 +5 +34 +2 +5 +23 +7 +3 +6 +8 +4 +4 +4 +2 +5 +3 +8 +5 +4 +8 +25 +2 +3 +17 +8 +3 +4 +8 +7 +3 +15 +6 +5 +7 +21 +9 +5 +6 +6 +5 +3 +2 +3 +10 +3 +6 +3 +14 +7 +4 +4 +8 +7 +8 +2 +6 +12 +4 +213 +6 +5 +21 +8 +2 +5 +23 +3 +11 +2 +3 +6 +25 +2 +3 +6 +7 +6 +6 +4 +4 +6 +3 +17 +9 +7 +6 +4 +3 +10 +7 +2 +3 +3 +3 +11 +8 +3 +7 +6 +4 +14 +36 +3 +4 +3 +3 +22 +13 +21 +4 +2 +7 +4 +4 +17 +15 +3 +7 +11 +2 +4 +7 +6 +209 +6 +3 +2 +2 +24 +4 +9 +4 +3 +3 +3 +29 +2 +2 +4 +3 +3 +5 +4 +6 +3 +3 +2 +4 diff --git a/vendor/github.com/beorn7/perks/quantile/stream_test.go b/vendor/github.com/beorn7/perks/quantile/stream_test.go new file mode 100644 index 0000000000000..4dba05449cb46 --- /dev/null +++ b/vendor/github.com/beorn7/perks/quantile/stream_test.go @@ -0,0 +1,188 @@ +package quantile + +import ( + "math" + "math/rand" + "sort" + "testing" +) + +var ( + Targets = map[float64]float64{ + 0.01: 0.001, + 0.10: 0.01, + 0.50: 0.05, + 0.90: 0.01, + 0.99: 0.001, + } + TargetsSmallEpsilon = map[float64]float64{ + 0.01: 0.0001, + 0.10: 0.001, + 0.50: 0.005, + 0.90: 0.001, + 0.99: 0.0001, + } + LowQuantiles = []float64{0.01, 0.1, 0.5} + HighQuantiles = []float64{0.99, 0.9, 0.5} +) + +const RelativeEpsilon = 0.01 + +func verifyPercsWithAbsoluteEpsilon(t *testing.T, a []float64, s *Stream) { + sort.Float64s(a) + for quantile, epsilon := range Targets { + n := float64(len(a)) + k := int(quantile * n) + lower := int((quantile - epsilon) * n) + if lower < 1 { + lower = 1 + } + upper := int(math.Ceil((quantile + epsilon) * n)) + if upper > len(a) { + upper = len(a) + } + w, min, max := a[k-1], a[lower-1], a[upper-1] + if g := s.Query(quantile); g < min || g > max { + t.Errorf("q=%f: want %v [%f,%f], got %v", quantile, w, min, max, g) + } + } +} + +func verifyLowPercsWithRelativeEpsilon(t *testing.T, a []float64, s *Stream) { + sort.Float64s(a) + for _, qu := range LowQuantiles { + n := float64(len(a)) + k := int(qu * n) + + lowerRank := int((1 - RelativeEpsilon) * qu * n) + upperRank := int(math.Ceil((1 + RelativeEpsilon) * qu * n)) + w, min, max := a[k-1], a[lowerRank-1], a[upperRank-1] + if g := s.Query(qu); g < min || g > max { + t.Errorf("q=%f: want %v [%f,%f], got %v", qu, w, min, max, g) + } + } +} + +func verifyHighPercsWithRelativeEpsilon(t *testing.T, a []float64, s *Stream) { + sort.Float64s(a) + for _, qu := range HighQuantiles { + n := float64(len(a)) + k := int(qu * n) + + lowerRank := int((1 - (1+RelativeEpsilon)*(1-qu)) * n) + upperRank := int(math.Ceil((1 - (1-RelativeEpsilon)*(1-qu)) * n)) + w, min, max := a[k-1], a[lowerRank-1], a[upperRank-1] + if g := s.Query(qu); g < min || g > max { + t.Errorf("q=%f: want %v [%f,%f], got %v", qu, w, min, max, g) + } + } +} + +func populateStream(s *Stream) []float64 { + a := make([]float64, 0, 1e5+100) + for i := 0; i < cap(a); i++ { + v := rand.NormFloat64() + // Add 5% asymmetric outliers. + if i%20 == 0 { + v = v*v + 1 + } + s.Insert(v) + a = append(a, v) + } + return a +} + +func TestTargetedQuery(t *testing.T) { + rand.Seed(42) + s := NewTargeted(Targets) + a := populateStream(s) + verifyPercsWithAbsoluteEpsilon(t, a, s) +} + +func TestLowBiasedQuery(t *testing.T) { + rand.Seed(42) + s := NewLowBiased(RelativeEpsilon) + a := populateStream(s) + verifyLowPercsWithRelativeEpsilon(t, a, s) +} + +func TestHighBiasedQuery(t *testing.T) { + rand.Seed(42) + s := NewHighBiased(RelativeEpsilon) + a := populateStream(s) + verifyHighPercsWithRelativeEpsilon(t, a, s) +} + +// BrokenTestTargetedMerge is broken, see Merge doc comment. +func BrokenTestTargetedMerge(t *testing.T) { + rand.Seed(42) + s1 := NewTargeted(Targets) + s2 := NewTargeted(Targets) + a := populateStream(s1) + a = append(a, populateStream(s2)...) + s1.Merge(s2.Samples()) + verifyPercsWithAbsoluteEpsilon(t, a, s1) +} + +// BrokenTestLowBiasedMerge is broken, see Merge doc comment. +func BrokenTestLowBiasedMerge(t *testing.T) { + rand.Seed(42) + s1 := NewLowBiased(RelativeEpsilon) + s2 := NewLowBiased(RelativeEpsilon) + a := populateStream(s1) + a = append(a, populateStream(s2)...) + s1.Merge(s2.Samples()) + verifyLowPercsWithRelativeEpsilon(t, a, s2) +} + +// BrokenTestHighBiasedMerge is broken, see Merge doc comment. +func BrokenTestHighBiasedMerge(t *testing.T) { + rand.Seed(42) + s1 := NewHighBiased(RelativeEpsilon) + s2 := NewHighBiased(RelativeEpsilon) + a := populateStream(s1) + a = append(a, populateStream(s2)...) + s1.Merge(s2.Samples()) + verifyHighPercsWithRelativeEpsilon(t, a, s2) +} + +func TestUncompressed(t *testing.T) { + q := NewTargeted(Targets) + for i := 100; i > 0; i-- { + q.Insert(float64(i)) + } + if g := q.Count(); g != 100 { + t.Errorf("want count 100, got %d", g) + } + // Before compression, Query should have 100% accuracy. + for quantile := range Targets { + w := quantile * 100 + if g := q.Query(quantile); g != w { + t.Errorf("want %f, got %f", w, g) + } + } +} + +func TestUncompressedSamples(t *testing.T) { + q := NewTargeted(map[float64]float64{0.99: 0.001}) + for i := 1; i <= 100; i++ { + q.Insert(float64(i)) + } + if g := q.Samples().Len(); g != 100 { + t.Errorf("want count 100, got %d", g) + } +} + +func TestUncompressedOne(t *testing.T) { + q := NewTargeted(map[float64]float64{0.99: 0.01}) + q.Insert(3.14) + if g := q.Query(0.90); g != 3.14 { + t.Error("want PI, got", g) + } +} + +func TestDefaults(t *testing.T) { + if g := NewTargeted(map[float64]float64{0.99: 0.001}).Query(0.99); g != 0 { + t.Errorf("want 0, got %f", g) + } +} diff --git a/vendor/github.com/blacktear23/go-proxyprotocol/.travis.yml b/vendor/github.com/blacktear23/go-proxyprotocol/.travis.yml new file mode 100644 index 0000000000000..67d2d12c8c25b --- /dev/null +++ b/vendor/github.com/blacktear23/go-proxyprotocol/.travis.yml @@ -0,0 +1,12 @@ +language: go + +go_import_path: github.com/blacktear23/go-proxyprotocol + +go: + - 1.9.x + +before_install: + - go get github.com/pingcap/check + +script: + - go test -cover diff --git a/vendor/github.com/blacktear23/go-proxyprotocol/README.md b/vendor/github.com/blacktear23/go-proxyprotocol/README.md new file mode 100644 index 0000000000000..f1de3b4a449ed --- /dev/null +++ b/vendor/github.com/blacktear23/go-proxyprotocol/README.md @@ -0,0 +1,37 @@ +# go-proxyprotocol + +PROXY protocol implementation in Go. + +## Usage + +import + +```go +import ( + proxyprotocol "github.com/blacktear23/go-proxyprotocol" +) +``` + +basic usage + +```go +// Create listener +l, err := net.Listen("tcp", "...") + +// Wrap listener as PROXY protocol listener +ppl, err := proxyprotocol.NewListener(l, "*", 5) + +for { + conn, err := ppl.Accept() + if err != nil { + // PROXY protocol related errors can be output by log and + // continue accept next one. + if proxyprotocol.IsProxyProtocolError(err) { + log.Errorf("PROXY protocol error: %s", err.Error()) + continue + } + panic(err) + } + go processConn(conn) +} +``` diff --git a/vendor/github.com/blacktear23/go-proxyprotocol/proxy_protocol_test.go b/vendor/github.com/blacktear23/go-proxyprotocol/proxy_protocol_test.go new file mode 100644 index 0000000000000..9fbe6ed006697 --- /dev/null +++ b/vendor/github.com/blacktear23/go-proxyprotocol/proxy_protocol_test.go @@ -0,0 +1,353 @@ +package proxyprotocol + +import ( + "bytes" + "encoding/binary" + "net" + "testing" + "time" + + . "github.com/pingcap/check" +) + +func Test(t *testing.T) { TestingT(t) } + +type ProxyProtocolTestSuite struct{} + +var _ = Suite(ProxyProtocolTestSuite{}) + +type mockBufferConn struct { + *bytes.Buffer + raddr net.Addr +} + +func newMockBufferConn(buffer *bytes.Buffer, raddr net.Addr) net.Conn { + return &mockBufferConn{ + Buffer: buffer, + raddr: raddr, + } +} + +func (c *mockBufferConn) Close() error { + return nil +} + +func (c *mockBufferConn) RemoteAddr() net.Addr { + if c.raddr != nil { + return c.raddr + } + addr, _ := net.ResolveTCPAddr("tcp", "127.0.0.1:12345") + return addr +} + +func (c *mockBufferConn) LocalAddr() net.Addr { + addr, _ := net.ResolveTCPAddr("tcp", "127.0.0.1:4000") + return addr +} + +func (c *mockBufferConn) SetDeadline(t time.Time) error { + return nil +} + +func (c *mockBufferConn) SetReadDeadline(t time.Time) error { + return nil +} + +func (c *mockBufferConn) SetWriteDeadline(t time.Time) error { + return nil +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolConnCheckAllowed(c *C) { + l, _ := newListener(nil, "*", 5) + raddr, _ := net.ResolveTCPAddr("tcp4", "192.168.1.100:8080") + c.Assert(l.checkAllowed(raddr), IsTrue) + l, _ = newListener(nil, "192.168.1.0/24,192.168.2.0/24", 5) + for _, ipstr := range []string{"192.168.1.100:8080", "192.168.2.100:8080"} { + raddr, _ := net.ResolveTCPAddr("tcp4", ipstr) + c.Assert(l.checkAllowed(raddr), IsTrue) + } + for _, ipstr := range []string{"192.168.3.100:8080", "192.168.4.100:8080"} { + raddr, _ := net.ResolveTCPAddr("tcp4", ipstr) + c.Assert(l.checkAllowed(raddr), IsFalse) + } +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolConnMustNotReadAnyDataAfterCLRF(c *C) { + buffer := []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data") + conn := newMockBufferConn(bytes.NewBuffer(buffer), nil) + + l, _ := newListener(nil, "*", 5) + wconn, err := l.createProxyProtocolConn(conn) + c.Assert(err, IsNil) + expectedString := "Other Data" + buf := make([]byte, 10) + n, _ := wconn.Read(buf) + c.Assert(n, Equals, 10) + c.Assert(string(buf[0:n]), Equals, expectedString) + + buffer = []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data") + conn = newMockBufferConn(bytes.NewBuffer(buffer), nil) + wconn, err = l.createProxyProtocolConn(conn) + c.Assert(err, IsNil) + buf = make([]byte, 5) + n, err = wconn.Read(buf) + c.Assert(err, IsNil) + c.Assert(n, Equals, 5) + c.Assert(string(buf[0:n]), Equals, "Other") + n, err = wconn.Read(buf) + c.Assert(err, IsNil) + c.Assert(n, Equals, 5) + c.Assert(string(buf[0:n]), Equals, " Data") + + buffer = []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data for a very long long long long long long long long long content") + expectedString = "Other Data for a very long long long long long long long long long content" + conn = newMockBufferConn(bytes.NewBuffer(buffer), nil) + wconn, err = l.createProxyProtocolConn(conn) + c.Assert(err, IsNil) + buf = make([]byte, 1024) + n, err = wconn.Read(buf) + c.Assert(err, IsNil) + c.Assert(string(buf[0:n]), Equals, expectedString) +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolV2ConnMustNotReadAnyDataAfterHeader(c *C) { + craddr, _ := net.ResolveTCPAddr("tcp4", "192.168.1.51:8080") + buffer := encodeProxyProtocolV2Header("tcp4", "192.168.1.100:5678", "192.168.1.5:4000") + expectedString := "Other Data" + buffer = append(buffer, []byte(expectedString)...) + l, _ := newListener(nil, "*", 5) + conn := newMockBufferConn(bytes.NewBuffer(buffer), craddr) + wconn, err := l.createProxyProtocolConn(conn) + buf := make([]byte, len(expectedString)) + n, err := wconn.Read(buf) + c.Assert(err, IsNil) + c.Assert(string(buf[0:n]), Equals, expectedString) +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolV1HeaderRead(c *C) { + buffer := []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data") + expectedString := "PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\n" + conn := newMockBufferConn(bytes.NewBuffer(buffer), nil) + wconn := &proxyProtocolConn{ + Conn: conn, + headerReadTimeout: 5, + } + ver, buf, err := wconn.readHeader() + c.Assert(err, IsNil) + c.Assert(ver, Equals, proxyProtocolV1) + c.Assert(string(buf), Equals, expectedString) +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolV1ExtractClientIP(c *C) { + craddr, _ := net.ResolveTCPAddr("tcp4", "192.168.1.51:8080") + tests := []struct { + buffer []byte + expectedIP string + expectedErr bool + }{ + { + buffer: []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data"), + expectedIP: "192.168.1.100:5678", + expectedErr: false, + }, + { + buffer: []byte("PROXY UNKNOWN 192.168.1.100 192.168.1.50 5678 3306\r\n"), + expectedIP: "192.168.1.51:8080", + expectedErr: false, + }, + { + buffer: []byte("PROXY TCP 192.168.1.100 192.168.1.50 5678 3306 3307\r\n"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306 jkasdjfkljaksldfjklajsdkfjsklafjldsafa"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY TCP4 192.168.1.100 192.168.1.50 5678 3306785478934785738275489275843728954782598345"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY TCP6 2001:0db8:85a3:0000:0000:8a2e:0370:7334 2001:0db8:85a3:0000:0000:8a2e:0390:7334 5678 3306\r\n"), + expectedIP: "[2001:db8:85a3::8a2e:370:7334]:5678", + expectedErr: false, + }, + { + buffer: []byte("this is a invalid header"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY MCP3 192.168.1.100 192.168.1.50 5678 3306\r\nOther Data"), + expectedIP: "", + expectedErr: true, + }, + { + buffer: []byte("PROXY UNKNOWN\r\n"), + expectedIP: "192.168.1.51:8080", + expectedErr: false, + }, + } + + l, _ := newListener(nil, "*", 5) + for _, t := range tests { + conn := newMockBufferConn(bytes.NewBuffer(t.buffer), craddr) + wconn, err := l.createProxyProtocolConn(conn) + if err == nil { + clientIP := wconn.RemoteAddr() + if t.expectedErr { + c.Assert(false, IsTrue, Commentf( + "Buffer:%s\nExpect Error", string(t.buffer))) + } + c.Assert(clientIP.String(), Equals, t.expectedIP, Commentf( + "Buffer:%s\nExpect: %s Got: %s", string(t.buffer), t.expectedIP, clientIP.String())) + } else { + if !t.expectedErr { + c.Assert(false, IsTrue, Commentf( + "Buffer:%s\nExpect %s But got Error: %v", string(t.buffer), t.expectedIP, err)) + } + } + } +} + +func encodeProxyProtocolV2Header(network, srcAddr, dstAddr string) []byte { + saddr, _ := net.ResolveTCPAddr(network, srcAddr) + daddr, _ := net.ResolveTCPAddr(network, dstAddr) + buffer := make([]byte, 1024) + copy(buffer, proxyProtocolV2Sig) + // Command + buffer[v2CmdPos] = 0x21 + // Famly + if network == "tcp4" { + buffer[v2FamlyPos] = 0x11 + binary.BigEndian.PutUint16(buffer[14:14+2], 12) + copy(buffer[16:16+4], []byte(saddr.IP.To4())) + copy(buffer[20:20+4], []byte(daddr.IP.To4())) + binary.BigEndian.PutUint16(buffer[24:24+2], uint16(saddr.Port)) + binary.BigEndian.PutUint16(buffer[26:26+2], uint16(saddr.Port)) + return buffer[0:28] + } else if network == "tcp6" { + buffer[v2FamlyPos] = 0x21 + binary.BigEndian.PutUint16(buffer[14:14+2], 36) + copy(buffer[16:16+16], []byte(saddr.IP.To16())) + copy(buffer[32:32+16], []byte(daddr.IP.To16())) + binary.BigEndian.PutUint16(buffer[48:48+2], uint16(saddr.Port)) + binary.BigEndian.PutUint16(buffer[50:50+2], uint16(saddr.Port)) + return buffer[0:52] + } + return buffer +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolV2HeaderRead(c *C) { + craddr, _ := net.ResolveTCPAddr("tcp4", "192.168.1.51:8080") + tests := []struct { + buffer []byte + expectedIP string + }{ + { + buffer: encodeProxyProtocolV2Header("tcp4", "192.168.1.100:5678", "192.168.1.5:4000"), + expectedIP: "192.168.1.100:5678", + }, + { + buffer: encodeProxyProtocolV2Header("tcp6", "[2001:db8:85a3::8a2e:370:7334]:5678", "[2001:db8:85a3::8a2e:370:8000]:4000"), + expectedIP: "[2001:db8:85a3::8a2e:370:7334]:5678", + }, + } + + l, _ := newListener(nil, "*", 5) + for _, t := range tests { + conn := newMockBufferConn(bytes.NewBuffer(t.buffer), craddr) + wconn, err := l.createProxyProtocolConn(conn) + clientIP := wconn.RemoteAddr() + if err == nil { + c.Assert(clientIP.String(), Equals, t.expectedIP, Commentf( + "Buffer:%v\nExpect: %s Got: %s", t.buffer, t.expectedIP, clientIP.String(), + )) + } else { + c.Assert(false, IsTrue, Commentf( + "Buffer:%v\nExpect: %s Got Error: %v", t.buffer, t.expectedIP, err, + )) + } + } +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolV2HeaderReadLocalCommand(c *C) { + craddr, _ := net.ResolveTCPAddr("tcp4", "192.168.1.51:8080") + buffer := encodeProxyProtocolV2Header("tcp4", "192.168.1.100:5678", "192.168.1.5:4000") + buffer[v2CmdPos] = 0x20 + l, _ := newListener(nil, "*", 5) + conn := newMockBufferConn(bytes.NewBuffer(buffer), craddr) + wconn, err := l.createProxyProtocolConn(conn) + clientIP := wconn.RemoteAddr() + c.Assert(err, IsNil) + c.Assert(clientIP.String(), Equals, craddr.String(), Commentf( + "Buffer:%v\nExpected: %s Got: %s", buffer, craddr.String(), clientIP.String(), + )) +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolListenerReadHeaderTimeout(c *C) { + addr := "127.0.0.1:18080" + go func() { + l, err := net.Listen("tcp", addr) + c.Assert(err, IsNil) + ppl, err := NewListener(l, "*", 1) + c.Assert(err, IsNil) + defer ppl.Close() + + conn, err := ppl.Accept() + c.Assert(conn, IsNil) + c.Assert(err, Equals, ErrHeaderReadTimeout) + }() + + conn, err := net.Dial("tcp", addr) + c.Assert(err, IsNil) + time.Sleep(2 * time.Second) + conn.Close() +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolListenerProxyNotAllowed(c *C) { + addr := "127.0.0.1:18080" + go func() { + l, err := net.Listen("tcp", addr) + c.Assert(err, IsNil) + ppl, err := NewListener(l, "192.168.1.1", 1) + c.Assert(err, IsNil) + defer ppl.Close() + + conn, err := ppl.Accept() + c.Assert(conn, IsNil) + c.Assert(err, Equals, ErrProxyAddressNotAllowed) + }() + + conn, err := net.Dial("tcp", addr) + c.Assert(err, IsNil) + time.Sleep(2 * time.Second) + conn.Close() +} + +func (ts ProxyProtocolTestSuite) TestProxyProtocolListenerCloseInOtherGoroutine(c *C) { + addr := "127.0.0.1:18081" + l, err := net.Listen("tcp", addr) + c.Assert(err, IsNil) + ppl, err := NewListener(l, "*", 1) + c.Assert(err, IsNil) + go func() { + conn, err := ppl.Accept() + c.Assert(conn, IsNil) + opErr, ok := err.(*net.OpError) + c.Assert(ok, IsTrue) + c.Assert(opErr.Err.Error(), Equals, "use of closed network connection") + }() + + time.Sleep(1 * time.Second) + ppl.Close() + time.Sleep(2 * time.Second) +} diff --git a/vendor/github.com/codahale/hdrhistogram/.travis.yml b/vendor/github.com/codahale/hdrhistogram/.travis.yml new file mode 100644 index 0000000000000..7960fc95b865d --- /dev/null +++ b/vendor/github.com/codahale/hdrhistogram/.travis.yml @@ -0,0 +1,5 @@ +language: go +go: + - 1.5 + - 1.6 + - tip diff --git a/vendor/github.com/codahale/hdrhistogram/README.md b/vendor/github.com/codahale/hdrhistogram/README.md new file mode 100644 index 0000000000000..614b197c3dd3a --- /dev/null +++ b/vendor/github.com/codahale/hdrhistogram/README.md @@ -0,0 +1,15 @@ +hdrhistogram +============ + +[![Build Status](https://travis-ci.org/codahale/hdrhistogram.png?branch=master)](https://travis-ci.org/codahale/hdrhistogram) + +A pure Go implementation of the [HDR Histogram](https://github.com/HdrHistogram/HdrHistogram). + +> A Histogram that supports recording and analyzing sampled data value counts +> across a configurable integer value range with configurable value precision +> within the range. Value precision is expressed as the number of significant +> digits in the value recording, and provides control over value quantization +> behavior across the value range and the subsequent value resolution at any +> given level. + +For documentation, check [godoc](http://godoc.org/github.com/codahale/hdrhistogram). diff --git a/vendor/github.com/codahale/hdrhistogram/hdr_test.go b/vendor/github.com/codahale/hdrhistogram/hdr_test.go new file mode 100644 index 0000000000000..309f0ea896978 --- /dev/null +++ b/vendor/github.com/codahale/hdrhistogram/hdr_test.go @@ -0,0 +1,388 @@ +package hdrhistogram_test + +import ( + "math" + "reflect" + "testing" + + "github.com/codahale/hdrhistogram" +) + +func TestHighSigFig(t *testing.T) { + input := []int64{ + 459876, 669187, 711612, 816326, 931423, 1033197, 1131895, 2477317, + 3964974, 12718782, + } + + hist := hdrhistogram.New(459876, 12718782, 5) + for _, sample := range input { + hist.RecordValue(sample) + } + + if v, want := hist.ValueAtQuantile(50), int64(1048575); v != want { + t.Errorf("Median was %v, but expected %v", v, want) + } +} + +func TestValueAtQuantile(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + data := []struct { + q float64 + v int64 + }{ + {q: 50, v: 500223}, + {q: 75, v: 750079}, + {q: 90, v: 900095}, + {q: 95, v: 950271}, + {q: 99, v: 990207}, + {q: 99.9, v: 999423}, + {q: 99.99, v: 999935}, + } + + for _, d := range data { + if v := h.ValueAtQuantile(d.q); v != d.v { + t.Errorf("P%v was %v, but expected %v", d.q, v, d.v) + } + } +} + +func TestMean(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + if v, want := h.Mean(), 500000.013312; v != want { + t.Errorf("Mean was %v, but expected %v", v, want) + } +} + +func TestStdDev(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + if v, want := h.StdDev(), 288675.1403682715; v != want { + t.Errorf("StdDev was %v, but expected %v", v, want) + } +} + +func TestTotalCount(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + if v, want := h.TotalCount(), int64(i+1); v != want { + t.Errorf("TotalCount was %v, but expected %v", v, want) + } + } +} + +func TestMax(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + if v, want := h.Max(), int64(1000447); v != want { + t.Errorf("Max was %v, but expected %v", v, want) + } +} + +func TestReset(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + h.Reset() + + if v, want := h.Max(), int64(0); v != want { + t.Errorf("Max was %v, but expected %v", v, want) + } +} + +func TestMerge(t *testing.T) { + h1 := hdrhistogram.New(1, 1000, 3) + h2 := hdrhistogram.New(1, 1000, 3) + + for i := 0; i < 100; i++ { + if err := h1.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + for i := 100; i < 200; i++ { + if err := h2.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + h1.Merge(h2) + + if v, want := h1.ValueAtQuantile(50), int64(99); v != want { + t.Errorf("Median was %v, but expected %v", v, want) + } +} + +func TestMin(t *testing.T) { + h := hdrhistogram.New(1, 10000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + if v, want := h.Min(), int64(0); v != want { + t.Errorf("Min was %v, but expected %v", v, want) + } +} + +func TestByteSize(t *testing.T) { + h := hdrhistogram.New(1, 100000, 3) + + if v, want := h.ByteSize(), 65604; v != want { + t.Errorf("ByteSize was %v, but expected %d", v, want) + } +} + +func TestRecordCorrectedValue(t *testing.T) { + h := hdrhistogram.New(1, 100000, 3) + + if err := h.RecordCorrectedValue(10, 100); err != nil { + t.Fatal(err) + } + + if v, want := h.ValueAtQuantile(75), int64(10); v != want { + t.Errorf("Corrected value was %v, but expected %v", v, want) + } +} + +func TestRecordCorrectedValueStall(t *testing.T) { + h := hdrhistogram.New(1, 100000, 3) + + if err := h.RecordCorrectedValue(1000, 100); err != nil { + t.Fatal(err) + } + + if v, want := h.ValueAtQuantile(75), int64(800); v != want { + t.Errorf("Corrected value was %v, but expected %v", v, want) + } +} + +func TestCumulativeDistribution(t *testing.T) { + h := hdrhistogram.New(1, 100000000, 3) + + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + actual := h.CumulativeDistribution() + expected := []hdrhistogram.Bracket{ + hdrhistogram.Bracket{Quantile: 0, Count: 1, ValueAt: 0}, + hdrhistogram.Bracket{Quantile: 50, Count: 500224, ValueAt: 500223}, + hdrhistogram.Bracket{Quantile: 75, Count: 750080, ValueAt: 750079}, + hdrhistogram.Bracket{Quantile: 87.5, Count: 875008, ValueAt: 875007}, + hdrhistogram.Bracket{Quantile: 93.75, Count: 937984, ValueAt: 937983}, + hdrhistogram.Bracket{Quantile: 96.875, Count: 969216, ValueAt: 969215}, + hdrhistogram.Bracket{Quantile: 98.4375, Count: 984576, ValueAt: 984575}, + hdrhistogram.Bracket{Quantile: 99.21875, Count: 992256, ValueAt: 992255}, + hdrhistogram.Bracket{Quantile: 99.609375, Count: 996352, ValueAt: 996351}, + hdrhistogram.Bracket{Quantile: 99.8046875, Count: 998400, ValueAt: 998399}, + hdrhistogram.Bracket{Quantile: 99.90234375, Count: 999424, ValueAt: 999423}, + hdrhistogram.Bracket{Quantile: 99.951171875, Count: 999936, ValueAt: 999935}, + hdrhistogram.Bracket{Quantile: 99.9755859375, Count: 999936, ValueAt: 999935}, + hdrhistogram.Bracket{Quantile: 99.98779296875, Count: 999936, ValueAt: 999935}, + hdrhistogram.Bracket{Quantile: 99.993896484375, Count: 1000000, ValueAt: 1000447}, + hdrhistogram.Bracket{Quantile: 100, Count: 1000000, ValueAt: 1000447}, + } + + if !reflect.DeepEqual(actual, expected) { + t.Errorf("CF was %#v, but expected %#v", actual, expected) + } +} + +func TestDistribution(t *testing.T) { + h := hdrhistogram.New(8, 1024, 3) + + for i := 0; i < 1024; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + actual := h.Distribution() + if len(actual) != 128 { + t.Errorf("Number of bars seen was %v, expected was 128", len(actual)) + } + for _, b := range actual { + if b.Count != 8 { + t.Errorf("Count per bar seen was %v, expected was 8", b.Count) + } + } +} + +func TestNaN(t *testing.T) { + h := hdrhistogram.New(1, 100000, 3) + if math.IsNaN(h.Mean()) { + t.Error("mean is NaN") + } + if math.IsNaN(h.StdDev()) { + t.Error("stddev is NaN") + } +} + +func TestSignificantFigures(t *testing.T) { + const sigFigs = 4 + h := hdrhistogram.New(1, 10, sigFigs) + if h.SignificantFigures() != sigFigs { + t.Errorf("Significant figures was %v, expected %d", h.SignificantFigures(), sigFigs) + } +} + +func TestLowestTrackableValue(t *testing.T) { + const minVal = 2 + h := hdrhistogram.New(minVal, 10, 3) + if h.LowestTrackableValue() != minVal { + t.Errorf("LowestTrackableValue figures was %v, expected %d", h.LowestTrackableValue(), minVal) + } +} + +func TestHighestTrackableValue(t *testing.T) { + const maxVal = 11 + h := hdrhistogram.New(1, maxVal, 3) + if h.HighestTrackableValue() != maxVal { + t.Errorf("HighestTrackableValue figures was %v, expected %d", h.HighestTrackableValue(), maxVal) + } +} + +func BenchmarkHistogramRecordValue(b *testing.B) { + h := hdrhistogram.New(1, 10000000, 3) + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + b.Fatal(err) + } + } + b.ResetTimer() + b.ReportAllocs() + + for i := 0; i < b.N; i++ { + h.RecordValue(100) + } +} + +func BenchmarkNew(b *testing.B) { + b.ReportAllocs() + + for i := 0; i < b.N; i++ { + hdrhistogram.New(1, 120000, 3) // this could track 1ms-2min + } +} + +func TestUnitMagnitudeOverflow(t *testing.T) { + h := hdrhistogram.New(0, 200, 4) + if err := h.RecordValue(11); err != nil { + t.Fatal(err) + } +} + +func TestSubBucketMaskOverflow(t *testing.T) { + hist := hdrhistogram.New(2e7, 1e8, 5) + for _, sample := range [...]int64{1e8, 2e7, 3e7} { + hist.RecordValue(sample) + } + + for q, want := range map[float64]int64{ + 50: 33554431, + 83.33: 33554431, + 83.34: 100663295, + 99: 100663295, + } { + if got := hist.ValueAtQuantile(q); got != want { + t.Errorf("got %d for %fth percentile. want: %d", got, q, want) + } + } +} + +func TestExportImport(t *testing.T) { + min := int64(1) + max := int64(10000000) + sigfigs := 3 + h := hdrhistogram.New(min, max, sigfigs) + for i := 0; i < 1000000; i++ { + if err := h.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + s := h.Export() + + if v := s.LowestTrackableValue; v != min { + t.Errorf("LowestTrackableValue was %v, but expected %v", v, min) + } + + if v := s.HighestTrackableValue; v != max { + t.Errorf("HighestTrackableValue was %v, but expected %v", v, max) + } + + if v := int(s.SignificantFigures); v != sigfigs { + t.Errorf("SignificantFigures was %v, but expected %v", v, sigfigs) + } + + if imported := hdrhistogram.Import(s); !imported.Equals(h) { + t.Error("Expected Histograms to be equivalent") + } + +} + +func TestEquals(t *testing.T) { + h1 := hdrhistogram.New(1, 10000000, 3) + for i := 0; i < 1000000; i++ { + if err := h1.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + h2 := hdrhistogram.New(1, 10000000, 3) + for i := 0; i < 10000; i++ { + if err := h1.RecordValue(int64(i)); err != nil { + t.Fatal(err) + } + } + + if h1.Equals(h2) { + t.Error("Expected Histograms to not be equivalent") + } + + h1.Reset() + h2.Reset() + + if !h1.Equals(h2) { + t.Error("Expected Histograms to be equivalent") + } +} diff --git a/vendor/github.com/codahale/hdrhistogram/window_test.go b/vendor/github.com/codahale/hdrhistogram/window_test.go new file mode 100644 index 0000000000000..7e787588af90e --- /dev/null +++ b/vendor/github.com/codahale/hdrhistogram/window_test.go @@ -0,0 +1,64 @@ +package hdrhistogram_test + +import ( + "testing" + + "github.com/codahale/hdrhistogram" +) + +func TestWindowedHistogram(t *testing.T) { + w := hdrhistogram.NewWindowed(2, 1, 1000, 3) + + for i := 0; i < 100; i++ { + w.Current.RecordValue(int64(i)) + } + w.Rotate() + + for i := 100; i < 200; i++ { + w.Current.RecordValue(int64(i)) + } + w.Rotate() + + for i := 200; i < 300; i++ { + w.Current.RecordValue(int64(i)) + } + + if v, want := w.Merge().ValueAtQuantile(50), int64(199); v != want { + t.Errorf("Median was %v, but expected %v", v, want) + } +} + +func BenchmarkWindowedHistogramRecordAndRotate(b *testing.B) { + w := hdrhistogram.NewWindowed(3, 1, 10000000, 3) + b.ReportAllocs() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + if err := w.Current.RecordValue(100); err != nil { + b.Fatal(err) + } + + if i%100000 == 1 { + w.Rotate() + } + } +} + +func BenchmarkWindowedHistogramMerge(b *testing.B) { + w := hdrhistogram.NewWindowed(3, 1, 10000000, 3) + for i := 0; i < 10000000; i++ { + if err := w.Current.RecordValue(100); err != nil { + b.Fatal(err) + } + + if i%100000 == 1 { + w.Rotate() + } + } + b.ReportAllocs() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + w.Merge() + } +} diff --git a/vendor/github.com/coreos/etcd/.dockerignore b/vendor/github.com/coreos/etcd/.dockerignore new file mode 100644 index 0000000000000..6b8710a711f3b --- /dev/null +++ b/vendor/github.com/coreos/etcd/.dockerignore @@ -0,0 +1 @@ +.git diff --git a/vendor/github.com/coreos/etcd/.gitignore b/vendor/github.com/coreos/etcd/.gitignore new file mode 100644 index 0000000000000..1a68387a7db71 --- /dev/null +++ b/vendor/github.com/coreos/etcd/.gitignore @@ -0,0 +1,22 @@ +/agent-* +/coverage +/covdir +/gopath +/gopath.proto +/go-bindata +/release +/machine* +/bin +.Dockerfile-test +.vagrant +*.etcd +*.log +/etcd +*.swp +/hack/insta-discovery/.env +*.test +tools/functional-tester/docker/bin +hack/scripts-dev/docker-dns/.Dockerfile +hack/scripts-dev/docker-dns-srv/.Dockerfile +hack/tls-setup/certs +.idea diff --git a/vendor/github.com/coreos/etcd/.godir b/vendor/github.com/coreos/etcd/.godir new file mode 100644 index 0000000000000..00ff6aa802e28 --- /dev/null +++ b/vendor/github.com/coreos/etcd/.godir @@ -0,0 +1 @@ +github.com/coreos/etcd diff --git a/vendor/github.com/coreos/etcd/.header b/vendor/github.com/coreos/etcd/.header new file mode 100644 index 0000000000000..0446af6d87763 --- /dev/null +++ b/vendor/github.com/coreos/etcd/.header @@ -0,0 +1,13 @@ +// Copyright 2016 The etcd 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. diff --git a/vendor/github.com/coreos/etcd/.semaphore.sh b/vendor/github.com/coreos/etcd/.semaphore.sh new file mode 100644 index 0000000000000..e7d80eb1b88cb --- /dev/null +++ b/vendor/github.com/coreos/etcd/.semaphore.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +TEST_SUFFIX=$(date +%s | base64 | head -c 15) + +TEST_OPTS="PASSES='build unit release integration_e2e functional' MANUAL_VER=v3.2.17" +if [ "$TEST_ARCH" == "386" ]; then + TEST_OPTS="GOARCH=386 PASSES='build unit integration_e2e'" +fi + +docker run \ + --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd \ + gcr.io/etcd-development/etcd-test:go1.8.7 \ + /bin/bash -c "${TEST_OPTS} ./test 2>&1 | tee test-${TEST_SUFFIX}.log" + +! egrep "(--- FAIL:|panic: test timed out|appears to have leaked)" -B50 -A10 test-${TEST_SUFFIX}.log diff --git a/vendor/github.com/coreos/etcd/.travis.yml b/vendor/github.com/coreos/etcd/.travis.yml new file mode 100644 index 0000000000000..2c4ff0e0541f7 --- /dev/null +++ b/vendor/github.com/coreos/etcd/.travis.yml @@ -0,0 +1,89 @@ +language: go +go_import_path: github.com/coreos/etcd + +sudo: required + +services: docker + +go: +- "1.8.7" +- tip + +notifications: + on_success: never + on_failure: never + +env: + matrix: + - TARGET=amd64 + - TARGET=amd64-go-tip + - TARGET=darwin-amd64 + - TARGET=windows-amd64 + - TARGET=arm64 + - TARGET=arm + - TARGET=386 + - TARGET=ppc64le + +matrix: + fast_finish: true + allow_failures: + - go: tip + env: TARGET=amd64-go-tip + exclude: + - go: "1.8.7" + env: TARGET=amd64-go-tip + - go: tip + env: TARGET=amd64 + - go: tip + env: TARGET=darwin-amd64 + - go: tip + env: TARGET=windows-amd64 + - go: tip + env: TARGET=arm + - go: tip + env: TARGET=arm64 + - go: tip + env: TARGET=386 + - go: tip + env: TARGET=ppc64le + +before_install: +- if [[ $TRAVIS_GO_VERSION == 1.* ]]; then docker pull gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION}; fi + +install: +- pushd cmd/etcd && go get -t -v ./... && popd + +script: + - echo "TRAVIS_GO_VERSION=${TRAVIS_GO_VERSION}" + - > + case "${TARGET}" in + amd64) + docker run --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ + /bin/bash -c "GOARCH=amd64 ./test" + ;; + amd64-go-tip) + GOARCH=amd64 ./test + ;; + darwin-amd64) + docker run --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ + /bin/bash -c "GO_BUILD_FLAGS='-a -v' GOOS=darwin GOARCH=amd64 ./build" + ;; + windows-amd64) + docker run --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ + /bin/bash -c "GO_BUILD_FLAGS='-a -v' GOOS=windows GOARCH=amd64 ./build" + ;; + 386) + docker run --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ + /bin/bash -c "GOARCH=386 PASSES='build unit' ./test" + ;; + *) + # test building out of gopath + docker run --rm \ + --volume=`pwd`:/go/src/github.com/coreos/etcd gcr.io/etcd-development/etcd-test:go${TRAVIS_GO_VERSION} \ + /bin/bash -c "GO_BUILD_FLAGS='-a -v' GOARCH='${TARGET}' ./build" + ;; + esac diff --git a/vendor/github.com/coreos/etcd/CONTRIBUTING.md b/vendor/github.com/coreos/etcd/CONTRIBUTING.md new file mode 100644 index 0000000000000..635f73a3056d9 --- /dev/null +++ b/vendor/github.com/coreos/etcd/CONTRIBUTING.md @@ -0,0 +1,62 @@ +# How to contribute + +etcd is Apache 2.0 licensed and accepts contributions via GitHub pull requests. This document outlines some of the conventions on commit message formatting, contact points for developers, and other resources to help get contributions into etcd. + +# Email and chat + +- Email: [etcd-dev](https://groups.google.com/forum/?hl=en#!forum/etcd-dev) +- IRC: #[coreos](irc://irc.freenode.org:6667/#coreos) IRC channel on freenode.org + +## Getting started + +- Fork the repository on GitHub +- Read the README.md for build instructions + +## Reporting bugs and creating issues + +Reporting bugs is one of the best ways to contribute. However, a good bug report has some very specific qualities, so please read over our short document on [reporting bugs](https://github.com/coreos/etcd/blob/master/Documentation/reporting_bugs.md) before submitting a bug report. This document might contain links to known issues, another good reason to take a look there before reporting a bug. + +## Contribution flow + +This is a rough outline of what a contributor's workflow looks like: + +- Create a topic branch from where to base the contribution. This is usually master. +- Make commits of logical units. +- Make sure commit messages are in the proper format (see below). +- Push changes in a topic branch to a personal fork of the repository. +- Submit a pull request to coreos/etcd. +- The PR must receive a LGTM from two maintainers found in the MAINTAINERS file. + +Thanks for contributing! + +### Code style + +The coding style suggested by the Golang community is used in etcd. See the [style doc](https://github.com/golang/go/wiki/CodeReviewComments) for details. + +Please follow this style to make etcd easy to review, maintain and develop. + +### Format of the commit message + +We follow a rough convention for commit messages that is designed to answer two +questions: what changed and why. The subject line should feature the what and +the body of the commit should describe the why. + +``` +scripts: add the test-cluster command + +this uses tmux to setup a test cluster that can easily be killed and started for debugging. + +Fixes #38 +``` + +The format can be described more formally as follows: + +``` +: + + + +