From 7add1c845f3d9085c6f317245b3a3a1615d84430 Mon Sep 17 00:00:00 2001 From: YangKeao Date: Wed, 30 Nov 2022 23:45:20 -0500 Subject: [PATCH] ttl: refractor ttl pkg into multiple sub packages (#39491) Signed-off-by: YangKeao --- ttl/cache/BUILD.bazel | 36 ++++++++++++++ ttl/{ => cache}/main_test.go | 2 +- ttl/{ => cache}/table.go | 5 +- ttl/{ => cache}/table_test.go | 17 ++++--- ttl/session/BUILD.bazel | 35 +++++++++++++ ttl/session/main_test.go | 33 +++++++++++++ ttl/{ => session}/session.go | 2 +- ttl/{ => session}/session_test.go | 2 +- ttl/{ => sqlbuilder}/BUILD.bazel | 27 +++-------- ttl/sqlbuilder/main_test.go | 33 +++++++++++++ ttl/{ => sqlbuilder}/sql.go | 13 ++--- ttl/{ => sqlbuilder}/sql_test.go | 81 ++++++++++++++++--------------- 12 files changed, 207 insertions(+), 79 deletions(-) create mode 100644 ttl/cache/BUILD.bazel rename ttl/{ => cache}/main_test.go (98%) rename ttl/{ => cache}/table.go (96%) rename ttl/{ => cache}/table_test.go (91%) create mode 100644 ttl/session/BUILD.bazel create mode 100644 ttl/session/main_test.go rename ttl/{ => session}/session.go (99%) rename ttl/{ => session}/session_test.go (99%) rename ttl/{ => sqlbuilder}/BUILD.bazel (59%) create mode 100644 ttl/sqlbuilder/main_test.go rename ttl/{ => sqlbuilder}/sql.go (96%) rename ttl/{ => sqlbuilder}/sql_test.go (94%) diff --git a/ttl/cache/BUILD.bazel b/ttl/cache/BUILD.bazel new file mode 100644 index 0000000000000..a1c84de044dbf --- /dev/null +++ b/ttl/cache/BUILD.bazel @@ -0,0 +1,36 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "cache", + srcs = ["table.go"], + importpath = "github.com/pingcap/tidb/ttl/cache", + visibility = ["//visibility:public"], + deps = [ + "//parser/ast", + "//parser/model", + "//parser/mysql", + "//table/tables", + "//ttl/session", + "//types", + "//util/chunk", + "@com_github_pingcap_errors//:errors", + ], +) + +go_test( + name = "cache_test", + srcs = [ + "main_test.go", + "table_test.go", + ], + flaky = True, + deps = [ + ":cache", + "//parser/model", + "//testkit", + "//testkit/testsetup", + "//ttl/session", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/ttl/main_test.go b/ttl/cache/main_test.go similarity index 98% rename from ttl/main_test.go rename to ttl/cache/main_test.go index 8bda0eb98eeef..9a9867d13d47d 100644 --- a/ttl/main_test.go +++ b/ttl/cache/main_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl_test +package cache_test import ( "testing" diff --git a/ttl/table.go b/ttl/cache/table.go similarity index 96% rename from ttl/table.go rename to ttl/cache/table.go index 4885da0e137b4..9b49c5186a2b6 100644 --- a/ttl/table.go +++ b/ttl/cache/table.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl +package cache import ( "context" @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) @@ -133,7 +134,7 @@ func (t *PhysicalTable) ValidateKey(key []types.Datum) error { } // EvalExpireTime returns the expired time -func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se Session, now time.Time) (expire time.Time, err error) { +func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session, now time.Time) (expire time.Time, err error) { tz := se.GetSessionVars().TimeZone expireExpr := t.TTLInfo.IntervalExprStr diff --git a/ttl/table_test.go b/ttl/cache/table_test.go similarity index 91% rename from ttl/table_test.go rename to ttl/cache/table_test.go index f77556c98dc09..fb450bd768f54 100644 --- a/ttl/table_test.go +++ b/ttl/cache/table_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl_test +package cache_test import ( "context" @@ -22,7 +22,8 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/ttl" + "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/session" "github.com/stretchr/testify/require" ) @@ -95,9 +96,9 @@ func TestNewTTLTable(t *testing.T) { tbl, err := is.TableByName(model.NewCIStr(c.db), model.NewCIStr(c.tbl)) require.NoError(t, err) tblInfo := tbl.Meta() - var physicalTbls []*ttl.PhysicalTable + var physicalTbls []*cache.PhysicalTable if tblInfo.Partition == nil { - ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr("")) + ttlTbl, err := cache.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr("")) if c.timeCol == "" { require.Error(t, err) continue @@ -106,7 +107,7 @@ func TestNewTTLTable(t *testing.T) { physicalTbls = append(physicalTbls, ttlTbl) } else { for _, partition := range tblInfo.Partition.Definitions { - ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr(partition.Name.O)) + ttlTbl, err := cache.NewPhysicalTable(model.NewCIStr(c.db), tblInfo, model.NewCIStr(partition.Name.O)) if c.timeCol == "" { require.Error(t, err) continue @@ -168,16 +169,16 @@ func TestEvalTTLExpireTime(t *testing.T) { tb, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() - ttlTbl, err := ttl.NewPhysicalTable(model.NewCIStr("test"), tblInfo, model.NewCIStr("")) + ttlTbl, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo, model.NewCIStr("")) require.NoError(t, err) tb2, err := do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) require.NoError(t, err) tblInfo2 := tb2.Meta() - ttlTbl2, err := ttl.NewPhysicalTable(model.NewCIStr("test"), tblInfo2, model.NewCIStr("")) + ttlTbl2, err := cache.NewPhysicalTable(model.NewCIStr("test"), tblInfo2, model.NewCIStr("")) require.NoError(t, err) - se := ttl.NewSession(tk.Session(), tk.Session(), nil) + se := session.NewSession(tk.Session(), tk.Session(), nil) now := time.UnixMilli(0) tz1, err := time.LoadLocation("Asia/Shanghai") diff --git a/ttl/session/BUILD.bazel b/ttl/session/BUILD.bazel new file mode 100644 index 0000000000000..36bdb259e6107 --- /dev/null +++ b/ttl/session/BUILD.bazel @@ -0,0 +1,35 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "session", + srcs = ["session.go"], + importpath = "github.com/pingcap/tidb/ttl/session", + visibility = ["//visibility:public"], + deps = [ + "//infoschema", + "//kv", + "//parser/terror", + "//sessionctx", + "//sessiontxn", + "//util/chunk", + "//util/sqlexec", + "@com_github_pingcap_errors//:errors", + ], +) + +go_test( + name = "session_test", + srcs = [ + "main_test.go", + "session_test.go", + ], + embed = [":session"], + flaky = True, + deps = [ + "//testkit", + "//testkit/testsetup", + "@com_github_pingcap_errors//:errors", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/ttl/session/main_test.go b/ttl/session/main_test.go new file mode 100644 index 0000000000000..330dcd581dbff --- /dev/null +++ b/ttl/session/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2022 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, +// 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 session_test + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/ttl/session.go b/ttl/session/session.go similarity index 99% rename from ttl/session.go rename to ttl/session/session.go index b3321e0d53c06..de03388e4cc2f 100644 --- a/ttl/session.go +++ b/ttl/session/session.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl +package session import ( "context" diff --git a/ttl/session_test.go b/ttl/session/session_test.go similarity index 99% rename from ttl/session_test.go rename to ttl/session/session_test.go index 90d47ed313e73..f23a717776b10 100644 --- a/ttl/session_test.go +++ b/ttl/session/session_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl +package session import ( "context" diff --git a/ttl/BUILD.bazel b/ttl/sqlbuilder/BUILD.bazel similarity index 59% rename from ttl/BUILD.bazel rename to ttl/sqlbuilder/BUILD.bazel index e5ec05168b29b..95d0371243158 100644 --- a/ttl/BUILD.bazel +++ b/ttl/sqlbuilder/BUILD.bazel @@ -1,44 +1,31 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( - name = "ttl", - srcs = [ - "session.go", - "sql.go", - "table.go", - ], - importpath = "github.com/pingcap/tidb/ttl", + name = "sqlbuilder", + srcs = ["sql.go"], + importpath = "github.com/pingcap/tidb/ttl/sqlbuilder", visibility = ["//visibility:public"], deps = [ - "//infoschema", - "//kv", "//parser/ast", "//parser/format", "//parser/model", "//parser/mysql", - "//parser/terror", - "//sessionctx", - "//sessiontxn", - "//table/tables", + "//ttl/cache", "//types", - "//util/chunk", "//util/sqlexec", - "@com_github_pingcap_errors//:errors", "@com_github_pkg_errors//:errors", ], ) go_test( - name = "ttl_test", + name = "sqlbuilder_test", srcs = [ "main_test.go", - "session_test.go", "sql_test.go", - "table_test.go", ], - embed = [":ttl"], flaky = True, deps = [ + ":sqlbuilder", "//kv", "//parser", "//parser/ast", @@ -46,9 +33,9 @@ go_test( "//parser/mysql", "//testkit", "//testkit/testsetup", + "//ttl/cache", "//types", "//util/sqlexec", - "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], diff --git a/ttl/sqlbuilder/main_test.go b/ttl/sqlbuilder/main_test.go new file mode 100644 index 0000000000000..76cecabaf752c --- /dev/null +++ b/ttl/sqlbuilder/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2022 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, +// 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 sqlbuilder_test + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/ttl/sql.go b/ttl/sqlbuilder/sql.go similarity index 96% rename from ttl/sql.go rename to ttl/sqlbuilder/sql.go index 3d100fd62eee7..4a27323da19a4 100644 --- a/ttl/sql.go +++ b/ttl/sqlbuilder/sql.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl +package sqlbuilder import ( "encoding/hex" @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/ttl/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" @@ -74,7 +75,7 @@ const ( // SQLBuilder is used to build SQLs for TTL type SQLBuilder struct { - tbl *PhysicalTable + tbl *cache.PhysicalTable sb strings.Builder restoreCtx *format.RestoreCtx state sqlBuilderState @@ -84,7 +85,7 @@ type SQLBuilder struct { } // NewSQLBuilder creates a new TTLSQLBuilder -func NewSQLBuilder(tbl *PhysicalTable) *SQLBuilder { +func NewSQLBuilder(tbl *cache.PhysicalTable) *SQLBuilder { b := &SQLBuilder{tbl: tbl, state: writeBegin} b.restoreCtx = format.NewRestoreCtx(format.DefaultRestoreFlags, &b.sb) return b @@ -304,7 +305,7 @@ func (b *SQLBuilder) writeDataPoint(cols []*model.ColumnInfo, dp []types.Datum) // ScanQueryGenerator generates SQLs for scan task type ScanQueryGenerator struct { - tbl *PhysicalTable + tbl *cache.PhysicalTable expire time.Time keyRangeStart []types.Datum keyRangeEnd []types.Datum @@ -314,7 +315,7 @@ type ScanQueryGenerator struct { } // NewScanQueryGenerator creates a new ScanQueryGenerator -func NewScanQueryGenerator(tbl *PhysicalTable, expire time.Time, rangeStart []types.Datum, rangeEnd []types.Datum) (*ScanQueryGenerator, error) { +func NewScanQueryGenerator(tbl *cache.PhysicalTable, expire time.Time, rangeStart []types.Datum, rangeEnd []types.Datum) (*ScanQueryGenerator, error) { if len(rangeStart) > 0 { if err := tbl.ValidateKey(rangeStart); err != nil { return nil, err @@ -446,7 +447,7 @@ func (g *ScanQueryGenerator) buildSQL() (string, error) { } // BuildDeleteSQL builds a delete SQL -func BuildDeleteSQL(tbl *PhysicalTable, rows [][]types.Datum, expire time.Time) (string, error) { +func BuildDeleteSQL(tbl *cache.PhysicalTable, rows [][]types.Datum, expire time.Time) (string, error) { if len(rows) == 0 { return "", errors.New("Cannot build delete SQL with empty rows") } diff --git a/ttl/sql_test.go b/ttl/sqlbuilder/sql_test.go similarity index 94% rename from ttl/sql_test.go rename to ttl/sqlbuilder/sql_test.go index f013d557402b5..dd6f58d9046ad 100644 --- a/ttl/sql_test.go +++ b/ttl/sqlbuilder/sql_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ttl_test +package sqlbuilder_test import ( "context" @@ -27,14 +27,15 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/ttl" + "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/sqlbuilder" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/require" ) func TestEscape(t *testing.T) { - tb := &ttl.PhysicalTable{ + tb := &cache.PhysicalTable{ Schema: model.NewCIStr("testp;\"';123`456"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("tp\"';123`456"), @@ -52,7 +53,7 @@ func TestEscape(t *testing.T) { } buildSelect := func(d []types.Datum) string { - b := ttl.NewSQLBuilder(tb) + b := sqlbuilder.NewSQLBuilder(tb) require.NoError(t, b.WriteSelect()) require.NoError(t, b.WriteCommonCondition(tb.KeyColumns, ">", d)) require.NoError(t, b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) @@ -62,7 +63,7 @@ func TestEscape(t *testing.T) { } buildDelete := func(ds ...[]types.Datum) string { - b := ttl.NewSQLBuilder(tb) + b := sqlbuilder.NewSQLBuilder(tb) require.NoError(t, b.WriteDelete()) require.NoError(t, b.WriteInCondition(tb.KeyColumns, ds...)) require.NoError(t, b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) @@ -288,7 +289,7 @@ func TestFormatSQLDatum(t *testing.T) { require.Equal(t, 1, len(rows), selectSQL) col := tbl.Meta().FindPublicColumnByName(colName) d := rows[0].GetDatum(0, &col.FieldType) - s, err := ttl.FormatSQLDatum(d, &col.FieldType) + s, err := sqlbuilder.FormatSQLDatum(d, &col.FieldType) if c.notSupport { require.Error(t, err) } else { @@ -308,15 +309,15 @@ func TestSQLBuilder(t *testing.T) { require.NoError(t, err) } - mustBuild := func(b *ttl.SQLBuilder, str string) { + mustBuild := func(b *sqlbuilder.SQLBuilder, str string) { s, err := b.Build() require.NoError(t, err) require.Equal(t, str, s) } - var b *ttl.SQLBuilder + var b *sqlbuilder.SQLBuilder - t1 := &ttl.PhysicalTable{ + t1 := &cache.PhysicalTable{ Schema: model.NewCIStr("test"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t1"), @@ -330,7 +331,7 @@ func TestSQLBuilder(t *testing.T) { }, } - t2 := &ttl.PhysicalTable{ + t2 := &cache.PhysicalTable{ Schema: model.NewCIStr("test2"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t2"), @@ -345,7 +346,7 @@ func TestSQLBuilder(t *testing.T) { }, } - tp := &ttl.PhysicalTable{ + tp := &cache.PhysicalTable{ Schema: model.NewCIStr("testp"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("tp"), @@ -358,57 +359,57 @@ func TestSQLBuilder(t *testing.T) { } // test build select queries - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1`") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteCommonCondition(t1.KeyColumns, ">", d("a1"))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `id` > 'a1'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteCommonCondition(t1.KeyColumns, ">", d("a1"))) must(b.WriteCommonCondition(t1.KeyColumns, "<=", d("c3"))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `id` > 'a1' AND `id` <= 'c3'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) shLoc, err := time.LoadLocation("Asia/Shanghai") require.NoError(t, err) must(b.WriteExpireCondition(time.UnixMilli(0).In(shLoc))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `time` < '1970-01-01 08:00:00'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteCommonCondition(t1.KeyColumns, ">", d("a1"))) must(b.WriteCommonCondition(t1.KeyColumns, "<=", d("c3"))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `id` > 'a1' AND `id` <= 'c3' AND `time` < '1970-01-01 00:00:00'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteOrderBy(t1.KeyColumns, false)) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` ORDER BY `id` ASC") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteOrderBy(t1.KeyColumns, true)) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` ORDER BY `id` DESC") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteOrderBy(t1.KeyColumns, false)) must(b.WriteLimit(128)) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` ORDER BY `id` ASC LIMIT 128") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteCommonCondition(t1.KeyColumns, ">", d("';``~?%\"\n"))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `id` > '\\';``~?%\\\"\\n'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteSelect()) must(b.WriteCommonCondition(t1.KeyColumns, ">", d("a1';'"))) must(b.WriteCommonCondition(t1.KeyColumns, "<=", d("a2\""))) @@ -417,12 +418,12 @@ func TestSQLBuilder(t *testing.T) { must(b.WriteLimit(128)) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `test`.`t1` WHERE `id` > 'a1\\';\\'' AND `id` <= 'a2\\\"' AND `time` < '1970-01-01 00:00:00' ORDER BY `id` ASC LIMIT 128") - b = ttl.NewSQLBuilder(t2) + b = sqlbuilder.NewSQLBuilder(t2) must(b.WriteSelect()) must(b.WriteCommonCondition(t2.KeyColumns, ">", d("x1", 20))) mustBuild(b, "SELECT LOW_PRIORITY `a`, `b` FROM `test2`.`t2` WHERE (`a`, `b`) > ('x1', 20)") - b = ttl.NewSQLBuilder(t2) + b = sqlbuilder.NewSQLBuilder(t2) must(b.WriteSelect()) must(b.WriteCommonCondition(t2.KeyColumns, "<=", d("x2", 21))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) @@ -430,7 +431,7 @@ func TestSQLBuilder(t *testing.T) { must(b.WriteLimit(100)) mustBuild(b, "SELECT LOW_PRIORITY `a`, `b` FROM `test2`.`t2` WHERE (`a`, `b`) <= ('x2', 21) AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b` ASC LIMIT 100") - b = ttl.NewSQLBuilder(t2) + b = sqlbuilder.NewSQLBuilder(t2) must(b.WriteSelect()) must(b.WriteCommonCondition(t2.KeyColumns[0:1], "=", d("x3"))) must(b.WriteCommonCondition(t2.KeyColumns[1:2], ">", d(31))) @@ -440,51 +441,51 @@ func TestSQLBuilder(t *testing.T) { mustBuild(b, "SELECT LOW_PRIORITY `a`, `b` FROM `test2`.`t2` WHERE `a` = 'x3' AND `b` > 31 AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b` ASC LIMIT 100") // test build delete queries - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) _, err = b.Build() require.EqualError(t, err, "expire condition not write") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) must(b.WriteInCondition(t1.KeyColumns, d("a"))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) mustBuild(b, "DELETE LOW_PRIORITY FROM `test`.`t1` WHERE `id` IN ('a') AND `time` < '1970-01-01 00:00:00'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) must(b.WriteInCondition(t1.KeyColumns, d("a"), d("b"))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) mustBuild(b, "DELETE LOW_PRIORITY FROM `test`.`t1` WHERE `id` IN ('a', 'b') AND `time` < '1970-01-01 00:00:00'") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) must(b.WriteInCondition(t2.KeyColumns, d("a", 1))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) must(b.WriteLimit(100)) mustBuild(b, "DELETE LOW_PRIORITY FROM `test`.`t1` WHERE (`a`, `b`) IN (('a', 1)) AND `time` < '1970-01-01 00:00:00' LIMIT 100") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) must(b.WriteInCondition(t2.KeyColumns, d("a", 1), d("b", 2))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) must(b.WriteLimit(100)) mustBuild(b, "DELETE LOW_PRIORITY FROM `test`.`t1` WHERE (`a`, `b`) IN (('a', 1), ('b', 2)) AND `time` < '1970-01-01 00:00:00' LIMIT 100") - b = ttl.NewSQLBuilder(t1) + b = sqlbuilder.NewSQLBuilder(t1) must(b.WriteDelete()) must(b.WriteInCondition(t2.KeyColumns, d("a", 1), d("b", 2))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) mustBuild(b, "DELETE LOW_PRIORITY FROM `test`.`t1` WHERE (`a`, `b`) IN (('a', 1), ('b', 2)) AND `time` < '1970-01-01 00:00:00'") // test select partition table - b = ttl.NewSQLBuilder(tp) + b = sqlbuilder.NewSQLBuilder(tp) must(b.WriteSelect()) must(b.WriteCommonCondition(tp.KeyColumns, ">", d("a1"))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) mustBuild(b, "SELECT LOW_PRIORITY `id` FROM `testp`.`tp` PARTITION(`p1`) WHERE `id` > 'a1' AND `time` < '1970-01-01 00:00:00'") - b = ttl.NewSQLBuilder(tp) + b = sqlbuilder.NewSQLBuilder(tp) must(b.WriteDelete()) must(b.WriteInCondition(tp.KeyColumns, d("a"), d("b"))) must(b.WriteExpireCondition(time.UnixMilli(0).In(time.UTC))) @@ -492,7 +493,7 @@ func TestSQLBuilder(t *testing.T) { } func TestScanQueryGenerator(t *testing.T) { - t1 := &ttl.PhysicalTable{ + t1 := &cache.PhysicalTable{ Schema: model.NewCIStr("test"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t1"), @@ -506,7 +507,7 @@ func TestScanQueryGenerator(t *testing.T) { }, } - t2 := &ttl.PhysicalTable{ + t2 := &cache.PhysicalTable{ Schema: model.NewCIStr("test2"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t2"), @@ -529,7 +530,7 @@ func TestScanQueryGenerator(t *testing.T) { } cases := []struct { - tbl *ttl.PhysicalTable + tbl *cache.PhysicalTable expire time.Time rangeStart []types.Datum rangeEnd []types.Datum @@ -704,7 +705,7 @@ func TestScanQueryGenerator(t *testing.T) { } for i, c := range cases { - g, err := ttl.NewScanQueryGenerator(c.tbl, c.expire, c.rangeStart, c.rangeEnd) + g, err := sqlbuilder.NewScanQueryGenerator(c.tbl, c.expire, c.rangeStart, c.rangeEnd) require.NoError(t, err, fmt.Sprintf("%d", i)) for j, p := range c.path { msg := fmt.Sprintf("%d-%d", i, j) @@ -728,7 +729,7 @@ func TestScanQueryGenerator(t *testing.T) { } func TestBuildDeleteSQL(t *testing.T) { - t1 := &ttl.PhysicalTable{ + t1 := &cache.PhysicalTable{ Schema: model.NewCIStr("test"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t1"), @@ -742,7 +743,7 @@ func TestBuildDeleteSQL(t *testing.T) { }, } - t2 := &ttl.PhysicalTable{ + t2 := &cache.PhysicalTable{ Schema: model.NewCIStr("test2"), TableInfo: &model.TableInfo{ Name: model.NewCIStr("t2"), @@ -758,7 +759,7 @@ func TestBuildDeleteSQL(t *testing.T) { } cases := []struct { - tbl *ttl.PhysicalTable + tbl *cache.PhysicalTable expire time.Time rows [][]types.Datum sql string @@ -790,7 +791,7 @@ func TestBuildDeleteSQL(t *testing.T) { } for _, c := range cases { - sql, err := ttl.BuildDeleteSQL(c.tbl, c.rows, c.expire) + sql, err := sqlbuilder.BuildDeleteSQL(c.tbl, c.rows, c.expire) require.NoError(t, err) require.Equal(t, c.sql, sql) }