Skip to content
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

chunk: support capacity grow #7473

Merged
merged 14 commits into from
Aug 28, 2018
Merged

chunk: support capacity grow #7473

merged 14 commits into from
Aug 28, 2018

Conversation

lysu
Copy link
Contributor

@lysu lysu commented Aug 23, 2018

What problem does this PR solve?

This is first reviewable part of #7141, in first step, we make chunk be growable, then we can review for each operator's chunk modifition carefully.

What is changed and how it works?

  • add New() Renew() and GrowReset() method to chunk
  • add eraseGrow() in column
  • adaptor NewChunkWithCapacity() and Reset() works in chunk.

Check List

Tests

  • Benchmark Test
goos: linux
goarch: amd64
pkg: github.com/pingcap/tidb/util/chunk
BenchmarkChunkRenewConsumeExec10000000In1024Slice-8   	      10	 136971920 ns/op	83128428 B/op	   58596 allocs/op
BenchmarkChunkRenewConsumeExec10000000In32Slice-8     	       5	 206753238 ns/op	142500092 B/op	 1875000 allocs/op
BenchmarkChunkRenewConsumeExec10000000InGrow-8        	      10	 136599018 ns/op	83129195 B/op	   58598 allocs/op
BenchmarkChunkRestConsumeExec10000000In1024Slice-8    	      10	 113129478 ns/op	       0 B/op	       0 allocs/op
BenchmarkChunkResetConsumeExec10000000In32Slice-8     	      10	 118094010 ns/op	       0 B/op	       0 allocs/op
BenchmarkChunkResetConsumeExec10000000InGrow-8        	      10	 105698574 ns/op	     820 B/op	       1 allocs/op
BenchmarkChunkRenewConsumeExec10In1024Slice-8         	 1000000	      1939 ns/op	    8512 B/op	       6 allocs/op
BenchmarkChunkRenewConsumeExec10In32Slice-8           	 3000000	       422 ns/op	     456 B/op	       6 allocs/op
BenchmarkChunkRenewConsumeExec10InGrow-8              	 5000000	       382 ns/op	     328 B/op	       6 allocs/op
BenchmarkChunkRestConsumeExec10In1024Slice-8          	10000000	       137 ns/op	       0 B/op	       0 allocs/op
BenchmarkChunkResetConsumeExec10In32Slice-8           	10000000	       138 ns/op	       0 B/op	       0 allocs/op
BenchmarkChunkResetConsumeExec10InGrow-8              	10000000	       133 ns/op	       0 B/op	       0 allocs/op
PASS

PR's InGrow solution work "well" in both small result(10 per exec.Next) and big result(10000000) situtation(Renew&Reset).

Code changes

  • Has exported function/method change

Side effects

  • No effect for master configuration
  • But later PR will base on this.

Related changes

  • Need to be included in the release note

This change is Reviewable

@lysu lysu added the sig/execution SIG execution label Aug 23, 2018
@lysu
Copy link
Contributor Author

lysu commented Aug 23, 2018

/run-all-tests

})
}

// addVarLenColumn adds a variable length column with initial data capacity.
func (c *Chunk) addVarLenColumn(initCap int) {
func (c *Chunk) addVarLenColumn(elemLen int8, initCap int, old *column) {
guessElemLen := 4
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ guessElemeLen/ fakeElemLen ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no it's not fake, it is guess

nullBitmap: make([]byte, 0, initCap>>3),
elemLen: elemLen,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will this be better?

const VarElemLen = -1
elemLen: VarElemLen

func (c *Chunk) addVarLenColumn(initCap int) {
func (c *Chunk) addVarLenColumn(elemLen int8, initCap int, old *column) {
guessElemLen := 4
if old != nil && old.length != 0 {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a comment for this check.

})
}

// addColumnByFieldType adds a column by field type.
func (c *Chunk) addColumnByFieldType(fieldTp *types.FieldType, initCap int) {
numFixedBytes := getFixedLen(fieldTp)
func (c *Chunk) addColumnByFieldType(numFixedBytes int8, initCap int, old *column) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ addColumnByFieldType/ addColumnByElemLen ?

return newChunkByChunk(chk, nextCapacity)
}

func newChunkByChunk(old *Chunk, cap int) *Chunk {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this function is only used in Renew,
I think we can put the implementation in Renew.


// Returns a power of two size for the given target size.
func powerOf2Capacity(size, capLimit int) int {
n := size - 1
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we'd better extract
line 201~206 as to a util function like NextPowerOfTwo

n |= n >> 2
n |= n >> 4
n |= n >> 8
n |= n >> 16
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should n++ after this?

n |= n >> 8
n |= n >> 16
if n < 0 {
return 1
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we return capLimit if n <= 0?

if n < 0 {
return 1
}
if n >= capLimit {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems this would not happen since currSize < c.capacity. capLimit is always the power of 2?


// GrowReset resets the chunk and grow capacity if need.
// TODO: this method will be used in continue PR.
func (c *Chunk) GrowReset(capLimit int) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ GrowReset/ GrowAndReset?

}
for _, col := range c.columns {
if needResize {
col.eraseGrow(nextCapacity)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use newChunkByChunk here instead of col.eraseGrow?

@@ -30,6 +30,10 @@ type Chunk struct {
// numVirtualRows indicates the number of virtual rows, which have zero column.
// It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0".
numVirtualRows int
// capacity indicate the max number of rows that the chunk want to hold.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

want -> wants, indicate -> indicates

@@ -30,6 +30,10 @@ type Chunk struct {
// numVirtualRows indicates the number of virtual rows, which have zero column.
// It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0".
numVirtualRows int
// capacity indicate the max number of rows that the chunk want to hold.
// The accurate row count after executor.Next is guaranteed by executor implement.
// It is used only during executor's execution, and be grow by `Renew` or `Reset`.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

be grow -> is grown or just grown.

}

// New creates a new chunk.
// cap: the max number of rows that the chunk want to hold.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

want -> wants

@@ -60,31 +94,36 @@ func (c *Chunk) MemoryUsage() (sum int64) {
}

// addFixedLenColumn adds a fixed length column with elemLen and initial data capacity.
func (c *Chunk) addFixedLenColumn(elemLen, initCap int) {
func (c *Chunk) addFixedLenColumn(elemLen int8, initCap int) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to specify it's int8 not int?

return newChk
}

func resizeColumns(oldCol []*column, cap int) []*column {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not name it renewColumns?
resize implies modification on the old columns.

util/misc.go Outdated
const maxCapacity int32 = 1 << 30

// NextPowerOfTwo returns next power of two size for the given target cap.
func NextPowerOfTwo(cap int32) int32 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

1 << uint(bits.Len64(x)) will do

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems wrong when x = 2, 4, 8.... o.o

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then what about 1 << uint(bits.Len32(x-1)) ?

@@ -50,6 +50,7 @@ type column struct {
offsets []int32
data []byte
elemBuf []byte
elemLen int8
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this?
Isn't it can be inferred by elemBuf?

// and check `chk.NumRows == 0` to stop loop, so for last loop we can use a small chunk to check.
// but for Reset() situation in which the chunk keep reusing reallocate a smaller chunk is expensive in most situation.
//
func checkCapacity(c *Chunk, capLimit int, tryScaleDown bool) (needScaleUp bool, newCap int) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we define different calculateCapacity functions for Renew and Reset?
I think that would be easier to read.

)

// NewChunkWithCapacity creates a new chunk with field types and capacity.
func NewChunkWithCapacity(fields []*types.FieldType, cap int) *Chunk {
return New(fields, cap, cap) //FIX ME in continue PR, this keep every thing same.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the next PR, we remove this function and use New instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes~

// New creates a new chunk.
// cap: the max number of rows that the chunk wants to hold.
// capLimit: the max limit for max number of rows.
func New(fields []*types.FieldType, cap, capLimit int) *Chunk {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. the meaning of cap and capLimit makes people confused...`
  2. seems capLimit is not used in this function

Copy link
Contributor Author

@lysu lysu Aug 24, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. having any good advise for this two variable~~~?
  2. I make a bug in previous, address comment orz..., this funcation will check cap not over capLimit..fixed

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is capLimit equal to tidb_max_chunk_size in the production use cases?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, for current plan, it will be equal to tidb_max_chunk_size just like previous does.

@lysu
Copy link
Contributor Author

lysu commented Aug 24, 2018

@zz-jason @coocood @XuHuaiyu PTAL thx~

if col.isFixed() {
elemLen = len(col.elemBuf)
}
columns = append(columns, newColumn(elemLen, cap, col))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this way is more straightforward:

if col.isFixed() {
    columns = append(columns, newFixedLenColumn(len(col.elemBuf), initCap))
} else {
    columns = append(columns, newVarLenColumn(initCap, col))
}

// so, in first executor.Next we using a experience value --- 4(so it maybe make `runtime.growslice`)
// but in continue Next call we estimated length as AVG elemLen of previous call.
if old != nil && old.length != 0 {
estimatedElemLen = len(old.data) / old.length
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we set it larger, make it more room, so it's more unlikely to grow slice.

//
// this method will be used in call Executor#Next many times with a new chunk situation.
// so it will `calculateResetCap` to get a new cap then create a new chunk, whatever happens.
func Renew(chk *Chunk, capLimit int) *Chunk {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

capLimit can be obtained by chk.capacity, should we remove this argument?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wuu..it's not easy..capacity will be capcity in current round, capLimit will always equal to sessionVars.max_chunk_size, and as review meeting result we should not store capLimit in chunk and got it from sessionVars...so we need pass this argument into New/Renew/GrowAndReset call- -

@@ -112,12 +159,68 @@ func (c *Chunk) SetNumVirtualRows(numVirtualRows int) {
// Reset resets the chunk, so the memory it allocated can be reused.
// Make sure all the data in the chunk is not used anymore before you reuse this chunk.
func (c *Chunk) Reset() {

This comment was marked as resolved.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

got it~

return true, mathutil.Min(c.capacity*2, capLimit)
}

// calculateResetCap checks and return suitable capacity in next execution.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/calculateResetCap/calculateRenewCap/

// using next power of 2 for renew if currSize < cap
// using capacity limit if full but capacity already meet capLimit
// using oldcap x 2 if chk is full but less than capLimit
func calculateRenewCap(c *Chunk, capLimit int) int {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we only need calculateRenewCap, and calculateResetCap can be removed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this two method has some different

calculateResetCap: if previous return chk not full, will keep not grow.
calculateRenewCap: if previous returned chk is not full, will have chance give a smaller capacity then previous.

as #7473 (comment) advise I divide them into two.

@@ -642,3 +658,209 @@ func BenchmarkChunkMemoryUsage(b *testing.B) {
chk.MemoryUsage()
}
}

type mockExec struct {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a comment for the usage of thsi, if it's hard to reuse it in other tests
in this file, it'll be better to make the name more clearly.

@@ -30,24 +32,69 @@ type Chunk struct {
// numVirtualRows indicates the number of virtual rows, which have zero column.
// It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0".
numVirtualRows int
// capacity indicates the max number of rows that the chunk wants to hold.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think wants to hold may make the reader somewhat confused.
Can we change it to can hold here?

@@ -30,24 +32,69 @@ type Chunk struct {
// numVirtualRows indicates the number of virtual rows, which have zero column.
// It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0".
numVirtualRows int
// capacity indicates the max number of rows that the chunk wants to hold.
// The accurate row count after executor.Next is guaranteed by executor implement.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ implement/ implementation

//
// this method will be used in call Executor#Next many times with a new chunk situation.
// so it will `calculateResetCap` to get a new cap then create a new chunk, whatever happens.
func Renew(chk *Chunk, capLimit int) *Chunk {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we rename capLimit to maxChunkSize, which seems to be more readable.

return newChk
}

func renewColumns(oldCol []*column, cap int) []*column {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add a comment for this func.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

2333...IMHO, this's method is too simple, code is comment...ok..I will try to add comment

}
c.numVirtualRows = 0
}

// GrowAndReset resets the chunk and grow capacity if need.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/ if need/ if needed

// It works like:
// keep old value if capacity == capLimit or chk isn't full, and no need renew columns.
// old value x 2 if chk is full, and need renew columns.
func calculateResetCap(c *Chunk, capLimit int) (requireRenewCol bool, newCap int) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/calculateResetCap/getCapForReset
s/ calculateRenewCap/ getCapForRenew ?

// capLimit: the max limit for max number of rows.
//
// this method will be used in call Executor#Next many times with a new chunk situation.
// so it will `calculateResetCap` to get a new cap then create a new chunk, whatever happens.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

calculateRenewCap?

// chk: old chunk(often used in previous call).
// capLimit: the max limit for max number of rows.
//
// this method will be used in call Executor#Next many times with a new chunk situation.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/in/ in the situation when calling ....

// capLimit: the max limit for max number of rows.
//
// this method will be used in call Executor#Next many times with a new chunk situation.
// so it will `calculateResetCap` to get a new cap then create a new chunk, whatever happens.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what does whatever happens mean?

Copy link
Contributor

@XuHuaiyu XuHuaiyu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rest LGTM

// chk: old chunk(often used in previous call).
// maxChunkSize: the max limit for max number of rows.
//
// this method will be used in the situation when calling call Executor#Next many times with a new chunk situation.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove call and the situation at the end.

// newVarLenColumn creates a variable length column with initial data capacity.
func newVarLenColumn(initCap int, old *column) *column {
estimatedElemLen := 4
// for varLenColumn(e.g. varchar) we could not take the accuracy length of element,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/for/For

Copy link
Contributor

@XuHuaiyu XuHuaiyu left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

for _, f := range fields {
chk.addColumnByFieldType(f, cap)
elemLen := getFixedLen(f)
if elemLen != varElemLen {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about:

if elemLen == varElemLen {
	chk.columns = append(chk.columns, newVarLenColumn(cap, nil))
} else ...

// so, in first executor.Next we using a experience value --- 4(so it maybe make `runtime.growslice`)
// but in continue Next call we estimated length as AVG x 1.5 elemLen of previous call.
if old != nil && old.length != 0 {
estimatedElemLen = (len(old.data) + len(old.data)/2) / old.length
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why the estimated length for the var-length element is changed?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To reduce the probability of growing slice when appending string.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

len(old.data)/old.length is enough to estimated the average length of the strings to be appended in the new column, no need to moltiply 1.5.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@coocood @zz-jason PTAL... I have conitune PR wait this...

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

len(old.data) + len(old.data)/8 may be large enough

func calculateCapForRenew(c *Chunk, maxChunkSize int) int {
currSize := c.NumRows()
if currSize < c.capacity {
return NextPowerOfTwo(currSize)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we use NextPowerOfTwo(currSize) rather than c.capacity?

if c.capacity == maxChunkSize {
return maxChunkSize
}
return mathutil.Min(c.capacity*2, maxChunkSize)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we remove line 203~205? IMO, mathutil.Min(c.capacity*2, maxChunkSize) can ccover the case of c.capacity == maxChunkSize.


func BenchmarkChunkRenewConsumeExec10000000In1024Slice(b *testing.B) {
b.ReportAllocs()
chk := New([]*types.FieldType{{Tp: mysql.TypeLong}}, 1024, 1024)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can extract a common function for benchmarks, avoid duplicated code.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I found testing.B.Run is powerful

- add New() Renew() and GrowReset() method to chunk
- add eraseGrow() in column
- adaptor NewChunkWithCapacity() and Reset() works in chunk
@coocood
Copy link
Member

coocood commented Aug 28, 2018

LGTM

// so, in first executor.Next we using a experience value --- 8(so it maybe make `runtime.growslice`)
// but in continue Next call we estimated length as AVG x 1.125 elemLen of previous call.
if old != nil && old.length != 0 {
estimatedElemLen = (len(old.data) + len(old.data)/8) / old.length
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why 1.125?

Copy link
Contributor Author

@lysu lysu Aug 28, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's "magic const" by @coocood 🤦‍♂️ , the idea is avg maybe not very suitable for next call

// capacity indicates the max number of rows that the chunk can hold.
// The accurate row count after executor.Next is guaranteed by executor implementation.
// It is used only during executor's execution, and grow by `Renew` or `GrowAndReset`.
capacity int
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about simplifying the comment to:

// capacity indicates the max number of rows this chunk can hold.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zz-jason I prefer "this" in your sentence and it is better than "the".

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to use "this" instead of "the" to refer to the current thing.

chk.addColumnByFieldType(f, cap)
elemLen := getFixedLen(f)
if elemLen == varElemLen {
chk.columns = append(chk.columns, newVarLenColumn(cap, nil))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should firstly calculate the capacity according to the input parameters:

chk.capacity = mathutil.Min(cap, maxChunkSize)

then use chk.capacity to create columns.

//
// this method will be used in the situation when calling call Executor#Next many times with a new chunk.
// so it will `calculateCapacity` to get a new cap then create a new chunk.
func Renew(chk *Chunk, maxChunkSize int) *Chunk {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about change the comment to:

// Renew creates a new Chunk based on an existing Chunk. The newly created Chunk
// has the same data schema with the old Chunk, the capacity of the new Chunk
// might be doubled based on the capacity of the old Chunk and the maxChunkSize.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Jason's suggestion is good but a comma should be changed to a period:
// Renew creates a new Chunk based on an existing Chunk. The newly created Chunk
// has the same data schema with the old Chunk**. The** capacity of the new Chunk
// might be doubled based on the capacity of the old Chunk and the maxChunkSize.


// renewColumns recreates columns in chunk.
// create FixedLenColumn or VarLenColumn based on whether column is fixed.
func renewColumns(oldCol []*column, cap int) []*column {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about change the comment to:

// renewColumns creates the columns of a Chunk, the capacity of the newly
// created columns is equal to cap.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Jason's suggestion is good but a comma should be changed to a period:
// renewColumns creates the columns of a Chunk**. The** capacity of the newly
// created columns is equal to cap.

func (c *Chunk) addVarLenColumn(initCap int) {
c.columns = append(c.columns, &column{
// newVarLenColumn creates a variable length column with initial data capacity.
func newVarLenColumn(initCap int, old *column) *column {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about s/initCap/cap/?

}
c.numVirtualRows = 0
}

// GrowAndReset resets the chunk and grow capacity if needed.
// TODO: this method will be used in continue PR.
func (c *Chunk) GrowAndReset(maxChunkSize int) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about changing the comment to:

// GrowAndReset doubles the capacity of the Chunk, the doubled capacity should
// not be larger than maxChunkSize, resets the content of the Chunk.

// using capacity limit if full but capacity already meet maxChunkSize
// using oldcap x 2 if chk is full but less than maxChunkSize
// keep oldcap if chk isn't full
func calculateCapacity(c *Chunk, maxChunkSize int) int {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about s/calculateCapacity/reCalcCapacity/ and changing the comment to:

// reCalcCapacity calculates the capacity for another Chunk based the current
// Chunk. The new capacity is doubled only when the current Chunk is full.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on Jason's suggestion:

// reCalcCapacity calculates the capacity for another Chunk based on the current
// Chunk. The new capacity is doubled only when the current Chunk is full.

return mathutil.Min(c.capacity*2, maxChunkSize)
}

// Capacity return the max number of rows that chunk want to hold.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

// Capacity returns the capacity of the Chunk.

@zz-jason
Copy link
Member

@lilin90 Could you help us to refine some code comments in this PR? Thanks!

@lilin90
Copy link
Member

lilin90 commented Aug 28, 2018

@CaitinChen Pls help review the code comments. Thanks!

@@ -30,6 +31,10 @@ type Chunk struct {
// numVirtualRows indicates the number of virtual rows, which have zero column.
// It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0".
numVirtualRows int
// capacity indicates the max number of rows that the chunk can hold.
// The accurate row count after executor.Next is guaranteed by executor implementation.
// It is used only during executor's execution, and grow by `Renew` or `GrowAndReset`.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

grow -> grows

// capacity indicates the max number of rows that the chunk can hold.
// The accurate row count after executor.Next is guaranteed by executor implementation.
// It is used only during executor's execution, and grow by `Renew` or `GrowAndReset`.
capacity int
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zz-jason I prefer "this" in your sentence and it is better than "the".

// capacity indicates the max number of rows that the chunk can hold.
// The accurate row count after executor.Next is guaranteed by executor implementation.
// It is used only during executor's execution, and grow by `Renew` or `GrowAndReset`.
capacity int
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to use "this" instead of "the" to refer to the current thing.


// New creates a new chunk.
// cap: the max number of rows that the chunk can hold.
// maxChunkSize: the max limit for max number of rows.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

-> maxChunkSize: the limit for the max number of rows.
or
-> maxChunkSize: the max limit for the number of rows.

return chk
}

// Renew recreates a new chunk base on old chunk.
// chk: old chunk(often used in previous call).
// maxChunkSize: the max limit for max number of rows.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

-> maxChunkSize: the limit for the max number of rows.
or
-> maxChunkSize: the max limit for the number of rows.

@@ -39,15 +44,57 @@ const (

// NewChunkWithCapacity creates a new chunk with field types and capacity.
func NewChunkWithCapacity(fields []*types.FieldType, cap int) *Chunk {
return New(fields, cap, cap) //FIX ME in continue PR, this keep every thing same.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

// FIX ME in the following PR. Keep everything consistent?

// newVarLenColumn creates a variable length column with initial data capacity.
func newVarLenColumn(initCap int, old *column) *column {
estimatedElemLen := 8
// For varLenColumn(e.g. varchar) we could not take the accuracy length of element,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For varLenColumn (e.g. varchar), the accurate length of an element is unknown.

func newVarLenColumn(initCap int, old *column) *column {
estimatedElemLen := 8
// For varLenColumn(e.g. varchar) we could not take the accuracy length of element,
// so, in first executor.Next we using a experience value --- 8(so it maybe make `runtime.growslice`)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Therefore, in the first executor.Next we use an experience value -- 8 (so it may make runtime.growslice)

estimatedElemLen := 8
// For varLenColumn(e.g. varchar) we could not take the accuracy length of element,
// so, in first executor.Next we using a experience value --- 8(so it maybe make `runtime.growslice`)
// but in continue Next call we estimated length as AVG x 1.125 elemLen of previous call.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but in the following Next call we estimate the length as AVG x 1.125 elemLen of the previous call.

@@ -163,6 +163,9 @@ func (c *Codec) bytesToI32Slice(b []byte) (i32s []int32) {
return i32s
}

// varElemLn indicates column is variable length column.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • 如果:varElemLn 取值有几种不同的情况,有的情况下,该 column 是 variable length column;有的情况下,该 column 不是 variable length column,那么:
    varElemLn indicates whether this column is a variable length column.
  • 如果:varElemLn 取值只能等于 -1 (我看代码中显示是常量,应该是固定值吧),column 一直是 variable length column,那么:
    varElemLn indicates this column is a variable length column.

我倾向于后者

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not change the name?

@lysu
Copy link
Contributor Author

lysu commented Aug 28, 2018

PTAL @zz-jason 🍂

// Therefore, in the first executor.Next we use an experience value -- 8 (so it may make runtime.growslice)
// but in the following Next call we estimate the length as AVG x 1.125 elemLen of the previous call.
if old != nil && old.length != 0 {
estimatedElemLen = (len(old.data) + len(old.data)/8) / old.length
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why the estimatedElemLen is len(old.data)/old.length * 1.125?

Copy link
Member

@zz-jason zz-jason left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@lysu lysu merged commit 2fb3145 into pingcap:master Aug 28, 2018
@lysu lysu deleted the dev-chunk-add-grow-mechanism branch August 28, 2018 12:08
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
sig/execution SIG execution
Projects
None yet
Development

Successfully merging this pull request may close these issues.

9 participants