Skip to content

Commit

Permalink
executor: Improve the performance of appending not fixed columns (#20969
Browse files Browse the repository at this point in the history
)
  • Loading branch information
mmyj authored May 13, 2021
1 parent acf2e82 commit 2105317
Show file tree
Hide file tree
Showing 2 changed files with 80 additions and 1 deletion.
4 changes: 3 additions & 1 deletion util/chunk/chunk.go
Original file line number Diff line number Diff line change
Expand Up @@ -505,8 +505,10 @@ func (c *Chunk) Append(other *Chunk, begin, end int) {
} else {
beginOffset, endOffset := src.offsets[begin], src.offsets[end]
dst.data = append(dst.data, src.data[beginOffset:endOffset]...)
lastOffset := dst.offsets[len(dst.offsets)-1]
for i := begin; i < end; i++ {
dst.offsets = append(dst.offsets, dst.offsets[len(dst.offsets)-1]+src.offsets[i+1]-src.offsets[i])
lastOffset += src.offsets[i+1] - src.offsets[i]
dst.offsets = append(dst.offsets, lastOffset)
}
}
for i := begin; i < end; i++ {
Expand Down
77 changes: 77 additions & 0 deletions util/chunk/chunk_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1179,3 +1179,80 @@ func BenchmarkBatchAppendRows(b *testing.B) {
})
}
}

func BenchmarkAppendRows(b *testing.B) {
b.ReportAllocs()
rowChk := newChunk(8, 8, 0, 0)

for i := 0; i < 4096; i++ {
rowChk.AppendNull(0)
rowChk.AppendInt64(1, 1)
rowChk.AppendString(2, "abcd")
rowChk.AppendBytes(3, []byte("abcd"))
}

type testCaseConf struct {
batchSize int
}
testCaseConfs := []testCaseConf{
{batchSize: 2},
{batchSize: 8},
{batchSize: 16},
{batchSize: 100},
{batchSize: 1000},
{batchSize: 4000},
}

chk := newChunk(8, 8, 0, 0)
for _, conf := range testCaseConfs {
b.ResetTimer()
b.Run(fmt.Sprintf("row-%d", conf.batchSize), func(b *testing.B) {
for i := 0; i < b.N; i++ {
chk.Reset()
for j := 0; j < conf.batchSize; j++ {
chk.AppendRow(rowChk.GetRow(j))
}
}
})
b.ResetTimer()
b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) {
for i := 0; i < b.N; i++ {
chk.Reset()
chk.Append(rowChk, 0, conf.batchSize)
}
})
}
}

func BenchmarkAppend(b *testing.B) {
b.ReportAllocs()
rowChk := newChunk(0, 0)

for i := 0; i < 4096; i++ {
rowChk.AppendString(0, "abcd")
rowChk.AppendBytes(1, []byte("abcd"))
}

type testCaseConf struct {
batchSize int
}
testCaseConfs := []testCaseConf{
{batchSize: 2},
{batchSize: 8},
{batchSize: 16},
{batchSize: 100},
{batchSize: 1000},
{batchSize: 4000},
}

chk := newChunk(0, 0)
for _, conf := range testCaseConfs {
b.ResetTimer()
b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) {
for i := 0; i < b.N; i++ {
chk.Reset()
chk.Append(rowChk, 0, conf.batchSize)
}
})
}
}

0 comments on commit 2105317

Please sign in to comment.