diff --git a/statistics/column.go b/statistics/column.go index 9d988c1eb7503..5ce06b5917854 100644 --- a/statistics/column.go +++ b/statistics/column.go @@ -184,7 +184,7 @@ func (c *Column) equalRowCount(sctx sessionctx.Context, val types.Datum, encoded } // GetColumnRowCount estimates the row count by a slice of Range. -func (c *Column) GetColumnRowCount(sctx sessionctx.Context, ranges []*ranger.Range, realtimeRowCount int64, pkIsHandle bool) (float64, error) { +func (c *Column) GetColumnRowCount(sctx sessionctx.Context, ranges []*ranger.Range, realtimeRowCount, modifyCount int64, pkIsHandle bool) (float64, error) { sc := sctx.GetSessionVars().StmtCtx var rowCount float64 for _, rg := range ranges { @@ -281,11 +281,7 @@ func (c *Column) GetColumnRowCount(sctx sessionctx.Context, ranges []*ranger.Ran // handling the out-of-range part if (c.outOfRange(lowVal) && !lowVal.IsNull()) || c.outOfRange(highVal) { - increaseCount := realtimeRowCount - int64(c.TotalRowCount()) - if increaseCount < 0 { - increaseCount = 0 - } - cnt += c.Histogram.outOfRangeRowCount(&lowVal, &highVal, increaseCount) + cnt += c.Histogram.outOfRangeRowCount(&lowVal, &highVal, modifyCount) } rowCount += cnt diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 8d50de9eada83..09d36759bf1fc 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -1474,10 +1474,10 @@ func (h *Handle) RecalculateExpectCount(q *statistics.QueryFeedback, enablePseud expected := 0.0 if isIndex { idx := t.Indices[id] - expected, err = idx.GetRowCount(sctx, nil, ranges, t.Count) + expected, err = idx.GetRowCount(sctx, nil, ranges, t.Count, t.ModifyCount) } else { c := t.Columns[id] - expected, err = c.GetColumnRowCount(sctx, ranges, t.Count, true) + expected, err = c.GetColumnRowCount(sctx, ranges, t.Count, t.ModifyCount, true) } q.Expected = int64(expected) return err diff --git a/statistics/histogram.go b/statistics/histogram.go index 8c662b6f04061..3cab23f0492d8 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -778,7 +778,7 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { // outOfRangeRowCount estimate the row count of part of [lDatum, rDatum] which is out of range of the histogram. // Here we assume the density of data is decreasing from the lower/upper bound of the histogram toward outside. -// The maximum row count it can get is the increaseCount. It reaches the maximum when out-of-range width reaches histogram range width. +// The maximum row count it can get is the modifyCount. It reaches the maximum when out-of-range width reaches histogram range width. // As it shows below. To calculate the out-of-range row count, we need to calculate the percentage of the shaded area. // Note that we assume histL-boundL == histR-histL == boundR-histR here. /* @@ -795,7 +795,7 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { │ │ lDatum rDatum */ -func (hg *Histogram) outOfRangeRowCount(lDatum, rDatum *types.Datum, increaseCount int64) float64 { +func (hg *Histogram) outOfRangeRowCount(lDatum, rDatum *types.Datum, modifyCount int64) float64 { if hg.Len() == 0 { return 0 } @@ -879,8 +879,14 @@ func (hg *Histogram) outOfRangeRowCount(lDatum, rDatum *types.Datum, increaseCou totalPercent = 1 } rowCount := totalPercent * hg.notNullCount() - if rowCount > float64(increaseCount) { - return float64(increaseCount) + + // Use the modifyCount as the upper bound. Note that modifyCount contains insert, delete and update. So this is + // a rather loose upper bound. + // There are some scenarios where we need to handle out-of-range estimation after both insert and delete happen. + // But we don't know how many increases are in the modifyCount. So we have to use this loose bound to ensure it + // can produce a reasonable results in this scenario. + if rowCount > float64(modifyCount) { + return float64(modifyCount) } return rowCount } diff --git a/statistics/index.go b/statistics/index.go index 731aaa5eb2964..40ba2b005843b 100644 --- a/statistics/index.go +++ b/statistics/index.go @@ -216,7 +216,7 @@ func (idx *Index) QueryBytes(d []byte) uint64 { // GetRowCount returns the row count of the given ranges. // It uses the modifyCount to adjust the influence of modifications on the table. -func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRanges []*ranger.Range, realtimeRowCount int64) (float64, error) { +func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRanges []*ranger.Range, realtimeRowCount, modifyCount int64) (float64, error) { idx.checkStats() sc := sctx.GetSessionVars().StmtCtx totalCount := float64(0) @@ -310,11 +310,7 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang // handling the out-of-range part if (idx.outOfRange(l) && !(isSingleCol && lowIsNull)) || idx.outOfRange(r) { - increaseCount := realtimeRowCount - int64(idx.TotalRowCount()) - if increaseCount < 0 { - increaseCount = 0 - } - count += idx.Histogram.outOfRangeRowCount(&l, &r, increaseCount) + count += idx.Histogram.outOfRangeRowCount(&l, &r, modifyCount) } totalCount += count } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 6f65b79d86d5a..3cd5211fc1c8d 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -128,7 +128,7 @@ func TestOutOfRangeEstimation(t *testing.T) { statsTbl := h.GetTableStats(table.Meta()) sctx := mock.NewContext() col := statsTbl.Columns[table.Meta().Columns[0].ID] - count, err := col.GetColumnRowCount(sctx, getRange(900, 900), statsTbl.Count, false) + count, err := col.GetColumnRowCount(sctx, getRange(900, 900), statsTbl.Count, statsTbl.ModifyCount, false) require.NoError(t, err) // Because the ANALYZE collect data by random sampling, so the result is not an accurate value. // so we use a range here. @@ -147,8 +147,9 @@ func TestOutOfRangeEstimation(t *testing.T) { statsSuiteData := statistics.GetStatsSuiteData() statsSuiteData.LoadTestCases(t, &input, &output) increasedTblRowCount := int64(float64(statsTbl.Count) * 1.5) + modifyCount := int64(float64(statsTbl.Count) * 0.5) for i, ran := range input { - count, err = col.GetColumnRowCount(sctx, getRange(ran.Start, ran.End), increasedTblRowCount, false) + count, err = col.GetColumnRowCount(sctx, getRange(ran.Start, ran.End), increasedTblRowCount, modifyCount, false) require.NoError(t, err) testdata.OnRecord(func() { output[i].Start = ran.Start @@ -160,6 +161,42 @@ func TestOutOfRangeEstimation(t *testing.T) { } } +// TestOutOfRangeEstimationAfterDelete tests the out-of-range estimation after deletion happen. +// The test result doesn't perfectly reflect the actual data distribution, but this is the expected behavior for now. +func TestOutOfRangeEstimationAfterDelete(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int unsigned)") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + for i := 0; i < 3000; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900) + } + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t with 1 samplerate, 0 topn") + testKit.MustExec("delete from t where a < 500") + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, h.Update(dom.InfoSchema())) + var ( + input []string + output []struct { + SQL string + Result []string + } + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.LoadTestCases(t, &input, &output) + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } +} + func TestEstimationForUnknownValues(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -544,6 +581,7 @@ func TestSelectivity(t *testing.T) { require.Truef(t, math.Abs(ratio-tt.selectivity) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio) histColl.Count *= 10 + histColl.ModifyCount = histColl.Count * 9 ratio, _, err = histColl.Selectivity(sctx, sel.Conditions, nil) require.NoErrorf(t, err, "for %s", tt.exprs) require.Truef(t, math.Abs(ratio-tt.selectivityAfterIncrease) < eps, "for %s, needed: %v, got: %v", tt.exprs, tt.selectivityAfterIncrease, ratio) @@ -749,7 +787,7 @@ func TestSmallRangeEstimation(t *testing.T) { statsSuiteData := statistics.GetStatsSuiteData() statsSuiteData.LoadTestCases(t, &input, &output) for i, ran := range input { - count, err := col.GetColumnRowCount(sctx, getRange(ran.Start, ran.End), statsTbl.Count, false) + count, err := col.GetColumnRowCount(sctx, getRange(ran.Start, ran.End), statsTbl.Count, statsTbl.ModifyCount, false) require.NoError(t, err) testdata.OnRecord(func() { output[i].Start = ran.Start @@ -1071,5 +1109,51 @@ func TestIssue39593(t *testing.T) { count, err = statsTbl.GetRowCountByIndexRanges(sctx, idxID, getRanges(vals, vals)) require.NoError(t, err) // estimated row count after mock modify on the table - require.Equal(t, float64(3870), count) + require.Equal(t, float64(3600), count) +} + +func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() + testKit.MustExec("use test") + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int unsigned) " + + "partition by range (a) " + + "(partition p0 values less than (400), " + + "partition p1 values less than (600), " + + "partition p2 values less than (800)," + + "partition p3 values less than (1000)," + + "partition p4 values less than (1200))") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + for i := 0; i < 3000; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900) + } + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + testKit.MustExec("analyze table t with 1 samplerate, 0 topn") + testKit.MustExec("delete from t where a < 500") + require.Nil(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, h.Update(dom.InfoSchema())) + var ( + input []string + output []struct { + SQL string + Result []string + } + ) + statsSuiteData := statistics.GetStatsSuiteData() + statsSuiteData.LoadTestCases(t, &input, &output) + for i := range input { + testdata.OnRecord(func() { + output[i].SQL = input[i] + output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } + testKit.MustExec("analyze table t partition p4 with 1 samplerate, 0 topn") + require.Nil(t, h.Update(dom.InfoSchema())) + for i := range input { + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) + } } diff --git a/statistics/table.go b/statistics/table.go index 20bc5d38b0a5e..f22699cfdb95b 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -566,7 +566,7 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sctx sessionctx.Context, colI } return result, nil } - result, err = c.GetColumnRowCount(sctx, intRanges, coll.Count, true) + result, err = c.GetColumnRowCount(sctx, intRanges, coll.Count, coll.ModifyCount, true) if sc.EnableOptimizerCETrace { CETraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats", uint64(result)) } @@ -587,7 +587,7 @@ func (coll *HistColl) GetRowCountByColumnRanges(sctx sessionctx.Context, colID i } return result, err } - result, err := c.GetColumnRowCount(sctx, colRanges, coll.Count, false) + result, err := c.GetColumnRowCount(sctx, colRanges, coll.Count, coll.ModifyCount, false) if sc.EnableOptimizerCETrace { CETraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, colRanges, "Column Stats", uint64(result)) } @@ -623,7 +623,7 @@ func (coll *HistColl) GetRowCountByIndexRanges(sctx sessionctx.Context, idxID in if idx.CMSketch != nil && idx.StatsVer == Version1 { result, err = coll.getIndexRowCount(sctx, idxID, indexRanges) } else { - result, err = idx.GetRowCount(sctx, coll, indexRanges, coll.Count) + result, err = idx.GetRowCount(sctx, coll, indexRanges, coll.Count, coll.ModifyCount) } if sc.EnableOptimizerCETrace { CETraceRange(sctx, coll.PhysicalID, colNames, indexRanges, "Index Stats", uint64(result)) @@ -959,7 +959,7 @@ func (coll *HistColl) crossValidationSelectivity(sctx sessionctx.Context, idx *I Collators: []collate.Collator{idxPointRange.Collators[i]}, } - rowCount, err := col.GetColumnRowCount(sctx, []*ranger.Range{&rang}, coll.Count, col.IsHandle) + rowCount, err := col.GetColumnRowCount(sctx, []*ranger.Range{&rang}, coll.Count, coll.ModifyCount, col.IsHandle) if err != nil { return 0, 0, err } @@ -1031,7 +1031,7 @@ func (coll *HistColl) getIndexRowCount(sctx sessionctx.Context, idxID int64, ind // on single-column index, use previous way as well, because CMSketch does not contain null // values in this case. if rangePosition == 0 || isSingleColIdxNullRange(idx, ran) { - count, err := idx.GetRowCount(sctx, nil, []*ranger.Range{ran}, coll.Count) + count, err := idx.GetRowCount(sctx, nil, []*ranger.Range{ran}, coll.Count, coll.ModifyCount) if err != nil { return 0, errors.Trace(err) } diff --git a/statistics/testdata/stats_suite_in.json b/statistics/testdata/stats_suite_in.json index ff76b09ac4c15..bf53c6726c974 100644 --- a/statistics/testdata/stats_suite_in.json +++ b/statistics/testdata/stats_suite_in.json @@ -259,5 +259,41 @@ "End": 0 } ] + }, + { + "name": "TestOutOfRangeEstimationAfterDelete", + "cases": [ + "explain format = 'brief' select * from t where a <= 300", + "explain format = 'brief' select * from t where a < 300", + "explain format = 'brief' select * from t where a <= 500", + "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "explain format = 'brief' select * from t where a >= 900", + "explain format = 'brief' select * from t where a > 900", + "explain format = 'brief' select * from t where a >= 300", + "explain format = 'brief' select * from t where a <= 900", + "explain format = 'brief' select * from t where a > 800 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1100", + "explain format = 'brief' select * from t where a > 200 and a < 300", + "explain format = 'brief' select * from t where a > 100 and a < 300" + ] + }, + { + "name": "TestGlobalStatsOutOfRangeEstimationAfterDelete", + "cases": [ + "explain format = 'brief' select * from t where a <= 300", + "explain format = 'brief' select * from t where a < 300", + "explain format = 'brief' select * from t where a <= 500", + "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "explain format = 'brief' select * from t where a >= 900", + "explain format = 'brief' select * from t where a > 900", + "explain format = 'brief' select * from t where a >= 300", + "explain format = 'brief' select * from t where a <= 900", + "explain format = 'brief' select * from t where a > 800 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1000", + "explain format = 'brief' select * from t where a > 900 and a < 1100", + "explain format = 'brief' select * from t where a > 200 and a < 300", + "explain format = 'brief' select * from t where a > 100 and a < 300" + ] } ] diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json index a78fafe87ade9..fea01ef77bee1 100644 --- a/statistics/testdata/stats_suite_out.json +++ b/statistics/testdata/stats_suite_out.json @@ -759,5 +759,223 @@ "Count": 7.5 } ] + }, + { + "Name": "TestOutOfRangeEstimationAfterDelete", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a <= 300", + "Result": [ + "TableReader 1003.33 root data:Selection", + "└─Selection 1003.33 cop[tikv] le(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a < 300", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 500", + "Result": [ + "TableReader 1670.00 root data:Selection", + "└─Selection 1670.00 cop[tikv] le(test.t.a, 500)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300), le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 900", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] ge(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900", + "Result": [ + "TableReader 1000.00 root data:Selection", + "└─Selection 1000.00 cop[tikv] gt(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 900", + "Result": [ + "TableReader 2000.00 root data:Selection", + "└─Selection 2000.00 cop[tikv] le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 800 and a < 1000", + "Result": [ + "TableReader 793.13 root data:Selection", + "└─Selection 793.13 cop[tikv] gt(test.t.a, 800), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1000", + "Result": [ + "TableReader 458.12 root data:Selection", + "└─Selection 458.12 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1100", + "Result": [ + "TableReader 832.49 root data:Selection", + "└─Selection 832.49 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1100)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 200 and a < 300", + "Result": [ + "TableReader 458.12 root data:Selection", + "└─Selection 458.12 cop[tikv] gt(test.t.a, 200), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 100 and a < 300", + "Result": [ + "TableReader 832.49 root data:Selection", + "└─Selection 832.49 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + } + ] + }, + { + "Name": "TestGlobalStatsOutOfRangeEstimationAfterDelete", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a <= 300", + "Result": [ + "TableReader 1003.33 root partition:p0 data:Selection", + "└─Selection 1003.33 cop[tikv] le(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a < 300", + "Result": [ + "TableReader 1000.00 root partition:p0 data:Selection", + "└─Selection 1000.00 cop[tikv] lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 500", + "Result": [ + "TableReader 1670.00 root partition:p0,p1 data:Selection", + "└─Selection 1670.00 cop[tikv] le(test.t.a, 500)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300 and a <= 900", + "Result": [ + "TableReader 2000.00 root partition:p0,p1,p2,p3 data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300), le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 900", + "Result": [ + "TableReader 1000.00 root partition:p3,p4 data:Selection", + "└─Selection 1000.00 cop[tikv] ge(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900", + "Result": [ + "TableReader 1000.00 root partition:p3,p4 data:Selection", + "└─Selection 1000.00 cop[tikv] gt(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a >= 300", + "Result": [ + "TableReader 2000.00 root partition:all data:Selection", + "└─Selection 2000.00 cop[tikv] ge(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a <= 900", + "Result": [ + "TableReader 2000.00 root partition:p0,p1,p2,p3 data:Selection", + "└─Selection 2000.00 cop[tikv] le(test.t.a, 900)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 800 and a < 1000", + "Result": [ + "TableReader 793.20 root partition:p3 data:Selection", + "└─Selection 793.20 cop[tikv] gt(test.t.a, 800), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1000", + "Result": [ + "TableReader 458.19 root partition:p3 data:Selection", + "└─Selection 458.19 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1000)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 900 and a < 1100", + "Result": [ + "TableReader 832.77 root partition:p3,p4 data:Selection", + "└─Selection 832.77 cop[tikv] gt(test.t.a, 900), lt(test.t.a, 1100)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 200 and a < 300", + "Result": [ + "TableReader 459.03 root partition:p0 data:Selection", + "└─Selection 459.03 cop[tikv] gt(test.t.a, 200), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a > 100 and a < 300", + "Result": [ + "TableReader 834.45 root partition:p0 data:Selection", + "└─Selection 834.45 cop[tikv] gt(test.t.a, 100), lt(test.t.a, 300)", + " └─TableFullScan 2000.00 cop[tikv] table:t keep order:false" + ] + } + ] } ]