-
Notifications
You must be signed in to change notification settings - Fork 5.9k
/
Copy pathdelete.go
206 lines (182 loc) · 6.08 KB
/
delete.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
// Copyright 2018 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
import (
"context"
"github.com/pingcap/tidb/config"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/memory"
)
// DeleteExec represents a delete executor.
// See https://dev.mysql.com/doc/refman/5.7/en/delete.html
type DeleteExec struct {
baseExecutor
IsMultiTable bool
tblID2Table map[int64]table.Table
// tblColPosInfos stores relationship between column ordinal to its table handle.
// the columns ordinals is present in ordinal range format, @see plannercore.TblColPosInfos
tblColPosInfos plannercore.TblColPosInfoSlice
memTracker *memory.Tracker
}
// Next implements the Executor Next interface.
func (e *DeleteExec) Next(ctx context.Context, req *chunk.Chunk) error {
req.Reset()
if e.IsMultiTable {
return e.deleteMultiTablesByChunk(ctx)
}
return e.deleteSingleTableByChunk(ctx)
}
func (e *DeleteExec) deleteOneRow(tbl table.Table, handleIndex int, isExtraHandle bool, row []types.Datum) error {
end := len(row)
if isExtraHandle {
end--
}
handle := row[handleIndex].GetInt64()
err := e.removeRow(e.ctx, tbl, handle, row[:end])
if err != nil {
return err
}
return nil
}
func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error {
var (
tbl table.Table
isExtrahandle bool
handleIndex int
rowCount int
)
for _, info := range e.tblColPosInfos {
tbl = e.tblID2Table[info.TblID]
handleIndex = info.HandleOrdinal
isExtrahandle = handleIsExtra(e.children[0].Schema().Columns[info.HandleOrdinal])
}
// If tidb_batch_delete is ON and not in a transaction, we could use BatchDelete mode.
batchDelete := e.ctx.GetSessionVars().BatchDelete && !e.ctx.GetSessionVars().InTxn() && config.GetGlobalConfig().EnableBatchDML
batchDMLSize := e.ctx.GetSessionVars().DMLBatchSize
fields := retTypes(e.children[0])
chk := newFirstChunk(e.children[0])
memUsageOfChk := int64(0)
for {
e.memTracker.Consume(-memUsageOfChk)
iter := chunk.NewIterator4Chunk(chk)
err := Next(ctx, e.children[0], chk)
if err != nil {
return err
}
if chk.NumRows() == 0 {
break
}
memUsageOfChk = chk.MemoryUsage()
e.memTracker.Consume(memUsageOfChk)
for chunkRow := iter.Begin(); chunkRow != iter.End(); chunkRow = iter.Next() {
if batchDelete && rowCount >= batchDMLSize {
if err = e.ctx.StmtCommit(e.memTracker); err != nil {
return err
}
if err = e.ctx.NewTxn(ctx); err != nil {
// We should return a special error for batch insert.
return ErrBatchInsertFail.GenWithStack("BatchDelete failed with error: %v", err)
}
rowCount = 0
}
datumRow := chunkRow.GetDatumRow(fields)
err = e.deleteOneRow(tbl, handleIndex, isExtrahandle, datumRow)
if err != nil {
return err
}
rowCount++
}
chk = chunk.Renew(chk, e.maxChunkSize)
}
return nil
}
func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []plannercore.TblColPosInfo, joinedRow []types.Datum) {
// iterate all the joined tables, and got the copresonding rows in joinedRow.
for _, info := range colPosInfos {
if tblRowMap[info.TblID] == nil {
tblRowMap[info.TblID] = make(map[int64][]types.Datum)
}
handle := joinedRow[info.HandleOrdinal].GetInt64()
// tblRowMap[info.TblID][handle] hold the row datas binding to this table and this handle.
tblRowMap[info.TblID][handle] = joinedRow[info.Start:info.End]
}
}
func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error {
colPosInfos := e.tblColPosInfos
tblRowMap := make(tableRowMapType)
fields := retTypes(e.children[0])
chk := newFirstChunk(e.children[0])
memUsageOfChk := int64(0)
for {
e.memTracker.Consume(-memUsageOfChk)
iter := chunk.NewIterator4Chunk(chk)
err := Next(ctx, e.children[0], chk)
if err != nil {
return err
}
if chk.NumRows() == 0 {
break
}
memUsageOfChk = chk.MemoryUsage()
e.memTracker.Consume(memUsageOfChk)
for joinedChunkRow := iter.Begin(); joinedChunkRow != iter.End(); joinedChunkRow = iter.Next() {
joinedDatumRow := joinedChunkRow.GetDatumRow(fields)
e.composeTblRowMap(tblRowMap, colPosInfos, joinedDatumRow)
}
chk = chunk.Renew(chk, e.maxChunkSize)
}
return e.removeRowsInTblRowMap(tblRowMap)
}
func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error {
for id, rowMap := range tblRowMap {
for handle, data := range rowMap {
err := e.removeRow(e.ctx, e.tblID2Table[id], handle, data)
if err != nil {
return err
}
}
}
return nil
}
func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, data []types.Datum) error {
txnState, err := e.ctx.Txn(false)
if err != nil {
return err
}
memUsageOfTxnState := txnState.Size()
err = t.RemoveRecord(ctx, h, data)
if err != nil {
return err
}
e.memTracker.Consume(int64(txnState.Size() - memUsageOfTxnState))
ctx.GetSessionVars().StmtCtx.AddAffectedRows(1)
return nil
}
// Close implements the Executor Close interface.
func (e *DeleteExec) Close() error {
return e.children[0].Close()
}
// Open implements the Executor Open interface.
func (e *DeleteExec) Open(ctx context.Context) error {
e.memTracker = memory.NewTracker(e.id, -1)
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker)
return e.children[0].Open(ctx)
}
// tableRowMapType is a map for unique (Table, Row) pair. key is the tableID.
// the key in map[int64]Row is the joined table handle, which represent a unique reference row.
// the value in map[int64]Row is the deleting row.
type tableRowMapType map[int64]map[int64][]types.Datum