diff --git a/br/pkg/restore/snap_client/systable_restore_test.go b/br/pkg/restore/snap_client/systable_restore_test.go index 504b3a45cd6fa..f9308576edc17 100644 --- a/br/pkg/restore/snap_client/systable_restore_test.go +++ b/br/pkg/restore/snap_client/systable_restore_test.go @@ -116,5 +116,5 @@ func TestCheckSysTableCompatibility(t *testing.T) { // // The above variables are in the file br/pkg/restore/systable_restore.go func TestMonitorTheSystemTableIncremental(t *testing.T) { - require.Equal(t, int64(241), session.CurrentBootstrapVersion) + require.Equal(t, int64(242), session.CurrentBootstrapVersion) } diff --git a/pkg/executor/hot_regions_history_table_test.go b/pkg/executor/hot_regions_history_table_test.go index 87966916f82a8..8edefa4f1d765 100644 --- a/pkg/executor/hot_regions_history_table_test.go +++ b/pkg/executor/hot_regions_history_table_test.go @@ -187,6 +187,9 @@ func TestTiDBHotRegionsHistory(t *testing.T) { // mysql table_id = 21 ,index_id = 1, table_name = STATS_META, index_name = IDX_VER {"2019-10-10 10:10:19", "MYSQL", "STATS_META", statsMetaTidStr, "IDX_VER", "1", "3", "3", "33333", "0", "1", "READ", "99", "99", "99", "99"}, {"2019-10-10 10:10:20", "MYSQL", "STATS_META", statsMetaTidStr, "IDX_VER", "1", "4", "4", "44444", "0", "0", "WRITE", "99", "99", "99", "99"}, + // mysql table_id = 21, index_id = 3, table_name = STATS_META, index_name = INDEX_ANALYZE_VERSION + {"2019-10-10 10:10:21", "MYSQL", "STATS_META", statsMetaTidStr, "IDX_ANALYZE_VERSION", "2", "5", "5", "55555", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:22", "MYSQL", "STATS_META", statsMetaTidStr, "IDX_ANALYZE_VERSION", "2", "6", "6", "66666", "0", "0", "WRITE", "99", "99", "99", "99"}, // mysql table_id = 21 ,index_id = 2, table_name = STATS_META, index_name = TBL {"2019-10-10 10:10:21", "MYSQL", "STATS_META", statsMetaTidStr, "TBL", "2", "5", "5", "55555", "0", "1", "READ", "99", "99", "99", "99"}, {"2019-10-10 10:10:22", "MYSQL", "STATS_META", statsMetaTidStr, "TBL", "2", "6", "6", "66666", "0", "0", "WRITE", "99", "99", "99", "99"}, diff --git a/pkg/executor/show_stats.go b/pkg/executor/show_stats.go index 38237baabc8c7..23e62229b1c93 100644 --- a/pkg/executor/show_stats.go +++ b/pkg/executor/show_stats.go @@ -146,27 +146,19 @@ func (e *ShowExec) appendTableForStatsMeta(dbName, tblName, partitionName string if statsTbl.Pseudo { return } + row := make([]any, 7) + row[0] = dbName + row[1] = tblName + row[2] = partitionName + row[3] = e.versionToTime(statsTbl.Version) + row[4] = statsTbl.ModifyCount + row[5] = statsTbl.RealtimeCount if !statsTbl.IsAnalyzed() { - e.appendRow([]any{ - dbName, - tblName, - partitionName, - e.versionToTime(statsTbl.Version), - statsTbl.ModifyCount, - statsTbl.RealtimeCount, - nil, - }) + row[6] = nil } else { - e.appendRow([]any{ - dbName, - tblName, - partitionName, - e.versionToTime(statsTbl.Version), - statsTbl.ModifyCount, - statsTbl.RealtimeCount, - e.versionToTime(statsTbl.LastAnalyzeVersion), - }) + row[6] = e.versionToTime(statsTbl.LastAnalyzeVersion) } + e.appendRow(row) } func (e *ShowExec) appendTableForStatsLocked(dbName, tblName, partitionName string) { diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 2aed5ddcd5993..02c3593be8489 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -5647,7 +5647,7 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString} case ast.ShowStatsMeta: names = []string{"Db_name", "Table_name", "Partition_name", "Update_time", "Modify_count", "Row_count", "Last_analyze_time"} - ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDatetime} + ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeDatetime, mysql.TypeDatetime} case ast.ShowStatsExtended: names = []string{"Db_name", "Table_name", "Stats_name", "Column_names", "Stats_type", "Stats_val", "Last_update_version"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong} diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 68aaea4b2a3bc..7773fd2d1ce7f 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -221,12 +221,16 @@ const ( // CreateStatsMetaTable stores the meta of table statistics. CreateStatsMetaTable = `CREATE TABLE IF NOT EXISTS mysql.stats_meta ( - version BIGINT(64) UNSIGNED NOT NULL, - table_id BIGINT(64) NOT NULL, - modify_count BIGINT(64) NOT NULL DEFAULT 0, - count BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, - snapshot BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, + version BIGINT(64) UNSIGNED NOT NULL, + table_id BIGINT(64) NOT NULL, + modify_count BIGINT(64) NOT NULL DEFAULT 0, + count BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, + snapshot BIGINT(64) UNSIGNED NOT NULL DEFAULT 0, + last_analyze_version BIGINT(64) UNSIGNED DEFAULT NULL, + last_affected_ddl_version BIGINT(64) UNSIGNED DEFAULT NULL, INDEX idx_ver(version), + INDEX idx_analyze_version(last_analyze_version), + INDEX idx_last_affected_ddl_version(last_affected_ddl_version), UNIQUE INDEX tbl(table_id) );` @@ -1242,11 +1246,15 @@ const ( // Add index on user field for some mysql tables. version241 = 241 + + // version 241 + // Add last_affected_ddl_version to mysql.stats_meta. + version242 = 242 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version241 +var currentBootstrapVersion int64 = version242 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -1423,6 +1431,7 @@ var ( upgradeToVer239, upgradeToVer240, upgradeToVer241, + upgradeToVer242, } ) @@ -3365,6 +3374,14 @@ func upgradeToVer241(s sessiontypes.Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.default_roles ADD INDEX i_user (user)", dbterror.ErrDupKeyName) } +func upgradeToVer242(s sessiontypes.Session, ver int64) { + if ver >= version242 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.stats_meta ADD COLUMN last_analyze_version bigint(20) unsigned DEFAULT NULL", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.stats_meta ADD COLUMN last_affected_ddl_version bigint(20) unsigned DEFAULT NULL", infoschema.ErrColumnExists) +} + // initGlobalVariableIfNotExists initialize a global variable with specific val if it does not exist. func initGlobalVariableIfNotExists(s sessiontypes.Session, name string, val any) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 4c50cd66317f5..2263dd7f0f2e5 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -67,22 +67,31 @@ func (*Handle) initStatsMeta4Chunk(cache statstypes.StatsCache, iter *chunk.Iter for row := iter.Begin(); row != iter.End(); row = iter.Next() { physicalID = row.GetInt64(1) maxPhysicalID = max(physicalID, maxPhysicalID) + // During the initialization phase, we need to initialize LastAnalyzeVersion with the snapshot, + // which ensures that we don't duplicate the auto-analyze of a particular type of table. + // When the predicate columns feature is turned on, if a table has neither predicate columns nor indexes, + // then auto-analyze will only analyze the _row_id and refresh stats_meta, + // but since we don't have any histograms or topn's created for _row_id at the moment. + // So if we don't initialize LastAnalyzeVersion with the snapshot here, + // it will stay at 0 and auto-analyze won't be able to detect that the table has been analyzed. + // But in the future, we maybe will create some records for _row_id, see: + // https://github.com/pingcap/tidb/issues/51098 newHistColl := *statistics.NewHistColl(physicalID, row.GetInt64(3), row.GetInt64(2), 4, 4) snapshot := row.GetUint64(4) + lastAnalyzeVersion, lastStatsFullUpdateVersion := snapshot, snapshot + if !row.IsNull(5) { + lastAnalyzeVersion = max(lastAnalyzeVersion, row.GetUint64(5)) + lastStatsFullUpdateVersion = max(lastStatsFullUpdateVersion, row.GetUint64(5)) + } + if !row.IsNull(6) { + lastStatsFullUpdateVersion = max(lastStatsFullUpdateVersion, row.GetUint64(6)) + } tbl := &statistics.Table{ - HistColl: newHistColl, - Version: row.GetUint64(0), - ColAndIdxExistenceMap: statistics.NewColAndIndexExistenceMapWithoutSize(), - // During the initialization phase, we need to initialize LastAnalyzeVersion with the snapshot, - // which ensures that we don't duplicate the auto-analyze of a particular type of table. - // When the predicate columns feature is turned on, if a table has neither predicate columns nor indexes, - // then auto-analyze will only analyze the _row_id and refresh stats_meta, - // but since we don't have any histograms or topn's created for _row_id at the moment. - // So if we don't initialize LastAnalyzeVersion with the snapshot here, - // it will stay at 0 and auto-analyze won't be able to detect that the table has been analyzed. - // But in the future, we maybe will create some records for _row_id, see: - // https://github.com/pingcap/tidb/issues/51098 - LastAnalyzeVersion: snapshot, + HistColl: newHistColl, + Version: row.GetUint64(0), + ColAndIdxExistenceMap: statistics.NewColAndIndexExistenceMapWithoutSize(), + LastAnalyzeVersion: lastAnalyzeVersion, + LastStatsUpdateVersion: lastStatsFullUpdateVersion, } cache.Put(physicalID, tbl) // put this table again since it is updated } @@ -95,7 +104,7 @@ func (*Handle) initStatsMeta4Chunk(cache statstypes.StatsCache, iter *chunk.Iter func (h *Handle) initStatsMeta(ctx context.Context) (statstypes.StatsCache, error) { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnStats) - sql := "select HIGH_PRIORITY version, table_id, modify_count, count, snapshot from mysql.stats_meta" + sql := "select HIGH_PRIORITY version, table_id, modify_count, count, snapshot, last_analyze_version, last_affected_ddl_version from mysql.stats_meta" rc, err := util.Exec(h.initStatsCtx, sql) if err != nil { return nil, errors.Trace(err) @@ -146,17 +155,14 @@ func (*Handle) initStatsHistograms4ChunkLite(cache statstypes.StatsCache, iter * } if isIndex > 0 { table.ColAndIdxExistenceMap.InsertIndex(id, statsVer != statistics.Version0) - if statsVer != statistics.Version0 { - // The LastAnalyzeVersion is added by ALTER table so its value might be 0. - table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4)) - } } else { table.ColAndIdxExistenceMap.InsertCol(id, statsVer != statistics.Version0 || ndv > 0 || nullCount > 0) - if statsVer != statistics.Version0 { - // The LastAnalyzeVersion is added by ALTER table so its value might be 0. - table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4)) - } } + // The LastXXXVersion can be added by ALTER table so its value might be 0, so we also need to update its memory value by the column/index's. + if statsVer != statistics.Version0 { + table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, row.GetUint64(4)) + } + table.LastStatsUpdateVersion = max(table.LastStatsUpdateVersion, row.GetUint64(4)) } if table != nil { cache.Put(table.PhysicalID, table) // put this table in the cache because all statstics of the table have been read. @@ -228,12 +234,14 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats StatsVer: statsVer, PhysicalID: tblID, } + // The LastXXXVersion can be added by ALTER table so its value might be 0, so we also need to update its memory value by the column/index's. if statsVer != statistics.Version0 { // We first set the StatsLoadedStatus as AllEvicted. when completing to load bucket, we will set it as ALlLoad. index.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus() - // The LastAnalyzeVersion is added by ALTER table so its value might be 0. table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version) } + table.LastStatsUpdateVersion = max(table.LastStatsUpdateVersion, version) + table.SetIdx(idxInfo.ID, index) table.ColAndIdxExistenceMap.InsertIndex(idxInfo.ID, statsVer != statistics.Version0) } else { @@ -258,8 +266,8 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats } table.SetCol(hist.ID, col) table.ColAndIdxExistenceMap.InsertCol(colInfo.ID, statsVer != statistics.Version0 || ndv > 0 || nullCount > 0) + // The LastXXXVersion can be added by ALTER table so its value might be 0, so we also need to update its memory value by the column/index's. if statsVer != statistics.Version0 { - // The LastAnalyzeVersion is added by ALTER table so its value might be 0. table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, version) // We will also set int primary key's loaded status to evicted. col.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus() @@ -269,6 +277,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats col.StatsLoadedStatus = statistics.NewStatsAllEvictedStatus() } // Otherwise the column's stats is not initialized. + table.LastStatsUpdateVersion = max(table.LastStatsUpdateVersion, version) } } if table != nil { diff --git a/pkg/statistics/handle/cache/statscache.go b/pkg/statistics/handle/cache/statscache.go index 3f36a26714b46..b4edbe884ee61 100644 --- a/pkg/statistics/handle/cache/statscache.go +++ b/pkg/statistics/handle/cache/statscache.go @@ -127,7 +127,7 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t err error ) if err := util.CallWithSCtx(s.statsHandle.SPool(), func(sctx sessionctx.Context) error { - query := "SELECT version, table_id, modify_count, count, snapshot from mysql.stats_meta where version > %? " + query := "SELECT version, table_id, modify_count, count, snapshot, last_analyze_version, last_affected_ddl_version from mysql.stats_meta where version > %? " args := []any{lastVersion} if len(tableAndPartitionIDs) > 0 { @@ -168,6 +168,13 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t modifyCount := row.GetInt64(2) count := row.GetInt64(3) snapshot := row.GetUint64(4) + var latestHistUpdateVersion uint64 + if !row.IsNull(5) { + latestHistUpdateVersion = row.GetUint64(5) + } + if !row.IsNull(6) { + latestHistUpdateVersion = max(latestHistUpdateVersion, row.GetUint64(6)) + } // Detect the context cancel signal, since it may take a long time for the loop. // TODO: add context to TableInfoByID and remove this code block? @@ -186,12 +193,21 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t } tableInfo := table.Meta() // If the table is not updated, we can skip it. - if oldTbl, ok := s.Get(physicalID); ok && - oldTbl.Version >= version && + + oldTbl, ok := s.Get(physicalID) + if ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { continue } - tbl, err := s.statsHandle.TableStatsFromStorage( + var tbl *statistics.Table + // If the column/index stats has not been updated, we can reuse the old table stats. + // Only need to update the count and modify count. + if ok && latestHistUpdateVersion > 0 && oldTbl.LastStatsUpdateVersion >= latestHistUpdateVersion { + tbl = oldTbl.Copy() + // count and modify count is updated in finalProcess + goto finalProcess + } + tbl, err = s.statsHandle.TableStatsFromStorage( tableInfo, physicalID, false, @@ -210,6 +226,7 @@ func (s *StatsCacheImpl) Update(ctx context.Context, is infoschema.InfoSchema, t tblToUpdateOrDelete.addToDelete(physicalID) continue } + finalProcess: tbl.Version = version tbl.RealtimeCount = count tbl.ModifyCount = modifyCount diff --git a/pkg/statistics/handle/storage/gc.go b/pkg/statistics/handle/storage/gc.go index 0e7889b63f33e..32b81b696e227 100644 --- a/pkg/statistics/handle/storage/gc.go +++ b/pkg/statistics/handle/storage/gc.go @@ -143,7 +143,7 @@ func DeleteTableStatsFromKV(sctx sessionctx.Context, statsIDs []int64, soft bool } for _, statsID := range statsIDs { // We only update the version so that other tidb will know that this table is deleted. - if _, err = util.Exec(sctx, "update mysql.stats_meta set version = %? where table_id = %? ", startTS, statsID); err != nil { + if _, err = util.Exec(sctx, "update mysql.stats_meta set version = %?, last_affected_ddl_version = %? where table_id = %? ", startTS, startTS, statsID); err != nil { return err } if soft { diff --git a/pkg/statistics/handle/storage/read.go b/pkg/statistics/handle/storage/read.go index c271494949d1b..e9f537a54282c 100644 --- a/pkg/statistics/handle/storage/read.go +++ b/pkg/statistics/handle/storage/read.go @@ -316,6 +316,7 @@ func indexStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *statis table.StatsVer = int(statsVer) table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, histVer) } + table.LastStatsUpdateVersion = max(table.LastStatsUpdateVersion, histVer) // We will not load buckets, topn and cmsketch if: // 1. lease > 0, and: // 2. the index doesn't have any of buckets, topn, cmsketch in memory before, and: @@ -406,6 +407,7 @@ func columnStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *stati table.StatsVer = int(statsVer) table.LastAnalyzeVersion = max(table.LastAnalyzeVersion, histVer) } + table.LastStatsUpdateVersion = max(table.LastStatsUpdateVersion, histVer) isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()) // We will not load buckets, topn and cmsketch if: // 1. lease > 0, and: diff --git a/pkg/statistics/handle/storage/save.go b/pkg/statistics/handle/storage/save.go index e339ab9addee1..7b15a9ddd8f75 100644 --- a/pkg/statistics/handle/storage/save.go +++ b/pkg/statistics/handle/storage/save.go @@ -172,11 +172,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, count = 0 } if _, err = util.Exec(sctx, - "replace into mysql.stats_meta (version, table_id, count, snapshot) values (%?, %?, %?, %?)", + "replace into mysql.stats_meta (version, table_id, count, snapshot, last_analyze_version) values (%?, %?, %?, %?, %?)", version, tableID, count, snapShot, + version, ); err != nil { return 0, err } @@ -185,7 +186,8 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, // 1-2. There's already an existing record for this table, and we are handling stats for mv index now. // In this case, we only update the version. See comments for AnalyzeResults.ForMVIndex for more details. if _, err = util.Exec(sctx, - "update mysql.stats_meta set version=%? where table_id=%?", + "update mysql.stats_meta set version=%?, last_analyze_version=%? where table_id=%?", + version, version, tableID, ); err != nil { @@ -225,11 +227,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, zap.Int64("count", cnt)) } if _, err = util.Exec(sctx, - "update mysql.stats_meta set version=%?, modify_count=%?, count=%?, snapshot=%? where table_id=%?", + "update mysql.stats_meta set version=%?, modify_count=%?, count=%?, snapshot=%?, last_analyze_version=%? where table_id=%?", version, modifyCnt, cnt, results.Snapshot, + version, tableID, ); err != nil { return 0, err @@ -410,8 +413,8 @@ func InsertColStats2KV( // First of all, we update the version. _, err = util.ExecWithCtx( ctx, sctx, - "update mysql.stats_meta set version = %? where table_id = %?", - startTS, physicalID, + "update mysql.stats_meta set version = %?, last_affected_ddl_version = %? where table_id = %?", + startTS, startTS, physicalID, ) if err != nil { return 0, errors.Trace(err) @@ -501,8 +504,8 @@ func InsertTableStats2KV( } if _, err = util.ExecWithCtx( ctx, sctx, - "insert into mysql.stats_meta (version, table_id) values(%?, %?)", - startTS, physicalID, + "insert into mysql.stats_meta (version, table_id, last_affected_ddl_version) values(%?, %?, %?)", + startTS, physicalID, startTS, ); err != nil { return 0, errors.Trace(err) } diff --git a/pkg/statistics/table.go b/pkg/statistics/table.go index 78f488bdc9494..1c8cbe5962a3a 100644 --- a/pkg/statistics/table.go +++ b/pkg/statistics/table.go @@ -72,6 +72,14 @@ type Table struct { // 1. Initialized by snapshot when loading stats_meta. // 2. Updated by the analysis time of a specific column or index when loading the histogram of the column or index. LastAnalyzeVersion uint64 + // LastStatsUpdateVersion is the mvcc version of the last full update of histograms. + // It differs from LastAnalyzeVersion because it can be influenced by some DDL. + // e.g. When we execute ALTER TABLE ADD COLUMN, there'll be new record inserted into mysql.stats_histograms. + // We need to load the corresponding one into memory too. + // It's used to skip redundant loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables, + // and the schema of the table does not change, we don't need to load the stats for this table again. + // Stats' sync load/async load should not change this field since they are not table-level update. + LastStatsUpdateVersion uint64 // TblInfoUpdateTS is the UpdateTS of the TableInfo used when filling this struct. // It is the schema version of the corresponding table. It is used to skip redundant // loading of stats, i.e, if the cached stats is already update-to-date with mysql.stats_xxx tables, @@ -605,10 +613,11 @@ func (t *Table) Copy() *Table { newHistColl.indices[id] = idx.Copy() } nt := &Table{ - HistColl: newHistColl, - Version: t.Version, - TblInfoUpdateTS: t.TblInfoUpdateTS, - LastAnalyzeVersion: t.LastAnalyzeVersion, + HistColl: newHistColl, + Version: t.Version, + TblInfoUpdateTS: t.TblInfoUpdateTS, + LastAnalyzeVersion: t.LastAnalyzeVersion, + LastStatsUpdateVersion: t.LastStatsUpdateVersion, } if t.ExtendedStats != nil { newExtStatsColl := &ExtendedStatsColl{ @@ -640,12 +649,13 @@ func (t *Table) ShallowCopy() *Table { StatsVer: t.StatsVer, } nt := &Table{ - HistColl: newHistColl, - Version: t.Version, - TblInfoUpdateTS: t.TblInfoUpdateTS, - ExtendedStats: t.ExtendedStats, - ColAndIdxExistenceMap: t.ColAndIdxExistenceMap, - LastAnalyzeVersion: t.LastAnalyzeVersion, + HistColl: newHistColl, + Version: t.Version, + TblInfoUpdateTS: t.TblInfoUpdateTS, + ExtendedStats: t.ExtendedStats, + ColAndIdxExistenceMap: t.ColAndIdxExistenceMap, + LastAnalyzeVersion: t.LastAnalyzeVersion, + LastStatsUpdateVersion: t.LastStatsUpdateVersion, } return nt }