-
Notifications
You must be signed in to change notification settings - Fork 5.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
util: add PreAlloc4Row and Insert for Chunk and List #7916
Conversation
/run-all-tests |
/run-all-tests |
util/chunk/chunk.go
Outdated
// The null elem info will be pre-written. | ||
func (c *Chunk) PreAlloc4Row(row Row) { | ||
for i, rowCol := range row.c.columns { | ||
chkCol := c.columns[i] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about:
- s/chkCol/dstCol/
- s/rowCol/srcCol/
util/chunk/chunk.go
Outdated
@@ -277,6 +278,60 @@ func (c *Chunk) AppendPartialRow(colIdx int, row Row) { | |||
} | |||
} | |||
|
|||
// PreAlloc4Row pre-allocates the memory space for a Row. | |||
// The null elem info will be pre-written. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- s/elem info/bitmap/
- should the schema of
row
andc
be the same?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, but it's should be promised by the caller.
util/chunk/chunk.go
Outdated
for i, rowCol := range row.c.columns { | ||
chkCol := c.columns[i] | ||
chkCol.appendNullBitmap(!rowCol.isNull(row.idx)) | ||
if rowCol.isFixed() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about:
elemLen := len(rowCol.elemBuf)
if !rowCol.isFixed() {
elemLen = int(rowCol.offsets[row.idx+1] - rowCol.offsets[row.idx])
chkCol.offsets = append(chkCol.offsets, len(chkCol.data)+elemLen)
}
if len(chkCol.data)+elemLen >= cap(chkCol.data) {
...
util/chunk/chunk.go
Outdated
@@ -277,6 +278,60 @@ func (c *Chunk) AppendPartialRow(colIdx int, row Row) { | |||
} | |||
} | |||
|
|||
// PreAlloc4Row pre-allocates the memory space for a Row. | |||
// The null elem info will be pre-written. | |||
func (c *Chunk) PreAlloc4Row(row Row) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we preallocate the memory for a batch of rows? The memory grow stratagem may allocate a lot of unused memory.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
-
Preallocate memory for a batch of rows cannot avoid the waste problem brought by memory usage increment of chunk.data.
Unless the batch contains all of the rows which should be pre-allocated. -
I adjust the strategy according to https://github.com/golang/go/blob/master/src/runtime/slice.go#L116-L135
util/chunk/chunk.go
Outdated
} | ||
chkCol.length++ | ||
} | ||
c.numVirtualRows++ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why do we increase numVirtualRows
?
util/chunk/chunk.go
Outdated
srcStart, srcEnd = int(rowCol.offsets[row.idx]), int(rowCol.offsets[row.idx+1]) | ||
destStart, destEnd = int(chkCol.offsets[rowIdx]), int(chkCol.offsets[rowIdx+1]) | ||
} | ||
if destEnd-destStart != srcEnd-srcStart { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we really need to add this length check?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In practice, this should be promised by the caller, as well as line 316.
util/chunk/chunk.go
Outdated
} | ||
} | ||
} | ||
dstCol.data = make([]byte, len(dstCol.data)+elemLen, newCap) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we copy the old content into the new allocated buffer?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is no old data in column.data.
We'll insert the values parallelly later on.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This line will not copy data into column.data, neither.
All the chunk in List is empty.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's better to add all the requirements in the comments of this API
util/chunk/chunk.go
Outdated
elemLen = int(srcCol.offsets[row.idx+1] - srcCol.offsets[row.idx]) | ||
dstCol.offsets = append(dstCol.offsets, int32(len(dstCol.data)+elemLen)) | ||
} | ||
if needCap := len(dstCol.data) + elemLen; needCap > cap(dstCol.data) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about:
dstCol.length++
if needCap := len(dstCol.data) + elemLen; needCap <= cap(dstCol.data) {
(*reflect.SliceHeader)(unsafe.Pointer(&dstCol.data)).Len = len(dstCol.data) + elemLen
continue
}
// Grow the capacity according to golang.growslice.
util/chunk/chunk.go
Outdated
// Cover the origin data if the upper check is valid. | ||
for i, rowCol := range row.c.columns { | ||
chkCol := c.columns[i] | ||
if chkCol.isFixed() != rowCol.isFixed() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this check can be removed
util/chunk/chunk.go
Outdated
func (c *Chunk) Insert(rowIdx int, row Row) { | ||
// Check data length between row and the origin data for every column. | ||
// Cover the origin data if the upper check is valid. | ||
for i, rowCol := range row.c.columns { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
s/rowCol/srcCol/
s/chkCol/dstCol/
util/chunk/chunk.go
Outdated
srcStart, srcEnd = int(rowCol.offsets[row.idx]), int(rowCol.offsets[row.idx+1]) | ||
destStart, destEnd = int(chkCol.offsets[rowIdx]), int(chkCol.offsets[rowIdx+1]) | ||
} | ||
if destEnd-destStart != srcEnd-srcStart { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this check can be removed.
util/chunk/list.go
Outdated
// PreAlloc4Row pre-allocate the storage memory for a Row. | ||
func (l *List) PreAlloc4Row(row Row) (ptr RowPtr) { | ||
chkIdx := len(l.chunks) - 1 | ||
if chkIdx == -1 || l.chunks[chkIdx].NumRows() >= l.chunks[chkIdx].Capacity() || chkIdx == l.consumedIdx { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this check can be removed: chkIdx == l.consumedIdx
PTAL @zz-jason |
func (c *Chunk) PreAlloc4Row(row Row) { | ||
for i, srcCol := range row.c.columns { | ||
dstCol := c.columns[i] | ||
dstCol.appendNullBitmap(!srcCol.isNull(row.idx)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's better to only pre-allocate the memory for null bitmap by calling dstCol.appendNullBitmap(true)
and set the null bit mask in the insert
function.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We write the nullBitMap info here is because that,
Insert
will be called parallelly later, every element in nullBitMap is a byte .
It will cause data race problem if we write the bit mask in Insert
.
util/chunk/chunk.go
Outdated
@@ -277,6 +278,65 @@ func (c *Chunk) AppendPartialRow(colIdx int, row Row) { | |||
} | |||
} | |||
|
|||
// PreAlloc4Row pre-allocates the memory space for a Row. | |||
// Nothing except for the nullBitMap of c.columns will be pre-written. | |||
func (c *Chunk) PreAlloc4Row(row Row) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about:
- s/PreAlloc4Row/PreAlloc/
- change the comment to:
// PreAlloc pre-allocates the memory space in a Chunk to store the Row.
// NOTE:
// 1. The Chunk must be empty or holds no useful data.
// 2. The schema of the Row must be the same with the Chunk.
// 3. This API is paired with the `Insert()` function, which inserts all the
// rows into the Chunk after the pre-allocation.
// 1. The Chunk must be empty or holds no useful data. | ||
// 2. The schema of the Row must be the same with the Chunk. | ||
// 3. This API is paired with the `Insert()` function, which inserts all the | ||
// rows data into the Chunk after the pre-allocation. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about adding another NOTE in the comment:
// 4. We set the null bitmap here instead of in the Insert() function because
// the Insert() function is called parallelly, the data race on a byte can
// not be avoided although the manipulated bits are different inside a byte.
// Note: this function will *ONLY* allocate the needed memory for `row`, the | ||
// data will *NOT* be written into the List. List.Insert can be called to write | ||
// the data later on. | ||
func (l *List) PreAlloc4Row(row Row) (ptr RowPtr) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the comment of this function should also be updated.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
/run-all-tests |
Emm, since the column may be unfixed length. So the order that the row inserted is preserved? |
@winoros Yep, so we can write into the chunk parallelly. |
got |
startWg, endWg := &sync.WaitGroup{}, &sync.WaitGroup{} | ||
startWg.Add(1) | ||
for i := 0; i < srcChk.NumRows(); i++ { | ||
destChk.PreAlloc(srcChk.GetRow(i)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
emm..I have a question too, there are any potential PreAlloc
many rows in one call?
or direct PreAlloc
last row in chunk....in our test case, it's PreAlloc row by row and memory grow as row handle, maybe direct alloc whole memory is better? 😆 ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
just a question not review... I read too slow 🤣
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@lysu Direct PreAlloc
the whole memory when meeting the last row seems to be reasonable.
May be we can add an input argument like doAlloc
for PreAlloc
?
Not that easy. We need a better way.
bugfix fixed pingcap#7518 expression: MySQL compatible current_user function (pingcap#7801) plan: propagate constant over outer join (pingcap#7794) - extract `outerCol = const` from join conditions and filter conditions, substitute `outerCol` in join conditions with `const`; - extract `outerCol = innerCol` from join conditions, derive new join conditions based on this column equal condition and `outerCol` related expressions in join conditions and filter conditions; util/timeutil: fix data race caused by forgetting set stats lease to 0 (pingcap#7901) stats: handle ddl event for partition table (pingcap#7903) plan: implement Operand and Pattern of cascades planner. (pingcap#7910) planner: not convert to TableDual if empty range is derived from deferred constants (pingcap#7808) plan: move projEliminate behind aggEliminate (pingcap#7909) admin: fix admin check table bug of byte compare (pingcap#7887) * admin: remove reflect deepEqual stats: fix panic caused by empty histogram (pingcap#7912) plan: fix panic caused by empty schema of LogicalTableDual (pingcap#7906) * fix drop view if exist error (pingcap#7833) executor: refine `explain analyze` (pingcap#7888) executor: add an variable to compatible with MySQL insert for OGG (pingcap#7863) expression: maintain `DeferredExpr` in aggressive constant folding. (pingcap#7915) stats: fix histogram boundaries overflow error (pingcap#7883) ddl:support the definition of `null` change to `not null` using `alter table` (pingcap#7771) * ddl:support the definition of null change to not null using alter table ddl: add check when create table with foreign key. (pingcap#7885) * ddl: add check when create table with foreign key planner: eliminate if null on non null column (pingcap#7924) executor: fix a bug in point get (pingcap#7934) planner, executor: refine ColumnPrune for LogicalUnionAll (pingcap#7930) executor: fix panic when limit is too large (pingcap#7936) ddl: add TiDB version to metrics (pingcap#7902) stats: limit the length of sample values (pingcap#7931) vendor: update tipb (pingcap#7893) planner: support the Group and GroupExpr for the cascades planner (pingcap#7917) store/tikv: log more information when other err occurs (pingcap#7948) types: fix date time parse (pingcap#7933) ddl: just print error message when ddl job is normal to calcel, to eliminate noisy log (pingcap#7875) stats: update delta info for partition table (pingcap#7947) explaintest: add explain test for partition pruning (pingcap#7505) util: move disjoint set to util package (pingcap#7950) util: add PreAlloc4Row and Insert for Chunk and List (pingcap#7916) executor: add the slow log for commit (pingcap#7951) expression: add builtin json_keys (pingcap#7776) privilege: add USAGE in `show grants` for mysql compatibility (pingcap#7955) ddl: fix invailid ddl job panic (pingcap#7940) *: move ast.NewValueExpr to standalone parser_driver package (pingcap#7952) Make the ast package get rid of the dependency of types.Datum server: allow cors http request (pingcap#7939) *: move `Statement` and `RecordSet` from ast to sqlexec package (pingcap#7970) pr suggestion update executor/aggfuncs: split unit tests to corresponding file (pingcap#7993) store/tikv: fix typo (pingcap#7990) executor, planner: clone proj schema for different children in buildProj4Union (pingcap#7999) executor: let information_schema be the first database in ShowDatabases (pingcap#7938) stats: use local feedback for partition table (pingcap#7963) executor: add unit test for aggfuncs (pingcap#7966) server: add log for binary execute statement (pingcap#7987) admin: refine admin check decoder (pingcap#7862) executor: improve wide table insert & update performance (pingcap#7935) ddl: fix reassigned partition id in `truncate table` does not take effect (pingcap#7919) fix reassigned partition id in truncate table does not take effect add changelog for 2.1.0 rc4 (pingcap#8020) *: make parser package dependency as small as possible (pingcap#7989) parser: support `:=` in the `set` syntax (pingcap#8018) According to MySQL document, `set` use the = assignment operator, but the := assignment operator is also permitted stats: garbage collect stats for partition table (pingcap#7962) docs: add the proposal for the column pool (pingcap#7988) expression: refine built-in func truncate to support uint arg (pingcap#8000) stats: support show stats for partition table (pingcap#8023) stats: update error rate for partition table (pingcap#8022) stats: fix estimation for out of range point queries (pingcap#8015) *: move parser to a separate repository (pingcap#8036) executor: fix wrong result when index join on union scan. (pingcap#8031) Do not modify Plan of dataReaderBuilder directly, because it would impact next batch of outer rows, as well as other concurrent inner workers. Instead, build a local child builder to store the child plan. planner: fix a panic of a cached prepared statement with IndexScan (pingcap#8017) *: fix the issue of executing DDL after executing SQL failure in txn (pingcap#8044) * ddl, executor: fix the issue of executing DDL after executing SQL failure in txn add unit test remove debug info add like evaluator case sensitive test ddl, domain: make schema correct after canceling jobs (pingcap#7997) unit test fix code format proposal: maintaining histograms in plan. (pingcap#7605) support _tidb_rowid for table scan range (pingcap#8047) var rename fix
What problem does this PR solve?
This is a preparation work for parallel partition.
Add two new member function for *chunk.Chunk and *chunk.List.
row
but does not write the data.Note: nullBitMap info will be written in this function, since Chunk.nullBitMap can not be accessed parallely.
row
at the position speficied byrowIdx
.Note: Insert can only be called after PreAlloc4Row, or it will cause unexpected errors.
What is changed and how it works?
N/A
Check List
Tests
Code changes
Side effects
N/A
Related changes
N/A