Skip to content

Commit

Permalink
util: Change the name of ListInDisk (#47777)
Browse files Browse the repository at this point in the history
ref #47733
  • Loading branch information
xzhangxian1008 authored Oct 20, 2023
1 parent 9b8890d commit 3c461dd
Show file tree
Hide file tree
Showing 11 changed files with 100 additions and 99 deletions.
18 changes: 9 additions & 9 deletions docs/design/2021-08-18-charsets.md
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ After receiving the non-utf-8 character set request, this solution will convert
- All ParseOneStmt/Parse usage needs to be checked
- For SQL strings that need to be temporarily saved, you need to bring character set information. For example, BindSQL, View Select Stmt, etc.
- For internally executed SQL statements, since they are already utf-8, they do not need to be processed. For example, the table creation statement in the perfschema package.

### Runtime
- Add a repertoire field to collationInfo to facilitate automatic character set conversion in expressions, so that many errors like "illegal mix of collations" can be avoided.
- The corresponding types of the Repertoire attribute are as follows:
Expand All @@ -75,11 +75,11 @@ After receiving the non-utf-8 character set request, this solution will convert

const (
// RepertoireASCII is pure ASCII and it’s Unicode range: U+0000..U+007F
RepertoireASCII Repertoire = 1
// RepertoireExtended is Extended characters and it’s Unicode range: U+0080..U+FFFF
RepertoireExtended Repertoire = 1 << 1
// RepertoireUnicode consists ASCII and EXTENDED, and it’s Unicode range: U+0000..U+FFFF
RepertoireUnicode Repertoire = ASCII | EXTENDED
RepertoireASCII Repertoire = 1
// RepertoireExtended is Extended characters and it’s Unicode range: U+0080..U+FFFF
RepertoireExtended Repertoire = 1 << 1
// RepertoireUnicode consists ASCII and EXTENDED, and it’s Unicode range: U+0000..U+FFFF
RepertoireUnicode Repertoire = ASCII | EXTENDED
)
```

Expand All @@ -90,7 +90,7 @@ After receiving the non-utf-8 character set request, this solution will convert

### Optimizer

- The statistics module may need special processing functions based on charset: AvgColSizeListInDisk, GetFixedLen, BucketToString and DecodedString, etc.
- The statistics module may need special processing functions based on charset: AvgColSizeDataInDiskByRows, GetFixedLen, BucketToString and DecodedString, etc.
- Ranger module
- The processing of prefix len needs to consider the charset.
- Functions such as BuildFromPatternLike and checkLikeFunc may need to consider charset.
Expand Down Expand Up @@ -123,7 +123,7 @@ Other behaviors that need to be dealt with:
- Upgrade compatibility:
- There may be compatibility issues when performing operations during the rolling upgrade.
- The new version of the cluster is expected to have no compatibility issues when reading old data.
- Downgrade compatibility:
- Downgrade compatibility:
- Downgrade is not compatible. The index key uses the table of gbk_bin/gbk_chinese_ci. The lower version of TiDB will have problems when decoding, and it needs to be transcoded before downgrading.

#### Compatibility with MySQL
Expand Down Expand Up @@ -189,7 +189,7 @@ Test the compatibility of some related features, such as SQL binding, SQL hints,
- Version 4.0 and above test upgrade:
- During the rolling upgrade process, gbk operation is not supported.
- After the upgrade, normal gbk related operations are supported.

### Downgrade compatibility

There will be incompatibility issues when downgrading tables that use gbk encoding.
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/aggregate/agg_hash_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,8 +127,8 @@ type HashAggExec struct {
stats *HashAggRuntimeStats

// listInDisk is the chunks to store row values for spilled data.
// The HashAggExec may be set to `spill mode` multiple times, and all spilled data will be appended to ListInDisk.
listInDisk *chunk.ListInDisk
// The HashAggExec may be set to `spill mode` multiple times, and all spilled data will be appended to DataInDiskByRows.
listInDisk *chunk.DataInDiskByRows
// numOfSpilledChks indicates the number of all the spilled chunks.
numOfSpilledChks int
// offsetOfSpilledChks indicates the offset of the chunk be read from the disk.
Expand Down Expand Up @@ -240,7 +240,7 @@ func (e *HashAggExec) initForUnparallelExec() {

e.offsetOfSpilledChks, e.numOfSpilledChks = 0, 0
e.executed, e.isChildDrained = false, false
e.listInDisk = chunk.NewListInDisk(exec.RetTypes(e.Children(0)))
e.listInDisk = chunk.NewDataInDiskByRows(exec.RetTypes(e.Children(0)))

e.tmpChkForSpill = exec.TryNewCacheChunk(e.Children(0))
if vars := e.Ctx().GetSessionVars(); vars.TrackAggregateMemoryUsage && variable.EnableTmpStorageOnOOM.Load() {
Expand Down
14 changes: 7 additions & 7 deletions pkg/planner/cardinality/row_size.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,8 @@ func GetAvgRowSize(ctx sessionctx.Context, coll *statistics.HistColl, cols []*ex
return size + float64(len(cols))
}

// GetAvgRowSizeListInDisk computes average row size for given columns.
func GetAvgRowSizeListInDisk(coll *statistics.HistColl, cols []*expression.Column) (size float64) {
// GetAvgRowSizeDataInDiskByRows computes average row size for given columns.
func GetAvgRowSizeDataInDiskByRows(coll *statistics.HistColl, cols []*expression.Column) (size float64) {
if coll.Pseudo || len(coll.Columns) == 0 || coll.RealtimeCount == 0 {
for _, col := range cols {
size += float64(chunk.EstimateTypeWidth(col.GetType()))
Expand All @@ -103,10 +103,10 @@ func GetAvgRowSizeListInDisk(coll *statistics.HistColl, cols []*expression.Colum
size += float64(chunk.EstimateTypeWidth(col.GetType()))
continue
}
size += AvgColSizeListInDisk(colHist, coll.RealtimeCount)
size += AvgColSizeDataInDiskByRows(colHist, coll.RealtimeCount)
}
}
// Add 8 byte for each column's size record. See `ListInDisk` for details.
// Add 8 byte for each column's size record. See `DataInDiskByRows` for details.
return size + float64(8*len(cols))
}

Expand Down Expand Up @@ -160,9 +160,9 @@ func AvgColSizeChunkFormat(c *statistics.Column, count int64) float64 {
return math.Round((avgSize-math.Log2(avgSize))*100)/100 + 8
}

// AvgColSizeListInDisk is the average column size of the histogram. These sizes are derived
// from `chunk.ListInDisk` so we need to update them if those 2 functions are changed.
func AvgColSizeListInDisk(c *statistics.Column, count int64) float64 {
// AvgColSizeDataInDiskByRows is the average column size of the histogram. These sizes are derived
// from `chunk.DataInDiskByRows` so we need to update them if those 2 functions are changed.
func AvgColSizeDataInDiskByRows(c *statistics.Column, count int64) float64 {
if count == 0 {
return 0
}
Expand Down
20 changes: 10 additions & 10 deletions pkg/planner/cardinality/row_size_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,36 +40,36 @@ func TestAvgColLen(t *testing.T) {
tableInfo := tbl.Meta()
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
require.Equal(t, 1.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))

// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0-3, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0-3, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0-3+8, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
require.Equal(t, 0.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
require.Equal(t, 0.0, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29', NULL)")
testKit.MustExec("analyze table t")
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
require.Equal(t, 1.5, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 10.5, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSize(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount, false))
require.Equal(t, 8.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[0].ID], statsTbl.RealtimeCount))
require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100+8, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[1].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[2].ID], statsTbl.RealtimeCount))
require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[3].ID], statsTbl.RealtimeCount))
require.Equal(t, 8.0, cardinality.AvgColSizeChunkFormat(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
require.Equal(t, 0.0, cardinality.AvgColSizeListInDisk(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
require.Equal(t, 0.0, cardinality.AvgColSizeDataInDiskByRows(statsTbl.Columns[tableInfo.Columns[4].ID], statsTbl.RealtimeCount))
}
2 changes: 1 addition & 1 deletion pkg/planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,7 +307,7 @@ func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task {
// RowSize for cost model ver2 is simplified, always use this function to calculate row size.
func getAvgRowSize(stats *property.StatsInfo, cols []*expression.Column) (size float64) {
if stats.HistColl != nil {
size = cardinality.GetAvgRowSizeListInDisk(stats.HistColl, cols)
size = cardinality.GetAvgRowSizeDataInDiskByRows(stats.HistColl, cols)
} else {
// Estimate using just the type info.
for _, col := range cols {
Expand Down
4 changes: 2 additions & 2 deletions pkg/util/chunk/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,14 +9,14 @@ go_library(
"codec.go",
"column.go",
"compare.go",
"disk.go",
"iterator.go",
"list.go",
"mutrow.go",
"pool.go",
"row.go",
"row_container.go",
"row_container_reader.go",
"row_in_disk.go",
],
importpath = "github.com/pingcap/tidb/pkg/util/chunk",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -48,13 +48,13 @@ go_test(
"chunk_util_test.go",
"codec_test.go",
"column_test.go",
"disk_test.go",
"iterator_test.go",
"list_test.go",
"main_test.go",
"mutrow_test.go",
"pool_test.go",
"row_container_test.go",
"row_in_disk_test.go",
],
embed = [":chunk"],
flaky = True,
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/chunk/list.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func (l *List) FieldTypes() []*types.FieldType {
return l.fieldTypes
}

// NumRowsOfChunk returns the number of rows of a chunk in the ListInDisk.
// NumRowsOfChunk returns the number of rows of a chunk in the DataInDiskByRows.
func (l *List) NumRowsOfChunk(chkID int) int {
return l.chunks[chkID].NumRows()
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/util/chunk/row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (

type rowContainerRecord struct {
inMemory *List
inDisk *ListInDisk
inDisk *DataInDiskByRows
// spillError stores the error when spilling.
spillError error
}
Expand Down Expand Up @@ -152,7 +152,7 @@ func (c *RowContainer) spillToDisk(preSpillError error) {
var err error
memory.QueryForceDisk.Add(1)
n := c.m.records.inMemory.NumChunks()
c.m.records.inDisk = NewListInDisk(c.m.records.inMemory.FieldTypes())
c.m.records.inDisk = NewDataInDiskByRows(c.m.records.inMemory.FieldTypes())
c.m.records.inDisk.diskTracker.AttachTo(c.diskTracker)
defer func() {
if r := recover(); r != nil {
Expand Down Expand Up @@ -221,7 +221,7 @@ func (c *RowContainer) NumRow() int {
return c.m.records.inMemory.Len()
}

// NumRowsOfChunk returns the number of rows of a chunk in the ListInDisk.
// NumRowsOfChunk returns the number of rows of a chunk in the DataInDiskByRows.
func (c *RowContainer) NumRowsOfChunk(chkID int) int {
c.m.RLock()
defer c.m.RUnlock()
Expand Down
Loading

0 comments on commit 3c461dd

Please sign in to comment.