From 0494095c78012dfd7f2057ce93a9cb3bf58c9019 Mon Sep 17 00:00:00 2001 From: Yiding Date: Fri, 29 Nov 2024 00:07:19 +0800 Subject: [PATCH] fix tests --- pkg/statistics/handle/bootstrap.go | 8 ++-- .../handle/handletest/statstest/stats_test.go | 3 ++ pkg/statistics/handle/syncload/BUILD.bazel | 1 - .../handle/syncload/stats_syncload.go | 46 +++++++++---------- .../integrationtest/r/executor/issues.result | 30 ++++++------ .../r/executor/partition/issues.result | 14 +++--- .../r/explain_complex_stats.result | 22 ++++----- .../r/explain_easy_stats.result | 10 ++-- .../explain_generate_column_substitute.result | 8 ++-- .../r/globalindex/analyze.result | 12 ++--- 10 files changed, 78 insertions(+), 76 deletions(-) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 42948261e2077..64ee703b395ff 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -322,7 +322,7 @@ func (h *Handle) initStatsHistogramsByPaging(is infoschema.InfoSchema, cache sta // Why do we need to add `is_index=1` in the SQL? // because it is aligned to the `initStatsTopN` function, which only loads the topn of the index too. // the other will be loaded by sync load. - sql := "select /*+ ORDER_INDEX(mysql.stats_histograms,tbl)*/ HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms where table_id >= %? and table_id < %? order by table_id" + sql := "select /*+ ORDER_INDEX(mysql.stats_histograms,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms where table_id >= %? and table_id < %? order by table_id" rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err) @@ -435,7 +435,7 @@ func (h *Handle) initStatsTopNByPaging(cache statstypes.StatsCache, task initsta } }() sctx := se.(sessionctx.Context) - sql := "select HIGH_PRIORITY table_id, hist_id, value, count from mysql.stats_top_n where is_index = 1 and table_id >= %? and table_id < %? order by table_id" + sql := "select HIGH_PRIORITY table_id, hist_id, value, count from mysql.stats_top_n where is_index = 1 and table_id >= %? and table_id < %? and is_index=1 order by table_id" rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err) @@ -629,7 +629,7 @@ func (h *Handle) initStatsBuckets(cache statstypes.StatsCache, totalMemory uint6 return errors.Trace(err) } } else { - sql := "select /*+ ORDER_INDEX(mysql.stats_buckets,tbl)*/ HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets order by table_id, is_index, hist_id, bucket_id" + sql := "select /*+ ORDER_INDEX(mysql.stats_buckets,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where is_index=1 order by table_id, is_index, hist_id, bucket_id" rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return errors.Trace(err) @@ -668,7 +668,7 @@ func (h *Handle) initStatsBucketsByPaging(cache statstypes.StatsCache, task init } }() sctx := se.(sessionctx.Context) - sql := "select HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where table_id >= %? and table_id < %? order by table_id, is_index, hist_id, bucket_id" + sql := "select HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets where table_id >= %? and table_id < %? and is_index=1 order by table_id, is_index, hist_id, bucket_id" rc, err := util.Exec(sctx, sql, task.StartTid, task.EndTid) if err != nil { return errors.Trace(err) diff --git a/pkg/statistics/handle/handletest/statstest/stats_test.go b/pkg/statistics/handle/handletest/statstest/stats_test.go index 5167c8e706604..5aac34e5bd856 100644 --- a/pkg/statistics/handle/handletest/statstest/stats_test.go +++ b/pkg/statistics/handle/handletest/statstest/stats_test.go @@ -273,6 +273,9 @@ func TestInitStats(t *testing.T) { require.NoError(t, h.Update(context.Background(), is)) // Index and pk are loaded. needed := fmt.Sprintf(`Table:%v RealtimeCount:6 +column:1 ndv:6 totColSize:0 +column:2 ndv:6 totColSize:6 +column:3 ndv:6 totColSize:6 index:1 ndv:6 num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0 num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0 diff --git a/pkg/statistics/handle/syncload/BUILD.bazel b/pkg/statistics/handle/syncload/BUILD.bazel index 21d6048beaaa1..4275311ac4579 100644 --- a/pkg/statistics/handle/syncload/BUILD.bazel +++ b/pkg/statistics/handle/syncload/BUILD.bazel @@ -18,7 +18,6 @@ go_library( "//pkg/statistics", "//pkg/statistics/handle/storage", "//pkg/statistics/handle/types", - "//pkg/table", "//pkg/types", "//pkg/util", "//pkg/util/intest", diff --git a/pkg/statistics/handle/syncload/stats_syncload.go b/pkg/statistics/handle/syncload/stats_syncload.go index 0eaebebd20844..38fb3998c5b4a 100644 --- a/pkg/statistics/handle/syncload/stats_syncload.go +++ b/pkg/statistics/handle/syncload/stats_syncload.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/storage" statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" - "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/intest" @@ -314,47 +313,48 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err } item := task.Item.TableItemID - tbl, ok := s.statsHandle.Get(item.TableID) + statsTbl, ok := s.statsHandle.Get(item.TableID) if !ok { return nil } is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) - tblInfo, ok := s.statsHandle.TableInfoByID(is, item.TableID) + tbl, ok := s.statsHandle.TableInfoByID(is, item.TableID) if !ok { return nil } - isPkIsHandle := tblInfo.Meta().PKIsHandle - if !tbl.ColAndIdxExistenceMap.Checked() { - tbl = tbl.Copy() - for _, col := range tbl.HistColl.GetColSlice() { - if tblInfo.Meta().FindColumnByID(col.ID) == nil { - tbl.HistColl.DelCol(col.ID) - tbl.ColAndIdxExistenceMap.DeleteColAnalyzed(col.ID) + tblInfo := tbl.Meta() + isPkIsHandle := tblInfo.PKIsHandle + if !statsTbl.ColAndIdxExistenceMap.Checked() { + statsTbl = statsTbl.Copy() + for _, col := range statsTbl.HistColl.GetColSlice() { + if tblInfo.FindColumnByID(col.ID) == nil { + statsTbl.HistColl.DelCol(col.ID) + statsTbl.ColAndIdxExistenceMap.DeleteColAnalyzed(col.ID) } } - for _, idx := range tbl.HistColl.GetIdxSlice() { - if tblInfo.Meta().FindIndexByID(idx.ID) == nil { - tbl.HistColl.DelIdx(idx.ID) - tbl.ColAndIdxExistenceMap.DeleteIdxAnalyzed(idx.ID) + for _, idx := range statsTbl.HistColl.GetIdxSlice() { + if tblInfo.FindIndexByID(idx.ID) == nil { + statsTbl.HistColl.DelIdx(idx.ID) + statsTbl.ColAndIdxExistenceMap.DeleteIdxAnalyzed(idx.ID) } } - tbl.ColAndIdxExistenceMap.SetChecked() - s.statsHandle.UpdateStatsCache([]*statistics.Table{tbl}, nil) + statsTbl.ColAndIdxExistenceMap.SetChecked() + s.statsHandle.UpdateStatsCache([]*statistics.Table{statsTbl}, nil) } wrapper := &statsWrapper{} if item.IsIndex { - index, loadNeeded := tbl.IndexIsLoadNeeded(item.ID) + index, loadNeeded := statsTbl.IndexIsLoadNeeded(item.ID) if !loadNeeded { return nil } if index != nil { wrapper.idxInfo = index.Info } else { - wrapper.idxInfo = tblInfo.Meta().FindIndexByID(item.ID) + wrapper.idxInfo = tblInfo.FindIndexByID(item.ID) } } else { - col, loadNeeded, analyzed := tbl.ColumnIsLoadNeeded(item.ID, task.Item.FullLoad) + col, loadNeeded, analyzed := statsTbl.ColumnIsLoadNeeded(item.ID, task.Item.FullLoad) if !loadNeeded { return nil } @@ -363,7 +363,7 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err } else { // Now, we cannot init the column info in the ColAndIdxExistenceMap when to disable lite-init-stats. // so we have to get the column info from the domain. - wrapper.colInfo = tblInfo.Meta().GetColumnByID(item.ID) + wrapper.colInfo = tblInfo.GetColumnByID(item.ID) } if skipTypes != nil { _, skip := skipTypes[types.TypeToStr(wrapper.colInfo.FieldType.GetType(), wrapper.colInfo.FieldType.GetCharset())] @@ -377,7 +377,7 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err // Otherwise, it will trigger the sync/async load again, even if the column has not been analyzed. if loadNeeded && !analyzed { wrapper.col = statistics.EmptyColumn(item.TableID, isPkIsHandle, wrapper.colInfo) - s.updateCachedItem(tblInfo, item, wrapper.col, wrapper.idx, task.Item.FullLoad) + s.updateCachedItem(item, wrapper.col, wrapper.idx, task.Item.FullLoad) return nil } } @@ -402,7 +402,7 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err } metrics.ReadStatsHistogram.Observe(float64(time.Since(t).Milliseconds())) if needUpdate { - s.updateCachedItem(tblInfo, item, wrapper.col, wrapper.idx, task.Item.FullLoad) + s.updateCachedItem(item, wrapper.col, wrapper.idx, task.Item.FullLoad) } return nil } @@ -578,7 +578,7 @@ func (*statsSyncLoad) writeToResultChan(resultCh chan stmtctx.StatsLoadResult, r } // updateCachedItem updates the column/index hist to global statsCache. -func (s *statsSyncLoad) updateCachedItem(tblInfo table.Table, item model.TableItemID, colHist *statistics.Column, idxHist *statistics.Index, fullLoaded bool) (updated bool) { +func (s *statsSyncLoad) updateCachedItem(item model.TableItemID, colHist *statistics.Column, idxHist *statistics.Index, fullLoaded bool) (updated bool) { s.StatsLoad.Lock() defer s.StatsLoad.Unlock() // Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions diff --git a/tests/integrationtest/r/executor/issues.result b/tests/integrationtest/r/executor/issues.result index 90fbd3efcd6d0..d9a81a809a111 100644 --- a/tests/integrationtest/r/executor/issues.result +++ b/tests/integrationtest/r/executor/issues.result @@ -969,18 +969,18 @@ Limit_7 256.00 root NULL NULL offset:0, count:100000 └─TableFullScan_9 256.00 cop[tikv] table:pt NULL keep order:false explain analyze select * from pt where val = 125 limit 100; # expected distsql concurrency 15 id estRows actRows task access object execution info operator info memory disk -Limit_8 100.00 root NULL NULL offset:0, count:100 -└─TableReader_13 100.00 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_12 100.00 cop[tikv] NULL NULL offset:0, count:100 - └─Selection_11 100.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) - └─TableFullScan_10 125.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:missing] +Limit_8 0.26 root NULL NULL offset:0, count:100 +└─TableReader_13 0.26 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_12 0.26 cop[tikv] NULL NULL offset:0, count:100 + └─Selection_11 0.26 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) + └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:unInitialized] explain analyze select * from pt where val = 125 limit 100000; # expected distsql concurrency 15 id estRows actRows task access object execution info operator info memory disk -Limit_8 204.80 root NULL NULL offset:0, count:100000 -└─TableReader_13 204.80 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_12 204.80 cop[tikv] NULL NULL offset:0, count:100000 - └─Selection_11 204.80 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) - └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:missing] +Limit_8 0.26 root NULL NULL offset:0, count:100000 +└─TableReader_13 0.26 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_12 0.26 cop[tikv] NULL NULL offset:0, count:100000 + └─Selection_11 0.26 cop[tikv] NULL NULL eq(executor__issues.pt.val, 125) + └─TableFullScan_10 256.00 cop[tikv] table:pt NULL keep order:false, stats:partial[val:unInitialized] explain analyze select * from pt order by id limit 100; # expected distsql concurrency 7, but currently get 1, see issue #55190 id estRows actRows task access object execution info operator info memory disk Limit_10 100.00 root NULL NULL offset:0, count:100 @@ -995,11 +995,11 @@ Limit_11 256.00 root NULL NULL offset:0, count:100000 └─TableFullScan_19 256.00 cop[tikv] table:pt NULL keep order:true explain analyze select * from pt where val = 126 order by id limit 100; # expected distsql concurrency 15 id estRows actRows task access object execution info operator info memory disk -Limit_11 100.00 root NULL NULL offset:0, count:100 -└─TableReader_20 100.00 root partition:all max_distsql_concurrency: 15 NULL - └─Limit_19 100.00 cop[tikv] NULL NULL offset:0, count:100 - └─Selection_18 100.00 cop[tikv] NULL NULL eq(executor__issues.pt.val, 126) - └─TableFullScan_17 125.00 cop[tikv] table:pt NULL keep order:true, stats:partial[val:missing] +Limit_11 0.26 root NULL NULL offset:0, count:100 +└─TableReader_20 0.26 root partition:all max_distsql_concurrency: 15 NULL + └─Limit_19 0.26 cop[tikv] NULL NULL offset:0, count:100 + └─Selection_18 0.26 cop[tikv] NULL NULL eq(executor__issues.pt.val, 126) + └─TableFullScan_17 256.00 cop[tikv] table:pt NULL keep order:true, stats:partial[val:unInitialized] CREATE TABLE test_55837 (col1 int(4) NOT NULL, col2 bigint(4) NOT NULL, KEY col2_index (col2)); insert into test_55837 values(0,1725292800),(0,1725292800); select from_unixtime( if(col2 >9999999999, col2/1000, col2), '%Y-%m-%d %H:%i:%s') as result from test_55837; diff --git a/tests/integrationtest/r/executor/partition/issues.result b/tests/integrationtest/r/executor/partition/issues.result index 06bc8e5645084..e4a5d31e140f3 100644 --- a/tests/integrationtest/r/executor/partition/issues.result +++ b/tests/integrationtest/r/executor/partition/issues.result @@ -302,13 +302,13 @@ join t on c.txt_account_id = t.txn_account_id and t.broker = '0009' and c.occur_trade_date = '2022-11-17' and c.serial_id = t.serial_id; id estRows task access object operator info -IndexJoin_20 0.80 root inner join, inner:TableReader_19, outer key:executor__partition__issues.t.txn_account_id, executor__partition__issues.t.serial_id, inner key:executor__partition__issues.c.txt_account_id, executor__partition__issues.c.serial_id, equal cond:eq(executor__partition__issues.t.serial_id, executor__partition__issues.c.serial_id), eq(executor__partition__issues.t.txn_account_id, executor__partition__issues.c.txt_account_id) -├─TableReader_25(Build) 0.80 root data:Selection_24 -│ └─Selection_24 0.80 cop[tikv] eq(executor__partition__issues.t.broker, "0009"), not(isnull(executor__partition__issues.t.serial_id)) -│ └─TableFullScan_23 1.00 cop[tikv] table:t keep order:false, stats:partial[serial_id:missing] -└─TableReader_19(Probe) 0.80 root partition:all data:Selection_18 - └─Selection_18 0.80 cop[tikv] eq(executor__partition__issues.c.occur_trade_date, 2022-11-17 00:00:00.000000) - └─TableRangeScan_17 0.80 cop[tikv] table:c range: decided by [eq(executor__partition__issues.c.txt_account_id, executor__partition__issues.t.txn_account_id) eq(executor__partition__issues.c.serial_id, executor__partition__issues.t.serial_id) eq(executor__partition__issues.c.occur_trade_date, 2022-11-17 00:00:00.000000)], keep order:false +IndexJoin_20 1.00 root inner join, inner:TableReader_19, outer key:executor__partition__issues.t.txn_account_id, executor__partition__issues.t.serial_id, inner key:executor__partition__issues.c.txt_account_id, executor__partition__issues.c.serial_id, equal cond:eq(executor__partition__issues.t.serial_id, executor__partition__issues.c.serial_id), eq(executor__partition__issues.t.txn_account_id, executor__partition__issues.c.txt_account_id) +├─TableReader_25(Build) 1.00 root data:Selection_24 +│ └─Selection_24 1.00 cop[tikv] eq(executor__partition__issues.t.broker, "0009"), not(isnull(executor__partition__issues.t.serial_id)) +│ └─TableFullScan_23 1.00 cop[tikv] table:t keep order:false, stats:partial[serial_id:unInitialized] +└─TableReader_19(Probe) 1.00 root partition:all data:Selection_18 + └─Selection_18 1.00 cop[tikv] eq(executor__partition__issues.c.occur_trade_date, 2022-11-17 00:00:00.000000) + └─TableRangeScan_17 1.00 cop[tikv] table:c range: decided by [eq(executor__partition__issues.c.txt_account_id, executor__partition__issues.t.txn_account_id) eq(executor__partition__issues.c.serial_id, executor__partition__issues.t.serial_id) eq(executor__partition__issues.c.occur_trade_date, 2022-11-17 00:00:00.000000)], keep order:false set @@tidb_opt_advanced_join_hint=default; set tidb_partition_prune_mode=default; drop table if exists t; diff --git a/tests/integrationtest/r/explain_complex_stats.result b/tests/integrationtest/r/explain_complex_stats.result index 9183896d7aefb..cb1eda95f1317 100644 --- a/tests/integrationtest/r/explain_complex_stats.result +++ b/tests/integrationtest/r/explain_complex_stats.result @@ -139,10 +139,10 @@ Projection 21.47 root explain_complex_stats.dt.ds, explain_complex_stats.dt.p1, └─Sort 21.47 root explain_complex_stats.dt.ds2:desc └─HashAgg 21.47 root group by:explain_complex_stats.dt.ds, explain_complex_stats.dt.p1, explain_complex_stats.dt.p2, explain_complex_stats.dt.p3, explain_complex_stats.dt.p4, explain_complex_stats.dt.p5, explain_complex_stats.dt.p6_md5, explain_complex_stats.dt.p7_md5, funcs:count(Column#32)->Column#21, funcs:firstrow(explain_complex_stats.dt.ds)->explain_complex_stats.dt.ds, funcs:firstrow(Column#34)->explain_complex_stats.dt.ds2, funcs:firstrow(explain_complex_stats.dt.p1)->explain_complex_stats.dt.p1, funcs:firstrow(explain_complex_stats.dt.p2)->explain_complex_stats.dt.p2, funcs:firstrow(explain_complex_stats.dt.p3)->explain_complex_stats.dt.p3, funcs:firstrow(explain_complex_stats.dt.p4)->explain_complex_stats.dt.p4, funcs:firstrow(explain_complex_stats.dt.p5)->explain_complex_stats.dt.p5, funcs:firstrow(explain_complex_stats.dt.p6_md5)->explain_complex_stats.dt.p6_md5, funcs:firstrow(explain_complex_stats.dt.p7_md5)->explain_complex_stats.dt.p7_md5 └─IndexLookUp 21.47 root - ├─IndexRangeScan(Build) 128.00 cop[tikv] table:dt, index:cm(cm) range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:partial[cm:missing] + ├─IndexRangeScan(Build) 128.00 cop[tikv] table:dt, index:cm(cm) range:[1062,1062], [1086,1086], [1423,1423], [1424,1424], [1425,1425], [1426,1426], [1427,1427], [1428,1428], [1429,1429], [1430,1430], [1431,1431], [1432,1432], [1433,1433], [1434,1434], [1435,1435], [1436,1436], [1437,1437], [1438,1438], [1439,1439], [1440,1440], [1441,1441], [1442,1442], [1443,1443], [1444,1444], [1445,1445], [1446,1446], [1447,1447], [1448,1448], [1449,1449], [1450,1450], [1451,1451], [1452,1452], [1488,1488], [1489,1489], [1490,1490], [1491,1491], [1492,1492], [1493,1493], [1494,1494], [1495,1495], [1496,1496], [1497,1497], [1550,1550], [1551,1551], [1552,1552], [1553,1553], [1554,1554], [1555,1555], [1556,1556], [1557,1557], [1558,1558], [1559,1559], [1597,1597], [1598,1598], [1599,1599], [1600,1600], [1601,1601], [1602,1602], [1603,1603], [1604,1604], [1605,1605], [1606,1606], [1607,1607], [1608,1608], [1609,1609], [1610,1610], [1611,1611], [1612,1612], [1613,1613], [1614,1614], [1615,1615], [1616,1616], [1623,1623], [1624,1624], [1625,1625], [1626,1626], [1627,1627], [1628,1628], [1629,1629], [1630,1630], [1631,1631], [1632,1632], [1709,1709], [1719,1719], [1720,1720], [1843,1843], [2813,2813], [2814,2814], [2815,2815], [2816,2816], [2817,2817], [2818,2818], [2819,2819], [2820,2820], [2821,2821], [2822,2822], [2823,2823], [2824,2824], [2825,2825], [2826,2826], [2827,2827], [2828,2828], [2829,2829], [2830,2830], [2831,2831], [2832,2832], [2833,2833], [2834,2834], [2835,2835], [2836,2836], [2837,2837], [2838,2838], [2839,2839], [2840,2840], [2841,2841], [2842,2842], [2843,2843], [2844,2844], [2845,2845], [2846,2846], [2847,2847], [2848,2848], [2849,2849], [2850,2850], [2851,2851], [2852,2852], [2853,2853], [2854,2854], [2855,2855], [2856,2856], [2857,2857], [2858,2858], [2859,2859], [2860,2860], [2861,2861], [2862,2862], [2863,2863], [2864,2864], [2865,2865], [2866,2866], [2867,2867], [2868,2868], [2869,2869], [2870,2870], [2871,2871], [2872,2872], [3139,3139], [3140,3140], [3141,3141], [3142,3142], [3143,3143], [3144,3144], [3145,3145], [3146,3146], [3147,3147], [3148,3148], [3149,3149], [3150,3150], [3151,3151], [3152,3152], [3153,3153], [3154,3154], [3155,3155], [3156,3156], [3157,3157], [3158,3158], [3386,3386], [3387,3387], [3388,3388], [3389,3389], [3390,3390], [3391,3391], [3392,3392], [3393,3393], [3394,3394], [3395,3395], [3664,3664], [3665,3665], [3666,3666], [3667,3667], [3668,3668], [3670,3670], [3671,3671], [3672,3672], [3673,3673], [3674,3674], [3676,3676], [3677,3677], [3678,3678], [3679,3679], [3680,3680], [3681,3681], [3682,3682], [3683,3683], [3684,3684], [3685,3685], [3686,3686], [3687,3687], [3688,3688], [3689,3689], [3690,3690], [3691,3691], [3692,3692], [3693,3693], [3694,3694], [3695,3695], [3696,3696], [3697,3697], [3698,3698], [3699,3699], [3700,3700], [3701,3701], [3702,3702], [3703,3703], [3704,3704], [3705,3705], [3706,3706], [3707,3707], [3708,3708], [3709,3709], [3710,3710], [3711,3711], [3712,3712], [3713,3713], [3714,3714], [3715,3715], [3960,3960], [3961,3961], [3962,3962], [3963,3963], [3964,3964], [3965,3965], [3966,3966], [3967,3967], [3968,3968], [3978,3978], [3979,3979], [3980,3980], [3981,3981], [3982,3982], [3983,3983], [3984,3984], [3985,3985], [3986,3986], [3987,3987], [4208,4208], [4209,4209], [4210,4210], [4211,4211], [4212,4212], [4304,4304], [4305,4305], [4306,4306], [4307,4307], [4308,4308], [4866,4866], [4867,4867], [4868,4868], [4869,4869], [4870,4870], [4871,4871], [4872,4872], [4873,4873], [4874,4874], [4875,4875], keep order:false, stats:partial[cm:unInitialized] └─HashAgg(Probe) 21.47 cop[tikv] group by:explain_complex_stats.dt.ds, explain_complex_stats.dt.p1, explain_complex_stats.dt.p2, explain_complex_stats.dt.p3, explain_complex_stats.dt.p4, explain_complex_stats.dt.p5, explain_complex_stats.dt.p6_md5, explain_complex_stats.dt.p7_md5, funcs:count(explain_complex_stats.dt.dic)->Column#32, funcs:firstrow(explain_complex_stats.dt.ds2)->Column#34 └─Selection 21.50 cop[tikv] ge(explain_complex_stats.dt.ds, 2016-09-01 00:00:00.000000), le(explain_complex_stats.dt.ds, 2016-11-03 00:00:00.000000) - └─TableRowIDScan 128.00 cop[tikv] table:dt keep order:false, stats:partial[cm:missing] + └─TableRowIDScan 128.00 cop[tikv] table:dt keep order:false, stats:partial[cm:unInitialized] explain format = 'brief' select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.bm = 0 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id estRows task access object operator info Projection 424.00 root explain_complex_stats.st.id, explain_complex_stats.dd.id, explain_complex_stats.st.aid, explain_complex_stats.st.cm, explain_complex_stats.dd.dic, explain_complex_stats.dd.ip, explain_complex_stats.dd.t, explain_complex_stats.st.p1, explain_complex_stats.st.p2, explain_complex_stats.st.p3, explain_complex_stats.st.p4, explain_complex_stats.st.p5, explain_complex_stats.st.p6_md5, explain_complex_stats.st.p7_md5, explain_complex_stats.st.ext, explain_complex_stats.st.t @@ -151,10 +151,10 @@ Projection 424.00 root explain_complex_stats.st.id, explain_complex_stats.dd.id └─HashJoin 424.00 root inner join, equal:[eq(explain_complex_stats.st.aid, explain_complex_stats.dd.aid) eq(explain_complex_stats.st.ip, explain_complex_stats.dd.ip)], other cond:gt(explain_complex_stats.dd.t, explain_complex_stats.st.t) ├─TableReader(Build) 424.00 root data:Selection │ └─Selection 424.00 cop[tikv] eq(explain_complex_stats.st.bm, 0), eq(explain_complex_stats.st.pt, "android"), gt(explain_complex_stats.st.t, 1478143908), not(isnull(explain_complex_stats.st.ip)) - │ └─TableFullScan 1999.00 cop[tikv] table:gad keep order:false, stats:partial[t:missing] + │ └─TableFullScan 1999.00 cop[tikv] table:gad keep order:false, stats:partial[t:unInitialized] └─TableReader(Probe) 450.56 root data:Selection └─Selection 450.56 cop[tikv] eq(explain_complex_stats.dd.bm, 0), eq(explain_complex_stats.dd.pt, "android"), gt(explain_complex_stats.dd.t, 1478143908), not(isnull(explain_complex_stats.dd.ip)), not(isnull(explain_complex_stats.dd.t)) - └─TableFullScan 2000.00 cop[tikv] table:dd keep order:false, stats:partial[ip:missing, t:missing] + └─TableFullScan 2000.00 cop[tikv] table:dd keep order:false, stats:partial[ip:unInitialized, t:unInitialized] explain format = 'brief' select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; id estRows task access object operator info Projection 170.34 root explain_complex_stats.st.id, explain_complex_stats.dd.id, explain_complex_stats.st.aid, explain_complex_stats.st.cm, explain_complex_stats.dd.dic, explain_complex_stats.dd.ip, explain_complex_stats.dd.t, explain_complex_stats.st.p1, explain_complex_stats.st.p2, explain_complex_stats.st.p3, explain_complex_stats.st.p4, explain_complex_stats.st.p5, explain_complex_stats.st.p6_md5, explain_complex_stats.st.p7_md5, explain_complex_stats.st.ext @@ -162,19 +162,19 @@ Projection 170.34 root explain_complex_stats.st.id, explain_complex_stats.dd.id └─IndexJoin 170.34 root inner join, inner:IndexLookUp, outer key:explain_complex_stats.st.aid, inner key:explain_complex_stats.dd.aid, equal cond:eq(explain_complex_stats.st.aid, explain_complex_stats.dd.aid), eq(explain_complex_stats.st.dic, explain_complex_stats.dd.mac), other cond:lt(explain_complex_stats.st.t, explain_complex_stats.dd.t) ├─TableReader(Build) 170.34 root data:Selection │ └─Selection 170.34 cop[tikv] eq(explain_complex_stats.st.bm, 0), eq(explain_complex_stats.st.dit, "mac"), eq(explain_complex_stats.st.pt, "ios"), gt(explain_complex_stats.st.t, 1477971479), not(isnull(explain_complex_stats.st.dic)) - │ └─TableFullScan 1999.00 cop[tikv] table:gad keep order:false, stats:partial[t:missing] + │ └─TableFullScan 1999.00 cop[tikv] table:gad keep order:false, stats:partial[t:unInitialized] └─IndexLookUp(Probe) 170.34 root - ├─IndexRangeScan(Build) 669.25 cop[tikv] table:sdk, index:aid(aid, dic) range: decided by [eq(explain_complex_stats.dd.aid, explain_complex_stats.st.aid)], keep order:false, stats:partial[t:missing] + ├─IndexRangeScan(Build) 669.25 cop[tikv] table:sdk, index:aid(aid, dic) range: decided by [eq(explain_complex_stats.dd.aid, explain_complex_stats.st.aid)], keep order:false, stats:partial[t:unInitialized] └─Selection(Probe) 170.34 cop[tikv] eq(explain_complex_stats.dd.bm, 0), eq(explain_complex_stats.dd.pt, "ios"), gt(explain_complex_stats.dd.t, 1477971479), not(isnull(explain_complex_stats.dd.mac)), not(isnull(explain_complex_stats.dd.t)) - └─TableRowIDScan 669.25 cop[tikv] table:sdk keep order:false, stats:partial[t:missing] + └─TableRowIDScan 669.25 cop[tikv] table:sdk keep order:false, stats:partial[t:unInitialized] explain format = 'brief' SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id estRows task access object operator info Projection 39.28 root explain_complex_stats.st.cm, explain_complex_stats.st.p1, explain_complex_stats.st.p2, explain_complex_stats.st.p3, explain_complex_stats.st.p4, explain_complex_stats.st.p5, explain_complex_stats.st.p6_md5, explain_complex_stats.st.p7_md5, Column#20, Column#21 └─HashAgg 39.28 root group by:explain_complex_stats.st.cm, explain_complex_stats.st.p1, explain_complex_stats.st.p2, explain_complex_stats.st.p3, explain_complex_stats.st.p4, explain_complex_stats.st.p5, explain_complex_stats.st.p6_md5, explain_complex_stats.st.p7_md5, funcs:count(1)->Column#20, funcs:count(distinct explain_complex_stats.st.ip)->Column#21, funcs:firstrow(explain_complex_stats.st.cm)->explain_complex_stats.st.cm, funcs:firstrow(explain_complex_stats.st.p1)->explain_complex_stats.st.p1, funcs:firstrow(explain_complex_stats.st.p2)->explain_complex_stats.st.p2, funcs:firstrow(explain_complex_stats.st.p3)->explain_complex_stats.st.p3, funcs:firstrow(explain_complex_stats.st.p4)->explain_complex_stats.st.p4, funcs:firstrow(explain_complex_stats.st.p5)->explain_complex_stats.st.p5, funcs:firstrow(explain_complex_stats.st.p6_md5)->explain_complex_stats.st.p6_md5, funcs:firstrow(explain_complex_stats.st.p7_md5)->explain_complex_stats.st.p7_md5 └─IndexLookUp 39.38 root - ├─IndexRangeScan(Build) 160.23 cop[tikv] table:st, index:t(t) range:[1478188800,1478275200], keep order:false, stats:partial[t:missing] + ├─IndexRangeScan(Build) 160.23 cop[tikv] table:st, index:t(t) range:[1478188800,1478275200], keep order:false, stats:partial[t:unInitialized] └─Selection(Probe) 39.38 cop[tikv] eq(explain_complex_stats.st.aid, "cn.sbkcq"), eq(explain_complex_stats.st.pt, "android") - └─TableRowIDScan 160.23 cop[tikv] table:st keep order:false, stats:partial[t:missing] + └─TableRowIDScan 160.23 cop[tikv] table:st keep order:false, stats:partial[t:unInitialized] explain format = 'brief' 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 estRows task access object operator info Projection 428.32 root explain_complex_stats.dt.id, explain_complex_stats.dt.aid, explain_complex_stats.dt.pt, explain_complex_stats.dt.dic, explain_complex_stats.dt.cm, explain_complex_stats.rr.gid, explain_complex_stats.rr.acd, explain_complex_stats.rr.t, explain_complex_stats.dt.p1, explain_complex_stats.dt.p2, explain_complex_stats.dt.p3, explain_complex_stats.dt.p4, explain_complex_stats.dt.p5, explain_complex_stats.dt.p6_md5, explain_complex_stats.dt.p7_md5 @@ -192,9 +192,9 @@ id estRows task access object operator info Projection 207.02 root explain_complex_stats.pp.pc, explain_complex_stats.pp.cr, Column#22, Column#23, Column#24 └─HashAgg 207.02 root group by:explain_complex_stats.pp.cr, explain_complex_stats.pp.pc, funcs:count(distinct explain_complex_stats.pp.uid)->Column#22, funcs:count(explain_complex_stats.pp.oid)->Column#23, funcs:sum(explain_complex_stats.pp.am)->Column#24, funcs:firstrow(explain_complex_stats.pp.pc)->explain_complex_stats.pp.pc, funcs:firstrow(explain_complex_stats.pp.cr)->explain_complex_stats.pp.cr └─IndexLookUp 207.02 root - ├─IndexRangeScan(Build) 627.00 cop[tikv] table:pp, index:ps(ps) range:[2,2], keep order:false, stats:partial[uid:missing, ppt:missing, ps:missing] + ├─IndexRangeScan(Build) 627.00 cop[tikv] table:pp, index:ps(ps) range:[2,2], keep order:false, stats:partial[uid:unInitialized, ppt:unInitialized, ps:unInitialized] └─Selection(Probe) 207.02 cop[tikv] ge(explain_complex_stats.pp.ppt, 1478188800), in(explain_complex_stats.pp.pi, 510017, 520017), in(explain_complex_stats.pp.uid, 18089709, 18090780), lt(explain_complex_stats.pp.ppt, 1478275200) - └─TableRowIDScan 627.00 cop[tikv] table:pp keep order:false, stats:partial[uid:missing, ppt:missing, ps:missing] + └─TableRowIDScan 627.00 cop[tikv] table:pp keep order:false, stats:partial[uid:unInitialized, ppt:unInitialized, ps:unInitialized] drop table if exists tbl_001; CREATE TABLE tbl_001 (a int, b int); load stats 's/explain_complex_stats_tbl_001.json'; diff --git a/tests/integrationtest/r/explain_easy_stats.result b/tests/integrationtest/r/explain_easy_stats.result index cbe78c65adc7d..65f8d3202a56d 100644 --- a/tests/integrationtest/r/explain_easy_stats.result +++ b/tests/integrationtest/r/explain_easy_stats.result @@ -44,13 +44,13 @@ TableReader 1999.00 root data:TableRangeScan explain format = 'brief' select t1.c1, t1.c2 from t1 where t1.c2 = 1; id estRows task access object operator info IndexReader 0.00 root index:IndexRangeScan -└─IndexRangeScan 0.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:partial[c2:missing] +└─IndexRangeScan 0.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:partial[c2:unInitialized] explain format = 'brief' select * from t1 left join t2 on t1.c2 = t2.c1 where t1.c1 > 1; id estRows task access object operator info HashJoin 2481.25 root left outer join, equal:[eq(explain_easy_stats.t1.c2, explain_easy_stats.t2.c1)] ├─TableReader(Build) 1985.00 root data:Selection │ └─Selection 1985.00 cop[tikv] not(isnull(explain_easy_stats.t2.c1)) -│ └─TableFullScan 1985.00 cop[tikv] table:t2 keep order:false, stats:partial[c1:missing] +│ └─TableFullScan 1985.00 cop[tikv] table:t2 keep order:false, stats:partial[c1:unInitialized] └─TableReader(Probe) 1998.00 root data:TableRangeScan └─TableRangeScan 1998.00 cop[tikv] table:t1 range:(1,+inf], keep order:false explain format = 'brief' update t1 set t1.c2 = 2 where t1.c1 = 1; @@ -61,7 +61,7 @@ explain format = 'brief' delete from t1 where t1.c2 = 1; id estRows task access object operator info Delete N/A root N/A └─IndexReader 0.00 root index:IndexRangeScan - └─IndexRangeScan 0.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:partial[c2:missing] + └─IndexRangeScan 0.00 cop[tikv] table:t1, index:c2(c2) range:[1,1], keep order:false, stats:partial[c2:unInitialized] explain format = 'brief' select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id estRows task access object operator info Projection 1985.00 root Column#7 @@ -81,9 +81,9 @@ TopN 1.00 root explain_easy_stats.t2.c2, offset:0, count:1 explain format = 'brief' select * from t1 where c1 > 1 and c2 = 1 and c3 < 1; id estRows task access object operator info IndexLookUp 0.00 root -├─IndexRangeScan(Build) 0.00 cop[tikv] table:t1, index:c2(c2) range:(1 1,1 +inf], keep order:false, stats:partial[c2:missing] +├─IndexRangeScan(Build) 0.00 cop[tikv] table:t1, index:c2(c2) range:(1 1,1 +inf], keep order:false, stats:partial[c2:unInitialized] └─Selection(Probe) 0.00 cop[tikv] lt(explain_easy_stats.t1.c3, 1) - └─TableRowIDScan 0.00 cop[tikv] table:t1 keep order:false, stats:partial[c2:missing] + └─TableRowIDScan 0.00 cop[tikv] table:t1 keep order:false, stats:partial[c2:unInitialized] explain format = 'brief' select * from t1 where c1 = 1 and c2 > 1; id estRows task access object operator info Selection 0.50 root gt(explain_easy_stats.t1.c2, 1) diff --git a/tests/integrationtest/r/explain_generate_column_substitute.result b/tests/integrationtest/r/explain_generate_column_substitute.result index 08cc423925984..b5a95d9c653bf 100644 --- a/tests/integrationtest/r/explain_generate_column_substitute.result +++ b/tests/integrationtest/r/explain_generate_column_substitute.result @@ -415,14 +415,14 @@ desc format = 'brief' select * from t where (lower(b) = "a" and a+1 = 2) or (low id estRows task access object operator info Projection 1.00 root explain_generate_column_substitute.t.a, explain_generate_column_substitute.t.b └─IndexLookUp 1.00 root - ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:["a" 2,"a" 2], ["b" 5,"b" 5], keep order:false - └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false + ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:["a" 2,"a" 2], ["b" 5,"b" 5], keep order:false, stats:partial[_v$_expression_index_0:unInitialized, _v$_expression_index_1:unInitialized] + └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:partial[_v$_expression_index_0:unInitialized, _v$_expression_index_1:unInitialized] desc format = 'brief' select * from t where not (lower(b) >= "a"); id estRows task access object operator info Projection 1.00 root explain_generate_column_substitute.t.a, explain_generate_column_substitute.t.b └─IndexLookUp 1.00 root - ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:[-inf,"a"), keep order:false - └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false + ├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:expression_index(lower(`b`), `a` + 1) range:[-inf,"a"), keep order:false, stats:partial[_v$_expression_index_0:unInitialized] + └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t keep order:false, stats:partial[_v$_expression_index_0:unInitialized] desc format = 'brief' select count(upper(b)) from t group by upper(b); id estRows task access object operator info StreamAgg 4.80 root group by:upper(explain_generate_column_substitute.t.b), funcs:count(upper(explain_generate_column_substitute.t.b))->Column#7 diff --git a/tests/integrationtest/r/globalindex/analyze.result b/tests/integrationtest/r/globalindex/analyze.result index a441d6de0a884..f6a6c4fc79883 100644 --- a/tests/integrationtest/r/globalindex/analyze.result +++ b/tests/integrationtest/r/globalindex/analyze.result @@ -21,11 +21,11 @@ analyze table t; explain select c+1 from t where (c+1)>3; id estRows task access object operator info IndexReader_7 3.00 root partition:all index:IndexRangeScan_5 -└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx(`c` + 1) range:(3,+inf], keep order:false +└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx(`c` + 1) range:(3,+inf], keep order:false, stats:partial[_v$_idx_0:unInitialized] explain select c+1 from t where (c+1)>4; id estRows task access object operator info IndexReader_7 2.00 root partition:all index:IndexRangeScan_5 -└─IndexRangeScan_5 2.00 cop[tikv] table:t, index:idx(`c` + 1) range:(4,+inf], keep order:false +└─IndexRangeScan_5 2.00 cop[tikv] table:t, index:idx(`c` + 1) range:(4,+inf], keep order:false, stats:partial[_v$_idx_0:unInitialized] #Test normal global index explain select c from t where c > 2; id estRows task access object operator info @@ -43,11 +43,11 @@ analyze table t index idx2; explain select c+2 from t where (c+2)>3; id estRows task access object operator info IndexReader_7 4.00 root partition:all index:IndexRangeScan_5 -└─IndexRangeScan_5 4.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(3,+inf], keep order:false +└─IndexRangeScan_5 4.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(3,+inf], keep order:false, stats:partial[_v$_idx2_0:unInitialized] explain select c+2 from t where (c+2)>4; id estRows task access object operator info IndexReader_7 3.00 root partition:all index:IndexRangeScan_5 -└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(4,+inf], keep order:false +└─IndexRangeScan_5 3.00 cop[tikv] table:t, index:idx2(`c` + 2) range:(4,+inf], keep order:false, stats:partial[_v$_idx2_0:unInitialized] #Test return error with analyze special global index analyze table t partition p0, p1 index idx; Error 1105 (HY000): Analyze global index 'idx' can't work with analyze specified partitions @@ -118,8 +118,8 @@ explain select * from t use index(b_s) where b + 1 > 3; id estRows task access object operator info Projection_4 3.00 root globalindex__analyze.t.a, globalindex__analyze.t.b, globalindex__analyze.t.c, globalindex__analyze.t.d └─IndexLookUp_9 3.00 root partition:all - ├─IndexRangeScan_6(Build) 3.00 cop[tikv] table:t, index:b_s(`b` + 1) range:(3,+inf], keep order:false - └─TableRowIDScan_7(Probe) 3.00 cop[tikv] table:t keep order:false + ├─IndexRangeScan_6(Build) 3.00 cop[tikv] table:t, index:b_s(`b` + 1) range:(3,+inf], keep order:false, stats:partial[_v$_b_s_0:unInitialized] + └─TableRowIDScan_7(Probe) 3.00 cop[tikv] table:t keep order:false, stats:partial[_v$_b_s_0:unInitialized] explain select * from t use index(b) where b > 3; id estRows task access object operator info IndexLookUp_7 2.00 root partition:all