github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/disk_spiller.go (about) 1 // Copyright 2019 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package colexec 12 13 import ( 14 "context" 15 "fmt" 16 "strings" 17 18 "github.com/cockroachdb/cockroach/pkg/col/coldata" 19 "github.com/cockroachdb/cockroach/pkg/sql/colexecbase" 20 "github.com/cockroachdb/cockroach/pkg/sql/colexecbase/colexecerror" 21 "github.com/cockroachdb/cockroach/pkg/sql/execinfra" 22 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 23 ) 24 25 // bufferingInMemoryOperator is an Operator that buffers up intermediate tuples 26 // in memory and knows how to export them once the memory limit has been 27 // reached. 28 type bufferingInMemoryOperator interface { 29 colexecbase.Operator 30 31 // ExportBuffered returns all the batches that have been buffered up from the 32 // input and have not yet been processed by the operator. It needs to be 33 // called once the memory limit has been reached in order to "dump" the 34 // buffered tuples into a disk-backed operator. It will return a zero-length 35 // batch once the buffer has been emptied. 36 // 37 // Calling ExportBuffered may invalidate the contents of the last batch 38 // returned by ExportBuffered. 39 ExportBuffered(input colexecbase.Operator) coldata.Batch 40 } 41 42 // oneInputDiskSpiller is an Operator that manages the fallback from a one 43 // input in-memory buffering operator to a disk-backed one when the former hits 44 // the memory limit. 45 // 46 // NOTE: if an out of memory error occurs during initialization, this operator 47 // simply propagates the error further. 48 // 49 // The diagram of the components involved is as follows: 50 // 51 // ------------- input ----------- 52 // | || | (2nd src) 53 // | || (1st src) ↓ 54 // | ----||---> bufferExportingOperator 55 // ↓ | || | 56 // inMemoryOp ---- || ↓ 57 // | || diskBackedOp 58 // | || | 59 // | ↓↓ | 60 // ---------> disk spiller <-------- 61 // || 62 // || 63 // ↓↓ 64 // output 65 // 66 // Here is the explanation: 67 // - the main chain of Operators is input -> disk spiller -> output. 68 // - the disk spiller will first try running everything through the left side 69 // chain of input -> inMemoryOp. If that succeeds, great! The disk spiller 70 // will simply propagate the batch to the output. If that fails with an OOM 71 // error, the disk spiller will then initialize the right side chain and will 72 // proceed to emit from there. 73 // - the right side chain is bufferExportingOperator -> diskBackedOp. The 74 // former will first export all the buffered tuples from inMemoryOp and then 75 // will proceed on emitting from input. 76 77 // newOneInputDiskSpiller returns a new oneInputDiskSpiller. It takes the 78 // following arguments: 79 // - inMemoryOp - the in-memory operator that will be consuming input and doing 80 // computations until it either successfully processes the whole input or 81 // reaches its memory limit. 82 // - inMemoryMemMonitorName - the name of the memory monitor of the in-memory 83 // operator. diskSpiller will catch an OOM error only if this name is 84 // contained within the error message. 85 // - diskBackedOpConstructor - the function to construct the disk-backed 86 // operator when given an input operator. We take in a constructor rather 87 // than an already created operator in order to hide the complexity of buffer 88 // exporting operator that serves as the input to the disk-backed operator. 89 // - spillingCallbackFn will be called when the spilling from in-memory to disk 90 // backed operator occurs. It should only be set in tests. 91 func newOneInputDiskSpiller( 92 input colexecbase.Operator, 93 inMemoryOp bufferingInMemoryOperator, 94 inMemoryMemMonitorName string, 95 diskBackedOpConstructor func(input colexecbase.Operator) colexecbase.Operator, 96 spillingCallbackFn func(), 97 ) colexecbase.Operator { 98 diskBackedOpInput := newBufferExportingOperator(inMemoryOp, input) 99 return &diskSpillerBase{ 100 inputs: []colexecbase.Operator{input}, 101 inMemoryOp: inMemoryOp, 102 inMemoryMemMonitorName: inMemoryMemMonitorName, 103 diskBackedOp: diskBackedOpConstructor(diskBackedOpInput), 104 spillingCallbackFn: spillingCallbackFn, 105 } 106 } 107 108 // twoInputDiskSpiller is an Operator that manages the fallback from a two 109 // input in-memory buffering operator to a disk-backed one when the former hits 110 // the memory limit. 111 // 112 // NOTE: if an out of memory error occurs during initialization, this operator 113 // simply propagates the error further. 114 // 115 // The diagram of the components involved is as follows: 116 // 117 // ----- input1 input2 ---------- 118 // || / | _____________________________________________| | || 119 // || / ↓ / | || 120 // || | inMemoryOp ------------------------------ | || 121 // || | / | | | || 122 // || | / ------------------ | | || 123 // || |/ (2nd src) ↓ (1st src) ↓ (1st src) ↓ (2nd src) || 124 // || / ----------> bufferExportingOperator1 bufferExportingOperator2 || 125 // || / | | || 126 // || | | | || 127 // || | -----> diskBackedOp <----- || 128 // || | | || 129 // || ------------------------------ | || 130 // || ↓ ↓ || 131 // ----------------------------> disk spiller <------------------------------- 132 // 133 // Here is the explanation: 134 // - the main chain of Operators is inputs -> disk spiller -> output. 135 // - the disk spiller will first try running everything through the left side 136 // chain of inputs -> inMemoryOp. If that succeeds, great! The disk spiller 137 // will simply propagate the batch to the output. If that fails with an OOM 138 // error, the disk spiller will then initialize the right side chain and will 139 // proceed to emit from there. 140 // - the right side chain is bufferExportingOperators -> diskBackedOp. The 141 // former will first export all the buffered tuples from inMemoryOp and then 142 // will proceed on emitting from input. 143 144 // newTwoInputDiskSpiller returns a new twoInputDiskSpiller. It takes the 145 // following arguments: 146 // - inMemoryOp - the in-memory operator that will be consuming inputs and 147 // doing computations until it either successfully processes the whole inputs 148 // or reaches its memory limit. 149 // - inMemoryMemMonitorName - the name of the memory monitor of the in-memory 150 // operator. diskSpiller will catch an OOM error only if this name is 151 // contained within the error message. 152 // - diskBackedOpConstructor - the function to construct the disk-backed 153 // operator when given two input operators. We take in a constructor rather 154 // than an already created operator in order to hide the complexity of buffer 155 // exporting operators that serves as inputs to the disk-backed operator. 156 // - spillingCallbackFn will be called when the spilling from in-memory to disk 157 // backed operator occurs. It should only be set in tests. 158 func newTwoInputDiskSpiller( 159 inputOne, inputTwo colexecbase.Operator, 160 inMemoryOp bufferingInMemoryOperator, 161 inMemoryMemMonitorName string, 162 diskBackedOpConstructor func(inputOne, inputTwo colexecbase.Operator) colexecbase.Operator, 163 spillingCallbackFn func(), 164 ) colexecbase.Operator { 165 diskBackedOpInputOne := newBufferExportingOperator(inMemoryOp, inputOne) 166 diskBackedOpInputTwo := newBufferExportingOperator(inMemoryOp, inputTwo) 167 return &diskSpillerBase{ 168 inputs: []colexecbase.Operator{inputOne, inputTwo}, 169 inMemoryOp: inMemoryOp, 170 inMemoryOpInitStatus: OperatorNotInitialized, 171 inMemoryMemMonitorName: inMemoryMemMonitorName, 172 diskBackedOp: diskBackedOpConstructor(diskBackedOpInputOne, diskBackedOpInputTwo), 173 distBackedOpInitStatus: OperatorNotInitialized, 174 spillingCallbackFn: spillingCallbackFn, 175 } 176 } 177 178 // diskSpillerBase is the common base for the one-input and two-input disk 179 // spillers. 180 type diskSpillerBase struct { 181 NonExplainable 182 183 closerHelper 184 185 inputs []colexecbase.Operator 186 spilled bool 187 188 inMemoryOp bufferingInMemoryOperator 189 inMemoryOpInitStatus OperatorInitStatus 190 inMemoryMemMonitorName string 191 diskBackedOp colexecbase.Operator 192 distBackedOpInitStatus OperatorInitStatus 193 spillingCallbackFn func() 194 } 195 196 var _ resettableOperator = &diskSpillerBase{} 197 198 func (d *diskSpillerBase) Init() { 199 if d.inMemoryOpInitStatus == OperatorInitialized { 200 return 201 } 202 // It is possible that Init() call below will hit an out of memory error, 203 // but we decide to bail on this query, so we do not catch internal panics. 204 // 205 // Also note that d.input is the input to d.inMemoryOp, so calling Init() 206 // only on the latter is sufficient. 207 d.inMemoryOp.Init() 208 d.inMemoryOpInitStatus = OperatorInitialized 209 } 210 211 func (d *diskSpillerBase) Next(ctx context.Context) coldata.Batch { 212 if d.spilled { 213 return d.diskBackedOp.Next(ctx) 214 } 215 var batch coldata.Batch 216 if err := colexecerror.CatchVectorizedRuntimeError( 217 func() { 218 batch = d.inMemoryOp.Next(ctx) 219 }, 220 ); err != nil { 221 if sqlbase.IsOutOfMemoryError(err) && 222 strings.Contains(err.Error(), d.inMemoryMemMonitorName) { 223 d.spilled = true 224 if d.spillingCallbackFn != nil { 225 d.spillingCallbackFn() 226 } 227 d.diskBackedOp.Init() 228 d.distBackedOpInitStatus = OperatorInitialized 229 return d.diskBackedOp.Next(ctx) 230 } 231 // Either not an out of memory error or an OOM error coming from a 232 // different operator, so we propagate it further. 233 colexecerror.InternalError(err) 234 } 235 return batch 236 } 237 238 func (d *diskSpillerBase) reset(ctx context.Context) { 239 for _, input := range d.inputs { 240 if r, ok := input.(resetter); ok { 241 r.reset(ctx) 242 } 243 } 244 if d.inMemoryOpInitStatus == OperatorInitialized { 245 if r, ok := d.inMemoryOp.(resetter); ok { 246 r.reset(ctx) 247 } 248 } 249 if d.distBackedOpInitStatus == OperatorInitialized { 250 if r, ok := d.diskBackedOp.(resetter); ok { 251 r.reset(ctx) 252 } 253 } 254 d.spilled = false 255 } 256 257 // Close closes the diskSpillerBase's input. 258 // TODO(asubiotto): Remove this method. It only exists so that we can call Close 259 // from some runTests subtests when not draining the input fully. The test 260 // should pass in the testing.T object used so that the caller can decide to 261 // explicitly close the input after checking the test. 262 func (d *diskSpillerBase) IdempotentClose(ctx context.Context) error { 263 if !d.close() { 264 return nil 265 } 266 if c, ok := d.diskBackedOp.(IdempotentCloser); ok { 267 return c.IdempotentClose(ctx) 268 } 269 return nil 270 } 271 272 func (d *diskSpillerBase) ChildCount(verbose bool) int { 273 if verbose { 274 return len(d.inputs) + 2 275 } 276 return 1 277 } 278 279 func (d *diskSpillerBase) Child(nth int, verbose bool) execinfra.OpNode { 280 // Note: although the main chain is d.inputs -> diskSpiller -> output (and 281 // the main chain should be under nth == 0), in order to make the output of 282 // EXPLAIN (VEC) less confusing we return the in-memory operator as being on 283 // the main chain. 284 if verbose { 285 switch nth { 286 case 0: 287 return d.inMemoryOp 288 case len(d.inputs) + 1: 289 return d.diskBackedOp 290 default: 291 return d.inputs[nth-1] 292 } 293 } 294 switch nth { 295 case 0: 296 return d.inMemoryOp 297 default: 298 colexecerror.InternalError(fmt.Sprintf("invalid index %d", nth)) 299 // This code is unreachable, but the compiler cannot infer that. 300 return nil 301 } 302 } 303 304 // bufferExportingOperator is an Operator that first returns all batches from 305 // firstSource, and once firstSource is exhausted, it proceeds on returning all 306 // batches from the second source. 307 // 308 // NOTE: bufferExportingOperator assumes that both sources will have been 309 // initialized when bufferExportingOperator.Init() is called. 310 // NOTE: it is assumed that secondSource is the input to firstSource. 311 type bufferExportingOperator struct { 312 colexecbase.ZeroInputNode 313 NonExplainable 314 315 firstSource bufferingInMemoryOperator 316 secondSource colexecbase.Operator 317 firstSourceDone bool 318 } 319 320 var _ resettableOperator = &bufferExportingOperator{} 321 322 func newBufferExportingOperator( 323 firstSource bufferingInMemoryOperator, secondSource colexecbase.Operator, 324 ) colexecbase.Operator { 325 return &bufferExportingOperator{ 326 firstSource: firstSource, 327 secondSource: secondSource, 328 } 329 } 330 331 func (b *bufferExportingOperator) Init() { 332 // Init here is a noop because the operator assumes that both sources have 333 // already been initialized. 334 } 335 336 func (b *bufferExportingOperator) Next(ctx context.Context) coldata.Batch { 337 if b.firstSourceDone { 338 return b.secondSource.Next(ctx) 339 } 340 batch := b.firstSource.ExportBuffered(b.secondSource) 341 if batch.Length() == 0 { 342 b.firstSourceDone = true 343 return b.secondSource.Next(ctx) 344 } 345 return batch 346 } 347 348 func (b *bufferExportingOperator) reset(ctx context.Context) { 349 if r, ok := b.firstSource.(resetter); ok { 350 r.reset(ctx) 351 } 352 if r, ok := b.secondSource.(resetter); ok { 353 r.reset(ctx) 354 } 355 b.firstSourceDone = false 356 }