Skip to content

Commit

Permalink
planner: move QueryTimeRange structure to util pkg (#53071)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored May 7, 2024
1 parent 81a7cf5 commit 264cd4e
Show file tree
Hide file tree
Showing 11 changed files with 60 additions and 53 deletions.
25 changes: 13 additions & 12 deletions pkg/executor/inspection_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/kv"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -58,7 +59,7 @@ type (

inspectionFilter struct {
set set.StringSet
timeRange plannercore.QueryTimeRange
timeRange plannerutil.QueryTimeRange
}

inspectionRule interface {
Expand Down Expand Up @@ -108,7 +109,7 @@ type inspectionResultRetriever struct {
dummyCloser
retrieved bool
extractor *plannercore.InspectionResultTableExtractor
timeRange plannercore.QueryTimeRange
timeRange plannerutil.QueryTimeRange
instanceToStatusAddress map[string]string
statusToInstanceAddress map[string]string
}
Expand Down Expand Up @@ -490,7 +491,7 @@ func (nodeLoadInspection) inspect(ctx context.Context, sctx sessionctx.Context,

type inspectVirtualMemUsage struct{}

func (inspectVirtualMemUsage) genSQL(timeRange plannercore.QueryTimeRange) string {
func (inspectVirtualMemUsage) genSQL(timeRange plannerutil.QueryTimeRange) string {
sql := fmt.Sprintf("select instance, max(value) as max_usage from metrics_schema.node_memory_usage %s group by instance having max_usage >= 70", timeRange.Condition())
return sql
}
Expand All @@ -513,7 +514,7 @@ func (inspectVirtualMemUsage) getItem() string {

type inspectSwapMemoryUsed struct{}

func (inspectSwapMemoryUsed) genSQL(timeRange plannercore.QueryTimeRange) string {
func (inspectSwapMemoryUsed) genSQL(timeRange plannerutil.QueryTimeRange) string {
sql := fmt.Sprintf("select instance, max(value) as max_used from metrics_schema.node_memory_swap_used %s group by instance having max_used > 0", timeRange.Condition())
return sql
}
Expand All @@ -535,7 +536,7 @@ func (inspectSwapMemoryUsed) getItem() string {

type inspectDiskUsage struct{}

func (inspectDiskUsage) genSQL(timeRange plannercore.QueryTimeRange) string {
func (inspectDiskUsage) genSQL(timeRange plannerutil.QueryTimeRange) string {
sql := fmt.Sprintf("select instance, device, max(value) as max_usage from metrics_schema.node_disk_usage %v and device like '/%%' group by instance, device having max_usage >= 70", timeRange.Condition())
return sql
}
Expand All @@ -561,7 +562,7 @@ type inspectCPULoad struct {
tbl string
}

func (i inspectCPULoad) genSQL(timeRange plannercore.QueryTimeRange) string {
func (i inspectCPULoad) genSQL(timeRange plannerutil.QueryTimeRange) string {
sql := fmt.Sprintf(`select t1.instance, t1.max_load , 0.7*t2.cpu_count from
(select instance,max(value) as max_load from metrics_schema.%[1]s %[2]s group by instance) as t1 join
(select instance,max(value) as cpu_count from metrics_schema.node_virtual_cpus %[2]s group by instance) as t2
Expand Down Expand Up @@ -1039,7 +1040,7 @@ func (thresholdCheckInspection) inspectThreshold2(ctx context.Context, sctx sess
}

type ruleChecker interface {
genSQL(timeRange plannercore.QueryTimeRange) string
genSQL(timeRange plannerutil.QueryTimeRange) string
genResult(sql string, row chunk.Row) inspectionResult
getItem() string
}
Expand All @@ -1050,10 +1051,10 @@ type compareStoreStatus struct {
threshold float64
}

func (c compareStoreStatus) genSQL(timeRange plannercore.QueryTimeRange) string {
func (c compareStoreStatus) genSQL(timeRange plannerutil.QueryTimeRange) string {
condition := fmt.Sprintf(`where t1.time>='%[1]s' and t1.time<='%[2]s' and
t2.time>='%[1]s' and t2.time<='%[2]s'`, timeRange.From.Format(plannercore.MetricTableTimeFormat),
timeRange.To.Format(plannercore.MetricTableTimeFormat))
t2.time>='%[1]s' and t2.time<='%[2]s'`, timeRange.From.Format(plannerutil.MetricTableTimeFormat),
timeRange.To.Format(plannerutil.MetricTableTimeFormat))
return fmt.Sprintf(`
SELECT t1.address,
max(t1.value),
Expand Down Expand Up @@ -1097,7 +1098,7 @@ func (c compareStoreStatus) getItem() string {

type checkRegionHealth struct{}

func (checkRegionHealth) genSQL(timeRange plannercore.QueryTimeRange) string {
func (checkRegionHealth) genSQL(timeRange plannerutil.QueryTimeRange) string {
condition := timeRange.Condition()
return fmt.Sprintf(`select instance, sum(value) as sum_value from metrics_schema.pd_region_health %s and
type in ('extra-peer-region-count','learner-peer-region-count','pending-peer-region-count') having sum_value>100`, condition)
Expand Down Expand Up @@ -1125,7 +1126,7 @@ func (checkRegionHealth) getItem() string {

type checkStoreRegionTooMuch struct{}

func (checkStoreRegionTooMuch) genSQL(timeRange plannercore.QueryTimeRange) string {
func (checkStoreRegionTooMuch) genSQL(timeRange plannerutil.QueryTimeRange) string {
condition := timeRange.Condition()
return fmt.Sprintf(`select address, max(value) from metrics_schema.pd_scheduler_store_status %s and type='region_count' and value > 20000 group by address`, condition)
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/inspection_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
Expand All @@ -34,7 +35,7 @@ type inspectionSummaryRetriever struct {
retrieved bool
table *model.TableInfo
extractor *plannercore.InspectionSummaryTableExtractor
timeRange plannercore.QueryTimeRange
timeRange plannerutil.QueryTimeRange
}

// inspectionSummaryRules is used to maintain
Expand Down
5 changes: 3 additions & 2 deletions pkg/executor/metrics_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
Expand Down Expand Up @@ -188,7 +189,7 @@ type MetricsSummaryRetriever struct {
dummyCloser
table *model.TableInfo
extractor *plannercore.MetricSummaryTableExtractor
timeRange plannercore.QueryTimeRange
timeRange plannerutil.QueryTimeRange
retrieved bool
}

Expand Down Expand Up @@ -265,7 +266,7 @@ type MetricsSummaryByLabelRetriever struct {
dummyCloser
table *model.TableInfo
extractor *plannercore.MetricSummaryTableExtractor
timeRange plannercore.QueryTimeRange
timeRange plannerutil.QueryTimeRange
retrieved bool
}

Expand Down
3 changes: 0 additions & 3 deletions pkg/planner/core/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1109,9 +1109,6 @@ func (p *TiKVSingleGather) ExplainInfo() string {
return buffer.String()
}

// MetricTableTimeFormat is the time format for metric table explain and format.
const MetricTableTimeFormat = "2006-01-02 15:04:05.999"

// ExplainInfo implements Plan interface.
func (p *PhysicalMemTable) ExplainInfo() string {
accessObject, operatorInfo := p.AccessObject().String(), p.OperatorInfo(false)
Expand Down
8 changes: 4 additions & 4 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -5211,19 +5211,19 @@ func (ds *DataSource) ExtractFD() *fd.FDSet {
return ds.fdSet
}

func (b *PlanBuilder) timeRangeForSummaryTable() QueryTimeRange {
func (b *PlanBuilder) timeRangeForSummaryTable() util.QueryTimeRange {
const defaultSummaryDuration = 30 * time.Minute
hints := b.TableHints()
// User doesn't use TIME_RANGE hint
if hints == nil || (hints.TimeRangeHint.From == "" && hints.TimeRangeHint.To == "") {
to := time.Now()
from := to.Add(-defaultSummaryDuration)
return QueryTimeRange{From: from, To: to}
return util.QueryTimeRange{From: from, To: to}
}

// Parse time specified by user via TIM_RANGE hint
parse := func(s string) (time.Time, bool) {
t, err := time.ParseInLocation(MetricTableTimeFormat, s, time.Local)
t, err := time.ParseInLocation(util.MetricTableTimeFormat, s, time.Local)
if err != nil {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
}
Expand All @@ -5241,7 +5241,7 @@ func (b *PlanBuilder) timeRangeForSummaryTable() QueryTimeRange {
from = to.Add(-defaultSummaryDuration)
}

return QueryTimeRange{From: from, To: to}
return util.QueryTimeRange{From: from, To: to}
}

func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, tableInfo *model.TableInfo) (base.LogicalPlan, error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1404,7 +1404,7 @@ type LogicalMemTable struct {
// select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from metrics_summary_by_label;
// select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from inspection_summary;
// select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from inspection_result;
QueryTimeRange QueryTimeRange
QueryTimeRange util.QueryTimeRange
}

// LogicalUnionScan is used in non read-only txn or for scanning a local temporary table whose snapshot data is located in memory.
Expand Down
9 changes: 5 additions & 4 deletions pkg/planner/core/memtable_predicate_extractor.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
Expand Down Expand Up @@ -813,11 +814,11 @@ func (e *ClusterLogTableExtractor) ExplainInfo(pp base.PhysicalPlan) string {
st, et := e.StartTime, e.EndTime
if st > 0 {
st := time.UnixMilli(st)
fmt.Fprintf(r, "start_time:%v, ", st.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(MetricTableTimeFormat))
fmt.Fprintf(r, "start_time:%v, ", st.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(util.MetricTableTimeFormat))
}
if et > 0 {
et := time.UnixMilli(et)
fmt.Fprintf(r, "end_time:%v, ", et.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(MetricTableTimeFormat))
fmt.Fprintf(r, "end_time:%v, ", et.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(util.MetricTableTimeFormat))
}
if len(e.NodeTypes) > 0 {
fmt.Fprintf(r, "node_types:[%s], ", extractStringFromStringSet(e.NodeTypes))
Expand Down Expand Up @@ -1067,8 +1068,8 @@ func (e *MetricTableExtractor) ExplainInfo(pp base.PhysicalPlan) string {
step := time.Second * time.Duration(p.SCtx().GetSessionVars().MetricSchemaStep)
return fmt.Sprintf("PromQL:%v, start_time:%v, end_time:%v, step:%v",
promQL,
startTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(MetricTableTimeFormat),
endTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(MetricTableTimeFormat),
startTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(util.MetricTableTimeFormat),
endTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone()).Format(util.MetricTableTimeFormat),
step,
)
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/memtable_predicate_extractor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/planner"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -544,7 +545,7 @@ func TestMetricTableExtractor(t *testing.T) {
require.NoError(t, err)

parseTime := func(t *testing.T, s string) time.Time {
tt, err := time.ParseInLocation(plannercore.MetricTableTimeFormat, s, time.Local)
tt, err := time.ParseInLocation(util.MetricTableTimeFormat, s, time.Local)
require.NoError(t, err)
return tt
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -839,7 +839,7 @@ type PhysicalMemTable struct {
Table *model.TableInfo
Columns []*model.ColumnInfo
Extractor base.MemTablePredicateExtractor
QueryTimeRange QueryTimeRange
QueryTimeRange util.QueryTimeRange
}

// MemoryUsage return the memory usage of PhysicalMemTable
Expand Down
24 changes: 0 additions & 24 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@ import (
"math"
"strconv"
"strings"
"time"
"unsafe"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/br/pkg/storage"
Expand Down Expand Up @@ -84,28 +82,6 @@ type visitInfo struct {
dynamicWithGrant bool
}

// QueryTimeRange represents a time range specified by TIME_RANGE hint
type QueryTimeRange struct {
From time.Time
To time.Time
}

// Condition returns a WHERE clause base on it's value
func (tr *QueryTimeRange) Condition() string {
return fmt.Sprintf("where time>='%s' and time<='%s'", tr.From.Format(MetricTableTimeFormat), tr.To.Format(MetricTableTimeFormat))
}

const emptyQueryTimeRangeSize = int64(unsafe.Sizeof(QueryTimeRange{}))

// MemoryUsage return the memory usage of QueryTimeRange
func (tr *QueryTimeRange) MemoryUsage() (sum int64) {
if tr == nil {
return
}

return emptyQueryTimeRangeSize
}

// clauseCode indicates in which clause the column is currently.
type clauseCode int

Expand Down
29 changes: 29 additions & 0 deletions pkg/planner/util/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,10 @@
package util

import (
"fmt"
"time"
"unsafe"

"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/util/ranger"
Expand Down Expand Up @@ -59,3 +63,28 @@ func CloneRanges(ranges []*ranger.Range) []*ranger.Range {
}
return cloned
}

// QueryTimeRange represents a time range specified by TIME_RANGE hint
type QueryTimeRange struct {
From time.Time
To time.Time
}

// Condition returns a WHERE clause base on it's value
func (tr *QueryTimeRange) Condition() string {
return fmt.Sprintf("where time>='%s' and time<='%s'",
tr.From.Format(MetricTableTimeFormat), tr.To.Format(MetricTableTimeFormat))
}

// MetricTableTimeFormat is the time format for metric table explain and format.
const MetricTableTimeFormat = "2006-01-02 15:04:05.999"

const emptyQueryTimeRangeSize = int64(unsafe.Sizeof(QueryTimeRange{}))

// MemoryUsage return the memory usage of QueryTimeRange
func (tr *QueryTimeRange) MemoryUsage() (sum int64) {
if tr == nil {
return
}
return emptyQueryTimeRangeSize
}

0 comments on commit 264cd4e

Please sign in to comment.