From 063f0a4e3a7a20748ca409816c358bcc31161773 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 30 Dec 2019 11:06:27 +0800 Subject: [PATCH 01/43] merge sort --- executor/sort.go | 90 +++++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 81 insertions(+), 9 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 65129c8c7f8c9..8ee0eeef2fa09 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -62,6 +62,16 @@ type SortExec struct { partitionList []*chunk.ListInDisk // partitionRowPtrs store the disk-chunk index and row index for each row for partitions. partitionRowPtrs [][]chunk.RowPtr + + // sortRows + sortRows []chunk.Row + // sortRowsIndex store the partition index for each row. + sortRowsIndex []int + // partitionConsumedRows store the consumed rows num for each partition. + partitionConsumedRows []int + // heapSort use heap sort for spill disk. + heapSort *topNChunkHeapWithIndex + // exceeded indicates that records have exceeded memQuota during // adding this chunk and we should spill now. exceeded uint32 @@ -150,14 +160,8 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { } if e.alreadySpilled() { - for !req.IsFull() && e.Idx < len(e.partitionRowPtrs[0]) { - rowPtr := e.partitionRowPtrs[0][e.Idx] - row, err := e.partitionList[0].GetRow(rowPtr) - if err != nil { - return err - } - req.AppendRow(row) - e.Idx++ + if err := e.externalSorting(req); err != nil { + return err } } else { for !req.IsFull() && e.Idx < len(e.rowPtrs) { @@ -190,9 +194,77 @@ func (e *SortExec) prepareExternalSorting() (err error) { e.rowChunks = nil e.partitionList = append(e.partitionList, listInDisk) e.partitionRowPtrs = append(e.partitionRowPtrs, e.initPointersForListInDisk(listInDisk)) + e.sortRowsIndex = make([]int, len(e.partitionList)) + e.partitionConsumedRows = make([]int, len(e.partitionList)) + e.heapSort = nil return err } +type topNChunkHeapWithIndex struct { + *SortExec +} + +func (h *topNChunkHeapWithIndex) Less(i, j int) bool { + rowI := h.sortRows[i] + rowJ := h.sortRows[j] + return h.lessRow(rowI, rowJ) +} + +func (h *topNChunkHeapWithIndex) Len() int { + return len(h.sortRows) +} + +func (h *topNChunkHeapWithIndex) Push(x interface{}) { + // Should never be called. +} + +func (h *topNChunkHeapWithIndex) Pop() interface{} { + return nil +} + +func (h *topNChunkHeapWithIndex) Swap(i, j int) { + h.sortRows[i], h.sortRows[j] = h.sortRows[j], h.sortRows[i] + h.sortRowsIndex[i], h.sortRowsIndex[j] = h.sortRowsIndex[j], h.sortRowsIndex[i] +} + +func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { + if e.heapSort == nil { + e.heapSort = &topNChunkHeapWithIndex{e} + heap.Init(e.heapSort) + for i := 0; i < len(e.partitionList); i++ { + e.partitionConsumedRows[i] = 0 + row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) + if err != nil { + return err + } + e.sortRows = append(e.sortRows, row) + e.sortRowsIndex = append(e.sortRowsIndex, i) + heap.Fix(e.heapSort, 0) + } + } + + for !req.IsFull() && e.heapSort.Len() > 0 { + length := e.heapSort.Len() - 1 + heap.Pop(e.heapSort) + row, idx := e.sortRows[length], e.sortRowsIndex[length] + e.sortRows = e.sortRows[:length] + e.sortRowsIndex = e.sortRowsIndex[:length] + req.AppendRow(row) + e.partitionConsumedRows[idx]++ + + if e.partitionConsumedRows[idx] < len(e.partitionRowPtrs[idx]) { + row, err := e.partitionList[idx].GetRow(e.partitionRowPtrs[idx][e.partitionConsumedRows[idx]]) + if err != nil { + return err + } + e.sortRows = append(e.sortRows, row) + e.sortRowsIndex = append(e.sortRowsIndex, idx) + heap.Fix(e.heapSort, 0) + } + } + return nil +} + func (e *SortExec) fetchRowChunks(ctx context.Context) error { fields := retTypes(e) e.rowChunks = chunk.NewList(fields, e.initCap, e.maxChunkSize) @@ -216,7 +288,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { } } else { e.rowChunks.Add(chk) - if atomic.LoadUint32(&e.exceeded) == 1 { + if atomic.LoadUint32(&e.exceeded) == 0 { e.rowChunksInDisk, err = e.spillToDisk() if err != nil { return err From d82339230fcd2e069c916b4c6a33d7c31d4f6f28 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 30 Dec 2019 13:42:35 +0800 Subject: [PATCH 02/43] split partition --- executor/sort.go | 31 ++++++++++++++++++++++++------- util/chunk/disk.go | 5 +++++ 2 files changed, 29 insertions(+), 7 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 8ee0eeef2fa09..9dce163f9d2a1 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -178,9 +178,29 @@ func (e *SortExec) prepareExternalSorting() (err error) { e.buildKeyColumns() e.rowPtrsInDisk = e.initPointersForListInDisk(e.rowChunksInDisk) // partition sort - // Now only have one partition. - // The partition will be adjusted in the next pr. - err = e.readPartition(e.rowChunksInDisk, e.rowPtrsInDisk) + partStartPtr := 0 + partStartOffset := e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[partStartPtr]) + for i := 0; i < len(e.rowPtrsInDisk); i++ { + size := e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[i]) - partStartOffset + if size > e.ctx.GetSessionVars().MemQuotaQuery { + if err := e.generatePartition(partStartPtr, i); err != nil { + return err + } + partStartPtr = i + partStartOffset = e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[partStartPtr]) + } + } + if err := e.generatePartition(partStartPtr, len(e.rowPtrsInDisk)); err != nil { + return nil + } + e.sortRowsIndex = make([]int, 0, len(e.partitionList)) + e.partitionConsumedRows = make([]int, len(e.partitionList)) + e.heapSort = nil + return err +} + +func (e *SortExec) generatePartition(st, ed int) error { + err := e.readPartition(e.rowChunksInDisk, e.rowPtrsInDisk[st:ed]) if err != nil { return err } @@ -194,10 +214,7 @@ func (e *SortExec) prepareExternalSorting() (err error) { e.rowChunks = nil e.partitionList = append(e.partitionList, listInDisk) e.partitionRowPtrs = append(e.partitionRowPtrs, e.initPointersForListInDisk(listInDisk)) - e.sortRowsIndex = make([]int, len(e.partitionList)) - e.partitionConsumedRows = make([]int, len(e.partitionList)) - e.heapSort = nil - return err + return nil } type topNChunkHeapWithIndex struct { diff --git a/util/chunk/disk.go b/util/chunk/disk.go index b1a8c707f0ee2..ae33f065c4835 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -143,6 +143,11 @@ func (l *ListInDisk) NumChunks() int { return len(l.offsets) } +// GetOffsetOfRow returns the offset of a row in the ListInDisk. +func (l *ListInDisk) GetOffsetOfRow(ptr RowPtr) int64 { + return l.offsets[ptr.ChkIdx][ptr.RowIdx] +} + // Close releases the disk resource. func (l *ListInDisk) Close() error { if l.disk != nil { From aca577742720c2ec78ad4e13d873ed68c9236c79 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 30 Dec 2019 14:43:07 +0800 Subject: [PATCH 03/43] mem track --- executor/sort.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/executor/sort.go b/executor/sort.go index 9dce163f9d2a1..ddfc29e2f7065 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -98,7 +98,11 @@ func (e *SortExec) Close() error { e.partitionList = e.partitionList[:0] e.memTracker.Consume(int64(-8 * cap(e.rowPtrsInDisk))) + e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) + e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedRows))) e.rowPtrsInDisk = nil + e.sortRowsIndex = nil + e.partitionConsumedRows = nil for _, partitionPtrs := range e.partitionRowPtrs { e.memTracker.Consume(int64(-8 * cap(partitionPtrs))) } @@ -195,6 +199,8 @@ func (e *SortExec) prepareExternalSorting() (err error) { } e.sortRowsIndex = make([]int, 0, len(e.partitionList)) e.partitionConsumedRows = make([]int, len(e.partitionList)) + e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) + e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRows))) e.heapSort = nil return err } From ef80a75813229d68c3ea83cafa141974afe36bc3 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 30 Dec 2019 15:11:40 +0800 Subject: [PATCH 04/43] comments --- executor/sort.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/sort.go b/executor/sort.go index ddfc29e2f7065..7a4513aba18d8 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -63,7 +63,7 @@ type SortExec struct { // partitionRowPtrs store the disk-chunk index and row index for each row for partitions. partitionRowPtrs [][]chunk.RowPtr - // sortRows + // sortRows is used to maintain a heap. sortRows []chunk.Row // sortRowsIndex store the partition index for each row. sortRowsIndex []int From b22f83bed4a1b1e0eaf890c08bdacad103a9ad06 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 11:55:59 +0800 Subject: [PATCH 05/43] address heap comments --- executor/sort.go | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 7a4513aba18d8..9b78f944f600d 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -242,6 +242,8 @@ func (h *topNChunkHeapWithIndex) Push(x interface{}) { } func (h *topNChunkHeapWithIndex) Pop() interface{} { + h.sortRows = h.sortRows[:len(h.sortRows)-1] + h.sortRowsIndex = h.sortRowsIndex[:len(h.sortRowsIndex)-1] return nil } @@ -253,7 +255,6 @@ func (h *topNChunkHeapWithIndex) Swap(i, j int) { func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if e.heapSort == nil { e.heapSort = &topNChunkHeapWithIndex{e} - heap.Init(e.heapSort) for i := 0; i < len(e.partitionList); i++ { e.partitionConsumedRows[i] = 0 row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) @@ -262,16 +263,13 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { } e.sortRows = append(e.sortRows, row) e.sortRowsIndex = append(e.sortRowsIndex, i) - heap.Fix(e.heapSort, 0) } + heap.Init(e.heapSort) } for !req.IsFull() && e.heapSort.Len() > 0 { - length := e.heapSort.Len() - 1 - heap.Pop(e.heapSort) - row, idx := e.sortRows[length], e.sortRowsIndex[length] - e.sortRows = e.sortRows[:length] - e.sortRowsIndex = e.sortRowsIndex[:length] + row, idx := e.sortRows[0], e.sortRowsIndex[0] + heap.Remove(e.heapSort, 0) req.AppendRow(row) e.partitionConsumedRows[idx]++ From 2ccadc631c55fd67db010a06707c6ad3c448d330 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 22 Dec 2019 10:38:10 +0800 Subject: [PATCH 06/43] add action --- executor/sort.go | 41 +++++++++++++++++++++++++++++++++++++++ executor/sort_test.go | 45 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 86 insertions(+) create mode 100644 executor/sort_test.go diff --git a/executor/sort.go b/executor/sort.go index 9b78f944f600d..856942173ffcb 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -17,14 +17,18 @@ import ( "container/heap" "context" "fmt" + "go.uber.org/zap" "sort" + "sync" "sync/atomic" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/disk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/stringutil" ) @@ -291,6 +295,10 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks = chunk.NewList(fields, e.initCap, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + if config.GetGlobalConfig().OOMUseTmpStorage { + actionSpill := e.ActionSpill() + e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) + } for { chk := newFirstChunk(e.children[0]) err := Next(ctx, e.children[0], chk) @@ -445,6 +453,39 @@ func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { return rowChunksInDisk, nil } +func (e *SortExec) ActionSpill() memory.ActionOnExceed { + return &spillSortDiskAction{e: e} +} + +// spillSortDiskAction implements memory.ActionOnExceed for chunk.List. If +// the memory quota of a query is exceeded, spillSortDiskAction.Action is +// triggered. +type spillSortDiskAction struct { + once sync.Once + e *SortExec + fallbackAction memory.ActionOnExceed +} + +// Action sends a signal to trigger spillToDisk method of SortExec +// and if it is already triggered before, call its fallbackAction. +func (a *spillSortDiskAction) Action(t *memory.Tracker) { + if a.e.alreadySpilledSafe() { + if a.fallbackAction != nil { + a.fallbackAction.Action(t) + } + } + a.once.Do(func() { + atomic.StoreUint32(&a.e.exceeded, 1) + logutil.BgLogger().Info("memory exceeds quota, spill to disk now.", zap.String("memory", t.String())) + }) +} + +func (a *spillSortDiskAction) SetFallback(fallback memory.ActionOnExceed) { + a.fallbackAction = fallback +} + +func (a *spillSortDiskAction) SetLogHook(hook func(uint64)) {} + // TopNExec implements a Top-N algorithm and it is built from a SELECT statement with ORDER BY and LIMIT. // Instead of sorting all the rows fetched from the table, it keeps the Top-N elements only in a heap to reduce memory usage. type TopNExec struct { diff --git a/executor/sort_test.go b/executor/sort_test.go new file mode 100644 index 0000000000000..a1a6792793328 --- /dev/null +++ b/executor/sort_test.go @@ -0,0 +1,45 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/testkit" +) + +func (s *testSuite) TestSortInDisk(c *C) { + originCfg := config.GetGlobalConfig() + newConf := *originCfg + newConf.OOMUseTmpStorage = true + config.StoreGlobalConfig(&newConf) + defer config.StoreGlobalConfig(originCfg) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + sm := &mockSessionManager1{ + PS: make([]*util.ProcessInfo, 0), + } + tk.Se.SetSessionManager(sm) + s.domain.ExpensiveQueryHandle().SetSessionManager(sm) + + tk.MustExec("set @@tidb_mem_quota_query=1;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int, c2 int)") + tk.MustExec("insert into t values(1,5),(2,4),(3,3),(4,2),(5,1)") + result := tk.MustQuery("select * from t order by c2") + result.Check(testkit.Rows("5 1", "4 2", "3 3", "2 4", "1 5")) +} From f6c4ebe7dac4ce20227f9988e1bc6eb3b1cf3491 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 11:00:32 +0800 Subject: [PATCH 07/43] add comments --- executor/sort.go | 1 + 1 file changed, 1 insertion(+) diff --git a/executor/sort.go b/executor/sort.go index 856942173ffcb..90796366ca2c0 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -453,6 +453,7 @@ func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { return rowChunksInDisk, nil } +// ActionSpill returns a memory.ActionOnExceed for spilling over to disk. func (e *SortExec) ActionSpill() memory.ActionOnExceed { return &spillSortDiskAction{e: e} } From d83e9af09064ea77df21b408168c6bab52d70124 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 15:01:18 +0800 Subject: [PATCH 08/43] address comments --- executor/sort.go | 109 ++++++++++++----------------------------------- 1 file changed, 27 insertions(+), 82 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 90796366ca2c0..e7cccf97bda9e 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -75,6 +75,8 @@ type SortExec struct { partitionConsumedRows []int // heapSort use heap sort for spill disk. heapSort *topNChunkHeapWithIndex + // action + spillAction *spillSortDiskAction // exceeded indicates that records have exceeded memQuota during // adding this chunk and we should spill now. @@ -85,7 +87,7 @@ type SortExec struct { // Close implements the Executor Close interface. func (e *SortExec) Close() error { - if e.alreadySpilled() { + if e.alreadySpilledSafe() { if e.rowChunksInDisk != nil { if err := e.rowChunksInDisk.Close(); err != nil { return err @@ -148,26 +150,24 @@ func (e *SortExec) Open(ctx context.Context) error { func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { req.Reset() if !e.fetched { + e.initCompareFuncs() + e.buildKeyColumns() err := e.fetchRowChunks(ctx) if err != nil { return err } - if e.alreadySpilled() { + if e.alreadySpilledSafe() { err = e.prepareExternalSorting() if err != nil { return err } - e.fetched = true - } else { - e.initPointers() - e.initCompareFuncs() - e.buildKeyColumns() - sort.Slice(e.rowPtrs, e.keyColumnsLess) - e.fetched = true } + e.initPointers() + sort.Slice(e.rowPtrs, e.keyColumnsLess) + e.fetched = true } - if e.alreadySpilled() { + if e.alreadySpilledSafe() { if err := e.externalSorting(req); err != nil { return err } @@ -182,25 +182,6 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { } func (e *SortExec) prepareExternalSorting() (err error) { - e.initCompareFuncs() - e.buildKeyColumns() - e.rowPtrsInDisk = e.initPointersForListInDisk(e.rowChunksInDisk) - // partition sort - partStartPtr := 0 - partStartOffset := e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[partStartPtr]) - for i := 0; i < len(e.rowPtrsInDisk); i++ { - size := e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[i]) - partStartOffset - if size > e.ctx.GetSessionVars().MemQuotaQuery { - if err := e.generatePartition(partStartPtr, i); err != nil { - return err - } - partStartPtr = i - partStartOffset = e.rowChunksInDisk.GetOffsetOfRow(e.rowPtrsInDisk[partStartPtr]) - } - } - if err := e.generatePartition(partStartPtr, len(e.rowPtrsInDisk)); err != nil { - return nil - } e.sortRowsIndex = make([]int, 0, len(e.partitionList)) e.partitionConsumedRows = make([]int, len(e.partitionList)) e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) @@ -209,19 +190,14 @@ func (e *SortExec) prepareExternalSorting() (err error) { return err } -func (e *SortExec) generatePartition(st, ed int) error { - err := e.readPartition(e.rowChunksInDisk, e.rowPtrsInDisk[st:ed]) - if err != nil { - return err - } +func (e *SortExec) generatePartition() error { e.initPointers() sort.Slice(e.rowPtrs, e.keyColumnsLess) listInDisk, err := e.spillToDiskByRowPtr() if err != nil { return err } - e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) - e.rowChunks = nil + e.rowChunks.Reset() e.partitionList = append(e.partitionList, listInDisk) e.partitionRowPtrs = append(e.partitionRowPtrs, e.initPointersForListInDisk(listInDisk)) return nil @@ -309,25 +285,19 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if rowCount == 0 { break } - if e.alreadySpilled() { - // append chk to disk. - err := e.rowChunksInDisk.Add(chk) + e.rowChunks.Add(chk) + if atomic.LoadUint32(&e.exceeded) == 1 { + err := e.generatePartition() if err != nil { return err } - } else { - e.rowChunks.Add(chk) - if atomic.LoadUint32(&e.exceeded) == 0 { - e.rowChunksInDisk, err = e.spillToDisk() - if err != nil { - return err - } - e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) - e.rowChunks = nil // GC its internal chunks. - atomic.StoreUint32(&e.spilled, 1) - } + atomic.StoreUint32(&e.spilled, 1) + e.spillAction.reset() } } + if e.alreadySpilledSafe() && e.rowChunks.Len() != 0 { + return e.generatePartition() + } return nil } @@ -397,40 +367,9 @@ func (e *SortExec) keyColumnsLess(i, j int) bool { return e.lessRow(rowI, rowJ) } -func (e *SortExec) readPartition(disk *chunk.ListInDisk, rowPtrs []chunk.RowPtr) error { - e.rowChunks = chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) - e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - for _, rowPtr := range rowPtrs { - rowPtr, err := disk.GetRow(rowPtr) - if err != nil { - return err - } - e.rowChunks.AppendRow(rowPtr) - } - return nil -} - -// alreadySpilled indicates that records have spilled out into disk. -func (e *SortExec) alreadySpilled() bool { return e.rowChunksInDisk != nil } - // alreadySpilledSafe indicates that records have spilled out into disk. It's thread-safe. func (e *SortExec) alreadySpilledSafe() bool { return atomic.LoadUint32(&e.spilled) == 1 } -func (e *SortExec) spillToDisk() (disk *chunk.ListInDisk, err error) { - N := e.rowChunks.NumChunks() - rowChunksInDisk := chunk.NewListInDisk(e.retFieldTypes) - rowChunksInDisk.GetDiskTracker().AttachTo(e.diskTracker) - for i := 0; i < N; i++ { - chk := e.rowChunks.GetChunk(i) - err = rowChunksInDisk.Add(chk) - if err != nil { - return nil, err - } - } - return rowChunksInDisk, nil -} - func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { rowChunksInDisk := chunk.NewListInDisk(e.retFieldTypes) rowChunksInDisk.GetDiskTracker().AttachTo(e.diskTracker) @@ -455,7 +394,8 @@ func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { // ActionSpill returns a memory.ActionOnExceed for spilling over to disk. func (e *SortExec) ActionSpill() memory.ActionOnExceed { - return &spillSortDiskAction{e: e} + e.spillAction = &spillSortDiskAction{e: e} + return e.spillAction } // spillSortDiskAction implements memory.ActionOnExceed for chunk.List. If @@ -487,6 +427,11 @@ func (a *spillSortDiskAction) SetFallback(fallback memory.ActionOnExceed) { func (a *spillSortDiskAction) SetLogHook(hook func(uint64)) {} +func (a *spillSortDiskAction) reset() { + atomic.StoreUint32(&a.e.exceeded, 0) + a.once = sync.Once{} +} + // TopNExec implements a Top-N algorithm and it is built from a SELECT statement with ORDER BY and LIMIT. // Instead of sorting all the rows fetched from the table, it keeps the Top-N elements only in a heap to reduce memory usage. type TopNExec struct { From 525d7796f84796acdfc2a1382d50427cb19614a0 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 15:43:45 +0800 Subject: [PATCH 09/43] add some test --- executor/sort_test.go | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/executor/sort_test.go b/executor/sort_test.go index a1a6792793328..3f6ad5794bb5e 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -14,6 +14,8 @@ package executor_test import ( + "fmt" + . "github.com/pingcap/check" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util" @@ -37,9 +39,16 @@ func (s *testSuite) TestSortInDisk(c *C) { s.domain.ExpensiveQueryHandle().SetSessionManager(sm) tk.MustExec("set @@tidb_mem_quota_query=1;") + tk.MustExec("set @@tidb_max_chunk_size=32;") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - tk.MustExec("insert into t values(1,5),(2,4),(3,3),(4,2),(5,1)") - result := tk.MustQuery("select * from t order by c2") - result.Check(testkit.Rows("5 1", "4 2", "3 3", "2 4", "1 5")) + tk.MustExec("create table t(c1 int)") + for i := 0; i < 5; i++ { + for j := i; j < 1024; j += 5 { + tk.MustExec(fmt.Sprintf("insert into t values(%v)", j)) + } + } + result := tk.MustQuery("select * from t order by c1") + for i := 0; i < 1024; i++ { + c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) + } } From 19ec95a3c72bc17abeeca7fdb31c0ae2e0a57bb6 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 16:09:10 +0800 Subject: [PATCH 10/43] remove useless field --- executor/sort.go | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index e7cccf97bda9e..9c713d93fa1b4 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -58,10 +58,6 @@ type SortExec struct { memTracker *memory.Tracker diskTracker *disk.Tracker - // rowChunksInDisk is the chunks to store row values in disk. - rowChunksInDisk *chunk.ListInDisk - // rowPtrsInDisk store the disk-chunk index and row index for each row. - rowPtrsInDisk []chunk.RowPtr // partitionList is the chunks to store row values in disk for partitions. partitionList []*chunk.ListInDisk // partitionRowPtrs store the disk-chunk index and row index for each row for partitions. @@ -75,7 +71,7 @@ type SortExec struct { partitionConsumedRows []int // heapSort use heap sort for spill disk. heapSort *topNChunkHeapWithIndex - // action + // spillAction save the spill action for the Sort Executor. spillAction *spillSortDiskAction // exceeded indicates that records have exceeded memQuota during @@ -88,11 +84,6 @@ type SortExec struct { // Close implements the Executor Close interface. func (e *SortExec) Close() error { if e.alreadySpilledSafe() { - if e.rowChunksInDisk != nil { - if err := e.rowChunksInDisk.Close(); err != nil { - return err - } - } for _, chunkInDisk := range e.partitionList { if chunkInDisk != nil { if err := chunkInDisk.Close(); err != nil { @@ -100,13 +91,10 @@ func (e *SortExec) Close() error { } } } - e.rowChunksInDisk = nil e.partitionList = e.partitionList[:0] - e.memTracker.Consume(int64(-8 * cap(e.rowPtrsInDisk))) e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedRows))) - e.rowPtrsInDisk = nil e.sortRowsIndex = nil e.partitionConsumedRows = nil for _, partitionPtrs := range e.partitionRowPtrs { @@ -139,8 +127,6 @@ func (e *SortExec) Open(ctx context.Context) error { } e.exceeded = 0 e.spilled = 0 - e.rowChunksInDisk = nil - e.rowPtrsInDisk = e.rowPtrsInDisk[:0] e.partitionList = e.partitionList[:0] e.partitionRowPtrs = e.partitionRowPtrs[:0] return e.children[0].Open(ctx) From 76af1d80e9bdd63f75481265cd5385db303d6c27 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 16:58:07 +0800 Subject: [PATCH 11/43] addres comments --- executor/sort.go | 30 ++++++++++++++++++++---------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 9c713d93fa1b4..73c26797580c3 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -168,8 +168,8 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { } func (e *SortExec) prepareExternalSorting() (err error) { - e.sortRowsIndex = make([]int, 0, len(e.partitionList)) - e.partitionConsumedRows = make([]int, len(e.partitionList)) + e.sortRowsIndex = make([]int, 0, len(e.partitionList)+1) + e.partitionConsumedRows = make([]int, len(e.partitionList)+1) e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRows))) e.heapSort = nil @@ -230,6 +230,11 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { e.sortRows = append(e.sortRows, row) e.sortRowsIndex = append(e.sortRowsIndex, i) } + if len(e.rowPtrs) != 0 { + e.partitionConsumedRows[len(e.partitionList)] = 0 + e.sortRows = append(e.sortRows, e.rowChunks.GetRow(e.rowPtrs[0])) + e.sortRowsIndex = append(e.sortRowsIndex, len(e.partitionList)) + } heap.Init(e.heapSort) } @@ -239,15 +244,23 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { req.AppendRow(row) e.partitionConsumedRows[idx]++ - if e.partitionConsumedRows[idx] < len(e.partitionRowPtrs[idx]) { - row, err := e.partitionList[idx].GetRow(e.partitionRowPtrs[idx][e.partitionConsumedRows[idx]]) + if idx == len(e.partitionList) { + if e.partitionConsumedRows[idx] >= len(e.rowPtrs) { + continue + } + row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRows[idx]]) + } else { + if e.partitionConsumedRows[idx] >= len(e.partitionRowPtrs[idx]) { + continue + } + row, err = e.partitionList[idx].GetRow(e.partitionRowPtrs[idx][e.partitionConsumedRows[idx]]) if err != nil { return err } - e.sortRows = append(e.sortRows, row) - e.sortRowsIndex = append(e.sortRowsIndex, idx) - heap.Fix(e.heapSort, 0) } + e.sortRows = append(e.sortRows, row) + e.sortRowsIndex = append(e.sortRowsIndex, idx) + heap.Fix(e.heapSort, 0) } return nil } @@ -281,9 +294,6 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.spillAction.reset() } } - if e.alreadySpilledSafe() && e.rowChunks.Len() != 0 { - return e.generatePartition() - } return nil } From 556a8589d4c46bdd10aa497406c1a1445dae31ea Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 31 Dec 2019 17:50:34 +0800 Subject: [PATCH 12/43] fix --- executor/sort.go | 6 +++++- executor/sort_test.go | 6 ++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/executor/sort.go b/executor/sort.go index 73c26797580c3..040213f9c91e9 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -183,7 +183,10 @@ func (e *SortExec) generatePartition() error { if err != nil { return err } - e.rowChunks.Reset() + e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) + e.rowChunks = chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) + e.rowChunks.GetMemTracker().AttachTo(e.memTracker) + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) e.partitionList = append(e.partitionList, listInDisk) e.partitionRowPtrs = append(e.partitionRowPtrs, e.initPointersForListInDisk(listInDisk)) return nil @@ -286,6 +289,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { } e.rowChunks.Add(chk) if atomic.LoadUint32(&e.exceeded) == 1 { + fmt.Println(e.rowChunks.Len()) err := e.generatePartition() if err != nil { return err diff --git a/executor/sort_test.go b/executor/sort_test.go index 3f6ad5794bb5e..259bf7a48fe98 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -51,4 +51,10 @@ func (s *testSuite) TestSortInDisk(c *C) { for i := 0; i < 1024; i++ { c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) } + + tk.MustExec("set @@tidb_mem_quota_query=500;") + result = tk.MustQuery("select * from t order by c1") + for i := 0; i < 1024; i++ { + c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) + } } From 90e8081d7d9dd06b62604e0d82d0033f9d0d4e7b Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 11:00:19 +0800 Subject: [PATCH 13/43] address comments --- executor/sort.go | 2 +- util/memory/tracker.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 040213f9c91e9..cfc274582f00c 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -183,7 +183,7 @@ func (e *SortExec) generatePartition() error { if err != nil { return err } - e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) + e.memTracker.ReplaceChild(e.rowChunks.GetMemTracker(), nil) e.rowChunks = chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 400c91c8bac61..ffb754037f937 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -133,7 +133,7 @@ func (t *Tracker) remove(oldChild *Tracker) { continue } - atomic.AddInt64(&t.bytesConsumed, -oldChild.BytesConsumed()) + t.Consume(-oldChild.BytesConsumed()) oldChild.parent = nil t.mu.children = append(t.mu.children[:i], t.mu.children[i+1:]...) break From 6483d740dd2b5b8c43f90e4de62c2283bee6bcc2 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 11:05:34 +0800 Subject: [PATCH 14/43] fix --- executor/sort.go | 1 - executor/sort_test.go | 6 ------ util/chunk/disk.go | 5 ----- 3 files changed, 12 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index cfc274582f00c..63aa249825563 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -289,7 +289,6 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { } e.rowChunks.Add(chk) if atomic.LoadUint32(&e.exceeded) == 1 { - fmt.Println(e.rowChunks.Len()) err := e.generatePartition() if err != nil { return err diff --git a/executor/sort_test.go b/executor/sort_test.go index 259bf7a48fe98..3f6ad5794bb5e 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -51,10 +51,4 @@ func (s *testSuite) TestSortInDisk(c *C) { for i := 0; i < 1024; i++ { c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) } - - tk.MustExec("set @@tidb_mem_quota_query=500;") - result = tk.MustQuery("select * from t order by c1") - for i := 0; i < 1024; i++ { - c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) - } } diff --git a/util/chunk/disk.go b/util/chunk/disk.go index ae33f065c4835..b1a8c707f0ee2 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -143,11 +143,6 @@ func (l *ListInDisk) NumChunks() int { return len(l.offsets) } -// GetOffsetOfRow returns the offset of a row in the ListInDisk. -func (l *ListInDisk) GetOffsetOfRow(ptr RowPtr) int64 { - return l.offsets[ptr.ChkIdx][ptr.RowIdx] -} - // Close releases the disk resource. func (l *ListInDisk) Close() error { if l.disk != nil { From bd11fd7c9759dc164da17a88f436d00951459231 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 14:13:00 +0800 Subject: [PATCH 15/43] optimize --- executor/sort.go | 48 ++++++++++++++----------------------------- executor/sort_test.go | 6 ++++-- util/chunk/disk.go | 13 ++++++++++++ 3 files changed, 32 insertions(+), 35 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 63aa249825563..853d1149cffc1 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -60,15 +60,13 @@ type SortExec struct { // partitionList is the chunks to store row values in disk for partitions. partitionList []*chunk.ListInDisk - // partitionRowPtrs store the disk-chunk index and row index for each row for partitions. - partitionRowPtrs [][]chunk.RowPtr // sortRows is used to maintain a heap. sortRows []chunk.Row // sortRowsIndex store the partition index for each row. sortRowsIndex []int // partitionConsumedRows store the consumed rows num for each partition. - partitionConsumedRows []int + partitionConsumedRowsPtr []chunk.RowPtr // heapSort use heap sort for spill disk. heapSort *topNChunkHeapWithIndex // spillAction save the spill action for the Sort Executor. @@ -94,13 +92,9 @@ func (e *SortExec) Close() error { e.partitionList = e.partitionList[:0] e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedRows))) + e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedRowsPtr))) e.sortRowsIndex = nil - e.partitionConsumedRows = nil - for _, partitionPtrs := range e.partitionRowPtrs { - e.memTracker.Consume(int64(-8 * cap(partitionPtrs))) - } - e.partitionRowPtrs = nil + e.partitionConsumedRowsPtr = nil } if e.rowChunks != nil { e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) @@ -128,7 +122,6 @@ func (e *SortExec) Open(ctx context.Context) error { e.exceeded = 0 e.spilled = 0 e.partitionList = e.partitionList[:0] - e.partitionRowPtrs = e.partitionRowPtrs[:0] return e.children[0].Open(ctx) } @@ -169,9 +162,9 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { func (e *SortExec) prepareExternalSorting() (err error) { e.sortRowsIndex = make([]int, 0, len(e.partitionList)+1) - e.partitionConsumedRows = make([]int, len(e.partitionList)+1) + e.partitionConsumedRowsPtr = make([]chunk.RowPtr, len(e.partitionList)+1) e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRows))) + e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRowsPtr))) e.heapSort = nil return err } @@ -188,7 +181,6 @@ func (e *SortExec) generatePartition() error { e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) e.partitionList = append(e.partitionList, listInDisk) - e.partitionRowPtrs = append(e.partitionRowPtrs, e.initPointersForListInDisk(listInDisk)) return nil } @@ -225,8 +217,8 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if e.heapSort == nil { e.heapSort = &topNChunkHeapWithIndex{e} for i := 0; i < len(e.partitionList); i++ { - e.partitionConsumedRows[i] = 0 - row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) + e.partitionConsumedRowsPtr[i] = chunk.RowPtr{ChkIdx: 0, RowIdx: 0} + row, err := e.partitionList[i].GetRow(e.partitionConsumedRowsPtr[i]) if err != nil { return err } @@ -234,7 +226,7 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { e.sortRowsIndex = append(e.sortRowsIndex, i) } if len(e.rowPtrs) != 0 { - e.partitionConsumedRows[len(e.partitionList)] = 0 + e.partitionConsumedRowsPtr[len(e.partitionList)] = chunk.RowPtr{ChkIdx: 0, RowIdx: 0} e.sortRows = append(e.sortRows, e.rowChunks.GetRow(e.rowPtrs[0])) e.sortRowsIndex = append(e.sortRowsIndex, len(e.partitionList)) } @@ -245,18 +237,19 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { row, idx := e.sortRows[0], e.sortRowsIndex[0] heap.Remove(e.heapSort, 0) req.AppendRow(row) - e.partitionConsumedRows[idx]++ - if idx == len(e.partitionList) { - if e.partitionConsumedRows[idx] >= len(e.rowPtrs) { + e.partitionConsumedRowsPtr[idx].RowIdx++ + if e.partitionConsumedRowsPtr[idx].RowIdx >= uint32(len(e.rowPtrs)) { continue } - row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRows[idx]]) + row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRowsPtr[idx].RowIdx]) } else { - if e.partitionConsumedRows[idx] >= len(e.partitionRowPtrs[idx]) { + rowPtr, ok := e.partitionList[idx].GetNextRowPtr(e.partitionConsumedRowsPtr[idx]) + if !ok { continue } - row, err = e.partitionList[idx].GetRow(e.partitionRowPtrs[idx][e.partitionConsumedRows[idx]]) + e.partitionConsumedRowsPtr[idx] = rowPtr + row, err = e.partitionList[idx].GetRow(rowPtr) if err != nil { return err } @@ -316,17 +309,6 @@ func (e *SortExec) initPointers() { e.memTracker.Consume(int64(8 * cap(e.rowPtrs))) } -func (e *SortExec) initPointersForListInDisk(disk *chunk.ListInDisk) []chunk.RowPtr { - rowPtrsInDisk := make([]chunk.RowPtr, 0) - for chkIdx := 0; chkIdx < disk.NumChunks(); chkIdx++ { - for rowIdx := 0; rowIdx < disk.NumRowsOfChunk(chkIdx); rowIdx++ { - rowPtrsInDisk = append(rowPtrsInDisk, chunk.RowPtr{ChkIdx: uint32(chkIdx), RowIdx: uint32(rowIdx)}) - } - } - e.memTracker.Consume(int64(8 * len(rowPtrsInDisk))) - return rowPtrsInDisk -} - func (e *SortExec) initCompareFuncs() { e.keyCmpFuncs = make([]chunk.CompareFunc, len(e.ByItems)) for i := range e.ByItems { diff --git a/executor/sort_test.go b/executor/sort_test.go index 3f6ad5794bb5e..9cafcefd8da2d 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -41,14 +41,16 @@ func (s *testSuite) TestSortInDisk(c *C) { tk.MustExec("set @@tidb_mem_quota_query=1;") tk.MustExec("set @@tidb_max_chunk_size=32;") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int)") + tk.MustExec("create table t(c1 int, c2 int, c3 int)") for i := 0; i < 5; i++ { for j := i; j < 1024; j += 5 { - tk.MustExec(fmt.Sprintf("insert into t values(%v)", j)) + tk.MustExec(fmt.Sprintf("insert into t values(%v, %v, %v)", j, j, j)) } } result := tk.MustQuery("select * from t order by c1") for i := 0; i < 1024; i++ { c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) + c.Assert(result.Rows()[i][1].(string), Equals, fmt.Sprint(i)) + c.Assert(result.Rows()[i][2].(string), Equals, fmt.Sprint(i)) } } diff --git a/util/chunk/disk.go b/util/chunk/disk.go index b1a8c707f0ee2..894a07607a19f 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -133,6 +133,19 @@ func (l *ListInDisk) GetRow(ptr RowPtr) (row Row, err error) { return row, err } +// GetNextRowPtr gets the next RowPtr for the ListInDisk by given RowPtr. +func (l *ListInDisk) GetNextRowPtr(ptr RowPtr) (RowPtr, bool) { + ptr.RowIdx++ + if ptr.RowIdx == uint32(l.NumRowsOfChunk(int(ptr.ChkIdx))) { + ptr.ChkIdx++ + ptr.RowIdx = 0 + } + if ptr.ChkIdx == uint32(l.NumChunks()) { + return ptr, false + } + return ptr, true +} + // NumRowsOfChunk returns the number of rows of a chunk in the ListInDisk. func (l *ListInDisk) NumRowsOfChunk(chkID int) int { return len(l.offsets[chkID]) From 7af49465312a12e73ae1d123640658abbca10cf9 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 14:38:31 +0800 Subject: [PATCH 16/43] fix --- executor/sort.go | 16 ++++++++++++++-- util/chunk/disk.go | 13 ------------- 2 files changed, 14 insertions(+), 15 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 853d1149cffc1..075bf32b4edff 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -244,7 +244,7 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { } row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRowsPtr[idx].RowIdx]) } else { - rowPtr, ok := e.partitionList[idx].GetNextRowPtr(e.partitionConsumedRowsPtr[idx]) + rowPtr, ok := getNextRowPtr(e.partitionList[idx], e.partitionConsumedRowsPtr[idx]) if !ok { continue } @@ -256,11 +256,23 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { } e.sortRows = append(e.sortRows, row) e.sortRowsIndex = append(e.sortRowsIndex, idx) - heap.Fix(e.heapSort, 0) + heap.Fix(e.heapSort, e.heapSort.Len()-1) } return nil } +func getNextRowPtr(l *chunk.ListInDisk, ptr chunk.RowPtr) (chunk.RowPtr, bool) { + ptr.RowIdx++ + if ptr.RowIdx == uint32(l.NumRowsOfChunk(int(ptr.ChkIdx))) { + ptr.ChkIdx++ + ptr.RowIdx = 0 + } + if ptr.ChkIdx == uint32(l.NumChunks()) { + return ptr, false + } + return ptr, true +} + func (e *SortExec) fetchRowChunks(ctx context.Context) error { fields := retTypes(e) e.rowChunks = chunk.NewList(fields, e.initCap, e.maxChunkSize) diff --git a/util/chunk/disk.go b/util/chunk/disk.go index 894a07607a19f..b1a8c707f0ee2 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -133,19 +133,6 @@ func (l *ListInDisk) GetRow(ptr RowPtr) (row Row, err error) { return row, err } -// GetNextRowPtr gets the next RowPtr for the ListInDisk by given RowPtr. -func (l *ListInDisk) GetNextRowPtr(ptr RowPtr) (RowPtr, bool) { - ptr.RowIdx++ - if ptr.RowIdx == uint32(l.NumRowsOfChunk(int(ptr.ChkIdx))) { - ptr.ChkIdx++ - ptr.RowIdx = 0 - } - if ptr.ChkIdx == uint32(l.NumChunks()) { - return ptr, false - } - return ptr, true -} - // NumRowsOfChunk returns the number of rows of a chunk in the ListInDisk. func (l *ListInDisk) NumRowsOfChunk(chkID int) int { return len(l.offsets[chkID]) From 37730235271f761fdaf037aabc2e193918386b97 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 17:02:20 +0800 Subject: [PATCH 17/43] add comments --- executor/sort.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/executor/sort.go b/executor/sort.go index 075bf32b4edff..c100b45592b77 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -126,6 +126,13 @@ func (e *SortExec) Open(ctx context.Context) error { } // Next implements the Executor Next interface. +// Sort constructs the result following these step: +// 1. Read as mush as rows into memory. +// 2. If memory quota is triggered, sort these rows in memory and put them into disk as partition 1, then reset +// the memory quota trigger and return to step 1 +// 3. If memory quota is not triggered and child is consumed, sort these rows in memory as partition N merge sort +// if the count of partitions is larger than 1. +// 4. If there is only one partition in step 4, it works just like in-memory sort before. func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { req.Reset() if !e.fetched { From 86f3f83eb9a619802a27400cfc1ae4b25ad8c43b Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 2 Jan 2020 17:19:00 +0800 Subject: [PATCH 18/43] address comments --- executor/sort.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index c100b45592b77..4acf772e93a53 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -242,17 +242,18 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { for !req.IsFull() && e.heapSort.Len() > 0 { row, idx := e.sortRows[0], e.sortRowsIndex[0] - heap.Remove(e.heapSort, 0) req.AppendRow(row) if idx == len(e.partitionList) { e.partitionConsumedRowsPtr[idx].RowIdx++ if e.partitionConsumedRowsPtr[idx].RowIdx >= uint32(len(e.rowPtrs)) { + heap.Remove(e.heapSort, 0) continue } row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRowsPtr[idx].RowIdx]) } else { rowPtr, ok := getNextRowPtr(e.partitionList[idx], e.partitionConsumedRowsPtr[idx]) if !ok { + heap.Remove(e.heapSort, 0) continue } e.partitionConsumedRowsPtr[idx] = rowPtr @@ -261,9 +262,8 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { return err } } - e.sortRows = append(e.sortRows, row) - e.sortRowsIndex = append(e.sortRowsIndex, idx) - heap.Fix(e.heapSort, e.heapSort.Len()-1) + e.sortRows[0] = row + heap.Fix(e.heapSort, 0) } return nil } From 0a481cb1aaeb72ae7017d91bc308f8c7cbb17a78 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 7 Jan 2020 11:14:47 +0800 Subject: [PATCH 19/43] row container --- executor/sort.go | 78 ++++++++++++++------------------ util/chunk/row_container.go | 7 +-- util/chunk/row_container_test.go | 2 +- 3 files changed, 40 insertions(+), 47 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 4acf772e93a53..9638dd5a0fcc1 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -58,8 +58,8 @@ type SortExec struct { memTracker *memory.Tracker diskTracker *disk.Tracker - // partitionList is the chunks to store row values in disk for partitions. - partitionList []*chunk.ListInDisk + // partitionList is the chunks to store row values for partitions. + partitionList []*chunk.RowContainer // sortRows is used to maintain a heap. sortRows []chunk.Row @@ -168,26 +168,23 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { } func (e *SortExec) prepareExternalSorting() (err error) { - e.sortRowsIndex = make([]int, 0, len(e.partitionList)+1) - e.partitionConsumedRowsPtr = make([]chunk.RowPtr, len(e.partitionList)+1) + e.sortRowsIndex = make([]int, 0, len(e.partitionList)) + e.partitionConsumedRowsPtr = make([]chunk.RowPtr, len(e.partitionList)) e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRowsPtr))) e.heapSort = nil return err } -func (e *SortExec) generatePartition() error { +func (e *SortExec) generatePartition(exceeded bool) error { e.initPointers() sort.Slice(e.rowPtrs, e.keyColumnsLess) - listInDisk, err := e.spillToDiskByRowPtr() + rowContainer, err := e.generateRowContainerByRowPtr(exceeded) if err != nil { return err } - e.memTracker.ReplaceChild(e.rowChunks.GetMemTracker(), nil) - e.rowChunks = chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) - e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - e.partitionList = append(e.partitionList, listInDisk) + e.rowChunks.Clear() + e.partitionList = append(e.partitionList, rowContainer) return nil } @@ -232,35 +229,21 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { e.sortRows = append(e.sortRows, row) e.sortRowsIndex = append(e.sortRowsIndex, i) } - if len(e.rowPtrs) != 0 { - e.partitionConsumedRowsPtr[len(e.partitionList)] = chunk.RowPtr{ChkIdx: 0, RowIdx: 0} - e.sortRows = append(e.sortRows, e.rowChunks.GetRow(e.rowPtrs[0])) - e.sortRowsIndex = append(e.sortRowsIndex, len(e.partitionList)) - } heap.Init(e.heapSort) } for !req.IsFull() && e.heapSort.Len() > 0 { row, idx := e.sortRows[0], e.sortRowsIndex[0] req.AppendRow(row) - if idx == len(e.partitionList) { - e.partitionConsumedRowsPtr[idx].RowIdx++ - if e.partitionConsumedRowsPtr[idx].RowIdx >= uint32(len(e.rowPtrs)) { - heap.Remove(e.heapSort, 0) - continue - } - row = e.rowChunks.GetRow(e.rowPtrs[e.partitionConsumedRowsPtr[idx].RowIdx]) - } else { - rowPtr, ok := getNextRowPtr(e.partitionList[idx], e.partitionConsumedRowsPtr[idx]) - if !ok { - heap.Remove(e.heapSort, 0) - continue - } - e.partitionConsumedRowsPtr[idx] = rowPtr - row, err = e.partitionList[idx].GetRow(rowPtr) - if err != nil { - return err - } + rowPtr, ok := getNextRowPtr(e.partitionList[idx], e.partitionConsumedRowsPtr[idx]) + if !ok { + heap.Remove(e.heapSort, 0) + continue + } + e.partitionConsumedRowsPtr[idx] = rowPtr + row, err = e.partitionList[idx].GetRow(rowPtr) + if err != nil { + return err } e.sortRows[0] = row heap.Fix(e.heapSort, 0) @@ -268,7 +251,7 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { return nil } -func getNextRowPtr(l *chunk.ListInDisk, ptr chunk.RowPtr) (chunk.RowPtr, bool) { +func getNextRowPtr(l *chunk.RowContainer, ptr chunk.RowPtr) (chunk.RowPtr, bool) { ptr.RowIdx++ if ptr.RowIdx == uint32(l.NumRowsOfChunk(int(ptr.ChkIdx))) { ptr.ChkIdx++ @@ -301,14 +284,20 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { } e.rowChunks.Add(chk) if atomic.LoadUint32(&e.exceeded) == 1 { - err := e.generatePartition() + err := e.generatePartition(true) if err != nil { return err } atomic.StoreUint32(&e.spilled, 1) + e.exceeded = 0 e.spillAction.reset() } } + if e.alreadySpilledSafe() && e.rowChunks.Len() != 0 { + if err := e.generatePartition(false); err != nil { + return err + } + } return nil } @@ -370,14 +359,18 @@ func (e *SortExec) keyColumnsLess(i, j int) bool { // alreadySpilledSafe indicates that records have spilled out into disk. It's thread-safe. func (e *SortExec) alreadySpilledSafe() bool { return atomic.LoadUint32(&e.spilled) == 1 } -func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { - rowChunksInDisk := chunk.NewListInDisk(e.retFieldTypes) - rowChunksInDisk.GetDiskTracker().AttachTo(e.diskTracker) +func (e *SortExec) generateRowContainerByRowPtr(exceeded bool) (disk *chunk.RowContainer, err error) { + rowContainer := chunk.NewRowContainer(e.retFieldTypes, e.maxChunkSize) + if exceeded { + if err := rowContainer.SpillToDisk(); err != nil { + return nil, err + } + } chk := newFirstChunk(e) for _, rowPtr := range e.rowPtrs { chk.AppendRow(e.rowChunks.GetRow(rowPtr)) if chk.IsFull() { - err := rowChunksInDisk.Add(chk) + err := rowContainer.Add(chk) if err != nil { return nil, err } @@ -385,11 +378,11 @@ func (e *SortExec) spillToDiskByRowPtr() (disk *chunk.ListInDisk, err error) { } } if chk.NumRows() != 0 { - if err := rowChunksInDisk.Add(chk); err != nil { + if err := rowContainer.Add(chk); err != nil { return nil, err } } - return rowChunksInDisk, nil + return rowContainer, nil } // ActionSpill returns a memory.ActionOnExceed for spilling over to disk. @@ -428,7 +421,6 @@ func (a *spillSortDiskAction) SetFallback(fallback memory.ActionOnExceed) { func (a *spillSortDiskAction) SetLogHook(hook func(uint64)) {} func (a *spillSortDiskAction) reset() { - atomic.StoreUint32(&a.e.exceeded, 0) a.once = sync.Once{} } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index 4f964e462b155..c5d3aadc1f5fa 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -58,7 +58,8 @@ func NewRowContainer(fieldType []*types.FieldType, chunkSize int) *RowContainer return rc } -func (c *RowContainer) spillToDisk() (err error) { +// SpillToDisk spill RowContainer to disk. +func (c *RowContainer) SpillToDisk() (err error) { N := c.records.NumChunks() c.recordsInDisk = NewListInDisk(c.records.FieldTypes()) c.recordsInDisk.diskTracker.AttachTo(c.diskTracker) @@ -127,7 +128,7 @@ func (c *RowContainer) Add(chk *Chunk) (err error) { } else { c.records.Add(chk) if atomic.LoadUint32(&c.exceeded) != 0 { - err = c.spillToDisk() + err = c.SpillToDisk() if err != nil { return err } @@ -189,7 +190,7 @@ type spillDiskAction struct { fallbackAction memory.ActionOnExceed } -// Action sends a signal to trigger spillToDisk method of RowContainer +// Action sends a signal to trigger SpillToDisk method of RowContainer // and if it is already triggered before, call its fallbackAction. func (a *spillDiskAction) Action(t *memory.Tracker) { if a.c.AlreadySpilledSafe() { diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index d1d5d6953e656..4a33d3802b6ec 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -72,7 +72,7 @@ func (r *rowContainerTestSuite) TestSel(c *check.C) { c.Assert(i, check.Equals, n-1) } checkByIter(NewMultiIterator(NewIterator4RowContainer(rc), NewIterator4Chunk(chk))) - err := rc.spillToDisk() + err := rc.SpillToDisk() c.Assert(err, check.IsNil) c.Assert(rc.AlreadySpilled(), check.Equals, true) checkByIter(NewMultiIterator(NewIterator4RowContainer(rc), NewIterator4Chunk(chk))) From cc5bed20b97348d889db6da32a95bcb6ddf1656d Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 7 Jan 2020 11:19:47 +0800 Subject: [PATCH 20/43] address comments --- executor/sort.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 9638dd5a0fcc1..01629de9c03ea 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -17,7 +17,6 @@ import ( "container/heap" "context" "fmt" - "go.uber.org/zap" "sort" "sync" "sync/atomic" @@ -31,6 +30,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/stringutil" + "go.uber.org/zap" ) var rowChunksLabel fmt.Stringer = stringutil.StringerStr("rowChunks") @@ -130,9 +130,9 @@ func (e *SortExec) Open(ctx context.Context) error { // 1. Read as mush as rows into memory. // 2. If memory quota is triggered, sort these rows in memory and put them into disk as partition 1, then reset // the memory quota trigger and return to step 1 -// 3. If memory quota is not triggered and child is consumed, sort these rows in memory as partition N merge sort -// if the count of partitions is larger than 1. -// 4. If there is only one partition in step 4, it works just like in-memory sort before. +// 3. If memory quota is not triggered and child is consumed, sort these rows in memory as partition N. +// 4. Merge sort if the count of partitions is larger than 1. If there is only one partition in step 4, it works +// just like in-memory sort before. func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { req.Reset() if !e.fetched { From 26f072f3b880708ca5c6f538779beadd1aaac71e Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 7 Jan 2020 18:35:48 +0800 Subject: [PATCH 21/43] temp --- executor/hash_table.go | 2 +- executor/merge_join.go | 2 +- executor/sort.go | 231 ++++++++++++------------------- util/chunk/row_container.go | 13 +- util/chunk/row_container_test.go | 6 +- 5 files changed, 102 insertions(+), 152 deletions(-) diff --git a/executor/hash_table.go b/executor/hash_table.go index 5b43876380cf2..42799b606ad19 100644 --- a/executor/hash_table.go +++ b/executor/hash_table.go @@ -164,7 +164,7 @@ func (c *hashRowContainer) PutChunk(chk *chunk.Chunk) error { func (c *hashRowContainer) PutChunkSelected(chk *chunk.Chunk, selected []bool) error { var chkIdx uint32 chkIdx = uint32(c.rowContainer.NumChunks()) - err := c.rowContainer.Add(chk) + err := c.rowContainer.Add(chk, nil) if err != nil { return err } diff --git a/executor/merge_join.go b/executor/merge_join.go index b6e00ab2ea32a..0a11f6517fa39 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -236,7 +236,7 @@ func (t *mergeJoinInnerTable) reallocCurChkForReader() (err error) { t.firstRow4Key = t.firstRow4Key.CopyConstruct() // curSel be never be nil, since the chunk is in use. t.curChk.SetSel(t.curSel) - err = t.rowContainer.Add(t.curChk) + err = t.rowContainer.Add(t.curChk, nil) if err != nil { return err } diff --git a/executor/sort.go b/executor/sort.go index 01629de9c03ea..5ca7a6294a5cc 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -17,20 +17,16 @@ import ( "container/heap" "context" "fmt" + "github.com/pingcap/tidb/config" "sort" - "sync" - "sync/atomic" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/disk" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/stringutil" - "go.uber.org/zap" ) var rowChunksLabel fmt.Stringer = stringutil.StringerStr("rowChunks") @@ -51,7 +47,7 @@ type SortExec struct { // keyCmpFuncs is used to compare each ByItem. keyCmpFuncs []chunk.CompareFunc // rowChunks is the chunks to store row values. - rowChunks *chunk.List + rowChunks *chunk.RowContainer // rowPointer store the chunk index and row index for each row. rowPtrs []chunk.RowPtr @@ -60,17 +56,17 @@ type SortExec struct { // partitionList is the chunks to store row values for partitions. partitionList []*chunk.RowContainer + // partitionRowPtrs + partitionRowPtrs [][]chunk.RowPtr // sortRows is used to maintain a heap. sortRows []chunk.Row // sortRowsIndex store the partition index for each row. sortRowsIndex []int // partitionConsumedRows store the consumed rows num for each partition. - partitionConsumedRowsPtr []chunk.RowPtr + partitionConsumedNum []int // heapSort use heap sort for spill disk. heapSort *topNChunkHeapWithIndex - // spillAction save the spill action for the Sort Executor. - spillAction *spillSortDiskAction // exceeded indicates that records have exceeded memQuota during // adding this chunk and we should spill now. @@ -81,7 +77,7 @@ type SortExec struct { // Close implements the Executor Close interface. func (e *SortExec) Close() error { - if e.alreadySpilledSafe() { + if len(e.partitionList) > 1 { for _, chunkInDisk := range e.partitionList { if chunkInDisk != nil { if err := chunkInDisk.Close(); err != nil { @@ -92,9 +88,9 @@ func (e *SortExec) Close() error { e.partitionList = e.partitionList[:0] e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedRowsPtr))) + e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedNum))) e.sortRowsIndex = nil - e.partitionConsumedRowsPtr = nil + e.partitionConsumedNum = nil } if e.rowChunks != nil { e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) @@ -142,50 +138,41 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - if e.alreadySpilledSafe() { - err = e.prepareExternalSorting() - if err != nil { - return err - } - } e.initPointers() sort.Slice(e.rowPtrs, e.keyColumnsLess) e.fetched = true } - if e.alreadySpilledSafe() { + if len(e.partitionList) > 0 { if err := e.externalSorting(req); err != nil { return err } } else { for !req.IsFull() && e.Idx < len(e.rowPtrs) { rowPtr := e.rowPtrs[e.Idx] - req.AppendRow(e.rowChunks.GetRow(rowPtr)) + row, err := e.rowChunks.GetRow(rowPtr) + if err != nil { + return err + } + req.AppendRow(row) e.Idx++ } } return nil } -func (e *SortExec) prepareExternalSorting() (err error) { - e.sortRowsIndex = make([]int, 0, len(e.partitionList)) - e.partitionConsumedRowsPtr = make([]chunk.RowPtr, len(e.partitionList)) - e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(8 * cap(e.partitionConsumedRowsPtr))) - e.heapSort = nil - return err -} - -func (e *SortExec) generatePartition(exceeded bool) error { +func (e *SortExec) generatePartition() { e.initPointers() sort.Slice(e.rowPtrs, e.keyColumnsLess) - rowContainer, err := e.generateRowContainerByRowPtr(exceeded) - if err != nil { - return err + e.partitionList = append(e.partitionList, e.rowChunks) + e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) + e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) + e.rowChunks.GetMemTracker().AttachTo(e.memTracker) + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + if config.GetGlobalConfig().OOMUseTmpStorage { + actionSpill := e.rowChunks.ActionSpill() + e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) } - e.rowChunks.Clear() - e.partitionList = append(e.partitionList, rowContainer) - return nil } type topNChunkHeapWithIndex struct { @@ -219,10 +206,14 @@ func (h *topNChunkHeapWithIndex) Swap(i, j int) { func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if e.heapSort == nil { + e.sortRowsIndex = make([]int, 0, len(e.partitionList)) + e.partitionConsumedNum = make([]int, len(e.partitionList)) + e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) + e.memTracker.Consume(int64(8 * cap(e.partitionConsumedNum))) e.heapSort = &topNChunkHeapWithIndex{e} for i := 0; i < len(e.partitionList); i++ { - e.partitionConsumedRowsPtr[i] = chunk.RowPtr{ChkIdx: 0, RowIdx: 0} - row, err := e.partitionList[i].GetRow(e.partitionConsumedRowsPtr[i]) + e.partitionConsumedNum[i] = 0 + row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) if err != nil { return err } @@ -233,15 +224,15 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { } for !req.IsFull() && e.heapSort.Len() > 0 { - row, idx := e.sortRows[0], e.sortRowsIndex[0] + row, partIdx := e.sortRows[0], e.sortRowsIndex[0] req.AppendRow(row) - rowPtr, ok := getNextRowPtr(e.partitionList[idx], e.partitionConsumedRowsPtr[idx]) - if !ok { + rowIdx := e.partitionConsumedNum[partIdx] + 1 + if rowIdx >= len(e.partitionRowPtrs[partIdx]) { heap.Remove(e.heapSort, 0) continue } - e.partitionConsumedRowsPtr[idx] = rowPtr - row, err = e.partitionList[idx].GetRow(rowPtr) + e.partitionConsumedNum[partIdx] = rowIdx + row, err = e.partitionList[partIdx].GetRow(e.partitionRowPtrs[partIdx][rowIdx]) if err != nil { return err } @@ -265,11 +256,11 @@ func getNextRowPtr(l *chunk.RowContainer, ptr chunk.RowPtr) (chunk.RowPtr, bool) func (e *SortExec) fetchRowChunks(ctx context.Context) error { fields := retTypes(e) - e.rowChunks = chunk.NewList(fields, e.initCap, e.maxChunkSize) + e.rowChunks = chunk.NewRowContainer(fields, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) if config.GetGlobalConfig().OOMUseTmpStorage { - actionSpill := e.ActionSpill() + actionSpill := e.rowChunks.ActionSpill() e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) } for { @@ -282,32 +273,18 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if rowCount == 0 { break } - e.rowChunks.Add(chk) - if atomic.LoadUint32(&e.exceeded) == 1 { - err := e.generatePartition(true) - if err != nil { - return err - } - atomic.StoreUint32(&e.spilled, 1) - e.exceeded = 0 - e.spillAction.reset() - } - } - if e.alreadySpilledSafe() && e.rowChunks.Len() != 0 { - if err := e.generatePartition(false); err != nil { + if err := e.rowChunks.Add(chk, e.generatePartition); err != nil { return err } } + if e.rowChunks.NumRow() > 0 && len(e.partitionList) > 0 { + e.generatePartition() + } return nil } func (e *SortExec) initPointers() { - if e.rowPtrs != nil { - e.memTracker.Consume(int64(-8 * cap(e.rowPtrs))) - e.rowPtrs = e.rowPtrs[:0] - } else { - e.rowPtrs = make([]chunk.RowPtr, 0, e.rowChunks.Len()) - } + e.rowPtrs = make([]chunk.RowPtr, 0, e.rowChunks.NumRow()) for chkIdx := 0; chkIdx < e.rowChunks.NumChunks(); chkIdx++ { rowChk := e.rowChunks.GetChunk(chkIdx) for rowIdx := 0; rowIdx < rowChk.NumRows(); rowIdx++ { @@ -351,77 +328,15 @@ func (e *SortExec) lessRow(rowI, rowJ chunk.Row) bool { // keyColumnsLess is the less function for key columns. func (e *SortExec) keyColumnsLess(i, j int) bool { - rowI := e.rowChunks.GetRow(e.rowPtrs[i]) - rowJ := e.rowChunks.GetRow(e.rowPtrs[j]) - return e.lessRow(rowI, rowJ) -} - -// alreadySpilledSafe indicates that records have spilled out into disk. It's thread-safe. -func (e *SortExec) alreadySpilledSafe() bool { return atomic.LoadUint32(&e.spilled) == 1 } - -func (e *SortExec) generateRowContainerByRowPtr(exceeded bool) (disk *chunk.RowContainer, err error) { - rowContainer := chunk.NewRowContainer(e.retFieldTypes, e.maxChunkSize) - if exceeded { - if err := rowContainer.SpillToDisk(); err != nil { - return nil, err - } - } - chk := newFirstChunk(e) - for _, rowPtr := range e.rowPtrs { - chk.AppendRow(e.rowChunks.GetRow(rowPtr)) - if chk.IsFull() { - err := rowContainer.Add(chk) - if err != nil { - return nil, err - } - chk = newFirstChunk(e) - } - } - if chk.NumRows() != 0 { - if err := rowContainer.Add(chk); err != nil { - return nil, err - } + rowI, err := e.rowChunks.GetRow(e.rowPtrs[i]) + if err != nil { + panic(err) } - return rowContainer, nil -} - -// ActionSpill returns a memory.ActionOnExceed for spilling over to disk. -func (e *SortExec) ActionSpill() memory.ActionOnExceed { - e.spillAction = &spillSortDiskAction{e: e} - return e.spillAction -} - -// spillSortDiskAction implements memory.ActionOnExceed for chunk.List. If -// the memory quota of a query is exceeded, spillSortDiskAction.Action is -// triggered. -type spillSortDiskAction struct { - once sync.Once - e *SortExec - fallbackAction memory.ActionOnExceed -} - -// Action sends a signal to trigger spillToDisk method of SortExec -// and if it is already triggered before, call its fallbackAction. -func (a *spillSortDiskAction) Action(t *memory.Tracker) { - if a.e.alreadySpilledSafe() { - if a.fallbackAction != nil { - a.fallbackAction.Action(t) - } + rowJ, err := e.rowChunks.GetRow(e.rowPtrs[j]) + if err != nil { + panic(err) } - a.once.Do(func() { - atomic.StoreUint32(&a.e.exceeded, 1) - logutil.BgLogger().Info("memory exceeds quota, spill to disk now.", zap.String("memory", t.String())) - }) -} - -func (a *spillSortDiskAction) SetFallback(fallback memory.ActionOnExceed) { - a.fallbackAction = fallback -} - -func (a *spillSortDiskAction) SetLogHook(hook func(uint64)) {} - -func (a *spillSortDiskAction) reset() { - a.once = sync.Once{} + return e.lessRow(rowI, rowJ) } // TopNExec implements a Top-N algorithm and it is built from a SELECT statement with ORDER BY and LIMIT. @@ -442,8 +357,14 @@ type topNChunkHeap struct { // Less implement heap.Interface, but since we mantains a max heap, // this function returns true if row i is greater than row j. func (h *topNChunkHeap) Less(i, j int) bool { - rowI := h.rowChunks.GetRow(h.rowPtrs[i]) - rowJ := h.rowChunks.GetRow(h.rowPtrs[j]) + rowI, err := h.rowChunks.GetRow(h.rowPtrs[i]) + if err != nil { + panic(err) + } + rowJ, err := h.rowChunks.GetRow(h.rowPtrs[j]) + if err != nil { + panic(err) + } return h.greaterRow(rowI, rowJ) } @@ -508,7 +429,10 @@ func (e *TopNExec) Next(ctx context.Context, req *chunk.Chunk) error { return nil } for !req.IsFull() && e.Idx < len(e.rowPtrs) { - row := e.rowChunks.GetRow(e.rowPtrs[e.Idx]) + row, err := e.rowChunks.GetRow(e.rowPtrs[e.Idx]) + if err != nil { + return err + } req.AppendRow(row) e.Idx++ } @@ -517,13 +441,13 @@ func (e *TopNExec) Next(ctx context.Context, req *chunk.Chunk) error { func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.chkHeap = &topNChunkHeap{e} - e.rowChunks = chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) + e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - for uint64(e.rowChunks.Len()) < e.totalLimit { + for uint64(e.rowChunks.NumRow()) < e.totalLimit { srcChk := newFirstChunk(e.children[0]) // adjust required rows by total limit - srcChk.SetRequiredRows(int(e.totalLimit-uint64(e.rowChunks.Len())), e.maxChunkSize) + srcChk.SetRequiredRows(int(e.totalLimit-uint64(e.rowChunks.NumRow())), e.maxChunkSize) err := Next(ctx, e.children[0], srcChk) if err != nil { return err @@ -531,7 +455,9 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { if srcChk.NumRows() == 0 { break } - e.rowChunks.Add(srcChk) + if err := e.rowChunks.Add(srcChk, nil); err != nil { + return err + } } e.initPointers() e.initCompareFuncs() @@ -560,7 +486,7 @@ func (e *TopNExec) executeTopN(ctx context.Context) error { if err != nil { return err } - if e.rowChunks.Len() > len(e.rowPtrs)*topNCompactionFactor { + if e.rowChunks.NumRow() > len(e.rowPtrs)*topNCompactionFactor { err = e.doCompaction() if err != nil { return err @@ -575,11 +501,17 @@ func (e *TopNExec) processChildChk(childRowChk *chunk.Chunk) error { for i := 0; i < childRowChk.NumRows(); i++ { heapMaxPtr := e.rowPtrs[0] var heapMax, next chunk.Row - heapMax = e.rowChunks.GetRow(heapMaxPtr) + heapMax, err := e.rowChunks.GetRow(heapMaxPtr) + if err != nil { + return err + } next = childRowChk.GetRow(i) if e.chkHeap.greaterRow(heapMax, next) { // Evict heap max, keep the next row. - e.rowPtrs[0] = e.rowChunks.AppendRow(childRowChk.GetRow(i)) + e.rowPtrs[0], err = e.rowChunks.AppendRow(childRowChk.GetRow(i)) + if err != nil { + return err + } heap.Fix(e.chkHeap, 0) } } @@ -591,10 +523,17 @@ func (e *TopNExec) processChildChk(childRowChk *chunk.Chunk) error { // but we want descending top N, then we will keep all data in memory. // But if data is distributed randomly, this function will be called log(n) times. func (e *TopNExec) doCompaction() error { - newRowChunks := chunk.NewList(retTypes(e), e.initCap, e.maxChunkSize) - newRowPtrs := make([]chunk.RowPtr, 0, e.rowChunks.Len()) + newRowChunks := chunk.NewRowContainer(retTypes(e), e.maxChunkSize) + newRowPtrs := make([]chunk.RowPtr, 0, e.rowChunks.NumRow()) for _, rowPtr := range e.rowPtrs { - newRowPtr := newRowChunks.AppendRow(e.rowChunks.GetRow(rowPtr)) + row, err := e.rowChunks.GetRow(rowPtr) + if err != nil { + return err + } + newRowPtr, err := newRowChunks.AppendRow(row) + if err != nil { + return err + } newRowPtrs = append(newRowPtrs, newRowPtr) } newRowChunks.GetMemTracker().SetLabel(rowChunksLabel) diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index c5d3aadc1f5fa..1e754e02ce478 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -14,6 +14,7 @@ package chunk import ( + "errors" "sync" "sync/atomic" @@ -122,12 +123,15 @@ func (c *RowContainer) NumChunks() int { } // Add appends a chunk into the RowContainer. -func (c *RowContainer) Add(chk *Chunk) (err error) { +func (c *RowContainer) Add(chk *Chunk, callback func()) (err error) { if c.AlreadySpilled() { err = c.recordsInDisk.Add(chk) } else { c.records.Add(chk) if atomic.LoadUint32(&c.exceeded) != 0 { + if callback != nil { + callback() + } err = c.SpillToDisk() if err != nil { return err @@ -138,6 +142,13 @@ func (c *RowContainer) Add(chk *Chunk) (err error) { return } +func (c *RowContainer) AppendRow(row Row) (RowPtr, error) { + if c.AlreadySpilled() { + return RowPtr{}, errors.New("ListInDisk don't support AppendRow") + } + return c.records.AppendRow(row), nil +} + // AllocChunk allocates a new chunk from RowContainer. func (c *RowContainer) AllocChunk() (chk *Chunk) { return c.records.allocChunk() diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index 4a33d3802b6ec..88cdb1313c9e7 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -47,7 +47,7 @@ func (r *rowContainerTestSuite) TestSel(c *check.C) { if chk.NumRows() == sz { chk.SetSel([]int{0, 2}) numRows += 2 - err := rc.Add(chk) + err := rc.Add(chk, nil) c.Assert(err, check.IsNil) chk = NewChunkWithCapacity(fields, sz) } @@ -96,14 +96,14 @@ func (r *rowContainerTestSuite) TestSpillAction(c *check.C) { c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(0)) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(0)) - err = rc.Add(chk) + err = rc.Add(chk, nil) c.Assert(err, check.IsNil) c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(0)) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(0)) c.Assert(rc.GetMemTracker().BytesConsumed(), check.Equals, chk.MemoryUsage()) // The following line is erroneous, since chk is already handled by rc, Add it again causes duplicated memory usage account. // It is only for test of spill, do not double-add a chunk elsewhere. - err = rc.Add(chk) + err = rc.Add(chk, nil) c.Assert(err, check.IsNil) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(1)) c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(1)) From 704bf10257df4c881fdf322f7afd266e6635d822 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 8 Jan 2020 14:50:23 +0800 Subject: [PATCH 22/43] address comments --- executor/executor_test.go | 2 +- executor/sort.go | 41 +++++++++++++++++-------------------- util/chunk/row_container.go | 31 ++++++++++++++++------------ 3 files changed, 38 insertions(+), 36 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 85336b9c99106..16c5783a90b1c 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -4446,7 +4446,7 @@ func (s *testSuite) TestOOMPanicAction(c *C) { c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*") tk.MustExec("set @@tidb_mem_quota_query=10000") tk.MustExec("insert into t1 values (1),(2),(3),(4),(5);") - tk.MustExec("set @@tidb_mem_quota_query=200;") + tk.MustExec("set @@tidb_mem_quota_query=10;") _, err = tk.Exec("insert into t select a from t1 order by a desc;") c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*") _, err = tk.Exec("replace into t select a from t1 order by a desc;") diff --git a/executor/sort.go b/executor/sort.go index 5ca7a6294a5cc..3e2d83e9121ea 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -67,6 +67,8 @@ type SortExec struct { partitionConsumedNum []int // heapSort use heap sort for spill disk. heapSort *topNChunkHeapWithIndex + // spillAction save the Action for spill disk. + spillAction *chunk.SpillDiskAction // exceeded indicates that records have exceeded memQuota during // adding this chunk and we should spill now. @@ -77,7 +79,7 @@ type SortExec struct { // Close implements the Executor Close interface. func (e *SortExec) Close() error { - if len(e.partitionList) > 1 { + if len(e.partitionList) > 0 { for _, chunkInDisk := range e.partitionList { if chunkInDisk != nil { if err := chunkInDisk.Close(); err != nil { @@ -87,6 +89,13 @@ func (e *SortExec) Close() error { } e.partitionList = e.partitionList[:0] + for _, chunkPtr := range e.partitionRowPtrs { + if chunkPtr != nil { + e.memTracker.Consume(int64(-8 * cap(chunkPtr))) + } + } + e.partitionRowPtrs = e.partitionRowPtrs[:0] + e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedNum))) e.sortRowsIndex = nil @@ -166,13 +175,6 @@ func (e *SortExec) generatePartition() { sort.Slice(e.rowPtrs, e.keyColumnsLess) e.partitionList = append(e.partitionList, e.rowChunks) e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) - e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) - e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - if config.GetGlobalConfig().OOMUseTmpStorage { - actionSpill := e.rowChunks.ActionSpill() - e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) - } } type topNChunkHeapWithIndex struct { @@ -242,26 +244,14 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { return nil } -func getNextRowPtr(l *chunk.RowContainer, ptr chunk.RowPtr) (chunk.RowPtr, bool) { - ptr.RowIdx++ - if ptr.RowIdx == uint32(l.NumRowsOfChunk(int(ptr.ChkIdx))) { - ptr.ChkIdx++ - ptr.RowIdx = 0 - } - if ptr.ChkIdx == uint32(l.NumChunks()) { - return ptr, false - } - return ptr, true -} - func (e *SortExec) fetchRowChunks(ctx context.Context) error { fields := retTypes(e) e.rowChunks = chunk.NewRowContainer(fields, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) if config.GetGlobalConfig().OOMUseTmpStorage { - actionSpill := e.rowChunks.ActionSpill() - e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) + e.spillAction = e.rowChunks.ActionSpill() + e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(e.spillAction) } for { chk := newFirstChunk(e.children[0]) @@ -276,6 +266,13 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if err := e.rowChunks.Add(chk, e.generatePartition); err != nil { return err } + if e.rowChunks.AlreadySpilled() { + e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) + e.rowChunks.GetMemTracker().AttachTo(e.memTracker) + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + e.spillAction.SetRowContainer(e.rowChunks) + e.spillAction.ResetOnce() + } } if e.rowChunks.NumRow() > 0 && len(e.partitionList) > 0 { e.generatePartition() diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index 1e754e02ce478..e8f632d1ec3da 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -47,7 +47,7 @@ type RowContainer struct { memTracker *memory.Tracker diskTracker *disk.Tracker - actionSpill *spillDiskAction + actionSpill *SpillDiskAction } // NewRowContainer creates a new RowContainer in memory. @@ -85,7 +85,7 @@ func (c *RowContainer) Reset() error { } atomic.StoreUint32(&c.exceeded, 0) atomic.StoreUint32(&c.spilled, 0) - c.actionSpill.reset() + c.actionSpill.ResetOnce() } else { c.records.Reset() } @@ -186,16 +186,16 @@ func (c *RowContainer) Close() (err error) { return } -// ActionSpill returns a memory.ActionOnExceed for spilling over to disk. -func (c *RowContainer) ActionSpill() memory.ActionOnExceed { - c.actionSpill = &spillDiskAction{c: c} +// ActionSpill returns a SpillDiskAction for spilling over to disk. +func (c *RowContainer) ActionSpill() *SpillDiskAction { + c.actionSpill = &SpillDiskAction{c: c} return c.actionSpill } -// spillDiskAction implements memory.ActionOnExceed for chunk.List. If -// the memory quota of a query is exceeded, spillDiskAction.Action is +// SpillDiskAction implements memory.ActionOnExceed for chunk.List. If +// the memory quota of a query is exceeded, SpillDiskAction.Action is // triggered. -type spillDiskAction struct { +type SpillDiskAction struct { once sync.Once c *RowContainer fallbackAction memory.ActionOnExceed @@ -203,7 +203,7 @@ type spillDiskAction struct { // Action sends a signal to trigger SpillToDisk method of RowContainer // and if it is already triggered before, call its fallbackAction. -func (a *spillDiskAction) Action(t *memory.Tracker) { +func (a *SpillDiskAction) Action(t *memory.Tracker) { if a.c.AlreadySpilledSafe() { if a.fallbackAction != nil { a.fallbackAction.Action(t) @@ -216,14 +216,19 @@ func (a *spillDiskAction) Action(t *memory.Tracker) { } // SetFallback sets the fallback action. -func (a *spillDiskAction) SetFallback(fallback memory.ActionOnExceed) { +func (a *SpillDiskAction) SetFallback(fallback memory.ActionOnExceed) { a.fallbackAction = fallback } // SetLogHook sets the hook, it does nothing just to form the memory.ActionOnExceed interface. -func (a *spillDiskAction) SetLogHook(hook func(uint64)) {} +func (a *SpillDiskAction) SetLogHook(hook func(uint64)) {} -// reset resets the spill action so that it can be triggered next time. -func (a *spillDiskAction) reset() { +// ResetOnce resets the spill action so that it can be triggered next time. +func (a *SpillDiskAction) ResetOnce() { a.once = sync.Once{} } + +// SetRowContainer sets the RowContainer for the SpillDiskAction. +func (a *SpillDiskAction) SetRowContainer(c *RowContainer) { + a.c = c +} From 1e6e99211d2eb441c0c82530ed5abaf844bff2f9 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 8 Jan 2020 14:58:14 +0800 Subject: [PATCH 23/43] fix --- executor/sort.go | 16 +++++++--------- util/chunk/row_container.go | 7 +++---- util/chunk/row_container_test.go | 2 +- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 3e2d83e9121ea..553f55c5ec73d 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -17,9 +17,9 @@ import ( "container/heap" "context" "fmt" - "github.com/pingcap/tidb/config" "sort" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" @@ -56,7 +56,7 @@ type SortExec struct { // partitionList is the chunks to store row values for partitions. partitionList []*chunk.RowContainer - // partitionRowPtrs + // partitionRowPtrs store the sorted RowPtrs for each row for partitions. partitionRowPtrs [][]chunk.RowPtr // sortRows is used to maintain a heap. @@ -175,6 +175,11 @@ func (e *SortExec) generatePartition() { sort.Slice(e.rowPtrs, e.keyColumnsLess) e.partitionList = append(e.partitionList, e.rowChunks) e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) + e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) + e.rowChunks.GetMemTracker().AttachTo(e.memTracker) + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + e.spillAction.SetRowContainer(e.rowChunks) + e.spillAction.ResetOnce() } type topNChunkHeapWithIndex struct { @@ -266,13 +271,6 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if err := e.rowChunks.Add(chk, e.generatePartition); err != nil { return err } - if e.rowChunks.AlreadySpilled() { - e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) - e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - e.spillAction.SetRowContainer(e.rowChunks) - e.spillAction.ResetOnce() - } } if e.rowChunks.NumRow() > 0 && len(e.partitionList) > 0 { e.generatePartition() diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index e8f632d1ec3da..0fbae4d4bed98 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -59,8 +59,7 @@ func NewRowContainer(fieldType []*types.FieldType, chunkSize int) *RowContainer return rc } -// SpillToDisk spill RowContainer to disk. -func (c *RowContainer) SpillToDisk() (err error) { +func (c *RowContainer) spillToDisk() (err error) { N := c.records.NumChunks() c.recordsInDisk = NewListInDisk(c.records.FieldTypes()) c.recordsInDisk.diskTracker.AttachTo(c.diskTracker) @@ -132,7 +131,7 @@ func (c *RowContainer) Add(chk *Chunk, callback func()) (err error) { if callback != nil { callback() } - err = c.SpillToDisk() + err = c.spillToDisk() if err != nil { return err } @@ -201,7 +200,7 @@ type SpillDiskAction struct { fallbackAction memory.ActionOnExceed } -// Action sends a signal to trigger SpillToDisk method of RowContainer +// Action sends a signal to trigger spillToDisk method of RowContainer // and if it is already triggered before, call its fallbackAction. func (a *SpillDiskAction) Action(t *memory.Tracker) { if a.c.AlreadySpilledSafe() { diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index 88cdb1313c9e7..d18fabcf11f83 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -72,7 +72,7 @@ func (r *rowContainerTestSuite) TestSel(c *check.C) { c.Assert(i, check.Equals, n-1) } checkByIter(NewMultiIterator(NewIterator4RowContainer(rc), NewIterator4Chunk(chk))) - err := rc.SpillToDisk() + err := rc.spillToDisk() c.Assert(err, check.IsNil) c.Assert(rc.AlreadySpilled(), check.Equals, true) checkByIter(NewMultiIterator(NewIterator4RowContainer(rc), NewIterator4Chunk(chk))) From ed0b16348e5c55dc2984e2e418cccaefdfcac41e Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 8 Jan 2020 15:05:39 +0800 Subject: [PATCH 24/43] fix --- util/chunk/row_container.go | 1 + 1 file changed, 1 insertion(+) diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index 0fbae4d4bed98..87fef994ecc2c 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -141,6 +141,7 @@ func (c *RowContainer) Add(chk *Chunk, callback func()) (err error) { return } +// AppendRow appends a row to the RowContainer, the row is copied to the RowContainer. func (c *RowContainer) AppendRow(row Row) (RowPtr, error) { if c.AlreadySpilled() { return RowPtr{}, errors.New("ListInDisk don't support AppendRow") From 2684c0656fe18675315c261564fdefe5e9ab015e Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 8 Jan 2020 15:22:10 +0800 Subject: [PATCH 25/43] fix --- executor/sort.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 553f55c5ec73d..d07c0a166da98 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -175,11 +175,6 @@ func (e *SortExec) generatePartition() { sort.Slice(e.rowPtrs, e.keyColumnsLess) e.partitionList = append(e.partitionList, e.rowChunks) e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) - e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) - e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - e.spillAction.SetRowContainer(e.rowChunks) - e.spillAction.ResetOnce() } type topNChunkHeapWithIndex struct { @@ -271,6 +266,13 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if err := e.rowChunks.Add(chk, e.generatePartition); err != nil { return err } + if e.rowChunks.AlreadySpilled() { + e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) + e.rowChunks.GetMemTracker().AttachTo(e.memTracker) + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + e.spillAction.SetRowContainer(e.rowChunks) + e.spillAction.ResetOnce() + } } if e.rowChunks.NumRow() > 0 && len(e.partitionList) > 0 { e.generatePartition() From 45d8a1615b2c38eb3b627c336b6bd58b553429bc Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 19 Jan 2020 12:11:27 +0800 Subject: [PATCH 26/43] address commets --- executor/sort.go | 16 ++++++++-------- executor/sort_test.go | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index d07c0a166da98..3c68e4258b558 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -66,7 +66,7 @@ type SortExec struct { // partitionConsumedRows store the consumed rows num for each partition. partitionConsumedNum []int // heapSort use heap sort for spill disk. - heapSort *topNChunkHeapWithIndex + heapSort *multiWayMerge // spillAction save the Action for spill disk. spillAction *chunk.SpillDiskAction @@ -177,31 +177,31 @@ func (e *SortExec) generatePartition() { e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) } -type topNChunkHeapWithIndex struct { +type multiWayMerge struct { *SortExec } -func (h *topNChunkHeapWithIndex) Less(i, j int) bool { +func (h *multiWayMerge) Less(i, j int) bool { rowI := h.sortRows[i] rowJ := h.sortRows[j] return h.lessRow(rowI, rowJ) } -func (h *topNChunkHeapWithIndex) Len() int { +func (h *multiWayMerge) Len() int { return len(h.sortRows) } -func (h *topNChunkHeapWithIndex) Push(x interface{}) { +func (h *multiWayMerge) Push(x interface{}) { // Should never be called. } -func (h *topNChunkHeapWithIndex) Pop() interface{} { +func (h *multiWayMerge) Pop() interface{} { h.sortRows = h.sortRows[:len(h.sortRows)-1] h.sortRowsIndex = h.sortRowsIndex[:len(h.sortRowsIndex)-1] return nil } -func (h *topNChunkHeapWithIndex) Swap(i, j int) { +func (h *multiWayMerge) Swap(i, j int) { h.sortRows[i], h.sortRows[j] = h.sortRows[j], h.sortRows[i] h.sortRowsIndex[i], h.sortRowsIndex[j] = h.sortRowsIndex[j], h.sortRowsIndex[i] } @@ -212,7 +212,7 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { e.partitionConsumedNum = make([]int, len(e.partitionList)) e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) e.memTracker.Consume(int64(8 * cap(e.partitionConsumedNum))) - e.heapSort = &topNChunkHeapWithIndex{e} + e.heapSort = &multiWayMerge{e} for i := 0; i < len(e.partitionList); i++ { e.partitionConsumedNum[i] = 0 row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) diff --git a/executor/sort_test.go b/executor/sort_test.go index 9cafcefd8da2d..4f2ccd5e0ff36 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 PingCAP, Inc. +// Copyright 2020 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 50ecaf1d54d24c53c287ede005c5d42845d66431 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 19 Jan 2020 16:07:45 +0800 Subject: [PATCH 27/43] address comments 1 --- executor/hash_table.go | 2 +- executor/merge_join.go | 2 +- executor/sort.go | 10 ++++++++-- util/chunk/row_container.go | 18 ++++++++++++------ util/chunk/row_container_test.go | 6 +++--- 5 files changed, 25 insertions(+), 13 deletions(-) diff --git a/executor/hash_table.go b/executor/hash_table.go index 5ccab42700228..709d924685ff0 100644 --- a/executor/hash_table.go +++ b/executor/hash_table.go @@ -163,7 +163,7 @@ func (c *hashRowContainer) PutChunk(chk *chunk.Chunk) error { // value of hash table: RowPtr of the corresponded row func (c *hashRowContainer) PutChunkSelected(chk *chunk.Chunk, selected []bool) error { chkIdx := uint32(c.rowContainer.NumChunks()) - err := c.rowContainer.Add(chk, nil) + err := c.rowContainer.Add(chk) if err != nil { return err } diff --git a/executor/merge_join.go b/executor/merge_join.go index 0a11f6517fa39..b6e00ab2ea32a 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -236,7 +236,7 @@ func (t *mergeJoinInnerTable) reallocCurChkForReader() (err error) { t.firstRow4Key = t.firstRow4Key.CopyConstruct() // curSel be never be nil, since the chunk is in use. t.curChk.SetSel(t.curSel) - err = t.rowContainer.Add(t.curChk, nil) + err = t.rowContainer.Add(t.curChk) if err != nil { return err } diff --git a/executor/sort.go b/executor/sort.go index 3c68e4258b558..73eb965e7f0e0 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -249,9 +249,14 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks = chunk.NewRowContainer(fields, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + var onExceededCallback func(chk *chunk.Chunk) if config.GetGlobalConfig().OOMUseTmpStorage { e.spillAction = e.rowChunks.ActionSpill() e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(e.spillAction) + onExceededCallback = func(chk *chunk.Chunk) { + e.generatePartition() + } + e.rowChunks.SetOnExceededCallback(onExceededCallback) } for { chk := newFirstChunk(e.children[0]) @@ -263,13 +268,14 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { if rowCount == 0 { break } - if err := e.rowChunks.Add(chk, e.generatePartition); err != nil { + if err := e.rowChunks.Add(chk); err != nil { return err } if e.rowChunks.AlreadySpilled() { e.rowChunks = chunk.NewRowContainer(retTypes(e), e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) + e.rowChunks.SetOnExceededCallback(onExceededCallback) e.spillAction.SetRowContainer(e.rowChunks) e.spillAction.ResetOnce() } @@ -452,7 +458,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { if srcChk.NumRows() == 0 { break } - if err := e.rowChunks.Add(srcChk, nil); err != nil { + if err := e.rowChunks.Add(srcChk); err != nil { return err } } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index b057980713829..0896c0194ccab 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -45,9 +45,10 @@ type RowContainer struct { // It's for concurrency usage, so access it with atomic. spilled uint32 - memTracker *memory.Tracker - diskTracker *disk.Tracker - actionSpill *SpillDiskAction + memTracker *memory.Tracker + diskTracker *disk.Tracker + actionSpill *SpillDiskAction + onExceededCallback func(chk *Chunk) } // NewRowContainer creates a new RowContainer in memory. @@ -122,14 +123,14 @@ func (c *RowContainer) NumChunks() int { } // Add appends a chunk into the RowContainer. -func (c *RowContainer) Add(chk *Chunk, callback func()) (err error) { +func (c *RowContainer) Add(chk *Chunk) (err error) { if c.AlreadySpilled() { err = c.recordsInDisk.Add(chk) } else { c.records.Add(chk) if atomic.LoadUint32(&c.exceeded) != 0 { - if callback != nil { - callback() + if c.onExceededCallback != nil { + c.onExceededCallback(chk) } err = c.spillToDisk() if err != nil { @@ -193,6 +194,11 @@ func (c *RowContainer) ActionSpill() *SpillDiskAction { return c.actionSpill } +// SetOnExceededCallback set a callback function for exceeded memory limit. +func (c *RowContainer) SetOnExceededCallback(f func(chk *Chunk)) { + c.onExceededCallback = f +} + // SpillDiskAction implements memory.ActionOnExceed for chunk.List. If // the memory quota of a query is exceeded, SpillDiskAction.Action is // triggered. diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index efa235c605071..cf22cd4f6262f 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -47,7 +47,7 @@ func (r *rowContainerTestSuite) TestSel(c *check.C) { if chk.NumRows() == sz { chk.SetSel([]int{0, 2}) numRows += 2 - err := rc.Add(chk, nil) + err := rc.Add(chk) c.Assert(err, check.IsNil) chk = NewChunkWithCapacity(fields, sz) } @@ -98,14 +98,14 @@ func (r *rowContainerTestSuite) TestSpillAction(c *check.C) { c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(0)) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(0)) - err = rc.Add(chk, nil) + err = rc.Add(chk) c.Assert(err, check.IsNil) c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(0)) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(0)) c.Assert(rc.GetMemTracker().BytesConsumed(), check.Equals, chk.MemoryUsage()) // The following line is erroneous, since chk is already handled by rc, Add it again causes duplicated memory usage account. // It is only for test of spill, do not double-add a chunk elsewhere. - err = rc.Add(chk, nil) + err = rc.Add(chk) c.Assert(err, check.IsNil) c.Assert(atomic.LoadUint32(&rc.exceeded), check.Equals, uint32(1)) c.Assert(atomic.LoadUint32(&rc.spilled), check.Equals, uint32(1)) From 0e65c364af248032ca92382587350f71bbca34b0 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 19 Jan 2020 17:20:10 +0800 Subject: [PATCH 28/43] address comments 2 --- executor/sort.go | 57 +++++++++++++++++++----------------------------- 1 file changed, 22 insertions(+), 35 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 73eb965e7f0e0..05be0a7036e5f 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -59,12 +59,6 @@ type SortExec struct { // partitionRowPtrs store the sorted RowPtrs for each row for partitions. partitionRowPtrs [][]chunk.RowPtr - // sortRows is used to maintain a heap. - sortRows []chunk.Row - // sortRowsIndex store the partition index for each row. - sortRowsIndex []int - // partitionConsumedRows store the consumed rows num for each partition. - partitionConsumedNum []int // heapSort use heap sort for spill disk. heapSort *multiWayMerge // spillAction save the Action for spill disk. @@ -95,11 +89,6 @@ func (e *SortExec) Close() error { } } e.partitionRowPtrs = e.partitionRowPtrs[:0] - - e.memTracker.Consume(int64(-8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(-8 * cap(e.partitionConsumedNum))) - e.sortRowsIndex = nil - e.partitionConsumedNum = nil } if e.rowChunks != nil { e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) @@ -177,18 +166,25 @@ func (e *SortExec) generatePartition() { e.partitionRowPtrs = append(e.partitionRowPtrs, e.rowPtrs) } +type partitionPointer struct { + row chunk.Row + partitionId int + consumed int +} + type multiWayMerge struct { - *SortExec + lessRowFunction func(rowI chunk.Row, rowJ chunk.Row) bool + elements []partitionPointer } func (h *multiWayMerge) Less(i, j int) bool { - rowI := h.sortRows[i] - rowJ := h.sortRows[j] - return h.lessRow(rowI, rowJ) + rowI := h.elements[i].row + rowJ := h.elements[j].row + return h.lessRowFunction(rowI, rowJ) } func (h *multiWayMerge) Len() int { - return len(h.sortRows) + return len(h.elements) } func (h *multiWayMerge) Push(x interface{}) { @@ -196,49 +192,40 @@ func (h *multiWayMerge) Push(x interface{}) { } func (h *multiWayMerge) Pop() interface{} { - h.sortRows = h.sortRows[:len(h.sortRows)-1] - h.sortRowsIndex = h.sortRowsIndex[:len(h.sortRowsIndex)-1] + h.elements = h.elements[:len(h.elements)-1] return nil } func (h *multiWayMerge) Swap(i, j int) { - h.sortRows[i], h.sortRows[j] = h.sortRows[j], h.sortRows[i] - h.sortRowsIndex[i], h.sortRowsIndex[j] = h.sortRowsIndex[j], h.sortRowsIndex[i] + h.elements[i], h.elements[j] = h.elements[j], h.elements[i] } func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if e.heapSort == nil { - e.sortRowsIndex = make([]int, 0, len(e.partitionList)) - e.partitionConsumedNum = make([]int, len(e.partitionList)) - e.memTracker.Consume(int64(8 * cap(e.sortRowsIndex))) - e.memTracker.Consume(int64(8 * cap(e.partitionConsumedNum))) - e.heapSort = &multiWayMerge{e} + e.heapSort = &multiWayMerge{e.lessRow, make([]partitionPointer, 0, len(e.partitionList))} for i := 0; i < len(e.partitionList); i++ { - e.partitionConsumedNum[i] = 0 row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) if err != nil { return err } - e.sortRows = append(e.sortRows, row) - e.sortRowsIndex = append(e.sortRowsIndex, i) + e.heapSort.elements = append(e.heapSort.elements, partitionPointer{row: row, partitionId: i, consumed: 0}) } heap.Init(e.heapSort) } for !req.IsFull() && e.heapSort.Len() > 0 { - row, partIdx := e.sortRows[0], e.sortRowsIndex[0] - req.AppendRow(row) - rowIdx := e.partitionConsumedNum[partIdx] + 1 - if rowIdx >= len(e.partitionRowPtrs[partIdx]) { + partitionPtr := e.heapSort.elements[0] + req.AppendRow(partitionPtr.row) + partitionPtr.consumed++ + if partitionPtr.consumed >= len(e.partitionRowPtrs[partitionPtr.partitionId]) { heap.Remove(e.heapSort, 0) continue } - e.partitionConsumedNum[partIdx] = rowIdx - row, err = e.partitionList[partIdx].GetRow(e.partitionRowPtrs[partIdx][rowIdx]) + partitionPtr.row, err = e.partitionList[partitionPtr.partitionId].GetRow(e.partitionRowPtrs[partitionPtr.partitionId][partitionPtr.consumed]) if err != nil { return err } - e.sortRows[0] = row + e.heapSort.elements[0] = partitionPtr heap.Fix(e.heapSort, 0) } return nil From 21c62034ad3da6a01a978a69033a6ce7c090760d Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 19 Jan 2020 17:22:46 +0800 Subject: [PATCH 29/43] fix --- executor/sort.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 05be0a7036e5f..005599f9ce7a2 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -168,7 +168,7 @@ func (e *SortExec) generatePartition() { type partitionPointer struct { row chunk.Row - partitionId int + partitionID int consumed int } @@ -208,7 +208,7 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { if err != nil { return err } - e.heapSort.elements = append(e.heapSort.elements, partitionPointer{row: row, partitionId: i, consumed: 0}) + e.heapSort.elements = append(e.heapSort.elements, partitionPointer{row: row, partitionID: i, consumed: 0}) } heap.Init(e.heapSort) } @@ -217,11 +217,11 @@ func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { partitionPtr := e.heapSort.elements[0] req.AppendRow(partitionPtr.row) partitionPtr.consumed++ - if partitionPtr.consumed >= len(e.partitionRowPtrs[partitionPtr.partitionId]) { + if partitionPtr.consumed >= len(e.partitionRowPtrs[partitionPtr.partitionID]) { heap.Remove(e.heapSort, 0) continue } - partitionPtr.row, err = e.partitionList[partitionPtr.partitionId].GetRow(e.partitionRowPtrs[partitionPtr.partitionId][partitionPtr.consumed]) + partitionPtr.row, err = e.partitionList[partitionPtr.partitionID].GetRow(e.partitionRowPtrs[partitionPtr.partitionID][partitionPtr.consumed]) if err != nil { return err } From 831aae75960178432b3d19fbc8198dea81b99e6b Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 20 Jan 2020 16:08:54 +0800 Subject: [PATCH 30/43] s/chunk /rowcontainer --- executor/sort.go | 4 ++-- util/chunk/row_container.go | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 005599f9ce7a2..297a669b493be 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -236,11 +236,11 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks = chunk.NewRowContainer(fields, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) - var onExceededCallback func(chk *chunk.Chunk) + var onExceededCallback func(rowContainer *chunk.RowContainer) if config.GetGlobalConfig().OOMUseTmpStorage { e.spillAction = e.rowChunks.ActionSpill() e.ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(e.spillAction) - onExceededCallback = func(chk *chunk.Chunk) { + onExceededCallback = func(rowContainer *chunk.RowContainer) { e.generatePartition() } e.rowChunks.SetOnExceededCallback(onExceededCallback) diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index 0896c0194ccab..bc73d69d820af 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -48,7 +48,7 @@ type RowContainer struct { memTracker *memory.Tracker diskTracker *disk.Tracker actionSpill *SpillDiskAction - onExceededCallback func(chk *Chunk) + onExceededCallback func(rowContainer *RowContainer) } // NewRowContainer creates a new RowContainer in memory. @@ -130,7 +130,7 @@ func (c *RowContainer) Add(chk *Chunk) (err error) { c.records.Add(chk) if atomic.LoadUint32(&c.exceeded) != 0 { if c.onExceededCallback != nil { - c.onExceededCallback(chk) + c.onExceededCallback(c) } err = c.spillToDisk() if err != nil { @@ -195,7 +195,7 @@ func (c *RowContainer) ActionSpill() *SpillDiskAction { } // SetOnExceededCallback set a callback function for exceeded memory limit. -func (c *RowContainer) SetOnExceededCallback(f func(chk *Chunk)) { +func (c *RowContainer) SetOnExceededCallback(f func(rowContainer *RowContainer)) { c.onExceededCallback = f } From a184a1ad1f59cc0236497937c4fa7e65c6665a05 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 20 Jan 2020 16:00:05 +0800 Subject: [PATCH 31/43] try 1 --- executor/benchmark_test.go | 174 ++++++++++++++++++++++++++++++------- 1 file changed, 144 insertions(+), 30 deletions(-) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 8e255cda29f16..4eb36b5b3f84d 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -15,6 +15,7 @@ package executor import ( "context" + "encoding/base64" "fmt" "math/rand" "sort" @@ -46,13 +47,12 @@ var ( ) type mockDataSourceParameters struct { - schema *expression.Schema - genDataFunc func(row int, typ *types.FieldType) interface{} - ndvs []int // number of distinct values on columns[i] and zero represents no limit - orders []bool // columns[i] should be ordered if orders[i] is true - rows int // number of rows the DataSource should output - ctx sessionctx.Context - isRawDataSmall bool // false: rawData, true: rawDataSmall + schema *expression.Schema + genDataFunc func(row int, typ *types.FieldType) interface{} + ndvs []int // number of distinct values on columns[i] and zero represents no limit + orders []bool // columns[i] should be ordered if orders[i] is true + rows int // number of rows the DataSource should output + ctx sessionctx.Context } type mockDataSource struct { @@ -154,10 +154,13 @@ func (mds *mockDataSource) randDatum(typ *types.FieldType) interface{} { case mysql.TypeDouble: return rand.Float64() case mysql.TypeVarString: - if mds.p.isRawDataSmall { - return rawDataSmall - } - return rawData + buff := make([]byte, 10) + rand.Read(buff) + return base64.RawURLEncoding.EncodeToString(buff) + //if mds.p.isRawDataSmall { + // return rawDataSmall + //} + //return rawData default: panic("not implement") } @@ -508,18 +511,15 @@ type windowTestCase struct { concurrency int dataSourceSorted bool ctx sessionctx.Context + rawDate string + rawDataSmall string } -var rawData = strings.Repeat("x", 5*1024) -var rawDataSmall = strings.Repeat("x", 16) - func (a windowTestCase) columns() []*expression.Column { - rawDataTp := new(types.FieldType) - types.DefaultTypeForValue(rawData, rawDataTp) return []*expression.Column{ {Index: 0, RetType: types.NewFieldType(mysql.TypeDouble)}, {Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)}, - {Index: 2, RetType: rawDataTp}, + {Index: 2, RetType: types.NewFieldType(mysql.TypeVarString)}, {Index: 3, RetType: types.NewFieldType(mysql.TypeLonglong)}, } } @@ -533,7 +533,7 @@ func defaultWindowTestCase() *windowTestCase { ctx := mock.NewContext() ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, true, ctx} + return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, true, ctx, strings.Repeat("x", 5*1024), strings.Repeat("x", 16)} } func benchmarkWindowExecWithCase(b *testing.B, casTest *windowTestCase) { @@ -544,12 +544,21 @@ func benchmarkWindowExecWithCase(b *testing.B, casTest *windowTestCase) { cols := casTest.columns() dataSource := buildMockDataSource(mockDataSourceParameters{ - schema: expression.NewSchema(cols...), - ndvs: []int{0, casTest.ndv, 0, 0}, - orders: []bool{false, casTest.dataSourceSorted, false, false}, - rows: casTest.rows, - ctx: casTest.ctx, - isRawDataSmall: true, + schema: expression.NewSchema(cols...), + ndvs: []int{0, casTest.ndv, 0, 0}, + orders: []bool{false, casTest.dataSourceSorted, false, false}, + rows: casTest.rows, + ctx: casTest.ctx, + genDataFunc: func(row int, typ *types.FieldType) interface{} { + switch typ.Tp { + case mysql.TypeLong, mysql.TypeLonglong: + return int64(row) + case mysql.TypeVarString: + return casTest.rawDataSmall + default: + panic("not implement") + } + }, }) b.ResetTimer() @@ -679,6 +688,7 @@ type hashJoinTestCase struct { joinType core.JoinType disk bool useOuterToBuild bool + rawDate string } func (tc hashJoinTestCase) columns() []*expression.Column { @@ -702,7 +712,7 @@ func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, us ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) ctx.GetSessionVars().IndexLookupJoinConcurrency = 4 - tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}} + tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}, rawDate: strings.Repeat("x", 5*1024)} tc.cols = cols tc.useOuterToBuild = useOuterToBuild tc.joinType = joinType @@ -762,7 +772,7 @@ func benchmarkHashJoinExecWithCase(b *testing.B, casTest *hashJoinTestCase) { case mysql.TypeLong, mysql.TypeLonglong: return int64(row) case mysql.TypeVarString: - return rawData + return casTest.rawDate case mysql.TypeDouble: return float64(row) default: @@ -915,7 +925,7 @@ func benchmarkBuildHashTableForList(b *testing.B, casTest *hashJoinTestCase) { case mysql.TypeLong, mysql.TypeLonglong: return int64(row) case mysql.TypeVarString: - return rawData + return casTest.rawDate default: panic("not implement") } @@ -994,6 +1004,7 @@ type indexJoinTestCase struct { innerJoinKeyIdx []int innerIdx []int needOuterSort bool + rawData string } func (tc indexJoinTestCase) columns() []*expression.Column { @@ -1019,6 +1030,7 @@ func defaultIndexJoinTestCase() *indexJoinTestCase { outerJoinKeyIdx: []int{0, 1}, innerJoinKeyIdx: []int{0, 1}, innerIdx: []int{0, 1}, + rawData: strings.Repeat("x", 5*1024), } return tc } @@ -1039,7 +1051,7 @@ func (tc indexJoinTestCase) getMockDataSourceOptByRows(rows int) mockDataSourceP case mysql.TypeDouble: return float64(row) case mysql.TypeVarString: - return rawData + return tc.rawData default: panic("not implement") } @@ -1317,6 +1329,7 @@ func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc outerJoinKeyIdx: []int{0, 1}, innerJoinKeyIdx: []int{0, 1}, innerIdx: []int{0, 1}, + rawData: strings.Repeat("x", 5*1024), } tc = &mergeJoinTestCase{*itc} outerOpt := mockDataSourceParameters{ @@ -1330,7 +1343,7 @@ func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc case mysql.TypeDouble: return float64(row) case mysql.TypeVarString: - return rawData + return tc.rawData default: panic("not implement") } @@ -1349,7 +1362,7 @@ func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc case mysql.TypeDouble: return float64(row) case mysql.TypeVarString: - return rawData + return tc.rawData default: panic("not implement") } @@ -1444,3 +1457,104 @@ func BenchmarkMergeJoinExec(b *testing.B) { }) } } + +type sortCase struct { + rows int + orderByIdx []int + ndvs []int + ctx sessionctx.Context +} + +func (tc sortCase) columns() []*expression.Column { + return []*expression.Column{ + {Index: 0, RetType: types.NewFieldType(mysql.TypeLonglong)}, + {Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)}, + } +} + +func (tc sortCase) String() string { + return fmt.Sprintf("(rows:%v, orderBy:%v, ndvs: %v)", tc.rows, tc.orderByIdx, tc.ndvs) +} + +func defaultSortTestCase() *sortCase { + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + tc := &sortCase{rows: 300000, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} + return tc +} + +func benchmarkSortExec(b *testing.B, cas *sortCase) { + opt := mockDataSourceParameters{ + schema: expression.NewSchema(cas.columns()...), + rows: cas.rows, + ctx: cas.ctx, + ndvs: cas.ndvs, + } + dataSource := buildMockDataSource(opt) + exec := &SortExec{ + baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), + ByItems: make([]*core.ByItems, 0, len(cas.orderByIdx)), + schema: dataSource.schema, + } + for _, idx := range cas.orderByIdx { + exec.ByItems = append(exec.ByItems, &core.ByItems{Expr: cas.columns()[idx]}) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + tmpCtx := context.Background() + chk := newFirstChunk(exec) + dataSource.prepareChunks() + + b.StartTimer() + if err := exec.Open(tmpCtx); err != nil { + b.Fatal(err) + } + for { + if err := exec.Next(tmpCtx, chk); err != nil { + b.Fatal(err) + } + if chk.NumRows() == 0 { + break + } + } + + if err := exec.Close(); err != nil { + b.Fatal(err) + } + b.StopTimer() + } +} + +func BenchmarkSortExec(b *testing.B) { + b.ReportAllocs() + cas := defaultSortTestCase() + // all random data + cas.ndvs = []int{0, 0} + cas.orderByIdx = []int{0, 1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + for ndv := 1; ndv < 10000; ndv *= 2 { + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{0, 1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{0} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + + cas.ndvs = []int{ndv, 0} + cas.orderByIdx = []int{1} + b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { + benchmarkSortExec(b, cas) + }) + } +} From 704d86fcc1d6f8d53e6041a5865222fda8cf47b9 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Fri, 31 Jan 2020 14:19:50 +0800 Subject: [PATCH 32/43] address --- executor/sort.go | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 297a669b493be..74191104e4f73 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -63,12 +63,6 @@ type SortExec struct { heapSort *multiWayMerge // spillAction save the Action for spill disk. spillAction *chunk.SpillDiskAction - - // exceeded indicates that records have exceeded memQuota during - // adding this chunk and we should spill now. - exceeded uint32 - // spilled indicates that records have spilled out into disk. - spilled uint32 } // Close implements the Executor Close interface. @@ -113,8 +107,6 @@ func (e *SortExec) Open(ctx context.Context) error { e.diskTracker = memory.NewTracker(e.id, -1) e.diskTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.DiskTracker) } - e.exceeded = 0 - e.spilled = 0 e.partitionList = e.partitionList[:0] return e.children[0].Open(ctx) } @@ -136,8 +128,10 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - e.initPointers() - sort.Slice(e.rowPtrs, e.keyColumnsLess) + if !e.rowChunks.AlreadySpilled() { + e.initPointers() + sort.Slice(e.rowPtrs, e.keyColumnsLess) + } e.fetched = true } From 38900618095079924694588c6ad722f96c258abd Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 2 Feb 2020 11:25:56 +0800 Subject: [PATCH 33/43] fix regression --- executor/sort.go | 26 ++++++-------------------- util/chunk/row_container.go | 5 +++++ 2 files changed, 11 insertions(+), 20 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 74191104e4f73..7d622a383da0c 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -140,12 +140,10 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } } else { + rowChunks := e.rowChunks.GetList() for !req.IsFull() && e.Idx < len(e.rowPtrs) { rowPtr := e.rowPtrs[e.Idx] - row, err := e.rowChunks.GetRow(rowPtr) - if err != nil { - return err - } + row := rowChunks.GetRow(rowPtr) req.AppendRow(row) e.Idx++ } @@ -312,14 +310,8 @@ func (e *SortExec) lessRow(rowI, rowJ chunk.Row) bool { // keyColumnsLess is the less function for key columns. func (e *SortExec) keyColumnsLess(i, j int) bool { - rowI, err := e.rowChunks.GetRow(e.rowPtrs[i]) - if err != nil { - panic(err) - } - rowJ, err := e.rowChunks.GetRow(e.rowPtrs[j]) - if err != nil { - panic(err) - } + rowI := e.rowChunks.GetList().GetRow(e.rowPtrs[i]) + rowJ := e.rowChunks.GetList().GetRow(e.rowPtrs[j]) return e.lessRow(rowI, rowJ) } @@ -341,14 +333,8 @@ type topNChunkHeap struct { // Less implement heap.Interface, but since we mantains a max heap, // this function returns true if row i is greater than row j. func (h *topNChunkHeap) Less(i, j int) bool { - rowI, err := h.rowChunks.GetRow(h.rowPtrs[i]) - if err != nil { - panic(err) - } - rowJ, err := h.rowChunks.GetRow(h.rowPtrs[j]) - if err != nil { - panic(err) - } + rowI := h.rowChunks.GetList().GetRow(h.rowPtrs[i]) + rowJ := h.rowChunks.GetList().GetRow(h.rowPtrs[j]) return h.greaterRow(rowI, rowJ) } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index bc73d69d820af..5bbc0864796e1 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -160,6 +160,11 @@ func (c *RowContainer) GetChunk(chkIdx int) *Chunk { return c.records.GetChunk(chkIdx) } +// GetList returns the list of in memory records. +func (c *RowContainer) GetList() *List { + return c.records +} + // GetRow returns the row the ptr pointed to. func (c *RowContainer) GetRow(ptr RowPtr) (Row, error) { if c.AlreadySpilled() { From 8517e1145dc783b752b6449c5111652a9c078000 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 2 Feb 2020 20:57:09 +0800 Subject: [PATCH 34/43] fix --- executor/benchmark_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 4eb36b5b3f84d..75d35986e199e 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -157,10 +157,6 @@ func (mds *mockDataSource) randDatum(typ *types.FieldType) interface{} { buff := make([]byte, 10) rand.Read(buff) return base64.RawURLEncoding.EncodeToString(buff) - //if mds.p.isRawDataSmall { - // return rawDataSmall - //} - //return rawData default: panic("not implement") } From 07d604e16eaec9e68b1054a94b94b0ecacd7039d Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Sun, 2 Feb 2020 21:41:04 +0800 Subject: [PATCH 35/43] fix --- executor/sort.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 7d622a383da0c..a15ec277ac00c 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -143,8 +143,7 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { rowChunks := e.rowChunks.GetList() for !req.IsFull() && e.Idx < len(e.rowPtrs) { rowPtr := e.rowPtrs[e.Idx] - row := rowChunks.GetRow(rowPtr) - req.AppendRow(row) + req.AppendRow(rowChunks.GetRow(rowPtr)) e.Idx++ } } From d50b58fbe4b19c574d77a660cd42a7796662e5e2 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Mon, 3 Feb 2020 15:33:10 +0800 Subject: [PATCH 36/43] fix --- executor/sort.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/executor/sort.go b/executor/sort.go index a15ec277ac00c..a9b74964ca9be 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -92,6 +92,8 @@ func (e *SortExec) Close() error { e.rowPtrs = nil e.memTracker = nil e.diskTracker = nil + e.heapSort = nil + e.spillAction = nil return e.children[0].Close() } From 777da74757db46425f4228b492c0dcb8ab79d155 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 4 Feb 2020 23:14:59 +0800 Subject: [PATCH 37/43] address comments --- executor/benchmark_test.go | 14 +++++++------- executor/sort.go | 31 ++++++++++++++----------------- 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 75d35986e199e..2918d340e8455 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -507,7 +507,6 @@ type windowTestCase struct { concurrency int dataSourceSorted bool ctx sessionctx.Context - rawDate string rawDataSmall string } @@ -529,7 +528,7 @@ func defaultWindowTestCase() *windowTestCase { ctx := mock.NewContext() ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, true, ctx, strings.Repeat("x", 5*1024), strings.Repeat("x", 16)} + return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, true, ctx, strings.Repeat("x", 16)} } func benchmarkWindowExecWithCase(b *testing.B, casTest *windowTestCase) { @@ -684,7 +683,7 @@ type hashJoinTestCase struct { joinType core.JoinType disk bool useOuterToBuild bool - rawDate string + rawData string } func (tc hashJoinTestCase) columns() []*expression.Column { @@ -708,7 +707,7 @@ func defaultHashJoinTestCase(cols []*types.FieldType, joinType core.JoinType, us ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(nil, -1) ctx.GetSessionVars().IndexLookupJoinConcurrency = 4 - tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}, rawDate: strings.Repeat("x", 5*1024)} + tc := &hashJoinTestCase{rows: 100000, concurrency: 4, ctx: ctx, keyIdx: []int{0, 1}, rawData: strings.Repeat("x", 5*1024)} tc.cols = cols tc.useOuterToBuild = useOuterToBuild tc.joinType = joinType @@ -768,7 +767,7 @@ func benchmarkHashJoinExecWithCase(b *testing.B, casTest *hashJoinTestCase) { case mysql.TypeLong, mysql.TypeLonglong: return int64(row) case mysql.TypeVarString: - return casTest.rawDate + return casTest.rawData case mysql.TypeDouble: return float64(row) default: @@ -921,7 +920,7 @@ func benchmarkBuildHashTableForList(b *testing.B, casTest *hashJoinTestCase) { case mysql.TypeLong, mysql.TypeLonglong: return int64(row) case mysql.TypeVarString: - return casTest.rawDate + return casTest.rawData default: panic("not implement") } @@ -1534,7 +1533,8 @@ func BenchmarkSortExec(b *testing.B) { benchmarkSortExec(b, cas) }) - for ndv := 1; ndv < 10000; ndv *= 2 { + ndvs := []int{1, 10000} + for _, ndv := range ndvs { cas.ndvs = []int{ndv, 0} cas.orderByIdx = []int{0, 1} b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { diff --git a/executor/sort.go b/executor/sort.go index a9b74964ca9be..2965421614c46 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -59,8 +59,9 @@ type SortExec struct { // partitionRowPtrs store the sorted RowPtrs for each row for partitions. partitionRowPtrs [][]chunk.RowPtr - // heapSort use heap sort for spill disk. - heapSort *multiWayMerge + // multiWayMerge uses multi-way merge for spill disk. + // The multi-way merge algorithm can refer to https://en.wikipedia.org/wiki/K-way_merge_algorithm + multiWayMerge *multiWayMerge // spillAction save the Action for spill disk. spillAction *chunk.SpillDiskAction } @@ -92,7 +93,7 @@ func (e *SortExec) Close() error { e.rowPtrs = nil e.memTracker = nil e.diskTracker = nil - e.heapSort = nil + e.multiWayMerge = nil e.spillAction = nil return e.children[0].Close() } @@ -130,10 +131,6 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - if !e.rowChunks.AlreadySpilled() { - e.initPointers() - sort.Slice(e.rowPtrs, e.keyColumnsLess) - } e.fetched = true } @@ -194,32 +191,32 @@ func (h *multiWayMerge) Swap(i, j int) { } func (e *SortExec) externalSorting(req *chunk.Chunk) (err error) { - if e.heapSort == nil { - e.heapSort = &multiWayMerge{e.lessRow, make([]partitionPointer, 0, len(e.partitionList))} + if e.multiWayMerge == nil { + e.multiWayMerge = &multiWayMerge{e.lessRow, make([]partitionPointer, 0, len(e.partitionList))} for i := 0; i < len(e.partitionList); i++ { row, err := e.partitionList[i].GetRow(e.partitionRowPtrs[i][0]) if err != nil { return err } - e.heapSort.elements = append(e.heapSort.elements, partitionPointer{row: row, partitionID: i, consumed: 0}) + e.multiWayMerge.elements = append(e.multiWayMerge.elements, partitionPointer{row: row, partitionID: i, consumed: 0}) } - heap.Init(e.heapSort) + heap.Init(e.multiWayMerge) } - for !req.IsFull() && e.heapSort.Len() > 0 { - partitionPtr := e.heapSort.elements[0] + for !req.IsFull() && e.multiWayMerge.Len() > 0 { + partitionPtr := e.multiWayMerge.elements[0] req.AppendRow(partitionPtr.row) partitionPtr.consumed++ if partitionPtr.consumed >= len(e.partitionRowPtrs[partitionPtr.partitionID]) { - heap.Remove(e.heapSort, 0) + heap.Remove(e.multiWayMerge, 0) continue } partitionPtr.row, err = e.partitionList[partitionPtr.partitionID].GetRow(e.partitionRowPtrs[partitionPtr.partitionID][partitionPtr.consumed]) if err != nil { return err } - e.heapSort.elements[0] = partitionPtr - heap.Fix(e.heapSort, 0) + e.multiWayMerge.elements[0] = partitionPtr + heap.Fix(e.multiWayMerge, 0) } return nil } @@ -260,7 +257,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.spillAction.ResetOnce() } } - if e.rowChunks.NumRow() > 0 && len(e.partitionList) > 0 { + if e.rowChunks.NumRow() > 0 { e.generatePartition() } return nil From 11ac7471a3e72d2b4236b503c666ba6f6b56e26c Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Tue, 4 Feb 2020 23:23:39 +0800 Subject: [PATCH 38/43] fix ci --- executor/sort.go | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 2965421614c46..8685c5a0e7276 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -68,28 +68,26 @@ type SortExec struct { // Close implements the Executor Close interface. func (e *SortExec) Close() error { - if len(e.partitionList) > 0 { - for _, chunkInDisk := range e.partitionList { - if chunkInDisk != nil { - if err := chunkInDisk.Close(); err != nil { - return err - } + for _, chunkInDisk := range e.partitionList { + if chunkInDisk != nil { + if err := chunkInDisk.Close(); err != nil { + return err } } - e.partitionList = e.partitionList[:0] + } + e.partitionList = e.partitionList[:0] - for _, chunkPtr := range e.partitionRowPtrs { - if chunkPtr != nil { - e.memTracker.Consume(int64(-8 * cap(chunkPtr))) - } + for _, chunkPtr := range e.partitionRowPtrs { + if chunkPtr != nil { + e.memTracker.Consume(int64(-8 * cap(chunkPtr))) } - e.partitionRowPtrs = e.partitionRowPtrs[:0] } + e.partitionRowPtrs = e.partitionRowPtrs[:0] + if e.rowChunks != nil { e.memTracker.Consume(-e.rowChunks.GetMemTracker().BytesConsumed()) e.rowChunks = nil } - e.memTracker.Consume(int64(-8 * cap(e.rowPtrs))) e.rowPtrs = nil e.memTracker = nil e.diskTracker = nil From 9e37ebb701aa4dd11670480e7a3f6e5498788c16 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 5 Feb 2020 14:02:32 +0800 Subject: [PATCH 39/43] address --- executor/sort.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/executor/sort.go b/executor/sort.go index 8685c5a0e7276..abb11b481a8df 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -132,7 +132,10 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { e.fetched = true } - if len(e.partitionList) > 0 { + if len(e.partitionList) == 0 { + return nil + } + if e.partitionList[0].AlreadySpilled() { if err := e.externalSorting(req); err != nil { return err } From 06e5ef6dddc520acfd384252193625365b6b4c29 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 5 Feb 2020 15:55:32 +0800 Subject: [PATCH 40/43] address --- executor/sort.go | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index abb11b481a8df..58ce3f90d3885 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -135,16 +135,30 @@ func (e *SortExec) Next(ctx context.Context, req *chunk.Chunk) error { if len(e.partitionList) == 0 { return nil } - if e.partitionList[0].AlreadySpilled() { + if len(e.partitionList) > 1 { if err := e.externalSorting(req); err != nil { return err } } else { - rowChunks := e.rowChunks.GetList() - for !req.IsFull() && e.Idx < len(e.rowPtrs) { - rowPtr := e.rowPtrs[e.Idx] - req.AppendRow(rowChunks.GetRow(rowPtr)) - e.Idx++ + // Check whether the one partition is spilled. + // If the partition is in memory, use List.GetRow() to get better performance. + if !e.partitionList[0].AlreadySpilled() { + rowChunks := e.partitionList[0].GetList() + for !req.IsFull() && e.Idx < len(e.rowPtrs) { + rowPtr := e.partitionRowPtrs[0][e.Idx] + req.AppendRow(rowChunks.GetRow(rowPtr)) + e.Idx++ + } + } else { + for !req.IsFull() && e.Idx < len(e.rowPtrs) { + rowPtr := e.partitionRowPtrs[0][e.Idx] + row, err := e.partitionList[0].GetRow(rowPtr) + if err != nil { + return err + } + req.AppendRow(row) + e.Idx++ + } } } return nil From dae9cdf8df13e9687ea929471a0a976134c8122e Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 5 Feb 2020 21:03:59 +0800 Subject: [PATCH 41/43] add ut --- executor/executor_pkg_test.go | 91 +++++++++++++++++++++++++++++++++++ 1 file changed, 91 insertions(+) diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 904abacb1c7f2..f0873875d1255 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -20,12 +20,17 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" ) var _ = Suite(&testExecSuite{}) @@ -234,3 +239,89 @@ func assertEqualStrings(c *C, got []field, expect []string) { c.Assert(string(got[i].str), Equals, expect[i]) } } + +func (s *testExecSuite) TestSortSpillDisk(c *C) { + originCfg := config.GetGlobalConfig() + newConf := *originCfg + newConf.OOMUseTmpStorage = true + newConf.MemQuotaQuery = 1 + config.StoreGlobalConfig(&newConf) + defer config.StoreGlobalConfig(originCfg) + + ctx := mock.NewContext() + ctx.GetSessionVars().InitChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, -1) + cas := &sortCase{rows: 2048, orderByIdx: []int{0, 1}, ndvs: []int{0, 0}, ctx: ctx} + opt := mockDataSourceParameters{ + schema: expression.NewSchema(cas.columns()...), + rows: cas.rows, + ctx: cas.ctx, + ndvs: cas.ndvs, + } + dataSource := buildMockDataSource(opt) + exec := &SortExec{ + baseExecutor: newBaseExecutor(cas.ctx, dataSource.schema, stringutil.StringerStr("sort"), dataSource), + ByItems: make([]*core.ByItems, 0, len(cas.orderByIdx)), + schema: dataSource.schema, + } + for _, idx := range cas.orderByIdx { + exec.ByItems = append(exec.ByItems, &core.ByItems{Expr: cas.columns()[idx]}) + } + tmpCtx := context.Background() + chk := newFirstChunk(exec) + dataSource.prepareChunks() + err := exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test only 1 partition and all data in memory. + c.Assert(len(exec.partitionList), Equals, 1) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, false) + c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) + err = exec.Close() + c.Assert(err, IsNil) + + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 1) + dataSource.prepareChunks() + err = exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test 2 partitions and all data in disk. + c.Assert(len(exec.partitionList), Equals, 2) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[1].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[0].NumRow(), Equals, 1024) + c.Assert(exec.partitionList[1].NumRow(), Equals, 1024) + err = exec.Close() + c.Assert(err, IsNil) + + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, 24000) + dataSource.prepareChunks() + err = exec.Open(tmpCtx) + c.Assert(err, IsNil) + for { + err = exec.Next(tmpCtx, chk) + c.Assert(err, IsNil) + if chk.NumRows() == 0 { + break + } + } + // Test only 1 partition but spill disk. + c.Assert(len(exec.partitionList), Equals, 1) + c.Assert(exec.partitionList[0].AlreadySpilled(), Equals, true) + c.Assert(exec.partitionList[0].NumRow(), Equals, 2048) + err = exec.Close() + c.Assert(err, IsNil) +} From 36ecd3894895a7b8df1ee95d57ffc454f2dbac4e Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 6 Feb 2020 16:27:04 +0800 Subject: [PATCH 42/43] fix rowcontainer data race --- util/chunk/row_container.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index 5bbc0864796e1..fb3c2ef35e070 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -211,11 +211,14 @@ type SpillDiskAction struct { once sync.Once c *RowContainer fallbackAction memory.ActionOnExceed + m sync.Mutex } // Action sends a signal to trigger spillToDisk method of RowContainer // and if it is already triggered before, call its fallbackAction. func (a *SpillDiskAction) Action(t *memory.Tracker) { + a.m.Lock() + defer a.m.Unlock() if a.c.AlreadySpilledSafe() { if a.fallbackAction != nil { a.fallbackAction.Action(t) @@ -237,10 +240,14 @@ func (a *SpillDiskAction) SetLogHook(hook func(uint64)) {} // ResetOnce resets the spill action so that it can be triggered next time. func (a *SpillDiskAction) ResetOnce() { + a.m.Lock() + defer a.m.Unlock() a.once = sync.Once{} } // SetRowContainer sets the RowContainer for the SpillDiskAction. func (a *SpillDiskAction) SetRowContainer(c *RowContainer) { + a.m.Lock() + defer a.m.Unlock() a.c = c } From 09b7253286e2b103e24e94968514c329d3e83861 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Thu, 6 Feb 2020 17:10:49 +0800 Subject: [PATCH 43/43] fix rowcontainer data race --- executor/sort.go | 3 +-- util/chunk/row_container.go | 5 +++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/executor/sort.go b/executor/sort.go index 58ce3f90d3885..b38470c27b808 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -268,8 +268,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) e.rowChunks.SetOnExceededCallback(onExceededCallback) - e.spillAction.SetRowContainer(e.rowChunks) - e.spillAction.ResetOnce() + e.spillAction.ResetOnceAndSetRowContainer(e.rowChunks) } } if e.rowChunks.NumRow() > 0 { diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index fb3c2ef35e070..7cd2076c49189 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -245,9 +245,10 @@ func (a *SpillDiskAction) ResetOnce() { a.once = sync.Once{} } -// SetRowContainer sets the RowContainer for the SpillDiskAction. -func (a *SpillDiskAction) SetRowContainer(c *RowContainer) { +// ResetOnceAndSetRowContainer resets the spill action and sets the RowContainer for the SpillDiskAction. +func (a *SpillDiskAction) ResetOnceAndSetRowContainer(c *RowContainer) { a.m.Lock() defer a.m.Unlock() + a.once = sync.Once{} a.c = c }