Skip to content

Commit

Permalink
ttl: refractor ttl pkg into multiple sub packages (#39491)
Browse files Browse the repository at this point in the history
Signed-off-by: YangKeao <[email protected]>
  • Loading branch information
YangKeao authored Dec 1, 2022
1 parent 8e0e49c commit 7add1c8
Show file tree
Hide file tree
Showing 12 changed files with 207 additions and 79 deletions.
36 changes: 36 additions & 0 deletions ttl/cache/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
],
)
2 changes: 1 addition & 1 deletion ttl/main_test.go → ttl/cache/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
5 changes: 3 additions & 2 deletions ttl/table.go → ttl/cache/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package ttl
package cache

import (
"context"
Expand All @@ -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"
)
Expand Down Expand Up @@ -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
Expand Down
17 changes: 9 additions & 8 deletions ttl/table_test.go → ttl/cache/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
)

Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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")
Expand Down
35 changes: 35 additions & 0 deletions ttl/session/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -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",
],
)
33 changes: 33 additions & 0 deletions ttl/session/main_test.go
Original file line number Diff line number Diff line change
@@ -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...)
}
2 changes: 1 addition & 1 deletion ttl/session.go → ttl/session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package ttl
package session

import (
"context"
Expand Down
2 changes: 1 addition & 1 deletion ttl/session_test.go → ttl/session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package ttl
package session

import (
"context"
Expand Down
27 changes: 7 additions & 20 deletions ttl/BUILD.bazel → ttl/sqlbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,54 +1,41 @@
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",
"//parser/model",
"//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",
],
Expand Down
33 changes: 33 additions & 0 deletions ttl/sqlbuilder/main_test.go
Original file line number Diff line number Diff line change
@@ -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...)
}
13 changes: 7 additions & 6 deletions ttl/sql.go → ttl/sqlbuilder/sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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")
}
Expand Down
Loading

0 comments on commit 7add1c8

Please sign in to comment.