github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/disttae/stats.go (about)

     1  // Copyright 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package disttae
    16  
    17  import (
    18  	"context"
    19  	"math"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/container/types"
    24  	"github.com/matrixorigin/matrixone/pkg/defines"
    25  	"github.com/matrixorigin/matrixone/pkg/fileservice"
    26  	"github.com/matrixorigin/matrixone/pkg/logutil"
    27  	"github.com/matrixorigin/matrixone/pkg/objectio"
    28  	"github.com/matrixorigin/matrixone/pkg/pb/gossip"
    29  	"github.com/matrixorigin/matrixone/pkg/pb/logtail"
    30  	"github.com/matrixorigin/matrixone/pkg/pb/query"
    31  	pb "github.com/matrixorigin/matrixone/pkg/pb/statsinfo"
    32  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    33  	"github.com/matrixorigin/matrixone/pkg/queryservice/client"
    34  	plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan"
    35  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    36  	"github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay"
    37  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index"
    38  )
    39  
    40  const (
    41  	// defaultLogtailUpdateStatsThreshold is the default value of threshold of receiving
    42  	// logtail entries to trigger to update stats info.
    43  	defaultLogtailUpdateStatsThreshold = 1000
    44  
    45  	// MinUpdateInterval is the minimal interval to update stats info as it
    46  	// is necessary to update stats every time.
    47  	MinUpdateInterval = time.Second * 30
    48  )
    49  
    50  type updateStatsRequest struct {
    51  	// statsInfo is the field which is to update.
    52  	statsInfo *pb.StatsInfo
    53  
    54  	// The following fields are needed to update the stats.
    55  
    56  	// tableDef is the main table definition.
    57  	tableDef *plan2.TableDef
    58  	// partitionsTableDef is the partitions table definition.
    59  	partitionsTableDef []*plan2.TableDef
    60  
    61  	partitionState  *logtailreplay.PartitionState
    62  	fs              fileservice.FileService
    63  	ts              types.TS
    64  	approxObjectNum int64
    65  }
    66  
    67  func newUpdateStatsRequest(
    68  	tableDef *plan2.TableDef,
    69  	partitionsTableDef []*plan2.TableDef,
    70  	partitionState *logtailreplay.PartitionState,
    71  	fs fileservice.FileService,
    72  	ts types.TS,
    73  	approxObjectNum int64,
    74  	stats *pb.StatsInfo,
    75  ) *updateStatsRequest {
    76  	return &updateStatsRequest{
    77  		statsInfo:          stats,
    78  		tableDef:           tableDef,
    79  		partitionsTableDef: partitionsTableDef,
    80  		partitionState:     partitionState,
    81  		fs:                 fs,
    82  		ts:                 ts,
    83  		approxObjectNum:    approxObjectNum,
    84  	}
    85  }
    86  
    87  type logtailUpdate struct {
    88  	c  chan uint64
    89  	mu struct {
    90  		sync.Mutex
    91  		updated map[uint64]struct{}
    92  	}
    93  }
    94  
    95  func newLogtailUpdate() *logtailUpdate {
    96  	u := &logtailUpdate{
    97  		c: make(chan uint64, 1000),
    98  	}
    99  	u.mu.updated = make(map[uint64]struct{})
   100  	return u
   101  }
   102  
   103  type GlobalStatsConfig struct {
   104  	LogtailUpdateStatsThreshold int
   105  }
   106  
   107  type GlobalStatsOption func(s *GlobalStats)
   108  
   109  func WithLogtailUpdateStatsThreshold(v int) GlobalStatsOption {
   110  	return func(s *GlobalStats) {
   111  		s.cfg.LogtailUpdateStatsThreshold = v
   112  	}
   113  }
   114  
   115  type GlobalStats struct {
   116  	ctx context.Context
   117  
   118  	cfg GlobalStatsConfig
   119  
   120  	// engine is the global Engine instance.
   121  	engine *Engine
   122  
   123  	// tailC is the chan to receive entries from logtail
   124  	// and then update the stats info map.
   125  	// TODO(volgariver6): add metrics of the chan length.
   126  	tailC chan *logtail.TableLogtail
   127  
   128  	updateC chan pb.StatsInfoKey
   129  
   130  	// statsUpdated is used to control the frequency of updating stats info.
   131  	// It is not necessary to update stats info too frequently.
   132  	// It records the update time of the stats info key.
   133  	statsUpdated sync.Map
   134  
   135  	logtailUpdate *logtailUpdate
   136  
   137  	// tableLogtailCounter is the counter of the logtail entry of stats info key.
   138  	tableLogtailCounter map[pb.StatsInfoKey]int
   139  
   140  	// statsInfoMap is the global stats info in engine which
   141  	// contains all subscribed tables stats info.
   142  	mu struct {
   143  		sync.Mutex
   144  
   145  		// cond is used to wait for stats updated for the first time.
   146  		// If sync parameter is false, it is unuseful.
   147  		cond *sync.Cond
   148  
   149  		// statsInfoMap is the real stats info data.
   150  		statsInfoMap map[pb.StatsInfoKey]*pb.StatsInfo
   151  	}
   152  
   153  	// KeyRouter is the router to decides which node should send to.
   154  	KeyRouter client.KeyRouter[pb.StatsInfoKey]
   155  }
   156  
   157  func NewGlobalStats(
   158  	ctx context.Context, e *Engine, keyRouter client.KeyRouter[pb.StatsInfoKey], opts ...GlobalStatsOption,
   159  ) *GlobalStats {
   160  	s := &GlobalStats{
   161  		ctx:                 ctx,
   162  		engine:              e,
   163  		tailC:               make(chan *logtail.TableLogtail, 10000),
   164  		updateC:             make(chan pb.StatsInfoKey, 1000),
   165  		logtailUpdate:       newLogtailUpdate(),
   166  		tableLogtailCounter: make(map[pb.StatsInfoKey]int),
   167  		KeyRouter:           keyRouter,
   168  	}
   169  	s.mu.statsInfoMap = make(map[pb.StatsInfoKey]*pb.StatsInfo)
   170  	s.mu.cond = sync.NewCond(&s.mu)
   171  	for _, opt := range opts {
   172  		opt(s)
   173  	}
   174  	s.fillConfig()
   175  	go s.consumeWorker(ctx)
   176  	go s.updateWorker(ctx)
   177  	return s
   178  }
   179  
   180  func (gs *GlobalStats) fillConfig() {
   181  	if gs.cfg.LogtailUpdateStatsThreshold == 0 {
   182  		gs.cfg.LogtailUpdateStatsThreshold = defaultLogtailUpdateStatsThreshold
   183  	}
   184  }
   185  
   186  func (gs *GlobalStats) Get(ctx context.Context, key pb.StatsInfoKey, sync bool) *pb.StatsInfo {
   187  	gs.mu.Lock()
   188  	defer gs.mu.Unlock()
   189  	info, ok := gs.mu.statsInfoMap[key]
   190  	if ok && info != nil {
   191  		return info
   192  	}
   193  
   194  	// Get stats info from remote node.
   195  	if gs.KeyRouter != nil {
   196  		client := gs.engine.qc
   197  		target := gs.KeyRouter.Target(key)
   198  		if len(target) != 0 && client != nil {
   199  			resp, err := client.SendMessage(ctx, target, client.NewRequest(query.CmdMethod_GetStatsInfo))
   200  			if err != nil || resp == nil {
   201  				logutil.Errorf("failed to send request to %s, err: %v, resp: %v", "", err, resp)
   202  			} else if resp.GetStatsInfoResponse != nil {
   203  				defer client.Release(resp)
   204  
   205  				info := resp.GetStatsInfoResponse.StatsInfo
   206  				// If we get stats info from remote node, update local stats info.
   207  				gs.mu.statsInfoMap[key] = info
   208  				return info
   209  			}
   210  		}
   211  	}
   212  
   213  	ok = false
   214  	if sync {
   215  		for !ok {
   216  			if ctx.Err() != nil {
   217  				return nil
   218  			}
   219  
   220  			// If the trigger condition is not satisfied, the stats will not be updated
   221  			// for long time. So we trigger the update here to get the stats info as soon
   222  			// as possible.
   223  			gs.triggerUpdate(key)
   224  
   225  			// Wait until stats info of the key is updated.
   226  			gs.mu.cond.Wait()
   227  
   228  			info, ok = gs.mu.statsInfoMap[key]
   229  		}
   230  	}
   231  	return info
   232  }
   233  
   234  func (gs *GlobalStats) enqueue(tail *logtail.TableLogtail) {
   235  	select {
   236  	case gs.tailC <- tail:
   237  	default:
   238  		logutil.Errorf("the channel of logtails is full")
   239  	}
   240  }
   241  
   242  func (gs *GlobalStats) consumeWorker(ctx context.Context) {
   243  	for {
   244  		select {
   245  		case <-ctx.Done():
   246  			return
   247  
   248  		case tail := <-gs.tailC:
   249  			gs.consumeLogtail(tail)
   250  		}
   251  	}
   252  }
   253  
   254  func (gs *GlobalStats) updateWorker(ctx context.Context) {
   255  	for {
   256  		select {
   257  		case <-ctx.Done():
   258  			return
   259  
   260  		case key := <-gs.updateC:
   261  			go gs.updateTableStats(key)
   262  		}
   263  	}
   264  }
   265  
   266  func (gs *GlobalStats) triggerUpdate(key pb.StatsInfoKey) {
   267  	select {
   268  	case gs.updateC <- key:
   269  	default:
   270  		logutil.Errorf("the channel of update table is full")
   271  	}
   272  }
   273  
   274  func (gs *GlobalStats) consumeLogtail(tail *logtail.TableLogtail) {
   275  	key := pb.StatsInfoKey{
   276  		DatabaseID: tail.Table.DbId,
   277  		TableID:    tail.Table.TbId,
   278  	}
   279  	if len(tail.CkpLocation) > 0 {
   280  		gs.triggerUpdate(key)
   281  	} else if tail.Table != nil {
   282  		var triggered bool
   283  		for _, cmd := range tail.Commands {
   284  			if logtailreplay.IsBlkTable(cmd.TableName) ||
   285  				logtailreplay.IsObjTable(cmd.TableName) ||
   286  				logtailreplay.IsMetaTable(cmd.TableName) {
   287  				triggered = true
   288  				gs.triggerUpdate(key)
   289  				break
   290  			}
   291  		}
   292  		if _, ok := gs.tableLogtailCounter[key]; !ok {
   293  			gs.tableLogtailCounter[key] = 1
   294  		} else {
   295  			gs.tableLogtailCounter[key]++
   296  		}
   297  		if !triggered && gs.tableLogtailCounter[key] > gs.cfg.LogtailUpdateStatsThreshold {
   298  			gs.tableLogtailCounter[key] = 0
   299  			gs.triggerUpdate(key)
   300  		}
   301  	}
   302  }
   303  
   304  func (gs *GlobalStats) notifyLogtailUpdate(tid uint64) {
   305  	gs.logtailUpdate.mu.Lock()
   306  	defer gs.logtailUpdate.mu.Unlock()
   307  	_, ok := gs.logtailUpdate.mu.updated[tid]
   308  	if ok {
   309  		return
   310  	}
   311  	gs.logtailUpdate.mu.updated[tid] = struct{}{}
   312  
   313  	select {
   314  	case gs.logtailUpdate.c <- tid:
   315  	default:
   316  	}
   317  }
   318  
   319  func (gs *GlobalStats) waitLogtailUpdated(tid uint64) {
   320  	gs.logtailUpdate.mu.Lock()
   321  	_, ok := gs.logtailUpdate.mu.updated[tid]
   322  	gs.logtailUpdate.mu.Unlock()
   323  	if ok {
   324  		return
   325  	}
   326  	var done bool
   327  	for {
   328  		if done {
   329  			return
   330  		}
   331  		gs.logtailUpdate.mu.Lock()
   332  		_, ok := gs.logtailUpdate.mu.updated[tid]
   333  		gs.logtailUpdate.mu.Unlock()
   334  		if ok {
   335  			return
   336  		}
   337  		select {
   338  		case <-gs.ctx.Done():
   339  			return
   340  
   341  		case i := <-gs.logtailUpdate.c:
   342  			if i == tid {
   343  				done = true
   344  			}
   345  		}
   346  	}
   347  }
   348  
   349  func (gs *GlobalStats) updateTableStats(key pb.StatsInfoKey) {
   350  	// wait until the table's logtail has been updated.
   351  	gs.waitLogtailUpdated(key.TableID)
   352  
   353  	ts, ok := gs.statsUpdated.Load(key)
   354  	if ok && time.Since(ts.(time.Time)) < MinUpdateInterval {
   355  		return
   356  	}
   357  
   358  	// updated is used to mark that the stats info is updated.
   359  	var updated bool
   360  
   361  	stats := plan2.NewStatsInfo()
   362  	defer func() {
   363  		gs.mu.Lock()
   364  		defer gs.mu.Unlock()
   365  
   366  		// If it is the first time that the stats info is updated,
   367  		// send it to key router.
   368  		if _, ok := gs.statsUpdated.Load(key); !ok && gs.KeyRouter != nil && updated {
   369  			gs.KeyRouter.AddItem(gossip.CommonItem{
   370  				Operation: gossip.Operation_Set,
   371  				Key: &gossip.CommonItem_StatsInfoKey{
   372  					StatsInfoKey: &pb.StatsInfoKey{
   373  						DatabaseID: key.DatabaseID,
   374  						TableID:    key.TableID,
   375  					},
   376  				},
   377  			})
   378  		}
   379  
   380  		// update the time to current time only if the stats is not nil.
   381  		if updated {
   382  			gs.mu.statsInfoMap[key] = stats
   383  			gs.statsUpdated.Store(key, time.Now())
   384  		} else if _, ok := gs.mu.statsInfoMap[key]; !ok {
   385  			gs.mu.statsInfoMap[key] = nil
   386  		}
   387  
   388  		// Notify all the waiters to read the new stats info.
   389  		gs.mu.cond.Broadcast()
   390  	}()
   391  
   392  	table := gs.engine.getLatestCatalogCache().GetTableById(key.DatabaseID, key.TableID)
   393  	// table or its definition is nil, means that the table is created but not committed yet.
   394  	if table == nil || table.TableDef == nil {
   395  		logutil.Errorf("cannot get table by ID %v", key)
   396  		return
   397  	}
   398  
   399  	partitionState := gs.engine.getOrCreateLatestPart(key.DatabaseID, key.TableID).Snapshot()
   400  	var partitionsTableDef []*plan2.TableDef
   401  	var approxObjectNum int64
   402  	if table.Partitioned > 0 {
   403  		partitionInfo := &plan2.PartitionByDef{}
   404  		if err := partitionInfo.UnMarshalPartitionInfo([]byte(table.Partition)); err != nil {
   405  			logutil.Errorf("failed to unmarshal partition table: %v", err)
   406  			return
   407  		}
   408  		for _, partitionTableName := range partitionInfo.PartitionTableNames {
   409  			partitionTable := gs.engine.getLatestCatalogCache().GetTableByName(key.DatabaseID, partitionTableName)
   410  			partitionsTableDef = append(partitionsTableDef, partitionTable.TableDef)
   411  			ps := gs.engine.getOrCreateLatestPart(key.DatabaseID, partitionTable.Id).Snapshot()
   412  			approxObjectNum += int64(ps.ApproxObjectsNum())
   413  		}
   414  	} else {
   415  		approxObjectNum = int64(partitionState.ApproxObjectsNum())
   416  	}
   417  
   418  	if approxObjectNum == 0 {
   419  		// There are no objects flushed yet.
   420  		return
   421  	}
   422  
   423  	// the time used to init stats info is not need to be too precise.
   424  	now := timestamp.Timestamp{PhysicalTime: time.Now().UnixNano()}
   425  	req := newUpdateStatsRequest(
   426  		table.TableDef,
   427  		partitionsTableDef,
   428  		partitionState,
   429  		gs.engine.fs,
   430  		types.TimestampToTS(now),
   431  		approxObjectNum,
   432  		stats,
   433  	)
   434  	if err := UpdateStats(gs.ctx, req); err != nil {
   435  		logutil.Errorf("failed to init stats info for table %v", key)
   436  		return
   437  	}
   438  	updated = true
   439  }
   440  
   441  func calcNdvUsingZonemap(zm objectio.ZoneMap, t *types.Type) float64 {
   442  	if !zm.IsInited() {
   443  		return -1 /*for new added column, its zonemap will be empty and not initialized*/
   444  	}
   445  	switch t.Oid {
   446  	case types.T_bool:
   447  		return 2
   448  	case types.T_bit:
   449  		return float64(types.DecodeFixed[uint64](zm.GetMaxBuf())) - float64(types.DecodeFixed[uint64](zm.GetMinBuf())) + 1
   450  	case types.T_int8:
   451  		return float64(types.DecodeFixed[int8](zm.GetMaxBuf())) - float64(types.DecodeFixed[int8](zm.GetMinBuf())) + 1
   452  	case types.T_int16:
   453  		return float64(types.DecodeFixed[int16](zm.GetMaxBuf())) - float64(types.DecodeFixed[int16](zm.GetMinBuf())) + 1
   454  	case types.T_int32:
   455  		return float64(types.DecodeFixed[int32](zm.GetMaxBuf())) - float64(types.DecodeFixed[int32](zm.GetMinBuf())) + 1
   456  	case types.T_int64:
   457  		return float64(types.DecodeFixed[int64](zm.GetMaxBuf())) - float64(types.DecodeFixed[int64](zm.GetMinBuf())) + 1
   458  	case types.T_uint8:
   459  		return float64(types.DecodeFixed[uint8](zm.GetMaxBuf())) - float64(types.DecodeFixed[uint8](zm.GetMinBuf())) + 1
   460  	case types.T_uint16:
   461  		return float64(types.DecodeFixed[uint16](zm.GetMaxBuf())) - float64(types.DecodeFixed[uint16](zm.GetMinBuf())) + 1
   462  	case types.T_uint32:
   463  		return float64(types.DecodeFixed[uint32](zm.GetMaxBuf())) - float64(types.DecodeFixed[uint32](zm.GetMinBuf())) + 1
   464  	case types.T_uint64:
   465  		return float64(types.DecodeFixed[uint64](zm.GetMaxBuf())) - float64(types.DecodeFixed[uint64](zm.GetMinBuf())) + 1
   466  	case types.T_decimal64:
   467  		return types.Decimal64ToFloat64(types.DecodeFixed[types.Decimal64](zm.GetMaxBuf()), t.Scale) -
   468  			types.Decimal64ToFloat64(types.DecodeFixed[types.Decimal64](zm.GetMinBuf()), t.Scale) + 1
   469  	case types.T_decimal128:
   470  		return types.Decimal128ToFloat64(types.DecodeFixed[types.Decimal128](zm.GetMaxBuf()), t.Scale) -
   471  			types.Decimal128ToFloat64(types.DecodeFixed[types.Decimal128](zm.GetMinBuf()), t.Scale) + 1
   472  	case types.T_float32:
   473  		return float64(types.DecodeFixed[float32](zm.GetMaxBuf())) - float64(types.DecodeFixed[float32](zm.GetMinBuf())) + 1
   474  	case types.T_float64:
   475  		return types.DecodeFixed[float64](zm.GetMaxBuf()) - types.DecodeFixed[float64](zm.GetMinBuf()) + 1
   476  	case types.T_timestamp:
   477  		return float64(types.DecodeFixed[types.Timestamp](zm.GetMaxBuf())) - float64(types.DecodeFixed[types.Timestamp](zm.GetMinBuf())) + 1
   478  	case types.T_date:
   479  		return float64(types.DecodeFixed[types.Date](zm.GetMaxBuf())) - float64(types.DecodeFixed[types.Date](zm.GetMinBuf())) + 1
   480  	case types.T_time:
   481  		return float64(types.DecodeFixed[types.Time](zm.GetMaxBuf())) - float64(types.DecodeFixed[types.Time](zm.GetMinBuf())) + 1
   482  	case types.T_datetime:
   483  		return float64(types.DecodeFixed[types.Datetime](zm.GetMaxBuf())) - float64(types.DecodeFixed[types.Datetime](zm.GetMinBuf())) + 1
   484  	case types.T_uuid, types.T_char, types.T_varchar, types.T_blob, types.T_json, types.T_text,
   485  		types.T_array_float32, types.T_array_float64:
   486  		//NDV Function
   487  		// An aggregate function that returns an approximate value similar to the result of COUNT(DISTINCT col),
   488  		// the "number of distinct values".
   489  		return -1
   490  	case types.T_enum:
   491  		return float64(types.DecodeFixed[types.Enum](zm.GetMaxBuf())) - float64(types.DecodeFixed[types.Enum](zm.GetMinBuf())) + 1
   492  	default:
   493  		return -1
   494  	}
   495  }
   496  
   497  func getMinMaxValueByFloat64(typ types.Type, buf []byte) float64 {
   498  	switch typ.Oid {
   499  	case types.T_bit:
   500  		return float64(types.DecodeUint64(buf))
   501  	case types.T_int8:
   502  		return float64(types.DecodeInt8(buf))
   503  	case types.T_int16:
   504  		return float64(types.DecodeInt16(buf))
   505  	case types.T_int32:
   506  		return float64(types.DecodeInt32(buf))
   507  	case types.T_int64:
   508  		return float64(types.DecodeInt64(buf))
   509  	case types.T_uint8:
   510  		return float64(types.DecodeUint8(buf))
   511  	case types.T_uint16:
   512  		return float64(types.DecodeUint16(buf))
   513  	case types.T_uint32:
   514  		return float64(types.DecodeUint32(buf))
   515  	case types.T_uint64:
   516  		return float64(types.DecodeUint64(buf))
   517  	case types.T_date:
   518  		return float64(types.DecodeDate(buf))
   519  	case types.T_time:
   520  		return float64(types.DecodeTime(buf))
   521  	case types.T_timestamp:
   522  		return float64(types.DecodeTimestamp(buf))
   523  	case types.T_datetime:
   524  		return float64(types.DecodeDatetime(buf))
   525  	//case types.T_char, types.T_varchar, types.T_text:
   526  	//return float64(plan2.ByteSliceToUint64(buf)), true
   527  	default:
   528  		panic("unsupported type")
   529  	}
   530  }
   531  
   532  // get ndv, minval , maxval, datatype from zonemap. Retrieve all columns except for rowid, return accurate number of objects
   533  func updateInfoFromZoneMap(ctx context.Context, req *updateStatsRequest, info *plan2.InfoFromZoneMap) error {
   534  	start := time.Now()
   535  	defer func() {
   536  		v2.TxnStatementUpdateInfoFromZonemapHistogram.Observe(time.Since(start).Seconds())
   537  	}()
   538  	lenCols := len(req.tableDef.Cols) - 1 /* row-id */
   539  	var (
   540  		init    bool
   541  		err     error
   542  		meta    objectio.ObjectDataMeta
   543  		objMeta objectio.ObjectMeta
   544  	)
   545  	fs, err := fileservice.Get[fileservice.FileService](req.fs, defines.SharedFileServiceName)
   546  	if err != nil {
   547  		return err
   548  	}
   549  
   550  	onObjFn := func(obj logtailreplay.ObjectEntry) error {
   551  		location := obj.Location()
   552  		if objMeta, err = objectio.FastLoadObjectMeta(ctx, &location, false, fs); err != nil {
   553  			return err
   554  		}
   555  		meta = objMeta.MustDataMeta()
   556  		info.AccurateObjectNumber++
   557  		info.BlockNumber += int64(obj.BlkCnt())
   558  		info.TableCnt += float64(meta.BlockHeader().Rows())
   559  		if !init {
   560  			init = true
   561  			for idx, col := range req.tableDef.Cols[:lenCols] {
   562  				objColMeta := meta.MustGetColumn(uint16(col.Seqnum))
   563  				info.NullCnts[idx] = int64(objColMeta.NullCnt())
   564  				info.ColumnZMs[idx] = objColMeta.ZoneMap().Clone()
   565  				info.DataTypes[idx] = types.T(col.Typ.Id).ToType()
   566  				info.ColumnNDVs[idx] = float64(objColMeta.Ndv())
   567  				info.ColumnSize[idx] = int64(meta.BlockHeader().ZoneMapArea().Length() +
   568  					meta.BlockHeader().BFExtent().Length() + objColMeta.Location().Length())
   569  				if info.ColumnNDVs[idx] > 100 || info.ColumnNDVs[idx] > 0.1*float64(meta.BlockHeader().Rows()) {
   570  					switch info.DataTypes[idx].Oid {
   571  					case types.T_int64, types.T_int32, types.T_int16, types.T_uint64, types.T_uint32, types.T_uint16, types.T_time, types.T_timestamp, types.T_date, types.T_datetime:
   572  						info.ShuffleRanges[idx] = plan2.NewShuffleRange(false)
   573  						if info.ColumnZMs[idx].IsInited() {
   574  							minvalue := getMinMaxValueByFloat64(info.DataTypes[idx], info.ColumnZMs[idx].GetMinBuf())
   575  							maxvalue := getMinMaxValueByFloat64(info.DataTypes[idx], info.ColumnZMs[idx].GetMaxBuf())
   576  							info.ShuffleRanges[idx].Update(minvalue, maxvalue, int64(meta.BlockHeader().Rows()), int64(objColMeta.NullCnt()))
   577  						}
   578  					case types.T_varchar, types.T_char, types.T_text:
   579  						info.ShuffleRanges[idx] = plan2.NewShuffleRange(true)
   580  						if info.ColumnZMs[idx].IsInited() {
   581  							info.ShuffleRanges[idx].UpdateString(info.ColumnZMs[idx].GetMinBuf(), info.ColumnZMs[idx].GetMaxBuf(), int64(meta.BlockHeader().Rows()), int64(objColMeta.NullCnt()))
   582  						}
   583  					}
   584  				}
   585  			}
   586  		} else {
   587  			for idx, col := range req.tableDef.Cols[:lenCols] {
   588  				objColMeta := meta.MustGetColumn(uint16(col.Seqnum))
   589  				info.NullCnts[idx] += int64(objColMeta.NullCnt())
   590  				zm := objColMeta.ZoneMap().Clone()
   591  				if !zm.IsInited() {
   592  					continue
   593  				}
   594  				index.UpdateZM(info.ColumnZMs[idx], zm.GetMaxBuf())
   595  				index.UpdateZM(info.ColumnZMs[idx], zm.GetMinBuf())
   596  				info.ColumnNDVs[idx] += float64(objColMeta.Ndv())
   597  				info.ColumnSize[idx] += int64(objColMeta.Location().Length())
   598  				if info.ShuffleRanges[idx] != nil {
   599  					switch info.DataTypes[idx].Oid {
   600  					case types.T_int64, types.T_int32, types.T_int16, types.T_uint64, types.T_uint32, types.T_uint16, types.T_time, types.T_timestamp, types.T_date, types.T_datetime:
   601  						minvalue := getMinMaxValueByFloat64(info.DataTypes[idx], zm.GetMinBuf())
   602  						maxvalue := getMinMaxValueByFloat64(info.DataTypes[idx], zm.GetMaxBuf())
   603  						info.ShuffleRanges[idx].Update(minvalue, maxvalue, int64(meta.BlockHeader().Rows()), int64(objColMeta.NullCnt()))
   604  					case types.T_varchar, types.T_char, types.T_text:
   605  						info.ShuffleRanges[idx].UpdateString(zm.GetMinBuf(), zm.GetMaxBuf(), int64(meta.BlockHeader().Rows()), int64(objColMeta.NullCnt()))
   606  					}
   607  				}
   608  			}
   609  		}
   610  		return nil
   611  	}
   612  	if err = ForeachVisibleDataObject(req.partitionState, req.ts, onObjFn); err != nil {
   613  		return err
   614  	}
   615  
   616  	return nil
   617  }
   618  
   619  func adjustNDV(info *plan2.InfoFromZoneMap, tableDef *plan2.TableDef) {
   620  	lenCols := len(tableDef.Cols) - 1 /* row-id */
   621  
   622  	if info.AccurateObjectNumber > 1 {
   623  		for idx := range tableDef.Cols[:lenCols] {
   624  			rate := info.ColumnNDVs[idx] / info.TableCnt
   625  			if rate > 1 {
   626  				rate = 1
   627  			}
   628  			if rate < 0.1 {
   629  				info.ColumnNDVs[idx] /= math.Pow(float64(info.AccurateObjectNumber), (1 - rate))
   630  			}
   631  			ndvUsingZonemap := calcNdvUsingZonemap(info.ColumnZMs[idx], &info.DataTypes[idx])
   632  			if ndvUsingZonemap != -1 && info.ColumnNDVs[idx] > ndvUsingZonemap {
   633  				info.ColumnNDVs[idx] = ndvUsingZonemap
   634  			}
   635  
   636  			if info.ColumnNDVs[idx] > info.TableCnt {
   637  				info.ColumnNDVs[idx] = info.TableCnt
   638  			}
   639  		}
   640  	}
   641  }
   642  
   643  // UpdateStats is the main function to calculate and update the stats for scan node.
   644  func UpdateStats(ctx context.Context, req *updateStatsRequest) error {
   645  	start := time.Now()
   646  	defer func() {
   647  		v2.TxnStatementUpdateStatsDurationHistogram.Observe(time.Since(start).Seconds())
   648  	}()
   649  	lenCols := len(req.tableDef.Cols) - 1 /* row-id */
   650  	info := plan2.NewInfoFromZoneMap(lenCols)
   651  	if req.approxObjectNum == 0 {
   652  		return nil
   653  	}
   654  
   655  	info.ApproxObjectNumber = req.approxObjectNum
   656  	baseTableDef := req.tableDef
   657  	if len(req.partitionsTableDef) > 0 {
   658  		for _, def := range req.partitionsTableDef {
   659  			req.tableDef = def
   660  			if err := updateInfoFromZoneMap(ctx, req, info); err != nil {
   661  				return err
   662  			}
   663  		}
   664  	} else if err := updateInfoFromZoneMap(ctx, req, info); err != nil {
   665  		return err
   666  	}
   667  	adjustNDV(info, baseTableDef)
   668  	plan2.UpdateStatsInfo(info, baseTableDef, req.statsInfo)
   669  	return nil
   670  }